Low IndexCoord weight (#21548)

Signed-off-by: cai.zhang <cai.zhang@zilliz.com>
pull/21564/head
cai.zhang 2023-01-06 14:21:37 +08:00 committed by GitHub
parent e8e2696430
commit aa203acfb3
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
80 changed files with 1478 additions and 18314 deletions

View File

@ -19,57 +19,38 @@ package components
import (
"context"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
grpcindexcoord "github.com/milvus-io/milvus/internal/distributed/indexcoord"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/util/dependency"
"github.com/milvus-io/milvus/internal/util/typeutil"
)
// IndexCoord implements IndexCoord grpc server
type IndexCoord struct {
svr *grpcindexcoord.Server
}
// NewIndexCoord creates a new IndexCoord
func NewIndexCoord(ctx context.Context, factory dependency.Factory) (*IndexCoord, error) {
var err error
s := &IndexCoord{}
svr, err := grpcindexcoord.NewServer(ctx, factory)
if err != nil {
return nil, err
}
s.svr = svr
return s, nil
return &IndexCoord{}, nil
}
// Run starts service
func (s *IndexCoord) Run() error {
if err := s.svr.Run(); err != nil {
return err
}
log.Debug("IndexCoord successfully started")
log.Info("IndexCoord running ...")
return nil
}
// Stop terminates service
func (s *IndexCoord) Stop() error {
if err := s.svr.Stop(); err != nil {
return err
}
log.Info("IndexCoord stopping ...")
return nil
}
// GetComponentStates returns indexnode's states
func (s *IndexCoord) Health(ctx context.Context) commonpb.StateCode {
resp, err := s.svr.GetComponentStates(ctx, &milvuspb.GetComponentStatesRequest{})
if err != nil {
return commonpb.StateCode_Abnormal
}
return resp.State.GetStateCode()
log.Info("IndexCoord is healthy")
return commonpb.StateCode_Healthy
}
func (s *IndexCoord) GetName() string {

View File

@ -26,8 +26,8 @@ type run struct {
svrAlias string
enableRootCoord bool
enableQueryCoord bool
enableIndexCoord bool
enableDataCoord bool
enableIndexCoord bool
enableQueryNode bool
enableDataNode bool
enableIndexNode bool

View File

@ -26,15 +26,14 @@ import (
"syscall"
"time"
"github.com/milvus-io/milvus/internal/management"
rocksmqimpl "github.com/milvus-io/milvus/internal/mq/mqimpl/rocksmq/server"
"go.uber.org/zap"
"github.com/milvus-io/milvus/cmd/components"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/management"
"github.com/milvus-io/milvus/internal/management/healthz"
"github.com/milvus-io/milvus/internal/metrics"
rocksmqimpl "github.com/milvus-io/milvus/internal/mq/mqimpl/rocksmq/server"
"github.com/milvus-io/milvus/internal/util/dependency"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/logutil"
@ -147,7 +146,7 @@ func (mr *MilvusRoles) runDataNode(ctx context.Context, localMsg bool) *componen
}
func (mr *MilvusRoles) runIndexCoord(ctx context.Context, localMsg bool) *components.IndexCoord {
return runComponent(ctx, localMsg, components.NewIndexCoord, metrics.RegisterIndexCoord)
return runComponent(ctx, localMsg, components.NewIndexCoord, func(registry *prometheus.Registry) {})
}
func (mr *MilvusRoles) runIndexNode(ctx context.Context, localMsg bool) *components.IndexNode {

View File

@ -241,22 +241,6 @@ queryNode:
maxNQ: 1000
topKMergeRatio: 10.0
indexCoord:
address: localhost
port: 31000
enableActiveStandby: false # Enable active-standby
minSegmentNumRowsToEnableIndex: 1024 # It's a threshold. When the segment num rows is less than this value, the segment will not be indexed
bindIndexNodeMode:
enable: false
address: "localhost:22930"
withCred: false
nodeID: 0
gc:
interval: 600 # gc interval in seconds
indexNode:
port: 21121
enableDisk: true # enable index node build disk vector index

View File

@ -303,7 +303,7 @@ func (gc *garbageCollector) recycleUnusedSegIndexes() {
if gc.meta.GetSegmentUnsafe(segIdx.SegmentID) == nil || !gc.meta.IsIndexExist(segIdx.CollectionID, segIdx.IndexID) {
if err := gc.meta.RemoveSegmentIndex(segIdx.CollectionID, segIdx.PartitionID, segIdx.SegmentID, segIdx.IndexID, segIdx.BuildID); err != nil {
log.Warn("delete index meta from etcd failed, wait to retry", zap.Int64("buildID", segIdx.BuildID),
zap.Int64("nodeID", segIdx.NodeID), zap.Error(err))
zap.Int64("segID", segIdx.SegmentID), zap.Int64("nodeID", segIdx.NodeID), zap.Error(err))
continue
}
log.Info("index meta recycle success", zap.Int64("buildID", segIdx.BuildID),

View File

@ -86,13 +86,13 @@ func (m *meta) updateIndexTasksMetrics() {
for k, v := range m {
switch k {
case commonpb.IndexState_Unissued:
metrics.IndexCoordIndexTaskNum.WithLabelValues(strconv.FormatInt(collID, 10), metrics.UnissuedIndexTaskLabel).Set(float64(v))
metrics.IndexTaskNum.WithLabelValues(strconv.FormatInt(collID, 10), metrics.UnissuedIndexTaskLabel).Set(float64(v))
case commonpb.IndexState_InProgress:
metrics.IndexCoordIndexTaskNum.WithLabelValues(strconv.FormatInt(collID, 10), metrics.InProgressIndexTaskLabel).Set(float64(v))
metrics.IndexTaskNum.WithLabelValues(strconv.FormatInt(collID, 10), metrics.InProgressIndexTaskLabel).Set(float64(v))
case commonpb.IndexState_Finished:
metrics.IndexCoordIndexTaskNum.WithLabelValues(strconv.FormatInt(collID, 10), metrics.FinishedIndexTaskLabel).Set(float64(v))
metrics.IndexTaskNum.WithLabelValues(strconv.FormatInt(collID, 10), metrics.FinishedIndexTaskLabel).Set(float64(v))
case commonpb.IndexState_Failed:
metrics.IndexCoordIndexTaskNum.WithLabelValues(strconv.FormatInt(collID, 10), metrics.FailedIndexTaskLabel).Set(float64(v))
metrics.IndexTaskNum.WithLabelValues(strconv.FormatInt(collID, 10), metrics.FailedIndexTaskLabel).Set(float64(v))
}
}
}
@ -630,10 +630,10 @@ func (m *meta) RemoveIndex(collID, indexID UniqueID) error {
delete(m.indexes[collID], indexID)
if len(m.indexes[collID]) == 0 {
delete(m.indexes, collID)
metrics.IndexCoordIndexTaskNum.Delete(prometheus.Labels{"collection_id": strconv.FormatInt(collID, 10), "index_task_status": metrics.UnissuedIndexTaskLabel})
metrics.IndexCoordIndexTaskNum.Delete(prometheus.Labels{"collection_id": strconv.FormatInt(collID, 10), "index_task_status": metrics.InProgressIndexTaskLabel})
metrics.IndexCoordIndexTaskNum.Delete(prometheus.Labels{"collection_id": strconv.FormatInt(collID, 10), "index_task_status": metrics.FinishedIndexTaskLabel})
metrics.IndexCoordIndexTaskNum.Delete(prometheus.Labels{"collection_id": strconv.FormatInt(collID, 10), "index_task_status": metrics.FailedIndexTaskLabel})
metrics.IndexTaskNum.Delete(prometheus.Labels{"collection_id": strconv.FormatInt(collID, 10), "index_task_status": metrics.UnissuedIndexTaskLabel})
metrics.IndexTaskNum.Delete(prometheus.Labels{"collection_id": strconv.FormatInt(collID, 10), "index_task_status": metrics.InProgressIndexTaskLabel})
metrics.IndexTaskNum.Delete(prometheus.Labels{"collection_id": strconv.FormatInt(collID, 10), "index_task_status": metrics.FinishedIndexTaskLabel})
metrics.IndexTaskNum.Delete(prometheus.Labels{"collection_id": strconv.FormatInt(collID, 10), "index_task_status": metrics.FailedIndexTaskLabel})
}
log.Info("IndexCoord meta table remove index success", zap.Int64("collID", collID), zap.Int64("indexID", indexID))
return nil

View File

@ -26,6 +26,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/metrics"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/util/metautil"
"github.com/milvus-io/milvus/internal/util/paramtable"
@ -140,11 +141,13 @@ func (s *Server) CreateIndex(ctx context.Context, req *datapb.CreateIndexRequest
errResp.Reason = msgDataCoordIsUnhealthy(paramtable.GetNodeID())
return errResp, nil
}
metrics.IndexRequestCounter.WithLabelValues(metrics.TotalLabel).Inc()
indexID, err := s.meta.CanCreateIndex(req)
if err != nil {
log.Error("CreateIndex failed", zap.Error(err))
errResp.Reason = err.Error()
metrics.IndexRequestCounter.WithLabelValues(metrics.FailLabel).Inc()
return errResp, nil
}
@ -153,12 +156,14 @@ func (s *Server) CreateIndex(ctx context.Context, req *datapb.CreateIndexRequest
if err != nil {
log.Warn("failed to alloc indexID", zap.Error(err))
errResp.Reason = "failed to alloc indexID"
metrics.IndexRequestCounter.WithLabelValues(metrics.FailLabel).Inc()
return errResp, nil
}
if getIndexType(req.GetIndexParams()) == diskAnnIndex && !s.indexNodeManager.ClientSupportDisk() {
errMsg := "all IndexNodes do not support disk indexes, please verify"
log.Warn(errMsg)
errResp.Reason = errMsg
metrics.IndexRequestCounter.WithLabelValues(metrics.FailLabel).Inc()
return errResp, nil
}
}
@ -182,6 +187,7 @@ func (s *Server) CreateIndex(ctx context.Context, req *datapb.CreateIndexRequest
log.Error("CreateIndex fail", zap.Int64("collectionID", req.GetCollectionID()),
zap.Int64("fieldID", req.GetFieldID()), zap.String("indexName", req.GetIndexName()), zap.Error(err))
errResp.Reason = err.Error()
metrics.IndexRequestCounter.WithLabelValues(metrics.FailLabel).Inc()
return errResp, nil
}
@ -194,6 +200,7 @@ func (s *Server) CreateIndex(ctx context.Context, req *datapb.CreateIndexRequest
zap.String("IndexName", req.GetIndexName()), zap.Int64("fieldID", req.GetFieldID()),
zap.Int64("IndexID", indexID))
errResp.ErrorCode = commonpb.ErrorCode_Success
metrics.IndexRequestCounter.WithLabelValues(metrics.SuccessLabel).Inc()
return errResp, nil
}

View File

@ -66,7 +66,7 @@ func (nm *IndexNodeManager) RemoveNode(nodeID UniqueID) {
defer nm.lock.Unlock()
delete(nm.nodeClients, nodeID)
delete(nm.stoppingNodes, nodeID)
metrics.IndexCoordIndexNodeNum.WithLabelValues().Dec()
metrics.IndexNodeNum.WithLabelValues().Dec()
}
func (nm *IndexNodeManager) StoppingNode(nodeID UniqueID) {
@ -84,13 +84,13 @@ func (nm *IndexNodeManager) AddNode(nodeID UniqueID, address string) error {
err error
)
nodeClient, err = grpcindexnodeclient.NewClient(context.TODO(), address, Params.IndexCoordCfg.WithCredential.GetAsBool())
nodeClient, err = grpcindexnodeclient.NewClient(context.TODO(), address, Params.DataCoordCfg.WithCredential.GetAsBool())
if err != nil {
log.Error("create IndexNode client fail", zap.Error(err))
return err
}
metrics.IndexCoordIndexNodeNum.WithLabelValues().Inc()
metrics.IndexNodeNum.WithLabelValues().Inc()
nm.setClient(nodeID, nodeClient)
return nil
}

View File

@ -20,6 +20,8 @@ import (
"context"
"errors"
"github.com/milvus-io/milvus/internal/types"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
@ -49,8 +51,9 @@ func (s *Server) getSystemInfoMetrics(
// get datacoord info
nodes := s.cluster.GetSessions()
clusterTopology := metricsinfo.DataClusterTopology{
Self: s.getDataCoordMetrics(),
ConnectedNodes: make([]metricsinfo.DataNodeInfos, 0, len(nodes)),
Self: s.getDataCoordMetrics(),
ConnectedDataNodes: make([]metricsinfo.DataNodeInfos, 0, len(nodes)),
ConnectedIndexNodes: make([]metricsinfo.IndexNodeInfos, 0),
}
// for each data node, fetch metrics info
@ -60,7 +63,17 @@ func (s *Server) getSystemInfoMetrics(
log.Warn("fails to get DataNode metrics", zap.Error(err))
continue
}
clusterTopology.ConnectedNodes = append(clusterTopology.ConnectedNodes, infos)
clusterTopology.ConnectedDataNodes = append(clusterTopology.ConnectedDataNodes, infos)
}
indexNodes := s.indexNodeManager.GetAllClients()
for _, node := range indexNodes {
infos, err := s.getIndexNodeMetrics(ctx, req, node)
if err != nil {
log.Warn("fails to get IndexNode metrics", zap.Error(err))
continue
}
clusterTopology.ConnectedIndexNodes = append(clusterTopology.ConnectedIndexNodes, infos)
}
// compose topolgoy struct
@ -168,3 +181,43 @@ func (s *Server) getDataNodeMetrics(ctx context.Context, req *milvuspb.GetMetric
infos.BaseComponentInfos.HasError = false
return infos, nil
}
func (s *Server) getIndexNodeMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest, node types.IndexNode) (metricsinfo.IndexNodeInfos, error) {
infos := metricsinfo.IndexNodeInfos{
BaseComponentInfos: metricsinfo.BaseComponentInfos{
HasError: true,
ID: int64(uniquegenerator.GetUniqueIntGeneratorIns().GetInt()),
},
}
if node == nil {
return infos, errors.New("IndexNode is nil")
}
metrics, err := node.GetMetrics(ctx, req)
if err != nil {
log.Warn("invalid metrics of IndexNode was found",
zap.Error(err))
infos.BaseComponentInfos.ErrorReason = err.Error()
// err handled, returns nil
return infos, nil
}
infos.BaseComponentInfos.Name = metrics.GetComponentName()
if metrics.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success {
log.Warn("invalid metrics of DataNode was found",
zap.Any("error_code", metrics.Status.ErrorCode),
zap.Any("error_reason", metrics.Status.Reason))
infos.BaseComponentInfos.ErrorReason = metrics.GetStatus().GetReason()
return infos, nil
}
err = metricsinfo.UnmarshalComponentInfos(metrics.GetResponse(), &infos)
if err != nil {
log.Warn("invalid metrics of IndexNode found",
zap.Error(err))
infos.BaseComponentInfos.ErrorReason = err.Error()
return infos, nil
}
infos.BaseComponentInfos.HasError = false
return infos, nil
}

View File

@ -41,6 +41,18 @@ func (c *mockMetricDataNodeClient) GetMetrics(ctx context.Context, req *milvuspb
return c.mock()
}
type mockMetricIndexNodeClient struct {
types.IndexNode
mock func() (*milvuspb.GetMetricsResponse, error)
}
func (m *mockMetricIndexNodeClient) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
if m.mock == nil {
return m.IndexNode.GetMetrics(ctx, req)
}
return m.mock()
}
func TestGetDataNodeMetrics(t *testing.T) {
svr := newTestServer(t, nil)
defer closeTestServer(t, svr)
@ -112,3 +124,93 @@ func TestGetDataNodeMetrics(t *testing.T) {
assert.True(t, info.HasError)
}
func TestGetIndexNodeMetrics(t *testing.T) {
svr := newTestServer(t, nil)
defer closeTestServer(t, svr)
ctx := context.Background()
req := &milvuspb.GetMetricsRequest{}
// nil node
_, err := svr.getIndexNodeMetrics(ctx, req, nil)
assert.NotNil(t, err)
// return error
info, err := svr.getIndexNodeMetrics(ctx, req, &mockMetricIndexNodeClient{mock: func() (*milvuspb.GetMetricsResponse, error) {
return nil, errors.New("mock error")
}})
assert.Nil(t, err)
assert.True(t, info.HasError)
// failed
info, err = svr.getIndexNodeMetrics(ctx, req, &mockMetricIndexNodeClient{
mock: func() (*milvuspb.GetMetricsResponse, error) {
return &milvuspb.GetMetricsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "mock fail",
},
Response: "",
ComponentName: "indexnode100",
}, nil
},
})
assert.Nil(t, err)
assert.True(t, info.HasError)
assert.Equal(t, metricsinfo.ConstructComponentName(typeutil.IndexNodeRole, 100), info.BaseComponentInfos.Name)
// return unexpected
info, err = svr.getIndexNodeMetrics(ctx, req, &mockMetricIndexNodeClient{
mock: func() (*milvuspb.GetMetricsResponse, error) {
return &milvuspb.GetMetricsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
},
Response: "XXXXXXXXXXXXX",
ComponentName: "indexnode100",
}, nil
},
})
assert.Nil(t, err)
assert.True(t, info.HasError)
assert.Equal(t, metricsinfo.ConstructComponentName(typeutil.IndexNodeRole, 100), info.BaseComponentInfos.Name)
// success
info, err = svr.getIndexNodeMetrics(ctx, req, &mockMetricIndexNodeClient{
mock: func() (*milvuspb.GetMetricsResponse, error) {
nodeID = UniqueID(100)
nodeInfos := metricsinfo.DataNodeInfos{
BaseComponentInfos: metricsinfo.BaseComponentInfos{
Name: metricsinfo.ConstructComponentName(typeutil.IndexNodeRole, nodeID),
ID: nodeID,
},
}
resp, err := metricsinfo.MarshalComponentInfos(nodeInfos)
if err != nil {
return &milvuspb.GetMetricsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: err.Error(),
},
Response: "",
ComponentName: metricsinfo.ConstructComponentName(typeutil.IndexNodeRole, nodeID),
}, nil
}
return &milvuspb.GetMetricsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
},
Response: resp,
ComponentName: metricsinfo.ConstructComponentName(typeutil.IndexNodeRole, nodeID),
}, nil
},
})
assert.Nil(t, err)
assert.False(t, info.HasError)
assert.Equal(t, metricsinfo.ConstructComponentName(typeutil.IndexNodeRole, 100), info.BaseComponentInfos.Name)
}

View File

@ -31,7 +31,6 @@ import (
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/proxypb"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/metricsinfo"
"github.com/milvus-io/milvus/internal/util/tsoutil"
"github.com/milvus-io/milvus/internal/util/typeutil"
@ -787,87 +786,3 @@ func newMockHandlerWithMeta(meta *meta) *mockHandler {
meta: meta,
}
}
type mockIndexCoord struct {
types.IndexCoord
}
func newMockIndexCoord() *mockIndexCoord {
return &mockIndexCoord{}
}
func (m *mockIndexCoord) Init() error {
return nil
}
func (m *mockIndexCoord) Start() error {
return nil
}
func (m *mockIndexCoord) DescribeIndex(ctx context.Context, req *datapb.DescribeIndexRequest) (*datapb.DescribeIndexResponse, error) {
if req.CollectionID == 10000 {
return nil, errors.New("server down")
}
// Has diskann index
if req.CollectionID == 1000 || req.CollectionID == 2000 ||
req.CollectionID == 3000 || req.CollectionID == 4000 {
return &datapb.DescribeIndexResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
IndexInfos: []*datapb.IndexInfo{
{
CollectionID: req.CollectionID,
FieldID: 0,
IndexName: "DISKANN",
IndexID: 0,
TypeParams: nil,
IndexParams: []*commonpb.KeyValuePair{
{
Key: "index_type",
Value: "DISKANN",
},
},
},
},
}, nil
}
// Has common index
return &datapb.DescribeIndexResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
IndexInfos: []*datapb.IndexInfo{
{
CollectionID: 1,
FieldID: 0,
IndexName: "default",
IndexID: 0,
TypeParams: nil,
IndexParams: nil,
},
},
}, nil
}
func (m *mockIndexCoord) GetIndexInfos(ctx context.Context, req *datapb.GetIndexInfoRequest) (*datapb.GetIndexInfoResponse, error) {
segmentID := req.GetSegmentIDs()[0]
collectionID := req.GetCollectionID()
return &datapb.GetIndexInfoResponse{
Status: &commonpb.Status{},
SegmentInfo: map[int64]*datapb.SegmentIndexInfo{
segmentID: {
EnableIndex: true,
CollectionID: collectionID,
SegmentID: segmentID,
IndexInfos: []*datapb.IndexFilePathInfo{
{
FieldID: int64(201),
},
},
},
},
}, nil
}

View File

@ -1136,8 +1136,8 @@ func TestServer_getSystemInfoMetrics(t *testing.T) {
var coordTopology metricsinfo.DataCoordTopology
err = metricsinfo.UnmarshalTopology(resp.Response, &coordTopology)
assert.Nil(t, err)
assert.Equal(t, len(svr.cluster.GetSessions()), len(coordTopology.Cluster.ConnectedNodes))
for _, nodeMetrics := range coordTopology.Cluster.ConnectedNodes {
assert.Equal(t, len(svr.cluster.GetSessions()), len(coordTopology.Cluster.ConnectedDataNodes))
for _, nodeMetrics := range coordTopology.Cluster.ConnectedDataNodes {
assert.Equal(t, false, nodeMetrics.HasError)
assert.Equal(t, 0, len(nodeMetrics.ErrorReason))
_, err = metricsinfo.MarshalComponentInfos(nodeMetrics)
@ -3601,14 +3601,6 @@ func newTestServer2(t *testing.T, receiveCh chan any, opts ...Option) *Server {
_, err = etcdCli.Delete(context.Background(), sessKey, clientv3.WithPrefix())
assert.Nil(t, err)
icSession := sessionutil.NewSession(context.Background(), Params.EtcdCfg.MetaRootPath.GetValue(), etcdCli)
icSession.Init(typeutil.IndexCoordRole, "localhost:31000", true, true)
icSession.Register()
qcSession := sessionutil.NewSession(context.Background(), Params.EtcdCfg.MetaRootPath.GetValue(), etcdCli)
qcSession.Init(typeutil.QueryCoordRole, "localhost:19532", true, true)
qcSession.Register()
svr := CreateServer(context.TODO(), factory, opts...)
svr.SetEtcdClient(etcdCli)
svr.dataNodeCreator = func(ctx context.Context, addr string) (types.DataNode, error) {

View File

@ -427,7 +427,6 @@ var roles = map[string]struct{}{
typeutil.RootCoordRole: {},
typeutil.QueryCoordRole: {},
typeutil.DataCoordRole: {},
typeutil.IndexCoordRole: {},
typeutil.QueryNodeRole: {},
typeutil.DataNodeRole: {},
typeutil.IndexNodeRole: {},

View File

@ -49,7 +49,6 @@ func TestConnectionManager(t *testing.T) {
cm.AddDependency(typeutil.RootCoordRole)
cm.AddDependency(typeutil.QueryCoordRole)
cm.AddDependency(typeutil.DataCoordRole)
cm.AddDependency(typeutil.IndexCoordRole)
cm.AddDependency(typeutil.QueryNodeRole)
cm.AddDependency(typeutil.DataNodeRole)
cm.AddDependency(typeutil.IndexNodeRole)

View File

@ -25,7 +25,6 @@ import (
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/stretchr/testify/assert"
clientv3 "go.etcd.io/etcd/client/v3"
@ -99,9 +98,6 @@ func (*MockDataCoord) SetAddress(address string) {
func (m *MockDataCoord) SetEtcdClient(etcdClient *clientv3.Client) {
}
func (m *MockDataCoord) SetIndexCoord(indexCoord types.IndexCoord) {
}
func (m *MockDataCoord) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
return m.states, m.err
}

View File

@ -1,297 +0,0 @@
// 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 grpcindexcoordclient
import (
"context"
"fmt"
"time"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap"
"google.golang.org/grpc"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/grpcclient"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/typeutil"
)
var Params *paramtable.ComponentParam = paramtable.Get()
// Client is the grpc client of IndexCoord.
type Client struct {
grpcClient grpcclient.GrpcClient[indexpb.IndexCoordClient]
sess *sessionutil.Session
}
// NewClient creates a new IndexCoord client.
func NewClient(ctx context.Context, metaRoot string, etcdCli *clientv3.Client) (*Client, error) {
sess := sessionutil.NewSession(ctx, metaRoot, etcdCli)
if sess == nil {
err := fmt.Errorf("new session error, maybe can not connect to etcd")
log.Debug("IndexCoordClient NewClient failed", zap.Error(err))
return nil, err
}
ClientParams := &Params.IndexCoordGrpcClientCfg
client := &Client{
grpcClient: &grpcclient.ClientBase[indexpb.IndexCoordClient]{
ClientMaxRecvSize: ClientParams.ClientMaxRecvSize.GetAsInt(),
ClientMaxSendSize: ClientParams.ClientMaxSendSize.GetAsInt(),
DialTimeout: ClientParams.DialTimeout.GetAsDuration(time.Millisecond),
KeepAliveTime: ClientParams.KeepAliveTime.GetAsDuration(time.Millisecond),
KeepAliveTimeout: ClientParams.KeepAliveTimeout.GetAsDuration(time.Millisecond),
RetryServiceNameConfig: "milvus.proto.index.IndexCoord",
MaxAttempts: ClientParams.MaxAttempts.GetAsInt(),
InitialBackoff: float32(ClientParams.InitialBackoff.GetAsFloat()),
MaxBackoff: float32(ClientParams.MaxBackoff.GetAsFloat()),
BackoffMultiplier: float32(ClientParams.BackoffMultiplier.GetAsFloat()),
},
sess: sess,
}
client.grpcClient.SetRole(typeutil.IndexCoordRole)
client.grpcClient.SetGetAddrFunc(client.getIndexCoordAddr)
client.grpcClient.SetNewGrpcClientFunc(client.newGrpcClient)
return client, nil
}
// Init initializes IndexCoord's grpc client.
func (c *Client) Init() error {
return nil
}
// Start starts IndexCoord's client service. But it does nothing here.
func (c *Client) Start() error {
return nil
}
// Stop stops IndexCoord's grpc client.
func (c *Client) Stop() error {
return c.grpcClient.Close()
}
// Register dummy
func (c *Client) Register() error {
return nil
}
func (c *Client) getIndexCoordAddr() (string, error) {
key := c.grpcClient.GetRole()
msess, _, err := c.sess.GetSessions(key)
if err != nil {
log.Debug("IndexCoordClient GetSessions failed", zap.Any("key", key), zap.Error(err))
return "", err
}
log.Debug("IndexCoordClient GetSessions success", zap.Any("key", key), zap.Any("msess", msess))
ms, ok := msess[key]
if !ok {
log.Debug("IndexCoordClient msess key not existed", zap.Any("key", key), zap.Any("len of msess", len(msess)))
return "", fmt.Errorf("find no available indexcoord, check indexcoord state")
}
return ms.Address, nil
}
// newGrpcClient create a new grpc client of IndexCoord.
func (c *Client) newGrpcClient(cc *grpc.ClientConn) indexpb.IndexCoordClient {
return indexpb.NewIndexCoordClient(cc)
}
// GetComponentStates gets the component states of IndexCoord.
func (c *Client) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client indexpb.IndexCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
}
return client.GetComponentStates(ctx, &milvuspb.GetComponentStatesRequest{})
})
if err != nil || ret == nil {
return nil, err
}
return ret.(*milvuspb.ComponentStates), err
}
// GetStatisticsChannel gets the statistics channel of IndexCoord.
func (c *Client) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client indexpb.IndexCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
}
return client.GetStatisticsChannel(ctx, &internalpb.GetStatisticsChannelRequest{})
})
if err != nil || ret == nil {
return nil, err
}
return ret.(*milvuspb.StringResponse), err
}
// CreateIndex sends the build index request to IndexCoord.
func (c *Client) CreateIndex(ctx context.Context, req *indexpb.CreateIndexRequest) (*commonpb.Status, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client indexpb.IndexCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
}
return client.CreateIndex(ctx, req)
})
if err != nil || ret == nil {
return nil, err
}
return ret.(*commonpb.Status), err
}
// GetIndexState gets the index states from IndexCoord.
func (c *Client) GetIndexState(ctx context.Context, req *indexpb.GetIndexStateRequest) (*indexpb.GetIndexStateResponse, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client indexpb.IndexCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
}
return client.GetIndexState(ctx, req)
})
if err != nil || ret == nil {
return nil, err
}
return ret.(*indexpb.GetIndexStateResponse), err
}
// GetSegmentIndexState gets the index states from IndexCoord.
func (c *Client) GetSegmentIndexState(ctx context.Context, req *indexpb.GetSegmentIndexStateRequest) (*indexpb.GetSegmentIndexStateResponse, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client indexpb.IndexCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
}
return client.GetSegmentIndexState(ctx, req)
})
if err != nil || ret == nil {
return nil, err
}
return ret.(*indexpb.GetSegmentIndexStateResponse), err
}
// GetIndexInfos gets the index file paths from IndexCoord.
func (c *Client) GetIndexInfos(ctx context.Context, req *indexpb.GetIndexInfoRequest) (*indexpb.GetIndexInfoResponse, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client indexpb.IndexCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
}
return client.GetIndexInfos(ctx, req)
})
if err != nil || ret == nil {
return nil, err
}
return ret.(*indexpb.GetIndexInfoResponse), err
}
// DescribeIndex describe the index info of the collection.
func (c *Client) DescribeIndex(ctx context.Context, req *indexpb.DescribeIndexRequest) (*indexpb.DescribeIndexResponse, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client indexpb.IndexCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
}
return client.DescribeIndex(ctx, req)
})
if err != nil || ret == nil {
return nil, err
}
return ret.(*indexpb.DescribeIndexResponse), err
}
// GetIndexBuildProgress describe the progress of the index.
func (c *Client) GetIndexBuildProgress(ctx context.Context, req *indexpb.GetIndexBuildProgressRequest) (*indexpb.GetIndexBuildProgressResponse, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client indexpb.IndexCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
}
return client.GetIndexBuildProgress(ctx, req)
})
if err != nil || ret == nil {
return nil, err
}
return ret.(*indexpb.GetIndexBuildProgressResponse), err
}
// DropIndex sends the drop index request to IndexCoord.
func (c *Client) DropIndex(ctx context.Context, req *indexpb.DropIndexRequest) (*commonpb.Status, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client indexpb.IndexCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
}
return client.DropIndex(ctx, req)
})
if err != nil || ret == nil {
return nil, err
}
return ret.(*commonpb.Status), err
}
// ShowConfigurations gets specified configurations para of IndexCoord
func (c *Client) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
commonpbutil.FillMsgBaseFromClient(paramtable.GetNodeID(), commonpbutil.WithTargetID(c.sess.ServerID)),
)
ret, err := c.grpcClient.ReCall(ctx, func(client indexpb.IndexCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
}
return client.ShowConfigurations(ctx, req)
})
if err != nil || ret == nil {
return nil, err
}
return ret.(*internalpb.ShowConfigurationsResponse), err
}
// GetMetrics gets the metrics info of IndexCoord.
func (c *Client) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
commonpbutil.FillMsgBaseFromClient(paramtable.GetNodeID(), commonpbutil.WithTargetID(c.sess.ServerID)),
)
ret, err := c.grpcClient.ReCall(ctx, func(client indexpb.IndexCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
}
return client.GetMetrics(ctx, req)
})
if err != nil || ret == nil {
return nil, err
}
return ret.(*milvuspb.GetMetricsResponse), err
}
func (c *Client) CheckHealth(ctx context.Context, req *milvuspb.CheckHealthRequest) (*milvuspb.CheckHealthResponse, error) {
ret, err := c.grpcClient.ReCall(ctx, func(client indexpb.IndexCoordClient) (any, error) {
if !funcutil.CheckCtxValid(ctx) {
return nil, ctx.Err()
}
return client.CheckHealth(ctx, req)
})
if err != nil || ret == nil {
return nil, err
}
return ret.(*milvuspb.CheckHealthResponse), err
}

View File

@ -1,215 +0,0 @@
// 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 grpcindexcoordclient
import (
"context"
"testing"
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
grpcindexcoord "github.com/milvus-io/milvus/internal/distributed/indexcoord"
"github.com/milvus-io/milvus/internal/indexcoord"
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/util/dependency"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/internal/util/typeutil"
)
func TestIndexCoordClient(t *testing.T) {
paramtable.Init()
ctx := context.Background()
factory := dependency.NewDefaultFactory(true)
server, err := grpcindexcoord.NewServer(ctx, factory)
assert.NoError(t, err)
icm := indexcoord.NewIndexCoordMock()
etcdCli, err := etcd.GetEtcdClient(
Params.EtcdCfg.UseEmbedEtcd.GetAsBool(),
Params.EtcdCfg.EtcdUseSSL.GetAsBool(),
Params.EtcdCfg.Endpoints.GetAsStrings(),
Params.EtcdCfg.EtcdTLSCert.GetValue(),
Params.EtcdCfg.EtcdTLSKey.GetValue(),
Params.EtcdCfg.EtcdTLSCACert.GetValue(),
Params.EtcdCfg.EtcdTLSMinVersion.GetValue())
assert.NoError(t, err)
var address string
icm.CallSetAddress = func(addr string) {
address = addr
}
icm.CallRegister = func() error {
session := sessionutil.NewSession(context.Background(), indexcoord.Params.EtcdCfg.MetaRootPath.GetValue(), etcdCli)
session.Init(typeutil.IndexCoordRole, address, true, false)
session.Register()
return err
}
icm.CallStop = func() error {
etcdKV := etcdkv.NewEtcdKV(etcdCli, indexcoord.Params.EtcdCfg.MetaRootPath.GetValue())
err = etcdKV.RemoveWithPrefix("session/" + typeutil.IndexCoordRole)
return err
}
err = server.SetClient(icm)
assert.NoError(t, err)
rcm := indexcoord.NewRootCoordMock()
err = server.SetRootCoord(rcm)
assert.NoError(t, err)
dcm := indexcoord.NewDataCoordMock()
err = server.SetDataCoord(dcm)
assert.NoError(t, err)
err = server.Run()
assert.NoError(t, err)
//
//etcdCli, err := etcd.GetEtcdClient(&indexcoord.Params.EtcdCfg)
//assert.NoError(t, err)
icc, err := NewClient(ctx, indexcoord.Params.EtcdCfg.MetaRootPath.GetValue(), etcdCli)
assert.NoError(t, err)
assert.NotNil(t, icc)
err = icc.Init()
assert.NoError(t, err)
err = icc.Register()
assert.NoError(t, err)
err = icc.Start()
assert.NoError(t, err)
t.Run("GetComponentStates", func(t *testing.T) {
states, err := icc.GetComponentStates(ctx)
assert.NoError(t, err)
assert.Equal(t, commonpb.StateCode_Healthy, states.State.StateCode)
assert.Equal(t, commonpb.ErrorCode_Success, states.Status.ErrorCode)
})
t.Run("GetStatisticsChannel", func(t *testing.T) {
resp, err := icc.GetStatisticsChannel(ctx)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
})
t.Run("CreateIndex", func(t *testing.T) {
req := &indexpb.CreateIndexRequest{
CollectionID: 0,
FieldID: 0,
IndexName: "default",
}
resp, err := icc.CreateIndex(ctx, req)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.ErrorCode)
})
t.Run("GetIndexState", func(t *testing.T) {
req := &indexpb.GetIndexStateRequest{
CollectionID: 0,
IndexName: "index",
}
resp, err := icc.GetIndexState(ctx, req)
assert.NoError(t, err)
assert.Equal(t, commonpb.IndexState_Finished, resp.State)
})
t.Run("GetSegmentIndexState", func(t *testing.T) {
req := &indexpb.GetSegmentIndexStateRequest{
CollectionID: 1,
IndexName: "index",
SegmentIDs: []int64{1, 2},
}
resp, err := icc.GetSegmentIndexState(ctx, req)
assert.NoError(t, err)
assert.Equal(t, len(req.SegmentIDs), len(resp.States))
})
t.Run("GetIndexInfos", func(t *testing.T) {
req := &indexpb.GetIndexInfoRequest{
CollectionID: 0,
SegmentIDs: []int64{0},
IndexName: "index",
}
resp, err := icc.GetIndexInfos(ctx, req)
assert.NoError(t, err)
assert.Equal(t, len(req.SegmentIDs), len(resp.SegmentInfo))
})
t.Run("DescribeIndex", func(t *testing.T) {
req := &indexpb.DescribeIndexRequest{
CollectionID: 1,
IndexName: "",
}
resp, err := icc.DescribeIndex(ctx, req)
assert.NoError(t, err)
assert.Equal(t, 1, len(resp.IndexInfos))
})
t.Run("GetIndexBuildProgress", func(t *testing.T) {
req := &indexpb.GetIndexBuildProgressRequest{
CollectionID: 1,
IndexName: "default",
}
resp, err := icc.GetIndexBuildProgress(ctx, req)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
assert.Equal(t, resp.TotalRows, resp.IndexedRows)
})
t.Run("DropIndex", func(t *testing.T) {
req := &indexpb.DropIndexRequest{
CollectionID: 0,
IndexName: "default",
}
resp, err := icc.DropIndex(ctx, req)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.ErrorCode)
})
t.Run("ShowConfigurations", func(t *testing.T) {
req := &internalpb.ShowConfigurationsRequest{
Pattern: "",
}
resp, err := icc.ShowConfigurations(ctx, req)
assert.Nil(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
})
t.Run("GetMetrics", func(t *testing.T) {
req := &milvuspb.GetMetricsRequest{
Request: "",
}
resp, err := icc.GetMetrics(ctx, req)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
assert.Equal(t, typeutil.IndexCoordRole, resp.ComponentName)
})
t.Run("CheckHealth", func(t *testing.T) {
req := &milvuspb.CheckHealthRequest{}
resp, err := icc.CheckHealth(ctx, req)
assert.NoError(t, err)
assert.Equal(t, true, resp.IsHealthy)
})
err = server.Stop()
assert.NoError(t, err)
err = icc.Stop()
assert.NoError(t, err)
}

View File

@ -1,363 +0,0 @@
// 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 grpcindexcoord
import (
"context"
"fmt"
"io"
"net"
"strconv"
"sync"
"time"
clientv3 "go.etcd.io/etcd/client/v3"
"go.uber.org/zap"
"google.golang.org/grpc"
"google.golang.org/grpc/keepalive"
grpc_middleware "github.com/grpc-ecosystem/go-grpc-middleware"
ot "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
dcc "github.com/milvus-io/milvus/internal/distributed/datacoord/client"
rcc "github.com/milvus-io/milvus/internal/distributed/rootcoord/client"
"github.com/milvus-io/milvus/internal/indexcoord"
ic "github.com/milvus-io/milvus/internal/indexcoord"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/dependency"
"github.com/milvus-io/milvus/internal/util/etcd"
"github.com/milvus-io/milvus/internal/util/funcutil"
"github.com/milvus-io/milvus/internal/util/logutil"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/trace"
"github.com/milvus-io/milvus/internal/util/typeutil"
)
// UniqueID is an alias of int64, is used as a unique identifier for the request.
type UniqueID = typeutil.UniqueID
// Server is the grpc wrapper of IndexCoord.
type Server struct {
indexcoord types.IndexCoordComponent
grpcServer *grpc.Server
grpcErrChan chan error
loopCtx context.Context
loopCancel func()
loopWg sync.WaitGroup
etcdCli *clientv3.Client
dataCoord types.DataCoord
rootCoord types.RootCoord
closer io.Closer
}
// Run initializes and starts IndexCoord's grpc service.
func (s *Server) Run() error {
if err := s.init(); err != nil {
return err
}
log.Debug("IndexCoord init done ...")
if err := s.start(); err != nil {
return err
}
log.Debug("IndexCoord start done ...")
return nil
}
// init initializes IndexCoord's grpc service.
func (s *Server) init() error {
etcdConfig := &paramtable.Get().EtcdCfg
Params := &paramtable.Get().IndexCoordGrpcServerCfg
closer := trace.InitTracing("IndexCoord")
s.closer = closer
etcdCli, err := etcd.GetEtcdClient(
etcdConfig.UseEmbedEtcd.GetAsBool(),
etcdConfig.EtcdUseSSL.GetAsBool(),
etcdConfig.Endpoints.GetAsStrings(),
etcdConfig.EtcdTLSCert.GetValue(),
etcdConfig.EtcdTLSKey.GetValue(),
etcdConfig.EtcdTLSCACert.GetValue(),
etcdConfig.EtcdTLSMinVersion.GetValue())
if err != nil {
log.Debug("IndexCoord connect to etcd failed", zap.Error(err))
return err
}
s.etcdCli = etcdCli
s.indexcoord.SetEtcdClient(s.etcdCli)
s.indexcoord.SetAddress(fmt.Sprintf("%s:%d", Params.IP, Params.Port.GetAsInt()))
s.loopWg.Add(1)
go s.startGrpcLoop(Params.Port.GetAsInt())
// wait for grpc IndexCoord loop start
if err := <-s.grpcErrChan; err != nil {
log.Error("IndexCoord", zap.Any("init error", err))
return err
}
if err := s.indexcoord.Init(); err != nil {
log.Error("IndexCoord", zap.Any("init error", err))
return err
}
// --- RootCoord ---
if s.rootCoord == nil {
s.rootCoord, err = rcc.NewClient(s.loopCtx, ic.Params.EtcdCfg.MetaRootPath.GetValue(), s.etcdCli)
if err != nil {
log.Debug("IndexCoord try to new RootCoord client failed", zap.Error(err))
panic(err)
}
}
if err = s.rootCoord.Init(); err != nil {
log.Debug("IndexCoord RootCoord client init failed", zap.Error(err))
panic(err)
}
if err = s.rootCoord.Start(); err != nil {
log.Debug("IndexCoord RootCoord client start failed", zap.Error(err))
panic(err)
}
log.Debug("IndexCoord try to wait for RootCoord ready")
err = funcutil.WaitForComponentHealthy(s.loopCtx, s.rootCoord, typeutil.RootCoordRole, 1000000, time.Millisecond*200)
if err != nil {
log.Debug("IndexCoord wait for RootCoord ready failed", zap.Error(err))
panic(err)
}
if err := s.SetRootCoord(s.rootCoord); err != nil {
panic(err)
}
// --- DataCoord ---
if s.dataCoord == nil {
s.dataCoord, err = dcc.NewClient(s.loopCtx, ic.Params.EtcdCfg.MetaRootPath.GetValue(), s.etcdCli)
if err != nil {
log.Debug("IndexCoord try to new DataCoord client failed", zap.Error(err))
panic(err)
}
}
if err = s.dataCoord.Init(); err != nil {
log.Debug("IndexCoord DataCoordClient Init failed", zap.Error(err))
panic(err)
}
if err = s.dataCoord.Start(); err != nil {
log.Debug("IndexCoord DataCoordClient Start failed", zap.Error(err))
panic(err)
}
log.Debug("IndexCoord try to wait for DataCoord ready")
err = funcutil.WaitForComponentHealthy(s.loopCtx, s.dataCoord, typeutil.DataCoordRole, 1000000, time.Millisecond*200)
if err != nil {
log.Debug("IndexCoord wait for DataCoord ready failed", zap.Error(err))
panic(err)
}
if err := s.SetDataCoord(s.dataCoord); err != nil {
panic(err)
}
return nil
}
// start starts IndexCoord's grpc service.
func (s *Server) start() error {
if err := s.indexcoord.Start(); err != nil {
return err
}
log.Debug("indexCoord started")
if err := s.indexcoord.Register(); err != nil {
log.Error("IndexCoord", zap.Any("register session error", err))
return err
}
log.Debug("IndexCoord registers service successfully")
return nil
}
// Stop stops IndexCoord's grpc service.
func (s *Server) Stop() error {
Params := &paramtable.Get().DataCoordGrpcServerCfg
log.Debug("Indexcoord stop", zap.String("Address", Params.GetAddress()))
if s.closer != nil {
if err := s.closer.Close(); err != nil {
return err
}
}
if s.indexcoord != nil {
s.indexcoord.Stop()
}
if s.etcdCli != nil {
defer s.etcdCli.Close()
}
s.loopCancel()
if s.grpcServer != nil {
log.Debug("Graceful stop grpc server...")
s.grpcServer.GracefulStop()
}
s.loopWg.Wait()
return nil
}
// SetClient sets the IndexCoord's instance.
func (s *Server) SetClient(indexCoordClient types.IndexCoordComponent) error {
s.indexcoord = indexCoordClient
return nil
}
// SetDataCoord sets the DataCoord's client for IndexCoord component.
func (s *Server) SetDataCoord(d types.DataCoord) error {
s.dataCoord = d
return s.indexcoord.SetDataCoord(d)
}
// SetRootCoord sets the RootCoord's client for IndexCoord component.
func (s *Server) SetRootCoord(d types.RootCoord) error {
s.rootCoord = d
return s.indexcoord.SetRootCoord(d)
}
// GetComponentStates gets the component states of IndexCoord.
func (s *Server) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) {
return s.indexcoord.GetComponentStates(ctx)
}
// GetStatisticsChannel gets the statistics channel of IndexCoord.
func (s *Server) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) {
return s.indexcoord.GetStatisticsChannel(ctx)
}
// CreateIndex sends the build index request to IndexCoord.
func (s *Server) CreateIndex(ctx context.Context, req *indexpb.CreateIndexRequest) (*commonpb.Status, error) {
return s.indexcoord.CreateIndex(ctx, req)
}
// GetIndexState gets the index states from IndexCoord.
// Deprecated: use DescribeIndex instead
func (s *Server) GetIndexState(ctx context.Context, req *indexpb.GetIndexStateRequest) (*indexpb.GetIndexStateResponse, error) {
return s.indexcoord.GetIndexState(ctx, req)
}
func (s *Server) GetSegmentIndexState(ctx context.Context, req *indexpb.GetSegmentIndexStateRequest) (*indexpb.GetSegmentIndexStateResponse, error) {
return s.indexcoord.GetSegmentIndexState(ctx, req)
}
// GetIndexInfos gets the index file paths from IndexCoord.
func (s *Server) GetIndexInfos(ctx context.Context, req *indexpb.GetIndexInfoRequest) (*indexpb.GetIndexInfoResponse, error) {
return s.indexcoord.GetIndexInfos(ctx, req)
}
// DescribeIndex gets all indexes of the collection.
func (s *Server) DescribeIndex(ctx context.Context, req *indexpb.DescribeIndexRequest) (*indexpb.DescribeIndexResponse, error) {
return s.indexcoord.DescribeIndex(ctx, req)
}
// DropIndex sends the drop index request to IndexCoord.
func (s *Server) DropIndex(ctx context.Context, request *indexpb.DropIndexRequest) (*commonpb.Status, error) {
return s.indexcoord.DropIndex(ctx, request)
}
// Deprecated: use DescribeIndex instead
func (s *Server) GetIndexBuildProgress(ctx context.Context, req *indexpb.GetIndexBuildProgressRequest) (*indexpb.GetIndexBuildProgressResponse, error) {
return s.indexcoord.GetIndexBuildProgress(ctx, req)
}
// ShowConfigurations gets specified configurations para of IndexCoord
func (s *Server) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
return s.indexcoord.ShowConfigurations(ctx, req)
}
// GetMetrics gets the metrics info of IndexCoord.
func (s *Server) GetMetrics(ctx context.Context, request *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
return s.indexcoord.GetMetrics(ctx, request)
}
func (s *Server) CheckHealth(ctx context.Context, request *milvuspb.CheckHealthRequest) (*milvuspb.CheckHealthResponse, error) {
return s.indexcoord.CheckHealth(ctx, request)
}
// startGrpcLoop starts the grep loop of IndexCoord component.
func (s *Server) startGrpcLoop(grpcPort int) {
defer s.loopWg.Done()
Params := &paramtable.Get().DataCoordGrpcServerCfg
var kaep = keepalive.EnforcementPolicy{
MinTime: 5 * time.Second, // If a client pings more than once every 5 seconds, terminate the connection
PermitWithoutStream: true, // Allow pings even when there are no active streams
}
var kasp = keepalive.ServerParameters{
Time: 60 * time.Second, // Ping the client if it is idle for 60 seconds to ensure the connection is still active
Timeout: 10 * time.Second, // Wait 10 second for the ping ack before assuming the connection is dead
}
log.Debug("IndexCoord", zap.String("network address", Params.IP), zap.Int("network port", grpcPort))
lis, err := net.Listen("tcp", ":"+strconv.Itoa(grpcPort))
if err != nil {
log.Warn("IndexCoord", zap.String("GrpcServer:failed to listen", err.Error()))
s.grpcErrChan <- err
return
}
ctx, cancel := context.WithCancel(s.loopCtx)
defer cancel()
opts := trace.GetInterceptorOpts()
s.grpcServer = grpc.NewServer(
grpc.KeepaliveEnforcementPolicy(kaep),
grpc.KeepaliveParams(kasp),
grpc.MaxRecvMsgSize(Params.ServerMaxRecvSize.GetAsInt()),
grpc.MaxSendMsgSize(Params.ServerMaxSendSize.GetAsInt()),
grpc.UnaryInterceptor(grpc_middleware.ChainUnaryServer(
ot.UnaryServerInterceptor(opts...),
logutil.UnaryTraceLoggerInterceptor)),
grpc.StreamInterceptor(grpc_middleware.ChainStreamServer(
ot.StreamServerInterceptor(opts...),
logutil.StreamTraceLoggerInterceptor)))
indexpb.RegisterIndexCoordServer(s.grpcServer, s)
go funcutil.CheckGrpcReady(ctx, s.grpcErrChan)
if err := s.grpcServer.Serve(lis); err != nil {
s.grpcErrChan <- err
}
log.Debug("IndexCoord grpcServer loop exit")
}
// NewServer create a new IndexCoord grpc server.
func NewServer(ctx context.Context, factory dependency.Factory) (*Server, error) {
ctx1, cancel := context.WithCancel(ctx)
serverImp, err := indexcoord.NewIndexCoord(ctx, factory)
if err != nil {
defer cancel()
return nil, err
}
s := &Server{
loopCtx: ctx1,
loopCancel: cancel,
indexcoord: serverImp,
grpcErrChan: make(chan error),
}
return s, nil
}

View File

@ -1,169 +0,0 @@
// 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 grpcindexcoord
import (
"context"
"testing"
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus/internal/indexcoord"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/util/dependency"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/typeutil"
)
func TestIndexCoordinateServer(t *testing.T) {
paramtable.Init()
ctx := context.Background()
factory := dependency.NewDefaultFactory(true)
server, err := NewServer(ctx, factory)
assert.NoError(t, err)
assert.NotNil(t, server)
indexCoordClient := indexcoord.NewIndexCoordMock()
err = server.SetClient(indexCoordClient)
assert.NoError(t, err)
rcm := indexcoord.NewRootCoordMock()
server.rootCoord = rcm
dcm := indexcoord.NewDataCoordMock()
server.dataCoord = dcm
err = server.Run()
assert.NoError(t, err)
t.Run("GetComponentStates", func(t *testing.T) {
req := &milvuspb.GetComponentStatesRequest{}
states, err := server.GetComponentStates(ctx, req)
assert.NoError(t, err)
assert.Equal(t, commonpb.StateCode_Healthy, states.State.StateCode)
})
t.Run("GetStatisticsChannel", func(t *testing.T) {
req := &internalpb.GetStatisticsChannelRequest{}
resp, err := server.GetStatisticsChannel(ctx, req)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
})
t.Run("CreateIndex", func(t *testing.T) {
req := &indexpb.CreateIndexRequest{
CollectionID: 0,
FieldID: 0,
IndexName: "index",
}
resp, err := server.CreateIndex(ctx, req)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.ErrorCode)
})
t.Run("GetIndexState", func(t *testing.T) {
req := &indexpb.GetIndexStateRequest{
CollectionID: 0,
IndexName: "index",
}
resp, err := server.GetIndexState(ctx, req)
assert.NoError(t, err)
assert.Equal(t, commonpb.IndexState_Finished, resp.State)
})
t.Run("GetSegmentIndexState", func(t *testing.T) {
req := &indexpb.GetSegmentIndexStateRequest{
CollectionID: 1,
IndexName: "index",
SegmentIDs: []UniqueID{1, 2},
}
resp, err := server.GetSegmentIndexState(ctx, req)
assert.NoError(t, err)
assert.Equal(t, len(req.SegmentIDs), len(resp.States))
})
t.Run("GetIndexInfos", func(t *testing.T) {
req := &indexpb.GetIndexInfoRequest{
CollectionID: 0,
SegmentIDs: []UniqueID{0},
IndexName: "index",
}
resp, err := server.GetIndexInfos(ctx, req)
assert.NoError(t, err)
assert.Equal(t, len(req.SegmentIDs), len(resp.SegmentInfo))
})
t.Run("DescribeIndex", func(t *testing.T) {
req := &indexpb.DescribeIndexRequest{
CollectionID: 1,
IndexName: "",
}
resp, err := server.DescribeIndex(ctx, req)
assert.NoError(t, err)
assert.Equal(t, 1, len(resp.IndexInfos))
})
t.Run("GetIndexBuildProgress", func(t *testing.T) {
req := &indexpb.GetIndexBuildProgressRequest{
CollectionID: 1,
IndexName: "default",
}
resp, err := server.GetIndexBuildProgress(ctx, req)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
assert.Equal(t, resp.TotalRows, resp.IndexedRows)
})
t.Run("DropIndex", func(t *testing.T) {
req := &indexpb.DropIndexRequest{
CollectionID: 0,
IndexName: "default",
}
resp, err := server.DropIndex(ctx, req)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.ErrorCode)
})
t.Run("ShowConfigurations", func(t *testing.T) {
req := &internalpb.ShowConfigurationsRequest{
Pattern: "",
}
resp, err := server.ShowConfigurations(ctx, req)
assert.Nil(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
})
t.Run("GetMetrics", func(t *testing.T) {
req := &milvuspb.GetMetricsRequest{
Request: "",
}
resp, err := server.GetMetrics(ctx, req)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.Status.ErrorCode)
assert.Equal(t, typeutil.IndexCoordRole, resp.ComponentName)
})
t.Run("CheckHealth", func(t *testing.T) {
ret, err := server.CheckHealth(ctx, nil)
assert.Nil(t, err)
assert.Equal(t, true, ret.IsHealthy)
})
err = server.Stop()
assert.NoError(t, err)
}

View File

@ -795,10 +795,6 @@ func (m *MockProxy) SetDataCoordClient(dataCoord types.DataCoord) {
}
func (m *MockProxy) SetIndexCoordClient(indexCoord types.IndexCoord) {
}
func (m *MockProxy) SetQueryCoordClient(queryCoord types.QueryCoord) {
}

View File

@ -291,12 +291,6 @@ func (s *Server) SetDataCoord(d types.DataCoord) error {
return nil
}
// SetIndexCoord sets the IndexCoord's client for QueryCoord component.
func (s *Server) SetIndexCoord(d types.IndexCoord) error {
s.queryCoord.SetIndexCoord(d)
return nil
}
// GetComponentStates gets the component states of QueryCoord.
func (s *Server) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) {
return s.queryCoord.GetComponentStates(ctx)

View File

@ -88,10 +88,6 @@ func (m *MockQueryCoord) SetDataCoord(types.DataCoord) error {
return nil
}
func (m *MockQueryCoord) SetIndexCoord(coord types.IndexCoord) error {
return nil
}
func (m *MockQueryCoord) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
log.Debug("MockQueryCoord::WaitForComponentStates")
return m.states, m.err

View File

@ -138,10 +138,6 @@ func (m *MockQueryNode) SetRootCoord(rc types.RootCoord) error {
return m.err
}
func (m *MockQueryNode) SetIndexCoord(index types.IndexCoord) error {
return m.err
}
func (m *MockQueryNode) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
return m.configResp, m.err
}
@ -191,42 +187,6 @@ func (m *MockRootCoord) GetComponentStates(ctx context.Context) (*milvuspb.Compo
}, nil
}
///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
type MockIndexCoord struct {
types.IndexCoord
initErr error
startErr error
regErr error
stopErr error
stateErr commonpb.ErrorCode
}
func (m *MockIndexCoord) Init() error {
return m.initErr
}
func (m *MockIndexCoord) Start() error {
return m.startErr
}
func (m *MockIndexCoord) Stop() error {
return m.stopErr
}
func (m *MockIndexCoord) Register() error {
return m.regErr
}
func (m *MockIndexCoord) SetEtcdClient(client *clientv3.Client) {
}
func (m *MockIndexCoord) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
return &milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{StateCode: commonpb.StateCode_Healthy},
Status: &commonpb.Status{ErrorCode: m.stateErr},
}, nil
}
///////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
func TestMain(m *testing.M) {

View File

@ -69,9 +69,6 @@ func (m *mockCore) SetEtcdClient(etcdClient *clientv3.Client) {
func (m *mockCore) SetDataCoord(context.Context, types.DataCoord) error {
return nil
}
func (m *mockCore) SetIndexCoord(types.IndexCoord) error {
return nil
}
func (m *mockCore) SetQueryCoord(types.QueryCoord) error {
return nil
@ -122,18 +119,6 @@ func (m *mockDataCoord) Stop() error {
return fmt.Errorf("stop error")
}
type mockIndex struct {
types.IndexCoord
}
func (m *mockIndex) Init() error {
return nil
}
func (m *mockIndex) Stop() error {
return fmt.Errorf("stop error")
}
type mockQuery struct {
types.QueryCoord
}

View File

@ -1,10 +0,0 @@
# order by contributions
reviewers:
- xiaocai2333
- sunby
- DragonDriver
approvers:
- maintainers

View File

@ -1,32 +0,0 @@
// 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 indexcoord
import "time"
const (
// IndexAddTaskName is the name of the operation to add index task.
IndexAddTaskName = "IndexAddTask"
CreateIndexTaskName = "CreateIndexTask"
flatIndex = "FLAT"
binFlatIndex = "BIN_FLAT"
diskAnnIndex = "DISKANN"
invalidIndex = "invalid"
reqTimeoutInterval = time.Second * 10
)

View File

@ -1,51 +0,0 @@
// 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 indexcoord
import (
"errors"
"fmt"
)
var (
ErrCompareVersion = errors.New("failed to save meta in etcd because version compare failure")
ErrNotIndexExist = errors.New("there is no index")
ErrSegmentNotFound = errors.New("failed to get segment")
ErrMsgAmbiguousIndexName = "there are multiple indexes, please specify the index_name"
)
// errIndexNodeIsNotOnService return an error that the specified IndexNode is not exists.
func errIndexNodeIsNotOnService(id UniqueID) error {
return fmt.Errorf("index node %d is not on service", id)
}
// msgIndexCoordIsUnhealthy return an error that the IndexCoord is not healthy.
func msgIndexCoordIsUnhealthy(coordID UniqueID) string {
return fmt.Sprintf("IndexCoord %d is not ready", coordID)
}
func errIndexCoordIsUnhealthy(coordID UniqueID) error {
return errors.New(msgIndexCoordIsUnhealthy(coordID))
}
func msgSegmentNotFound(segID UniqueID) string {
return fmt.Sprintf("failed to get segment %d", segID)
}
func errSegmentNotFound(segID UniqueID) error {
return fmt.Errorf("%w %d", ErrSegmentNotFound, segID)
}

View File

@ -1,61 +0,0 @@
// 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 indexcoord
import (
"errors"
"testing"
"github.com/stretchr/testify/assert"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/log"
)
func TestErrIndexNodeIsNotOnService(t *testing.T) {
indexNodeIDList := []UniqueID{
1,
}
for _, id := range indexNodeIDList {
log.Info("TestErrIndexNodeIsNotOnService",
zap.Error(errIndexNodeIsNotOnService(id)))
}
}
func TestMsgIndexCoordIsUnhealthy(t *testing.T) {
nodeIDList := []UniqueID{1, 2, 3}
for _, nodeID := range nodeIDList {
log.Info("TestMsgIndexCoordIsUnhealthy", zap.String("msg", msgIndexCoordIsUnhealthy(nodeID)))
}
}
func TestErrIndexCoordIsUnhealthy(t *testing.T) {
nodeIDList := []UniqueID{1, 2, 3}
for _, nodeID := range nodeIDList {
log.Info("TestErrIndexCoordIsUnhealthy", zap.Error(errIndexCoordIsUnhealthy(nodeID)))
}
}
func TestErrSegmentNotExist(t *testing.T) {
segID := UniqueID(435846569243121068)
errMsg := msgSegmentNotFound(segID)
log.Info("TestErrSegmentNotExist", zap.String("errMsg", errMsg))
err := errSegmentNotFound(segID)
assert.True(t, errors.Is(err, ErrSegmentNotFound))
}

View File

@ -1,360 +0,0 @@
// 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 indexcoord
import (
"context"
"errors"
"fmt"
"sort"
"strconv"
"sync"
"time"
"github.com/golang/protobuf/proto"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/kv"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/util"
"github.com/milvus-io/milvus/internal/util/timerecord"
)
type flushedSegmentWatcher struct {
ctx context.Context
cancel context.CancelFunc
kvClient kv.MetaKv
wg sync.WaitGroup
scheduleDuration time.Duration
internalTaskMutex sync.RWMutex
internalNotify chan struct{}
etcdRevision int64
meta *metaTable
builder *indexBuilder
ic *IndexCoord
handoff *handoff
internalTasks map[UniqueID]*internalTask
}
type internalTask struct {
state indexTaskState
segmentInfo *datapb.SegmentInfo
}
func newFlushSegmentWatcher(ctx context.Context, kv kv.MetaKv, meta *metaTable, builder *indexBuilder,
handoff *handoff, ic *IndexCoord) (*flushedSegmentWatcher, error) {
ctx, cancel := context.WithCancel(ctx)
fsw := &flushedSegmentWatcher{
ctx: ctx,
cancel: cancel,
kvClient: kv,
wg: sync.WaitGroup{},
internalTaskMutex: sync.RWMutex{},
scheduleDuration: time.Second * 3,
internalNotify: make(chan struct{}, 1),
meta: meta,
builder: builder,
handoff: handoff,
ic: ic,
}
err := fsw.reloadFromKV()
if err != nil {
return nil, err
}
return fsw, nil
}
func (fsw *flushedSegmentWatcher) reloadFromKV() error {
record := timerecord.NewTimeRecorder("indexcoord")
log.Ctx(fsw.ctx).Info("flushSegmentWatcher reloadFromKV")
fsw.internalTasks = make(map[UniqueID]*internalTask)
_, values, version, err := fsw.kvClient.LoadWithRevision(util.FlushedSegmentPrefix)
if err != nil {
log.Ctx(fsw.ctx).Error("flushSegmentWatcher reloadFromKV fail", zap.String("prefix", util.FlushedSegmentPrefix), zap.Error(err))
return err
}
for _, value := range values {
segmentInfo := &datapb.SegmentInfo{}
if err := proto.Unmarshal([]byte(value), segmentInfo); err != nil {
// just for backward compatibility
segID, err := strconv.ParseInt(value, 10, 64)
if err != nil {
log.Ctx(fsw.ctx).Error("flushSegmentWatcher unmarshal fail", zap.String("value", value), zap.Error(err))
return err
}
segmentInfo.ID = segID
}
fsw.enqueueInternalTask(segmentInfo)
}
fsw.etcdRevision = version
log.Ctx(fsw.ctx).Info("flushSegmentWatcher reloadFromKV success", zap.Int64("etcdRevision", version))
record.Record("flushedSegmentWatcher reloadFromKV")
return nil
}
func (fsw *flushedSegmentWatcher) Start() {
fsw.wg.Add(1)
go fsw.internalScheduler()
}
func (fsw *flushedSegmentWatcher) Stop() {
fsw.cancel()
fsw.wg.Wait()
}
func (fsw *flushedSegmentWatcher) enqueueInternalTask(segment *datapb.SegmentInfo) {
defer fsw.internalNotifyFunc()
fsw.internalTaskMutex.Lock()
defer fsw.internalTaskMutex.Unlock()
if _, ok := fsw.internalTasks[segment.GetID()]; !ok {
if segment.GetIsFake() {
fsw.internalTasks[segment.GetID()] = &internalTask{
state: indexTaskPrepare,
segmentInfo: segment,
}
} else {
fsw.internalTasks[segment.GetID()] = &internalTask{
state: indexTaskPrepare,
segmentInfo: nil,
}
}
}
log.Ctx(fsw.ctx).Info("flushed segment task enqueue successfully", zap.Int64("segID", segment.GetID()),
zap.Bool("isFake", segment.GetIsFake()))
}
func (fsw *flushedSegmentWatcher) internalScheduler() {
log.Ctx(fsw.ctx).Info("IndexCoord flushedSegmentWatcher internalScheduler start...")
defer fsw.wg.Done()
ticker := time.NewTicker(fsw.scheduleDuration)
defer ticker.Stop()
for {
select {
case <-fsw.ctx.Done():
log.Ctx(fsw.ctx).Warn("IndexCoord flushedSegmentWatcher context done")
return
case <-ticker.C:
fsw.internalRun()
case <-fsw.internalNotify:
fsw.internalRun()
}
}
}
func (fsw *flushedSegmentWatcher) internalRun() {
fsw.internalTaskMutex.RLock()
segmentIDs := make([]UniqueID, 0, len(fsw.internalTasks))
if len(fsw.internalTasks) > 0 {
for segID := range fsw.internalTasks {
segmentIDs = append(segmentIDs, segID)
}
sort.Slice(segmentIDs, func(i, j int) bool {
return segmentIDs[i] < segmentIDs[j]
})
}
fsw.internalTaskMutex.RUnlock()
for _, segmentID := range segmentIDs {
fsw.internalProcess(segmentID)
}
}
func (fsw *flushedSegmentWatcher) internalNotifyFunc() {
select {
case fsw.internalNotify <- struct{}{}:
default:
}
}
func (fsw *flushedSegmentWatcher) Len() int {
fsw.internalTaskMutex.RLock()
defer fsw.internalTaskMutex.RUnlock()
return len(fsw.internalTasks)
}
func (fsw *flushedSegmentWatcher) updateInternalTaskState(segID UniqueID, state indexTaskState) {
fsw.internalTaskMutex.Lock()
defer fsw.internalTaskMutex.Unlock()
log.Ctx(fsw.ctx).Debug("flushedSegmentWatcher updateInternalTaskState", zap.Int64("segID", segID),
zap.String("state", state.String()))
if _, ok := fsw.internalTasks[segID]; ok {
fsw.internalTasks[segID].state = state
}
}
func (fsw *flushedSegmentWatcher) deleteInternalTask(segID UniqueID) {
fsw.internalTaskMutex.Lock()
defer fsw.internalTaskMutex.Unlock()
delete(fsw.internalTasks, segID)
log.Ctx(fsw.ctx).Debug("flushedSegmentWatcher delete the internal task", zap.Int64("segID", segID))
}
func (fsw *flushedSegmentWatcher) getInternalTask(segID UniqueID) *internalTask {
fsw.internalTaskMutex.RLock()
defer fsw.internalTaskMutex.RUnlock()
return &internalTask{
state: fsw.internalTasks[segID].state,
segmentInfo: fsw.internalTasks[segID].segmentInfo,
}
}
func (fsw *flushedSegmentWatcher) setInternalTaskSegmentInfo(segID UniqueID, segInfo *datapb.SegmentInfo) {
fsw.internalTaskMutex.Lock()
defer fsw.internalTaskMutex.Unlock()
if _, ok := fsw.internalTasks[segID]; ok {
fsw.internalTasks[segID].segmentInfo = segInfo
}
}
func (fsw *flushedSegmentWatcher) internalProcess(segID UniqueID) {
t := fsw.getInternalTask(segID)
switch t.state {
case indexTaskPrepare:
if t.segmentInfo.GetIsFake() {
fsw.handoff.enqueue(t.segmentInfo)
fsw.updateInternalTaskState(segID, indexTaskDone)
fsw.internalNotifyFunc()
return
}
if err := fsw.prepare(segID); err != nil {
log.Ctx(fsw.ctx).Warn("flushedSegmentWatcher prepare internal task fail", zap.Int64("segID", segID), zap.Error(err))
return
}
fsw.updateInternalTaskState(segID, indexTaskInit)
case indexTaskInit:
if err := fsw.constructTask(t); err != nil {
log.Ctx(fsw.ctx).Warn("flushedSegmentWatcher construct task fail", zap.Int64("segID", segID), zap.Error(err))
return
}
fsw.updateInternalTaskState(segID, indexTaskInProgress)
fsw.internalNotifyFunc()
case indexTaskInProgress:
if fsw.handoff.taskDone(segID) {
fsw.updateInternalTaskState(segID, indexTaskDone)
fsw.internalNotifyFunc()
}
case indexTaskDone:
if err := fsw.removeFlushedSegment(t); err != nil {
return
}
fsw.deleteInternalTask(segID)
fsw.internalNotifyFunc()
default:
log.Info("IndexCoord flushedSegmentWatcher internal task get invalid state", zap.Int64("segID", segID),
zap.String("state", t.state.String()))
}
}
func (fsw *flushedSegmentWatcher) constructTask(t *internalTask) error {
// Make sure index is not being written.
fsw.ic.indexGCLock.Lock()
defer fsw.ic.indexGCLock.Unlock()
fieldIndexes := fsw.meta.GetIndexesForCollection(t.segmentInfo.CollectionID, "")
if len(fieldIndexes) == 0 {
log.Ctx(fsw.ctx).Debug("segment no need to build index", zap.Int64("segmentID", t.segmentInfo.ID),
zap.Int64("num of rows", t.segmentInfo.NumOfRows), zap.Int("collection indexes num", len(fieldIndexes)))
// no need to build index
return nil
}
resp, err := fsw.ic.rootCoordClient.AllocTimestamp(fsw.ctx, &rootcoordpb.AllocTimestampRequest{
Count: 1,
})
if err != nil {
return err
}
if resp.Status.GetErrorCode() != commonpb.ErrorCode_Success {
return errors.New(resp.Status.GetReason())
}
for _, index := range fieldIndexes {
segIdx := &model.SegmentIndex{
SegmentID: t.segmentInfo.ID,
CollectionID: t.segmentInfo.CollectionID,
PartitionID: t.segmentInfo.PartitionID,
NumRows: t.segmentInfo.NumOfRows,
IndexID: index.IndexID,
CreateTime: resp.Timestamp,
}
//create index task for metaTable
// send to indexBuilder
have, buildID, err := fsw.ic.createIndexForSegment(segIdx)
if err != nil {
return err
}
if !have {
fsw.builder.enqueue(buildID)
}
log.Ctx(fsw.ctx).Info("flushedSegmentWatcher construct task success", zap.Int64("segID", t.segmentInfo.ID),
zap.Int64("buildID", buildID), zap.Bool("already have index task", have))
}
fsw.handoff.enqueue(t.segmentInfo)
return nil
}
func (fsw *flushedSegmentWatcher) removeFlushedSegment(t *internalTask) error {
deletedKeys := fmt.Sprintf("%s/%d/%d/%d", util.FlushedSegmentPrefix, t.segmentInfo.CollectionID, t.segmentInfo.PartitionID, t.segmentInfo.ID)
err := fsw.kvClient.RemoveWithPrefix(deletedKeys)
if err != nil {
log.Ctx(fsw.ctx).Warn("IndexCoord remove flushed segment key fail", zap.Int64("collID", t.segmentInfo.CollectionID),
zap.Int64("partID", t.segmentInfo.PartitionID), zap.Int64("segID", t.segmentInfo.ID), zap.Error(err))
return err
}
log.Ctx(fsw.ctx).Info("IndexCoord remove flushed segment key success", zap.Int64("collID", t.segmentInfo.CollectionID),
zap.Int64("partID", t.segmentInfo.PartitionID), zap.Int64("segID", t.segmentInfo.ID))
return nil
}
func (fsw *flushedSegmentWatcher) prepare(segID UniqueID) error {
defer fsw.internalNotifyFunc()
t := fsw.getInternalTask(segID)
if t.segmentInfo != nil {
return nil
}
info, err := fsw.ic.pullSegmentInfo(fsw.ctx, segID)
if err != nil {
log.Warn("flushedSegmentWatcher get segment info fail", zap.Int64("segID", segID),
zap.Error(err))
if errors.Is(err, ErrSegmentNotFound) {
fsw.deleteInternalTask(segID)
return err
}
return err
}
fsw.setInternalTaskSegmentInfo(segID, info)
log.Ctx(fsw.ctx).Info("flushedSegmentWatcher prepare task success", zap.Int64("segID", segID))
return nil
}

View File

@ -1,675 +0,0 @@
// 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 indexcoord
import (
"context"
"errors"
"sync"
"testing"
"time"
"github.com/golang/protobuf/proto"
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/metastore/kv/indexcoord"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
)
var (
segBytes = marshalSegment()
)
func marshalSegment() string {
segment := &datapb.SegmentInfo{ID: segID}
bytes, err := proto.Marshal(segment)
if err != nil {
panic(err)
}
return string(bytes)
}
func Test_flushSegmentWatcher(t *testing.T) {
ctx := context.Background()
fsw, err := newFlushSegmentWatcher(ctx,
&mockETCDKV{
loadWithRevision: func(key string) ([]string, []string, int64, error) {
return []string{"1"}, []string{segBytes}, 1, nil
},
removeWithPrefix: func(key string) error {
return nil
},
},
&metaTable{
catalog: &indexcoord.Catalog{
Txn: NewMockEtcdKV(),
},
indexLock: sync.RWMutex{},
segmentIndexLock: sync.RWMutex{},
collectionIndexes: map[UniqueID]map[UniqueID]*model.Index{},
segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{},
buildID2SegmentIndex: map[UniqueID]*model.SegmentIndex{},
},
&indexBuilder{}, &handoff{}, &IndexCoord{
dataCoordClient: NewDataCoordMock(),
})
assert.NoError(t, err)
assert.NotNil(t, fsw)
fsw.enqueueInternalTask(&datapb.SegmentInfo{ID: 1})
fsw.Start()
// hold ticker.C
time.Sleep(time.Second * 2)
for fsw.Len() != 0 {
time.Sleep(time.Second)
}
fsw.Stop()
}
func Test_flushSegmentWatcher_newFlushSegmentWatcher(t *testing.T) {
t.Run("success", func(t *testing.T) {
fsw, err := newFlushSegmentWatcher(context.Background(),
&mockETCDKV{
loadWithRevision: func(key string) ([]string, []string, int64, error) {
return []string{"segID1"}, []string{segBytes}, 1, nil
},
}, &metaTable{}, &indexBuilder{}, &handoff{}, &IndexCoord{
dataCoordClient: NewDataCoordMock(),
})
assert.NoError(t, err)
assert.Equal(t, 1, len(fsw.internalTasks))
fsw, err = newFlushSegmentWatcher(context.Background(),
&mockETCDKV{
loadWithRevision: func(key string) ([]string, []string, int64, error) {
return []string{"segID1"}, []string{"10"}, 1, nil
},
}, &metaTable{}, &indexBuilder{}, &handoff{}, &IndexCoord{
dataCoordClient: NewDataCoordMock(),
})
assert.NoError(t, err)
assert.Equal(t, 1, len(fsw.internalTasks))
})
t.Run("load fail", func(t *testing.T) {
fsw, err := newFlushSegmentWatcher(context.Background(),
&mockETCDKV{
loadWithRevision: func(key string) ([]string, []string, int64, error) {
return []string{"segID1"}, []string{segBytes}, 1, errors.New("error")
},
}, &metaTable{}, &indexBuilder{}, &handoff{}, &IndexCoord{
dataCoordClient: NewDataCoordMock(),
})
assert.Error(t, err)
assert.Nil(t, fsw)
})
t.Run("parse fail", func(t *testing.T) {
fsw, err := newFlushSegmentWatcher(context.Background(),
&mockETCDKV{
loadWithRevision: func(key string) ([]string, []string, int64, error) {
return []string{"segID1"}, []string{"segID"}, 1, nil
},
}, &metaTable{}, &indexBuilder{}, &handoff{}, &IndexCoord{
dataCoordClient: NewDataCoordMock(),
})
assert.Error(t, err)
assert.Nil(t, fsw)
})
}
func Test_flushedSegmentWatcher_internalRun(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
fsw := &flushedSegmentWatcher{
ctx: ctx,
cancel: cancel,
kvClient: NewMockEtcdKV(),
wg: sync.WaitGroup{},
scheduleDuration: time.Second,
internalTaskMutex: sync.RWMutex{},
internalNotify: make(chan struct{}, 1),
etcdRevision: 0,
meta: nil,
builder: nil,
ic: &IndexCoord{
dataCoordClient: NewDataCoordMock(),
},
handoff: nil,
internalTasks: map[UniqueID]*internalTask{
segID: {
state: indexTaskPrepare,
segmentInfo: &datapb.SegmentInfo{
CollectionID: collID,
PartitionID: partID,
ID: segID,
},
},
segID + 1: {
state: indexTaskPrepare,
segmentInfo: nil,
},
segID - 1: {
state: indexTaskPrepare,
segmentInfo: nil,
},
},
}
fsw.internalRun()
assert.Equal(t, 3, fsw.Len())
}
func Test_flushSegmentWatcher_enqueueInternalTask(t *testing.T) {
fsw := &flushedSegmentWatcher{internalTasks: map[UniqueID]*internalTask{}}
segment := &datapb.SegmentInfo{ID: segID}
fakedSegment := &datapb.SegmentInfo{
ID: segID + 1,
CollectionID: collID,
PartitionID: partID,
IsFake: true,
}
fsw.enqueueInternalTask(segment)
fsw.enqueueInternalTask(fakedSegment)
assert.Equal(t, 2, len(fsw.internalTasks))
}
func Test_flushSegmentWatcher_internalProcess_success(t *testing.T) {
meta := &metaTable{
segmentIndexLock: sync.RWMutex{},
indexLock: sync.RWMutex{},
catalog: &indexcoord.Catalog{Txn: &mockETCDKV{
multiSave: func(m map[string]string) error {
return nil
},
}},
collectionIndexes: map[UniqueID]map[UniqueID]*model.Index{
collID: {
indexID: {
TenantID: "",
CollectionID: collID,
FieldID: fieldID,
IndexID: indexID,
IndexName: indexName,
IsDeleted: false,
CreateTime: 1,
TypeParams: nil,
IndexParams: nil,
},
},
},
segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{
segID: {
indexID: {
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
NumRows: 1000,
IndexID: indexID,
BuildID: buildID,
IndexState: commonpb.IndexState_Finished,
},
},
},
buildID2SegmentIndex: map[UniqueID]*model.SegmentIndex{
buildID: {
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
NumRows: 1000,
IndexID: indexID,
BuildID: buildID,
IndexState: commonpb.IndexState_Finished,
},
},
}
task := &internalTask{
state: indexTaskPrepare,
segmentInfo: nil,
}
task2 := &internalTask{
state: indexTaskPrepare,
segmentInfo: &datapb.SegmentInfo{
ID: segID + 1,
CollectionID: collID,
PartitionID: partID,
IsFake: true,
},
}
fsw := &flushedSegmentWatcher{
ctx: context.Background(),
handoff: &handoff{
segments: map[UniqueID]*datapb.SegmentInfo{},
taskMutex: sync.RWMutex{},
wg: sync.WaitGroup{},
meta: meta,
notifyChan: make(chan struct{}, 1),
scheduleDuration: time.Second,
kvClient: nil,
},
ic: &IndexCoord{
dataCoordClient: &DataCoordMock{
CallGetSegmentInfo: func(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) {
return &datapb.GetSegmentInfoResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
},
Infos: []*datapb.SegmentInfo{
{
ID: segID,
CollectionID: collID,
PartitionID: partID,
NumOfRows: 10000,
State: commonpb.SegmentState_Flushed,
CreatedByCompaction: true,
CompactionFrom: []int64{},
StartPosition: &internalpb.MsgPosition{
ChannelName: "",
MsgID: nil,
MsgGroup: "",
Timestamp: 1,
},
},
},
}, nil
},
},
rootCoordClient: NewRootCoordMock(),
metaTable: meta,
},
internalTasks: map[UniqueID]*internalTask{
segID: task,
segID + 1: task2,
},
meta: meta,
builder: &indexBuilder{
taskMutex: sync.RWMutex{},
scheduleDuration: 0,
tasks: map[int64]indexTaskState{},
notifyChan: nil,
meta: meta,
},
kvClient: &mockETCDKV{
multiSave: func(m map[string]string) error {
return nil
},
save: func(s string, s2 string) error {
return nil
},
removeWithPrefix: func(key string) error {
return nil
},
},
}
t.Run("prepare", func(t *testing.T) {
fsw.internalProcess(segID)
fsw.internalTaskMutex.RLock()
assert.Equal(t, indexTaskInit, fsw.internalTasks[segID].state)
fsw.internalTaskMutex.RUnlock()
})
t.Run("prepare for fake segment", func(t *testing.T) {
fsw.internalProcess(segID + 1)
fsw.internalTaskMutex.RLock()
assert.Equal(t, indexTaskDone, fsw.internalTasks[segID+1].state)
fsw.internalTaskMutex.RUnlock()
})
t.Run("init", func(t *testing.T) {
fsw.internalProcess(segID)
fsw.internalTaskMutex.RLock()
assert.Equal(t, indexTaskInProgress, fsw.internalTasks[segID].state)
fsw.internalTaskMutex.RUnlock()
})
fsw.handoff.deleteTask(segID)
t.Run("inProgress", func(t *testing.T) {
fsw.internalProcess(segID)
fsw.internalTaskMutex.RLock()
assert.Equal(t, indexTaskDone, fsw.internalTasks[segID].state)
fsw.internalTaskMutex.RUnlock()
})
t.Run("done", func(t *testing.T) {
fsw.internalProcess(segID)
fsw.internalTaskMutex.RLock()
_, ok := fsw.internalTasks[segID]
assert.False(t, ok)
fsw.internalTaskMutex.RUnlock()
})
}
func Test_flushSegmentWatcher_internalProcess_error(t *testing.T) {
task := &internalTask{
state: indexTaskPrepare,
segmentInfo: nil,
}
fsw := &flushedSegmentWatcher{
ctx: context.Background(),
ic: &IndexCoord{
loopCtx: context.Background(),
dataCoordClient: &DataCoordMock{
CallGetSegmentInfo: func(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) {
return &datapb.GetSegmentInfoResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
},
Infos: []*datapb.SegmentInfo{
{
ID: segID + 100,
CollectionID: collID,
PartitionID: partID,
NumOfRows: 10000,
State: commonpb.SegmentState_Flushed,
CreatedByCompaction: true,
CompactionFrom: []int64{segID},
},
},
}, nil
},
},
rootCoordClient: NewRootCoordMock(),
metaTable: &metaTable{},
},
internalTasks: map[UniqueID]*internalTask{
segID: task,
},
meta: &metaTable{},
builder: &indexBuilder{},
}
t.Run("fail", func(t *testing.T) {
fsw.ic.dataCoordClient = &DataCoordMock{
CallGetSegmentInfo: func(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) {
return nil, errors.New("error")
},
}
fsw.internalProcess(segID)
fsw.ic.dataCoordClient = &DataCoordMock{
CallGetSegmentInfo: func(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) {
return &datapb.GetSegmentInfoResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "fail reason",
},
}, nil
},
}
fsw.internalProcess(segID)
fsw.internalTaskMutex.RLock()
assert.Equal(t, indexTaskPrepare, fsw.internalTasks[segID].state)
fsw.internalTaskMutex.RUnlock()
})
t.Run("remove flushed segment fail", func(t *testing.T) {
fsw.kvClient = &mockETCDKV{
removeWithPrefix: func(key string) error {
return errors.New("error")
},
}
fsw.internalTasks = map[UniqueID]*internalTask{
segID: {
state: indexTaskDone,
segmentInfo: &datapb.SegmentInfo{
ID: segID,
CollectionID: collID,
PartitionID: partID,
InsertChannel: "",
NumOfRows: 0,
State: 0,
MaxRowNum: 0,
LastExpireTime: 0,
StartPosition: nil,
DmlPosition: nil,
Binlogs: nil,
Statslogs: nil,
Deltalogs: nil,
CreatedByCompaction: false,
CompactionFrom: nil,
DroppedAt: 0,
},
},
}
fsw.internalProcess(segID)
fsw.internalTaskMutex.RLock()
assert.Equal(t, indexTaskDone, fsw.internalTasks[segID].state)
fsw.internalTaskMutex.RUnlock()
})
t.Run("invalid state", func(t *testing.T) {
fsw.internalTasks = map[UniqueID]*internalTask{
segID: {
state: indexTaskDeleted,
segmentInfo: nil,
},
}
fsw.internalProcess(segID)
})
}
func Test_flushSegmentWatcher_prepare_error(t *testing.T) {
t.Run("segmentInfo already exist", func(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
fsw := &flushedSegmentWatcher{
ctx: ctx,
cancel: cancel,
kvClient: NewMockEtcdKV(),
wg: sync.WaitGroup{},
scheduleDuration: time.Second,
internalTaskMutex: sync.RWMutex{},
internalNotify: make(chan struct{}, 1),
etcdRevision: 0,
meta: nil,
builder: nil,
ic: &IndexCoord{
loopCtx: context.Background(),
dataCoordClient: NewDataCoordMock(),
},
handoff: nil,
internalTasks: map[UniqueID]*internalTask{
segID: {
state: indexTaskPrepare,
segmentInfo: &datapb.SegmentInfo{
CollectionID: collID,
PartitionID: partID,
ID: segID,
},
},
},
}
err := fsw.prepare(segID)
assert.NoError(t, err)
})
t.Run("segment is not exist", func(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
fsw := &flushedSegmentWatcher{
ctx: ctx,
cancel: cancel,
kvClient: NewMockEtcdKV(),
wg: sync.WaitGroup{},
scheduleDuration: time.Second,
internalTaskMutex: sync.RWMutex{},
internalNotify: make(chan struct{}, 1),
etcdRevision: 0,
meta: nil,
builder: nil,
ic: &IndexCoord{
loopCtx: context.Background(),
dataCoordClient: &DataCoordMock{
CallGetSegmentInfo: func(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) {
return &datapb.GetSegmentInfoResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
Infos: nil,
}, nil
},
},
},
handoff: nil,
internalTasks: map[UniqueID]*internalTask{
segID: {
state: indexTaskPrepare,
segmentInfo: nil,
},
},
}
err := fsw.prepare(segID)
assert.ErrorIs(t, err, ErrSegmentNotFound)
})
}
func Test_flushSegmentWatcher_removeFlushedSegment(t *testing.T) {
task := &internalTask{
state: indexTaskDone,
segmentInfo: &datapb.SegmentInfo{
ID: segID,
CollectionID: collID,
PartitionID: partID,
},
}
t.Run("success", func(t *testing.T) {
fsw := &flushedSegmentWatcher{
ctx: context.Background(),
kvClient: &mockETCDKV{
removeWithPrefix: func(key string) error {
return nil
},
},
}
err := fsw.removeFlushedSegment(task)
assert.NoError(t, err)
})
t.Run("fail", func(t *testing.T) {
fsw := &flushedSegmentWatcher{
kvClient: &mockETCDKV{
removeWithPrefix: func(key string) error {
return errors.New("error")
},
},
}
err := fsw.removeFlushedSegment(task)
assert.Error(t, err)
})
}
func Test_flushSegmentWatcher_constructTask_error(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
meta := &metaTable{
segmentIndexLock: sync.RWMutex{},
indexLock: sync.RWMutex{},
catalog: &indexcoord.Catalog{Txn: &mockETCDKV{
multiSave: func(m map[string]string) error {
return nil
},
}},
collectionIndexes: map[UniqueID]map[UniqueID]*model.Index{
collID: {
indexID: {
TenantID: "",
CollectionID: collID,
FieldID: fieldID,
IndexID: indexID,
IndexName: indexName,
IsDeleted: false,
CreateTime: 1,
TypeParams: nil,
IndexParams: nil,
},
},
},
segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{},
buildID2SegmentIndex: map[UniqueID]*model.SegmentIndex{},
}
task := &internalTask{
state: indexTaskInit,
segmentInfo: &datapb.SegmentInfo{
ID: segID,
CollectionID: collID,
PartitionID: partID,
},
}
fsw := &flushedSegmentWatcher{
ctx: ctx,
cancel: cancel,
kvClient: nil,
wg: sync.WaitGroup{},
scheduleDuration: 100 * time.Millisecond,
internalTaskMutex: sync.RWMutex{},
internalNotify: make(chan struct{}, 1),
etcdRevision: 0,
meta: meta,
builder: nil,
ic: &IndexCoord{
rootCoordClient: NewRootCoordMock(),
},
handoff: nil,
internalTasks: map[UniqueID]*internalTask{
segID: task,
},
}
t.Run("alloc timestamp error", func(t *testing.T) {
fsw.ic.rootCoordClient = &RootCoordMock{
CallAllocTimestamp: func(ctx context.Context, req *rootcoordpb.AllocTimestampRequest) (*rootcoordpb.AllocTimestampResponse, error) {
return nil, errors.New("error")
},
}
err := fsw.constructTask(task)
assert.Error(t, err)
})
t.Run("alloc timestamp not success", func(t *testing.T) {
fsw.ic.rootCoordClient = &RootCoordMock{
CallAllocTimestamp: func(ctx context.Context, req *rootcoordpb.AllocTimestampRequest) (*rootcoordpb.AllocTimestampResponse, error) {
return &rootcoordpb.AllocTimestampResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "fail reason",
},
}, nil
},
}
err := fsw.constructTask(task)
assert.Error(t, err)
})
}

View File

@ -1,288 +0,0 @@
// 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 indexcoord
import (
"context"
"path"
"sync"
"time"
"github.com/milvus-io/milvus/internal/util/metautil"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/storage"
)
type garbageCollector struct {
ctx context.Context
cancel context.CancelFunc
wg sync.WaitGroup
gcFileDuration time.Duration
gcMetaDuration time.Duration
metaTable *metaTable
chunkManager storage.ChunkManager
indexCoordClient *IndexCoord
}
func newGarbageCollector(ctx context.Context, meta *metaTable, chunkManager storage.ChunkManager, ic *IndexCoord) *garbageCollector {
ctx, cancel := context.WithCancel(ctx)
return &garbageCollector{
ctx: ctx,
cancel: cancel,
gcFileDuration: Params.IndexCoordCfg.GCInterval.GetAsDuration(time.Second),
gcMetaDuration: time.Minute,
metaTable: meta,
chunkManager: chunkManager,
indexCoordClient: ic,
}
}
func (gc *garbageCollector) Start() {
//gc.wg.Add(1)
//go gc.recycleUnusedIndexes()
//
//gc.wg.Add(1)
//go gc.recycleUnusedSegIndexes()
//
//gc.wg.Add(1)
//go gc.recycleUnusedIndexFiles()
}
func (gc *garbageCollector) Stop() {
gc.cancel()
gc.wg.Wait()
}
func (gc *garbageCollector) recycleUnusedIndexes() {
defer gc.wg.Done()
log.Ctx(gc.ctx).Info("IndexCoord garbageCollector recycleUnusedIndexes start")
ticker := time.NewTicker(gc.gcMetaDuration)
defer ticker.Stop()
for {
select {
case <-gc.ctx.Done():
log.Ctx(gc.ctx).Info("IndexCoord garbageCollector recycleUnusedMetaLoop context has done")
return
case <-ticker.C:
deletedIndexes := gc.metaTable.GetDeletedIndexes()
for _, index := range deletedIndexes {
buildIDs := gc.metaTable.GetBuildIDsFromIndexID(index.IndexID)
if len(buildIDs) == 0 {
if err := gc.metaTable.RemoveIndex(index.CollectionID, index.IndexID); err != nil {
log.Ctx(gc.ctx).Warn("IndexCoord remove index on collection fail", zap.Int64("collID", index.CollectionID),
zap.Int64("indexID", index.IndexID), zap.Error(err))
continue
}
} else {
for _, buildID := range buildIDs {
segIdx, ok := gc.metaTable.GetMeta(buildID)
if !ok {
log.Ctx(gc.ctx).Debug("IndexCoord get segment index is not exist", zap.Int64("buildID", buildID))
continue
}
if segIdx.NodeID != 0 {
// wait for releasing reference lock
continue
}
if err := gc.metaTable.RemoveSegmentIndex(segIdx.CollectionID, segIdx.PartitionID, segIdx.SegmentID, segIdx.BuildID); err != nil {
log.Ctx(gc.ctx).Warn("delete index meta from etcd failed, wait to retry", zap.Int64("buildID", segIdx.BuildID),
zap.Int64("nodeID", segIdx.NodeID), zap.Error(err))
continue
}
log.Ctx(gc.ctx).Info("IndexCoord remove segment index meta success", zap.Int64("buildID", segIdx.BuildID),
zap.Int64("nodeID", segIdx.NodeID))
}
log.Ctx(gc.ctx).Info("garbageCollector remove index success", zap.Int64("collID", index.CollectionID),
zap.Int64("indexID", index.IndexID))
}
}
}
}
}
func (gc *garbageCollector) recycleSegIndexesMeta() {
gc.indexCoordClient.indexGCLock.Lock()
segIndexes := gc.metaTable.GetAllSegIndexes()
gc.indexCoordClient.indexGCLock.Unlock()
collID2segID := make(map[int64]map[int64]struct{})
for segID, segIdx := range segIndexes {
if _, ok := collID2segID[segIdx.CollectionID]; !ok {
collID2segID[segIdx.CollectionID] = make(map[int64]struct{})
}
collID2segID[segIdx.CollectionID][segID] = struct{}{}
}
for collID, segIDs := range collID2segID {
resp, err := gc.indexCoordClient.dataCoordClient.GetFlushedSegments(gc.ctx, &datapb.GetFlushedSegmentsRequest{
CollectionID: collID,
PartitionID: -1,
IncludeUnhealthy: true,
})
if err != nil {
log.Ctx(gc.ctx).Warn("IndexCoord garbageCollector get flushed segments from DataCoord fail",
zap.Int64("collID", collID), zap.Error(err))
return
}
if resp.Status.ErrorCode != commonpb.ErrorCode_Success {
log.Ctx(gc.ctx).Warn("IndexCoord garbageCollector get flushed segments from DataCoord fail", zap.Int64("collID", collID),
zap.String("fail reason", resp.Status.Reason))
return
}
flushedSegments := make(map[int64]struct{})
for _, segID := range resp.Segments {
flushedSegments[segID] = struct{}{}
}
for segID := range segIDs {
if segIndexes[segID].IsDeleted {
continue
}
if _, ok := flushedSegments[segID]; !ok {
log.Ctx(gc.ctx).Info("segment is already not exist, mark it deleted", zap.Int64("collID", collID),
zap.Int64("segID", segID))
if err := gc.metaTable.MarkSegmentsIndexAsDeleted(func(segIndex *model.SegmentIndex) bool {
return segIndex.SegmentID == segID
}); err != nil {
continue
}
}
}
}
//segIndexes := gc.metaTable.GetDeletedSegmentIndexes()
for _, meta := range segIndexes {
if meta.IsDeleted || gc.metaTable.IsIndexDeleted(meta.CollectionID, meta.IndexID) {
if meta.NodeID != 0 {
// wait for releasing reference lock
continue
}
if err := gc.metaTable.RemoveSegmentIndex(meta.CollectionID, meta.PartitionID, meta.SegmentID, meta.BuildID); err != nil {
log.Ctx(gc.ctx).Warn("delete index meta from etcd failed, wait to retry", zap.Int64("buildID", meta.BuildID),
zap.Int64("nodeID", meta.NodeID), zap.Error(err))
continue
}
log.Ctx(gc.ctx).Info("index meta recycle success", zap.Int64("buildID", meta.BuildID),
zap.Int64("segID", meta.SegmentID))
}
}
}
func (gc *garbageCollector) recycleUnusedSegIndexes() {
defer gc.wg.Done()
log.Ctx(gc.ctx).Info("IndexCoord garbageCollector recycleUnusedSegIndexes start")
ticker := time.NewTicker(gc.gcMetaDuration)
defer ticker.Stop()
for {
select {
case <-gc.ctx.Done():
log.Ctx(gc.ctx).Info("IndexCoord garbageCollector recycleUnusedMetaLoop context has done")
return
case <-ticker.C:
gc.recycleSegIndexesMeta()
}
}
}
// recycleUnusedIndexFiles is used to delete those index files that no longer exist in the meta.
func (gc *garbageCollector) recycleUnusedIndexFiles() {
defer gc.wg.Done()
log.Ctx(gc.ctx).Info("IndexCoord garbageCollector start recycleUnusedIndexFiles loop")
ticker := time.NewTicker(gc.gcFileDuration)
defer ticker.Stop()
for {
select {
case <-gc.ctx.Done():
return
case <-ticker.C:
prefix := path.Join(gc.chunkManager.RootPath(), common.SegmentIndexPath) + "/"
// list dir first
keys, _, err := gc.chunkManager.ListWithPrefix(gc.ctx, prefix, false)
if err != nil {
log.Ctx(gc.ctx).Error("IndexCoord garbageCollector recycleUnusedIndexFiles list keys from chunk manager failed", zap.Error(err))
continue
}
for _, key := range keys {
log.Ctx(gc.ctx).Debug("indexFiles keys", zap.String("key", key))
buildID, err := parseBuildIDFromFilePath(key)
if err != nil {
log.Ctx(gc.ctx).Error("IndexCoord garbageCollector recycleUnusedIndexFiles parseIndexFileKey", zap.String("key", key), zap.Error(err))
continue
}
log.Ctx(gc.ctx).Info("IndexCoord garbageCollector will recycle index files", zap.Int64("buildID", buildID))
if !gc.metaTable.HasBuildID(buildID) {
// buildID no longer exists in meta, remove all index files
log.Ctx(gc.ctx).Info("IndexCoord garbageCollector recycleUnusedIndexFiles find meta has not exist, remove index files",
zap.Int64("buildID", buildID))
err = gc.chunkManager.RemoveWithPrefix(gc.ctx, key)
if err != nil {
log.Ctx(gc.ctx).Warn("IndexCoord garbageCollector recycleUnusedIndexFiles remove index files failed",
zap.Int64("buildID", buildID), zap.String("prefix", key), zap.Error(err))
continue
}
continue
}
log.Ctx(gc.ctx).Info("index meta can be recycled, recycle index files", zap.Int64("buildID", buildID))
canRecycle, segIdx := gc.metaTable.GetSegmentIndexByBuildID(buildID)
if !canRecycle {
// Even if the index is marked as deleted, the index file will not be recycled, wait for the next gc,
// and delete all index files about the buildID at one time.
log.Ctx(gc.ctx).Warn("IndexCoord garbageCollector can not recycle index files", zap.Int64("buildID", buildID))
continue
}
filesMap := make(map[string]struct{})
for _, fileID := range segIdx.IndexFileKeys {
filepath := metautil.BuildSegmentIndexFilePath(gc.chunkManager.RootPath(), segIdx.BuildID, segIdx.IndexVersion,
segIdx.PartitionID, segIdx.SegmentID, fileID)
filesMap[filepath] = struct{}{}
}
files, _, err := gc.chunkManager.ListWithPrefix(gc.ctx, key, true)
if err != nil {
log.Ctx(gc.ctx).Warn("IndexCoord garbageCollector recycleUnusedIndexFiles list files failed",
zap.Int64("buildID", buildID), zap.String("prefix", key), zap.Error(err))
continue
}
log.Ctx(gc.ctx).Info("recycle index files", zap.Int64("buildID", buildID), zap.Int("meta files num", len(filesMap)),
zap.Int("chunkManager files num", len(files)))
deletedFilesNum := 0
for _, file := range files {
if _, ok := filesMap[file]; !ok {
if err = gc.chunkManager.Remove(gc.ctx, file); err != nil {
log.Ctx(gc.ctx).Warn("IndexCoord garbageCollector recycleUnusedIndexFiles remove file failed",
zap.Int64("buildID", buildID), zap.String("file", file), zap.Error(err))
continue
}
deletedFilesNum++
}
}
log.Ctx(gc.ctx).Info("index files recycle success", zap.Int64("buildID", buildID),
zap.Int("delete index files num", deletedFilesNum))
}
}
}
}

View File

@ -1,748 +0,0 @@
// 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 indexcoord
import (
"sync"
"github.com/milvus-io/milvus/internal/metastore"
"github.com/milvus-io/milvus/internal/metastore/model"
)
func createGarbageCollectorMetaTable(catalog metastore.IndexCoordCatalog) *metaTable {
return &metaTable{
catalog: catalog,
indexLock: sync.RWMutex{},
segmentIndexLock: sync.RWMutex{},
collectionIndexes: map[UniqueID]map[UniqueID]*model.Index{
collID: {
indexID: {
TenantID: "",
CollectionID: collID,
FieldID: fieldID,
IndexID: indexID,
IndexName: indexName,
IsDeleted: true,
CreateTime: 1,
TypeParams: nil,
IndexParams: nil,
},
indexID + 1: {
TenantID: "",
CollectionID: collID,
FieldID: fieldID + 1,
IndexID: indexID + 1,
IndexName: "indexName2",
IsDeleted: true,
CreateTime: 0,
TypeParams: nil,
IndexParams: nil,
},
indexID + 2: {
TenantID: "",
CollectionID: collID,
FieldID: fieldID + 2,
IndexID: indexID + 2,
IndexName: "indexName3",
IsDeleted: false,
CreateTime: 0,
TypeParams: nil,
IndexParams: nil,
},
},
},
segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{
segID: {
indexID: {
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
NumRows: 1025,
IndexID: indexID,
BuildID: buildID,
NodeID: 0,
IndexVersion: 1,
IndexState: 3,
FailReason: "",
IsDeleted: false,
CreateTime: 1,
IndexFileKeys: nil,
IndexSize: 100,
WriteHandoff: false,
},
},
segID + 1: {
indexID: {
SegmentID: segID + 1,
CollectionID: collID,
PartitionID: partID,
NumRows: 1025,
IndexID: indexID,
BuildID: buildID + 1,
NodeID: 0,
IndexVersion: 1,
IndexState: 2,
FailReason: "",
IsDeleted: false,
CreateTime: 1,
IndexFileKeys: nil,
IndexSize: 100,
WriteHandoff: false,
},
},
segID + 2: {
indexID + 2: {
SegmentID: segID + 2,
CollectionID: collID,
PartitionID: partID,
NumRows: 10000,
IndexID: indexID + 2,
BuildID: buildID + 2,
NodeID: 0,
IndexVersion: 1,
IndexState: 1,
FailReason: "",
IsDeleted: true,
CreateTime: 1,
IndexFileKeys: nil,
IndexSize: 0,
WriteHandoff: false,
},
},
segID + 3: {
indexID + 2: {
SegmentID: segID + 3,
CollectionID: collID,
PartitionID: partID,
NumRows: 10000,
IndexID: indexID + 2,
BuildID: buildID + 3,
NodeID: 0,
IndexVersion: 1,
IndexState: 1,
FailReason: "",
IsDeleted: false,
CreateTime: 1,
IndexFileKeys: []string{"file1", "file2"},
IndexSize: 0,
WriteHandoff: false,
},
},
segID + 4: {
indexID + 2: {
SegmentID: segID + 4,
CollectionID: collID,
PartitionID: partID,
NumRows: 10000,
IndexID: indexID + 2,
BuildID: buildID + 4,
NodeID: 0,
IndexVersion: 1,
IndexState: 2,
FailReason: "",
IsDeleted: false,
CreateTime: 1,
IndexFileKeys: []string{},
IndexSize: 0,
WriteHandoff: false,
},
},
},
buildID2SegmentIndex: map[UniqueID]*model.SegmentIndex{
buildID: {
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
NumRows: 1025,
IndexID: indexID,
BuildID: buildID,
NodeID: 0,
IndexVersion: 1,
IndexState: 3,
FailReason: "",
IsDeleted: false,
CreateTime: 1,
IndexFileKeys: nil,
IndexSize: 100,
WriteHandoff: false,
},
buildID + 1: {
SegmentID: segID + 1,
CollectionID: collID,
PartitionID: partID,
NumRows: 1025,
IndexID: indexID,
BuildID: buildID + 1,
NodeID: 0,
IndexVersion: 1,
IndexState: 2,
FailReason: "",
IsDeleted: false,
CreateTime: 1,
IndexFileKeys: nil,
IndexSize: 100,
WriteHandoff: false,
},
buildID + 2: {
SegmentID: segID + 2,
CollectionID: collID,
PartitionID: partID,
NumRows: 10000,
IndexID: indexID + 2,
BuildID: buildID + 2,
NodeID: 0,
IndexVersion: 1,
IndexState: 1,
FailReason: "",
IsDeleted: true,
CreateTime: 1,
IndexFileKeys: nil,
IndexSize: 0,
WriteHandoff: false,
},
buildID + 3: {
SegmentID: segID + 3,
CollectionID: collID,
PartitionID: partID,
NumRows: 10000,
IndexID: indexID + 2,
BuildID: buildID + 3,
NodeID: 0,
IndexVersion: 1,
IndexState: 3,
FailReason: "",
IsDeleted: false,
CreateTime: 1,
IndexFileKeys: []string{"file1", "file2"},
IndexSize: 0,
WriteHandoff: false,
},
buildID + 4: {
SegmentID: segID + 4,
CollectionID: collID,
PartitionID: partID,
NumRows: 10000,
IndexID: indexID + 2,
BuildID: buildID + 4,
NodeID: 0,
IndexVersion: 1,
IndexState: 2,
FailReason: "",
IsDeleted: false,
CreateTime: 1,
IndexFileKeys: []string{},
IndexSize: 0,
WriteHandoff: false,
},
},
}
}
//func TestGarbageCollector(t *testing.T) {
// meta := createGarbageCollectorMetaTable(&indexcoord.Catalog{Txn: NewMockEtcdKV()})
// gc := newGarbageCollector(context.Background(), meta, &chunkManagerMock{
// removeWithPrefix: func(s string) error {
// return nil
// },
// listWithPrefix: func(s string, recursive bool) ([]string, []time.Time, error) {
// return []string{strconv.FormatInt(buildID, 10), strconv.FormatInt(buildID+1, 10),
// strconv.FormatInt(buildID+3, 10), strconv.FormatInt(buildID+4, 10)}, []time.Time{}, nil
// },
// remove: func(s string) error {
// return nil
// },
// }, &IndexCoord{
// dataCoordClient: &DataCoordMock{
// CallGetFlushedSegment: func(ctx context.Context, req *datapb.GetFlushedSegmentsRequest) (*datapb.GetFlushedSegmentsResponse, error) {
// return &datapb.GetFlushedSegmentsResponse{
// Status: &commonpb.Status{
// ErrorCode: commonpb.ErrorCode_Success,
// },
// Segments: []int64{segID, segID + 1, segID + 2, segID + 3, segID + 4},
// }, nil
// },
// },
// })
//
// gc.gcMetaDuration = time.Millisecond * 300
// gc.gcFileDuration = time.Millisecond * 300
//
// gc.Start()
// time.Sleep(time.Second * 2)
// err := gc.metaTable.MarkSegmentsIndexAsDeletedByBuildID([]UniqueID{buildID + 3, buildID + 4})
// assert.NoError(t, err)
// segIndexes := gc.metaTable.GetAllSegIndexes()
// for len(segIndexes) != 0 {
// time.Sleep(time.Second)
// segIndexes = gc.metaTable.GetAllSegIndexes()
// }
// gc.Stop()
//}
//
//func TestGarbageCollector_error(t *testing.T) {
// meta := createGarbageCollectorMetaTable(&indexcoord.Catalog{
// Txn: &mockETCDKV{
// multiSave: func(m map[string]string) error {
// return errors.New("error")
// },
// remove: func(s string) error {
// return errors.New("error")
// },
// },
// })
// gc := newGarbageCollector(context.Background(), meta, &chunkManagerMock{
// removeWithPrefix: func(s string) error {
// return errors.New("error")
// },
// listWithPrefix: func(s string, recursive bool) ([]string, []time.Time, error) {
// return nil, nil, errors.New("error")
// },
// remove: func(s string) error {
// return errors.New("error")
// },
// }, &IndexCoord{
// dataCoordClient: &DataCoordMock{
// CallGetFlushedSegment: func(ctx context.Context, req *datapb.GetFlushedSegmentsRequest) (*datapb.GetFlushedSegmentsResponse, error) {
// return &datapb.GetFlushedSegmentsResponse{
// Status: &commonpb.Status{
// ErrorCode: commonpb.ErrorCode_Success,
// },
// Segments: []int64{segID, segID + 1, segID + 2, segID + 3, segID + 4},
// }, nil
// },
// },
// })
//
// gc.gcMetaDuration = time.Millisecond * 300
// gc.gcFileDuration = time.Millisecond * 300
//
// gc.Start()
// time.Sleep(time.Second * 3)
// gc.Stop()
//}
//
//func TestGarbageCollectorGetFlushedSegment_error(t *testing.T) {
// t.Run("error", func(t *testing.T) {
// meta := createGarbageCollectorMetaTable(&indexcoord.Catalog{
// Txn: &mockETCDKV{
// multiSave: func(m map[string]string) error {
// return errors.New("error")
// },
// remove: func(s string) error {
// return errors.New("error")
// },
// },
// })
// gc := newGarbageCollector(context.Background(), meta, &chunkManagerMock{
// removeWithPrefix: func(s string) error {
// return errors.New("error")
// },
// listWithPrefix: func(s string, recursive bool) ([]string, []time.Time, error) {
// return nil, nil, errors.New("error")
// },
// remove: func(s string) error {
// return errors.New("error")
// },
// }, &IndexCoord{
// dataCoordClient: &DataCoordMock{
// CallGetFlushedSegment: func(ctx context.Context, req *datapb.GetFlushedSegmentsRequest) (*datapb.GetFlushedSegmentsResponse, error) {
// return &datapb.GetFlushedSegmentsResponse{
// Status: &commonpb.Status{
// ErrorCode: commonpb.ErrorCode_Success,
// },
// Segments: []int64{},
// }, errors.New("error")
// },
// },
// })
//
// gc.recycleSegIndexesMeta()
// })
//
// t.Run("fail", func(t *testing.T) {
// meta := createGarbageCollectorMetaTable(&indexcoord.Catalog{
// Txn: &mockETCDKV{
// multiSave: func(m map[string]string) error {
// return errors.New("error")
// },
// remove: func(s string) error {
// return errors.New("error")
// },
// },
// })
// gc := newGarbageCollector(context.Background(), meta, &chunkManagerMock{
// removeWithPrefix: func(s string) error {
// return errors.New("error")
// },
// listWithPrefix: func(s string, recursive bool) ([]string, []time.Time, error) {
// return nil, nil, errors.New("error")
// },
// remove: func(s string) error {
// return errors.New("error")
// },
// }, &IndexCoord{
// dataCoordClient: &DataCoordMock{
// CallGetFlushedSegment: func(ctx context.Context, req *datapb.GetFlushedSegmentsRequest) (*datapb.GetFlushedSegmentsResponse, error) {
// return &datapb.GetFlushedSegmentsResponse{
// Status: &commonpb.Status{
// ErrorCode: commonpb.ErrorCode_UnexpectedError,
// Reason: "fail reason",
// },
// Segments: []int64{},
// }, nil
// },
// },
// })
//
// gc.recycleSegIndexesMeta()
// })
//
//}
//
//func TestGarbageCollector_recycleUnusedIndexFiles(t *testing.T) {
// t.Run("index not in meta and remove with prefix failed", func(t *testing.T) {
// ctx, cancel := context.WithCancel(context.Background())
// gc := &garbageCollector{
// ctx: ctx,
// cancel: cancel,
// wg: sync.WaitGroup{},
// gcFileDuration: time.Millisecond * 300,
// gcMetaDuration: time.Millisecond * 300,
// metaTable: constructMetaTable(&indexcoord.Catalog{
// Txn: &mockETCDKV{
// remove: func(s string) error {
// return fmt.Errorf("error")
// },
// },
// }),
// chunkManager: &chunkManagerMock{
// removeWithPrefix: func(s string) error {
// return fmt.Errorf("error")
// },
// listWithPrefix: func(s string, recursive bool) ([]string, []time.Time, error) {
// if !recursive {
// return []string{"a/b/1/", "a/b/2/"}, []time.Time{{}, {}}, nil
// }
// return []string{"a/b/1/c", "a/b/2/d"}, []time.Time{{}, {}}, nil
// },
// remove: func(s string) error {
// return nil
// },
// },
// }
//
// gc.wg.Add(1)
// go gc.recycleUnusedIndexFiles()
// time.Sleep(time.Second)
// cancel()
// gc.wg.Wait()
// })
//
// t.Run("load dir failed", func(t *testing.T) {
// ctx, cancel := context.WithCancel(context.Background())
// gc := &garbageCollector{
// ctx: ctx,
// cancel: cancel,
// wg: sync.WaitGroup{},
// gcFileDuration: time.Millisecond * 300,
// gcMetaDuration: time.Millisecond * 300,
// metaTable: constructMetaTable(&indexcoord.Catalog{
// Txn: &mockETCDKV{
// remove: func(s string) error {
// return fmt.Errorf("error")
// },
// },
// }),
// chunkManager: &chunkManagerMock{
// removeWithPrefix: func(s string) error {
// return nil
// },
// listWithPrefix: func(s string, recursive bool) ([]string, []time.Time, error) {
// if !recursive {
// return nil, nil, fmt.Errorf("error")
// }
// return []string{"a/b/1/c", "a/b/2/d"}, []time.Time{{}, {}}, nil
// },
// remove: func(s string) error {
// return nil
// },
// },
// }
//
// gc.wg.Add(1)
// go gc.recycleUnusedIndexFiles()
// time.Sleep(time.Second)
// cancel()
// gc.wg.Wait()
// })
//
// t.Run("parse failed", func(t *testing.T) {
// ctx, cancel := context.WithCancel(context.Background())
// gc := &garbageCollector{
// ctx: ctx,
// cancel: cancel,
// wg: sync.WaitGroup{},
// gcFileDuration: time.Millisecond * 300,
// gcMetaDuration: time.Millisecond * 300,
// metaTable: constructMetaTable(&indexcoord.Catalog{
// Txn: &mockETCDKV{
// remove: func(s string) error {
// return fmt.Errorf("error")
// },
// },
// }),
// chunkManager: &chunkManagerMock{
// removeWithPrefix: func(s string) error {
// return nil
// },
// listWithPrefix: func(s string, recursive bool) ([]string, []time.Time, error) {
// if !recursive {
// return []string{"a/b/c/"}, []time.Time{{}}, nil
// }
// return []string{"a/b/1/c", "a/b/2/d"}, []time.Time{{}, {}}, nil
// },
// remove: func(s string) error {
// return nil
// },
// },
// }
//
// gc.wg.Add(1)
// go gc.recycleUnusedIndexFiles()
// time.Sleep(time.Second)
// cancel()
// gc.wg.Wait()
// })
//
// t.Run("ListWithPrefix failed", func(t *testing.T) {
// ctx, cancel := context.WithCancel(context.Background())
// gc := &garbageCollector{
// ctx: ctx,
// cancel: cancel,
// wg: sync.WaitGroup{},
// gcFileDuration: time.Millisecond * 300,
// gcMetaDuration: time.Millisecond * 300,
// metaTable: constructMetaTable(&indexcoord.Catalog{
// Txn: &mockETCDKV{
// remove: func(s string) error {
// return fmt.Errorf("error")
// },
// },
// }),
// chunkManager: &chunkManagerMock{
// removeWithPrefix: func(s string) error {
// return nil
// },
// listWithPrefix: func(s string, recursive bool) ([]string, []time.Time, error) {
// if !recursive {
// return []string{"a/b/1/"}, []time.Time{{}}, nil
// }
// return nil, nil, fmt.Errorf("error")
// },
// remove: func(s string) error {
// return nil
// },
// },
// }
//
// gc.wg.Add(1)
// go gc.recycleUnusedIndexFiles()
// time.Sleep(time.Second)
// cancel()
// gc.wg.Wait()
// })
//
// t.Run("remove failed", func(t *testing.T) {
// ctx, cancel := context.WithCancel(context.Background())
// gc := &garbageCollector{
// ctx: ctx,
// cancel: cancel,
// wg: sync.WaitGroup{},
// gcFileDuration: time.Millisecond * 300,
// gcMetaDuration: time.Millisecond * 300,
// metaTable: constructMetaTable(&indexcoord.Catalog{
// Txn: &mockETCDKV{
// remove: func(s string) error {
// return fmt.Errorf("error")
// },
// },
// }),
// chunkManager: &chunkManagerMock{
// removeWithPrefix: func(s string) error {
// return nil
// },
// listWithPrefix: func(s string, recursive bool) ([]string, []time.Time, error) {
// if !recursive {
// return []string{"a/b/1/"}, []time.Time{{}}, nil
// }
// return []string{"a/b/1/c"}, []time.Time{{}}, nil
// },
// remove: func(s string) error {
// return fmt.Errorf("error")
// },
// },
// }
//
// gc.wg.Add(1)
// go gc.recycleUnusedIndexFiles()
// time.Sleep(time.Second)
// cancel()
// gc.wg.Wait()
// })
//
// t.Run("meta mark deleted", func(t *testing.T) {
// ctx, cancel := context.WithCancel(context.Background())
// gc := &garbageCollector{
// ctx: ctx,
// cancel: cancel,
// wg: sync.WaitGroup{},
// gcFileDuration: time.Millisecond * 300,
// gcMetaDuration: time.Millisecond * 300,
// metaTable: constructMetaTable(&indexcoord.Catalog{
// Txn: &mockETCDKV{
// remove: func(s string) error {
// return fmt.Errorf("error")
// },
// },
// }),
// chunkManager: &chunkManagerMock{
// removeWithPrefix: func(s string) error {
// return nil
// },
// listWithPrefix: func(s string, recursive bool) ([]string, []time.Time, error) {
// if !recursive {
// return []string{"a/b/1/"}, []time.Time{{}}, nil
// }
// return []string{"a/b/1/c"}, []time.Time{{}}, nil
// },
// remove: func(s string) error {
// return fmt.Errorf("error")
// },
// },
// }
//
// gc.wg.Add(1)
// go gc.recycleUnusedIndexFiles()
// time.Sleep(time.Second)
// cancel()
// gc.wg.Wait()
// })
//}
//
//func TestIndexCoord_recycleUnusedMetaLoop(t *testing.T) {
// t.Run("success", func(t *testing.T) {
// ctx, cancel := context.WithCancel(context.Background())
// gc := &garbageCollector{
// ctx: ctx,
// cancel: cancel,
// wg: sync.WaitGroup{},
// gcFileDuration: time.Millisecond * 300,
// gcMetaDuration: time.Millisecond * 300,
// metaTable: constructMetaTable(&indexcoord.Catalog{
// Txn: &mockETCDKV{
// remove: func(s string) error {
// return fmt.Errorf("error")
// },
// multiSave: func(m map[string]string) error {
// return nil
// },
// },
// }),
// indexCoordClient: &IndexCoord{
// dataCoordClient: NewDataCoordMock(),
// },
// }
// gc.wg.Add(1)
// go gc.recycleUnusedSegIndexes()
// time.Sleep(time.Second)
// cancel()
// gc.wg.Wait()
// })
//
// t.Run("remove meta failed", func(t *testing.T) {
// ctx, cancel := context.WithCancel(context.Background())
// gc := &garbageCollector{
// ctx: ctx,
// cancel: cancel,
// wg: sync.WaitGroup{},
// gcFileDuration: time.Millisecond * 300,
// gcMetaDuration: time.Millisecond * 300,
// metaTable: &metaTable{
// buildID2SegmentIndex: map[UniqueID]*model.SegmentIndex{
// 1: {
// SegmentID: 0,
// CollectionID: 0,
// PartitionID: 0,
// NumRows: 0,
// IndexID: 0,
// BuildID: 0,
// NodeID: 0,
// IndexVersion: 0,
// IndexState: 0,
// FailReason: "",
// IsDeleted: true,
// CreateTime: 0,
// IndexFileKeys: nil,
// IndexSize: 0,
// },
// },
// catalog: &indexcoord.Catalog{
// Txn: &mockETCDKV{
// remove: func(s string) error {
// return fmt.Errorf("error")
// },
// },
// },
// },
// indexCoordClient: &IndexCoord{
// dataCoordClient: NewDataCoordMock(),
// },
// }
// gc.wg.Add(1)
// go gc.recycleUnusedSegIndexes()
// time.Sleep(time.Second)
// cancel()
// gc.wg.Wait()
// })
//}
//
//func TestGarbageCollector_Recycle(t *testing.T) {
// ctx, canel := context.WithCancel(context.Background())
// gc := &garbageCollector{
// ctx: ctx,
// cancel: canel,
// wg: sync.WaitGroup{},
// gcFileDuration: 300 * time.Millisecond,
// gcMetaDuration: 300 * time.Millisecond,
// metaTable: createMetaTable(&indexcoord.Catalog{Txn: NewMockEtcdKV()}),
// chunkManager: &chunkManagerMock{
// listWithPrefix: func(s string, b bool) ([]string, []time.Time, error) {
// return nil, nil, nil
// },
// },
// indexCoordClient: &IndexCoord{
// dataCoordClient: NewDataCoordMock(),
// },
// }
//
// gc.Start()
// time.Sleep(time.Second)
// err := gc.metaTable.MarkIndexAsDeleted(collID, []int64{indexID})
// assert.NoError(t, err)
// time.Sleep(time.Second * 10)
// gc.Stop()
//}

View File

@ -1,332 +0,0 @@
// 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 indexcoord
import (
"context"
"errors"
"sort"
"sync"
"time"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/golang/protobuf/proto"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/kv"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/querypb"
)
type handoff struct {
ctx context.Context
cancel context.CancelFunc
segments map[UniqueID]*datapb.SegmentInfo
taskMutex sync.RWMutex
wg sync.WaitGroup
meta *metaTable
notifyChan chan struct{}
scheduleDuration time.Duration
kvClient kv.MetaKv
ic *IndexCoord
}
func newHandoff(ctx context.Context, metaTable *metaTable, kvClient kv.MetaKv, ic *IndexCoord) *handoff {
ctx, cancel := context.WithCancel(ctx)
hd := &handoff{
ctx: ctx,
cancel: cancel,
segments: make(map[UniqueID]*datapb.SegmentInfo),
taskMutex: sync.RWMutex{},
wg: sync.WaitGroup{},
meta: metaTable,
notifyChan: make(chan struct{}, 1),
scheduleDuration: time.Second,
kvClient: kvClient,
ic: ic,
}
hd.recoveryFromMeta()
log.Ctx(ctx).Info("new handoff success")
return hd
}
func (hd *handoff) recoveryFromMeta() {
allSegIndexes := hd.meta.GetAllSegIndexes()
hd.taskMutex.Lock()
defer hd.taskMutex.Unlock()
hd.segments = make(map[UniqueID]*datapb.SegmentInfo, 0)
for segID, segIdx := range allSegIndexes {
if segIdx.IsDeleted {
continue
}
if segIdx.WriteHandoff {
continue
}
hd.segments[segID] = &datapb.SegmentInfo{ID: segID}
}
log.Ctx(hd.ctx).Info("recovery from meta success", zap.Int("task num", len(hd.segments)))
}
func (hd *handoff) enqueue(segment *datapb.SegmentInfo) {
defer hd.Notify()
hd.taskMutex.Lock()
defer hd.taskMutex.Unlock()
// note: don't reset state if the task contains state
hd.segments[segment.GetID()] = segment
log.Ctx(hd.ctx).Info("handoff task enqueue successfully",
zap.Int64("segID", segment.GetID()),
zap.Bool("isFake", segment.GetIsFake()),
)
}
func (hd *handoff) Start() {
hd.wg.Add(1)
go hd.scheduler()
}
func (hd *handoff) Stop() {
hd.cancel()
hd.wg.Wait()
}
func (hd *handoff) Notify() {
select {
case hd.notifyChan <- struct{}{}:
default:
}
}
func (hd *handoff) scheduler() {
log.Ctx(hd.ctx).Info("IndexCoord handoff start...")
defer hd.wg.Done()
ticker := time.NewTicker(hd.scheduleDuration)
defer ticker.Stop()
for {
select {
case <-hd.ctx.Done():
log.Info("IndexCoord handoff context done, exit...")
return
case <-ticker.C:
hd.run()
case <-hd.notifyChan:
hd.run()
}
}
}
func (hd *handoff) run() {
hd.taskMutex.RLock()
segIDs := make([]UniqueID, 0, len(hd.segments))
for segID := range hd.segments {
segIDs = append(segIDs, segID)
}
hd.taskMutex.RUnlock()
sort.Slice(segIDs, func(i, j int) bool {
return segIDs[i] < segIDs[j]
})
if len(segIDs) > 0 {
log.Ctx(hd.ctx).Debug("handoff process...", zap.Int("task num", len(segIDs)))
}
for _, segID := range segIDs {
hd.process(segID)
}
}
func (hd *handoff) handoffFakedSegment(segment *datapb.SegmentInfo) {
if hd.allParentsDone(segment.GetCompactionFrom()) {
handoffSegment := &querypb.SegmentInfo{
SegmentID: segment.GetID(),
CollectionID: segment.GetCollectionID(),
PartitionID: segment.GetPartitionID(),
CompactionFrom: segment.GetCompactionFrom(),
CreatedByCompaction: segment.GetCreatedByCompaction(),
IsFake: segment.GetIsFake(),
}
if err := hd.writeHandoffSegment(handoffSegment); err != nil {
log.Ctx(hd.ctx).Warn("write handoff task fail, need to retry", zap.Int64("segID", segment.GetID()), zap.Error(err))
return
}
log.Ctx(hd.ctx).Info("write handoff task success",
zap.Int64("segID", segment.GetID()),
zap.Bool("isFake", segment.GetIsFake()),
zap.Any("segment", segment))
hd.deleteTask(segment.GetID())
}
}
func (hd *handoff) process(segID UniqueID) {
hd.taskMutex.RLock()
segment, ok := hd.segments[segID]
hd.taskMutex.RUnlock()
if !ok {
log.Ctx(hd.ctx).Warn("handoff get task fail", zap.Int64("segID", segID))
return
}
if segment.GetIsFake() {
hd.handoffFakedSegment(segment)
return
}
state := hd.meta.GetSegmentIndexState(segID)
log.Ctx(hd.ctx).RatedDebug(30, "handoff task is process", zap.Int64("segID", segID),
zap.String("state", state.state.String()))
if state.state == commonpb.IndexState_Failed {
log.Ctx(hd.ctx).Error("build index failed, may be need manual intervention", zap.Int64("segID", segID),
zap.String("fail reason", state.failReason))
hd.deleteTask(segID)
// TODO @xiaocai2333: need write handoff event?
return
}
if state.state == commonpb.IndexState_Finished {
log.Ctx(hd.ctx).Debug("build index for segment success, write handoff event...", zap.Int64("segID", segID))
info, err := hd.ic.pullSegmentInfo(hd.ctx, segID)
if err != nil {
if errors.Is(err, ErrSegmentNotFound) {
log.Ctx(hd.ctx).Warn("handoff get segment fail, remove task", zap.Error(err))
hd.deleteTask(segID)
return
}
log.Ctx(hd.ctx).Warn("handoff get segment fail, need to retry", zap.Error(err))
return
}
if info.IsImporting {
log.Debug("segment is importing, can't write handoff event", zap.Int64("segID", segID))
return
}
if hd.allParentsDone(info.CompactionFrom) {
log.Ctx(hd.ctx).Debug("segment can write handoff event", zap.Int64("segID", segID),
zap.Int64s("compactionFrom", info.CompactionFrom))
indexInfos := hd.meta.GetSegmentIndexes(segID)
if len(indexInfos) == 0 {
log.Ctx(hd.ctx).Warn("ready to write handoff, but there is no index, may be dropped, remove task",
zap.Int64("segID", segID))
hd.deleteTask(segID)
return
}
handoffTask := &querypb.SegmentInfo{
SegmentID: segID,
CollectionID: info.CollectionID,
PartitionID: info.PartitionID,
NumRows: info.NumOfRows,
DmChannel: info.GetInsertChannel(),
CompactionFrom: info.CompactionFrom,
CreatedByCompaction: info.CreatedByCompaction,
SegmentState: info.State,
IndexInfos: make([]*querypb.FieldIndexInfo, 0),
EnableIndex: true,
}
for _, indexInfo := range indexInfos {
handoffTask.IndexInfos = append(handoffTask.IndexInfos, &querypb.FieldIndexInfo{
FieldID: hd.meta.GetFieldIDByIndexID(info.CollectionID, indexInfo.IndexID),
EnableIndex: true,
IndexName: hd.meta.GetIndexNameByID(info.CollectionID, indexInfo.IndexID),
IndexID: indexInfo.IndexID,
BuildID: indexInfo.BuildID,
IndexParams: hd.meta.GetIndexParams(info.CollectionID, indexInfo.IndexID),
//IndexFileKeys: nil,
//IndexSize: 0,
})
}
if !hd.meta.AlreadyWrittenHandoff(segID) {
if err := hd.writeHandoffSegment(handoffTask); err != nil {
log.Ctx(hd.ctx).Warn("write handoff task fail, need to retry", zap.Int64("segID", segID), zap.Error(err))
return
}
log.Ctx(hd.ctx).Info("write handoff success", zap.Int64("segID", segID))
if err := hd.meta.MarkSegmentWriteHandoff(segID); err != nil {
log.Ctx(hd.ctx).Warn("mark segment as write handoff fail, need to retry", zap.Int64("segID", segID), zap.Error(err))
return
}
}
log.Ctx(hd.ctx).Info("mark segment as write handoff success, remove task", zap.Int64("segID", segID))
hd.deleteTask(segID)
return
}
log.Ctx(hd.ctx).RatedDebug(5, "the handoff of the parent segment has not been written yet",
zap.Int64("segID", segID), zap.Int64s("compactionFrom", info.CompactionFrom))
}
}
func (hd *handoff) Len() int {
hd.taskMutex.RLock()
defer hd.taskMutex.RUnlock()
return len(hd.segments)
}
func (hd *handoff) deleteTask(segID UniqueID) {
hd.taskMutex.Lock()
defer hd.taskMutex.Unlock()
delete(hd.segments, segID)
}
func (hd *handoff) taskDone(segID UniqueID) bool {
hd.taskMutex.RLock()
defer hd.taskMutex.RUnlock()
_, ok := hd.segments[segID]
return !ok
}
func (hd *handoff) allParentsDone(segIDs []UniqueID) bool {
hd.taskMutex.RLock()
defer hd.taskMutex.RUnlock()
for _, segID := range segIDs {
if _, ok := hd.segments[segID]; ok {
return false
}
}
return true
}
func (hd *handoff) writeHandoffSegment(info *querypb.SegmentInfo) error {
key := buildHandoffKey(info.CollectionID, info.PartitionID, info.SegmentID)
value, err := proto.Marshal(info)
if err != nil {
log.Error("IndexCoord marshal handoff task fail", zap.Int64("collID", info.CollectionID),
zap.Int64("partID", info.PartitionID), zap.Int64("segID", info.SegmentID), zap.Error(err))
return err
}
err = hd.kvClient.Save(key, string(value))
if err != nil {
log.Error("IndexCoord save handoff task fail", zap.Int64("collID", info.CollectionID),
zap.Int64("partID", info.PartitionID), zap.Int64("segID", info.SegmentID), zap.Error(err))
return err
}
log.Info("IndexCoord write handoff task success", zap.Int64("collID", info.CollectionID),
zap.Int64("partID", info.PartitionID), zap.Int64("segID", info.SegmentID))
return nil
}

View File

@ -1,483 +0,0 @@
// 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 indexcoord
import (
"context"
"errors"
"sync"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/metastore"
"github.com/milvus-io/milvus/internal/metastore/kv/indexcoord"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
)
func createMetaForHandoff(catalog metastore.IndexCoordCatalog) *metaTable {
return &metaTable{
catalog: catalog,
segmentIndexLock: sync.RWMutex{},
indexLock: sync.RWMutex{},
collectionIndexes: map[UniqueID]map[UniqueID]*model.Index{
collID: {
indexID: {
TenantID: "",
CollectionID: collID,
FieldID: fieldID,
IndexID: indexID,
IndexName: indexName,
IsDeleted: false,
CreateTime: 0,
TypeParams: nil,
IndexParams: nil,
},
},
},
segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{
segID: {
indexID: {
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
BuildID: buildID,
IndexState: 1,
IsDeleted: false,
WriteHandoff: false,
},
},
segID + 1: {
indexID: {
SegmentID: segID + 1,
CollectionID: collID,
PartitionID: partID,
BuildID: buildID + 1,
IndexState: 1,
IsDeleted: true,
WriteHandoff: false,
},
},
segID + 2: {
indexID: {
SegmentID: segID + 2,
CollectionID: collID,
PartitionID: partID,
BuildID: buildID + 2,
IndexState: 1,
IsDeleted: false,
WriteHandoff: true,
},
},
},
buildID2SegmentIndex: map[UniqueID]*model.SegmentIndex{
buildID: {
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
BuildID: buildID,
IndexID: indexID,
IndexState: 1,
IsDeleted: false,
WriteHandoff: false,
},
buildID + 1: {
SegmentID: segID + 1,
CollectionID: collID,
PartitionID: partID,
BuildID: buildID + 1,
IndexID: indexID,
IndexState: 1,
IsDeleted: true,
WriteHandoff: false,
},
buildID + 2: {
SegmentID: segID + 2,
CollectionID: collID,
PartitionID: partID,
BuildID: buildID + 2,
IndexID: indexID,
IndexState: 1,
IsDeleted: false,
WriteHandoff: true,
},
},
}
}
func Test_newHandoff(t *testing.T) {
ctx := context.Background()
hd := newHandoff(ctx, createMetaForHandoff(&indexcoord.Catalog{Txn: NewMockEtcdKV()}), NewMockEtcdKV(), &IndexCoord{dataCoordClient: NewDataCoordMock()})
assert.NotNil(t, hd)
assert.Equal(t, 1, len(hd.segments))
hd.enqueue(&datapb.SegmentInfo{ID: segID})
assert.Equal(t, 1, len(hd.segments))
err := hd.meta.AddIndex(&model.SegmentIndex{
SegmentID: segID + 3,
CollectionID: collID,
PartitionID: partID,
NumRows: 0,
IndexID: indexID,
BuildID: buildID + 3,
})
assert.NoError(t, err)
hd.enqueue(&datapb.SegmentInfo{ID: segID + 3})
assert.Equal(t, 2, len(hd.segments))
hd.Start()
err = hd.meta.FinishTask(&indexpb.IndexTaskInfo{
BuildID: buildID,
State: commonpb.IndexState_Finished,
IndexFileKeys: []string{"file1", "file2"},
SerializedSize: 100,
FailReason: "",
})
assert.NoError(t, err)
err = hd.meta.FinishTask(&indexpb.IndexTaskInfo{
BuildID: buildID + 3,
State: commonpb.IndexState_Failed,
IndexFileKeys: nil,
SerializedSize: 0,
FailReason: "failed",
})
assert.NoError(t, err)
// handle ticker
time.Sleep(time.Second * 2)
for hd.Len() != 0 {
time.Sleep(500 * time.Millisecond)
}
assert.True(t, hd.taskDone(segID))
assert.True(t, hd.taskDone(segID+3))
hd.Stop()
}
func Test_process(t *testing.T) {
t.Run("not found segment", func(t *testing.T) {
hd := &handoff{segments: map[UniqueID]*datapb.SegmentInfo{}}
hd.process(segID)
assert.Equal(t, 0, hd.Len())
})
t.Run("write handoff ok for faked segment", func(t *testing.T) {
hd := &handoff{
segments: map[UniqueID]*datapb.SegmentInfo{
segID: {
ID: segID,
IsFake: true,
},
},
taskMutex: sync.RWMutex{},
kvClient: &mockETCDKV{
save: func(s string, s2 string) error {
return nil
},
},
}
hd.process(segID)
assert.Equal(t, 0, hd.Len())
})
}
func Test_handoff_error(t *testing.T) {
t.Run("pullSegmentInfo fail", func(t *testing.T) {
hd := &handoff{
ctx: context.Background(),
segments: map[UniqueID]*datapb.SegmentInfo{
segID: {},
},
taskMutex: sync.RWMutex{},
wg: sync.WaitGroup{},
meta: &metaTable{
segmentIndexLock: sync.RWMutex{},
segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{
segID: {
indexID: {
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
IndexID: indexID,
BuildID: buildID,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
WriteHandoff: false,
},
},
},
},
notifyChan: make(chan struct{}, 1),
scheduleDuration: 0,
kvClient: nil,
ic: &IndexCoord{
dataCoordClient: &DataCoordMock{
CallGetSegmentInfo: func(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) {
return nil, errors.New("error")
},
},
},
}
hd.process(segID)
assert.Equal(t, 1, hd.Len())
hd.ic.dataCoordClient = &DataCoordMock{
CallGetSegmentInfo: func(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) {
return nil, errSegmentNotFound(segID)
},
}
hd.process(segID)
assert.Equal(t, 0, hd.Len())
})
t.Run("is importing", func(t *testing.T) {
hd := &handoff{
ctx: context.Background(),
segments: map[UniqueID]*datapb.SegmentInfo{
segID: {},
},
taskMutex: sync.RWMutex{},
wg: sync.WaitGroup{},
meta: &metaTable{
segmentIndexLock: sync.RWMutex{},
segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{
segID: {
indexID: {
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
IndexID: indexID,
BuildID: buildID,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
WriteHandoff: false,
},
},
},
},
notifyChan: make(chan struct{}, 1),
scheduleDuration: 0,
kvClient: nil,
ic: &IndexCoord{
dataCoordClient: &DataCoordMock{
CallGetSegmentInfo: func(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) {
return &datapb.GetSegmentInfoResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
Infos: []*datapb.SegmentInfo{
{
ID: segID,
CollectionID: collID,
PartitionID: partID,
InsertChannel: "",
NumOfRows: 1024,
State: commonpb.SegmentState_Flushed,
IsImporting: true,
},
},
}, nil
},
},
},
}
hd.process(segID)
assert.Equal(t, 1, hd.Len())
})
t.Run("get index info fail", func(t *testing.T) {
hd := &handoff{
ctx: context.Background(),
segments: map[UniqueID]*datapb.SegmentInfo{
segID: {},
},
taskMutex: sync.RWMutex{},
wg: sync.WaitGroup{},
meta: &metaTable{
segmentIndexLock: sync.RWMutex{},
segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{
segID: {
indexID: {
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
IndexID: indexID,
BuildID: buildID,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: true,
WriteHandoff: false,
},
},
},
},
notifyChan: make(chan struct{}, 1),
scheduleDuration: 0,
kvClient: nil,
ic: &IndexCoord{
dataCoordClient: NewDataCoordMock(),
},
}
hd.process(segID)
assert.Equal(t, 0, hd.Len())
})
t.Run("write handoff fail", func(t *testing.T) {
hd := &handoff{
ctx: context.Background(),
segments: map[UniqueID]*datapb.SegmentInfo{
segID: {},
},
taskMutex: sync.RWMutex{},
wg: sync.WaitGroup{},
meta: &metaTable{
catalog: &indexcoord.Catalog{Txn: NewMockEtcdKV()},
segmentIndexLock: sync.RWMutex{},
segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{
segID: {
indexID: {
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
IndexID: indexID,
BuildID: buildID,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
WriteHandoff: false,
},
},
},
},
notifyChan: make(chan struct{}, 1),
scheduleDuration: 0,
kvClient: &mockETCDKV{
save: func(s string, s2 string) error {
return errors.New("error")
},
},
ic: &IndexCoord{
dataCoordClient: NewDataCoordMock(),
},
}
hd.process(segID)
assert.Equal(t, 1, hd.Len())
})
t.Run("write handoff fail for faked segment", func(t *testing.T) {
hd := &handoff{
ctx: context.Background(),
segments: map[UniqueID]*datapb.SegmentInfo{
segID: {
ID: segID,
IsFake: true,
},
},
taskMutex: sync.RWMutex{},
kvClient: &mockETCDKV{
save: func(s string, s2 string) error {
return errors.New("error")
},
},
}
hd.process(segID)
assert.Equal(t, 1, hd.Len())
})
t.Run("mark meta as write handoff fail", func(t *testing.T) {
hd := &handoff{
ctx: context.Background(),
segments: map[UniqueID]*datapb.SegmentInfo{
segID: {},
},
taskMutex: sync.RWMutex{},
wg: sync.WaitGroup{},
meta: &metaTable{
catalog: &indexcoord.Catalog{Txn: &mockETCDKV{
multiSave: func(m map[string]string) error {
return errors.New("error")
},
}},
segmentIndexLock: sync.RWMutex{},
segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{
segID: {
indexID: {
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
IndexID: indexID,
BuildID: buildID,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IsDeleted: false,
WriteHandoff: false,
},
},
},
},
notifyChan: make(chan struct{}, 1),
scheduleDuration: 0,
kvClient: NewMockEtcdKV(),
ic: &IndexCoord{
dataCoordClient: NewDataCoordMock(),
},
}
hd.process(segID)
assert.Equal(t, 1, hd.Len())
})
}
func Test_handoff_allParentsDone(t *testing.T) {
t.Run("done", func(t *testing.T) {
hd := &handoff{
segments: map[UniqueID]*datapb.SegmentInfo{
segID: {},
},
taskMutex: sync.RWMutex{},
}
done := hd.allParentsDone([]UniqueID{segID + 1, segID + 2, segID + 3})
assert.True(t, done)
})
t.Run("not done", func(t *testing.T) {
hd := &handoff{
segments: map[UniqueID]*datapb.SegmentInfo{
segID: {},
segID + 1: {},
},
taskMutex: sync.RWMutex{},
}
done := hd.allParentsDone([]UniqueID{segID + 1, segID + 2, segID + 3})
assert.False(t, done)
})
}

View File

@ -1,510 +0,0 @@
// 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 indexcoord
import (
"context"
"errors"
"path"
"sort"
"sync"
"time"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/common"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/indexpb"
)
type indexBuilder struct {
ctx context.Context
cancel context.CancelFunc
wg sync.WaitGroup
taskMutex sync.RWMutex
scheduleDuration time.Duration
// TODO @xiaocai2333: use priority queue
tasks map[int64]indexTaskState
notifyChan chan struct{}
ic *IndexCoord
meta *metaTable
}
func newIndexBuilder(ctx context.Context, ic *IndexCoord, metaTable *metaTable, aliveNodes []UniqueID) *indexBuilder {
ctx, cancel := context.WithCancel(ctx)
ib := &indexBuilder{
ctx: ctx,
cancel: cancel,
meta: metaTable,
ic: ic,
tasks: make(map[int64]indexTaskState),
notifyChan: make(chan struct{}, 1),
scheduleDuration: time.Second,
}
ib.reloadFromKV(aliveNodes)
return ib
}
func (ib *indexBuilder) Start() {
ib.wg.Add(1)
go ib.schedule()
}
func (ib *indexBuilder) Stop() {
ib.cancel()
ib.wg.Wait()
}
func (ib *indexBuilder) reloadFromKV(aliveNodes []UniqueID) {
metas := ib.meta.GetAllIndexMeta()
for build, indexMeta := range metas {
// deleted, need to release lock and clean meta
if indexMeta.IsDeleted || ib.meta.IsIndexDeleted(indexMeta.CollectionID, indexMeta.IndexID) {
if indexMeta.NodeID != 0 {
ib.tasks[build] = indexTaskDeleted
}
} else if indexMeta.IndexState == commonpb.IndexState_Unissued && indexMeta.NodeID == 0 {
// unissued, need to acquire lock and assign task
ib.tasks[build] = indexTaskInit
} else if indexMeta.IndexState == commonpb.IndexState_Unissued && indexMeta.NodeID != 0 {
// retry, need to release lock and reassign task
// need to release reference lock
ib.tasks[build] = indexTaskRetry
} else if indexMeta.IndexState == commonpb.IndexState_InProgress {
// need to check IndexNode is still alive.
alive := false
for _, nodeID := range aliveNodes {
if nodeID == indexMeta.NodeID {
alive = true
break
}
}
if !alive {
// IndexNode is down, need to retry
ib.tasks[build] = indexTaskRetry
} else {
// in_progress, nothing to do
ib.tasks[build] = indexTaskInProgress
}
} else if indexMeta.IndexState == commonpb.IndexState_Finished || indexMeta.IndexState == commonpb.IndexState_Failed {
if indexMeta.NodeID != 0 {
// task is done, but the lock has not been released, need to release.
ib.tasks[build] = indexTaskDone
}
// else: task is done, and lock has been released, no need to add to index builder.
}
}
}
// notify is an unblocked notify function
func (ib *indexBuilder) notify() {
select {
case ib.notifyChan <- struct{}{}:
default:
}
}
func (ib *indexBuilder) enqueue(buildID UniqueID) {
defer ib.notify()
ib.taskMutex.Lock()
defer ib.taskMutex.Unlock()
if _, ok := ib.tasks[buildID]; !ok {
ib.tasks[buildID] = indexTaskInit
}
log.Info("indexBuilder enqueue task", zap.Int64("buildID", buildID))
}
func (ib *indexBuilder) schedule() {
// receive notifyChan
// time ticker
log.Ctx(ib.ctx).Info("index builder schedule loop start")
defer ib.wg.Done()
ticker := time.NewTicker(ib.scheduleDuration)
defer ticker.Stop()
for {
select {
case <-ib.ctx.Done():
log.Ctx(ib.ctx).Warn("index builder ctx done")
return
case _, ok := <-ib.notifyChan:
if ok {
ib.run()
}
// !ok means indexBuild is closed.
case <-ticker.C:
ib.run()
}
}
}
func (ib *indexBuilder) run() {
ib.taskMutex.RLock()
buildIDs := make([]UniqueID, 0, len(ib.tasks))
for tID := range ib.tasks {
buildIDs = append(buildIDs, tID)
}
ib.taskMutex.RUnlock()
sort.Slice(buildIDs, func(i, j int) bool {
return buildIDs[i] < buildIDs[j]
})
if len(buildIDs) > 0 {
log.Ctx(ib.ctx).Info("index builder task schedule", zap.Int("task num", len(buildIDs)))
}
for _, buildID := range buildIDs {
ok := ib.process(buildID)
if !ok {
log.Ctx(ib.ctx).Info("there is no IndexNode available or etcd is not serviceable, wait a minute...")
break
}
}
}
func (ib *indexBuilder) process(buildID UniqueID) bool {
ib.taskMutex.RLock()
state := ib.tasks[buildID]
ib.taskMutex.RUnlock()
updateStateFunc := func(buildID UniqueID, state indexTaskState) {
ib.taskMutex.Lock()
defer ib.taskMutex.Unlock()
ib.tasks[buildID] = state
}
deleteFunc := func(buildID UniqueID) {
ib.taskMutex.Lock()
defer ib.taskMutex.Unlock()
delete(ib.tasks, buildID)
}
meta, exist := ib.meta.GetMeta(buildID)
if !exist {
log.Ctx(ib.ctx).RatedDebug(5, "index task has not exist in meta table, remove task", zap.Int64("buildID", buildID))
deleteFunc(buildID)
return true
}
switch state {
case indexTaskInit:
if !ib.meta.NeedIndex(meta.CollectionID, meta.IndexID) {
log.Ctx(ib.ctx).RatedDebug(5, "task is no need to build index, remove it", zap.Int64("buildID", buildID))
deleteFunc(buildID)
return true
}
indexParams := ib.meta.GetIndexParams(meta.CollectionID, meta.IndexID)
if isFlatIndex(getIndexType(indexParams)) || meta.NumRows < Params.IndexCoordCfg.MinSegmentNumRowsToEnableIndex.GetAsInt64() {
log.Ctx(ib.ctx).Debug("segment does not need index really", zap.Int64("buildID", buildID),
zap.Int64("segID", meta.SegmentID), zap.Int64("num rows", meta.NumRows))
if err := ib.meta.FinishTask(&indexpb.IndexTaskInfo{
BuildID: buildID,
State: commonpb.IndexState_Finished,
IndexFileKeys: nil,
SerializedSize: 0,
FailReason: "",
}); err != nil {
log.Ctx(ib.ctx).Warn("IndexCoord update index state fail", zap.Int64("buildID", buildID), zap.Error(err))
return false
}
updateStateFunc(buildID, indexTaskDone)
return true
}
// peek client
// if all IndexNodes are executing task, wait for one of them to finish the task.
nodeID, client := ib.ic.nodeManager.PeekClient(meta)
if client == nil {
log.Ctx(ib.ctx).RatedInfo(5, "index builder peek client error, there is no available")
return false
}
// update version and set nodeID
if err := ib.meta.UpdateVersion(buildID, nodeID); err != nil {
log.Ctx(ib.ctx).Warn("index builder update index version failed", zap.Int64("build", buildID), zap.Error(err))
return false
}
// acquire lock
//if err := ib.ic.tryAcquireSegmentReferLock(ib.ctx, buildID, nodeID, []UniqueID{meta.SegmentID}); err != nil {
// log.Ctx(ib.ctx).Warn("index builder acquire segment reference lock failed", zap.Int64("buildID", buildID),
// zap.Int64("nodeID", nodeID), zap.Error(err))
// updateStateFunc(buildID, indexTaskRetry)
// return false
//}
info, err := ib.ic.pullSegmentInfo(ib.ctx, meta.SegmentID)
if err != nil {
log.Ctx(ib.ctx).Warn("IndexCoord get segment info from DataCoord fail", zap.Int64("segID", meta.SegmentID),
zap.Int64("buildID", buildID), zap.Error(err))
if errors.Is(err, ErrSegmentNotFound) {
updateStateFunc(buildID, indexTaskDeleted)
return true
}
updateStateFunc(buildID, indexTaskRetry)
return false
}
binLogs := make([]string, 0)
fieldID := ib.meta.GetFieldIDByIndexID(meta.CollectionID, meta.IndexID)
for _, fieldBinLog := range info.GetBinlogs() {
if fieldBinLog.GetFieldID() == fieldID {
for _, binLog := range fieldBinLog.GetBinlogs() {
binLogs = append(binLogs, binLog.LogPath)
}
break
}
}
typeParams := ib.meta.GetTypeParams(meta.CollectionID, meta.IndexID)
var storageConfig *indexpb.StorageConfig
if Params.CommonCfg.StorageType.GetValue() == "local" {
storageConfig = &indexpb.StorageConfig{
RootPath: Params.LocalStorageCfg.Path.GetValue(),
StorageType: Params.CommonCfg.StorageType.GetValue(),
}
} else {
storageConfig = &indexpb.StorageConfig{
Address: Params.MinioCfg.Address.GetValue(),
AccessKeyID: Params.MinioCfg.AccessKeyID.GetValue(),
SecretAccessKey: Params.MinioCfg.SecretAccessKey.GetValue(),
UseSSL: Params.MinioCfg.UseSSL.GetAsBool(),
BucketName: Params.MinioCfg.BucketName.GetValue(),
RootPath: Params.MinioCfg.RootPath.GetValue(),
UseIAM: Params.MinioCfg.UseIAM.GetAsBool(),
IAMEndpoint: Params.MinioCfg.IAMEndpoint.GetValue(),
StorageType: Params.CommonCfg.StorageType.GetValue(),
}
}
req := &indexpb.CreateJobRequest{
ClusterID: Params.CommonCfg.ClusterPrefix.GetValue(),
IndexFilePrefix: path.Join(ib.ic.chunkManager.RootPath(), common.SegmentIndexPath),
BuildID: buildID,
DataPaths: binLogs,
IndexVersion: meta.IndexVersion + 1,
StorageConfig: storageConfig,
IndexParams: indexParams,
TypeParams: typeParams,
NumRows: meta.NumRows,
}
if err := ib.ic.assignTask(client, req); err != nil {
// need to release lock then reassign, so set task state to retry
log.Ctx(ib.ctx).RatedWarn(10, "index builder assign task to IndexNode failed", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID), zap.Error(err))
updateStateFunc(buildID, indexTaskRetry)
return false
}
log.Ctx(ib.ctx).Info("index task assigned successfully", zap.Int64("buildID", buildID),
zap.Int64("segID", meta.SegmentID), zap.Int64("nodeID", nodeID))
// update index meta state to InProgress
if err := ib.meta.BuildIndex(buildID); err != nil {
// need to release lock then reassign, so set task state to retry
log.Ctx(ib.ctx).Warn("index builder update index meta to InProgress failed", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID), zap.Error(err))
updateStateFunc(buildID, indexTaskRetry)
return false
}
updateStateFunc(buildID, indexTaskInProgress)
case indexTaskDone:
if !ib.meta.NeedIndex(meta.CollectionID, meta.IndexID) {
log.Ctx(ib.ctx).Info("task is no need to build index, remove it", zap.Int64("buildID", buildID),
zap.Int64("segID", meta.SegmentID))
updateStateFunc(buildID, indexTaskDeleted)
return true
}
if !ib.dropIndexTask(buildID, meta.NodeID) {
return true
}
if err := ib.releaseLockAndResetNode(buildID, meta.NodeID); err != nil {
// release lock failed, no need to modify state, wait to retry
return false
}
deleteFunc(buildID)
case indexTaskRetry:
if !ib.meta.NeedIndex(meta.CollectionID, meta.IndexID) {
log.Ctx(ib.ctx).Info("task is no need to build index, remove it", zap.Int64("buildID", buildID))
updateStateFunc(buildID, indexTaskDeleted)
return true
}
if !ib.dropIndexTask(buildID, meta.NodeID) {
return true
}
if err := ib.releaseLockAndResetTask(buildID, meta.NodeID); err != nil {
// release lock failed, no need to modify state, wait to retry
return false
}
updateStateFunc(buildID, indexTaskInit)
case indexTaskDeleted:
log.Ctx(ib.ctx).Debug("index task state is deleted, try to release reference lock", zap.Int64("buildID", buildID))
if err := ib.meta.MarkSegmentsIndexAsDeletedByBuildID([]int64{buildID}); err != nil {
return false
}
if meta.NodeID != 0 {
if !ib.dropIndexTask(buildID, meta.NodeID) {
log.Ctx(ib.ctx).Warn("index task state is deleted and drop index job for node fail", zap.Int64("build", buildID),
zap.Int64("nodeID", meta.NodeID))
return true
}
if err := ib.releaseLockAndResetNode(buildID, meta.NodeID); err != nil {
// release lock failed, no need to modify state, wait to retry
return false
}
}
// reset nodeID success, remove task.
deleteFunc(buildID)
default:
// state: in_progress
if !ib.meta.NeedIndex(meta.CollectionID, meta.IndexID) {
log.Ctx(ib.ctx).Debug("task is no need to build index, remove it", zap.Int64("buildID", buildID))
updateStateFunc(buildID, indexTaskDeleted)
return true
}
updateStateFunc(buildID, ib.getTaskState(buildID, meta.NodeID))
}
return true
}
func (ib *indexBuilder) getTaskState(buildID, nodeID UniqueID) indexTaskState {
client, exist := ib.ic.nodeManager.GetClientByID(nodeID)
if exist {
ctx1, cancel := context.WithTimeout(ib.ctx, reqTimeoutInterval)
defer cancel()
response, err := client.QueryJobs(ctx1, &indexpb.QueryJobsRequest{
ClusterID: Params.CommonCfg.ClusterPrefix.GetValue(),
BuildIDs: []int64{buildID},
})
if err != nil {
log.Ctx(ib.ctx).Warn("IndexCoord get jobs info from IndexNode fail", zap.Int64("nodeID", nodeID),
zap.Error(err))
return indexTaskInProgress
}
if response.Status.ErrorCode != commonpb.ErrorCode_Success {
log.Ctx(ib.ctx).Warn("IndexCoord get jobs info from IndexNode fail", zap.Int64("nodeID", nodeID),
zap.Int64("buildID", buildID), zap.String("fail reason", response.Status.Reason))
return indexTaskInProgress
}
// indexInfos length is always one.
for _, info := range response.IndexInfos {
if info.BuildID == buildID {
if info.State == commonpb.IndexState_Failed || info.State == commonpb.IndexState_Finished {
log.Ctx(ib.ctx).Info("this task has been finished", zap.Int64("buildID", info.BuildID),
zap.String("index state", info.State.String()))
if err := ib.meta.FinishTask(info); err != nil {
log.Ctx(ib.ctx).Warn("IndexCoord update index state fail", zap.Int64("buildID", info.BuildID),
zap.String("index state", info.State.String()), zap.Error(err))
return indexTaskInProgress
}
return indexTaskDone
} else if info.State == commonpb.IndexState_Retry || info.State == commonpb.IndexState_IndexStateNone {
log.Ctx(ib.ctx).Info("this task should be retry", zap.Int64("buildID", buildID), zap.String("fail reason", info.FailReason))
return indexTaskRetry
}
return indexTaskInProgress
}
}
log.Ctx(ib.ctx).Info("this task should be retry, indexNode does not have this task", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID))
return indexTaskRetry
}
// !exist --> node down
log.Ctx(ib.ctx).Info("this task should be retry, indexNode is no longer exist", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID))
return indexTaskRetry
}
func (ib *indexBuilder) dropIndexTask(buildID, nodeID UniqueID) bool {
client, exist := ib.ic.nodeManager.GetClientByID(nodeID)
if exist {
ctx1, cancel := context.WithTimeout(ib.ctx, reqTimeoutInterval)
defer cancel()
status, err := client.DropJobs(ctx1, &indexpb.DropJobsRequest{
ClusterID: Params.CommonCfg.ClusterPrefix.GetValue(),
BuildIDs: []UniqueID{buildID},
})
if err != nil {
log.Ctx(ib.ctx).Warn("IndexCoord notify IndexNode drop the index task fail", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID), zap.Error(err))
return false
}
if status.ErrorCode != commonpb.ErrorCode_Success {
log.Ctx(ib.ctx).Warn("IndexCoord notify IndexNode drop the index task fail", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID), zap.String("fail reason", status.Reason))
return false
}
log.Ctx(ib.ctx).Info("IndexCoord notify IndexNode drop the index task success",
zap.Int64("buildID", buildID), zap.Int64("nodeID", nodeID))
return true
}
log.Ctx(ib.ctx).Info("IndexNode no longer exist, no need to drop index task",
zap.Int64("buildID", buildID), zap.Int64("nodeID", nodeID))
return true
}
func (ib *indexBuilder) releaseLockAndResetNode(buildID UniqueID, nodeID UniqueID) error {
//if err := ib.ic.tryReleaseSegmentReferLock(ib.ctx, buildID, nodeID); err != nil {
// // release lock failed, no need to modify state, wait to retry
// log.Ctx(ib.ctx).Warn("index builder try to release reference lock failed", zap.Int64("buildID", buildID),
// zap.Int64("nodeID", nodeID), zap.Error(err))
// return err
//}
if err := ib.meta.ResetNodeID(buildID); err != nil {
log.Ctx(ib.ctx).Warn("index builder try to reset nodeID failed", zap.Error(err))
return err
}
log.Ctx(ib.ctx).Info("release segment reference lock and reset nodeID success", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID))
return nil
}
func (ib *indexBuilder) releaseLockAndResetTask(buildID UniqueID, nodeID UniqueID) error {
//if nodeID != 0 {
//if err := ib.ic.tryReleaseSegmentReferLock(ib.ctx, buildID, nodeID); err != nil {
// // release lock failed, no need to modify state, wait to retry
// log.Ctx(ib.ctx).Warn("index builder try to release reference lock failed", zap.Error(err))
// return err
//}
//}
if err := ib.meta.ResetMeta(buildID); err != nil {
log.Ctx(ib.ctx).Warn("index builder try to reset task failed", zap.Error(err))
return err
}
log.Ctx(ib.ctx).Info("release segment reference lock and reset task success", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID))
return nil
}
func (ib *indexBuilder) nodeDown(nodeID UniqueID) {
defer ib.notify()
metas := ib.meta.GetMetasByNodeID(nodeID)
ib.taskMutex.Lock()
defer ib.taskMutex.Unlock()
for _, meta := range metas {
if ib.tasks[meta.BuildID] != indexTaskDone {
ib.tasks[meta.BuildID] = indexTaskRetry
}
}
}

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

View File

@ -1,666 +0,0 @@
// 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 indexcoord
import (
"context"
"math/rand"
"sync"
"time"
clientv3 "go.etcd.io/etcd/client/v3"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus/internal/kv"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/typeutil"
)
// Mock is an alternative to IndexCoord, it will return specific results based on specific parameters.
type Mock struct {
types.IndexCoord
CallInit func() error
CallStart func() error
CallStop func() error
CallGetComponentStates func(ctx context.Context) (*milvuspb.ComponentStates, error)
CallGetStatisticsChannel func(ctx context.Context) (*milvuspb.StringResponse, error)
CallRegister func() error
CallSetAddress func(address string)
CallSetEtcdClient func(etcdClient *clientv3.Client)
CallSetDataCoord func(dataCoord types.DataCoord) error
CallSetRootCoord func(rootCoord types.RootCoord) error
CallUpdateStateCode func(stateCode commonpb.StateCode)
CallCreateIndex func(ctx context.Context, req *indexpb.CreateIndexRequest) (*commonpb.Status, error)
CallGetIndexState func(ctx context.Context, req *indexpb.GetIndexStateRequest) (*indexpb.GetIndexStateResponse, error)
CallGetSegmentIndexState func(ctx context.Context, req *indexpb.GetSegmentIndexStateRequest) (*indexpb.GetSegmentIndexStateResponse, error)
CallGetIndexInfos func(ctx context.Context, req *indexpb.GetIndexInfoRequest) (*indexpb.GetIndexInfoResponse, error)
CallDescribeIndex func(ctx context.Context, req *indexpb.DescribeIndexRequest) (*indexpb.DescribeIndexResponse, error)
CallGetIndexBuildProgress func(ctx context.Context, req *indexpb.GetIndexBuildProgressRequest) (*indexpb.GetIndexBuildProgressResponse, error)
CallDropIndex func(ctx context.Context, req *indexpb.DropIndexRequest) (*commonpb.Status, error)
CallShowConfigurations func(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error)
CallGetMetrics func(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error)
}
// Init initializes the Mock of IndexCoord. When param `Failure` is true, it will return an error.
func (m *Mock) Init() error {
return m.CallInit()
}
// Start starts the Mock of IndexCoord. When param `Failure` is true, it will return an error.
func (m *Mock) Start() error {
return m.CallStart()
}
// Stop stops the Mock of IndexCoord. When param `Failure` is true, it will return an error.
func (m *Mock) Stop() error {
return m.CallStop()
}
// Register registers an IndexCoord role in ETCD, if Param `Failure` is true, it will return an error.
func (m *Mock) Register() error {
return m.CallRegister()
}
func (m *Mock) SetAddress(address string) {
m.CallSetAddress(address)
}
func (m *Mock) SetEtcdClient(client *clientv3.Client) {
m.CallSetEtcdClient(client)
}
func (m *Mock) SetDataCoord(dataCoord types.DataCoord) error {
return m.CallSetDataCoord(dataCoord)
}
func (m *Mock) SetRootCoord(rootCoord types.RootCoord) error {
return m.CallSetRootCoord(rootCoord)
}
func (m *Mock) UpdateStateCode(stateCode commonpb.StateCode) {
m.CallUpdateStateCode(stateCode)
}
// GetComponentStates gets the component states of the mocked IndexCoord.
func (m *Mock) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
return m.CallGetComponentStates(ctx)
}
// GetStatisticsChannel gets the statistics channel of the mocked IndexCoord, if Param `Failure` is true, it will return an error.
func (m *Mock) GetStatisticsChannel(ctx context.Context) (*milvuspb.StringResponse, error) {
return m.CallGetStatisticsChannel(ctx)
}
func (m *Mock) CreateIndex(ctx context.Context, req *indexpb.CreateIndexRequest) (*commonpb.Status, error) {
return m.CallCreateIndex(ctx, req)
}
func (m *Mock) GetIndexState(ctx context.Context, req *indexpb.GetIndexStateRequest) (*indexpb.GetIndexStateResponse, error) {
return m.CallGetIndexState(ctx, req)
}
func (m *Mock) GetSegmentIndexState(ctx context.Context, req *indexpb.GetSegmentIndexStateRequest) (*indexpb.GetSegmentIndexStateResponse, error) {
return m.CallGetSegmentIndexState(ctx, req)
}
func (m *Mock) GetIndexInfos(ctx context.Context, req *indexpb.GetIndexInfoRequest) (*indexpb.GetIndexInfoResponse, error) {
return m.CallGetIndexInfos(ctx, req)
}
func (m *Mock) DescribeIndex(ctx context.Context, req *indexpb.DescribeIndexRequest) (*indexpb.DescribeIndexResponse, error) {
return m.CallDescribeIndex(ctx, req)
}
func (m *Mock) GetIndexBuildProgress(ctx context.Context, req *indexpb.GetIndexBuildProgressRequest) (*indexpb.GetIndexBuildProgressResponse, error) {
return m.CallGetIndexBuildProgress(ctx, req)
}
func (m *Mock) DropIndex(ctx context.Context, req *indexpb.DropIndexRequest) (*commonpb.Status, error) {
return m.CallDropIndex(ctx, req)
}
func (m *Mock) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
return m.CallShowConfigurations(ctx, req)
}
func (m *Mock) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
return m.CallGetMetrics(ctx, req)
}
func (m *Mock) CheckHealth(ctx context.Context, req *milvuspb.CheckHealthRequest) (*milvuspb.CheckHealthResponse, error) {
return &milvuspb.CheckHealthResponse{
IsHealthy: true,
}, nil
}
func NewIndexCoordMock() *Mock {
return &Mock{
CallInit: func() error {
return nil
},
CallStart: func() error {
return nil
},
CallRegister: func() error {
return nil
},
CallStop: func() error {
return nil
},
CallSetAddress: func(address string) {
},
CallSetEtcdClient: func(etcdClient *clientv3.Client) {
},
CallSetDataCoord: func(dataCoord types.DataCoord) error {
return nil
},
CallSetRootCoord: func(rootCoord types.RootCoord) error {
return nil
},
CallGetComponentStates: func(ctx context.Context) (*milvuspb.ComponentStates, error) {
return &milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{
NodeID: 1,
Role: typeutil.IndexCoordRole,
StateCode: commonpb.StateCode_Healthy,
},
SubcomponentStates: nil,
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
}, nil
},
CallGetStatisticsChannel: func(ctx context.Context) (*milvuspb.StringResponse, error) {
return &milvuspb.StringResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
}, nil
},
CallCreateIndex: func(ctx context.Context, req *indexpb.CreateIndexRequest) (*commonpb.Status, error) {
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
}, nil
},
CallGetIndexState: func(ctx context.Context, req *indexpb.GetIndexStateRequest) (*indexpb.GetIndexStateResponse, error) {
return &indexpb.GetIndexStateResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
State: commonpb.IndexState_Finished,
}, nil
},
CallGetSegmentIndexState: func(ctx context.Context, req *indexpb.GetSegmentIndexStateRequest) (*indexpb.GetSegmentIndexStateResponse, error) {
segmentStates := make([]*indexpb.SegmentIndexState, 0)
for _, segID := range req.SegmentIDs {
segmentStates = append(segmentStates, &indexpb.SegmentIndexState{
SegmentID: segID,
State: commonpb.IndexState_Finished,
})
}
return &indexpb.GetSegmentIndexStateResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
States: segmentStates,
}, nil
},
CallGetIndexInfos: func(ctx context.Context, req *indexpb.GetIndexInfoRequest) (*indexpb.GetIndexInfoResponse, error) {
segmentInfos := make(map[int64]*indexpb.SegmentInfo)
filePaths := make([]*indexpb.IndexFilePathInfo, 0)
for _, segID := range req.SegmentIDs {
filePaths = append(filePaths, &indexpb.IndexFilePathInfo{
SegmentID: segID,
IndexName: "default",
IndexFilePaths: []string{"file1", "file2"},
})
segmentInfos[segID] = &indexpb.SegmentInfo{
CollectionID: req.CollectionID,
SegmentID: segID,
EnableIndex: true,
IndexInfos: filePaths,
}
}
return &indexpb.GetIndexInfoResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
SegmentInfo: segmentInfos,
}, nil
},
CallDescribeIndex: func(ctx context.Context, req *indexpb.DescribeIndexRequest) (*indexpb.DescribeIndexResponse, error) {
return &indexpb.DescribeIndexResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
IndexInfos: []*indexpb.IndexInfo{
{
CollectionID: 1,
FieldID: 0,
IndexName: "default",
IndexID: 0,
TypeParams: nil,
IndexParams: nil,
},
},
}, nil
},
CallGetIndexBuildProgress: func(ctx context.Context, req *indexpb.GetIndexBuildProgressRequest) (*indexpb.GetIndexBuildProgressResponse, error) {
return &indexpb.GetIndexBuildProgressResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
IndexedRows: 10240,
TotalRows: 10240,
}, nil
},
CallDropIndex: func(ctx context.Context, req *indexpb.DropIndexRequest) (*commonpb.Status, error) {
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
}, nil
},
CallShowConfigurations: func(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
return &internalpb.ShowConfigurationsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
}, nil
},
CallGetMetrics: func(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
return &milvuspb.GetMetricsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
ComponentName: typeutil.IndexCoordRole,
}, nil
},
}
}
type RootCoordMock struct {
types.RootCoord
CallInit func() error
CallStart func() error
CallGetComponentStates func(ctx context.Context) (*milvuspb.ComponentStates, error)
CallAllocID func(ctx context.Context, req *rootcoordpb.AllocIDRequest) (*rootcoordpb.AllocIDResponse, error)
CallAllocTimestamp func(ctx context.Context, req *rootcoordpb.AllocTimestampRequest) (*rootcoordpb.AllocTimestampResponse, error)
}
func (rcm *RootCoordMock) Init() error {
return rcm.CallInit()
}
func (rcm *RootCoordMock) Start() error {
return rcm.CallStart()
}
func (rcm *RootCoordMock) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
return rcm.CallGetComponentStates(ctx)
}
func (rcm *RootCoordMock) AllocTimestamp(ctx context.Context, req *rootcoordpb.AllocTimestampRequest) (*rootcoordpb.AllocTimestampResponse, error) {
return rcm.CallAllocTimestamp(ctx, req)
}
func (rcm *RootCoordMock) AllocID(ctx context.Context, req *rootcoordpb.AllocIDRequest) (*rootcoordpb.AllocIDResponse, error) {
return rcm.CallAllocID(ctx, req)
}
func NewRootCoordMock() *RootCoordMock {
return &RootCoordMock{
CallInit: func() error {
return nil
},
CallStart: func() error {
return nil
},
CallGetComponentStates: func(ctx context.Context) (*milvuspb.ComponentStates, error) {
return &milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{
NodeID: 1,
Role: typeutil.IndexCoordRole,
StateCode: commonpb.StateCode_Healthy,
},
SubcomponentStates: nil,
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
}, nil
},
CallAllocID: func(ctx context.Context, req *rootcoordpb.AllocIDRequest) (*rootcoordpb.AllocIDResponse, error) {
return &rootcoordpb.AllocIDResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
ID: rand.Int63(),
Count: req.Count,
}, nil
},
CallAllocTimestamp: func(ctx context.Context, req *rootcoordpb.AllocTimestampRequest) (*rootcoordpb.AllocTimestampResponse, error) {
return &rootcoordpb.AllocTimestampResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
Timestamp: 1,
Count: req.Count,
}, nil
},
}
}
type DataCoordMock struct {
types.DataCoord
FuncLock sync.RWMutex
CallInit func() error
CallStart func() error
CallGetComponentStates func(ctx context.Context) (*milvuspb.ComponentStates, error)
CallGetSegmentInfo func(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error)
CallGetFlushedSegment func(ctx context.Context, req *datapb.GetFlushedSegmentsRequest) (*datapb.GetFlushedSegmentsResponse, error)
CallAcquireSegmentLock func(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error)
CallReleaseSegmentLock func(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error)
}
func (dcm *DataCoordMock) Init() error {
return dcm.CallInit()
}
func (dcm *DataCoordMock) Start() error {
return dcm.CallStart()
}
func (dcm *DataCoordMock) GetComponentStates(ctx context.Context) (*milvuspb.ComponentStates, error) {
return dcm.CallGetComponentStates(ctx)
}
func (dcm *DataCoordMock) SetFunc(f func()) {
dcm.FuncLock.Lock()
defer dcm.FuncLock.Unlock()
f()
}
func (dcm *DataCoordMock) GetSegmentInfo(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) {
dcm.FuncLock.RLock()
defer dcm.FuncLock.RUnlock()
return dcm.CallGetSegmentInfo(ctx, req)
}
func (dcm *DataCoordMock) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) {
return dcm.CallAcquireSegmentLock(ctx, req)
}
func (dcm *DataCoordMock) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error) {
return dcm.CallReleaseSegmentLock(ctx, req)
}
func (dcm *DataCoordMock) GetFlushedSegments(ctx context.Context, req *datapb.GetFlushedSegmentsRequest) (*datapb.GetFlushedSegmentsResponse, error) {
return dcm.CallGetFlushedSegment(ctx, req)
}
func NewDataCoordMock() *DataCoordMock {
return &DataCoordMock{
CallInit: func() error {
return nil
},
CallStart: func() error {
return nil
},
CallGetComponentStates: func(ctx context.Context) (*milvuspb.ComponentStates, error) {
return &milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{
NodeID: 1,
Role: typeutil.IndexCoordRole,
StateCode: commonpb.StateCode_Healthy,
},
SubcomponentStates: nil,
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
}, nil
},
CallGetSegmentInfo: func(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) {
segInfos := make([]*datapb.SegmentInfo, 0)
for _, segID := range req.SegmentIDs {
segInfos = append(segInfos, &datapb.SegmentInfo{
ID: segID,
CollectionID: 100,
PartitionID: 200,
InsertChannel: "",
NumOfRows: 1026,
State: commonpb.SegmentState_Flushed,
MaxRowNum: 0,
LastExpireTime: 0,
StartPosition: nil,
DmlPosition: nil,
Binlogs: []*datapb.FieldBinlog{
{
Binlogs: []*datapb.Binlog{
{
LogPath: "file1",
},
{
LogPath: "file2",
},
},
},
},
Statslogs: nil,
Deltalogs: nil,
CreatedByCompaction: false,
CompactionFrom: nil,
DroppedAt: 0,
})
}
return &datapb.GetSegmentInfoResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
Infos: segInfos,
}, nil
},
CallGetFlushedSegment: func(ctx context.Context, req *datapb.GetFlushedSegmentsRequest) (*datapb.GetFlushedSegmentsResponse, error) {
return &datapb.GetFlushedSegmentsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
},
}, nil
},
CallAcquireSegmentLock: func(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) {
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
}, nil
},
CallReleaseSegmentLock: func(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error) {
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
}, nil
},
}
}
type mockETCDKV struct {
kv.MetaKv
save func(string, string) error
load func(string) (string, error)
remove func(string) error
multiSave func(map[string]string) error
watchWithRevision func(string, int64) clientv3.WatchChan
loadWithRevisionAndVersions func(string) ([]string, []string, []int64, int64, error)
compareVersionAndSwap func(key string, version int64, target string, opts ...clientv3.OpOption) (bool, error)
loadWithPrefix2 func(key string) ([]string, []string, []int64, error)
loadWithPrefix func(key string) ([]string, []string, error)
loadWithRevision func(key string) ([]string, []string, int64, error)
removeWithPrefix func(key string) error
walkWithPrefix func(prefix string, paginationSize int, fn func([]byte, []byte) error) error
}
func NewMockEtcdKV() *mockETCDKV {
return &mockETCDKV{
save: func(s string, s2 string) error {
return nil
},
load: func(s string) (string, error) {
return "", nil
},
remove: func(s string) error {
return nil
},
multiSave: func(m map[string]string) error {
return nil
},
loadWithRevisionAndVersions: func(s string) ([]string, []string, []int64, int64, error) {
return []string{}, []string{}, []int64{}, 0, nil
},
compareVersionAndSwap: func(key string, version int64, target string, opts ...clientv3.OpOption) (bool, error) {
return true, nil
},
loadWithPrefix2: func(key string) ([]string, []string, []int64, error) {
return []string{}, []string{}, []int64{}, nil
},
loadWithRevision: func(key string) ([]string, []string, int64, error) {
return []string{}, []string{}, 0, nil
},
removeWithPrefix: func(key string) error {
return nil
},
walkWithPrefix: func(prefix string, paginationSize int, fn func([]byte, []byte) error) error {
return nil
},
}
}
func NewMockEtcdKVWithReal(real kv.MetaKv) *mockETCDKV {
return &mockETCDKV{
save: func(s string, s2 string) error {
return real.Save(s, s2)
},
load: func(s string) (string, error) {
return real.Load(s)
},
remove: func(s string) error {
return real.Remove(s)
},
multiSave: func(m map[string]string) error {
return real.MultiSave(m)
},
loadWithRevisionAndVersions: func(s string) ([]string, []string, []int64, int64, error) {
return real.LoadWithRevisionAndVersions(s)
},
compareVersionAndSwap: func(key string, version int64, target string, opts ...clientv3.OpOption) (bool, error) {
return real.CompareVersionAndSwap(key, version, target, opts...)
},
loadWithPrefix: func(key string) ([]string, []string, error) {
return real.LoadWithPrefix(key)
},
loadWithPrefix2: func(key string) ([]string, []string, []int64, error) {
return real.LoadWithPrefix2(key)
},
loadWithRevision: func(key string) ([]string, []string, int64, error) {
return real.LoadWithRevision(key)
},
removeWithPrefix: func(key string) error {
return real.RemoveWithPrefix(key)
},
}
}
func (mk *mockETCDKV) WalkWithPrefix(prefix string, paginationSize int, fn func([]byte, []byte) error) error {
return mk.walkWithPrefix(prefix, paginationSize, fn)
}
func (mk *mockETCDKV) Save(key string, value string) error {
return mk.save(key, value)
}
func (mk *mockETCDKV) Remove(key string) error {
return mk.remove(key)
}
func (mk *mockETCDKV) MultiSave(kvs map[string]string) error {
return mk.multiSave(kvs)
}
func (mk *mockETCDKV) LoadWithRevisionAndVersions(prefix string) ([]string, []string, []int64, int64, error) {
return mk.loadWithRevisionAndVersions(prefix)
}
func (mk *mockETCDKV) CompareVersionAndSwap(key string, version int64, target string, opts ...clientv3.OpOption) (bool, error) {
return mk.compareVersionAndSwap(key, version, target, opts...)
}
func (mk *mockETCDKV) LoadWithPrefix(key string) ([]string, []string, error) {
return mk.loadWithPrefix(key)
}
func (mk *mockETCDKV) LoadWithPrefix2(key string) ([]string, []string, []int64, error) {
return mk.loadWithPrefix2(key)
}
func (mk *mockETCDKV) WatchWithRevision(key string, revision int64) clientv3.WatchChan {
return mk.watchWithRevision(key, revision)
}
func (mk *mockETCDKV) LoadWithRevision(key string) ([]string, []string, int64, error) {
return mk.loadWithRevision(key)
}
func (mk *mockETCDKV) RemoveWithPrefix(key string) error {
return mk.removeWithPrefix(key)
}
func (mk *mockETCDKV) Load(key string) (string, error) {
return mk.load(key)
}
type chunkManagerMock struct {
storage.ChunkManager
removeWithPrefix func(string) error
listWithPrefix func(string, bool) ([]string, []time.Time, error)
remove func(string) error
}
func (cmm *chunkManagerMock) RootPath() string {
return ""
}
func (cmm *chunkManagerMock) RemoveWithPrefix(ctx context.Context, prefix string) error {
return cmm.removeWithPrefix(prefix)
}
func (cmm *chunkManagerMock) ListWithPrefix(ctx context.Context, prefix string, recursive bool) ([]string, []time.Time, error) {
return cmm.listWithPrefix(prefix, recursive)
}
func (cmm *chunkManagerMock) Remove(ctx context.Context, key string) error {
return cmm.remove(key)
}

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

View File

@ -1,148 +0,0 @@
// 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 indexcoord
import (
"context"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/util/hardware"
"github.com/milvus-io/milvus/internal/util/metricsinfo"
"github.com/milvus-io/milvus/internal/util/paramtable"
"github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/milvus-io/milvus/internal/util/uniquegenerator"
)
// TODO(dragondriver): add more detailed metrics
func getSystemInfoMetrics(
ctx context.Context,
req *milvuspb.GetMetricsRequest,
coord *IndexCoord,
) (*milvuspb.GetMetricsResponse, error) {
clusterTopology := metricsinfo.IndexClusterTopology{
Self: metricsinfo.IndexCoordInfos{
BaseComponentInfos: metricsinfo.BaseComponentInfos{
Name: metricsinfo.ConstructComponentName(typeutil.IndexCoordRole, coord.session.ServerID),
HardwareInfos: metricsinfo.HardwareMetrics{
IP: coord.session.Address,
CPUCoreCount: hardware.GetCPUNum(),
CPUCoreUsage: hardware.GetCPUUsage(),
Memory: hardware.GetMemoryCount(),
MemoryUsage: hardware.GetUsedMemoryCount(),
Disk: hardware.GetDiskCount(),
DiskUsage: hardware.GetDiskUsage(),
},
SystemInfo: metricsinfo.DeployMetrics{},
CreatedTime: paramtable.GetCreateTime().String(),
UpdatedTime: paramtable.GetUpdateTime().String(),
Type: typeutil.IndexCoordRole,
ID: coord.session.ServerID,
},
SystemConfigurations: metricsinfo.IndexCoordConfiguration{
MinioBucketName: Params.MinioCfg.BucketName.GetValue(),
},
},
ConnectedNodes: make([]metricsinfo.IndexNodeInfos, 0),
}
metricsinfo.FillDeployMetricsWithEnv(&clusterTopology.Self.SystemInfo)
nodesMetrics := coord.nodeManager.getMetrics(ctx, req)
for _, nodeMetrics := range nodesMetrics {
if nodeMetrics.err != nil {
log.Warn("invalid metrics of index node was found",
zap.Error(nodeMetrics.err))
clusterTopology.ConnectedNodes = append(clusterTopology.ConnectedNodes, metricsinfo.IndexNodeInfos{
BaseComponentInfos: metricsinfo.BaseComponentInfos{
HasError: true,
ErrorReason: nodeMetrics.err.Error(),
// Name doesn't matter here cause we can't get it when error occurs, using address as the Name?
Name: "",
ID: int64(uniquegenerator.GetUniqueIntGeneratorIns().GetInt()),
},
})
continue
}
if nodeMetrics.resp.Status.ErrorCode != commonpb.ErrorCode_Success {
log.Warn("invalid metrics of index node was found",
zap.Any("error_code", nodeMetrics.resp.Status.ErrorCode),
zap.Any("error_reason", nodeMetrics.resp.Status.Reason))
clusterTopology.ConnectedNodes = append(clusterTopology.ConnectedNodes, metricsinfo.IndexNodeInfos{
BaseComponentInfos: metricsinfo.BaseComponentInfos{
HasError: true,
ErrorReason: nodeMetrics.resp.Status.Reason,
Name: nodeMetrics.resp.ComponentName,
ID: int64(uniquegenerator.GetUniqueIntGeneratorIns().GetInt()),
},
})
continue
}
infos := metricsinfo.IndexNodeInfos{}
err := metricsinfo.UnmarshalComponentInfos(nodeMetrics.resp.Response, &infos)
if err != nil {
log.Warn("invalid metrics of index node was found",
zap.Error(err))
clusterTopology.ConnectedNodes = append(clusterTopology.ConnectedNodes, metricsinfo.IndexNodeInfos{
BaseComponentInfos: metricsinfo.BaseComponentInfos{
HasError: true,
ErrorReason: err.Error(),
Name: nodeMetrics.resp.ComponentName,
ID: int64(uniquegenerator.GetUniqueIntGeneratorIns().GetInt()),
},
})
continue
}
clusterTopology.ConnectedNodes = append(clusterTopology.ConnectedNodes, infos)
}
coordTopology := metricsinfo.IndexCoordTopology{
Cluster: clusterTopology,
Connections: metricsinfo.ConnTopology{
Name: metricsinfo.ConstructComponentName(typeutil.IndexCoordRole, coord.session.ServerID),
// TODO(dragondriver): fill ConnectedComponents if necessary
ConnectedComponents: []metricsinfo.ConnectionInfo{},
},
}
resp, err := metricsinfo.MarshalTopology(coordTopology)
if err != nil {
return &milvuspb.GetMetricsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: err.Error(),
},
Response: "",
ComponentName: metricsinfo.ConstructComponentName(typeutil.IndexCoordRole, coord.session.ServerID),
}, nil
}
return &milvuspb.GetMetricsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
},
Response: resp,
ComponentName: metricsinfo.ConstructComponentName(typeutil.IndexCoordRole, coord.session.ServerID),
}, nil
}

View File

@ -1,70 +0,0 @@
// 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 indexcoord
//func TestGetSystemInfoMetrics(t *testing.T) {
// ctx := context.Background()
// factory := dependency.NewDefaultFactory(true)
// ic, err := NewIndexCoord(ctx, factory)
// assert.Nil(t, err)
// Params.Init()
//
// etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
// defer etcdCli.Close()
// assert.NoError(t, err)
//
// ic.SetEtcdClient(etcdCli)
// err = ic.Init()
// assert.Nil(t, err)
// err = ic.Start()
// assert.Nil(t, err)
// err = ic.Register()
// assert.Nil(t, err)
//
// t.Run("getSystemInfoMetrics", func(t *testing.T) {
// req, err := metricsinfo.ConstructRequestByMetricType(metricsinfo.SystemInfoMetrics)
// assert.Nil(t, err)
//
// resp, err := getSystemInfoMetrics(ctx, req, ic)
// assert.Nil(t, err)
// assert.NotNil(t, resp)
// })
//
// t.Run("getSystemInfoMetrics error", func(t *testing.T) {
// req, err := metricsinfo.ConstructRequestByMetricType(metricsinfo.SystemInfoMetrics)
// assert.Nil(t, err)
//
// inm1 := &indexnode.Mock{
// Failure: true,
// Err: true,
// }
// inm2 := &indexnode.Mock{
// Failure: true,
// Err: false,
// }
//
// ic.nodeManager.setClient(1, inm1)
// ic.nodeManager.setClient(2, inm2)
//
// resp, err := getSystemInfoMetrics(ctx, req, ic)
// assert.Nil(t, err)
// assert.NotNil(t, resp)
// })
//
// err = ic.Stop()
// assert.Nil(t, err)
//}

View File

@ -1,277 +0,0 @@
// 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 indexcoord
import (
"context"
"sync"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/milvuspb"
grpcindexnodeclient "github.com/milvus-io/milvus/internal/distributed/indexnode/client"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/metrics"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/types"
)
// NodeManager is used by IndexCoord to manage the client of IndexNode.
type NodeManager struct {
nodeClients map[UniqueID]types.IndexNode
stoppingNodes map[UniqueID]struct{}
pq *PriorityQueue
lock sync.RWMutex
ctx context.Context
}
// NewNodeManager is used to create a new NodeManager.
func NewNodeManager(ctx context.Context) *NodeManager {
return &NodeManager{
nodeClients: make(map[UniqueID]types.IndexNode),
stoppingNodes: make(map[UniqueID]struct{}),
pq: &PriorityQueue{
policy: PeekClientV1,
},
lock: sync.RWMutex{},
ctx: ctx,
}
}
// setClient sets IndexNode client to node manager.
func (nm *NodeManager) setClient(nodeID UniqueID, client types.IndexNode) {
log.Debug("IndexCoord NodeManager setClient", zap.Int64("nodeID", nodeID))
item := &PQItem{
key: nodeID,
priority: 0,
weight: 0,
totalMem: 0,
}
nm.lock.Lock()
nm.nodeClients[nodeID] = client
log.Debug("IndexNode NodeManager setClient success", zap.Int64("nodeID", nodeID), zap.Int("IndexNode num", len(nm.nodeClients)))
nm.lock.Unlock()
nm.pq.Push(item)
}
// RemoveNode removes the unused client of IndexNode.
func (nm *NodeManager) RemoveNode(nodeID UniqueID) {
log.Debug("IndexCoord", zap.Any("Remove node with ID", nodeID))
nm.lock.Lock()
delete(nm.nodeClients, nodeID)
delete(nm.stoppingNodes, nodeID)
nm.lock.Unlock()
nm.pq.Remove(nodeID)
metrics.IndexCoordIndexNodeNum.WithLabelValues().Dec()
}
func (nm *NodeManager) StoppingNode(nodeID UniqueID) {
log.Info("IndexCoord", zap.Any("Stopping node with ID", nodeID))
nm.lock.Lock()
defer nm.lock.Unlock()
nm.stoppingNodes[nodeID] = struct{}{}
}
// AddNode adds the client of IndexNode.
func (nm *NodeManager) AddNode(nodeID UniqueID, address string) error {
log.Debug("IndexCoord addNode", zap.Any("nodeID", nodeID), zap.Any("node address", address))
if nm.pq.CheckExist(nodeID) {
log.Warn("IndexCoord", zap.Any("Node client already exist with ID:", nodeID))
return nil
}
var (
nodeClient types.IndexNode
err error
)
nodeClient, err = grpcindexnodeclient.NewClient(context.TODO(), address, Params.IndexCoordCfg.WithCredential.GetAsBool())
if err != nil {
log.Error("IndexCoord NodeManager", zap.Any("Add node err", err))
return err
}
err = nodeClient.Init()
if err != nil {
log.Error("IndexCoord NodeManager", zap.Any("Add node err", err))
return err
}
metrics.IndexCoordIndexNodeNum.WithLabelValues().Inc()
nm.setClient(nodeID, nodeClient)
return nil
}
// PeekClient peeks the client with the least load.
func (nm *NodeManager) PeekClient(meta *model.SegmentIndex) (UniqueID, types.IndexNode) {
allClients := nm.GetAllClients()
if len(allClients) == 0 {
log.Error("there is no IndexNode online")
return -1, nil
}
// Note: In order to quickly end other goroutines, an error is returned when the client is successfully selected
ctx, cancel := context.WithCancel(nm.ctx)
var (
peekNodeID = UniqueID(0)
nodeMutex = sync.Mutex{}
wg = sync.WaitGroup{}
)
for nodeID, client := range allClients {
nodeID := nodeID
client := client
wg.Add(1)
go func() {
defer wg.Done()
resp, err := client.GetJobStats(ctx, &indexpb.GetJobStatsRequest{})
if err != nil {
log.Warn("get IndexNode slots failed", zap.Int64("nodeID", nodeID), zap.Error(err))
return
}
if resp.Status.ErrorCode != commonpb.ErrorCode_Success {
log.Warn("get IndexNode slots failed", zap.Int64("nodeID", nodeID),
zap.String("reason", resp.Status.Reason))
return
}
if resp.TaskSlots > 0 {
nodeMutex.Lock()
defer nodeMutex.Unlock()
log.Info("peek client success", zap.Int64("nodeID", nodeID))
if peekNodeID == 0 {
peekNodeID = nodeID
}
cancel()
// Note: In order to quickly end other goroutines, an error is returned when the client is successfully selected
return
}
}()
}
wg.Wait()
cancel()
if peekNodeID != 0 {
log.Info("IndexCoord peek client success", zap.Int64("nodeID", peekNodeID))
return peekNodeID, allClients[peekNodeID]
}
log.RatedDebug(30, "IndexCoord peek client fail")
return 0, nil
}
func (nm *NodeManager) ClientSupportDisk() bool {
log.Info("IndexCoord check if client support disk index")
allClients := nm.GetAllClients()
if len(allClients) == 0 {
log.Warn("there is no IndexNode online")
return false
}
// Note: In order to quickly end other goroutines, an error is returned when the client is successfully selected
ctx, cancel := context.WithCancel(nm.ctx)
var (
enableDisk = false
nodeMutex = sync.Mutex{}
wg = sync.WaitGroup{}
)
for nodeID, client := range allClients {
nodeID := nodeID
client := client
wg.Add(1)
go func() {
defer wg.Done()
resp, err := client.GetJobStats(ctx, &indexpb.GetJobStatsRequest{})
if err != nil {
log.Warn("get IndexNode slots failed", zap.Int64("nodeID", nodeID), zap.Error(err))
return
}
if resp.Status.ErrorCode != commonpb.ErrorCode_Success {
log.Warn("get IndexNode slots failed", zap.Int64("nodeID", nodeID),
zap.String("reason", resp.Status.Reason))
return
}
log.Debug("get job stats success", zap.Int64("nodeID", nodeID), zap.Bool("enable disk", resp.EnableDisk))
if resp.EnableDisk {
nodeMutex.Lock()
defer nodeMutex.Unlock()
cancel()
if !enableDisk {
enableDisk = true
}
return
}
}()
}
wg.Wait()
cancel()
if enableDisk {
log.Info("IndexNode support disk index")
return true
}
log.Error("all IndexNodes do not support disk indexes")
return false
}
func (nm *NodeManager) GetAllClients() map[UniqueID]types.IndexNode {
nm.lock.RLock()
defer nm.lock.RUnlock()
allClients := make(map[UniqueID]types.IndexNode, len(nm.nodeClients))
for nodeID, client := range nm.nodeClients {
if _, ok := nm.stoppingNodes[nodeID]; !ok {
allClients[nodeID] = client
}
}
return allClients
}
func (nm *NodeManager) GetClientByID(nodeID UniqueID) (types.IndexNode, bool) {
nm.lock.RLock()
defer nm.lock.RUnlock()
client, ok := nm.nodeClients[nodeID]
return client, ok
}
// indexNodeGetMetricsResponse record the metrics information of IndexNode.
type indexNodeGetMetricsResponse struct {
resp *milvuspb.GetMetricsResponse
err error
}
// getMetrics get metrics information of all IndexNode.
func (nm *NodeManager) getMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) []indexNodeGetMetricsResponse {
var clients []types.IndexNode
nm.lock.RLock()
for _, node := range nm.nodeClients {
clients = append(clients, node)
}
nm.lock.RUnlock()
ret := make([]indexNodeGetMetricsResponse, 0, len(nm.nodeClients))
for _, node := range clients {
resp, err := node.GetMetrics(ctx, req)
ret = append(ret, indexNodeGetMetricsResponse{
resp: resp,
err: err,
})
}
return ret
}

View File

@ -1,266 +0,0 @@
// 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 indexcoord
import (
"context"
"errors"
"sync"
"testing"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/indexnode"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/types"
"github.com/stretchr/testify/assert"
)
func TestNodeManager_PeekClient(t *testing.T) {
t.Run("success", func(t *testing.T) {
nm := NewNodeManager(context.Background())
nodeID, client := nm.PeekClient(&model.SegmentIndex{})
assert.Equal(t, int64(-1), nodeID)
assert.Nil(t, client)
err := nm.AddNode(1, "indexnode-1")
assert.Nil(t, err)
nm.pq.SetMemory(1, 100)
nodeID2, client2 := nm.PeekClient(&model.SegmentIndex{})
assert.Equal(t, int64(0), nodeID2)
assert.Nil(t, client2)
})
t.Run("multiple unavailable IndexNode", func(t *testing.T) {
nm := &NodeManager{
ctx: context.TODO(),
nodeClients: map[UniqueID]types.IndexNode{
1: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, req *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
},
}, errors.New("error")
},
},
2: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, req *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
},
}, errors.New("error")
},
},
3: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, req *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
},
}, errors.New("error")
},
},
4: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, req *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
},
}, errors.New("error")
},
},
5: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, req *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "fail reason",
},
}, nil
},
},
6: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, req *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "fail reason",
},
}, nil
},
},
7: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, req *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "fail reason",
},
}, nil
},
},
8: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, req *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
TaskSlots: 1,
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
},
}, nil
},
},
9: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, req *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
TaskSlots: 10,
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
},
}, nil
},
},
},
}
nodeID, client := nm.PeekClient(&model.SegmentIndex{})
assert.NotNil(t, client)
assert.Contains(t, []UniqueID{8, 9}, nodeID)
})
}
func TestNodeManager_ClientSupportDisk(t *testing.T) {
t.Run("support", func(t *testing.T) {
nm := &NodeManager{
ctx: context.Background(),
lock: sync.RWMutex{},
nodeClients: map[UniqueID]types.IndexNode{
1: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, in *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
},
TaskSlots: 1,
JobInfos: nil,
EnableDisk: true,
}, nil
},
},
},
}
support := nm.ClientSupportDisk()
assert.True(t, support)
})
t.Run("not support", func(t *testing.T) {
nm := &NodeManager{
ctx: context.Background(),
lock: sync.RWMutex{},
nodeClients: map[UniqueID]types.IndexNode{
1: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, in *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_Success,
Reason: "",
},
TaskSlots: 1,
JobInfos: nil,
EnableDisk: false,
}, nil
},
},
},
}
support := nm.ClientSupportDisk()
assert.False(t, support)
})
t.Run("no indexnode", func(t *testing.T) {
nm := &NodeManager{
ctx: context.Background(),
lock: sync.RWMutex{},
nodeClients: map[UniqueID]types.IndexNode{},
}
support := nm.ClientSupportDisk()
assert.False(t, support)
})
t.Run("error", func(t *testing.T) {
nm := &NodeManager{
ctx: context.Background(),
lock: sync.RWMutex{},
nodeClients: map[UniqueID]types.IndexNode{
1: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, in *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return nil, errors.New("error")
},
},
},
}
support := nm.ClientSupportDisk()
assert.False(t, support)
})
t.Run("fail reason", func(t *testing.T) {
nm := &NodeManager{
ctx: context.Background(),
lock: sync.RWMutex{},
nodeClients: map[UniqueID]types.IndexNode{
1: &indexnode.Mock{
CallGetJobStats: func(ctx context.Context, in *indexpb.GetJobStatsRequest) (*indexpb.GetJobStatsResponse, error) {
return &indexpb.GetJobStatsResponse{
Status: &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UnexpectedError,
Reason: "fail reason",
},
TaskSlots: 0,
JobInfos: nil,
EnableDisk: false,
}, nil
},
},
},
}
support := nm.ClientSupportDisk()
assert.False(t, support)
})
}
func TestNodeManager_StoppingNode(t *testing.T) {
nm := NewNodeManager(context.Background())
err := nm.AddNode(1, "indexnode-1")
assert.NoError(t, err)
assert.Equal(t, 1, len(nm.GetAllClients()))
nm.StoppingNode(1)
assert.Equal(t, 0, len(nm.GetAllClients()))
assert.Equal(t, 1, len(nm.stoppingNodes))
nm.RemoveNode(1)
assert.Equal(t, 0, len(nm.GetAllClients()))
assert.Equal(t, 0, len(nm.stoppingNodes))
}

View File

@ -1,41 +0,0 @@
// 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 indexcoord
import "github.com/milvus-io/milvus-proto/go-api/commonpb"
// PeekClientPolicy defines how to choose IndexNode.
type PeekClientPolicy func(memorySize uint64, indexParams []*commonpb.KeyValuePair,
typeParams []*commonpb.KeyValuePair, pq *PriorityQueue) UniqueID
// PeekClientV0 choose the IndexNode with the smallest amount of tasks.
func PeekClientV0(memorySize uint64, indexParams []*commonpb.KeyValuePair,
typeParams []*commonpb.KeyValuePair, pq *PriorityQueue) UniqueID {
return pq.items[0].key
}
// PeekClientV1 choose the one with the smallest amount of tasks among all IndexNodes
// whose memory size meets the requirements for building the index.
func PeekClientV1(memorySize uint64, indexParams []*commonpb.KeyValuePair,
typeParams []*commonpb.KeyValuePair, pq *PriorityQueue) UniqueID {
for i := range pq.items {
if pq.items[i].totalMem > memorySize && pq.items[i].priority < 2 {
return pq.items[i].key
}
}
return UniqueID(0)
}

View File

@ -1,39 +0,0 @@
// 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 indexcoord
import (
"testing"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/stretchr/testify/assert"
)
func TestPeekClientV0(t *testing.T) {
pq := newPriorityQueue()
key := PeekClientV0(10, []*commonpb.KeyValuePair{}, []*commonpb.KeyValuePair{}, pq)
assert.Equal(t, UniqueID(1), key)
}
func TestPeekClientV1(t *testing.T) {
pq := newPriorityQueue()
key := PeekClientV1(10, []*commonpb.KeyValuePair{}, []*commonpb.KeyValuePair{}, pq)
assert.Equal(t, UniqueID(1), key)
key2 := PeekClientV1(10000, []*commonpb.KeyValuePair{}, []*commonpb.KeyValuePair{}, pq)
assert.Equal(t, UniqueID(0), key2)
}

View File

@ -1,194 +0,0 @@
// 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 indexcoord
import (
"container/heap"
"sync"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
)
// PQItem is something we manage in a priority queue.
type PQItem struct {
key UniqueID
priority int // The priority of the item in the queue.
// The index is needed by update and is maintained by the heap.Interface methods.
weight int // The weight of the item in the queue.
// When the priority is the same, a smaller weight is more preferred.
index int // The index of the item in the heap.
totalMem uint64 // The total memory of the IndexNode.
}
// PriorityQueue implements heap.Interface and holds Items.
type PriorityQueue struct {
items []*PQItem
lock sync.RWMutex
policy PeekClientPolicy
}
// Len is the length of the priority queue.
func (pq *PriorityQueue) Len() int {
return len(pq.items)
}
// Less reports whether the element with index i
// must sort before the element with index j.
func (pq *PriorityQueue) Less(i, j int) bool {
// We want Pop to give us the highest, not lowest, priority so we use greater than here.
return (pq.items[i].priority < pq.items[j].priority) ||
(pq.items[i].priority == pq.items[j].priority && pq.items[i].weight < pq.items[j].weight)
}
// Swap swaps the elements with indexes i and j.
func (pq *PriorityQueue) Swap(i, j int) {
pq.items[i], pq.items[j] = pq.items[j], pq.items[i]
pq.items[i].index = i
pq.items[j].index = j
}
// Push adds an element to the priority.
func (pq *PriorityQueue) Push(x interface{}) {
pq.lock.Lock()
defer pq.lock.Unlock()
n := (*pq).Len()
item := x.(*PQItem)
item.index = n
pq.items = append(pq.items, item)
}
// Pop do not call this directly.
func (pq *PriorityQueue) Pop() interface{} {
old := pq.items
n := len(old)
item := old[n-1]
item.index = -1 // for safety
pq.items = old[0 : n-1]
return item
}
// CheckExist checks whether the nodeID is already in the priority.
func (pq *PriorityQueue) CheckExist(nodeID UniqueID) bool {
pq.lock.RLock()
defer pq.lock.RUnlock()
for _, item := range pq.items {
if nodeID == item.key {
return true
}
}
return false
}
func (pq *PriorityQueue) getItemByKey(key UniqueID) interface{} {
var ret interface{}
for _, item := range pq.items {
if item.key == key {
ret = item
break
}
}
return ret
}
// IncPriority update modifies the priority and value of an Item in the queue.
func (pq *PriorityQueue) IncPriority(key UniqueID, priority int) {
pq.lock.Lock()
defer pq.lock.Unlock()
item := pq.getItemByKey(key)
if item != nil {
item.(*PQItem).priority += priority
if priority > 0 {
item.(*PQItem).weight += priority
}
heap.Fix(pq, item.(*PQItem).index)
}
}
// UpdatePriority update modifies the priority and value of an Item in the queue.
func (pq *PriorityQueue) UpdatePriority(key UniqueID, priority int) {
pq.lock.Lock()
defer pq.lock.Unlock()
item := pq.getItemByKey(key)
if item != nil {
item.(*PQItem).priority = priority
item.(*PQItem).weight = priority
heap.Fix(pq, item.(*PQItem).index)
}
}
// Remove deletes the corresponding item according to the key.
func (pq *PriorityQueue) Remove(key UniqueID) {
pq.lock.Lock()
defer pq.lock.Unlock()
item := pq.getItemByKey(key)
if item != nil {
heap.Remove(pq, item.(*PQItem).index)
}
}
// Peek picks an key with the lowest load.
func (pq *PriorityQueue) Peek(memorySize uint64, indexParams []*commonpb.KeyValuePair, typeParams []*commonpb.KeyValuePair) UniqueID {
pq.lock.RLock()
defer pq.lock.RUnlock()
if pq.Len() == 0 {
return UniqueID(-1)
}
return pq.policy(memorySize, indexParams, typeParams, pq)
}
// PeekAll return the key of all the items.
func (pq *PriorityQueue) PeekAll() []UniqueID {
pq.lock.RLock()
defer pq.lock.RUnlock()
var ret []UniqueID
for _, item := range pq.items {
ret = append(ret, item.key)
}
return ret
}
// GetMemory get the memory info for the speicied key.
func (pq *PriorityQueue) GetMemory(key UniqueID) uint64 {
pq.lock.RLock()
defer pq.lock.RUnlock()
for i := range pq.items {
if pq.items[i].key == key {
return pq.items[i].totalMem
}
}
return 0
}
// SetMemory sets the memory info for IndexNode.
func (pq *PriorityQueue) SetMemory(key UniqueID, memorySize uint64) {
pq.lock.Lock()
defer pq.lock.Unlock()
for i := range pq.items {
if pq.items[i].key == key {
pq.items[i].totalMem = memorySize
return
}
}
}

View File

@ -1,146 +0,0 @@
// 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 indexcoord
import (
"container/heap"
"testing"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/stretchr/testify/assert"
)
const QueueLen = 10
func newPriorityQueue() *PriorityQueue {
ret := &PriorityQueue{
policy: PeekClientV0,
}
for i := 1; i <= QueueLen; i++ {
item := &PQItem{
key: UniqueID(i),
priority: i - 1,
index: i - 1,
totalMem: 1000,
}
ret.items = append(ret.items, item)
}
heap.Init(ret)
return ret
}
func TestPriorityQueue_Len(t *testing.T) {
pq := newPriorityQueue()
assert.Equal(t, QueueLen, pq.Len())
pq = nil
}
func TestPriorityQueue_Push(t *testing.T) {
pq := newPriorityQueue()
for i := 1; i <= QueueLen; i++ {
item := &PQItem{
key: UniqueID(i),
priority: i,
index: i,
}
pq.Push(item)
assert.Equal(t, i+QueueLen, pq.Len())
}
}
func TestPriorityQueue_Remove(t *testing.T) {
pq := newPriorityQueue()
cnt := 0
for i := 1; i <= QueueLen; i++ {
if i%2 == 0 {
continue
}
pq.Remove(UniqueID(i))
cnt++
}
assert.Equal(t, QueueLen-cnt, pq.Len())
}
func TestPriorityQueue_UpdatePriority(t *testing.T) {
pq := newPriorityQueue()
key := UniqueID(pq.Len() / 2)
pq.UpdatePriority(key, -pq.Len())
peekKey := pq.Peek(10, []*commonpb.KeyValuePair{}, []*commonpb.KeyValuePair{})
assert.Equal(t, key, peekKey)
}
func TestPriorityQueue_IncPriority(t *testing.T) {
pq := newPriorityQueue()
key := UniqueID(pq.Len() / 2)
pq.IncPriority(key, -pq.Len())
peekKey := pq.Peek(10, []*commonpb.KeyValuePair{}, []*commonpb.KeyValuePair{})
assert.Equal(t, key, peekKey)
}
func TestPriorityQueue_SetMemory(t *testing.T) {
ret := &PriorityQueue{
policy: PeekClientV1,
}
for i := 0; i < QueueLen; i++ {
item := &PQItem{
key: UniqueID(i),
priority: 0,
index: i,
totalMem: 1000,
}
ret.items = append(ret.items, item)
}
heap.Init(ret)
ret.SetMemory(1, 100000)
peeKey := ret.Peek(99999, []*commonpb.KeyValuePair{}, []*commonpb.KeyValuePair{})
assert.Equal(t, int64(1), peeKey)
}
func TestPriorityQueue(t *testing.T) {
ret := &PriorityQueue{
policy: PeekClientV0,
}
for i := 0; i < 4; i++ {
item := &PQItem{
key: UniqueID(i),
priority: 0,
index: i,
totalMem: 1000,
}
ret.items = append(ret.items, item)
}
heap.Init(ret)
peeKey1 := ret.Peek(10, []*commonpb.KeyValuePair{}, []*commonpb.KeyValuePair{})
assert.Equal(t, int64(0), peeKey1)
ret.IncPriority(peeKey1, 1)
peeKey2 := ret.Peek(100, []*commonpb.KeyValuePair{}, []*commonpb.KeyValuePair{})
assert.Equal(t, int64(1), peeKey2)
ret.IncPriority(peeKey2, 1)
ret.IncPriority(peeKey1, -1)
ret.IncPriority(peeKey2, -1)
peeKey1 = ret.Peek(10, []*commonpb.KeyValuePair{}, []*commonpb.KeyValuePair{})
assert.Equal(t, int64(3), peeKey1)
ret.IncPriority(peeKey1, 1)
peeKey2 = ret.Peek(10, []*commonpb.KeyValuePair{}, []*commonpb.KeyValuePair{})
assert.Equal(t, int64(2), peeKey2)
}

View File

@ -1,316 +0,0 @@
// 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 indexcoord
import (
"context"
"errors"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/commonpbutil"
"github.com/milvus-io/milvus/internal/util/paramtable"
)
type task interface {
Ctx() context.Context
ID() UniqueID // return ReqID
SetID(uid UniqueID) // set ReqID
Name() string // set task name
PreExecute(ctx context.Context) error
Execute(ctx context.Context) error
PostExecute(ctx context.Context) error
WaitToFinish() error
Notify(err error) // notify if task is terminated
OnEnqueue() error
}
// BaseTask is an basic instance of task.
type BaseTask struct {
done chan error
ctx context.Context
id UniqueID
table *metaTable
}
// ID returns the id of index task.
func (bt *BaseTask) ID() UniqueID {
return bt.id
}
func (bt *BaseTask) setID(id UniqueID) {
bt.id = id
}
// WaitToFinish will wait for the task to complete, if the context is done,
// it means that the execution of the task has timed out.
func (bt *BaseTask) WaitToFinish() error {
select {
case <-bt.ctx.Done():
return errors.New("Task wait to finished timeout")
case err := <-bt.done:
return err
}
}
// Notify will notify WaitToFinish that the task is completed or failed.
func (bt *BaseTask) Notify(err error) {
bt.done <- err
}
// CreateIndexTask is used to create an index on field.
type CreateIndexTask struct {
BaseTask
dataCoordClient types.DataCoord
rootCoordClient types.RootCoord
indexCoordClient *IndexCoord
req *indexpb.CreateIndexRequest
indexID UniqueID
}
// Ctx returns the context of the index task.
func (cit *CreateIndexTask) Ctx() context.Context {
return cit.ctx
}
// ID returns the id of the index task.
func (cit *CreateIndexTask) ID() UniqueID {
return cit.id
}
// SetID sets the id for index tasks.
func (cit *CreateIndexTask) SetID(ID UniqueID) {
cit.BaseTask.setID(ID)
}
// Name returns the task name.
func (cit *CreateIndexTask) Name() string {
return CreateIndexTaskName
}
// OnEnqueue assigns the indexBuildID to index task.
func (cit *CreateIndexTask) OnEnqueue() error {
resp, err := cit.rootCoordClient.AllocID(cit.Ctx(), &rootcoordpb.AllocIDRequest{
Count: 1,
})
if err != nil {
return err
}
if resp.Status.ErrorCode != commonpb.ErrorCode_Success {
return errors.New(resp.Status.Reason)
}
cit.indexID = resp.ID
return nil
}
// PreExecute do nothing.
func (cit *CreateIndexTask) PreExecute(ctx context.Context) error {
log.Info("IndexCoord CreateIndexTask PreExecute", zap.Int64("collectionID", cit.req.CollectionID),
zap.Int64("fieldID", cit.req.FieldID), zap.String("indexName", cit.req.IndexName))
// TODO: check index type is disk index.
if getIndexType(cit.req.GetIndexParams()) == diskAnnIndex && !cit.indexCoordClient.nodeManager.ClientSupportDisk() {
return errors.New("all IndexNodes do not support disk indexes, please verify")
}
return nil
}
func (cit *CreateIndexTask) createIndexAtomic(index *model.Index, segmentsInfo []*datapb.SegmentInfo) ([]UniqueID, []*datapb.SegmentInfo, error) {
buildIDs := make([]UniqueID, 0)
segments := make([]*datapb.SegmentInfo, 0)
for _, segmentInfo := range segmentsInfo {
segIdx := &model.SegmentIndex{
SegmentID: segmentInfo.ID,
CollectionID: segmentInfo.CollectionID,
PartitionID: segmentInfo.PartitionID,
NumRows: segmentInfo.NumOfRows,
IndexID: cit.indexID,
CreateTime: cit.req.GetTimestamp(),
}
have, buildID, err := cit.indexCoordClient.createIndexForSegment(segIdx)
if err != nil {
log.Error("IndexCoord create index on segment fail", zap.Int64("collectionID", cit.req.CollectionID),
zap.Int64("fieldID", cit.req.FieldID), zap.String("indexName", cit.req.IndexName),
zap.Int64("segmentID", segIdx.SegmentID), zap.Error(err))
return nil, nil, err
}
if have || buildID == 0 {
continue
}
segments = append(segments, segmentInfo)
buildIDs = append(buildIDs, buildID)
}
err := cit.table.CreateIndex(index)
if err != nil {
log.Error("IndexCoord create index fail", zap.Int64("collectionID", cit.req.CollectionID),
zap.Int64("fieldID", cit.req.FieldID), zap.String("indexName", cit.req.IndexName), zap.Error(err))
return nil, nil, err
}
return buildIDs, segments, nil
}
// Execute adds the index task to meta table.
func (cit *CreateIndexTask) Execute(ctx context.Context) error {
log.Info("IndexCoord CreateIndexTask Execute", zap.Int64("collectionID", cit.req.CollectionID),
zap.Int64("fieldID", cit.req.FieldID), zap.String("indexName", cit.req.IndexName))
hasIndex, indexID := cit.table.HasSameReq(cit.req)
if hasIndex {
cit.indexID = indexID
}
index := &model.Index{
CollectionID: cit.req.GetCollectionID(),
FieldID: cit.req.GetFieldID(),
IndexID: cit.indexID,
IndexName: cit.req.GetIndexName(),
TypeParams: cit.req.GetTypeParams(),
IndexParams: cit.req.GetIndexParams(),
CreateTime: cit.req.GetTimestamp(),
IsAutoIndex: cit.req.GetIsAutoIndex(),
UserIndexParams: cit.req.GetUserIndexParams(),
}
// lock before GetFlushedSegments,
// prevent the flush watcher watches the new flushed segment just after getting the flushed segments, and it locks firstly.
cit.indexCoordClient.indexGCLock.RLock()
defer cit.indexCoordClient.indexGCLock.RUnlock()
// Get flushed segments
flushedSegments, err := cit.dataCoordClient.GetFlushedSegments(cit.ctx, &datapb.GetFlushedSegmentsRequest{
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(0),
commonpbutil.WithMsgID(cit.indexID),
commonpbutil.WithTimeStamp(cit.req.Timestamp),
commonpbutil.WithSourceID(paramtable.GetNodeID()),
),
CollectionID: cit.req.CollectionID,
PartitionID: -1,
})
if err != nil {
log.Error("IndexCoord get flushed segments from datacoord fail", zap.Int64("collectionID", cit.req.CollectionID),
zap.Int64("fieldID", cit.req.FieldID), zap.String("indexName", cit.req.IndexName), zap.Error(err))
return err
}
log.Debug("IndexCoord get flushed segment from DataCoord success", zap.Int64("collectionID", cit.req.CollectionID),
zap.Int64s("flushed segments", flushedSegments.Segments))
segmentsInfo, err := cit.dataCoordClient.GetSegmentInfo(cit.ctx, &datapb.GetSegmentInfoRequest{
SegmentIDs: flushedSegments.Segments,
IncludeUnHealthy: true,
})
if err != nil {
log.Error("IndexCoord get segment info from DataCoord fail", zap.Int64s("segIDs", flushedSegments.Segments),
zap.Error(err))
return err
}
buildIDs, segments, err := cit.createIndexAtomic(index, segmentsInfo.GetInfos())
if err != nil {
log.Error("IndexCoord create index fail", zap.Int64("collectionID", cit.req.CollectionID),
zap.Int64("fieldID", cit.req.FieldID), zap.String("indexName", cit.req.IndexName), zap.Error(err))
return err
}
for _, buildID := range buildIDs {
cit.indexCoordClient.indexBuilder.enqueue(buildID)
}
// If the handoff is not notified here, the segment that has been loaded will not be able to replace the index
for _, segment := range segments {
cit.indexCoordClient.handoff.enqueue(segment)
}
return nil
}
// PostExecute does nothing here.
func (cit *CreateIndexTask) PostExecute(ctx context.Context) error {
log.Info("IndexCoord CreateIndexTask PostExecute", zap.Int64("collectionID", cit.req.CollectionID),
zap.Int64("fieldID", cit.req.FieldID), zap.String("indexName", cit.req.IndexName))
return nil
}
// IndexAddTask is used to record index task on segment.
type IndexAddTask struct {
BaseTask
segmentIndex *model.SegmentIndex
rootcoordClient types.RootCoord
}
// Ctx returns the context of the index task.
func (it *IndexAddTask) Ctx() context.Context {
return it.ctx
}
// ID returns the id of the index task.
func (it *IndexAddTask) ID() UniqueID {
return it.id
}
// SetID sets the id for index tasks.
func (it *IndexAddTask) SetID(ID UniqueID) {
it.BaseTask.setID(ID)
}
// Name returns the task name.
func (it *IndexAddTask) Name() string {
return IndexAddTaskName
}
// OnEnqueue assigns the indexBuildID to index task.
func (it *IndexAddTask) OnEnqueue() error {
resp, err := it.rootcoordClient.AllocID(it.Ctx(), &rootcoordpb.AllocIDRequest{
Count: 1,
})
if err != nil {
return err
}
if resp.Status.ErrorCode != commonpb.ErrorCode_Success {
return errors.New(resp.Status.Reason)
}
it.segmentIndex.BuildID = resp.ID
return nil
}
// PreExecute sets the indexBuildID to index task request.
func (it *IndexAddTask) PreExecute(ctx context.Context) error {
log.Info("IndexCoord IndexAddTask PreExecute", zap.Int64("segID", it.segmentIndex.SegmentID),
zap.Int64("IndexBuildID", it.segmentIndex.BuildID))
return nil
}
// Execute adds the index task to meta table.
func (it *IndexAddTask) Execute(ctx context.Context) error {
log.Info("IndexCoord IndexAddTask Execute", zap.Int64("segID", it.segmentIndex.SegmentID),
zap.Int64("IndexBuildID", it.segmentIndex.BuildID))
err := it.table.AddIndex(it.segmentIndex)
if err != nil {
return err
}
return nil
}
// PostExecute does nothing here.
func (it *IndexAddTask) PostExecute(ctx context.Context) error {
log.Info("IndexCoord IndexAddTask PostExecute", zap.Int64("segID", it.segmentIndex.SegmentID),
zap.Int64("IndexBuildID", it.segmentIndex.BuildID))
return nil
}

View File

@ -1,303 +0,0 @@
// 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 indexcoord
import (
"container/list"
"context"
"errors"
"sync"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/trace"
"github.com/opentracing/opentracing-go"
oplog "github.com/opentracing/opentracing-go/log"
)
// TaskQueue is a queue used to store tasks.
type TaskQueue interface {
utChan() <-chan int
utEmpty() bool
utFull() bool
addUnissuedTask(t task) error
//FrontUnissuedTask() task
PopUnissuedTask() task
AddActiveTask(t task)
PopActiveTask(tID UniqueID) task
Enqueue(t task) error
tryToRemoveUselessIndexAddTask(indexID UniqueID) []UniqueID
}
// BaseTaskQueue is a basic instance of TaskQueue.
type BaseTaskQueue struct {
unissuedTasks *list.List
activeTasks map[UniqueID]task
utLock sync.Mutex
atLock sync.Mutex
// maxTaskNum should keep still
maxTaskNum int64
utBufChan chan int // to block scheduler
sched *TaskScheduler
}
func (queue *BaseTaskQueue) utChan() <-chan int {
return queue.utBufChan
}
func (queue *BaseTaskQueue) utEmpty() bool {
return queue.unissuedTasks.Len() == 0
}
func (queue *BaseTaskQueue) utFull() bool {
return int64(queue.unissuedTasks.Len()) >= queue.maxTaskNum
}
func (queue *BaseTaskQueue) addUnissuedTask(t task) error {
queue.utLock.Lock()
defer queue.utLock.Unlock()
if queue.utFull() {
return errors.New("task queue is full")
}
queue.unissuedTasks.PushBack(t)
queue.utBufChan <- 1
return nil
}
//func (queue *BaseTaskQueue) FrontUnissuedTask() task {
// queue.utLock.Lock()
// defer queue.utLock.Unlock()
//
// if queue.unissuedTasks.Len() <= 0 {
// log.Warn("sorry, but the unissued task list is empty!")
// return nil
// }
//
// return queue.unissuedTasks.Front().Value.(task)
//}
// PopUnissuedTask pops a task from tasks queue.
func (queue *BaseTaskQueue) PopUnissuedTask() task {
queue.utLock.Lock()
defer queue.utLock.Unlock()
if queue.unissuedTasks.Len() <= 0 {
return nil
}
ft := queue.unissuedTasks.Front()
queue.unissuedTasks.Remove(ft)
return ft.Value.(task)
}
// AddActiveTask adds a task to activeTasks.
func (queue *BaseTaskQueue) AddActiveTask(t task) {
queue.atLock.Lock()
defer queue.atLock.Unlock()
tID := t.ID()
_, ok := queue.activeTasks[tID]
if ok {
log.Warn("indexcoord", zap.Int64("task with ID already in active task list!", tID))
}
queue.activeTasks[tID] = t
}
// PopActiveTask tasks out a task from activateTask and the task will be executed.
func (queue *BaseTaskQueue) PopActiveTask(tID UniqueID) task {
queue.atLock.Lock()
defer queue.atLock.Unlock()
t, ok := queue.activeTasks[tID]
if ok {
delete(queue.activeTasks, tID)
return t
}
log.Debug("indexcoord", zap.Int64("sorry, but the ID was not found in the active task list!", tID))
return nil
}
// Enqueue adds a task to TaskQueue.
func (queue *BaseTaskQueue) Enqueue(t task) error {
err := t.OnEnqueue()
if err != nil {
return err
}
return queue.addUnissuedTask(t)
}
// IndexAddTaskQueue is a task queue used to store building index tasks.
type IndexAddTaskQueue struct {
BaseTaskQueue
lock sync.Mutex
}
// Enqueue adds a building index task to IndexAddTaskQueue.
func (queue *IndexAddTaskQueue) Enqueue(t task) error {
queue.lock.Lock()
defer queue.lock.Unlock()
return queue.BaseTaskQueue.Enqueue(t)
}
// Note: tryToRemoveUselessIndexAddTask must be called by DropIndex
func (queue *IndexAddTaskQueue) tryToRemoveUselessIndexAddTask(indexID UniqueID) []UniqueID {
queue.lock.Lock()
defer queue.lock.Unlock()
var indexBuildIDs []UniqueID
var next *list.Element
for e := queue.unissuedTasks.Front(); e != nil; e = next {
next = e.Next()
indexAddTask, ok := e.Value.(*IndexAddTask)
if !ok {
continue
}
if indexAddTask.segmentIndex.IndexID == indexID {
queue.unissuedTasks.Remove(e)
indexAddTask.Notify(nil)
indexBuildIDs = append(indexBuildIDs, indexAddTask.segmentIndex.BuildID)
}
}
return indexBuildIDs
}
// NewIndexAddTaskQueue creates a new IndexAddTaskQueue.
func NewIndexAddTaskQueue(sched *TaskScheduler) *IndexAddTaskQueue {
return &IndexAddTaskQueue{
BaseTaskQueue: BaseTaskQueue{
unissuedTasks: list.New(),
activeTasks: make(map[UniqueID]task),
maxTaskNum: 1024,
utBufChan: make(chan int, 1024),
sched: sched,
},
}
}
// TaskScheduler is a scheduler of indexing tasks.
type TaskScheduler struct {
IndexAddQueue TaskQueue
rootcoordClient types.RootCoord
metaTable *metaTable
cm storage.ChunkManager
wg sync.WaitGroup
ctx context.Context
cancel context.CancelFunc
}
// NewTaskScheduler creates a new task scheduler of indexing tasks.
func NewTaskScheduler(ctx context.Context,
client types.RootCoord,
cm storage.ChunkManager,
table *metaTable) (*TaskScheduler, error) {
ctx1, cancel := context.WithCancel(ctx)
s := &TaskScheduler{
metaTable: table,
rootcoordClient: client,
cm: cm,
ctx: ctx1,
cancel: cancel,
}
s.IndexAddQueue = NewIndexAddTaskQueue(s)
return s, nil
}
func (sched *TaskScheduler) scheduleIndexAddTask() task {
return sched.IndexAddQueue.PopUnissuedTask()
}
//func (sched *TaskScheduler) scheduleIndexBuildClient() indexnode.Interface {
// return sched.IndexAddQueue.PopUnissuedTask()
//}
func (sched *TaskScheduler) processTask(t task, q TaskQueue) {
span, ctx := trace.StartSpanFromContext(t.Ctx(),
opentracing.Tags{
"Type": t.Name(),
})
defer span.Finish()
span.LogFields(oplog.String("scheduler process PreExecute", t.Name()))
err := t.PreExecute(ctx)
defer func() {
t.Notify(err)
}()
if err != nil {
trace.LogError(span, err)
return
}
span.LogFields(oplog.String("scheduler process AddActiveTask", t.Name()))
q.AddActiveTask(t)
defer func() {
span.LogFields(oplog.String("scheduler process PopActiveTask", t.Name()))
q.PopActiveTask(t.ID())
}()
span.LogFields(oplog.String("scheduler process Execute", t.Name()))
err = t.Execute(ctx)
if err != nil {
trace.LogError(span, err)
return
}
span.LogFields(oplog.String("scheduler process PostExecute", t.Name()))
err = t.PostExecute(ctx)
}
func (sched *TaskScheduler) indexAddLoop() {
defer sched.wg.Done()
for {
select {
case <-sched.ctx.Done():
return
case <-sched.IndexAddQueue.utChan():
if !sched.IndexAddQueue.utEmpty() {
t := sched.scheduleIndexAddTask()
go sched.processTask(t, sched.IndexAddQueue)
}
}
}
}
// Start stats the task scheduler of indexing tasks.
func (sched *TaskScheduler) Start() error {
sched.wg.Add(1)
go sched.indexAddLoop()
return nil
}
// Close closes the task scheduler of indexing tasks.
func (sched *TaskScheduler) Close() {
if sched.cancel != nil {
sched.cancel()
}
sched.wg.Wait()
}

View File

@ -1,51 +0,0 @@
// 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 indexcoord
type indexTaskState int32
const (
// when we receive a index task
indexTaskInit indexTaskState = iota
// we've sent index task to scheduler, and wait for building index.
indexTaskInProgress
// task done, wait to be cleaned
indexTaskDone
// index task need to retry.
indexTaskRetry
// task has been deleted.
indexTaskDeleted
// task needs to prepare segment info on IndexNode
indexTaskPrepare
)
var TaskStateNames = map[indexTaskState]string{
0: "Init",
1: "InProgress",
2: "Done",
3: "Retry",
4: "Deleted",
5: "Prepare",
}
func (x indexTaskState) String() string {
ret, ok := TaskStateNames[x]
if !ok {
return "None"
}
return ret
}

View File

@ -1,32 +0,0 @@
// 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 indexcoord
import (
"testing"
"github.com/stretchr/testify/assert"
)
func TestIndexTaskState_String(t *testing.T) {
assert.Equal(t, indexTaskInit.String(), "Init")
assert.Equal(t, indexTaskInProgress.String(), "InProgress")
assert.Equal(t, indexTaskDone.String(), "Done")
assert.Equal(t, indexTaskDeleted.String(), "Deleted")
assert.Equal(t, indexTaskPrepare.String(), "Prepare")
assert.Equal(t, indexTaskRetry.String(), "Retry")
}

View File

@ -1,144 +0,0 @@
// 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 indexcoord
import (
"errors"
"sync"
"testing"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/metastore/kv/indexcoord"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/stretchr/testify/assert"
)
func Test_createIndexAtomic(t *testing.T) {
meta := &metaTable{
catalog: &indexcoord.Catalog{Txn: &mockETCDKV{
save: func(s string, s2 string) error {
return errors.New("error")
},
}},
indexLock: sync.RWMutex{},
segmentIndexLock: sync.RWMutex{},
collectionIndexes: map[UniqueID]map[UniqueID]*model.Index{},
segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{
segID: {
indexID: &model.SegmentIndex{
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
NumRows: 1025,
IndexID: indexID,
BuildID: buildID,
NodeID: nodeID,
IndexVersion: 0,
IndexState: commonpb.IndexState_Unissued,
FailReason: "",
IsDeleted: false,
CreateTime: 0,
IndexFileKeys: nil,
IndexSize: 0,
WriteHandoff: false,
},
},
},
buildID2SegmentIndex: map[UniqueID]*model.SegmentIndex{
buildID: {
SegmentID: segID,
CollectionID: collID,
PartitionID: partID,
NumRows: 1025,
IndexID: indexID,
BuildID: buildID,
NodeID: nodeID,
IndexVersion: 0,
IndexState: commonpb.IndexState_Unissued,
FailReason: "",
IsDeleted: false,
CreateTime: 0,
IndexFileKeys: nil,
IndexSize: 0,
WriteHandoff: false,
},
},
}
ic := &IndexCoord{
metaTable: meta,
}
cit := &CreateIndexTask{
BaseTask: BaseTask{
table: meta,
},
indexCoordClient: ic,
indexID: indexID,
req: &indexpb.CreateIndexRequest{
CollectionID: collID,
FieldID: fieldID,
IndexName: indexName,
Timestamp: createTs,
},
}
index := &model.Index{
TenantID: "",
CollectionID: collID,
FieldID: fieldID,
IndexID: indexID,
IndexName: indexName,
IsDeleted: false,
CreateTime: createTs,
TypeParams: nil,
IndexParams: nil,
IsAutoIndex: false,
UserIndexParams: nil,
}
segmentsInfo := []*datapb.SegmentInfo{
{
ID: segID,
CollectionID: collID,
PartitionID: partID,
InsertChannel: "",
NumOfRows: 1025,
State: commonpb.SegmentState_Flushed,
MaxRowNum: 65535,
LastExpireTime: 0,
StartPosition: nil,
DmlPosition: nil,
Binlogs: nil,
Statslogs: nil,
Deltalogs: nil,
CreatedByCompaction: false,
CompactionFrom: nil,
DroppedAt: 0,
IsImporting: false,
IsFake: false,
XXX_NoUnkeyedLiteral: struct{}{},
XXX_unrecognized: nil,
XXX_sizecache: 0,
},
}
buildIDs, segs, err := cit.createIndexAtomic(index, segmentsInfo)
// index already exist
assert.Equal(t, 0, len(buildIDs))
assert.Equal(t, 0, len(segs))
assert.Error(t, err)
}

View File

@ -1,88 +0,0 @@
// 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 indexcoord
import (
"errors"
"fmt"
"strconv"
"strings"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/schemapb"
"github.com/milvus-io/milvus/internal/log"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/util"
)
// getDimension gets the dimension of data from building index request.
func getDimension(req *indexpb.CreateIndexRequest) (int64, error) {
for _, kvPair := range req.GetTypeParams() {
key, value := kvPair.GetKey(), kvPair.GetValue()
if key == "dim" {
dim, err := strconv.ParseInt(value, 10, 64)
if err != nil {
errMsg := "dimension is invalid"
log.Error(errMsg)
return 0, errors.New(errMsg)
}
return dim, nil
}
}
errMsg := "dimension is not in type params"
log.Error(errMsg)
return 0, errors.New(errMsg)
}
// estimateIndexSize estimates how much memory will be occupied by IndexNode when building an index.
func estimateIndexSize(dim int64, numRows int64, dataType schemapb.DataType) (uint64, error) {
if dataType == schemapb.DataType_FloatVector {
return uint64(dim) * uint64(numRows) * 4, nil
}
if dataType == schemapb.DataType_BinaryVector {
return uint64(dim) / 8 * uint64(numRows), nil
}
// TODO: optimize here.
return 0, nil
}
func parseBuildIDFromFilePath(key string) (UniqueID, error) {
ss := strings.Split(key, "/")
if strings.HasSuffix(key, "/") {
return strconv.ParseInt(ss[len(ss)-2], 10, 64)
}
return strconv.ParseInt(ss[len(ss)-1], 10, 64)
}
func buildHandoffKey(collID, partID, segID UniqueID) string {
return fmt.Sprintf("%s/%d/%d/%d", util.HandoffSegmentPrefix, collID, partID, segID)
}
func getIndexType(indexParams []*commonpb.KeyValuePair) string {
for _, param := range indexParams {
if param.Key == "index_type" {
return param.Value
}
}
return invalidIndex
}
func isFlatIndex(indexType string) bool {
return indexType == flatIndex || indexType == binFlatIndex
}

View File

@ -1,98 +0,0 @@
// 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 indexcoord
import (
"testing"
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus-proto/go-api/schemapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
)
func Test_getDimension(t *testing.T) {
req := &indexpb.CreateIndexRequest{
TypeParams: []*commonpb.KeyValuePair{
{
Key: "dim",
Value: "128",
},
},
}
dim, err := getDimension(req)
assert.Equal(t, int64(128), dim)
assert.Nil(t, err)
req2 := &indexpb.CreateIndexRequest{
TypeParams: []*commonpb.KeyValuePair{
{
Key: "dim",
Value: "one",
},
},
}
dim, err = getDimension(req2)
assert.Error(t, err)
assert.Equal(t, int64(0), dim)
req3 := &indexpb.CreateIndexRequest{
TypeParams: []*commonpb.KeyValuePair{
{
Key: "TypeParam-Key-1",
Value: "TypeParam-Value-1",
},
},
}
dim, err = getDimension(req3)
assert.Error(t, err)
assert.Equal(t, int64(0), dim)
}
func Test_estimateIndexSize(t *testing.T) {
memorySize, err := estimateIndexSize(10, 100, schemapb.DataType_FloatVector)
assert.Nil(t, err)
assert.Equal(t, uint64(4000), memorySize)
memorySize, err = estimateIndexSize(16, 100, schemapb.DataType_BinaryVector)
assert.Nil(t, err)
assert.Equal(t, uint64(200), memorySize)
memorySize, err = estimateIndexSize(10, 100, schemapb.DataType_Float)
assert.Nil(t, err)
assert.Equal(t, uint64(0), memorySize)
// assert.Error(t, err)
// assert.Equal(t, uint64(0), memorySize)
}
func Test_parseKey(t *testing.T) {
key := "test-ListObjects/1/"
buildID, err := parseBuildIDFromFilePath(key)
assert.Nil(t, err)
assert.Equal(t, int64(1), buildID)
key2 := "test-ListObjects/key1/"
_, err2 := parseBuildIDFromFilePath(key2)
assert.Error(t, err2)
}
func Test_isFlatIndex(t *testing.T) {
assert.True(t, isFlatIndex(flatIndex))
assert.True(t, isFlatIndex(binFlatIndex))
assert.False(t, isFlatIndex(diskAnnIndex))
}

View File

@ -80,7 +80,7 @@ func NewIndexNodeMock() *Mock {
return &milvuspb.ComponentStates{
State: &milvuspb.ComponentInfo{
NodeID: 1,
Role: typeutil.IndexCoordRole,
Role: typeutil.IndexNodeRole,
StateCode: commonpb.StateCode_Healthy,
},
SubcomponentStates: nil,

View File

@ -1,194 +0,0 @@
package indexcoord
import (
"context"
"fmt"
"github.com/milvus-io/milvus/internal/util"
"github.com/milvus-io/milvus/internal/log"
"github.com/golang/protobuf/proto"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/kv"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/util/typeutil"
)
type Catalog struct {
Txn kv.TxnKV
}
func BuildIndexKey(collectionID, indexID int64) string {
return fmt.Sprintf("%s/%d/%d", util.FieldIndexPrefix, collectionID, indexID)
}
func BuildSegmentIndexKey(collectionID, partitionID, segmentID, buildID int64) string {
return fmt.Sprintf("%s/%d/%d/%d/%d", util.SegmentIndexPrefix, collectionID, partitionID, segmentID, buildID)
}
func (kc *Catalog) CreateIndex(ctx context.Context, index *model.Index) error {
key := BuildIndexKey(index.CollectionID, index.IndexID)
value, err := proto.Marshal(model.MarshalIndexModel(index))
if err != nil {
return err
}
err = kc.Txn.Save(key, string(value))
if err != nil {
return err
}
return nil
}
func (kc *Catalog) ListIndexes(ctx context.Context) ([]*model.Index, error) {
_, values, err := kc.Txn.LoadWithPrefix(util.FieldIndexPrefix)
if err != nil {
log.Error("list index meta fail", zap.String("prefix", util.FieldIndexPrefix), zap.Error(err))
return nil, err
}
indexes := make([]*model.Index, 0)
for _, value := range values {
meta := &indexpb.FieldIndex{}
err = proto.Unmarshal([]byte(value), meta)
if err != nil {
log.Warn("unmarshal index info failed", zap.Error(err))
return nil, err
}
indexes = append(indexes, &model.Index{
TenantID: "",
CollectionID: meta.IndexInfo.CollectionID,
FieldID: meta.IndexInfo.FieldID,
IndexID: meta.IndexInfo.IndexID,
IndexName: meta.IndexInfo.IndexName,
IsDeleted: meta.Deleted,
CreateTime: meta.GetCreateTime(),
TypeParams: meta.IndexInfo.TypeParams,
IndexParams: meta.IndexInfo.IndexParams,
IsAutoIndex: meta.IndexInfo.IsAutoIndex,
UserIndexParams: meta.IndexInfo.UserIndexParams,
})
}
return indexes, nil
}
func (kc *Catalog) AlterIndex(ctx context.Context, index *model.Index) error {
return kc.CreateIndex(ctx, index)
}
func (kc *Catalog) AlterIndexes(ctx context.Context, indexes []*model.Index) error {
kvs := make(map[string]string)
for _, index := range indexes {
key := BuildIndexKey(index.CollectionID, index.IndexID)
value, err := proto.Marshal(model.MarshalIndexModel(index))
if err != nil {
return err
}
kvs[key] = string(value)
}
return kc.Txn.MultiSave(kvs)
}
func (kc *Catalog) DropIndex(ctx context.Context, collID typeutil.UniqueID, dropIdxID typeutil.UniqueID) error {
key := BuildIndexKey(collID, dropIdxID)
err := kc.Txn.Remove(key)
if err != nil {
log.Error("drop collection index meta fail", zap.Int64("collectionID", collID),
zap.Int64("indexID", dropIdxID), zap.Error(err))
return err
}
return nil
}
func (kc *Catalog) CreateSegmentIndex(ctx context.Context, segIdx *model.SegmentIndex) error {
key := BuildSegmentIndexKey(segIdx.CollectionID, segIdx.PartitionID, segIdx.SegmentID, segIdx.BuildID)
value, err := proto.Marshal(model.MarshalSegmentIndexModel(segIdx))
if err != nil {
return err
}
err = kc.Txn.Save(key, string(value))
if err != nil {
log.Error("failed to save segment index meta in etcd", zap.Int64("buildID", segIdx.BuildID),
zap.Int64("segmentID", segIdx.SegmentID), zap.Error(err))
return err
}
return nil
}
func (kc *Catalog) ListSegmentIndexes(ctx context.Context) ([]*model.SegmentIndex, error) {
_, values, err := kc.Txn.LoadWithPrefix(util.SegmentIndexPrefix)
if err != nil {
log.Error("list segment index meta fail", zap.String("prefix", util.SegmentIndexPrefix), zap.Error(err))
return nil, err
}
segIndexes := make([]*model.SegmentIndex, 0)
for _, value := range values {
segmentIndexInfo := &indexpb.SegmentIndex{}
err = proto.Unmarshal([]byte(value), segmentIndexInfo)
if err != nil {
log.Warn("unmarshal segment index info failed", zap.Error(err))
return segIndexes, err
}
segIndexes = append(segIndexes, &model.SegmentIndex{
SegmentID: segmentIndexInfo.SegmentID,
CollectionID: segmentIndexInfo.CollectionID,
PartitionID: segmentIndexInfo.PartitionID,
NumRows: segmentIndexInfo.NumRows,
IndexID: segmentIndexInfo.IndexID,
BuildID: segmentIndexInfo.BuildID,
NodeID: segmentIndexInfo.NodeID,
IndexVersion: segmentIndexInfo.IndexVersion,
IndexState: segmentIndexInfo.State,
FailReason: segmentIndexInfo.FailReason,
IsDeleted: segmentIndexInfo.Deleted,
CreateTime: segmentIndexInfo.CreateTime,
IndexFileKeys: segmentIndexInfo.IndexFileKeys,
IndexSize: segmentIndexInfo.SerializeSize,
WriteHandoff: segmentIndexInfo.WriteHandoff,
})
}
return segIndexes, nil
}
func (kc *Catalog) AlterSegmentIndex(ctx context.Context, segIdx *model.SegmentIndex) error {
return kc.CreateSegmentIndex(ctx, segIdx)
}
func (kc *Catalog) AlterSegmentIndexes(ctx context.Context, segIdxes []*model.SegmentIndex) error {
kvs := make(map[string]string)
for _, segIdx := range segIdxes {
key := BuildSegmentIndexKey(segIdx.CollectionID, segIdx.PartitionID, segIdx.SegmentID, segIdx.BuildID)
value, err := proto.Marshal(model.MarshalSegmentIndexModel(segIdx))
if err != nil {
return err
}
kvs[key] = string(value)
}
return kc.Txn.MultiSave(kvs)
}
func (kc *Catalog) DropSegmentIndex(ctx context.Context, collID, partID, segID, buildID typeutil.UniqueID) error {
key := BuildSegmentIndexKey(collID, partID, segID, buildID)
err := kc.Txn.Remove(key)
if err != nil {
log.Error("drop segment index meta fail", zap.Int64("buildID", buildID), zap.Error(err))
return err
}
return nil
}

View File

@ -1,415 +0,0 @@
package indexcoord
import (
"context"
"errors"
"testing"
"github.com/golang/protobuf/proto"
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus-proto/go-api/commonpb"
"github.com/milvus-io/milvus/internal/kv"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/proto/indexpb"
)
type MockedTxnKV struct {
kv.TxnKV
multiSave func(kvs map[string]string) error
save func(key, value string) error
loadWithPrefix func(key string) ([]string, []string, error)
remove func(key string) error
}
func (mc *MockedTxnKV) MultiSave(kvs map[string]string) error {
return mc.multiSave(kvs)
}
func (mc *MockedTxnKV) Save(key, value string) error {
return mc.save(key, value)
}
func (mc *MockedTxnKV) LoadWithPrefix(key string) ([]string, []string, error) {
return mc.loadWithPrefix(key)
}
func (mc *MockedTxnKV) Remove(key string) error {
return mc.remove(key)
}
func TestCatalog_CreateIndex(t *testing.T) {
t.Run("success", func(t *testing.T) {
txn := &MockedTxnKV{
save: func(key, value string) error {
return nil
},
}
catalog := &Catalog{
Txn: txn,
}
err := catalog.CreateIndex(context.Background(), &model.Index{})
assert.NoError(t, err)
})
t.Run("failed", func(t *testing.T) {
txn := &MockedTxnKV{
save: func(key, value string) error {
return errors.New("error")
},
}
catalog := &Catalog{
Txn: txn,
}
err := catalog.CreateIndex(context.Background(), &model.Index{})
assert.Error(t, err)
})
}
func TestCatalog_ListIndexes(t *testing.T) {
t.Run("success", func(t *testing.T) {
txn := &MockedTxnKV{
loadWithPrefix: func(key string) ([]string, []string, error) {
i := &indexpb.FieldIndex{
IndexInfo: &indexpb.IndexInfo{
CollectionID: 0,
FieldID: 0,
IndexName: "",
IndexID: 0,
TypeParams: nil,
IndexParams: nil,
},
Deleted: false,
CreateTime: 0,
}
v, err := proto.Marshal(i)
assert.NoError(t, err)
return []string{"1"}, []string{string(v)}, nil
},
}
catalog := &Catalog{
Txn: txn,
}
indexes, err := catalog.ListIndexes(context.Background())
assert.NoError(t, err)
assert.Equal(t, 1, len(indexes))
})
t.Run("failed", func(t *testing.T) {
txn := &MockedTxnKV{
loadWithPrefix: func(key string) ([]string, []string, error) {
return []string{}, []string{}, errors.New("error")
},
}
catalog := &Catalog{
Txn: txn,
}
_, err := catalog.ListIndexes(context.Background())
assert.Error(t, err)
})
t.Run("unmarshal failed", func(t *testing.T) {
txn := &MockedTxnKV{
loadWithPrefix: func(key string) ([]string, []string, error) {
return []string{"1"}, []string{"invalid"}, nil
},
}
catalog := &Catalog{
Txn: txn,
}
_, err := catalog.ListIndexes(context.Background())
assert.Error(t, err)
})
}
func TestCatalog_AlterIndex(t *testing.T) {
i := &model.Index{
CollectionID: 0,
FieldID: 0,
IndexID: 0,
IndexName: "",
IsDeleted: false,
CreateTime: 0,
TypeParams: nil,
IndexParams: nil,
}
t.Run("add", func(t *testing.T) {
txn := &MockedTxnKV{
save: func(key, value string) error {
return nil
},
}
catalog := &Catalog{
Txn: txn,
}
err := catalog.AlterIndex(context.Background(), i)
assert.NoError(t, err)
})
}
func TestCatalog_AlterIndexes(t *testing.T) {
i := &model.Index{
CollectionID: 0,
FieldID: 0,
IndexID: 0,
IndexName: "",
IsDeleted: false,
CreateTime: 0,
TypeParams: nil,
IndexParams: nil,
}
txn := &MockedTxnKV{
multiSave: func(kvs map[string]string) error {
return nil
},
}
catalog := &Catalog{
Txn: txn,
}
err := catalog.AlterIndexes(context.Background(), []*model.Index{i})
assert.NoError(t, err)
}
func TestCatalog_DropIndex(t *testing.T) {
t.Run("success", func(t *testing.T) {
txn := &MockedTxnKV{
remove: func(key string) error {
return nil
},
}
catalog := &Catalog{
Txn: txn,
}
err := catalog.DropIndex(context.Background(), 0, 0)
assert.NoError(t, err)
})
t.Run("failed", func(t *testing.T) {
txn := &MockedTxnKV{
remove: func(key string) error {
return errors.New("error")
},
}
catalog := &Catalog{
Txn: txn,
}
err := catalog.DropIndex(context.Background(), 0, 0)
assert.Error(t, err)
})
}
func TestCatalog_CreateSegmentIndex(t *testing.T) {
segIdx := &model.SegmentIndex{
SegmentID: 1,
CollectionID: 2,
PartitionID: 3,
NumRows: 1024,
IndexID: 4,
BuildID: 5,
NodeID: 6,
IndexState: commonpb.IndexState_Finished,
FailReason: "",
IndexVersion: 0,
IsDeleted: false,
CreateTime: 0,
IndexFileKeys: nil,
IndexSize: 0,
}
t.Run("success", func(t *testing.T) {
txn := &MockedTxnKV{
save: func(key, value string) error {
return nil
},
}
catalog := &Catalog{
Txn: txn,
}
err := catalog.CreateSegmentIndex(context.Background(), segIdx)
assert.NoError(t, err)
})
t.Run("failed", func(t *testing.T) {
txn := &MockedTxnKV{
save: func(key, value string) error {
return errors.New("error")
},
}
catalog := &Catalog{
Txn: txn,
}
err := catalog.CreateSegmentIndex(context.Background(), segIdx)
assert.Error(t, err)
})
}
func TestCatalog_ListSegmentIndexes(t *testing.T) {
t.Run("success", func(t *testing.T) {
segIdx := &indexpb.SegmentIndex{
CollectionID: 0,
PartitionID: 0,
SegmentID: 0,
NumRows: 0,
IndexID: 0,
BuildID: 0,
NodeID: 0,
IndexVersion: 0,
State: 0,
FailReason: "",
IndexFileKeys: nil,
Deleted: false,
CreateTime: 0,
SerializeSize: 0,
}
v, err := proto.Marshal(segIdx)
assert.NoError(t, err)
txn := &MockedTxnKV{
loadWithPrefix: func(key string) ([]string, []string, error) {
return []string{"key"}, []string{string(v)}, nil
},
}
catalog := &Catalog{
Txn: txn,
}
segIdxes, err := catalog.ListSegmentIndexes(context.Background())
assert.NoError(t, err)
assert.Equal(t, 1, len(segIdxes))
})
t.Run("failed", func(t *testing.T) {
txn := &MockedTxnKV{
loadWithPrefix: func(key string) ([]string, []string, error) {
return []string{}, []string{}, errors.New("error")
},
}
catalog := &Catalog{
Txn: txn,
}
_, err := catalog.ListSegmentIndexes(context.Background())
assert.Error(t, err)
})
t.Run("unmarshal failed", func(t *testing.T) {
txn := &MockedTxnKV{
loadWithPrefix: func(key string) ([]string, []string, error) {
return []string{"key"}, []string{"invalid"}, nil
},
}
catalog := &Catalog{
Txn: txn,
}
_, err := catalog.ListSegmentIndexes(context.Background())
assert.Error(t, err)
})
}
func TestCatalog_AlterSegmentIndex(t *testing.T) {
segIdx := &model.SegmentIndex{
SegmentID: 0,
CollectionID: 0,
PartitionID: 0,
NumRows: 0,
IndexID: 0,
BuildID: 0,
NodeID: 0,
IndexState: 0,
FailReason: "",
IndexVersion: 0,
IsDeleted: false,
CreateTime: 0,
IndexFileKeys: nil,
IndexSize: 0,
}
t.Run("add", func(t *testing.T) {
txn := &MockedTxnKV{
save: func(key, value string) error {
return nil
},
}
catalog := &Catalog{
Txn: txn,
}
err := catalog.AlterSegmentIndex(context.Background(), segIdx)
assert.NoError(t, err)
})
}
func TestCatalog_AlterSegmentIndexes(t *testing.T) {
segIdx := &model.SegmentIndex{
SegmentID: 0,
CollectionID: 0,
PartitionID: 0,
NumRows: 0,
IndexID: 0,
BuildID: 0,
NodeID: 0,
IndexState: 0,
FailReason: "",
IndexVersion: 0,
IsDeleted: false,
CreateTime: 0,
IndexFileKeys: nil,
IndexSize: 0,
}
t.Run("add", func(t *testing.T) {
txn := &MockedTxnKV{
multiSave: func(kvs map[string]string) error {
return nil
},
}
catalog := &Catalog{
Txn: txn,
}
err := catalog.AlterSegmentIndexes(context.Background(), []*model.SegmentIndex{segIdx})
assert.NoError(t, err)
})
}
func TestCatalog_DropSegmentIndex(t *testing.T) {
t.Run("success", func(t *testing.T) {
txn := &MockedTxnKV{
remove: func(key string) error {
return nil
},
}
catalog := &Catalog{
Txn: txn,
}
err := catalog.DropSegmentIndex(context.Background(), 0, 0, 0, 0)
assert.NoError(t, err)
})
t.Run("fail", func(t *testing.T) {
txn := &MockedTxnKV{
remove: func(key string) error {
return errors.New("error")
},
}
catalog := &Catalog{
Txn: txn,
}
err := catalog.DropSegmentIndex(context.Background(), 0, 0, 0, 0)
assert.Error(t, err)
})
}

View File

@ -140,6 +140,32 @@ var (
}, []string{statusLabelName})
*/
// IndexRequestCounter records the number of the index requests.
IndexRequestCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.DataCoordRole,
Name: "index_req_count",
Help: "number of building index requests ",
}, []string{statusLabelName})
// IndexTaskNum records the number of index tasks of each type.
IndexTaskNum = prometheus.NewGaugeVec(
prometheus.GaugeOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.DataCoordRole,
Name: "index_task_count",
Help: "number of index tasks of each type",
}, []string{collectionIDLabelName, indexTaskStatusLabelName})
// IndexNodeNum records the number of IndexNodes managed by IndexCoord.
IndexNodeNum = prometheus.NewGaugeVec(
prometheus.GaugeOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.DataCoordRole,
Name: "index_node_num",
Help: "number of IndexNodes managed by IndexCoord",
}, []string{})
)
// RegisterDataCoord registers DataCoord metrics
@ -151,4 +177,7 @@ func RegisterDataCoord(registry *prometheus.Registry) {
registry.MustRegister(DataCoordNumStoredRowsCounter)
registry.MustRegister(DataCoordConsumeDataNodeTimeTickLag)
registry.MustRegister(DataCoordStoredBinlogSize)
registry.MustRegister(IndexRequestCounter)
registry.MustRegister(IndexTaskNum)
registry.MustRegister(IndexNodeNum)
}

View File

@ -1,58 +0,0 @@
// 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 metrics
import (
"github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/prometheus/client_golang/prometheus"
)
var (
// IndexCoordIndexRequestCounter records the number of the index requests.
IndexCoordIndexRequestCounter = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.IndexCoordRole,
Name: "index_req_count",
Help: "number of building index requests ",
}, []string{statusLabelName})
// IndexCoordIndexTaskNum records the number of index tasks of each type.
IndexCoordIndexTaskNum = prometheus.NewGaugeVec(
prometheus.GaugeOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.IndexCoordRole,
Name: "index_task_count",
Help: "number of index tasks of each type",
}, []string{collectionIDLabelName, indexTaskStatusLabelName})
// IndexCoordIndexNodeNum records the number of IndexNodes managed by IndexCoord.
IndexCoordIndexNodeNum = prometheus.NewGaugeVec(
prometheus.GaugeOpts{
Namespace: milvusNamespace,
Subsystem: typeutil.IndexCoordRole,
Name: "index_node_num",
Help: "number of IndexNodes managed by IndexCoord",
}, []string{})
)
//RegisterIndexCoord registers IndexCoord metrics
func RegisterIndexCoord(registry *prometheus.Registry) {
registry.MustRegister(IndexCoordIndexRequestCounter)
registry.MustRegister(IndexCoordIndexTaskNum)
registry.MustRegister(IndexCoordIndexNodeNum)
}

View File

@ -29,7 +29,6 @@ func TestRegisterMetrics(t *testing.T) {
RegisterDataNode(r)
RegisterDataCoord(r)
RegisterIndexNode(r)
RegisterIndexCoord(r)
RegisterProxy(r)
RegisterQueryNode(r)
RegisterQueryCoord(r)

View File

@ -1,265 +0,0 @@
syntax = "proto3";
package milvus.proto.index;
option go_package = "github.com/milvus-io/milvus/internal/proto/indexpb";
import "common.proto";
import "internal.proto";
import "milvus.proto";
service IndexCoord {
rpc GetComponentStates(milvus.GetComponentStatesRequest) returns (milvus.ComponentStates) {}
rpc GetStatisticsChannel(internal.GetStatisticsChannelRequest) returns(milvus.StringResponse){}
rpc CreateIndex(CreateIndexRequest) returns (common.Status){}
// Deprecated: use DescribeIndex instead
rpc GetIndexState(GetIndexStateRequest) returns (GetIndexStateResponse) {}
rpc GetSegmentIndexState(GetSegmentIndexStateRequest) returns (GetSegmentIndexStateResponse) {}
rpc GetIndexInfos(GetIndexInfoRequest) returns (GetIndexInfoResponse){}
rpc DropIndex(DropIndexRequest) returns (common.Status) {}
rpc DescribeIndex(DescribeIndexRequest) returns (DescribeIndexResponse) {}
// Deprecated: use DescribeIndex instead
rpc GetIndexBuildProgress(GetIndexBuildProgressRequest) returns (GetIndexBuildProgressResponse) {}
rpc ShowConfigurations(internal.ShowConfigurationsRequest) returns (internal.ShowConfigurationsResponse){}
// https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy
rpc GetMetrics(milvus.GetMetricsRequest) returns (milvus.GetMetricsResponse) {}
rpc CheckHealth(milvus.CheckHealthRequest) returns (milvus.CheckHealthResponse) {}
}
service IndexNode {
rpc GetComponentStates(milvus.GetComponentStatesRequest) returns (milvus.ComponentStates) {}
rpc GetStatisticsChannel(internal.GetStatisticsChannelRequest) returns(milvus.StringResponse){}
rpc CreateJob(CreateJobRequest) returns (common.Status) {}
rpc QueryJobs(QueryJobsRequest) returns (QueryJobsResponse) {}
rpc DropJobs(DropJobsRequest) returns (common.Status) {}
rpc GetJobStats(GetJobStatsRequest) returns (GetJobStatsResponse) {}
rpc ShowConfigurations(internal.ShowConfigurationsRequest) returns (internal.ShowConfigurationsResponse){}
// https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy
rpc GetMetrics(milvus.GetMetricsRequest) returns (milvus.GetMetricsResponse) {}
}
message IndexInfo {
int64 collectionID = 1;
int64 fieldID = 2;
string index_name = 3;
int64 indexID = 4;
repeated common.KeyValuePair type_params = 5;
repeated common.KeyValuePair index_params = 6;
// index build progress
// The real-time statistics may not be expected due to the existence of the compaction mechanism.
int64 indexed_rows = 7;
int64 total_rows = 8;
// index state
common.IndexState state = 9;
string index_state_fail_reason = 10;
bool is_auto_index = 11;
repeated common.KeyValuePair user_index_params = 12;
}
message FieldIndex {
IndexInfo index_info = 1;
bool deleted = 2;
uint64 create_time = 3;
}
message SegmentIndex {
int64 collectionID = 1;
int64 partitionID = 2;
int64 segmentID = 3;
int64 num_rows = 4;
int64 indexID = 5;
int64 buildID = 6;
int64 nodeID = 7;
int64 index_version = 8;
common.IndexState state = 9;
string fail_reason = 10;
repeated string index_file_keys = 11;
bool deleted = 12;
uint64 create_time = 13;
uint64 serialize_size = 14;
bool write_handoff = 15;
}
message RegisterNodeRequest {
common.MsgBase base = 1;
common.Address address = 2;
int64 nodeID = 3;
}
message RegisterNodeResponse {
common.Status status = 1;
internal.InitParams init_params = 2;
}
message GetIndexStateRequest {
int64 collectionID = 1;
string index_name = 2;
}
message GetIndexStateResponse {
common.Status status = 1;
common.IndexState state = 2;
string fail_reason = 3;
}
message GetSegmentIndexStateRequest {
int64 collectionID = 1;
string index_name = 2;
repeated int64 segmentIDs = 3;
}
message SegmentIndexState {
int64 segmentID = 1;
common.IndexState state = 2;
string fail_reason = 3;
}
message GetSegmentIndexStateResponse {
common.Status status = 1;
repeated SegmentIndexState states = 2;
}
message CreateIndexRequest {
int64 collectionID = 1;
int64 fieldID = 2;
string index_name = 3;
repeated common.KeyValuePair type_params = 4;
repeated common.KeyValuePair index_params = 5;
uint64 timestamp = 6;
bool is_auto_index = 7;
repeated common.KeyValuePair user_index_params = 8;
}
message GetIndexInfoRequest {
int64 collectionID = 1;
repeated int64 segmentIDs = 2;
string index_name = 3;
}
message IndexFilePathInfo {
int64 segmentID = 1;
int64 fieldID = 2;
int64 indexID = 3;
int64 buildID = 4;
string index_name = 5;
repeated common.KeyValuePair index_params = 6;
repeated string index_file_paths = 7;
uint64 serialized_size = 8;
int64 index_version = 9;
int64 num_rows = 10;
}
message SegmentInfo {
int64 collectionID = 1;
int64 segmentID = 2;
bool enable_index = 3;
repeated IndexFilePathInfo index_infos = 4;
}
message GetIndexInfoResponse {
common.Status status = 1;
map<int64, SegmentInfo> segment_info = 2;
}
message DropIndexRequest {
int64 collectionID = 1;
repeated int64 partitionIDs = 2;
string index_name = 3;
bool drop_all = 4;
}
message DescribeIndexRequest {
int64 collectionID = 1;
string index_name = 2;
}
message DescribeIndexResponse {
common.Status status = 1;
repeated IndexInfo index_infos = 2;
}
message GetIndexBuildProgressRequest {
int64 collectionID = 1;
string index_name = 2;
}
message GetIndexBuildProgressResponse {
common.Status status = 1;
int64 indexed_rows = 2;
int64 total_rows = 3;
}
message StorageConfig {
string address = 1;
string access_keyID = 2;
string secret_access_key = 3;
bool useSSL = 4;
string bucket_name = 5;
string root_path = 6;
bool useIAM = 7;
string IAMEndpoint = 8;
string storage_type = 9;
}
message CreateJobRequest {
string clusterID = 1;
string index_file_prefix = 2;
int64 buildID = 3;
repeated string data_paths = 4;
int64 index_version = 5;
int64 indexID = 6;
string index_name = 7;
StorageConfig storage_config = 8;
repeated common.KeyValuePair index_params = 9;
repeated common.KeyValuePair type_params = 10;
int64 num_rows = 11;
}
message QueryJobsRequest {
string clusterID = 1;
repeated int64 buildIDs = 2;
}
message IndexTaskInfo {
int64 buildID = 1;
common.IndexState state = 2;
repeated string index_file_keys = 3;
uint64 serialized_size = 4;
string fail_reason = 5;
}
message QueryJobsResponse {
common.Status status = 1;
string clusterID = 2;
repeated IndexTaskInfo index_infos = 3;
}
message DropJobsRequest {
string clusterID = 1;
repeated int64 buildIDs = 2;
}
message JobInfo {
int64 num_rows = 1;
int64 dim = 2;
int64 start_time = 3;
int64 end_time = 4;
repeated common.KeyValuePair index_params = 5;
int64 podID = 6;
}
message GetJobStatsRequest {
}
message GetJobStatsResponse {
common.Status status = 1;
int64 total_job_num = 2;
int64 in_progress_job_num = 3;
int64 enqueue_job_num = 4;
int64 task_slots = 5;
repeated JobInfo job_infos = 6;
bool enable_disk = 7;
}

View File

@ -0,0 +1,94 @@
syntax = "proto3";
package milvus.proto.index;
option go_package = "github.com/milvus-io/milvus/internal/proto/indexpb";
import "common.proto";
import "internal.proto";
import "milvus.proto";
service IndexNode {
rpc GetComponentStates(milvus.GetComponentStatesRequest) returns (milvus.ComponentStates) {}
rpc GetStatisticsChannel(internal.GetStatisticsChannelRequest) returns(milvus.StringResponse){}
rpc CreateJob(CreateJobRequest) returns (common.Status) {}
rpc QueryJobs(QueryJobsRequest) returns (QueryJobsResponse) {}
rpc DropJobs(DropJobsRequest) returns (common.Status) {}
rpc GetJobStats(GetJobStatsRequest) returns (GetJobStatsResponse) {}
rpc ShowConfigurations(internal.ShowConfigurationsRequest) returns (internal.ShowConfigurationsResponse){}
// https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy
rpc GetMetrics(milvus.GetMetricsRequest) returns (milvus.GetMetricsResponse) {}
}
message StorageConfig {
string address = 1;
string access_keyID = 2;
string secret_access_key = 3;
bool useSSL = 4;
string bucket_name = 5;
string root_path = 6;
bool useIAM = 7;
string IAMEndpoint = 8;
string storage_type = 9;
}
message CreateJobRequest {
string clusterID = 1;
string index_file_prefix = 2;
int64 buildID = 3;
repeated string data_paths = 4;
int64 index_version = 5;
int64 indexID = 6;
string index_name = 7;
StorageConfig storage_config = 8;
repeated common.KeyValuePair index_params = 9;
repeated common.KeyValuePair type_params = 10;
int64 num_rows = 11;
}
message QueryJobsRequest {
string clusterID = 1;
repeated int64 buildIDs = 2;
}
message IndexTaskInfo {
int64 buildID = 1;
common.IndexState state = 2;
repeated string index_file_keys = 3;
uint64 serialized_size = 4;
string fail_reason = 5;
}
message QueryJobsResponse {
common.Status status = 1;
string clusterID = 2;
repeated IndexTaskInfo index_infos = 3;
}
message DropJobsRequest {
string clusterID = 1;
repeated int64 buildIDs = 2;
}
message JobInfo {
int64 num_rows = 1;
int64 dim = 2;
int64 start_time = 3;
int64 end_time = 4;
repeated common.KeyValuePair index_params = 5;
int64 podID = 6;
}
message GetJobStatsRequest {
}
message GetJobStatsResponse {
common.Status status = 1;
int64 total_job_num = 2;
int64 in_progress_job_num = 3;
int64 enqueue_job_num = 4;
int64 task_slots = 5;
repeated JobInfo job_infos = 6;
bool enable_disk = 7;
}

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

View File

@ -173,11 +173,6 @@ func getSystemInfoMetrics(
var dataCoordTopology metricsinfo.DataCoordTopology
dataCoordRoleName := ""
var indexCoordResp *milvuspb.GetMetricsResponse
var indexCoordErr error
var indexCoordTopology metricsinfo.IndexCoordTopology
indexCoordRoleName := ""
var rootCoordResp *milvuspb.GetMetricsResponse
var rootCoordErr error
var rootCoordTopology metricsinfo.RootCoordTopology
@ -214,7 +209,6 @@ func getSystemInfoMetrics(
identifierMap[queryCoordRoleName] = int(queryCoordTopology.Cluster.Self.ID)
identifierMap[dataCoordRoleName] = int(dataCoordTopology.Cluster.Self.ID)
identifierMap[indexCoordRoleName] = int(indexCoordTopology.Cluster.Self.ID)
identifierMap[rootCoordRoleName] = int(rootCoordTopology.Self.ID)
if queryCoordErr == nil && queryCoordResp != nil {
@ -250,14 +244,6 @@ func getSystemInfoMetrics(
TargetType: typeutil.DataCoordRole,
})
}
case typeutil.IndexCoordRole:
if indexCoordErr == nil && indexCoordResp != nil {
queryCoordTopologyNode.Connected = append(queryCoordTopologyNode.Connected, metricsinfo.ConnectionEdge{
ConnectedIdentifier: identifierMap[indexCoordRoleName],
Type: metricsinfo.Forward,
TargetType: typeutil.IndexCoordRole,
})
}
case typeutil.QueryCoordRole:
queryCoordTopologyNode.Connected = append(queryCoordTopologyNode.Connected, metricsinfo.ConnectionEdge{
ConnectedIdentifier: identifierMap[queryCoordRoleName],
@ -320,14 +306,6 @@ func getSystemInfoMetrics(
Type: metricsinfo.Forward,
TargetType: typeutil.DataCoordRole,
})
case typeutil.IndexCoordRole:
if indexCoordErr == nil && indexCoordResp != nil {
dataCoordTopologyNode.Connected = append(dataCoordTopologyNode.Connected, metricsinfo.ConnectionEdge{
ConnectedIdentifier: identifierMap[indexCoordRoleName],
Type: metricsinfo.Forward,
TargetType: typeutil.IndexCoordRole,
})
}
case typeutil.QueryCoordRole:
if queryCoordErr == nil && queryCoordResp != nil {
dataCoordTopologyNode.Connected = append(dataCoordTopologyNode.Connected, metricsinfo.ConnectionEdge{
@ -340,7 +318,7 @@ func getSystemInfoMetrics(
}
// add data nodes to system topology graph
for _, dataNode := range dataCoordTopology.Cluster.ConnectedNodes {
for _, dataNode := range dataCoordTopology.Cluster.ConnectedDataNodes {
node := dataNode
identifier := int(node.ID)
identifierMap[dataNode.Name] = identifier
@ -357,62 +335,8 @@ func getSystemInfoMetrics(
})
}
// add DataCoord to system topology graph
systemTopology.NodesInfo = append(systemTopology.NodesInfo, dataCoordTopologyNode)
}
if indexCoordErr == nil && indexCoordResp != nil {
proxyTopologyNode.Connected = append(proxyTopologyNode.Connected, metricsinfo.ConnectionEdge{
ConnectedIdentifier: identifierMap[indexCoordRoleName],
Type: metricsinfo.Forward,
TargetType: typeutil.IndexCoordRole,
})
// index coord in system topology graph
indexCoordTopologyNode := metricsinfo.SystemTopologyNode{
Identifier: identifierMap[indexCoordRoleName],
Connected: make([]metricsinfo.ConnectionEdge, 0),
Infos: &indexCoordTopology.Cluster.Self,
}
// fill connection edge, a little trick here
for _, edge := range indexCoordTopology.Connections.ConnectedComponents {
switch edge.TargetType {
case typeutil.RootCoordRole:
if rootCoordErr == nil && rootCoordResp != nil {
indexCoordTopologyNode.Connected = append(indexCoordTopologyNode.Connected, metricsinfo.ConnectionEdge{
ConnectedIdentifier: identifierMap[rootCoordRoleName],
Type: metricsinfo.Forward,
TargetType: typeutil.RootCoordRole,
})
}
case typeutil.DataCoordRole:
if dataCoordErr == nil && dataCoordResp != nil {
indexCoordTopologyNode.Connected = append(indexCoordTopologyNode.Connected, metricsinfo.ConnectionEdge{
ConnectedIdentifier: identifierMap[dataCoordRoleName],
Type: metricsinfo.Forward,
TargetType: typeutil.DataCoordRole,
})
}
case typeutil.IndexCoordRole:
indexCoordTopologyNode.Connected = append(indexCoordTopologyNode.Connected, metricsinfo.ConnectionEdge{
ConnectedIdentifier: identifierMap[indexCoordRoleName],
Type: metricsinfo.Forward,
TargetType: typeutil.IndexCoordRole,
})
case typeutil.QueryCoordRole:
if queryCoordErr == nil && queryCoordResp != nil {
indexCoordTopologyNode.Connected = append(indexCoordTopologyNode.Connected, metricsinfo.ConnectionEdge{
ConnectedIdentifier: identifierMap[queryCoordRoleName],
Type: metricsinfo.Forward,
TargetType: typeutil.QueryCoordRole,
})
}
}
}
// add index nodes to system topology graph
for _, indexNode := range indexCoordTopology.Cluster.ConnectedNodes {
// add data nodes to system topology graph
for _, indexNode := range dataCoordTopology.Cluster.ConnectedIndexNodes {
node := indexNode
identifier := int(node.ID)
identifierMap[indexNode.Name] = identifier
@ -422,15 +346,15 @@ func getSystemInfoMetrics(
Infos: &node,
}
systemTopology.NodesInfo = append(systemTopology.NodesInfo, indexNodeTopologyNode)
indexCoordTopologyNode.Connected = append(indexCoordTopologyNode.Connected, metricsinfo.ConnectionEdge{
dataCoordTopologyNode.Connected = append(dataCoordTopologyNode.Connected, metricsinfo.ConnectionEdge{
ConnectedIdentifier: identifier,
Type: metricsinfo.CoordConnectToNode,
TargetType: typeutil.IndexNodeRole,
})
}
// add index coord to system topology graph
systemTopology.NodesInfo = append(systemTopology.NodesInfo, indexCoordTopologyNode)
// add DataCoord to system topology graph
systemTopology.NodesInfo = append(systemTopology.NodesInfo, dataCoordTopologyNode)
}
if rootCoordErr == nil && rootCoordResp != nil {
@ -464,12 +388,6 @@ func getSystemInfoMetrics(
TargetType: typeutil.DataCoordRole,
})
}
case typeutil.IndexCoordRole:
rootCoordTopologyNode.Connected = append(rootCoordTopologyNode.Connected, metricsinfo.ConnectionEdge{
ConnectedIdentifier: identifierMap[indexCoordRoleName],
Type: metricsinfo.Forward,
TargetType: typeutil.IndexCoordRole,
})
case typeutil.QueryCoordRole:
if queryCoordErr == nil && queryCoordResp != nil {
rootCoordTopologyNode.Connected = append(rootCoordTopologyNode.Connected, metricsinfo.ConnectionEdge{

View File

@ -75,10 +75,6 @@ func TestProxy_metrics(t *testing.T) {
Connections: metricsinfo.ConnTopology{
Name: metricsinfo.ConstructComponentName(typeutil.RootCoordRole, id),
ConnectedComponents: []metricsinfo.ConnectionInfo{
{
TargetName: metricsinfo.ConstructComponentName(typeutil.IndexCoordRole, id),
TargetType: typeutil.IndexCoordRole,
},
{
TargetName: metricsinfo.ConstructComponentName(typeutil.QueryCoordRole, id),
TargetType: typeutil.QueryCoordRole,
@ -139,10 +135,6 @@ func TestProxy_metrics(t *testing.T) {
TargetName: metricsinfo.ConstructComponentName(typeutil.DataCoordRole, id),
TargetType: typeutil.DataCoordRole,
},
{
TargetName: metricsinfo.ConstructComponentName(typeutil.IndexCoordRole, id),
TargetType: typeutil.IndexCoordRole,
},
},
},
}
@ -173,14 +165,21 @@ func TestProxy_metrics(t *testing.T) {
},
SystemConfigurations: metricsinfo.DataCoordConfiguration{},
},
ConnectedNodes: make([]metricsinfo.DataNodeInfos, 0),
ConnectedDataNodes: make([]metricsinfo.DataNodeInfos, 0),
ConnectedIndexNodes: make([]metricsinfo.IndexNodeInfos, 0),
}
infos := metricsinfo.DataNodeInfos{
BaseComponentInfos: metricsinfo.BaseComponentInfos{},
SystemConfigurations: metricsinfo.DataNodeConfiguration{},
}
clusterTopology.ConnectedNodes = append(clusterTopology.ConnectedNodes, infos)
clusterTopology.ConnectedDataNodes = append(clusterTopology.ConnectedDataNodes, infos)
indexNodeInfos := metricsinfo.IndexNodeInfos{
BaseComponentInfos: metricsinfo.BaseComponentInfos{},
SystemConfigurations: metricsinfo.IndexNodeConfiguration{},
}
clusterTopology.ConnectedIndexNodes = append(clusterTopology.ConnectedIndexNodes, indexNodeInfos)
coordTopology := metricsinfo.DataCoordTopology{
Cluster: clusterTopology,
@ -195,10 +194,6 @@ func TestProxy_metrics(t *testing.T) {
TargetName: metricsinfo.ConstructComponentName(typeutil.QueryCoordRole, id),
TargetType: typeutil.QueryCoordRole,
},
{
TargetName: metricsinfo.ConstructComponentName(typeutil.IndexCoordRole, id),
TargetType: typeutil.IndexCoordRole,
},
},
},
}

View File

@ -49,19 +49,16 @@ type Broker interface {
}
type CoordinatorBroker struct {
dataCoord types.DataCoord
rootCoord types.RootCoord
indexCoord types.IndexCoord
dataCoord types.DataCoord
rootCoord types.RootCoord
}
func NewCoordinatorBroker(
dataCoord types.DataCoord,
rootCoord types.RootCoord,
indexCoord types.IndexCoord) *CoordinatorBroker {
rootCoord types.RootCoord) *CoordinatorBroker {
return &CoordinatorBroker{
dataCoord,
rootCoord,
indexCoord,
}
}

View File

@ -76,9 +76,8 @@ type Server struct {
metricsCacheManager *metricsinfo.MetricsCacheManager
// Coordinators
dataCoord types.DataCoord
rootCoord types.RootCoord
indexCoord types.IndexCoord
dataCoord types.DataCoord
rootCoord types.RootCoord
// Meta
store meta.Store
@ -270,7 +269,6 @@ func (s *Server) initMeta() error {
s.broker = meta.NewCoordinatorBroker(
s.dataCoord,
s.rootCoord,
s.indexCoord,
)
s.targetMgr = meta.NewTargetManager(s.broker, s.meta)
@ -482,16 +480,6 @@ func (s *Server) SetDataCoord(dataCoord types.DataCoord) error {
return nil
}
// SetIndexCoord sets index coordinator's client
func (s *Server) SetIndexCoord(indexCoord types.IndexCoord) error {
if indexCoord == nil {
return errors.New("null IndexCoord interface")
}
s.indexCoord = indexCoord
return nil
}
func (s *Server) recover() error {
// Recover target managers
group, ctx := errgroup.WithContext(s.ctx)

View File

@ -229,7 +229,7 @@ func (q *QuotaCenter) syncMetrics() error {
if err != nil {
return err
}
for _, dataNodeMetric := range dataCoordTopology.Cluster.ConnectedNodes {
for _, dataNodeMetric := range dataCoordTopology.Cluster.ConnectedDataNodes {
if dataNodeMetric.QuotaMetrics != nil {
q.dataNodeMetrics[dataNodeMetric.ID] = dataNodeMetric.QuotaMetrics
}

View File

@ -415,70 +415,6 @@ type IndexNodeComponent interface {
UpdateStateCode(stateCode commonpb.StateCode)
}
// IndexCoord is the interface `indexcoord` package implements
type IndexCoord interface {
Component
//TimeTickProvider
// CreateIndex create an index on collection.
// Index building is asynchronous, so when an index building request comes, an IndexID is assigned to the task and
// will get all flushed segments from DataCoord and record tasks with these segments. The background process
// indexBuilder will find this task and assign it to IndexNode for execution.
CreateIndex(ctx context.Context, req *indexpb.CreateIndexRequest) (*commonpb.Status, error)
// GetIndexState gets the index state of the index name in the request from Proxy.
// Deprecated: use DescribeIndex instead
GetIndexState(ctx context.Context, req *indexpb.GetIndexStateRequest) (*indexpb.GetIndexStateResponse, error)
// GetSegmentIndexState gets the index state of the segments in the request from RootCoord.
GetSegmentIndexState(ctx context.Context, req *indexpb.GetSegmentIndexStateRequest) (*indexpb.GetSegmentIndexStateResponse, error)
// GetIndexInfos gets the index files of the IndexBuildIDs in the request from RootCoordinator.
GetIndexInfos(ctx context.Context, req *indexpb.GetIndexInfoRequest) (*indexpb.GetIndexInfoResponse, error)
// DescribeIndex describe the index info of the collection.
DescribeIndex(ctx context.Context, req *indexpb.DescribeIndexRequest) (*indexpb.DescribeIndexResponse, error)
// GetIndexBuildProgress get the index building progress by num rows.
// Deprecated: use DescribeIndex instead
GetIndexBuildProgress(ctx context.Context, req *indexpb.GetIndexBuildProgressRequest) (*indexpb.GetIndexBuildProgressResponse, error)
// DropIndex deletes indexes based on IndexID. One IndexID corresponds to the index of an entire column. A column is
// divided into many segments, and each segment corresponds to an IndexBuildID. IndexCoord uses IndexBuildID to record
// index tasks. Therefore, when DropIndex is called, delete all tasks corresponding to IndexBuildID corresponding to IndexID.
DropIndex(ctx context.Context, req *indexpb.DropIndexRequest) (*commonpb.Status, error)
//// GetIndexStates gets the index states of the IndexBuildIDs in the request from RootCoordinator.
//GetIndexStates(ctx context.Context, req *indexpb.GetIndexStatesRequest) (*indexpb.GetIndexStatesResponse, error)
//
//// GetIndexFilePaths gets the index files of the IndexBuildIDs in the request from RootCoordinator.
//GetIndexFilePaths(ctx context.Context, req *indexpb.GetIndexFilePathsRequest) (*indexpb.GetIndexFilePathsResponse, error)
// ShowConfigurations gets specified configurations para of IndexCoord
ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error)
// GetMetrics gets the metrics about IndexCoord.
GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error)
CheckHealth(ctx context.Context, req *milvuspb.CheckHealthRequest) (*milvuspb.CheckHealthResponse, error)
}
// IndexCoordComponent is used by grpc server of IndexCoord
type IndexCoordComponent interface {
IndexCoord
SetAddress(address string)
// SetEtcdClient set etcd client for IndexCoordComponent
SetEtcdClient(etcdClient *clientv3.Client)
SetDataCoord(dataCoord DataCoord) error
SetRootCoord(rootCoord RootCoord) error
// UpdateStateCode updates state code for IndexCoordComponent
// `stateCode` is current statement of this IndexCoordComponent, indicating whether it's healthy.
UpdateStateCode(stateCode commonpb.StateCode)
}
// RootCoord is the interface `rootcoord` package implements
type RootCoord interface {
Component
@ -1449,13 +1385,4 @@ type QueryCoordComponent interface {
// Return nil in status:
// The rootCoord is not nil.
SetRootCoord(rootCoord RootCoord) error
// SetIndexCoord set IndexCoord for QueryCoord
// `IndexCoord` is a client of index coordinator.
//
// Return a generic error in status:
// If the indexCoord is nil.
// Return nil in status:
// The indexCoord is not nil.
SetIndexCoord(indexCoord IndexCoord) error
}

View File

@ -185,47 +185,6 @@ func TestIndexNodeInfos_Codec(t *testing.T) {
assert.Equal(t, infos1, infos2)
}
func TestIndexCoordInfos_Codec(t *testing.T) {
infos1 := IndexCoordInfos{
BaseComponentInfos: BaseComponentInfos{
HasError: false,
ErrorReason: "",
Name: ConstructComponentName(typeutil.IndexCoordRole, 1),
HardwareInfos: HardwareMetrics{
IP: "193.168.1.2",
CPUCoreCount: 4,
CPUCoreUsage: 0.5,
Memory: 32 * 1024,
MemoryUsage: 4 * 1024,
Disk: 100 * 1024,
DiskUsage: 2 * 1024,
},
SystemInfo: DeployMetrics{
SystemVersion: "8b1ae98fa97ce1c7ba853e8b9ff1c7ce24458dc1",
DeployMode: ClusterDeployMode,
BuildVersion: "2.0.0-rc8",
BuildTime: "2021-11-24, 11:37:25",
UsedGoVersion: "go version go1.16.9 linux/amd64",
},
CreatedTime: time.Now().String(),
UpdatedTime: time.Now().String(),
Type: typeutil.IndexCoordRole,
ID: 1,
},
SystemConfigurations: IndexCoordConfiguration{
MinioBucketName: "a-bucket",
},
}
s, err := MarshalComponentInfos(infos1)
assert.Equal(t, nil, err)
log.Info("TestIndexCoordInfos_Codec",
zap.String("marshaled_result", s))
var infos2 IndexCoordInfos
err = UnmarshalComponentInfos(s, &infos2)
assert.Equal(t, nil, err)
assert.Equal(t, infos1, infos2)
}
func TestDataNodeInfos_Codec(t *testing.T) {
infos1 := DataNodeInfos{
BaseComponentInfos: BaseComponentInfos{

View File

@ -96,8 +96,9 @@ type IndexCoordTopology struct {
// DataClusterTopology shows the topology between DataCoord and DataNodes
type DataClusterTopology struct {
Self DataCoordInfos `json:"self"`
ConnectedNodes []DataNodeInfos `json:"connected_nodes"`
Self DataCoordInfos `json:"self"`
ConnectedDataNodes []DataNodeInfos `json:"connected_data_nodes"`
ConnectedIndexNodes []IndexNodeInfos `json:"connected_index_nodes"`
}
// DataCoordTopology shows the whole metrics of index cluster

View File

@ -127,95 +127,6 @@ func TestQueryCoordTopology_Codec(t *testing.T) {
}
}
func TestIndexClusterTopology_Codec(t *testing.T) {
topology1 := IndexClusterTopology{
Self: IndexCoordInfos{
BaseComponentInfos: BaseComponentInfos{
Name: ConstructComponentName(typeutil.IndexCoordRole, 1),
ID: 1,
},
},
ConnectedNodes: []IndexNodeInfos{
{
BaseComponentInfos: BaseComponentInfos{
Name: ConstructComponentName(typeutil.IndexNodeRole, 2),
ID: 2,
},
},
{
BaseComponentInfos: BaseComponentInfos{
Name: ConstructComponentName(typeutil.IndexNodeRole, 3),
ID: 3,
},
},
},
}
s, err := MarshalTopology(topology1)
assert.Equal(t, nil, err)
log.Info("TestIndexClusterTopology_Codec",
zap.String("marshaled_result", s))
var topology2 IndexClusterTopology
err = UnmarshalTopology(s, &topology2)
assert.Equal(t, nil, err)
assert.Equal(t, topology1.Self, topology2.Self)
assert.Equal(t, len(topology1.ConnectedNodes), len(topology2.ConnectedNodes))
for i := range topology1.ConnectedNodes {
assert.Equal(t, topology1.ConnectedNodes[i], topology2.ConnectedNodes[i])
}
}
func TestIndexCoordTopology_Codec(t *testing.T) {
topology1 := IndexCoordTopology{
Cluster: IndexClusterTopology{
Self: IndexCoordInfos{
BaseComponentInfos: BaseComponentInfos{
Name: ConstructComponentName(typeutil.IndexCoordRole, 1),
ID: 1,
},
},
ConnectedNodes: []IndexNodeInfos{
{
BaseComponentInfos: BaseComponentInfos{
Name: ConstructComponentName(typeutil.IndexNodeRole, 2),
ID: 2,
},
},
{
BaseComponentInfos: BaseComponentInfos{
Name: ConstructComponentName(typeutil.IndexNodeRole, 3),
ID: 3,
},
},
},
},
Connections: ConnTopology{
Name: ConstructComponentName(typeutil.IndexCoordRole, 1),
ConnectedComponents: []ConnectionInfo{
{
TargetType: typeutil.RootCoordRole,
},
},
},
}
s, err := MarshalTopology(topology1)
assert.Equal(t, nil, err)
log.Info("TestIndexCoordTopology_Codec",
zap.String("marshaled_result", s))
var topology2 IndexCoordTopology
err = UnmarshalTopology(s, &topology2)
assert.Equal(t, nil, err)
assert.Equal(t, topology1.Cluster.Self, topology2.Cluster.Self)
assert.Equal(t, len(topology1.Cluster.ConnectedNodes), len(topology2.Cluster.ConnectedNodes))
for i := range topology1.Cluster.ConnectedNodes {
assert.Equal(t, topology1.Cluster.ConnectedNodes[i], topology2.Cluster.ConnectedNodes[i])
}
assert.Equal(t, topology1.Connections.Name, topology2.Connections.Name)
assert.Equal(t, len(topology1.Connections.ConnectedComponents), len(topology1.Connections.ConnectedComponents))
for i := range topology1.Connections.ConnectedComponents {
assert.Equal(t, topology1.Connections.ConnectedComponents[i], topology2.Connections.ConnectedComponents[i])
}
}
func TestDataClusterTopology_Codec(t *testing.T) {
topology1 := DataClusterTopology{
Self: DataCoordInfos{
@ -224,7 +135,7 @@ func TestDataClusterTopology_Codec(t *testing.T) {
ID: 1,
},
},
ConnectedNodes: []DataNodeInfos{
ConnectedDataNodes: []DataNodeInfos{
{
BaseComponentInfos: BaseComponentInfos{
Name: ConstructComponentName(typeutil.DataNodeRole, 2),
@ -238,6 +149,20 @@ func TestDataClusterTopology_Codec(t *testing.T) {
},
},
},
ConnectedIndexNodes: []IndexNodeInfos{
{
BaseComponentInfos: BaseComponentInfos{
Name: ConstructComponentName(typeutil.IndexNodeRole, 4),
ID: 4,
},
},
{
BaseComponentInfos: BaseComponentInfos{
Name: ConstructComponentName(typeutil.IndexNodeRole, 5),
ID: 5,
},
},
},
}
s, err := MarshalTopology(topology1)
assert.Equal(t, nil, err)
@ -247,9 +172,13 @@ func TestDataClusterTopology_Codec(t *testing.T) {
err = UnmarshalTopology(s, &topology2)
assert.Equal(t, nil, err)
assert.Equal(t, topology1.Self, topology2.Self)
assert.Equal(t, len(topology1.ConnectedNodes), len(topology2.ConnectedNodes))
for i := range topology1.ConnectedNodes {
assert.Equal(t, topology1.ConnectedNodes[i], topology2.ConnectedNodes[i])
assert.Equal(t, len(topology1.ConnectedDataNodes), len(topology2.ConnectedDataNodes))
assert.Equal(t, len(topology1.ConnectedIndexNodes), len(topology2.ConnectedIndexNodes))
for i := range topology1.ConnectedDataNodes {
assert.Equal(t, topology1.ConnectedDataNodes[i], topology2.ConnectedDataNodes[i])
}
for i := range topology1.ConnectedIndexNodes {
assert.Equal(t, topology1.ConnectedIndexNodes[i], topology2.ConnectedIndexNodes[i])
}
}
@ -262,7 +191,7 @@ func TestDataCoordTopology_Codec(t *testing.T) {
ID: 1,
},
},
ConnectedNodes: []DataNodeInfos{
ConnectedDataNodes: []DataNodeInfos{
{
BaseComponentInfos: BaseComponentInfos{
Name: ConstructComponentName(typeutil.DataNodeRole, 2),
@ -276,6 +205,20 @@ func TestDataCoordTopology_Codec(t *testing.T) {
},
},
},
ConnectedIndexNodes: []IndexNodeInfos{
{
BaseComponentInfos: BaseComponentInfos{
Name: ConstructComponentName(typeutil.IndexNodeRole, 4),
ID: 4,
},
},
{
BaseComponentInfos: BaseComponentInfos{
Name: ConstructComponentName(typeutil.IndexNodeRole, 5),
ID: 5,
},
},
},
},
Connections: ConnTopology{
Name: ConstructComponentName(typeutil.DataCoordRole, 1),
@ -294,9 +237,13 @@ func TestDataCoordTopology_Codec(t *testing.T) {
err = UnmarshalTopology(s, &topology2)
assert.Equal(t, nil, err)
assert.Equal(t, topology1.Cluster.Self, topology2.Cluster.Self)
assert.Equal(t, len(topology1.Cluster.ConnectedNodes), len(topology2.Cluster.ConnectedNodes))
for i := range topology1.Cluster.ConnectedNodes {
assert.Equal(t, topology1.Cluster.ConnectedNodes[i], topology2.Cluster.ConnectedNodes[i])
assert.Equal(t, len(topology1.Cluster.ConnectedDataNodes), len(topology2.Cluster.ConnectedDataNodes))
assert.Equal(t, len(topology1.Cluster.ConnectedIndexNodes), len(topology2.Cluster.ConnectedIndexNodes))
for i := range topology1.Cluster.ConnectedDataNodes {
assert.Equal(t, topology1.Cluster.ConnectedDataNodes[i], topology2.Cluster.ConnectedDataNodes[i])
}
for i := range topology1.Cluster.ConnectedIndexNodes {
assert.Equal(t, topology1.Cluster.ConnectedIndexNodes[i], topology2.Cluster.ConnectedIndexNodes[i])
}
assert.Equal(t, topology1.Connections.Name, topology2.Connections.Name)
assert.Equal(t, len(topology1.Connections.ConnectedComponents), len(topology1.Connections.ConnectedComponents))
@ -314,12 +261,8 @@ func TestRootCoordTopology_Codec(t *testing.T) {
},
},
Connections: ConnTopology{
Name: ConstructComponentName(typeutil.RootCoordRole, 1),
ConnectedComponents: []ConnectionInfo{
{
TargetType: typeutil.IndexCoordRole,
},
},
Name: ConstructComponentName(typeutil.RootCoordRole, 1),
ConnectedComponents: []ConnectionInfo{},
},
}
s, err := MarshalTopology(topology1)
@ -341,9 +284,6 @@ func TestConnTopology_Codec(t *testing.T) {
topology1 := ConnTopology{
Name: ConstructComponentName(typeutil.ProxyRole, 1),
ConnectedComponents: []ConnectionInfo{
{
TargetType: typeutil.IndexCoordRole,
},
{
TargetType: typeutil.DataCoordRole,
},

View File

@ -61,7 +61,6 @@ type ComponentParam struct {
QueryNodeCfg queryNodeConfig
DataCoordCfg dataCoordConfig
DataNodeCfg dataNodeConfig
IndexCoordCfg indexCoordConfig
IndexNodeCfg indexNodeConfig
HTTPCfg httpConfig
HookCfg hookConfig
@ -106,7 +105,6 @@ func (p *ComponentParam) Init() {
p.QueryNodeCfg.init(&p.BaseTable)
p.DataCoordCfg.init(&p.BaseTable)
p.DataNodeCfg.init(&p.BaseTable)
p.IndexCoordCfg.init(&p.BaseTable)
p.IndexNodeCfg.init(&p.BaseTable)
p.HTTPCfg.init(&p.BaseTable)
p.HookCfg.init()
@ -117,7 +115,6 @@ func (p *ComponentParam) Init() {
p.QueryNodeGrpcServerCfg.Init(typeutil.QueryNodeRole, &p.BaseTable)
p.DataCoordGrpcServerCfg.Init(typeutil.DataCoordRole, &p.BaseTable)
p.DataNodeGrpcServerCfg.Init(typeutil.DataNodeRole, &p.BaseTable)
p.IndexCoordGrpcServerCfg.Init(typeutil.IndexCoordRole, &p.BaseTable)
p.IndexNodeGrpcServerCfg.Init(typeutil.IndexNodeRole, &p.BaseTable)
p.RootCoordGrpcClientCfg.Init(typeutil.RootCoordRole, &p.BaseTable)
@ -126,7 +123,6 @@ func (p *ComponentParam) Init() {
p.QueryNodeGrpcClientCfg.Init(typeutil.QueryNodeRole, &p.BaseTable)
p.DataCoordGrpcClientCfg.Init(typeutil.DataCoordRole, &p.BaseTable)
p.DataNodeGrpcClientCfg.Init(typeutil.DataNodeRole, &p.BaseTable)
p.IndexCoordGrpcClientCfg.Init(typeutil.IndexCoordRole, &p.BaseTable)
p.IndexNodeGrpcClientCfg.Init(typeutil.IndexNodeRole, &p.BaseTable)
}
@ -1644,72 +1640,6 @@ func (p *dataNodeConfig) init(base *BaseTable) {
}
// /////////////////////////////////////////////////////////////////////////////
// --- indexcoord ---
type indexCoordConfig struct {
BindIndexNodeMode ParamItem `refreshable:"false"`
IndexNodeAddress ParamItem `refreshable:"false"`
WithCredential ParamItem `refreshable:"false"`
IndexNodeID ParamItem `refreshable:"false"`
MinSegmentNumRowsToEnableIndex ParamItem `refreshable:"true"`
GCInterval ParamItem `refreshable:"false"`
EnableActiveStandby ParamItem `refreshable:"false"`
}
func (p *indexCoordConfig) init(base *BaseTable) {
p.GCInterval = ParamItem{
Key: "indexCoord.gc.interval",
Version: "2.0.0",
DefaultValue: "600",
}
p.GCInterval.Init(base.mgr)
p.MinSegmentNumRowsToEnableIndex = ParamItem{
Key: "indexCoord.minSegmentNumRowsToEnableIndex",
Version: "2.0.0",
DefaultValue: "1024",
}
p.MinSegmentNumRowsToEnableIndex.Init(base.mgr)
p.BindIndexNodeMode = ParamItem{
Key: "indexCoord.bindIndexNodeMode.enable",
Version: "2.0.0",
DefaultValue: "false",
}
p.BindIndexNodeMode.Init(base.mgr)
p.IndexNodeAddress = ParamItem{
Key: "indexCoord.bindIndexNodeMode.address",
Version: "2.0.0",
DefaultValue: "localhost:22930",
}
p.IndexNodeAddress.Init(base.mgr)
p.WithCredential = ParamItem{
Key: "indexCoord.bindIndexNodeMode.withCred",
Version: "2.0.0",
DefaultValue: "false",
}
p.WithCredential.Init(base.mgr)
p.IndexNodeID = ParamItem{
Key: "indexCoord.bindIndexNodeMode.nodeID",
Version: "2.0.0",
DefaultValue: "0",
}
p.IndexNodeID.Init(base.mgr)
p.EnableActiveStandby = ParamItem{
Key: "indexCoord.enableActiveStandby",
Version: "2.0.0",
DefaultValue: "false",
}
p.EnableActiveStandby.Init(base.mgr)
}
// /////////////////////////////////////////////////////////////////////////////
// --- indexnode ---
type indexNodeConfig struct {

View File

@ -335,18 +335,6 @@ func TestComponentParam(t *testing.T) {
assert.Equal(t, 10*time.Minute, Params.SyncPeriod.GetAsDuration(time.Second))
})
t.Run("test indexCoordConfig", func(t *testing.T) {
Params := params.IndexCoordCfg
assert.False(t, Params.BindIndexNodeMode.GetAsBool())
assert.Equal(t, "localhost:22930", Params.IndexNodeAddress.GetValue())
assert.False(t, Params.WithCredential.GetAsBool())
assert.Equal(t, int64(0), Params.IndexNodeID.GetAsInt64())
assert.Equal(t, Params.EnableActiveStandby.GetAsBool(), false)
t.Logf("indexCoord EnableActiveStandby = %t", Params.EnableActiveStandby.GetAsBool())
})
t.Run("test indexNodeConfig", func(t *testing.T) {
Params := params.IndexNodeCfg
params.Save(Params.GracefulStopTimeout.Key, "50")

View File

@ -41,11 +41,11 @@ const (
// IndexCoordRole is a constant represent IndexCoord
IndexCoordRole = "indexcoord"
// IndexNodeRole is a constant represent IndexNode
IndexNodeRole = "indexnode"
// DataCoordRole is a constant represent DataCoord
DataCoordRole = "datacoord"
// DataNodeRole is a constant represent DataNode
DataNodeRole = "datanode"
// IndexNodeRole is a constant represent IndexNode
IndexNodeRole = "indexnode"
)
const Unlimited int64 = -1
@ -58,7 +58,6 @@ func ServerTypeMap() map[string]interface{} {
ProxyRole: nil,
QueryCoordRole: nil,
QueryNodeRole: nil,
IndexCoordRole: nil,
IndexNodeRole: nil,
DataCoordRole: nil,
DataNodeRole: nil,

View File

@ -57,7 +57,7 @@ ${protoc_opt} --go_out=plugins=grpc,paths=source_relative:./indexcgopb index_cgo
${protoc_opt} --go_out=plugins=grpc,paths=source_relative:./rootcoordpb root_coord.proto
${protoc_opt} --go_out=plugins=grpc,paths=source_relative:./internalpb internal.proto
${protoc_opt} --go_out=plugins=grpc,paths=source_relative:./proxypb proxy.proto
${protoc_opt} --go_out=plugins=grpc,paths=source_relative:./indexpb index_coord.proto
${protoc_opt} --go_out=plugins=grpc,paths=source_relative:./indexpb index_node.proto
${protoc_opt} --go_out=plugins=grpc,paths=source_relative:./datapb data_coord.proto
${protoc_opt} --go_out=plugins=grpc,paths=source_relative:./querypb query_coord.proto
${protoc_opt} --go_out=plugins=grpc,paths=source_relative:./planpb plan.proto

View File

@ -48,4 +48,4 @@ echo "Starting indexcoord..."
nohup ./bin/milvus run indexcoord > /tmp/indexcoord.log 2>&1 &
echo "Starting indexnode..."
nohup ./bin/milvus run indexnode > /tmp/indexnode.log 2>&1 &
nohup ./bin/milvus run indexnode > /tmp/indexnode.log 2>&1 &