From 354ab2f55e742fb8e0e9f79c03f0959fc592cecd Mon Sep 17 00:00:00 2001 From: aoiasd <45024769+aoiasd@users.noreply.github.com> Date: Thu, 4 Dec 2025 16:23:11 +0800 Subject: [PATCH] enhance: sync file resource to querynode and datanode (#44480) relate:https://github.com/milvus-io/milvus/issues/43687 Support use file resource with sync mode. Auto download or remove file resource to local when user add or remove file resource. Sync file resource to node when find new node session. --------- Signed-off-by: aoiasd --- internal/coordinator/mix_coord.go | 8 + internal/datacoord/file_resource_manager.go | 157 ++++++++ .../datacoord/file_resource_manager_test.go | 222 +++++++++++ internal/datacoord/import_checker_test.go | 4 +- internal/datacoord/import_inspector_test.go | 2 +- internal/datacoord/import_util_test.go | 12 +- internal/datacoord/meta.go | 26 +- internal/datacoord/meta_test.go | 8 +- internal/datacoord/metrics_info_test.go | 1 + internal/datacoord/mock_test.go | 4 + internal/datacoord/server.go | 18 +- internal/datacoord/services.go | 25 +- internal/datacoord/services_test.go | 51 +-- internal/datanode/data_node.go | 15 + internal/datanode/services.go | 17 + .../distributed/datanode/client/client.go | 6 + internal/distributed/datanode/service.go | 4 + .../distributed/querynode/client/client.go | 7 + internal/distributed/querynode/service.go | 4 + internal/metastore/catalog.go | 7 +- internal/metastore/kv/datacoord/constant.go | 3 +- internal/metastore/kv/datacoord/kv_catalog.go | 50 ++- .../metastore/mocks/mock_datacoord_catalog.go | 77 ++-- internal/metastore/model/resource.go | 35 -- internal/mocks/mock_datanode.go | 59 +++ internal/mocks/mock_datanode_client.go | 74 ++++ internal/mocks/mock_mixcoord.go | 48 +++ internal/mocks/mock_querynode.go | 59 +++ internal/mocks/mock_querynode_client.go | 74 ++++ internal/querycoordv2/mocks/mock_querynode.go | 59 +++ .../observers/file_resource_observer.go | 167 ++++++++ .../observers/file_resource_observer_test.go | 261 +++++++++++++ internal/querycoordv2/server.go | 24 +- internal/querycoordv2/session/cluster.go | 17 + internal/querycoordv2/session/mock_cluster.go | 62 +++ internal/querynodev2/server.go | 3 + internal/querynodev2/services.go | 18 + internal/streamingnode/server/server.go | 5 + internal/types/types.go | 3 + internal/util/fileresource/manager.go | 288 ++++++++++++++ internal/util/fileresource/manager_test.go | 338 +++++++++++++++++ internal/util/fileresource/util.go | 36 ++ internal/util/mock/grpc_datanode_client.go | 4 + internal/util/mock/grpc_querynode_client.go | 4 + internal/util/pathutil/path_util.go | 12 +- internal/util/wrappers/qn_wrapper.go | 4 + pkg/proto/data_coord.proto | 3 + pkg/proto/datapb/data_coord.pb.go | 207 +++++----- pkg/proto/datapb/data_coord_grpc.pb.go | 39 ++ pkg/proto/internal.proto | 12 + pkg/proto/internalpb/internal.pb.go | 359 +++++++++++++----- pkg/proto/query_coord.proto | 3 + pkg/proto/querypb/query_coord.pb.go | 183 ++++----- pkg/proto/querypb/query_coord_grpc.pb.go | 39 ++ pkg/util/paramtable/component_param.go | 17 + 55 files changed, 2814 insertions(+), 430 deletions(-) create mode 100644 internal/datacoord/file_resource_manager.go create mode 100644 internal/datacoord/file_resource_manager_test.go delete mode 100644 internal/metastore/model/resource.go create mode 100644 internal/querycoordv2/observers/file_resource_observer.go create mode 100644 internal/querycoordv2/observers/file_resource_observer_test.go create mode 100644 internal/util/fileresource/manager.go create mode 100644 internal/util/fileresource/manager_test.go create mode 100644 internal/util/fileresource/util.go diff --git a/internal/coordinator/mix_coord.go b/internal/coordinator/mix_coord.go index 742a1caac5..b0bba01bc2 100644 --- a/internal/coordinator/mix_coord.go +++ b/internal/coordinator/mix_coord.go @@ -211,6 +211,10 @@ func (s *mixCoordImpl) initInternal() error { log.Error("queryCoord start failed", zap.Error(err)) return err } + + if err := s.datacoordServer.SyncFileResources(s.ctx); err != nil { + log.Error("init file resources failed", zap.Error(err)) + } return nil } @@ -883,6 +887,10 @@ func (s *mixCoordImpl) GetQcMetrics(ctx context.Context, in *milvuspb.GetMetrics return s.queryCoordServer.GetMetrics(ctx, in) } +func (s *mixCoordImpl) SyncQcFileResource(ctx context.Context, resources []*internalpb.FileResourceInfo, version uint64) error { + return s.queryCoordServer.SyncFileResource(ctx, resources, version) +} + // QueryCoordServer func (s *mixCoordImpl) ActivateChecker(ctx context.Context, req *querypb.ActivateCheckerRequest) (*commonpb.Status, error) { return s.queryCoordServer.ActivateChecker(ctx, req) diff --git a/internal/datacoord/file_resource_manager.go b/internal/datacoord/file_resource_manager.go new file mode 100644 index 0000000000..8484de009a --- /dev/null +++ b/internal/datacoord/file_resource_manager.go @@ -0,0 +1,157 @@ +// 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 datacoord contains core functions in datacoord +package datacoord + +import ( + "context" + "sync" + "time" + + "go.uber.org/zap" + + "github.com/milvus-io/milvus/internal/datacoord/session" + "github.com/milvus-io/milvus/internal/util/fileresource" + "github.com/milvus-io/milvus/pkg/v2/log" + "github.com/milvus-io/milvus/pkg/v2/proto/internalpb" + "github.com/milvus-io/milvus/pkg/v2/util/conc" + "github.com/milvus-io/milvus/pkg/v2/util/merr" + "github.com/milvus-io/milvus/pkg/v2/util/paramtable" +) + +type FileResourceManager struct { + ctx context.Context + meta *meta + + // version distribution + nodeManager session.NodeManager + distribution map[int64]uint64 + + notifyCh chan struct{} + sf conc.Singleflight[any] + once sync.Once + + // close + closeCh chan struct{} + wg sync.WaitGroup +} + +func NewFileResourceManager(ctx context.Context, meta *meta, nodeManager session.NodeManager) *FileResourceManager { + return &FileResourceManager{ + ctx: ctx, + meta: meta, + nodeManager: nodeManager, + distribution: map[int64]uint64{}, + + closeCh: make(chan struct{}), + sf: conc.Singleflight[any]{}, + } +} + +func (m *FileResourceManager) syncLoop() { + defer m.wg.Done() + for { + select { + case <-m.notifyCh: + err := m.sync() + if err != nil { + // retry if error exist + m.sf.Do("retry", func() (any, error) { + time.Sleep(5 * time.Second) + m.Notify() + return nil, nil + }) + } + case <-m.ctx.Done(): + return + case <-m.closeCh: + return + } + } +} + +func (m *FileResourceManager) Start() { + if fileresource.IsSyncMode(paramtable.Get().DataCoordCfg.FileResourceMode.GetValue()) { + m.once.Do(func() { + m.notifyCh = make(chan struct{}, 1) + m.wg.Add(1) + go m.syncLoop() + }) + } +} + +func (m *FileResourceManager) Close() { + close(m.closeCh) + m.wg.Wait() +} + +// notify sync file resource to datanode +// if file resource mode was Sync +func (m *FileResourceManager) Notify() { + if m == nil || m.notifyCh == nil { + return + } + + select { + case m.notifyCh <- struct{}{}: + default: + } +} + +func (m *FileResourceManager) sync() error { + nodes := m.nodeManager.GetClientIDs() + + var syncErr error + + resources, version := m.meta.ListFileResource(m.ctx) + + newDistribution := make(map[int64]uint64) + for _, node := range nodes { + newDistribution[node] = m.distribution[node] + if m.distribution[node] < version { + c, err := m.nodeManager.GetClient(node) + if err != nil { + log.Warn("sync file resource failed, fetch client failed", zap.Error(err)) + syncErr = err + continue + } + status, err := c.SyncFileResource(m.ctx, &internalpb.SyncFileResourceRequest{ + Resources: resources, + Version: version, + }) + if err != nil { + syncErr = err + log.Warn("sync file resource failed", zap.Int64("nodeID", node), zap.Error(err)) + continue + } + + if err = merr.Error(status); err != nil { + log.Warn("sync file resource failed", zap.Int64("nodeID", node), zap.Error(err)) + syncErr = err + continue + } + newDistribution[node] = version + log.Info("finish sync file resource to data node", zap.Int64("node", node), zap.Uint64("version", version)) + } + } + m.distribution = newDistribution + + if syncErr != nil { + return syncErr + } + return nil +} diff --git a/internal/datacoord/file_resource_manager_test.go b/internal/datacoord/file_resource_manager_test.go new file mode 100644 index 0000000000..34f5930fdd --- /dev/null +++ b/internal/datacoord/file_resource_manager_test.go @@ -0,0 +1,222 @@ +// 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 datacoord + +import ( + "context" + "testing" + "time" + + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/suite" + "google.golang.org/grpc" + + "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + "github.com/milvus-io/milvus/internal/datacoord/session" + metamock "github.com/milvus-io/milvus/internal/metastore/mocks" + "github.com/milvus-io/milvus/internal/mocks" + "github.com/milvus-io/milvus/pkg/v2/proto/internalpb" + "github.com/milvus-io/milvus/pkg/v2/util/merr" + "github.com/milvus-io/milvus/pkg/v2/util/paramtable" +) + +type FileResourceManagerSuite struct { + suite.Suite + + ctx context.Context + manager *FileResourceManager + + // Test objects + testMeta *meta + mockNodeManager *session.MockNodeManager + mockDataNode *mocks.MockDataNodeClient + mockCatalog *metamock.DataCoordCatalog +} + +func (suite *FileResourceManagerSuite) SetupSuite() { + paramtable.Init() +} + +func (suite *FileResourceManagerSuite) SetupTest() { + suite.ctx = context.Background() + + // Create mocks + suite.mockNodeManager = session.NewMockNodeManager(suite.T()) + suite.mockDataNode = mocks.NewMockDataNodeClient(suite.T()) + suite.mockCatalog = metamock.NewDataCoordCatalog(suite.T()) + + // Create test meta with minimal initialization + suite.testMeta = &meta{ + catalog: suite.mockCatalog, + resourceMeta: make(map[string]*internalpb.FileResourceInfo), + resourceVersion: 0, + } + + // Create FileResourceManager + suite.manager = NewFileResourceManager(suite.ctx, suite.testMeta, suite.mockNodeManager) + suite.manager.Start() +} + +func (suite *FileResourceManagerSuite) TearDownTest() { + suite.manager.Close() + // Assert mock expectations + suite.mockNodeManager.AssertExpectations(suite.T()) + suite.mockDataNode.AssertExpectations(suite.T()) +} + +func (suite *FileResourceManagerSuite) TestNormal() { + testResource := &internalpb.FileResourceInfo{ + Id: 1, + Name: "test", + Path: "/tmp/test", + } + + suite.mockNodeManager.EXPECT().GetClientIDs().Return([]int64{1}) + suite.mockNodeManager.EXPECT().GetClient(int64(1)).Return(suite.mockDataNode, nil) + + syncCh := make(chan struct{}, 1) + suite.mockDataNode.EXPECT().SyncFileResource(mock.Anything, mock.Anything, mock.Anything).Run(func(ctx context.Context, in *internalpb.SyncFileResourceRequest, opts ...grpc.CallOption) { + suite.Equal(1, len(in.Resources)) + suite.Equal(testResource.Id, in.Resources[0].Id) + suite.Equal(testResource.Name, in.Resources[0].Name) + suite.Equal(testResource.Path, in.Resources[0].Path) + syncCh <- struct{}{} + }).Return(merr.Success(), nil).Once() + suite.mockCatalog.EXPECT().SaveFileResource(mock.Anything, mock.Anything, mock.Anything).Return(nil) + suite.testMeta.AddFileResource(suite.ctx, testResource) + + // notify sync + suite.manager.Notify() + + suite.Eventually(func() bool { + select { + case <-syncCh: + return true + default: + return false + } + }, 2*time.Second, 100*time.Millisecond) +} + +func (suite *FileResourceManagerSuite) TestSync_Success() { + // Prepare test data + nodeID := int64(1) + resources := []*internalpb.FileResourceInfo{ + { + Id: 1, + Name: "test.file", + Path: "/test/test.file", + }, + } + version := uint64(100) + + // Setup meta state directly + suite.testMeta.resourceMeta["test.file"] = resources[0] + suite.testMeta.resourceVersion = version + + // Setup mocks + suite.mockNodeManager.EXPECT().GetClientIDs().Return([]int64{nodeID}) + suite.mockNodeManager.EXPECT().GetClient(nodeID).Return(suite.mockDataNode, nil) + suite.mockDataNode.EXPECT().SyncFileResource( + suite.ctx, + &internalpb.SyncFileResourceRequest{ + Resources: resources, + Version: version, + }, + ).Return(merr.Success(), nil) + + // Execute sync + err := suite.manager.sync() + + // Verify + suite.NoError(err) + suite.Equal(version, suite.manager.distribution[nodeID]) +} + +func (suite *FileResourceManagerSuite) TestSync_NodeClientError() { + // Prepare test data + nodeID := int64(1) + version := uint64(100) + + // Setup meta state directly + suite.testMeta.resourceVersion = version + + // Setup mocks - GetClient fails + suite.mockNodeManager.EXPECT().GetClientIDs().Return([]int64{nodeID}) + suite.mockNodeManager.EXPECT().GetClient(nodeID).Return(nil, merr.WrapErrNodeNotFound(nodeID)) + + // Execute sync + err := suite.manager.sync() + + // Verify error is returned and distribution not updated + suite.Error(err) + suite.Equal(uint64(0), suite.manager.distribution[nodeID]) +} + +func (suite *FileResourceManagerSuite) TestSync_SyncFileResourceError() { + // Prepare test data + nodeID := int64(1) + version := uint64(100) + + // Setup meta state directly + suite.testMeta.resourceVersion = version + + // Setup mocks - SyncFileResource fails + suite.mockNodeManager.EXPECT().GetClientIDs().Return([]int64{nodeID}) + suite.mockNodeManager.EXPECT().GetClient(nodeID).Return(suite.mockDataNode, nil) + suite.mockDataNode.EXPECT().SyncFileResource( + suite.ctx, + mock.AnythingOfType("*internalpb.SyncFileResourceRequest"), + ).Return(nil, merr.WrapErrServiceInternal("sync failed")) + + // Execute sync + err := suite.manager.sync() + + // Verify error is returned and distribution not updated + suite.Error(err) + suite.Equal(uint64(0), suite.manager.distribution[nodeID]) +} + +func (suite *FileResourceManagerSuite) TestSync_SyncFileResourceStatusError() { + // Prepare test data + nodeID := int64(1) + version := uint64(100) + + // Setup mocks - SyncFileResource returns error status + // Setup meta state directly + suite.testMeta.resourceVersion = version + suite.mockNodeManager.EXPECT().GetClientIDs().Return([]int64{nodeID}) + suite.mockNodeManager.EXPECT().GetClient(nodeID).Return(suite.mockDataNode, nil) + suite.mockDataNode.EXPECT().SyncFileResource( + suite.ctx, + mock.AnythingOfType("*internalpb.SyncFileResourceRequest"), + ).Return(&commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + Reason: "internal error", + }, nil) + + // Execute sync + err := suite.manager.sync() + + // Verify error is returned and distribution not updated + suite.Error(err) + suite.Equal(uint64(0), suite.manager.distribution[nodeID]) +} + +func TestFileResourceManagerSuite(t *testing.T) { + suite.Run(t, new(FileResourceManagerSuite)) +} diff --git a/internal/datacoord/import_checker_test.go b/internal/datacoord/import_checker_test.go index c4a8a5b670..ec5277e79f 100644 --- a/internal/datacoord/import_checker_test.go +++ b/internal/datacoord/import_checker_test.go @@ -64,7 +64,7 @@ func (s *ImportCheckerSuite) SetupTest() { catalog.EXPECT().ListCompactionTask(mock.Anything).Return(nil, nil) catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil) catalog.EXPECT().ListStatsTasks(mock.Anything).Return(nil, nil) - catalog.EXPECT().ListFileResource(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListFileResource(mock.Anything).Return(nil, 0, nil) s.alloc = allocator.NewMockAllocator(s.T()) @@ -575,7 +575,7 @@ func TestImportCheckerCompaction(t *testing.T) { catalog.EXPECT().ListCompactionTask(mock.Anything).Return(nil, nil) catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil) catalog.EXPECT().ListStatsTasks(mock.Anything).Return(nil, nil) - catalog.EXPECT().ListFileResource(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListFileResource(mock.Anything).Return(nil, 0, nil) alloc := allocator.NewMockAllocator(t) diff --git a/internal/datacoord/import_inspector_test.go b/internal/datacoord/import_inspector_test.go index 302a50ad7e..8aa340ad7e 100644 --- a/internal/datacoord/import_inspector_test.go +++ b/internal/datacoord/import_inspector_test.go @@ -63,7 +63,7 @@ func (s *ImportInspectorSuite) SetupTest() { s.catalog.EXPECT().ListCompactionTask(mock.Anything).Return(nil, nil) s.catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil) s.catalog.EXPECT().ListStatsTasks(mock.Anything).Return(nil, nil) - s.catalog.EXPECT().ListFileResource(mock.Anything).Return(nil, nil) + s.catalog.EXPECT().ListFileResource(mock.Anything).Return(nil, 0, nil) s.alloc = allocator.NewMockAllocator(s.T()) broker := broker.NewMockBroker(s.T()) diff --git a/internal/datacoord/import_util_test.go b/internal/datacoord/import_util_test.go index e4ac6c3d04..7639fb59d9 100644 --- a/internal/datacoord/import_util_test.go +++ b/internal/datacoord/import_util_test.go @@ -133,7 +133,7 @@ func TestImportUtil_NewImportTasks(t *testing.T) { catalog.EXPECT().ListCompactionTask(mock.Anything).Return(nil, nil) catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil) catalog.EXPECT().ListStatsTasks(mock.Anything).Return(nil, nil) - catalog.EXPECT().ListFileResource(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListFileResource(mock.Anything).Return(nil, 0, nil) broker := broker.NewMockBroker(t) broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(nil, nil) @@ -206,7 +206,7 @@ func TestImportUtil_NewImportTasksWithDataTt(t *testing.T) { catalog.EXPECT().ListCompactionTask(mock.Anything).Return(nil, nil) catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil) catalog.EXPECT().ListStatsTasks(mock.Anything).Return(nil, nil) - catalog.EXPECT().ListFileResource(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListFileResource(mock.Anything).Return(nil, 0, nil) broker := broker2.NewMockBroker(t) broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(&rootcoordpb.ShowCollectionIDsResponse{}, nil) @@ -267,7 +267,7 @@ func TestImportUtil_AssembleRequest(t *testing.T) { catalog.EXPECT().ListCompactionTask(mock.Anything).Return(nil, nil) catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil) catalog.EXPECT().ListStatsTasks(mock.Anything).Return(nil, nil) - catalog.EXPECT().ListFileResource(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListFileResource(mock.Anything).Return(nil, 0, nil) alloc := allocator.NewMockAllocator(t) alloc.EXPECT().AllocN(mock.Anything).RunAndReturn(func(n int64) (int64, int64, error) { @@ -343,7 +343,7 @@ func TestImportUtil_AssembleRequestWithDataTt(t *testing.T) { catalog.EXPECT().ListCompactionTask(mock.Anything).Return(nil, nil) catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil) catalog.EXPECT().ListStatsTasks(mock.Anything).Return(nil, nil) - catalog.EXPECT().ListFileResource(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListFileResource(mock.Anything).Return(nil, 0, nil) alloc := allocator.NewMockAllocator(t) alloc.EXPECT().AllocN(mock.Anything).RunAndReturn(func(n int64) (int64, int64, error) { @@ -428,7 +428,7 @@ func TestImportUtil_CheckDiskQuota(t *testing.T) { catalog.EXPECT().ListCompactionTask(mock.Anything).Return(nil, nil) catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil) catalog.EXPECT().ListStatsTasks(mock.Anything).Return(nil, nil) - catalog.EXPECT().ListFileResource(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListFileResource(mock.Anything).Return(nil, 0, nil) importMeta, err := NewImportMeta(context.TODO(), catalog, nil, nil) assert.NoError(t, err) @@ -616,7 +616,7 @@ func TestImportUtil_GetImportProgress(t *testing.T) { catalog.EXPECT().ListCompactionTask(mock.Anything).Return(nil, nil) catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil) catalog.EXPECT().ListStatsTasks(mock.Anything).Return(nil, nil) - catalog.EXPECT().ListFileResource(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListFileResource(mock.Anything).Return(nil, 0, nil) importMeta, err := NewImportMeta(context.TODO(), catalog, nil, nil) assert.NoError(t, err) diff --git a/internal/datacoord/meta.go b/internal/datacoord/meta.go index 7a603f746f..b9ba0c099e 100644 --- a/internal/datacoord/meta.go +++ b/internal/datacoord/meta.go @@ -36,13 +36,13 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/datacoord/broker" "github.com/milvus-io/milvus/internal/metastore" - "github.com/milvus-io/milvus/internal/metastore/model" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/segmentutil" "github.com/milvus-io/milvus/pkg/v2/common" "github.com/milvus-io/milvus/pkg/v2/log" "github.com/milvus-io/milvus/pkg/v2/metrics" "github.com/milvus-io/milvus/pkg/v2/proto/datapb" + "github.com/milvus-io/milvus/pkg/v2/proto/internalpb" "github.com/milvus-io/milvus/pkg/v2/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/v2/util/conc" "github.com/milvus-io/milvus/pkg/v2/util/funcutil" @@ -102,8 +102,9 @@ type meta struct { externalCollectionTaskMeta *externalCollectionTaskMeta // File Resource Meta - resourceMeta map[string]*model.FileResource - resourceLock lock.RWMutex + resourceMeta map[string]*internalpb.FileResourceInfo // name -> info + resourceVersion uint64 + resourceLock lock.RWMutex } func (m *meta) GetIndexMeta() *indexMeta { @@ -204,7 +205,7 @@ func newMeta(ctx context.Context, catalog metastore.DataCoordCatalog, chunkManag compactionTaskMeta: ctm, statsTaskMeta: stm, // externalCollectionTaskMeta: ectm, - resourceMeta: make(map[string]*model.FileResource), + resourceMeta: make(map[string]*internalpb.FileResourceInfo), } err = mt.reloadFromKV(ctx, broker) if err != nil { @@ -2417,20 +2418,21 @@ func (m *meta) reloadFileResourceMeta(ctx context.Context) error { m.resourceLock.Lock() defer m.resourceLock.Unlock() - resources, err := m.catalog.ListFileResource(ctx) + resources, version, err := m.catalog.ListFileResource(ctx) if err != nil { return err } - m.resourceMeta = make(map[string]*model.FileResource) + m.resourceMeta = make(map[string]*internalpb.FileResourceInfo) for _, resource := range resources { m.resourceMeta[resource.Name] = resource } + m.resourceVersion = version return nil } // AddFileResource add file resource to meta -func (m *meta) AddFileResource(ctx context.Context, resource *model.FileResource) error { +func (m *meta) AddFileResource(ctx context.Context, resource *internalpb.FileResourceInfo) error { m.resourceLock.Lock() defer m.resourceLock.Unlock() @@ -2438,12 +2440,13 @@ func (m *meta) AddFileResource(ctx context.Context, resource *model.FileResource return merr.WrapErrAsInputError(fmt.Errorf("create resource failed: resource name exist")) } - err := m.catalog.SaveFileResource(ctx, resource) + err := m.catalog.SaveFileResource(ctx, resource, m.resourceVersion+1) if err != nil { return err } m.resourceMeta[resource.Name] = resource + m.resourceVersion += 1 return nil } @@ -2453,21 +2456,22 @@ func (m *meta) RemoveFileResource(ctx context.Context, name string) error { defer m.resourceLock.Unlock() if resource, ok := m.resourceMeta[name]; ok { - err := m.catalog.RemoveFileResource(ctx, resource.ID) + err := m.catalog.RemoveFileResource(ctx, resource.Id, m.resourceVersion+1) if err != nil { return err } delete(m.resourceMeta, name) + m.resourceVersion += 1 } return nil } // ListFileResource list file resources from meta -func (m *meta) ListFileResource(ctx context.Context) []*model.FileResource { +func (m *meta) ListFileResource(ctx context.Context) ([]*internalpb.FileResourceInfo, uint64) { m.resourceLock.RLock() defer m.resourceLock.RUnlock() - return lo.Values(m.resourceMeta) + return lo.Values(m.resourceMeta), m.resourceVersion } diff --git a/internal/datacoord/meta_test.go b/internal/datacoord/meta_test.go index a5712d192b..e6c39cd4bb 100644 --- a/internal/datacoord/meta_test.go +++ b/internal/datacoord/meta_test.go @@ -45,6 +45,7 @@ import ( "github.com/milvus-io/milvus/pkg/v2/kv" "github.com/milvus-io/milvus/pkg/v2/metrics" "github.com/milvus-io/milvus/pkg/v2/proto/datapb" + "github.com/milvus-io/milvus/pkg/v2/proto/internalpb" "github.com/milvus-io/milvus/pkg/v2/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/v2/util" "github.com/milvus-io/milvus/pkg/v2/util/merr" @@ -129,7 +130,7 @@ func (suite *MetaReloadSuite) TestReloadFromKV() { }, }, nil) - suite.catalog.EXPECT().ListFileResource(mock.Anything).Return([]*model.FileResource{}, nil) + suite.catalog.EXPECT().ListFileResource(mock.Anything).Return([]*internalpb.FileResourceInfo{}, 0, nil) suite.catalog.EXPECT().ListIndexes(mock.Anything).Return([]*model.Index{}, nil) suite.catalog.EXPECT().ListSegmentIndexes(mock.Anything).Return([]*model.SegmentIndex{}, nil) suite.catalog.EXPECT().ListAnalyzeTasks(mock.Anything).Return(nil, nil) @@ -175,7 +176,7 @@ func (suite *MetaReloadSuite) TestReloadFromKV() { }, }, nil) - suite.catalog.EXPECT().ListFileResource(mock.Anything).Return([]*model.FileResource{}, nil) + suite.catalog.EXPECT().ListFileResource(mock.Anything).Return([]*internalpb.FileResourceInfo{}, 0, nil) suite.catalog.EXPECT().ListIndexes(mock.Anything).Return([]*model.Index{}, nil) suite.catalog.EXPECT().ListSegmentIndexes(mock.Anything).Return([]*model.SegmentIndex{}, nil) suite.catalog.EXPECT().ListAnalyzeTasks(mock.Anything).Return(nil, nil) @@ -703,6 +704,7 @@ func TestMeta_Basic(t *testing.T) { metakv.EXPECT().MultiSave(mock.Anything, mock.Anything).Return(errors.New("failed")).Maybe() metakv.EXPECT().WalkWithPrefix(mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil).Maybe() metakv.EXPECT().LoadWithPrefix(mock.Anything, mock.Anything).Return(nil, nil, nil).Maybe() + metakv.EXPECT().Has(mock.Anything, datacoord.FileResourceVersionKey).Return(false, nil).Maybe() catalog := datacoord.NewCatalog(metakv, "", "") broker := broker.NewMockBroker(t) broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(nil, nil) @@ -714,6 +716,7 @@ func TestMeta_Basic(t *testing.T) { metakv2 := mockkv.NewMetaKv(t) metakv2.EXPECT().Save(mock.Anything, mock.Anything, mock.Anything).Return(nil).Maybe() + metakv2.EXPECT().Has(mock.Anything, datacoord.FileResourceVersionKey).Return(false, nil).Maybe() metakv2.EXPECT().MultiSave(mock.Anything, mock.Anything).Return(nil).Maybe() metakv2.EXPECT().Remove(mock.Anything, mock.Anything).Return(errors.New("failed")).Maybe() metakv2.EXPECT().MultiRemove(mock.Anything, mock.Anything).Return(errors.New("failed")).Maybe() @@ -1173,6 +1176,7 @@ func TestUpdateSegmentsInfo(t *testing.T) { metakv.EXPECT().MultiSave(mock.Anything, mock.Anything).Return(errors.New("mocked fail")).Maybe() metakv.EXPECT().WalkWithPrefix(mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil).Maybe() metakv.EXPECT().LoadWithPrefix(mock.Anything, mock.Anything).Return(nil, nil, nil).Maybe() + metakv.EXPECT().Has(mock.Anything, mock.Anything).Return(false, nil).Maybe() catalog := datacoord.NewCatalog(metakv, "", "") broker := broker.NewMockBroker(t) broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(nil, nil) diff --git a/internal/datacoord/metrics_info_test.go b/internal/datacoord/metrics_info_test.go index 3ded0ce3e1..f645070395 100644 --- a/internal/datacoord/metrics_info_test.go +++ b/internal/datacoord/metrics_info_test.go @@ -77,6 +77,7 @@ func TestGetDataNodeMetrics(t *testing.T) { } mockNodeManager.EXPECT().Startup(mock.Anything, mock.Anything).Return(nil) + mockNodeManager.EXPECT().GetClientIDs().Return([]int64{1}) mockNodeManager.EXPECT().GetClient(mock.Anything).RunAndReturn(func(nodeID int64) (types.DataNodeClient, error) { switch nodeID { case 100: diff --git a/internal/datacoord/mock_test.go b/internal/datacoord/mock_test.go index 79fb958711..8105375e04 100644 --- a/internal/datacoord/mock_test.go +++ b/internal/datacoord/mock_test.go @@ -252,6 +252,10 @@ func (m *mockMixCoord) GetComponentStates(ctx context.Context, req *milvuspb.Get }, nil } +func (h *mockMixCoord) SyncQcFileResource(ctx context.Context, resources []*internalpb.FileResourceInfo, verion uint64) error { + return nil +} + func (m *mockMixCoord) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) { panic("not implemented") // TODO: Implement } diff --git a/internal/datacoord/server.go b/internal/datacoord/server.go index 1916ce76f9..535c62f3a7 100644 --- a/internal/datacoord/server.go +++ b/internal/datacoord/server.go @@ -166,6 +166,9 @@ type Server struct { broker broker.Broker metricsRequest *metricsinfo.MetricsRequest + + // file resource + fileManager *FileResourceManager } type CollectionNameInfo struct { @@ -337,6 +340,8 @@ func (s *Server) initDataCoord() error { s.importChecker = NewImportChecker(s.ctx, s.meta, s.broker, s.allocator, s.importMeta, s.compactionInspector, s.handler, s.compactionTriggerManager) + s.fileManager = NewFileResourceManager(s.ctx, s.meta, s.nodeManager) + s.serverLoopCtx, s.serverLoopCancel = context.WithCancel(s.ctx) RegisterDDLCallbacks(s) @@ -432,7 +437,8 @@ func (s *Server) Start() error { func (s *Server) startDataCoord() { s.startTaskScheduler() s.startServerLoop() - + s.fileManager.Start() + s.fileManager.Notify() s.afterStart() s.UpdateStateCode(commonpb.StateCode_Healthy) sessionutil.SaveServerInfo(typeutil.MixCoordRole, s.session.GetServerID()) @@ -868,7 +874,14 @@ func (s *Server) handleSessionEvent(ctx context.Context, role string, event *ses zap.String("event type", event.EventType.String())) return nil } - return s.nodeManager.AddNode(event.Session.ServerID, event.Session.Address) + err := s.nodeManager.AddNode(event.Session.ServerID, event.Session.Address) + if err != nil { + return err + } + + // notify file manager sync file resource to new node + s.fileManager.Notify() + return nil case sessionutil.SessionDelEvent: log.Info("received datanode unregister", zap.String("address", info.Address), @@ -1051,6 +1064,7 @@ func (s *Server) Stop() error { s.stopServerLoop() log.Info("datacoord stopServerLoop stopped") + s.fileManager.Close() s.globalScheduler.Stop() s.importInspector.Close() s.importChecker.Close() diff --git a/internal/datacoord/services.go b/internal/datacoord/services.go index 3603331063..6ba8b8681e 100644 --- a/internal/datacoord/services.go +++ b/internal/datacoord/services.go @@ -35,7 +35,6 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/internal/coordinator/snmanager" "github.com/milvus-io/milvus/internal/metastore/kv/binlog" - "github.com/milvus-io/milvus/internal/metastore/model" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/componentutil" "github.com/milvus-io/milvus/internal/util/importutilv2" @@ -2035,9 +2034,9 @@ func (s *Server) AddFileResource(ctx context.Context, req *milvuspb.AddFileResou return merr.Status(err), nil } - // Convert to model.FileResource - resource := &model.FileResource{ - ID: id, + // Convert to internalpb.FileResourceInfo + resource := &internalpb.FileResourceInfo{ + Id: id, Name: req.GetName(), Path: req.GetPath(), } @@ -2047,6 +2046,10 @@ func (s *Server) AddFileResource(ctx context.Context, req *milvuspb.AddFileResou log.Ctx(ctx).Warn("AddFileResource fail", zap.Error(err)) return merr.Status(err), nil } + s.fileManager.Notify() + + resources, version := s.meta.ListFileResource(ctx) + s.mixCoord.SyncQcFileResource(ctx, resources, version) log.Ctx(ctx).Info("AddFileResource success") return merr.Success(), nil @@ -2066,6 +2069,10 @@ func (s *Server) RemoveFileResource(ctx context.Context, req *milvuspb.RemoveFil log.Ctx(ctx).Warn("RemoveFileResource fail", zap.Error(err)) return merr.Status(err), nil } + s.fileManager.Notify() + + resources, version := s.meta.ListFileResource(ctx) + s.mixCoord.SyncQcFileResource(ctx, resources, version) log.Ctx(ctx).Info("RemoveFileResource success") return merr.Success(), nil @@ -2081,9 +2088,9 @@ func (s *Server) ListFileResources(ctx context.Context, req *milvuspb.ListFileRe log.Ctx(ctx).Info("receive ListFileResources request") - resources := s.meta.ListFileResource(ctx) + resources, _ := s.meta.ListFileResource(ctx) - // Convert model.FileResource to milvuspb.FileResourceInfo + // Convert internal.FileResourceInfo to milvuspb.FileResourceInfo fileResources := make([]*milvuspb.FileResourceInfo, 0, len(resources)) for _, resource := range resources { fileResources = append(fileResources, &milvuspb.FileResourceInfo{ @@ -2143,3 +2150,9 @@ func (s *Server) CreateExternalCollection(ctx context.Context, req *msgpb.Create Status: merr.Success(), }, nil } + +// first sync file resource data to qc when all coord init finished +func (s *Server) SyncFileResources(ctx context.Context) error { + resources, version := s.meta.ListFileResource(ctx) + return s.mixCoord.SyncQcFileResource(ctx, resources, version) +} diff --git a/internal/datacoord/services_test.go b/internal/datacoord/services_test.go index 873892e27c..ff5649c94b 100644 --- a/internal/datacoord/services_test.go +++ b/internal/datacoord/services_test.go @@ -1778,8 +1778,9 @@ func TestServer_AddFileResource(t *testing.T) { server := &Server{ idAllocator: globalIDAllocator.NewTestGlobalIDAllocator(mockAllocator), + mixCoord: newMockMixCoord(), meta: &meta{ - resourceMeta: make(map[string]*model.FileResource), + resourceMeta: make(map[string]*internalpb.FileResourceInfo), catalog: mockCatalog, }, } @@ -1791,9 +1792,9 @@ func TestServer_AddFileResource(t *testing.T) { Path: "/path/to/resource", } - mockCatalog.EXPECT().SaveFileResource(mock.Anything, mock.MatchedBy(func(resource *model.FileResource) bool { + mockCatalog.EXPECT().SaveFileResource(mock.Anything, mock.MatchedBy(func(resource *internalpb.FileResourceInfo) bool { return resource.Name == "test_resource" && resource.Path == "/path/to/resource" - })).Return(nil) + }), mock.Anything).Return(nil) resp, err := server.AddFileResource(context.Background(), req) assert.NoError(t, err) @@ -1822,7 +1823,7 @@ func TestServer_AddFileResource(t *testing.T) { server := &Server{ idAllocator: globalIDAllocator.NewTestGlobalIDAllocator(mockAllocator), meta: &meta{ - resourceMeta: make(map[string]*model.FileResource), + resourceMeta: make(map[string]*internalpb.FileResourceInfo), catalog: mockCatalog, }, } @@ -1846,7 +1847,7 @@ func TestServer_AddFileResource(t *testing.T) { server := &Server{ idAllocator: globalIDAllocator.NewTestGlobalIDAllocator(mockAllocator), meta: &meta{ - resourceMeta: make(map[string]*model.FileResource), + resourceMeta: make(map[string]*internalpb.FileResourceInfo), catalog: mockCatalog, }, } @@ -1857,7 +1858,7 @@ func TestServer_AddFileResource(t *testing.T) { Path: "/path/to/resource", } - mockCatalog.EXPECT().SaveFileResource(mock.Anything, mock.Anything).Return(errors.New("catalog error")) + mockCatalog.EXPECT().SaveFileResource(mock.Anything, mock.Anything, mock.Anything).Return(errors.New("catalog error")) resp, err := server.AddFileResource(context.Background(), req) assert.NoError(t, err) @@ -1869,8 +1870,8 @@ func TestServer_AddFileResource(t *testing.T) { mockAllocator := tso.NewMockAllocator() mockAllocator.GenerateTSOF = func(count uint32) (uint64, error) { return 100, nil } - existingResource := &model.FileResource{ - ID: 1, + existingResource := &internalpb.FileResourceInfo{ + Id: 1, Name: "test_resource", Path: "/existing/path", } @@ -1878,7 +1879,7 @@ func TestServer_AddFileResource(t *testing.T) { server := &Server{ idAllocator: globalIDAllocator.NewTestGlobalIDAllocator(mockAllocator), meta: &meta{ - resourceMeta: map[string]*model.FileResource{ + resourceMeta: map[string]*internalpb.FileResourceInfo{ "test_resource": existingResource, }, catalog: mockCatalog, @@ -1902,19 +1903,20 @@ func TestServer_RemoveFileResource(t *testing.T) { t.Run("success", func(t *testing.T) { mockCatalog := mocks.NewDataCoordCatalog(t) - existingResource := &model.FileResource{ - ID: 1, + existingResource := &internalpb.FileResourceInfo{ + Id: 1, Name: "test_resource", Path: "/path/to/resource", } server := &Server{ meta: &meta{ - resourceMeta: map[string]*model.FileResource{ + resourceMeta: map[string]*internalpb.FileResourceInfo{ "test_resource": existingResource, }, catalog: mockCatalog, }, + mixCoord: newMockMixCoord(), } server.stateCode.Store(commonpb.StateCode_Healthy) @@ -1923,7 +1925,7 @@ func TestServer_RemoveFileResource(t *testing.T) { Name: "test_resource", } - mockCatalog.EXPECT().RemoveFileResource(mock.Anything, int64(1)).Return(nil) + mockCatalog.EXPECT().RemoveFileResource(mock.Anything, mock.Anything, mock.Anything).Return(nil) resp, err := server.RemoveFileResource(context.Background(), req) assert.NoError(t, err) @@ -1948,9 +1950,10 @@ func TestServer_RemoveFileResource(t *testing.T) { server := &Server{ meta: &meta{ - resourceMeta: make(map[string]*model.FileResource), + resourceMeta: make(map[string]*internalpb.FileResourceInfo), catalog: mockCatalog, }, + mixCoord: newMockMixCoord(), } server.stateCode.Store(commonpb.StateCode_Healthy) @@ -1966,15 +1969,15 @@ func TestServer_RemoveFileResource(t *testing.T) { t.Run("catalog remove error", func(t *testing.T) { mockCatalog := mocks.NewDataCoordCatalog(t) - existingResource := &model.FileResource{ - ID: 1, + existingResource := &internalpb.FileResourceInfo{ + Id: 1, Name: "test_resource", Path: "/path/to/resource", } server := &Server{ meta: &meta{ - resourceMeta: map[string]*model.FileResource{ + resourceMeta: map[string]*internalpb.FileResourceInfo{ "test_resource": existingResource, }, catalog: mockCatalog, @@ -1986,7 +1989,7 @@ func TestServer_RemoveFileResource(t *testing.T) { Name: "test_resource", } - mockCatalog.EXPECT().RemoveFileResource(mock.Anything, int64(1)).Return(errors.New("catalog error")) + mockCatalog.EXPECT().RemoveFileResource(mock.Anything, int64(1), mock.Anything).Return(errors.New("catalog error")) resp, err := server.RemoveFileResource(context.Background(), req) assert.NoError(t, err) @@ -2000,7 +2003,7 @@ func TestServer_ListFileResources(t *testing.T) { server := &Server{ meta: &meta{ - resourceMeta: make(map[string]*model.FileResource), + resourceMeta: make(map[string]*internalpb.FileResourceInfo), catalog: mockCatalog, }, } @@ -2020,20 +2023,20 @@ func TestServer_ListFileResources(t *testing.T) { t.Run("success with resources", func(t *testing.T) { mockCatalog := mocks.NewDataCoordCatalog(t) - resource1 := &model.FileResource{ - ID: 1, + resource1 := &internalpb.FileResourceInfo{ + Id: 1, Name: "resource1", Path: "/path/to/resource1", } - resource2 := &model.FileResource{ - ID: 2, + resource2 := &internalpb.FileResourceInfo{ + Id: 2, Name: "resource2", Path: "/path/to/resource2", } server := &Server{ meta: &meta{ - resourceMeta: map[string]*model.FileResource{ + resourceMeta: map[string]*internalpb.FileResourceInfo{ "resource1": resource1, "resource2": resource2, }, diff --git a/internal/datanode/data_node.go b/internal/datanode/data_node.go index 810f20510f..cdcea68569 100644 --- a/internal/datanode/data_node.go +++ b/internal/datanode/data_node.go @@ -34,11 +34,13 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" + "github.com/milvus-io/milvus/internal/compaction" "github.com/milvus-io/milvus/internal/datanode/compactor" "github.com/milvus-io/milvus/internal/datanode/importv2" "github.com/milvus-io/milvus/internal/datanode/index" "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/internal/util/fileresource" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/v2/log" "github.com/milvus-io/milvus/pkg/v2/metrics" @@ -194,6 +196,19 @@ func (node *DataNode) Init() error { syncMgr := syncmgr.NewSyncManager(nil) node.syncMgr = syncMgr + fileMode := fileresource.ParseMode(paramtable.Get().DataCoordCfg.FileResourceMode.GetValue()) + if fileMode == fileresource.SyncMode { + cm, err := node.storageFactory.NewChunkManager(node.ctx, compaction.CreateStorageConfig()) + if err != nil { + log.Error("Init chunk manager for file resource manager failed", zap.Error(err)) + initError = err + return + } + fileresource.InitManager(cm, fileMode) + } else { + fileresource.InitManager(nil, fileMode) + } + node.importTaskMgr = importv2.NewTaskManager() node.importScheduler = importv2.NewScheduler(node.importTaskMgr) diff --git a/internal/datanode/services.go b/internal/datanode/services.go index fd4a3d37c2..50464e170b 100644 --- a/internal/datanode/services.go +++ b/internal/datanode/services.go @@ -32,6 +32,7 @@ import ( "github.com/milvus-io/milvus/internal/datanode/compactor" "github.com/milvus-io/milvus/internal/datanode/importv2" "github.com/milvus-io/milvus/internal/flushcommon/io" + "github.com/milvus-io/milvus/internal/util/fileresource" "github.com/milvus-io/milvus/internal/util/hookutil" "github.com/milvus-io/milvus/internal/util/importutilv2" "github.com/milvus-io/milvus/pkg/v2/common" @@ -773,3 +774,19 @@ func (node *DataNode) DropTask(ctx context.Context, request *workerpb.DropTaskRe return merr.Status(err), nil } } + +func (node *DataNode) SyncFileResource(ctx context.Context, req *internalpb.SyncFileResourceRequest) (*commonpb.Status, error) { + log := log.Ctx(ctx).With(zap.Uint64("version", req.GetVersion())) + log.Info("sync file resource", zap.Any("resources", req.Resources)) + + if !node.isHealthy() { + log.Warn("failed to sync file resource, DataNode is not healthy") + return merr.Status(merr.ErrServiceNotReady), nil + } + + err := fileresource.Sync(req.GetResources()) + if err != nil { + return merr.Status(err), nil + } + return merr.Success(), nil +} diff --git a/internal/distributed/datanode/client/client.go b/internal/distributed/datanode/client/client.go index 2b84d1f36b..bf667f2d69 100644 --- a/internal/distributed/datanode/client/client.go +++ b/internal/distributed/datanode/client/client.go @@ -359,3 +359,9 @@ func (c *Client) DropTask(ctx context.Context, in *workerpb.DropTaskRequest, opt return client.DropTask(ctx, in) }) } + +func (c *Client) SyncFileResource(ctx context.Context, req *internalpb.SyncFileResourceRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + return wrapGrpcCall(ctx, c, func(client DataNodeClient) (*commonpb.Status, error) { + return client.SyncFileResource(ctx, req) + }) +} diff --git a/internal/distributed/datanode/service.go b/internal/distributed/datanode/service.go index f82f83df30..3927ff1982 100644 --- a/internal/distributed/datanode/service.go +++ b/internal/distributed/datanode/service.go @@ -399,3 +399,7 @@ func (s *Server) QueryTask(ctx context.Context, request *workerpb.QueryTaskReque func (s *Server) DropTask(ctx context.Context, request *workerpb.DropTaskRequest) (*commonpb.Status, error) { return s.datanode.DropTask(ctx, request) } + +func (s *Server) SyncFileResource(ctx context.Context, req *internalpb.SyncFileResourceRequest) (*commonpb.Status, error) { + return s.datanode.SyncFileResource(ctx, req) +} diff --git a/internal/distributed/querynode/client/client.go b/internal/distributed/querynode/client/client.go index a39fa9f3f5..f05314b00c 100644 --- a/internal/distributed/querynode/client/client.go +++ b/internal/distributed/querynode/client/client.go @@ -416,3 +416,10 @@ func (c *Client) GetHighlight(ctx context.Context, req *querypb.GetHighlightRequ return client.GetHighlight(ctx, req) }) } + +func (c *Client) SyncFileResource(ctx context.Context, req *internalpb.SyncFileResourceRequest, _ ...grpc.CallOption) (*commonpb.Status, error) { + req = typeutil.Clone(req) + return wrapGrpcCall(ctx, c, func(client querypb.QueryNodeClient) (*commonpb.Status, error) { + return client.SyncFileResource(ctx, req) + }) +} diff --git a/internal/distributed/querynode/service.go b/internal/distributed/querynode/service.go index aa36497f78..7b4898a10c 100644 --- a/internal/distributed/querynode/service.go +++ b/internal/distributed/querynode/service.go @@ -415,3 +415,7 @@ func (s *Server) ValidateAnalyzer(ctx context.Context, req *querypb.ValidateAnal func (s *Server) GetHighlight(ctx context.Context, req *querypb.GetHighlightRequest) (*querypb.GetHighlightResponse, error) { return s.querynode.GetHighlight(ctx, req) } + +func (s *Server) SyncFileResource(ctx context.Context, req *internalpb.SyncFileResourceRequest) (*commonpb.Status, error) { + return s.querynode.SyncFileResource(ctx, req) +} diff --git a/internal/metastore/catalog.go b/internal/metastore/catalog.go index fad2fbd26d..9a3e3f6c8d 100644 --- a/internal/metastore/catalog.go +++ b/internal/metastore/catalog.go @@ -8,6 +8,7 @@ import ( "github.com/milvus-io/milvus/internal/metastore/model" "github.com/milvus-io/milvus/pkg/v2/proto/datapb" "github.com/milvus-io/milvus/pkg/v2/proto/indexpb" + "github.com/milvus-io/milvus/pkg/v2/proto/internalpb" "github.com/milvus-io/milvus/pkg/v2/proto/querypb" "github.com/milvus-io/milvus/pkg/v2/proto/streamingpb" "github.com/milvus-io/milvus/pkg/v2/util/typeutil" @@ -185,9 +186,9 @@ type DataCoordCatalog interface { DropUpdateExternalCollectionTask(ctx context.Context, taskID typeutil.UniqueID) error // Analyzer Resource - SaveFileResource(ctx context.Context, resource *model.FileResource) error - RemoveFileResource(ctx context.Context, resourceID int64) error - ListFileResource(ctx context.Context) ([]*model.FileResource, error) + SaveFileResource(ctx context.Context, resource *internalpb.FileResourceInfo, version uint64) error + RemoveFileResource(ctx context.Context, resourceID int64, version uint64) error + ListFileResource(ctx context.Context) ([]*internalpb.FileResourceInfo, uint64, error) } type QueryCoordCatalog interface { diff --git a/internal/metastore/kv/datacoord/constant.go b/internal/metastore/kv/datacoord/constant.go index 47ac3bbeec..0f7cfbf93d 100644 --- a/internal/metastore/kv/datacoord/constant.go +++ b/internal/metastore/kv/datacoord/constant.go @@ -34,7 +34,8 @@ const ( PartitionStatsCurrentVersionPrefix = MetaPrefix + "/current-partition-stats-version" StatsTaskPrefix = MetaPrefix + "/stats-task" UpdateExternalCollectionTaskPrefix = MetaPrefix + "/update-external-collection-task" - FileResourceMetaPrefix = MetaPrefix + "/file_resource" + FileResourceMetaPrefix = MetaPrefix + "/file_resource_info" + FileResourceVersionKey = MetaPrefix + "/file_resource_version" NonRemoveFlagTomestone = "non-removed" RemoveFlagTomestone = "removed" diff --git a/internal/metastore/kv/datacoord/kv_catalog.go b/internal/metastore/kv/datacoord/kv_catalog.go index 243381487b..8fa89e7ec5 100644 --- a/internal/metastore/kv/datacoord/kv_catalog.go +++ b/internal/metastore/kv/datacoord/kv_catalog.go @@ -41,6 +41,7 @@ import ( "github.com/milvus-io/milvus/pkg/v2/log" "github.com/milvus-io/milvus/pkg/v2/proto/datapb" "github.com/milvus-io/milvus/pkg/v2/proto/indexpb" + "github.com/milvus-io/milvus/pkg/v2/proto/internalpb" "github.com/milvus-io/milvus/pkg/v2/util" "github.com/milvus-io/milvus/pkg/v2/util/etcd" "github.com/milvus-io/milvus/pkg/v2/util/merr" @@ -1002,46 +1003,69 @@ func (kc *Catalog) DropUpdateExternalCollectionTask(ctx context.Context, taskID return kc.MetaKv.Remove(ctx, key) } -func (kc *Catalog) SaveFileResource(ctx context.Context, resource *model.FileResource) error { - k := BuildFileResourceKey(resource.ID) - v, err := proto.Marshal(resource.Marshal()) +func (kc *Catalog) SaveFileResource(ctx context.Context, resource *internalpb.FileResourceInfo, version uint64) error { + kvs := make(map[string]string) + + k := BuildFileResourceKey(resource.Id) + v, err := proto.Marshal(resource) if err != nil { log.Ctx(ctx).Error("failed to marshal resource info", zap.Error(err)) return err } - if err = kc.MetaKv.Save(ctx, k, string(v)); err != nil { + kvs[k] = string(v) + kvs[FileResourceVersionKey] = fmt.Sprint(version) + + if err = kc.MetaKv.MultiSave(ctx, kvs); err != nil { log.Ctx(ctx).Warn("fail to save resource info", zap.String("key", k), zap.Error(err)) return err } return nil } -func (kc *Catalog) RemoveFileResource(ctx context.Context, resourceID int64) error { +func (kc *Catalog) RemoveFileResource(ctx context.Context, resourceID int64, version uint64) error { k := BuildFileResourceKey(resourceID) - if err := kc.MetaKv.Remove(ctx, k); err != nil { + if err := kc.MetaKv.MultiSaveAndRemove(ctx, map[string]string{FileResourceVersionKey: fmt.Sprint(version)}, []string{k}); err != nil { log.Ctx(ctx).Warn("fail to remove resource info", zap.String("key", k), zap.Error(err)) return err } return nil } -func (kc *Catalog) ListFileResource(ctx context.Context) ([]*model.FileResource, error) { +func (kc *Catalog) ListFileResource(ctx context.Context) ([]*internalpb.FileResourceInfo, uint64, error) { _, values, err := kc.MetaKv.LoadWithPrefix(ctx, FileResourceMetaPrefix) if err != nil { - return nil, err + return nil, 0, err } - infos := make([]*model.FileResource, 0, len(values)) + var version uint64 = 0 + exist, err := kc.MetaKv.Has(ctx, FileResourceVersionKey) + if err != nil { + return nil, 0, err + } + + if exist { + strVersion, err := kc.MetaKv.Load(ctx, FileResourceVersionKey) + if err != nil { + return nil, 0, err + } + v, err := strconv.ParseUint(strVersion, 10, 64) + if err != nil { + return nil, 0, err + } + version = v + } + + infos := make([]*internalpb.FileResourceInfo, 0, len(values)) for _, v := range values { - info := &datapb.FileResourceInfo{} + info := &internalpb.FileResourceInfo{} err := proto.Unmarshal([]byte(v), info) if err != nil { - return nil, err + return nil, 0, err } - infos = append(infos, model.UnmarshalFileResourceInfo(info)) + infos = append(infos, info) } - return infos, nil + return infos, version, nil } func BuildFileResourceKey(resourceID typeutil.UniqueID) string { diff --git a/internal/metastore/mocks/mock_datacoord_catalog.go b/internal/metastore/mocks/mock_datacoord_catalog.go index dbec681da4..b9fb05df0c 100644 --- a/internal/metastore/mocks/mock_datacoord_catalog.go +++ b/internal/metastore/mocks/mock_datacoord_catalog.go @@ -8,6 +8,8 @@ import ( datapb "github.com/milvus-io/milvus/pkg/v2/proto/datapb" indexpb "github.com/milvus-io/milvus/pkg/v2/proto/indexpb" + internalpb "github.com/milvus-io/milvus/pkg/v2/proto/internalpb" + metastore "github.com/milvus-io/milvus/internal/metastore" mock "github.com/stretchr/testify/mock" @@ -1320,33 +1322,40 @@ func (_c *DataCoordCatalog_ListCompactionTask_Call) RunAndReturn(run func(contex } // ListFileResource provides a mock function with given fields: ctx -func (_m *DataCoordCatalog) ListFileResource(ctx context.Context) ([]*model.FileResource, error) { +func (_m *DataCoordCatalog) ListFileResource(ctx context.Context) ([]*internalpb.FileResourceInfo, uint64, error) { ret := _m.Called(ctx) if len(ret) == 0 { panic("no return value specified for ListFileResource") } - var r0 []*model.FileResource - var r1 error - if rf, ok := ret.Get(0).(func(context.Context) ([]*model.FileResource, error)); ok { + var r0 []*internalpb.FileResourceInfo + var r1 uint64 + var r2 error + if rf, ok := ret.Get(0).(func(context.Context) ([]*internalpb.FileResourceInfo, uint64, error)); ok { return rf(ctx) } - if rf, ok := ret.Get(0).(func(context.Context) []*model.FileResource); ok { + if rf, ok := ret.Get(0).(func(context.Context) []*internalpb.FileResourceInfo); ok { r0 = rf(ctx) } else { if ret.Get(0) != nil { - r0 = ret.Get(0).([]*model.FileResource) + r0 = ret.Get(0).([]*internalpb.FileResourceInfo) } } - if rf, ok := ret.Get(1).(func(context.Context) error); ok { + if rf, ok := ret.Get(1).(func(context.Context) uint64); ok { r1 = rf(ctx) } else { - r1 = ret.Error(1) + r1 = ret.Get(1).(uint64) } - return r0, r1 + if rf, ok := ret.Get(2).(func(context.Context) error); ok { + r2 = rf(ctx) + } else { + r2 = ret.Error(2) + } + + return r0, r1, r2 } // DataCoordCatalog_ListFileResource_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ListFileResource' @@ -1367,12 +1376,12 @@ func (_c *DataCoordCatalog_ListFileResource_Call) Run(run func(ctx context.Conte return _c } -func (_c *DataCoordCatalog_ListFileResource_Call) Return(_a0 []*model.FileResource, _a1 error) *DataCoordCatalog_ListFileResource_Call { - _c.Call.Return(_a0, _a1) +func (_c *DataCoordCatalog_ListFileResource_Call) Return(_a0 []*internalpb.FileResourceInfo, _a1 uint64, _a2 error) *DataCoordCatalog_ListFileResource_Call { + _c.Call.Return(_a0, _a1, _a2) return _c } -func (_c *DataCoordCatalog_ListFileResource_Call) RunAndReturn(run func(context.Context) ([]*model.FileResource, error)) *DataCoordCatalog_ListFileResource_Call { +func (_c *DataCoordCatalog_ListFileResource_Call) RunAndReturn(run func(context.Context) ([]*internalpb.FileResourceInfo, uint64, error)) *DataCoordCatalog_ListFileResource_Call { _c.Call.Return(run) return _c } @@ -1994,17 +2003,17 @@ func (_c *DataCoordCatalog_MarkChannelDeleted_Call) RunAndReturn(run func(contex return _c } -// RemoveFileResource provides a mock function with given fields: ctx, resourceID -func (_m *DataCoordCatalog) RemoveFileResource(ctx context.Context, resourceID int64) error { - ret := _m.Called(ctx, resourceID) +// RemoveFileResource provides a mock function with given fields: ctx, resourceID, version +func (_m *DataCoordCatalog) RemoveFileResource(ctx context.Context, resourceID int64, version uint64) error { + ret := _m.Called(ctx, resourceID, version) if len(ret) == 0 { panic("no return value specified for RemoveFileResource") } var r0 error - if rf, ok := ret.Get(0).(func(context.Context, int64) error); ok { - r0 = rf(ctx, resourceID) + if rf, ok := ret.Get(0).(func(context.Context, int64, uint64) error); ok { + r0 = rf(ctx, resourceID, version) } else { r0 = ret.Error(0) } @@ -2020,13 +2029,14 @@ type DataCoordCatalog_RemoveFileResource_Call struct { // RemoveFileResource is a helper method to define mock.On call // - ctx context.Context // - resourceID int64 -func (_e *DataCoordCatalog_Expecter) RemoveFileResource(ctx interface{}, resourceID interface{}) *DataCoordCatalog_RemoveFileResource_Call { - return &DataCoordCatalog_RemoveFileResource_Call{Call: _e.mock.On("RemoveFileResource", ctx, resourceID)} +// - version uint64 +func (_e *DataCoordCatalog_Expecter) RemoveFileResource(ctx interface{}, resourceID interface{}, version interface{}) *DataCoordCatalog_RemoveFileResource_Call { + return &DataCoordCatalog_RemoveFileResource_Call{Call: _e.mock.On("RemoveFileResource", ctx, resourceID, version)} } -func (_c *DataCoordCatalog_RemoveFileResource_Call) Run(run func(ctx context.Context, resourceID int64)) *DataCoordCatalog_RemoveFileResource_Call { +func (_c *DataCoordCatalog_RemoveFileResource_Call) Run(run func(ctx context.Context, resourceID int64, version uint64)) *DataCoordCatalog_RemoveFileResource_Call { _c.Call.Run(func(args mock.Arguments) { - run(args[0].(context.Context), args[1].(int64)) + run(args[0].(context.Context), args[1].(int64), args[2].(uint64)) }) return _c } @@ -2036,7 +2046,7 @@ func (_c *DataCoordCatalog_RemoveFileResource_Call) Return(_a0 error) *DataCoord return _c } -func (_c *DataCoordCatalog_RemoveFileResource_Call) RunAndReturn(run func(context.Context, int64) error) *DataCoordCatalog_RemoveFileResource_Call { +func (_c *DataCoordCatalog_RemoveFileResource_Call) RunAndReturn(run func(context.Context, int64, uint64) error) *DataCoordCatalog_RemoveFileResource_Call { _c.Call.Return(run) return _c } @@ -2327,17 +2337,17 @@ func (_c *DataCoordCatalog_SaveDroppedSegmentsInBatch_Call) RunAndReturn(run fun return _c } -// SaveFileResource provides a mock function with given fields: ctx, resource -func (_m *DataCoordCatalog) SaveFileResource(ctx context.Context, resource *model.FileResource) error { - ret := _m.Called(ctx, resource) +// SaveFileResource provides a mock function with given fields: ctx, resource, version +func (_m *DataCoordCatalog) SaveFileResource(ctx context.Context, resource *internalpb.FileResourceInfo, version uint64) error { + ret := _m.Called(ctx, resource, version) if len(ret) == 0 { panic("no return value specified for SaveFileResource") } var r0 error - if rf, ok := ret.Get(0).(func(context.Context, *model.FileResource) error); ok { - r0 = rf(ctx, resource) + if rf, ok := ret.Get(0).(func(context.Context, *internalpb.FileResourceInfo, uint64) error); ok { + r0 = rf(ctx, resource, version) } else { r0 = ret.Error(0) } @@ -2352,14 +2362,15 @@ type DataCoordCatalog_SaveFileResource_Call struct { // SaveFileResource is a helper method to define mock.On call // - ctx context.Context -// - resource *model.FileResource -func (_e *DataCoordCatalog_Expecter) SaveFileResource(ctx interface{}, resource interface{}) *DataCoordCatalog_SaveFileResource_Call { - return &DataCoordCatalog_SaveFileResource_Call{Call: _e.mock.On("SaveFileResource", ctx, resource)} +// - resource *internalpb.FileResourceInfo +// - version uint64 +func (_e *DataCoordCatalog_Expecter) SaveFileResource(ctx interface{}, resource interface{}, version interface{}) *DataCoordCatalog_SaveFileResource_Call { + return &DataCoordCatalog_SaveFileResource_Call{Call: _e.mock.On("SaveFileResource", ctx, resource, version)} } -func (_c *DataCoordCatalog_SaveFileResource_Call) Run(run func(ctx context.Context, resource *model.FileResource)) *DataCoordCatalog_SaveFileResource_Call { +func (_c *DataCoordCatalog_SaveFileResource_Call) Run(run func(ctx context.Context, resource *internalpb.FileResourceInfo, version uint64)) *DataCoordCatalog_SaveFileResource_Call { _c.Call.Run(func(args mock.Arguments) { - run(args[0].(context.Context), args[1].(*model.FileResource)) + run(args[0].(context.Context), args[1].(*internalpb.FileResourceInfo), args[2].(uint64)) }) return _c } @@ -2369,7 +2380,7 @@ func (_c *DataCoordCatalog_SaveFileResource_Call) Return(_a0 error) *DataCoordCa return _c } -func (_c *DataCoordCatalog_SaveFileResource_Call) RunAndReturn(run func(context.Context, *model.FileResource) error) *DataCoordCatalog_SaveFileResource_Call { +func (_c *DataCoordCatalog_SaveFileResource_Call) RunAndReturn(run func(context.Context, *internalpb.FileResourceInfo, uint64) error) *DataCoordCatalog_SaveFileResource_Call { _c.Call.Return(run) return _c } diff --git a/internal/metastore/model/resource.go b/internal/metastore/model/resource.go deleted file mode 100644 index 209bdbb056..0000000000 --- a/internal/metastore/model/resource.go +++ /dev/null @@ -1,35 +0,0 @@ -package model - -import ( - pb "github.com/milvus-io/milvus/pkg/v2/proto/datapb" -) - -type FileResource struct { - ID int64 - Name string - Path string -} - -func (resource *FileResource) Marshal() *pb.FileResourceInfo { - if resource == nil { - return nil - } - - return &pb.FileResourceInfo{ - ResourceId: resource.ID, - Name: resource.Name, - Path: resource.Path, - } -} - -func UnmarshalFileResourceInfo(resource *pb.FileResourceInfo) *FileResource { - if resource == nil { - return nil - } - - return &FileResource{ - ID: resource.ResourceId, - Name: resource.Name, - Path: resource.Path, - } -} diff --git a/internal/mocks/mock_datanode.go b/internal/mocks/mock_datanode.go index 76a8df37ef..7eaf5e6f90 100644 --- a/internal/mocks/mock_datanode.go +++ b/internal/mocks/mock_datanode.go @@ -2065,6 +2065,65 @@ func (_c *MockDataNode_Stop_Call) RunAndReturn(run func() error) *MockDataNode_S return _c } +// SyncFileResource provides a mock function with given fields: _a0, _a1 +func (_m *MockDataNode) SyncFileResource(_a0 context.Context, _a1 *internalpb.SyncFileResourceRequest) (*commonpb.Status, error) { + ret := _m.Called(_a0, _a1) + + if len(ret) == 0 { + panic("no return value specified for SyncFileResource") + } + + var r0 *commonpb.Status + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, *internalpb.SyncFileResourceRequest) (*commonpb.Status, error)); ok { + return rf(_a0, _a1) + } + if rf, ok := ret.Get(0).(func(context.Context, *internalpb.SyncFileResourceRequest) *commonpb.Status); ok { + r0 = rf(_a0, _a1) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*commonpb.Status) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, *internalpb.SyncFileResourceRequest) error); ok { + r1 = rf(_a0, _a1) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockDataNode_SyncFileResource_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SyncFileResource' +type MockDataNode_SyncFileResource_Call struct { + *mock.Call +} + +// SyncFileResource is a helper method to define mock.On call +// - _a0 context.Context +// - _a1 *internalpb.SyncFileResourceRequest +func (_e *MockDataNode_Expecter) SyncFileResource(_a0 interface{}, _a1 interface{}) *MockDataNode_SyncFileResource_Call { + return &MockDataNode_SyncFileResource_Call{Call: _e.mock.On("SyncFileResource", _a0, _a1)} +} + +func (_c *MockDataNode_SyncFileResource_Call) Run(run func(_a0 context.Context, _a1 *internalpb.SyncFileResourceRequest)) *MockDataNode_SyncFileResource_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(*internalpb.SyncFileResourceRequest)) + }) + return _c +} + +func (_c *MockDataNode_SyncFileResource_Call) Return(_a0 *commonpb.Status, _a1 error) *MockDataNode_SyncFileResource_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockDataNode_SyncFileResource_Call) RunAndReturn(run func(context.Context, *internalpb.SyncFileResourceRequest) (*commonpb.Status, error)) *MockDataNode_SyncFileResource_Call { + _c.Call.Return(run) + return _c +} + // SyncSegments provides a mock function with given fields: _a0, _a1 func (_m *MockDataNode) SyncSegments(_a0 context.Context, _a1 *datapb.SyncSegmentsRequest) (*commonpb.Status, error) { ret := _m.Called(_a0, _a1) diff --git a/internal/mocks/mock_datanode_client.go b/internal/mocks/mock_datanode_client.go index 4610d6970c..f4736dc82c 100644 --- a/internal/mocks/mock_datanode_client.go +++ b/internal/mocks/mock_datanode_client.go @@ -2150,6 +2150,80 @@ func (_c *MockDataNodeClient_ShowConfigurations_Call) RunAndReturn(run func(cont return _c } +// SyncFileResource provides a mock function with given fields: ctx, in, opts +func (_m *MockDataNodeClient) SyncFileResource(ctx context.Context, in *internalpb.SyncFileResourceRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + _va := make([]interface{}, len(opts)) + for _i := range opts { + _va[_i] = opts[_i] + } + var _ca []interface{} + _ca = append(_ca, ctx, in) + _ca = append(_ca, _va...) + ret := _m.Called(_ca...) + + if len(ret) == 0 { + panic("no return value specified for SyncFileResource") + } + + var r0 *commonpb.Status + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, *internalpb.SyncFileResourceRequest, ...grpc.CallOption) (*commonpb.Status, error)); ok { + return rf(ctx, in, opts...) + } + if rf, ok := ret.Get(0).(func(context.Context, *internalpb.SyncFileResourceRequest, ...grpc.CallOption) *commonpb.Status); ok { + r0 = rf(ctx, in, opts...) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*commonpb.Status) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, *internalpb.SyncFileResourceRequest, ...grpc.CallOption) error); ok { + r1 = rf(ctx, in, opts...) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockDataNodeClient_SyncFileResource_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SyncFileResource' +type MockDataNodeClient_SyncFileResource_Call struct { + *mock.Call +} + +// SyncFileResource is a helper method to define mock.On call +// - ctx context.Context +// - in *internalpb.SyncFileResourceRequest +// - opts ...grpc.CallOption +func (_e *MockDataNodeClient_Expecter) SyncFileResource(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_SyncFileResource_Call { + return &MockDataNodeClient_SyncFileResource_Call{Call: _e.mock.On("SyncFileResource", + append([]interface{}{ctx, in}, opts...)...)} +} + +func (_c *MockDataNodeClient_SyncFileResource_Call) Run(run func(ctx context.Context, in *internalpb.SyncFileResourceRequest, opts ...grpc.CallOption)) *MockDataNodeClient_SyncFileResource_Call { + _c.Call.Run(func(args mock.Arguments) { + variadicArgs := make([]grpc.CallOption, len(args)-2) + for i, a := range args[2:] { + if a != nil { + variadicArgs[i] = a.(grpc.CallOption) + } + } + run(args[0].(context.Context), args[1].(*internalpb.SyncFileResourceRequest), variadicArgs...) + }) + return _c +} + +func (_c *MockDataNodeClient_SyncFileResource_Call) Return(_a0 *commonpb.Status, _a1 error) *MockDataNodeClient_SyncFileResource_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockDataNodeClient_SyncFileResource_Call) RunAndReturn(run func(context.Context, *internalpb.SyncFileResourceRequest, ...grpc.CallOption) (*commonpb.Status, error)) *MockDataNodeClient_SyncFileResource_Call { + _c.Call.Return(run) + return _c +} + // SyncSegments provides a mock function with given fields: ctx, in, opts func (_m *MockDataNodeClient) SyncSegments(ctx context.Context, in *datapb.SyncSegmentsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { _va := make([]interface{}, len(opts)) diff --git a/internal/mocks/mock_mixcoord.go b/internal/mocks/mock_mixcoord.go index 9539df2dfb..6561ec2f41 100644 --- a/internal/mocks/mock_mixcoord.go +++ b/internal/mocks/mock_mixcoord.go @@ -8436,6 +8436,54 @@ func (_c *MixCoord_SyncNewCreatedPartition_Call) RunAndReturn(run func(context.C return _c } +// SyncQcFileResource provides a mock function with given fields: ctx, resources, version +func (_m *MixCoord) SyncQcFileResource(ctx context.Context, resources []*internalpb.FileResourceInfo, version uint64) error { + ret := _m.Called(ctx, resources, version) + + if len(ret) == 0 { + panic("no return value specified for SyncQcFileResource") + } + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, []*internalpb.FileResourceInfo, uint64) error); ok { + r0 = rf(ctx, resources, version) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// MixCoord_SyncQcFileResource_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SyncQcFileResource' +type MixCoord_SyncQcFileResource_Call struct { + *mock.Call +} + +// SyncQcFileResource is a helper method to define mock.On call +// - ctx context.Context +// - resources []*internalpb.FileResourceInfo +// - version uint64 +func (_e *MixCoord_Expecter) SyncQcFileResource(ctx interface{}, resources interface{}, version interface{}) *MixCoord_SyncQcFileResource_Call { + return &MixCoord_SyncQcFileResource_Call{Call: _e.mock.On("SyncQcFileResource", ctx, resources, version)} +} + +func (_c *MixCoord_SyncQcFileResource_Call) Run(run func(ctx context.Context, resources []*internalpb.FileResourceInfo, version uint64)) *MixCoord_SyncQcFileResource_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].([]*internalpb.FileResourceInfo), args[2].(uint64)) + }) + return _c +} + +func (_c *MixCoord_SyncQcFileResource_Call) Return(_a0 error) *MixCoord_SyncQcFileResource_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MixCoord_SyncQcFileResource_Call) RunAndReturn(run func(context.Context, []*internalpb.FileResourceInfo, uint64) error) *MixCoord_SyncQcFileResource_Call { + _c.Call.Return(run) + return _c +} + // TransferChannel provides a mock function with given fields: _a0, _a1 func (_m *MixCoord) TransferChannel(_a0 context.Context, _a1 *querypb.TransferChannelRequest) (*commonpb.Status, error) { ret := _m.Called(_a0, _a1) diff --git a/internal/mocks/mock_querynode.go b/internal/mocks/mock_querynode.go index ba6d9016c1..4b52e4e97e 100644 --- a/internal/mocks/mock_querynode.go +++ b/internal/mocks/mock_querynode.go @@ -1817,6 +1817,65 @@ func (_c *MockQueryNode_SyncDistribution_Call) RunAndReturn(run func(context.Con return _c } +// SyncFileResource provides a mock function with given fields: _a0, _a1 +func (_m *MockQueryNode) SyncFileResource(_a0 context.Context, _a1 *internalpb.SyncFileResourceRequest) (*commonpb.Status, error) { + ret := _m.Called(_a0, _a1) + + if len(ret) == 0 { + panic("no return value specified for SyncFileResource") + } + + var r0 *commonpb.Status + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, *internalpb.SyncFileResourceRequest) (*commonpb.Status, error)); ok { + return rf(_a0, _a1) + } + if rf, ok := ret.Get(0).(func(context.Context, *internalpb.SyncFileResourceRequest) *commonpb.Status); ok { + r0 = rf(_a0, _a1) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*commonpb.Status) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, *internalpb.SyncFileResourceRequest) error); ok { + r1 = rf(_a0, _a1) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockQueryNode_SyncFileResource_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SyncFileResource' +type MockQueryNode_SyncFileResource_Call struct { + *mock.Call +} + +// SyncFileResource is a helper method to define mock.On call +// - _a0 context.Context +// - _a1 *internalpb.SyncFileResourceRequest +func (_e *MockQueryNode_Expecter) SyncFileResource(_a0 interface{}, _a1 interface{}) *MockQueryNode_SyncFileResource_Call { + return &MockQueryNode_SyncFileResource_Call{Call: _e.mock.On("SyncFileResource", _a0, _a1)} +} + +func (_c *MockQueryNode_SyncFileResource_Call) Run(run func(_a0 context.Context, _a1 *internalpb.SyncFileResourceRequest)) *MockQueryNode_SyncFileResource_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(*internalpb.SyncFileResourceRequest)) + }) + return _c +} + +func (_c *MockQueryNode_SyncFileResource_Call) Return(_a0 *commonpb.Status, _a1 error) *MockQueryNode_SyncFileResource_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockQueryNode_SyncFileResource_Call) RunAndReturn(run func(context.Context, *internalpb.SyncFileResourceRequest) (*commonpb.Status, error)) *MockQueryNode_SyncFileResource_Call { + _c.Call.Return(run) + return _c +} + // SyncReplicaSegments provides a mock function with given fields: _a0, _a1 func (_m *MockQueryNode) SyncReplicaSegments(_a0 context.Context, _a1 *querypb.SyncReplicaSegmentsRequest) (*commonpb.Status, error) { ret := _m.Called(_a0, _a1) diff --git a/internal/mocks/mock_querynode_client.go b/internal/mocks/mock_querynode_client.go index f7054b94ba..139380e8eb 100644 --- a/internal/mocks/mock_querynode_client.go +++ b/internal/mocks/mock_querynode_client.go @@ -1926,6 +1926,80 @@ func (_c *MockQueryNodeClient_SyncDistribution_Call) RunAndReturn(run func(conte return _c } +// SyncFileResource provides a mock function with given fields: ctx, in, opts +func (_m *MockQueryNodeClient) SyncFileResource(ctx context.Context, in *internalpb.SyncFileResourceRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + _va := make([]interface{}, len(opts)) + for _i := range opts { + _va[_i] = opts[_i] + } + var _ca []interface{} + _ca = append(_ca, ctx, in) + _ca = append(_ca, _va...) + ret := _m.Called(_ca...) + + if len(ret) == 0 { + panic("no return value specified for SyncFileResource") + } + + var r0 *commonpb.Status + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, *internalpb.SyncFileResourceRequest, ...grpc.CallOption) (*commonpb.Status, error)); ok { + return rf(ctx, in, opts...) + } + if rf, ok := ret.Get(0).(func(context.Context, *internalpb.SyncFileResourceRequest, ...grpc.CallOption) *commonpb.Status); ok { + r0 = rf(ctx, in, opts...) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*commonpb.Status) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, *internalpb.SyncFileResourceRequest, ...grpc.CallOption) error); ok { + r1 = rf(ctx, in, opts...) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockQueryNodeClient_SyncFileResource_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SyncFileResource' +type MockQueryNodeClient_SyncFileResource_Call struct { + *mock.Call +} + +// SyncFileResource is a helper method to define mock.On call +// - ctx context.Context +// - in *internalpb.SyncFileResourceRequest +// - opts ...grpc.CallOption +func (_e *MockQueryNodeClient_Expecter) SyncFileResource(ctx interface{}, in interface{}, opts ...interface{}) *MockQueryNodeClient_SyncFileResource_Call { + return &MockQueryNodeClient_SyncFileResource_Call{Call: _e.mock.On("SyncFileResource", + append([]interface{}{ctx, in}, opts...)...)} +} + +func (_c *MockQueryNodeClient_SyncFileResource_Call) Run(run func(ctx context.Context, in *internalpb.SyncFileResourceRequest, opts ...grpc.CallOption)) *MockQueryNodeClient_SyncFileResource_Call { + _c.Call.Run(func(args mock.Arguments) { + variadicArgs := make([]grpc.CallOption, len(args)-2) + for i, a := range args[2:] { + if a != nil { + variadicArgs[i] = a.(grpc.CallOption) + } + } + run(args[0].(context.Context), args[1].(*internalpb.SyncFileResourceRequest), variadicArgs...) + }) + return _c +} + +func (_c *MockQueryNodeClient_SyncFileResource_Call) Return(_a0 *commonpb.Status, _a1 error) *MockQueryNodeClient_SyncFileResource_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockQueryNodeClient_SyncFileResource_Call) RunAndReturn(run func(context.Context, *internalpb.SyncFileResourceRequest, ...grpc.CallOption) (*commonpb.Status, error)) *MockQueryNodeClient_SyncFileResource_Call { + _c.Call.Return(run) + return _c +} + // SyncReplicaSegments provides a mock function with given fields: ctx, in, opts func (_m *MockQueryNodeClient) SyncReplicaSegments(ctx context.Context, in *querypb.SyncReplicaSegmentsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { _va := make([]interface{}, len(opts)) diff --git a/internal/querycoordv2/mocks/mock_querynode.go b/internal/querycoordv2/mocks/mock_querynode.go index 7bd6e44f44..f3a82f4f55 100644 --- a/internal/querycoordv2/mocks/mock_querynode.go +++ b/internal/querycoordv2/mocks/mock_querynode.go @@ -1480,6 +1480,65 @@ func (_c *MockQueryNodeServer_SyncDistribution_Call) RunAndReturn(run func(conte return _c } +// SyncFileResource provides a mock function with given fields: _a0, _a1 +func (_m *MockQueryNodeServer) SyncFileResource(_a0 context.Context, _a1 *internalpb.SyncFileResourceRequest) (*commonpb.Status, error) { + ret := _m.Called(_a0, _a1) + + if len(ret) == 0 { + panic("no return value specified for SyncFileResource") + } + + var r0 *commonpb.Status + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, *internalpb.SyncFileResourceRequest) (*commonpb.Status, error)); ok { + return rf(_a0, _a1) + } + if rf, ok := ret.Get(0).(func(context.Context, *internalpb.SyncFileResourceRequest) *commonpb.Status); ok { + r0 = rf(_a0, _a1) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*commonpb.Status) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, *internalpb.SyncFileResourceRequest) error); ok { + r1 = rf(_a0, _a1) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockQueryNodeServer_SyncFileResource_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SyncFileResource' +type MockQueryNodeServer_SyncFileResource_Call struct { + *mock.Call +} + +// SyncFileResource is a helper method to define mock.On call +// - _a0 context.Context +// - _a1 *internalpb.SyncFileResourceRequest +func (_e *MockQueryNodeServer_Expecter) SyncFileResource(_a0 interface{}, _a1 interface{}) *MockQueryNodeServer_SyncFileResource_Call { + return &MockQueryNodeServer_SyncFileResource_Call{Call: _e.mock.On("SyncFileResource", _a0, _a1)} +} + +func (_c *MockQueryNodeServer_SyncFileResource_Call) Run(run func(_a0 context.Context, _a1 *internalpb.SyncFileResourceRequest)) *MockQueryNodeServer_SyncFileResource_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(*internalpb.SyncFileResourceRequest)) + }) + return _c +} + +func (_c *MockQueryNodeServer_SyncFileResource_Call) Return(_a0 *commonpb.Status, _a1 error) *MockQueryNodeServer_SyncFileResource_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockQueryNodeServer_SyncFileResource_Call) RunAndReturn(run func(context.Context, *internalpb.SyncFileResourceRequest) (*commonpb.Status, error)) *MockQueryNodeServer_SyncFileResource_Call { + _c.Call.Return(run) + return _c +} + // SyncReplicaSegments provides a mock function with given fields: _a0, _a1 func (_m *MockQueryNodeServer) SyncReplicaSegments(_a0 context.Context, _a1 *querypb.SyncReplicaSegmentsRequest) (*commonpb.Status, error) { ret := _m.Called(_a0, _a1) diff --git a/internal/querycoordv2/observers/file_resource_observer.go b/internal/querycoordv2/observers/file_resource_observer.go new file mode 100644 index 0000000000..f41121759d --- /dev/null +++ b/internal/querycoordv2/observers/file_resource_observer.go @@ -0,0 +1,167 @@ +// 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 observers + +import ( + "context" + "sync" + "time" + + "go.uber.org/zap" + + "github.com/milvus-io/milvus/internal/querycoordv2/session" + "github.com/milvus-io/milvus/internal/util/fileresource" + "github.com/milvus-io/milvus/pkg/v2/log" + "github.com/milvus-io/milvus/pkg/v2/proto/internalpb" + "github.com/milvus-io/milvus/pkg/v2/util/conc" + "github.com/milvus-io/milvus/pkg/v2/util/lock" + "github.com/milvus-io/milvus/pkg/v2/util/merr" + "github.com/milvus-io/milvus/pkg/v2/util/paramtable" +) + +type FileResourceObserver struct { + lock.RWMutex + resources []*internalpb.FileResourceInfo + version uint64 + + ctx context.Context + + distribution map[int64]uint64 + + // version distribution + nodeManager *session.NodeManager + cluster session.Cluster + + notifyCh chan struct{} + closeCh chan struct{} + wg sync.WaitGroup + sf conc.Singleflight[any] + once sync.Once + closeOnce sync.Once +} + +func NewFileResourceObserver(ctx context.Context, nodeManager *session.NodeManager, cluster session.Cluster) *FileResourceObserver { + return &FileResourceObserver{ + ctx: ctx, + nodeManager: nodeManager, + cluster: cluster, + distribution: map[int64]uint64{}, + + notifyCh: make(chan struct{}, 1), + closeCh: make(chan struct{}), + sf: conc.Singleflight[any]{}, + } +} + +func (m *FileResourceObserver) getResources() ([]*internalpb.FileResourceInfo, uint64) { + m.RLock() + defer m.RUnlock() + return m.resources, m.version +} + +func (m *FileResourceObserver) syncLoop() { + defer m.wg.Done() + for { + select { + case <-m.notifyCh: + resources, version := m.getResources() + err := m.sync(resources, version) + if err != nil { + // retry if error exist + m.sf.Do("retry", func() (any, error) { + time.Sleep(5 * time.Second) + m.Notify() + return nil, nil + }) + } + case <-m.closeCh: + log.Info("file resource observer close") + return + case <-m.ctx.Done(): + log.Info("file resource observer context done") + return + } + } +} + +func (m *FileResourceObserver) Start() { + if fileresource.IsSyncMode(paramtable.Get().QueryCoordCfg.FileResourceMode.GetValue()) { + m.once.Do(func() { + m.wg.Add(1) + go m.syncLoop() + m.Notify() + }) + } +} + +func (m *FileResourceObserver) Stop() { + m.closeOnce.Do(func() { + close(m.closeCh) + m.wg.Wait() + }) +} + +func (m *FileResourceObserver) Notify() { + select { + case m.notifyCh <- struct{}{}: + default: + } +} + +func (m *FileResourceObserver) sync(resources []*internalpb.FileResourceInfo, version uint64) error { + nodes := m.nodeManager.GetAll() + var syncErr error + + newDistribution := make(map[int64]uint64) + for _, node := range nodes { + newDistribution[node.ID()] = m.distribution[node.ID()] + if m.distribution[node.ID()] < version { + status, err := m.cluster.SyncFileResource(m.ctx, node.ID(), &internalpb.SyncFileResourceRequest{ + Resources: resources, + Version: version, + }) + if err != nil { + log.Warn("sync file resource failed", zap.Int64("nodeID", node.ID()), zap.Error(err)) + syncErr = err + continue + } + + if err = merr.Error(status); err != nil { + log.Warn("sync file resource failed", zap.Int64("nodeID", node.ID()), zap.Error(err)) + syncErr = err + continue + } + + newDistribution[node.ID()] = version + log.Info("finish sync file resource to query node", zap.Int64("node", node.ID()), zap.Uint64("version", version)) + } + } + m.distribution = newDistribution + + if syncErr != nil { + return syncErr + } + return nil +} + +func (m *FileResourceObserver) UpdateResources(resources []*internalpb.FileResourceInfo, version uint64) { + m.Lock() + defer m.Unlock() + m.resources = resources + m.version = version + m.Notify() +} diff --git a/internal/querycoordv2/observers/file_resource_observer_test.go b/internal/querycoordv2/observers/file_resource_observer_test.go new file mode 100644 index 0000000000..3b0c06a314 --- /dev/null +++ b/internal/querycoordv2/observers/file_resource_observer_test.go @@ -0,0 +1,261 @@ +// 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 observers + +import ( + "context" + "testing" + "time" + + "github.com/stretchr/testify/suite" + + "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + "github.com/milvus-io/milvus/internal/querycoordv2/session" + "github.com/milvus-io/milvus/pkg/v2/proto/internalpb" + "github.com/milvus-io/milvus/pkg/v2/util/paramtable" +) + +type FileResourceObserverSuite struct { + suite.Suite + + ctx context.Context + observer *FileResourceObserver + + // Real components + nodeManager *session.NodeManager + mockCluster *session.MockCluster +} + +func (suite *FileResourceObserverSuite) SetupSuite() { + paramtable.Init() +} + +func (suite *FileResourceObserverSuite) SetupTest() { + suite.ctx = context.Background() + + // Create real NodeManager and mock Cluster + suite.nodeManager = session.NewNodeManager() + suite.mockCluster = session.NewMockCluster(suite.T()) + + // Create FileResourceObserver + suite.observer = NewFileResourceObserver(suite.ctx, suite.nodeManager, suite.mockCluster) +} + +func (suite *FileResourceObserverSuite) TearDownTest() { + // Assert mock expectations for cluster only + suite.mockCluster.AssertExpectations(suite.T()) +} + +func (suite *FileResourceObserverSuite) TestNewFileResourceObserver() { + observer := NewFileResourceObserver(suite.ctx, suite.nodeManager, suite.mockCluster) + suite.NotNil(observer) + suite.Equal(suite.ctx, observer.ctx) + suite.Equal(suite.nodeManager, observer.nodeManager) + suite.Equal(suite.mockCluster, observer.cluster) + suite.NotNil(observer.distribution) + suite.NotNil(observer.notifyCh) +} + +func (suite *FileResourceObserverSuite) TestNotify() { + // Test notify without blocking + suite.observer.Notify() + + // Verify notification was sent + select { + case <-suite.observer.notifyCh: + // Expected + case <-time.After(100 * time.Millisecond): + suite.Fail("Expected notification but got none") + } + + // Test notify when channel is full (should not block) + suite.observer.Notify() + suite.observer.Notify() // This should not block even if channel is full +} + +func (suite *FileResourceObserverSuite) TestUpdateResources() { + resources := []*internalpb.FileResourceInfo{ + { + Id: 1, + Name: "test.file", + Path: "/test/test.file", + }, + } + version := uint64(100) + + // Update resources + suite.observer.UpdateResources(resources, version) + + // Verify resources and version are updated + resultResources, resultVersion := suite.observer.getResources() + suite.Equal(resources, resultResources) + suite.Equal(version, resultVersion) + + // Verify notification was sent + select { + case <-suite.observer.notifyCh: + // Expected + case <-time.After(100 * time.Millisecond): + suite.Fail("Expected notification but got none") + } +} + +func (suite *FileResourceObserverSuite) TestGetResources() { + resources := []*internalpb.FileResourceInfo{ + { + Id: 1, + Name: "test.file", + Path: "/test/test.file", + }, + } + version := uint64(100) + + // Set resources directly + suite.observer.resources = resources + suite.observer.version = version + + // Get resources + resultResources, resultVersion := suite.observer.getResources() + suite.Equal(resources, resultResources) + suite.Equal(version, resultVersion) +} + +func (suite *FileResourceObserverSuite) TestSync_Success() { + // Prepare test data + resources := []*internalpb.FileResourceInfo{ + { + Id: 1, + Name: "test.file", + Path: "/test/test.file", + }, + } + version := uint64(100) + + // Real nodeManager starts with empty node list + // Execute sync + err := suite.observer.sync(resources, version) + + // Verify no error since no nodes to sync + suite.NoError(err) +} + +func (suite *FileResourceObserverSuite) TestSync_WithNodes() { + // Add some nodes to the real nodeManager + node1 := session.NewNodeInfo(session.ImmutableNodeInfo{ + NodeID: 1, + Address: "localhost:19530", + Hostname: "node1", + }) + node2 := session.NewNodeInfo(session.ImmutableNodeInfo{ + NodeID: 2, + Address: "localhost:19531", + Hostname: "node2", + }) + suite.nodeManager.Add(node1) + suite.nodeManager.Add(node2) + + // Prepare test data + resources := []*internalpb.FileResourceInfo{ + { + Id: 1, + Name: "test.file", + Path: "/test/test.file", + }, + } + version := uint64(100) + + // Mock cluster sync calls for each node + req1 := &internalpb.SyncFileResourceRequest{Resources: resources, Version: version} + req2 := &internalpb.SyncFileResourceRequest{Resources: resources, Version: version} + suite.mockCluster.EXPECT().SyncFileResource(suite.ctx, int64(1), req1).Return(&commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil) + suite.mockCluster.EXPECT().SyncFileResource(suite.ctx, int64(2), req2).Return(&commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil) + + // Execute sync + err := suite.observer.sync(resources, version) + + // Verify no error + suite.NoError(err) + + // Verify version was updated for both nodes + suite.Equal(version, suite.observer.distribution[1]) + suite.Equal(version, suite.observer.distribution[2]) +} + +func (suite *FileResourceObserverSuite) TestSync_NodeSyncError() { + // Prepare test data + resources := []*internalpb.FileResourceInfo{} + version := uint64(100) + + // Real nodeManager starts with empty node list + // Execute sync + err := suite.observer.sync(resources, version) + + // Verify no error since no nodes to sync + suite.NoError(err) +} + +func (suite *FileResourceObserverSuite) TestStart_SyncModeEnabled() { + // Mock paramtable to enable sync mode + paramtable.Get().QueryCoordCfg.FileResourceMode.SwapTempValue("sync") + + // Start observer - real nodeManager starts with empty node list + suite.observer.Start() + + // Wait a bit for goroutine to start + time.Sleep(10 * time.Millisecond) + + // Verify observer started (no specific expectations to check since real nodeManager is used) + // The test passes if no panic or error occurs +} + +func (suite *FileResourceObserverSuite) TestStart_SyncModeDisabled() { + // Mock paramtable to disable sync mode + paramtable.Get().QueryCoordCfg.FileResourceMode.SwapTempValue("async") + + // Start observer - no mocks should be called + suite.observer.Start() + + // Wait a bit + time.Sleep(10 * time.Millisecond) + + // No sync should have been triggered, so no expectations needed +} + +func (suite *FileResourceObserverSuite) TestMultipleUpdatesAndNotifications() { + // Test multiple rapid updates + for i := 0; i < 5; i++ { + resources := []*internalpb.FileResourceInfo{ + { + Id: int64(i + 1), + Name: "test.file", + Path: "/test/test.file", + }, + } + version := uint64(i + 1) + + suite.observer.UpdateResources(resources, version) + + // Verify latest update + resultResources, resultVersion := suite.observer.getResources() + suite.Equal(resources, resultResources) + suite.Equal(version, resultVersion) + } +} + +func TestFileResourceObserverSuite(t *testing.T) { + suite.Run(t, new(FileResourceObserverSuite)) +} diff --git a/internal/querycoordv2/server.go b/internal/querycoordv2/server.go index 288382d26f..37da8dc0fb 100644 --- a/internal/querycoordv2/server.go +++ b/internal/querycoordv2/server.go @@ -59,6 +59,7 @@ import ( "github.com/milvus-io/milvus/pkg/v2/kv" "github.com/milvus-io/milvus/pkg/v2/log" "github.com/milvus-io/milvus/pkg/v2/metrics" + "github.com/milvus-io/milvus/pkg/v2/proto/internalpb" "github.com/milvus-io/milvus/pkg/v2/util" "github.com/milvus-io/milvus/pkg/v2/util/expr" "github.com/milvus-io/milvus/pkg/v2/util/metricsinfo" @@ -111,11 +112,12 @@ type Server struct { checkerController *checkers.CheckerController // Observers - collectionObserver *observers.CollectionObserver - targetObserver *observers.TargetObserver - replicaObserver *observers.ReplicaObserver - resourceObserver *observers.ResourceObserver - leaderCacheObserver *observers.LeaderCacheObserver + collectionObserver *observers.CollectionObserver + targetObserver *observers.TargetObserver + replicaObserver *observers.ReplicaObserver + resourceObserver *observers.ResourceObserver + leaderCacheObserver *observers.LeaderCacheObserver + fileResourceObserver *observers.FileResourceObserver getBalancerFunc checkers.GetBalancerFunc balancerMap map[string]balance.Balance @@ -467,6 +469,8 @@ func (s *Server) initObserver() { s.leaderCacheObserver = observers.NewLeaderCacheObserver( s.proxyClientManager, ) + + s.fileResourceObserver = observers.NewFileResourceObserver(s.ctx, s.nodeMgr, s.cluster) } func (s *Server) afterStart() {} @@ -530,6 +534,7 @@ func (s *Server) startServerLoop() { s.targetObserver.Start() s.replicaObserver.Start() s.resourceObserver.Start() + s.fileResourceObserver.Start() log.Info("start task scheduler...") s.taskScheduler.Start() @@ -585,6 +590,9 @@ func (s *Server) Stop() error { if s.leaderCacheObserver != nil { s.leaderCacheObserver.Stop() } + if s.fileResourceObserver != nil { + s.fileResourceObserver.Stop() + } if s.distController != nil { log.Info("stop dist controller...") @@ -687,6 +695,7 @@ func (s *Server) watchNodes(revision int64) { Labels: event.Session.GetServerLabel(), })) s.handleNodeUp(nodeID) + s.fileResourceObserver.Notify() case sessionutil.SessionUpdateEvent: log.Info("stopping the node") @@ -934,3 +943,8 @@ func (s *Server) watchLoadConfigChanges() { }) paramtable.Get().Watch(paramtable.Get().QueryCoordCfg.ClusterLevelLoadResourceGroups.Key, rgHandler) } + +func (s *Server) SyncFileResource(ctx context.Context, resources []*internalpb.FileResourceInfo, version uint64) error { + s.fileResourceObserver.UpdateResources(resources, version) + return nil +} diff --git a/internal/querycoordv2/session/cluster.go b/internal/querycoordv2/session/cluster.go index 624eff7412..1551b94f46 100644 --- a/internal/querycoordv2/session/cluster.go +++ b/internal/querycoordv2/session/cluster.go @@ -31,6 +31,7 @@ import ( grpcquerynodeclient "github.com/milvus-io/milvus/internal/distributed/querynode/client" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/pkg/v2/log" + "github.com/milvus-io/milvus/pkg/v2/proto/internalpb" "github.com/milvus-io/milvus/pkg/v2/proto/querypb" "github.com/milvus-io/milvus/pkg/v2/util/paramtable" ) @@ -55,6 +56,7 @@ type Cluster interface { DropIndex(ctx context.Context, nodeID int64, req *querypb.DropIndexRequest) (*commonpb.Status, error) RunAnalyzer(ctx context.Context, nodeID int64, req *querypb.RunAnalyzerRequest) (*milvuspb.RunAnalyzerResponse, error) ValidateAnalyzer(ctx context.Context, nodeID int64, req *querypb.ValidateAnalyzerRequest) (*commonpb.Status, error) + SyncFileResource(ctx context.Context, nodeID int64, req *internalpb.SyncFileResourceRequest) (*commonpb.Status, error) Start() Stop() } @@ -315,6 +317,21 @@ func (c *QueryCluster) ValidateAnalyzer(ctx context.Context, nodeID int64, req * return resp, err } +func (c *QueryCluster) SyncFileResource(ctx context.Context, nodeID int64, req *internalpb.SyncFileResourceRequest) (*commonpb.Status, error) { + var ( + resp *commonpb.Status + err error + ) + + err1 := c.send(ctx, nodeID, func(cli types.QueryNodeClient) { + resp, err = cli.SyncFileResource(ctx, req) + }) + if err1 != nil { + return nil, err1 + } + return resp, err +} + func (c *QueryCluster) send(ctx context.Context, nodeID int64, fn func(cli types.QueryNodeClient)) error { node := c.nodeManager.Get(nodeID) if node == nil { diff --git a/internal/querycoordv2/session/mock_cluster.go b/internal/querycoordv2/session/mock_cluster.go index ce404a5bab..ffd5d6e508 100644 --- a/internal/querycoordv2/session/mock_cluster.go +++ b/internal/querycoordv2/session/mock_cluster.go @@ -7,6 +7,8 @@ import ( commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + internalpb "github.com/milvus-io/milvus/pkg/v2/proto/internalpb" + milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" mock "github.com/stretchr/testify/mock" @@ -690,6 +692,66 @@ func (_c *MockCluster_SyncDistribution_Call) RunAndReturn(run func(context.Conte return _c } +// SyncFileResource provides a mock function with given fields: ctx, nodeID, req +func (_m *MockCluster) SyncFileResource(ctx context.Context, nodeID int64, req *internalpb.SyncFileResourceRequest) (*commonpb.Status, error) { + ret := _m.Called(ctx, nodeID, req) + + if len(ret) == 0 { + panic("no return value specified for SyncFileResource") + } + + var r0 *commonpb.Status + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, int64, *internalpb.SyncFileResourceRequest) (*commonpb.Status, error)); ok { + return rf(ctx, nodeID, req) + } + if rf, ok := ret.Get(0).(func(context.Context, int64, *internalpb.SyncFileResourceRequest) *commonpb.Status); ok { + r0 = rf(ctx, nodeID, req) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*commonpb.Status) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, int64, *internalpb.SyncFileResourceRequest) error); ok { + r1 = rf(ctx, nodeID, req) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockCluster_SyncFileResource_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SyncFileResource' +type MockCluster_SyncFileResource_Call struct { + *mock.Call +} + +// SyncFileResource is a helper method to define mock.On call +// - ctx context.Context +// - nodeID int64 +// - req *internalpb.SyncFileResourceRequest +func (_e *MockCluster_Expecter) SyncFileResource(ctx interface{}, nodeID interface{}, req interface{}) *MockCluster_SyncFileResource_Call { + return &MockCluster_SyncFileResource_Call{Call: _e.mock.On("SyncFileResource", ctx, nodeID, req)} +} + +func (_c *MockCluster_SyncFileResource_Call) Run(run func(ctx context.Context, nodeID int64, req *internalpb.SyncFileResourceRequest)) *MockCluster_SyncFileResource_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(int64), args[2].(*internalpb.SyncFileResourceRequest)) + }) + return _c +} + +func (_c *MockCluster_SyncFileResource_Call) Return(_a0 *commonpb.Status, _a1 error) *MockCluster_SyncFileResource_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockCluster_SyncFileResource_Call) RunAndReturn(run func(context.Context, int64, *internalpb.SyncFileResourceRequest) (*commonpb.Status, error)) *MockCluster_SyncFileResource_Call { + _c.Call.Return(run) + return _c +} + // UnsubDmChannel provides a mock function with given fields: ctx, nodeID, req func (_m *MockCluster) UnsubDmChannel(ctx context.Context, nodeID int64, req *querypb.UnsubDmChannelRequest) (*commonpb.Status, error) { ret := _m.Called(ctx, nodeID, req) diff --git a/internal/querynodev2/server.go b/internal/querynodev2/server.go index e76fafad81..47d8f4eb17 100644 --- a/internal/querynodev2/server.go +++ b/internal/querynodev2/server.go @@ -56,6 +56,7 @@ import ( "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/analyzer" "github.com/milvus-io/milvus/internal/util/dependency" + "github.com/milvus-io/milvus/internal/util/fileresource" "github.com/milvus-io/milvus/internal/util/hookutil" "github.com/milvus-io/milvus/internal/util/initcore" "github.com/milvus-io/milvus/internal/util/searchutil/optimizers" @@ -353,6 +354,8 @@ func (node *QueryNode) Init() error { // init pipeline manager node.pipelineManager = pipeline.NewManager(node.manager, node.dispClient, node.delegators) + fileresource.InitManager(node.chunkManager, fileresource.ParseMode(paramtable.Get().QueryCoordCfg.FileResourceMode.GetValue())) + err = initcore.InitQueryNode(node.ctx) if err != nil { log.Error("QueryNode init segcore failed", zap.Error(err)) diff --git a/internal/querynodev2/services.go b/internal/querynodev2/services.go index 53ba81fa7e..459fdf13d7 100644 --- a/internal/querynodev2/services.go +++ b/internal/querynodev2/services.go @@ -38,6 +38,7 @@ import ( "github.com/milvus-io/milvus/internal/querynodev2/tasks" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/analyzer" + "github.com/milvus-io/milvus/internal/util/fileresource" "github.com/milvus-io/milvus/internal/util/searchutil/scheduler" "github.com/milvus-io/milvus/internal/util/streamrpc" "github.com/milvus-io/milvus/pkg/v2/common" @@ -1774,3 +1775,20 @@ func (node *QueryNode) GetHighlight(ctx context.Context, req *querypb.GetHighlig Results: results, }, nil } + +func (node *QueryNode) SyncFileResource(ctx context.Context, req *internalpb.SyncFileResourceRequest) (*commonpb.Status, error) { + log := log.Ctx(ctx).With(zap.Uint64("version", req.GetVersion())) + log.Info("sync file resource") + + if err := node.lifetime.Add(merr.IsHealthy); err != nil { + log.Warn("failed to sync file resource, QueryNode is not healthy") + return merr.Status(err), nil + } + defer node.lifetime.Done() + + err := fileresource.Sync(req.GetResources()) + if err != nil { + return merr.Status(err), nil + } + return merr.Success(), nil +} diff --git a/internal/streamingnode/server/server.go b/internal/streamingnode/server/server.go index 0a013032c2..0316380e38 100644 --- a/internal/streamingnode/server/server.go +++ b/internal/streamingnode/server/server.go @@ -10,6 +10,7 @@ import ( "github.com/milvus-io/milvus/internal/streamingnode/server/resource" "github.com/milvus-io/milvus/internal/streamingnode/server/service" "github.com/milvus-io/milvus/internal/streamingnode/server/walmanager" + "github.com/milvus-io/milvus/internal/util/fileresource" "github.com/milvus-io/milvus/internal/util/initcore" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/v2/log" @@ -17,6 +18,7 @@ import ( _ "github.com/milvus-io/milvus/pkg/v2/streaming/walimpls/impls/kafka" _ "github.com/milvus-io/milvus/pkg/v2/streaming/walimpls/impls/pulsar" _ "github.com/milvus-io/milvus/pkg/v2/streaming/walimpls/impls/rmq" + "github.com/milvus-io/milvus/pkg/v2/util/paramtable" ) // Server is the streamingnode server. @@ -42,6 +44,9 @@ func (s *Server) init() { // init all service. s.initService() + // init file resource manager + fileresource.InitManager(resource.Resource().ChunkManager(), fileresource.ParseMode(paramtable.Get().QueryCoordCfg.FileResourceMode.GetValue())) + log.Info("init query segcore...") if err := initcore.InitQueryNode(context.TODO()); err != nil { panic(fmt.Sprintf("init query node segcore failed, %+v", err)) diff --git a/internal/types/types.go b/internal/types/types.go index 5042d82bcb..57e9f43e1c 100644 --- a/internal/types/types.go +++ b/internal/types/types.go @@ -291,8 +291,11 @@ type MixCoord interface { // GetMetrics notifies MixCoordComponent to collect metrics for specified component GetQcMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) + // GetMetrics notifies MixCoordComponent to collect metrics for specified component NotifyDropPartition(ctx context.Context, channel string, partitionIDs []int64) error + + SyncQcFileResource(ctx context.Context, resources []*internalpb.FileResourceInfo, version uint64) error } // MixCoordComponent is used by grpc server of MixCoord diff --git a/internal/util/fileresource/manager.go b/internal/util/fileresource/manager.go new file mode 100644 index 0000000000..49b62db074 --- /dev/null +++ b/internal/util/fileresource/manager.go @@ -0,0 +1,288 @@ +/* + * # 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 fileresource + +import ( + "context" + "fmt" + "io" + "os" + "path" + "sync" + "time" + + "go.uber.org/zap" + + "github.com/milvus-io/milvus/internal/storage" + "github.com/milvus-io/milvus/internal/util/pathutil" + "github.com/milvus-io/milvus/pkg/v2/log" + "github.com/milvus-io/milvus/pkg/v2/proto/internalpb" + "github.com/milvus-io/milvus/pkg/v2/util/conc" + "github.com/milvus-io/milvus/pkg/v2/util/paramtable" + "github.com/milvus-io/milvus/pkg/v2/util/typeutil" +) + +var ( + GlobalFileManager Manager + once sync.Once +) + +func InitManager(storage storage.ChunkManager, mode Mode) { + once.Do(func() { + m := NewManager(storage, mode) + GlobalFileManager = m + }) +} + +func Sync(resourceList []*internalpb.FileResourceInfo) error { + if GlobalFileManager == nil { + log.Error("sync file resource to file manager not init") + return nil + } + + return GlobalFileManager.Sync(resourceList) +} + +// Manager manage file resource +type Manager interface { + // sync resource to local + Sync(resourceList []*internalpb.FileResourceInfo) error + + Download(downloader storage.ChunkManager, resources ...*internalpb.FileResourceInfo) error + Release(resources ...*internalpb.FileResourceInfo) + Mode() Mode +} + +type Mode int + +// manager mode +// Sync: sync when file resource list changed and download all file resource to local. +// Ref: install before use and delete local file if no one own it. +// Close: skip all action but don't return error. +const ( + SyncMode Mode = iota + 1 + RefMode + CloseMode +) + +type BaseManager struct { + localPath string +} + +func (m *BaseManager) Sync(resourceList []*internalpb.FileResourceInfo) error { return nil } +func (m *BaseManager) Download(downloader storage.ChunkManager, resources ...*internalpb.FileResourceInfo) error { + return nil +} +func (m *BaseManager) Release(resources ...*internalpb.FileResourceInfo) {} +func (m *BaseManager) Mode() Mode { return CloseMode } + +// Manager with Sync Mode +// mixcoord should sync all node after add or remove file resource. +type SyncManager struct { + BaseManager + sync.RWMutex + downloader storage.ChunkManager + resourceSet map[int64]struct{} +} + +// sync file to local if file mode was Sync +func (m *SyncManager) Sync(resourceList []*internalpb.FileResourceInfo) error { + m.Lock() + defer m.Unlock() + + ctx := context.Background() + newSet := make(map[int64]struct{}) + for _, resource := range resourceList { + newSet[resource.GetId()] = struct{}{} + if _, ok := m.resourceSet[resource.GetId()]; ok { + continue + } + + localResourcePath := path.Join(m.localPath, fmt.Sprint(resource.GetId())) + + err := os.MkdirAll(localResourcePath, os.ModePerm) + if err != nil { + return err + } + + reader, err := m.downloader.Reader(ctx, resource.GetPath()) + if err != nil { + log.Info("download resource failed", zap.String("path", resource.GetPath()), zap.Error(err)) + return err + } + defer reader.Close() + + fileName := path.Join(localResourcePath, path.Base(resource.GetPath())) + file, err := os.Create(fileName) + if err != nil { + return err + } + defer file.Close() + + if _, err = io.Copy(file, reader); err != nil { + log.Info("download resource failed", zap.String("path", resource.GetPath()), zap.Error(err)) + return err + } + log.Info("sync file to local", zap.String("name", fileName)) + } + + for resourceId := range m.resourceSet { + if _, ok := newSet[resourceId]; !ok { + err := os.RemoveAll(path.Join(m.localPath, fmt.Sprint(resourceId))) + if err != nil { + log.Warn("remove local resource failed", zap.Error(err)) + newSet[resourceId] = struct{}{} + } + } + } + m.resourceSet = newSet + return nil +} + +func (m *SyncManager) Mode() Mode { return SyncMode } + +func NewSyncManager(downloader storage.ChunkManager) *SyncManager { + return &SyncManager{ + BaseManager: BaseManager{localPath: pathutil.GetPath(pathutil.FileResourcePath, paramtable.GetNodeID())}, + downloader: downloader, + resourceSet: make(map[int64]struct{}), + } +} + +type RefManager struct { + BaseManager + sync.RWMutex + ref map[string]int + + finished *typeutil.ConcurrentMap[string, bool] + sf *conc.Singleflight[interface{}] +} + +func (m *RefManager) Download(downloader storage.ChunkManager, resources ...*internalpb.FileResourceInfo) error { + m.Lock() + // inc ref count and set storage name with storage root path + for _, resource := range resources { + key := fmt.Sprintf("%s/%d", downloader.RootPath(), resource.GetId()) + resource.StorageName = downloader.RootPath() + m.ref[key] += 1 + } + m.Unlock() + + ctx := context.Background() + for _, r := range resources { + resource := r + key := fmt.Sprintf("%s/%d", downloader.RootPath(), resource.GetId()) + if ok, exist := m.finished.Get(key); exist && ok { + continue + } + + _, err, _ := m.sf.Do(key, func() (interface{}, error) { + if ok, exist := m.finished.Get(key); exist && ok { + return nil, nil + } + + localResourcePath := path.Join(m.localPath, key) + + err := os.MkdirAll(localResourcePath, os.ModePerm) + if err != nil { + return nil, err + } + + reader, err := downloader.Reader(ctx, resource.GetPath()) + if err != nil { + log.Info("download resource failed", zap.String("path", resource.GetPath()), zap.Error(err)) + return nil, err + } + defer reader.Close() + + fileName := path.Join(localResourcePath, path.Base(resource.GetPath())) + file, err := os.Create(fileName) + if err != nil { + return nil, err + } + defer file.Close() + + if _, err = io.Copy(file, reader); err != nil { + return nil, err + } + m.finished.Insert(key, true) + return nil, nil + }) + + if err != nil { + return err + } + } + return nil +} + +func (m *RefManager) Release(resources ...*internalpb.FileResourceInfo) { + m.Lock() + defer m.Unlock() + // dec ref + for _, resource := range resources { + key := fmt.Sprintf("%s/%d", resource.GetStorageName(), resource.GetId()) + m.ref[key] -= 1 + } +} + +func (m *RefManager) Mode() Mode { return RefMode } + +// clean file resource with no ref. +func (m *RefManager) CleanResource() { + m.Lock() + defer m.Unlock() + + for key, cnt := range m.ref { + if cnt <= 0 { + localResourcePath := path.Join(m.localPath, key) + os.RemoveAll(localResourcePath) + delete(m.ref, key) + m.finished.Remove(key) + } + } +} + +func (m *RefManager) GcLoop() { + ticker := time.NewTicker(15 * time.Minute) + + for range ticker.C { + m.CleanResource() + } +} + +func NewRefManger() *RefManager { + return &RefManager{ + BaseManager: BaseManager{localPath: pathutil.GetPath(pathutil.FileResourcePath, paramtable.GetNodeID())}, + ref: map[string]int{}, + finished: typeutil.NewConcurrentMap[string, bool](), + sf: &conc.Singleflight[interface{}]{}, + } +} + +func NewManager(storage storage.ChunkManager, mode Mode) Manager { + switch mode { + case CloseMode: + return &BaseManager{} + case SyncMode: + return NewSyncManager(storage) + default: + panic(fmt.Sprintf("Unknown file resource mananger mod: %v", mode)) + } +} diff --git a/internal/util/fileresource/manager_test.go b/internal/util/fileresource/manager_test.go new file mode 100644 index 0000000000..fa91203d16 --- /dev/null +++ b/internal/util/fileresource/manager_test.go @@ -0,0 +1,338 @@ +/* + * # 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 fileresource + +import ( + "context" + "fmt" + "io" + "os" + "path" + "strings" + "sync" + "testing" + + "github.com/stretchr/testify/suite" + + "github.com/milvus-io/milvus/internal/mocks" + "github.com/milvus-io/milvus/pkg/v2/proto/internalpb" + "github.com/milvus-io/milvus/pkg/v2/util/conc" + "github.com/milvus-io/milvus/pkg/v2/util/typeutil" +) + +// MockReader implements storage.FileReader using composition +type MockReader struct { + io.Reader + io.Closer + io.ReaderAt + io.Seeker + size int64 +} + +func (mr *MockReader) Size() (int64, error) { + return mr.size, nil +} + +func newMockReader(s string) *MockReader { + ioReader := strings.NewReader(s) + return &MockReader{Reader: ioReader, Closer: io.NopCloser(ioReader), size: int64(len(s))} +} + +// BaseManagerSuite tests BaseManager +type BaseManagerSuite struct { + suite.Suite + manager *BaseManager +} + +func (suite *BaseManagerSuite) SetupTest() { + suite.manager = &BaseManager{localPath: "/tmp/test"} +} + +func (suite *BaseManagerSuite) TestSync() { + resources := []*internalpb.FileResourceInfo{ + {Id: 1, Name: "test.file", Path: "/test/test.file"}, + } + + err := suite.manager.Sync(resources) + suite.NoError(err) +} + +func (suite *BaseManagerSuite) TestDownload() { + mockStorage := mocks.NewChunkManager(suite.T()) + resources := []*internalpb.FileResourceInfo{ + {Id: 1, Name: "test.file", Path: "/test/test.file"}, + } + + err := suite.manager.Download(mockStorage, resources...) + suite.NoError(err) +} + +func (suite *BaseManagerSuite) TestRelease() { + resources := []*internalpb.FileResourceInfo{ + {Id: 1, Name: "test.file", Path: "/test/test.file"}, + } + + suite.manager.Release(resources...) + // Should not panic +} + +func (suite *BaseManagerSuite) TestMode() { + mode := suite.manager.Mode() + suite.Equal(CloseMode, mode) +} + +func TestBaseManagerSuite(t *testing.T) { + suite.Run(t, new(BaseManagerSuite)) +} + +// SyncManagerSuite tests SyncManager +type SyncManagerSuite struct { + suite.Suite + manager *SyncManager + mockStorage *mocks.ChunkManager + tempDir string +} + +func (suite *SyncManagerSuite) SetupTest() { + // Create temporary directory for tests + var err error + suite.tempDir, err = os.MkdirTemp(os.TempDir(), "fileresource_test_sync_*") + suite.NoError(err) + + suite.mockStorage = mocks.NewChunkManager(suite.T()) + suite.manager = &SyncManager{ + BaseManager: BaseManager{localPath: suite.tempDir}, + downloader: suite.mockStorage, + resourceSet: make(map[int64]struct{}), + } +} + +func (suite *SyncManagerSuite) TearDownTest() { + if suite.tempDir != "" { + os.RemoveAll(suite.tempDir) + } +} + +func (suite *SyncManagerSuite) TestSync_Success() { + resources := []*internalpb.FileResourceInfo{ + {Id: 1, Name: "test1.file", Path: "/storage/test1.file"}, + {Id: 2, Name: "test2.file", Path: "/storage/test2.file"}, + } + + // Mock the Reader calls + suite.mockStorage.EXPECT().Reader(context.Background(), "/storage/test1.file").Return(newMockReader("test content 1"), nil) + suite.mockStorage.EXPECT().Reader(context.Background(), "/storage/test2.file").Return(newMockReader("test content 2"), nil) + + err := suite.manager.Sync(resources) + suite.NoError(err) + + // Verify files were created + file1Path := path.Join(suite.tempDir, "1", "test1.file") + file2Path := path.Join(suite.tempDir, "2", "test2.file") + + suite.FileExists(file1Path) + suite.FileExists(file2Path) + + // Verify content + content1, err := os.ReadFile(file1Path) + suite.NoError(err) + suite.Equal("test content 1", string(content1)) + + content2, err := os.ReadFile(file2Path) + suite.NoError(err) + suite.Equal("test content 2", string(content2)) +} + +func (suite *SyncManagerSuite) TestSync_ReaderError() { + resources := []*internalpb.FileResourceInfo{ + {Id: 1, Name: "test.file", Path: "/storage/nonexistent.file"}, + } + + // Mock reader to return error + suite.mockStorage.EXPECT().Reader(context.Background(), "/storage/nonexistent.file").Return(nil, io.ErrUnexpectedEOF) + + err := suite.manager.Sync(resources) + suite.Error(err) + suite.ErrorIs(err, io.ErrUnexpectedEOF) +} + +func (suite *SyncManagerSuite) TestMode() { + mode := suite.manager.Mode() + suite.Equal(SyncMode, mode) +} + +func TestSyncManagerSuite(t *testing.T) { + suite.Run(t, new(SyncManagerSuite)) +} + +// RefManagerSuite tests RefManager +type RefManagerSuite struct { + suite.Suite + manager *RefManager + mockStorage *mocks.ChunkManager + tempDir string +} + +func (suite *RefManagerSuite) SetupTest() { + // Create temporary directory for tests + var err error + suite.tempDir, err = os.MkdirTemp(os.TempDir(), "fileresource_test_ref_*") + suite.NoError(err) + + suite.mockStorage = mocks.NewChunkManager(suite.T()) + suite.manager = &RefManager{ + BaseManager: BaseManager{localPath: suite.tempDir}, + ref: map[string]int{}, + finished: typeutil.NewConcurrentMap[string, bool](), + sf: &conc.Singleflight[interface{}]{}, + } +} + +func (suite *RefManagerSuite) TearDownTest() { + if suite.tempDir != "" { + os.RemoveAll(suite.tempDir) + } +} + +func (suite *RefManagerSuite) TestNormal() { + resources := []*internalpb.FileResourceInfo{ + {Id: 1, Name: "test", Path: "/storage/test.file"}, + } + + // Set up mock + suite.mockStorage.EXPECT().RootPath().Return("/test/storage") + suite.mockStorage.EXPECT().Reader(context.Background(), "/storage/test.file").Return(newMockReader("test content"), nil) + + err := suite.manager.Download(suite.mockStorage, resources...) + suite.Require().NoError(err) + + // Verify ref count + key := "/test/storage/1" + suite.Equal(1, suite.manager.ref[key]) + + // Verify storage name is set + suite.Equal("/test/storage", resources[0].StorageName) + + // Verify file was downloaded + // {local_path}/{storage_name}/{resource_id}/{file_name} + filePath := path.Join(suite.tempDir, "/test/storage", fmt.Sprint(1), path.Base(resources[0].GetPath())) + suite.FileExists(filePath) + + content, err := os.ReadFile(filePath) + suite.NoError(err) + suite.Equal("test content", string(content)) + + // release and clean all file + suite.manager.Release(resources...) + suite.manager.CleanResource() + suite.NoFileExists(filePath) +} + +func (suite *RefManagerSuite) TestMode() { + mode := suite.manager.Mode() + suite.Equal(RefMode, mode) +} + +func TestRefManagerSuite(t *testing.T) { + suite.Run(t, new(RefManagerSuite)) +} + +// ManagerFactorySuite tests NewManager factory function +type ManagerFactorySuite struct { + suite.Suite + mockStorage *mocks.ChunkManager +} + +func (suite *ManagerFactorySuite) SetupTest() { + suite.mockStorage = mocks.NewChunkManager(suite.T()) +} + +func (suite *ManagerFactorySuite) TestNewManager_BaseManager() { + manager := NewManager(suite.mockStorage, CloseMode) + suite.IsType(&BaseManager{}, manager) + suite.Equal(CloseMode, manager.Mode()) +} + +func (suite *ManagerFactorySuite) TestNewManager_SyncManager() { + manager := NewManager(suite.mockStorage, SyncMode) + suite.IsType(&SyncManager{}, manager) + suite.Equal(SyncMode, manager.Mode()) +} + +func (suite *ManagerFactorySuite) TestNewManager_InvalidMode() { + suite.Panics(func() { + NewManager(suite.mockStorage, Mode(999)) + }) +} + +func TestManagerFactorySuite(t *testing.T) { + suite.Run(t, new(ManagerFactorySuite)) +} + +// GlobalFunctionsSuite tests global functions +type GlobalFunctionsSuite struct { + suite.Suite + mockStorage *mocks.ChunkManager +} + +func (suite *GlobalFunctionsSuite) SetupTest() { + suite.mockStorage = mocks.NewChunkManager(suite.T()) + // Reset global state + GlobalFileManager = nil + once = sync.Once{} +} + +func (suite *GlobalFunctionsSuite) TestInitManager() { + InitManager(suite.mockStorage, SyncMode) + + suite.NotNil(GlobalFileManager) + suite.Equal(SyncMode, GlobalFileManager.Mode()) + + // Test that calling InitManager again doesn't change the manager + oldManager := GlobalFileManager + InitManager(suite.mockStorage, RefMode) + suite.Equal(oldManager, GlobalFileManager) + suite.Equal(SyncMode, GlobalFileManager.Mode()) // Should still be SyncMode +} + +func (suite *GlobalFunctionsSuite) TestSync_NotInitialized() { + GlobalFileManager = nil + + resources := []*internalpb.FileResourceInfo{ + {Id: 1, Name: "test.file", Path: "/test/test.file"}, + } + + err := Sync(resources) + suite.NoError(err) // Should not error when not initialized +} + +func (suite *GlobalFunctionsSuite) TestSync_Initialized() { + InitManager(suite.mockStorage, CloseMode) + + resources := []*internalpb.FileResourceInfo{ + {Id: 1, Name: "test.file", Path: "/test/test.file"}, + } + + err := Sync(resources) + suite.NoError(err) +} + +func TestGlobalFunctionsSuite(t *testing.T) { + suite.Run(t, new(GlobalFunctionsSuite)) +} diff --git a/internal/util/fileresource/util.go b/internal/util/fileresource/util.go new file mode 100644 index 0000000000..bd7bd14dac --- /dev/null +++ b/internal/util/fileresource/util.go @@ -0,0 +1,36 @@ +/* + * # 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 fileresource + +func ParseMode(value string) Mode { + switch value { + case "close": + return CloseMode + case "sync": + return SyncMode + case "ref": + return RefMode + default: + return CloseMode + } +} + +func IsSyncMode(value string) bool { + return value == "sync" +} diff --git a/internal/util/mock/grpc_datanode_client.go b/internal/util/mock/grpc_datanode_client.go index 0f761041b3..115b5463e5 100644 --- a/internal/util/mock/grpc_datanode_client.go +++ b/internal/util/mock/grpc_datanode_client.go @@ -153,3 +153,7 @@ func (m *GrpcDataNodeClient) QuerySlot(ctx context.Context, req *datapb.QuerySlo func (m *GrpcDataNodeClient) DropCompactionPlan(ctx context.Context, req *datapb.DropCompactionPlanRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { return &commonpb.Status{}, m.Err } + +func (m *GrpcDataNodeClient) SyncFileResource(ctx context.Context, req *internalpb.SyncFileResourceRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + return &commonpb.Status{}, m.Err +} diff --git a/internal/util/mock/grpc_querynode_client.go b/internal/util/mock/grpc_querynode_client.go index 22f7fd245d..e9748fa175 100644 --- a/internal/util/mock/grpc_querynode_client.go +++ b/internal/util/mock/grpc_querynode_client.go @@ -157,3 +157,7 @@ func (m *GrpcQueryNodeClient) Close() error { func (m *GrpcQueryNodeClient) DropIndex(ctx context.Context, in *querypb.DropIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { return &commonpb.Status{}, m.Err } + +func (m *GrpcQueryNodeClient) SyncFileResource(ctx context.Context, req *internalpb.SyncFileResourceRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + return &commonpb.Status{}, m.Err +} diff --git a/internal/util/pathutil/path_util.go b/internal/util/pathutil/path_util.go index 42e042638a..11c8c94fbe 100644 --- a/internal/util/pathutil/path_util.go +++ b/internal/util/pathutil/path_util.go @@ -17,13 +17,15 @@ const ( LocalChunkPath BM25Path RootCachePath + FileResourcePath ) const ( - CachePathPrefix = "cache" - GrowingMMapPathPrefix = "growing_mmap" - LocalChunkPathPrefix = "local_chunk" - BM25PathPrefix = "bm25" + CachePathPrefix = "cache" + GrowingMMapPathPrefix = "growing_mmap" + LocalChunkPathPrefix = "local_chunk" + BM25PathPrefix = "bm25" + FileResourcePathPrefix = "file_resource" ) func GetPath(pathType PathType, nodeID int64) string { @@ -37,6 +39,8 @@ func GetPath(pathType PathType, nodeID int64) string { path = filepath.Join(path, fmt.Sprintf("%d", nodeID), LocalChunkPathPrefix) case BM25Path: path = filepath.Join(path, fmt.Sprintf("%d", nodeID), BM25PathPrefix) + case FileResourcePath: + path = filepath.Join(path, fmt.Sprintf("%d", nodeID), FileResourcePathPrefix) case RootCachePath: } log.Info("Get path for", zap.Any("pathType", pathType), zap.Int64("nodeID", nodeID), zap.String("path", path)) diff --git a/internal/util/wrappers/qn_wrapper.go b/internal/util/wrappers/qn_wrapper.go index a48aacba27..81f8370dd0 100644 --- a/internal/util/wrappers/qn_wrapper.go +++ b/internal/util/wrappers/qn_wrapper.go @@ -172,6 +172,10 @@ func (qn *qnServerWrapper) ValidateAnalyzer(ctx context.Context, in *querypb.Val return qn.QueryNode.ValidateAnalyzer(ctx, in) } +func (qn *qnServerWrapper) SyncFileResource(ctx context.Context, in *internalpb.SyncFileResourceRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + return qn.QueryNode.SyncFileResource(ctx, in) +} + func WrapQueryNodeServerAsClient(qn types.QueryNode) types.QueryNodeClient { return &qnServerWrapper{ QueryNode: qn, diff --git a/pkg/proto/data_coord.proto b/pkg/proto/data_coord.proto index 4de61ef1b7..e591108da8 100644 --- a/pkg/proto/data_coord.proto +++ b/pkg/proto/data_coord.proto @@ -145,6 +145,9 @@ service DataNode { rpc QuerySlot(QuerySlotRequest) returns(QuerySlotResponse) {} rpc DropCompactionPlan(DropCompactionPlanRequest) returns(common.Status) {} + + // file resource + rpc SyncFileResource(internal.SyncFileResourceRequest) returns(common.Status) {} } message FlushRequest { diff --git a/pkg/proto/datapb/data_coord.pb.go b/pkg/proto/datapb/data_coord.pb.go index 880a8f8823..c8d1ccb2f3 100644 --- a/pkg/proto/datapb/data_coord.pb.go +++ b/pkg/proto/datapb/data_coord.pb.go @@ -11359,7 +11359,7 @@ var file_data_coord_proto_rawDesc = []byte{ 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x32, - 0xbf, 0x0f, 0x0a, 0x08, 0x44, 0x61, 0x74, 0x61, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x6c, 0x0a, 0x12, + 0xa2, 0x10, 0x0a, 0x08, 0x44, 0x61, 0x74, 0x61, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x6c, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, @@ -11483,10 +11483,16 @@ var file_data_coord_proto_rawDesc = []byte{ 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, - 0x00, 0x42, 0x31, 0x5a, 0x2f, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x64, 0x61, - 0x74, 0x61, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x00, 0x12, 0x61, 0x0a, 0x10, 0x53, 0x79, 0x6e, 0x63, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x79, + 0x6e, 0x63, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x22, 0x00, 0x42, 0x31, 0x5a, 0x2f, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, + 0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2f, 0x64, 0x61, 0x74, 0x61, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -11673,27 +11679,28 @@ var file_data_coord_proto_goTypes = []interface{}{ (*milvuspb.ListFileResourcesRequest)(nil), // 166: milvus.proto.milvus.ListFileResourcesRequest (*milvuspb.GetComponentStatesRequest)(nil), // 167: milvus.proto.milvus.GetComponentStatesRequest (*internalpb.GetStatisticsChannelRequest)(nil), // 168: milvus.proto.internal.GetStatisticsChannelRequest - (*milvuspb.StringResponse)(nil), // 169: milvus.proto.milvus.StringResponse - (*milvuspb.GetFlushAllStateResponse)(nil), // 170: milvus.proto.milvus.GetFlushAllStateResponse - (*internalpb.ShowConfigurationsResponse)(nil), // 171: milvus.proto.internal.ShowConfigurationsResponse - (*milvuspb.GetMetricsResponse)(nil), // 172: milvus.proto.milvus.GetMetricsResponse - (*milvuspb.ManualCompactionResponse)(nil), // 173: milvus.proto.milvus.ManualCompactionResponse - (*milvuspb.GetCompactionStateResponse)(nil), // 174: milvus.proto.milvus.GetCompactionStateResponse - (*milvuspb.GetCompactionPlansResponse)(nil), // 175: milvus.proto.milvus.GetCompactionPlansResponse - (*milvuspb.GetFlushStateResponse)(nil), // 176: milvus.proto.milvus.GetFlushStateResponse - (*milvuspb.CheckHealthResponse)(nil), // 177: milvus.proto.milvus.CheckHealthResponse - (*indexpb.GetIndexStateResponse)(nil), // 178: milvus.proto.index.GetIndexStateResponse - (*indexpb.GetSegmentIndexStateResponse)(nil), // 179: milvus.proto.index.GetSegmentIndexStateResponse - (*indexpb.GetIndexInfoResponse)(nil), // 180: milvus.proto.index.GetIndexInfoResponse - (*indexpb.DescribeIndexResponse)(nil), // 181: milvus.proto.index.DescribeIndexResponse - (*indexpb.GetIndexStatisticsResponse)(nil), // 182: milvus.proto.index.GetIndexStatisticsResponse - (*indexpb.GetIndexBuildProgressResponse)(nil), // 183: milvus.proto.index.GetIndexBuildProgressResponse - (*indexpb.ListIndexesResponse)(nil), // 184: milvus.proto.index.ListIndexesResponse - (*internalpb.ImportResponse)(nil), // 185: milvus.proto.internal.ImportResponse - (*internalpb.GetImportProgressResponse)(nil), // 186: milvus.proto.internal.GetImportProgressResponse - (*internalpb.ListImportsResponse)(nil), // 187: milvus.proto.internal.ListImportsResponse - (*milvuspb.ListFileResourcesResponse)(nil), // 188: milvus.proto.milvus.ListFileResourcesResponse - (*milvuspb.ComponentStates)(nil), // 189: milvus.proto.milvus.ComponentStates + (*internalpb.SyncFileResourceRequest)(nil), // 169: milvus.proto.internal.SyncFileResourceRequest + (*milvuspb.StringResponse)(nil), // 170: milvus.proto.milvus.StringResponse + (*milvuspb.GetFlushAllStateResponse)(nil), // 171: milvus.proto.milvus.GetFlushAllStateResponse + (*internalpb.ShowConfigurationsResponse)(nil), // 172: milvus.proto.internal.ShowConfigurationsResponse + (*milvuspb.GetMetricsResponse)(nil), // 173: milvus.proto.milvus.GetMetricsResponse + (*milvuspb.ManualCompactionResponse)(nil), // 174: milvus.proto.milvus.ManualCompactionResponse + (*milvuspb.GetCompactionStateResponse)(nil), // 175: milvus.proto.milvus.GetCompactionStateResponse + (*milvuspb.GetCompactionPlansResponse)(nil), // 176: milvus.proto.milvus.GetCompactionPlansResponse + (*milvuspb.GetFlushStateResponse)(nil), // 177: milvus.proto.milvus.GetFlushStateResponse + (*milvuspb.CheckHealthResponse)(nil), // 178: milvus.proto.milvus.CheckHealthResponse + (*indexpb.GetIndexStateResponse)(nil), // 179: milvus.proto.index.GetIndexStateResponse + (*indexpb.GetSegmentIndexStateResponse)(nil), // 180: milvus.proto.index.GetSegmentIndexStateResponse + (*indexpb.GetIndexInfoResponse)(nil), // 181: milvus.proto.index.GetIndexInfoResponse + (*indexpb.DescribeIndexResponse)(nil), // 182: milvus.proto.index.DescribeIndexResponse + (*indexpb.GetIndexStatisticsResponse)(nil), // 183: milvus.proto.index.GetIndexStatisticsResponse + (*indexpb.GetIndexBuildProgressResponse)(nil), // 184: milvus.proto.index.GetIndexBuildProgressResponse + (*indexpb.ListIndexesResponse)(nil), // 185: milvus.proto.index.ListIndexesResponse + (*internalpb.ImportResponse)(nil), // 186: milvus.proto.internal.ImportResponse + (*internalpb.GetImportProgressResponse)(nil), // 187: milvus.proto.internal.GetImportProgressResponse + (*internalpb.ListImportsResponse)(nil), // 188: milvus.proto.internal.ListImportsResponse + (*milvuspb.ListFileResourcesResponse)(nil), // 189: milvus.proto.milvus.ListFileResourcesResponse + (*milvuspb.ComponentStates)(nil), // 190: milvus.proto.milvus.ComponentStates } var file_data_coord_proto_depIdxs = []int32{ 129, // 0: milvus.proto.data.FlushRequest.base:type_name -> milvus.proto.common.MsgBase @@ -11984,79 +11991,81 @@ var file_data_coord_proto_depIdxs = []int32{ 104, // 285: milvus.proto.data.DataNode.DropImport:input_type -> milvus.proto.data.DropImportRequest 110, // 286: milvus.proto.data.DataNode.QuerySlot:input_type -> milvus.proto.data.QuerySlotRequest 114, // 287: milvus.proto.data.DataNode.DropCompactionPlan:input_type -> milvus.proto.data.DropCompactionPlanRequest - 10, // 288: milvus.proto.data.DataCoord.Flush:output_type -> milvus.proto.data.FlushResponse - 14, // 289: milvus.proto.data.DataCoord.FlushAll:output_type -> milvus.proto.data.FlushAllResponse - 116, // 290: milvus.proto.data.DataCoord.CreateExternalCollection:output_type -> milvus.proto.data.CreateExternalCollectionResponse - 18, // 291: milvus.proto.data.DataCoord.AllocSegment:output_type -> milvus.proto.data.AllocSegmentResponse - 21, // 292: milvus.proto.data.DataCoord.AssignSegmentID:output_type -> milvus.proto.data.AssignSegmentIDResponse - 26, // 293: milvus.proto.data.DataCoord.GetSegmentInfo:output_type -> milvus.proto.data.GetSegmentInfoResponse - 24, // 294: milvus.proto.data.DataCoord.GetSegmentStates:output_type -> milvus.proto.data.GetSegmentStatesResponse - 28, // 295: milvus.proto.data.DataCoord.GetInsertBinlogPaths:output_type -> milvus.proto.data.GetInsertBinlogPathsResponse - 30, // 296: milvus.proto.data.DataCoord.GetCollectionStatistics:output_type -> milvus.proto.data.GetCollectionStatisticsResponse - 32, // 297: milvus.proto.data.DataCoord.GetPartitionStatistics:output_type -> milvus.proto.data.GetPartitionStatisticsResponse - 169, // 298: milvus.proto.data.DataCoord.GetSegmentInfoChannel:output_type -> milvus.proto.milvus.StringResponse - 130, // 299: milvus.proto.data.DataCoord.SaveBinlogPaths:output_type -> milvus.proto.common.Status - 50, // 300: milvus.proto.data.DataCoord.GetRecoveryInfo:output_type -> milvus.proto.data.GetRecoveryInfoResponse - 52, // 301: milvus.proto.data.DataCoord.GetRecoveryInfoV2:output_type -> milvus.proto.data.GetRecoveryInfoResponseV2 - 55, // 302: milvus.proto.data.DataCoord.GetChannelRecoveryInfo:output_type -> milvus.proto.data.GetChannelRecoveryInfoResponse - 60, // 303: milvus.proto.data.DataCoord.GetFlushedSegments:output_type -> milvus.proto.data.GetFlushedSegmentsResponse - 58, // 304: milvus.proto.data.DataCoord.GetSegmentsByStates:output_type -> milvus.proto.data.GetSegmentsByStatesResponse - 170, // 305: milvus.proto.data.DataCoord.GetFlushAllState:output_type -> milvus.proto.milvus.GetFlushAllStateResponse - 171, // 306: milvus.proto.data.DataCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse - 172, // 307: milvus.proto.data.DataCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse - 173, // 308: milvus.proto.data.DataCoord.ManualCompaction:output_type -> milvus.proto.milvus.ManualCompactionResponse - 174, // 309: milvus.proto.data.DataCoord.GetCompactionState:output_type -> milvus.proto.milvus.GetCompactionStateResponse - 175, // 310: milvus.proto.data.DataCoord.GetCompactionStateWithPlans:output_type -> milvus.proto.milvus.GetCompactionPlansResponse - 73, // 311: milvus.proto.data.DataCoord.WatchChannels:output_type -> milvus.proto.data.WatchChannelsResponse - 176, // 312: milvus.proto.data.DataCoord.GetFlushState:output_type -> milvus.proto.milvus.GetFlushStateResponse - 78, // 313: milvus.proto.data.DataCoord.DropVirtualChannel:output_type -> milvus.proto.data.DropVirtualChannelResponse - 75, // 314: milvus.proto.data.DataCoord.SetSegmentState:output_type -> milvus.proto.data.SetSegmentStateResponse - 130, // 315: milvus.proto.data.DataCoord.UpdateSegmentStatistics:output_type -> milvus.proto.common.Status - 130, // 316: milvus.proto.data.DataCoord.UpdateChannelCheckpoint:output_type -> milvus.proto.common.Status - 130, // 317: milvus.proto.data.DataCoord.MarkSegmentsDropped:output_type -> milvus.proto.common.Status - 130, // 318: milvus.proto.data.DataCoord.BroadcastAlteredCollection:output_type -> milvus.proto.common.Status - 177, // 319: milvus.proto.data.DataCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse - 130, // 320: milvus.proto.data.DataCoord.CreateIndex:output_type -> milvus.proto.common.Status - 130, // 321: milvus.proto.data.DataCoord.AlterIndex:output_type -> milvus.proto.common.Status - 178, // 322: milvus.proto.data.DataCoord.GetIndexState:output_type -> milvus.proto.index.GetIndexStateResponse - 179, // 323: milvus.proto.data.DataCoord.GetSegmentIndexState:output_type -> milvus.proto.index.GetSegmentIndexStateResponse - 180, // 324: milvus.proto.data.DataCoord.GetIndexInfos:output_type -> milvus.proto.index.GetIndexInfoResponse - 130, // 325: milvus.proto.data.DataCoord.DropIndex:output_type -> milvus.proto.common.Status - 181, // 326: milvus.proto.data.DataCoord.DescribeIndex:output_type -> milvus.proto.index.DescribeIndexResponse - 182, // 327: milvus.proto.data.DataCoord.GetIndexStatistics:output_type -> milvus.proto.index.GetIndexStatisticsResponse - 183, // 328: milvus.proto.data.DataCoord.GetIndexBuildProgress:output_type -> milvus.proto.index.GetIndexBuildProgressResponse - 184, // 329: milvus.proto.data.DataCoord.ListIndexes:output_type -> milvus.proto.index.ListIndexesResponse - 88, // 330: milvus.proto.data.DataCoord.GcConfirm:output_type -> milvus.proto.data.GcConfirmResponse - 130, // 331: milvus.proto.data.DataCoord.ReportDataNodeTtMsgs:output_type -> milvus.proto.common.Status - 130, // 332: milvus.proto.data.DataCoord.GcControl:output_type -> milvus.proto.common.Status - 185, // 333: milvus.proto.data.DataCoord.ImportV2:output_type -> milvus.proto.internal.ImportResponse - 186, // 334: milvus.proto.data.DataCoord.GetImportProgress:output_type -> milvus.proto.internal.GetImportProgressResponse - 187, // 335: milvus.proto.data.DataCoord.ListImports:output_type -> milvus.proto.internal.ListImportsResponse - 130, // 336: milvus.proto.data.DataCoord.AddFileResource:output_type -> milvus.proto.common.Status - 130, // 337: milvus.proto.data.DataCoord.RemoveFileResource:output_type -> milvus.proto.common.Status - 188, // 338: milvus.proto.data.DataCoord.ListFileResources:output_type -> milvus.proto.milvus.ListFileResourcesResponse - 189, // 339: milvus.proto.data.DataNode.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates - 169, // 340: milvus.proto.data.DataNode.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse - 130, // 341: milvus.proto.data.DataNode.WatchDmChannels:output_type -> milvus.proto.common.Status - 130, // 342: milvus.proto.data.DataNode.FlushSegments:output_type -> milvus.proto.common.Status - 171, // 343: milvus.proto.data.DataNode.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse - 172, // 344: milvus.proto.data.DataNode.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse - 130, // 345: milvus.proto.data.DataNode.CompactionV2:output_type -> milvus.proto.common.Status - 70, // 346: milvus.proto.data.DataNode.GetCompactionState:output_type -> milvus.proto.data.CompactionStateResponse - 130, // 347: milvus.proto.data.DataNode.SyncSegments:output_type -> milvus.proto.common.Status - 82, // 348: milvus.proto.data.DataNode.ResendSegmentStats:output_type -> milvus.proto.data.ResendSegmentStatsResponse - 130, // 349: milvus.proto.data.DataNode.FlushChannels:output_type -> milvus.proto.common.Status - 130, // 350: milvus.proto.data.DataNode.NotifyChannelOperation:output_type -> milvus.proto.common.Status - 92, // 351: milvus.proto.data.DataNode.CheckChannelOperationProgress:output_type -> milvus.proto.data.ChannelOperationProgressResponse - 130, // 352: milvus.proto.data.DataNode.PreImport:output_type -> milvus.proto.common.Status - 130, // 353: milvus.proto.data.DataNode.ImportV2:output_type -> milvus.proto.common.Status - 100, // 354: milvus.proto.data.DataNode.QueryPreImport:output_type -> milvus.proto.data.QueryPreImportResponse - 103, // 355: milvus.proto.data.DataNode.QueryImport:output_type -> milvus.proto.data.QueryImportResponse - 130, // 356: milvus.proto.data.DataNode.DropImport:output_type -> milvus.proto.common.Status - 111, // 357: milvus.proto.data.DataNode.QuerySlot:output_type -> milvus.proto.data.QuerySlotResponse - 130, // 358: milvus.proto.data.DataNode.DropCompactionPlan:output_type -> milvus.proto.common.Status - 288, // [288:359] is the sub-list for method output_type - 217, // [217:288] is the sub-list for method input_type + 169, // 288: milvus.proto.data.DataNode.SyncFileResource:input_type -> milvus.proto.internal.SyncFileResourceRequest + 10, // 289: milvus.proto.data.DataCoord.Flush:output_type -> milvus.proto.data.FlushResponse + 14, // 290: milvus.proto.data.DataCoord.FlushAll:output_type -> milvus.proto.data.FlushAllResponse + 116, // 291: milvus.proto.data.DataCoord.CreateExternalCollection:output_type -> milvus.proto.data.CreateExternalCollectionResponse + 18, // 292: milvus.proto.data.DataCoord.AllocSegment:output_type -> milvus.proto.data.AllocSegmentResponse + 21, // 293: milvus.proto.data.DataCoord.AssignSegmentID:output_type -> milvus.proto.data.AssignSegmentIDResponse + 26, // 294: milvus.proto.data.DataCoord.GetSegmentInfo:output_type -> milvus.proto.data.GetSegmentInfoResponse + 24, // 295: milvus.proto.data.DataCoord.GetSegmentStates:output_type -> milvus.proto.data.GetSegmentStatesResponse + 28, // 296: milvus.proto.data.DataCoord.GetInsertBinlogPaths:output_type -> milvus.proto.data.GetInsertBinlogPathsResponse + 30, // 297: milvus.proto.data.DataCoord.GetCollectionStatistics:output_type -> milvus.proto.data.GetCollectionStatisticsResponse + 32, // 298: milvus.proto.data.DataCoord.GetPartitionStatistics:output_type -> milvus.proto.data.GetPartitionStatisticsResponse + 170, // 299: milvus.proto.data.DataCoord.GetSegmentInfoChannel:output_type -> milvus.proto.milvus.StringResponse + 130, // 300: milvus.proto.data.DataCoord.SaveBinlogPaths:output_type -> milvus.proto.common.Status + 50, // 301: milvus.proto.data.DataCoord.GetRecoveryInfo:output_type -> milvus.proto.data.GetRecoveryInfoResponse + 52, // 302: milvus.proto.data.DataCoord.GetRecoveryInfoV2:output_type -> milvus.proto.data.GetRecoveryInfoResponseV2 + 55, // 303: milvus.proto.data.DataCoord.GetChannelRecoveryInfo:output_type -> milvus.proto.data.GetChannelRecoveryInfoResponse + 60, // 304: milvus.proto.data.DataCoord.GetFlushedSegments:output_type -> milvus.proto.data.GetFlushedSegmentsResponse + 58, // 305: milvus.proto.data.DataCoord.GetSegmentsByStates:output_type -> milvus.proto.data.GetSegmentsByStatesResponse + 171, // 306: milvus.proto.data.DataCoord.GetFlushAllState:output_type -> milvus.proto.milvus.GetFlushAllStateResponse + 172, // 307: milvus.proto.data.DataCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse + 173, // 308: milvus.proto.data.DataCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse + 174, // 309: milvus.proto.data.DataCoord.ManualCompaction:output_type -> milvus.proto.milvus.ManualCompactionResponse + 175, // 310: milvus.proto.data.DataCoord.GetCompactionState:output_type -> milvus.proto.milvus.GetCompactionStateResponse + 176, // 311: milvus.proto.data.DataCoord.GetCompactionStateWithPlans:output_type -> milvus.proto.milvus.GetCompactionPlansResponse + 73, // 312: milvus.proto.data.DataCoord.WatchChannels:output_type -> milvus.proto.data.WatchChannelsResponse + 177, // 313: milvus.proto.data.DataCoord.GetFlushState:output_type -> milvus.proto.milvus.GetFlushStateResponse + 78, // 314: milvus.proto.data.DataCoord.DropVirtualChannel:output_type -> milvus.proto.data.DropVirtualChannelResponse + 75, // 315: milvus.proto.data.DataCoord.SetSegmentState:output_type -> milvus.proto.data.SetSegmentStateResponse + 130, // 316: milvus.proto.data.DataCoord.UpdateSegmentStatistics:output_type -> milvus.proto.common.Status + 130, // 317: milvus.proto.data.DataCoord.UpdateChannelCheckpoint:output_type -> milvus.proto.common.Status + 130, // 318: milvus.proto.data.DataCoord.MarkSegmentsDropped:output_type -> milvus.proto.common.Status + 130, // 319: milvus.proto.data.DataCoord.BroadcastAlteredCollection:output_type -> milvus.proto.common.Status + 178, // 320: milvus.proto.data.DataCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse + 130, // 321: milvus.proto.data.DataCoord.CreateIndex:output_type -> milvus.proto.common.Status + 130, // 322: milvus.proto.data.DataCoord.AlterIndex:output_type -> milvus.proto.common.Status + 179, // 323: milvus.proto.data.DataCoord.GetIndexState:output_type -> milvus.proto.index.GetIndexStateResponse + 180, // 324: milvus.proto.data.DataCoord.GetSegmentIndexState:output_type -> milvus.proto.index.GetSegmentIndexStateResponse + 181, // 325: milvus.proto.data.DataCoord.GetIndexInfos:output_type -> milvus.proto.index.GetIndexInfoResponse + 130, // 326: milvus.proto.data.DataCoord.DropIndex:output_type -> milvus.proto.common.Status + 182, // 327: milvus.proto.data.DataCoord.DescribeIndex:output_type -> milvus.proto.index.DescribeIndexResponse + 183, // 328: milvus.proto.data.DataCoord.GetIndexStatistics:output_type -> milvus.proto.index.GetIndexStatisticsResponse + 184, // 329: milvus.proto.data.DataCoord.GetIndexBuildProgress:output_type -> milvus.proto.index.GetIndexBuildProgressResponse + 185, // 330: milvus.proto.data.DataCoord.ListIndexes:output_type -> milvus.proto.index.ListIndexesResponse + 88, // 331: milvus.proto.data.DataCoord.GcConfirm:output_type -> milvus.proto.data.GcConfirmResponse + 130, // 332: milvus.proto.data.DataCoord.ReportDataNodeTtMsgs:output_type -> milvus.proto.common.Status + 130, // 333: milvus.proto.data.DataCoord.GcControl:output_type -> milvus.proto.common.Status + 186, // 334: milvus.proto.data.DataCoord.ImportV2:output_type -> milvus.proto.internal.ImportResponse + 187, // 335: milvus.proto.data.DataCoord.GetImportProgress:output_type -> milvus.proto.internal.GetImportProgressResponse + 188, // 336: milvus.proto.data.DataCoord.ListImports:output_type -> milvus.proto.internal.ListImportsResponse + 130, // 337: milvus.proto.data.DataCoord.AddFileResource:output_type -> milvus.proto.common.Status + 130, // 338: milvus.proto.data.DataCoord.RemoveFileResource:output_type -> milvus.proto.common.Status + 189, // 339: milvus.proto.data.DataCoord.ListFileResources:output_type -> milvus.proto.milvus.ListFileResourcesResponse + 190, // 340: milvus.proto.data.DataNode.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates + 170, // 341: milvus.proto.data.DataNode.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse + 130, // 342: milvus.proto.data.DataNode.WatchDmChannels:output_type -> milvus.proto.common.Status + 130, // 343: milvus.proto.data.DataNode.FlushSegments:output_type -> milvus.proto.common.Status + 172, // 344: milvus.proto.data.DataNode.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse + 173, // 345: milvus.proto.data.DataNode.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse + 130, // 346: milvus.proto.data.DataNode.CompactionV2:output_type -> milvus.proto.common.Status + 70, // 347: milvus.proto.data.DataNode.GetCompactionState:output_type -> milvus.proto.data.CompactionStateResponse + 130, // 348: milvus.proto.data.DataNode.SyncSegments:output_type -> milvus.proto.common.Status + 82, // 349: milvus.proto.data.DataNode.ResendSegmentStats:output_type -> milvus.proto.data.ResendSegmentStatsResponse + 130, // 350: milvus.proto.data.DataNode.FlushChannels:output_type -> milvus.proto.common.Status + 130, // 351: milvus.proto.data.DataNode.NotifyChannelOperation:output_type -> milvus.proto.common.Status + 92, // 352: milvus.proto.data.DataNode.CheckChannelOperationProgress:output_type -> milvus.proto.data.ChannelOperationProgressResponse + 130, // 353: milvus.proto.data.DataNode.PreImport:output_type -> milvus.proto.common.Status + 130, // 354: milvus.proto.data.DataNode.ImportV2:output_type -> milvus.proto.common.Status + 100, // 355: milvus.proto.data.DataNode.QueryPreImport:output_type -> milvus.proto.data.QueryPreImportResponse + 103, // 356: milvus.proto.data.DataNode.QueryImport:output_type -> milvus.proto.data.QueryImportResponse + 130, // 357: milvus.proto.data.DataNode.DropImport:output_type -> milvus.proto.common.Status + 111, // 358: milvus.proto.data.DataNode.QuerySlot:output_type -> milvus.proto.data.QuerySlotResponse + 130, // 359: milvus.proto.data.DataNode.DropCompactionPlan:output_type -> milvus.proto.common.Status + 130, // 360: milvus.proto.data.DataNode.SyncFileResource:output_type -> milvus.proto.common.Status + 289, // [289:361] is the sub-list for method output_type + 217, // [217:289] is the sub-list for method input_type 217, // [217:217] is the sub-list for extension type_name 217, // [217:217] is the sub-list for extension extendee 0, // [0:217] is the sub-list for field type_name diff --git a/pkg/proto/datapb/data_coord_grpc.pb.go b/pkg/proto/datapb/data_coord_grpc.pb.go index d5fb949d66..a86cd92b7d 100644 --- a/pkg/proto/datapb/data_coord_grpc.pb.go +++ b/pkg/proto/datapb/data_coord_grpc.pb.go @@ -2002,6 +2002,7 @@ const ( DataNode_DropImport_FullMethodName = "/milvus.proto.data.DataNode/DropImport" DataNode_QuerySlot_FullMethodName = "/milvus.proto.data.DataNode/QuerySlot" DataNode_DropCompactionPlan_FullMethodName = "/milvus.proto.data.DataNode/DropCompactionPlan" + DataNode_SyncFileResource_FullMethodName = "/milvus.proto.data.DataNode/SyncFileResource" ) // DataNodeClient is the client API for DataNode service. @@ -2031,6 +2032,8 @@ type DataNodeClient interface { DropImport(ctx context.Context, in *DropImportRequest, opts ...grpc.CallOption) (*commonpb.Status, error) QuerySlot(ctx context.Context, in *QuerySlotRequest, opts ...grpc.CallOption) (*QuerySlotResponse, error) DropCompactionPlan(ctx context.Context, in *DropCompactionPlanRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + // file resource + SyncFileResource(ctx context.Context, in *internalpb.SyncFileResourceRequest, opts ...grpc.CallOption) (*commonpb.Status, error) } type dataNodeClient struct { @@ -2221,6 +2224,15 @@ func (c *dataNodeClient) DropCompactionPlan(ctx context.Context, in *DropCompact return out, nil } +func (c *dataNodeClient) SyncFileResource(ctx context.Context, in *internalpb.SyncFileResourceRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, DataNode_SyncFileResource_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + // DataNodeServer is the server API for DataNode service. // All implementations should embed UnimplementedDataNodeServer // for forward compatibility @@ -2248,6 +2260,8 @@ type DataNodeServer interface { DropImport(context.Context, *DropImportRequest) (*commonpb.Status, error) QuerySlot(context.Context, *QuerySlotRequest) (*QuerySlotResponse, error) DropCompactionPlan(context.Context, *DropCompactionPlanRequest) (*commonpb.Status, error) + // file resource + SyncFileResource(context.Context, *internalpb.SyncFileResourceRequest) (*commonpb.Status, error) } // UnimplementedDataNodeServer should be embedded to have forward compatible implementations. @@ -2314,6 +2328,9 @@ func (UnimplementedDataNodeServer) QuerySlot(context.Context, *QuerySlotRequest) func (UnimplementedDataNodeServer) DropCompactionPlan(context.Context, *DropCompactionPlanRequest) (*commonpb.Status, error) { return nil, status.Errorf(codes.Unimplemented, "method DropCompactionPlan not implemented") } +func (UnimplementedDataNodeServer) SyncFileResource(context.Context, *internalpb.SyncFileResourceRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method SyncFileResource not implemented") +} // UnsafeDataNodeServer may be embedded to opt out of forward compatibility for this service. // Use of this interface is not recommended, as added methods to DataNodeServer will @@ -2686,6 +2703,24 @@ func _DataNode_DropCompactionPlan_Handler(srv interface{}, ctx context.Context, return interceptor(ctx, in, info, handler) } +func _DataNode_SyncFileResource_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.SyncFileResourceRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataNodeServer).SyncFileResource(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: DataNode_SyncFileResource_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataNodeServer).SyncFileResource(ctx, req.(*internalpb.SyncFileResourceRequest)) + } + return interceptor(ctx, in, info, handler) +} + // DataNode_ServiceDesc is the grpc.ServiceDesc for DataNode service. // It's only intended for direct use with grpc.RegisterService, // and not to be introspected or modified (even as a copy) @@ -2773,6 +2808,10 @@ var DataNode_ServiceDesc = grpc.ServiceDesc{ MethodName: "DropCompactionPlan", Handler: _DataNode_DropCompactionPlan_Handler, }, + { + MethodName: "SyncFileResource", + Handler: _DataNode_SyncFileResource_Handler, + }, }, Streams: []grpc.StreamDesc{}, Metadata: "data_coord.proto", diff --git a/pkg/proto/internal.proto b/pkg/proto/internal.proto index 7c3489bebe..ca2e38a5c5 100644 --- a/pkg/proto/internal.proto +++ b/pkg/proto/internal.proto @@ -448,3 +448,15 @@ message GetQuotaMetricsResponse { common.Status status = 1; string metrics_info = 2; } + +message FileResourceInfo { + string name = 1; + string path = 2; + int64 id = 3; + string storage_name = 5; +} + +message SyncFileResourceRequest{ + repeated FileResourceInfo resources = 1; + uint64 version = 2; +} \ No newline at end of file diff --git a/pkg/proto/internalpb/internal.pb.go b/pkg/proto/internalpb/internal.pb.go index c09d336cc9..c2495ac242 100644 --- a/pkg/proto/internalpb/internal.pb.go +++ b/pkg/proto/internalpb/internal.pb.go @@ -4030,6 +4030,132 @@ func (x *GetQuotaMetricsResponse) GetMetricsInfo() string { return "" } +type FileResourceInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + Path string `protobuf:"bytes,2,opt,name=path,proto3" json:"path,omitempty"` + Id int64 `protobuf:"varint,3,opt,name=id,proto3" json:"id,omitempty"` + StorageName string `protobuf:"bytes,5,opt,name=storage_name,json=storageName,proto3" json:"storage_name,omitempty"` +} + +func (x *FileResourceInfo) Reset() { + *x = FileResourceInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[46] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *FileResourceInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FileResourceInfo) ProtoMessage() {} + +func (x *FileResourceInfo) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[46] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use FileResourceInfo.ProtoReflect.Descriptor instead. +func (*FileResourceInfo) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{46} +} + +func (x *FileResourceInfo) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *FileResourceInfo) GetPath() string { + if x != nil { + return x.Path + } + return "" +} + +func (x *FileResourceInfo) GetId() int64 { + if x != nil { + return x.Id + } + return 0 +} + +func (x *FileResourceInfo) GetStorageName() string { + if x != nil { + return x.StorageName + } + return "" +} + +type SyncFileResourceRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Resources []*FileResourceInfo `protobuf:"bytes,1,rep,name=resources,proto3" json:"resources,omitempty"` + Version uint64 `protobuf:"varint,2,opt,name=version,proto3" json:"version,omitempty"` +} + +func (x *SyncFileResourceRequest) Reset() { + *x = SyncFileResourceRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_internal_proto_msgTypes[47] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SyncFileResourceRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SyncFileResourceRequest) ProtoMessage() {} + +func (x *SyncFileResourceRequest) ProtoReflect() protoreflect.Message { + mi := &file_internal_proto_msgTypes[47] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SyncFileResourceRequest.ProtoReflect.Descriptor instead. +func (*SyncFileResourceRequest) Descriptor() ([]byte, []int) { + return file_internal_proto_rawDescGZIP(), []int{47} +} + +func (x *SyncFileResourceRequest) GetResources() []*FileResourceInfo { + if x != nil { + return x.Resources + } + return nil +} + +func (x *SyncFileResourceRequest) GetVersion() uint64 { + if x != nil { + return x.Version + } + return 0 +} + var File_internal_proto protoreflect.FileDescriptor var file_internal_proto_rawDesc = []byte{ @@ -4737,36 +4863,50 @@ var file_internal_proto_rawDesc = []byte{ 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x49, 0x6e, 0x66, - 0x6f, 0x2a, 0x45, 0x0a, 0x09, 0x52, 0x61, 0x74, 0x65, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x0b, - 0x0a, 0x07, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x44, - 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x10, 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x43, 0x6f, 0x6c, - 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x50, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x03, 0x2a, 0xc4, 0x01, 0x0a, 0x08, 0x52, 0x61, 0x74, - 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x44, 0x4c, 0x43, 0x6f, 0x6c, 0x6c, - 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x00, 0x12, 0x10, 0x0a, 0x0c, 0x44, 0x44, 0x4c, 0x50, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x01, 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x44, - 0x4c, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x10, 0x02, 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x44, 0x4c, 0x46, - 0x6c, 0x75, 0x73, 0x68, 0x10, 0x03, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x44, 0x4c, 0x43, 0x6f, 0x6d, - 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x04, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x4d, 0x4c, - 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x10, 0x05, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x4d, 0x4c, 0x44, - 0x65, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x06, 0x12, 0x0f, 0x0a, 0x0b, 0x44, 0x4d, 0x4c, 0x42, 0x75, - 0x6c, 0x6b, 0x4c, 0x6f, 0x61, 0x64, 0x10, 0x07, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x51, 0x4c, 0x53, - 0x65, 0x61, 0x72, 0x63, 0x68, 0x10, 0x08, 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x51, 0x4c, 0x51, 0x75, - 0x65, 0x72, 0x79, 0x10, 0x09, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x4d, 0x4c, 0x55, 0x70, 0x73, 0x65, - 0x72, 0x74, 0x10, 0x0a, 0x12, 0x09, 0x0a, 0x05, 0x44, 0x44, 0x4c, 0x44, 0x42, 0x10, 0x0b, 0x2a, - 0x83, 0x01, 0x0a, 0x0e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, - 0x74, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, - 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x10, 0x01, 0x12, 0x10, 0x0a, 0x0c, 0x50, 0x72, 0x65, - 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x49, - 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x10, 0x03, 0x12, 0x0a, 0x0a, 0x06, 0x46, 0x61, - 0x69, 0x6c, 0x65, 0x64, 0x10, 0x04, 0x12, 0x0d, 0x0a, 0x09, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, - 0x74, 0x65, 0x64, 0x10, 0x05, 0x12, 0x11, 0x0a, 0x0d, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, - 0x69, 0x6c, 0x64, 0x69, 0x6e, 0x67, 0x10, 0x06, 0x12, 0x0b, 0x0a, 0x07, 0x53, 0x6f, 0x72, 0x74, - 0x69, 0x6e, 0x67, 0x10, 0x07, 0x42, 0x35, 0x5a, 0x33, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, - 0x63, 0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x33, + 0x6f, 0x22, 0x6d, 0x0a, 0x10, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74, + 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x70, 0x61, 0x74, 0x68, 0x12, 0x0e, 0x0a, + 0x02, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, 0x69, 0x64, 0x12, 0x21, 0x0a, + 0x0c, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x4e, 0x61, 0x6d, 0x65, + 0x22, 0x7a, 0x0a, 0x17, 0x53, 0x79, 0x6e, 0x63, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x45, 0x0a, 0x09, 0x72, + 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, + 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x09, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x04, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x2a, 0x45, 0x0a, 0x09, + 0x52, 0x61, 0x74, 0x65, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x43, 0x6c, 0x75, + 0x73, 0x74, 0x65, 0x72, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, + 0x73, 0x65, 0x10, 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x10, 0x03, 0x2a, 0xc4, 0x01, 0x0a, 0x08, 0x52, 0x61, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, + 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x44, 0x4c, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x10, 0x00, 0x12, 0x10, 0x0a, 0x0c, 0x44, 0x44, 0x4c, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x10, 0x01, 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x44, 0x4c, 0x49, 0x6e, 0x64, 0x65, + 0x78, 0x10, 0x02, 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x44, 0x4c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x10, + 0x03, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x44, 0x4c, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x10, 0x04, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x4d, 0x4c, 0x49, 0x6e, 0x73, 0x65, 0x72, + 0x74, 0x10, 0x05, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x4d, 0x4c, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, + 0x10, 0x06, 0x12, 0x0f, 0x0a, 0x0b, 0x44, 0x4d, 0x4c, 0x42, 0x75, 0x6c, 0x6b, 0x4c, 0x6f, 0x61, + 0x64, 0x10, 0x07, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x51, 0x4c, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, + 0x10, 0x08, 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x51, 0x4c, 0x51, 0x75, 0x65, 0x72, 0x79, 0x10, 0x09, + 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x4d, 0x4c, 0x55, 0x70, 0x73, 0x65, 0x72, 0x74, 0x10, 0x0a, 0x12, + 0x09, 0x0a, 0x05, 0x44, 0x44, 0x4c, 0x44, 0x42, 0x10, 0x0b, 0x2a, 0x83, 0x01, 0x0a, 0x0e, 0x49, + 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x08, 0x0a, + 0x04, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x50, 0x65, 0x6e, 0x64, 0x69, + 0x6e, 0x67, 0x10, 0x01, 0x12, 0x10, 0x0a, 0x0c, 0x50, 0x72, 0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, + 0x74, 0x69, 0x6e, 0x67, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, + 0x69, 0x6e, 0x67, 0x10, 0x03, 0x12, 0x0a, 0x0a, 0x06, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x10, + 0x04, 0x12, 0x0d, 0x0a, 0x09, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x10, 0x05, + 0x12, 0x11, 0x0a, 0x0d, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x69, 0x6e, + 0x67, 0x10, 0x06, 0x12, 0x0b, 0x0a, 0x07, 0x53, 0x6f, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x10, 0x07, + 0x42, 0x35, 0x5a, 0x33, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2f, + 0x70, 0x6b, 0x67, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x69, 0x6e, 0x74, + 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -4782,7 +4922,7 @@ func file_internal_proto_rawDescGZIP() []byte { } var file_internal_proto_enumTypes = make([]protoimpl.EnumInfo, 3) -var file_internal_proto_msgTypes = make([]protoimpl.MessageInfo, 47) +var file_internal_proto_msgTypes = make([]protoimpl.MessageInfo, 49) var file_internal_proto_goTypes = []interface{}{ (RateScope)(0), // 0: milvus.proto.internal.RateScope (RateType)(0), // 1: milvus.proto.internal.RateType @@ -4833,87 +4973,90 @@ var file_internal_proto_goTypes = []interface{}{ (*GetSegmentsInfoResponse)(nil), // 46: milvus.proto.internal.GetSegmentsInfoResponse (*GetQuotaMetricsRequest)(nil), // 47: milvus.proto.internal.GetQuotaMetricsRequest (*GetQuotaMetricsResponse)(nil), // 48: milvus.proto.internal.GetQuotaMetricsResponse - nil, // 49: milvus.proto.internal.SearchResults.ChannelsMvccEntry - (*commonpb.Address)(nil), // 50: milvus.proto.common.Address - (*commonpb.KeyValuePair)(nil), // 51: milvus.proto.common.KeyValuePair - (*commonpb.Status)(nil), // 52: milvus.proto.common.Status - (*commonpb.MsgBase)(nil), // 53: milvus.proto.common.MsgBase - (commonpb.DslType)(0), // 54: milvus.proto.common.DslType - (commonpb.ConsistencyLevel)(0), // 55: milvus.proto.common.ConsistencyLevel - (*schemapb.IDs)(nil), // 56: milvus.proto.schema.IDs - (*schemapb.FieldData)(nil), // 57: milvus.proto.schema.FieldData - (*milvuspb.PrivilegeGroupInfo)(nil), // 58: milvus.proto.milvus.PrivilegeGroupInfo - (*schemapb.CollectionSchema)(nil), // 59: milvus.proto.schema.CollectionSchema - (commonpb.SegmentState)(0), // 60: milvus.proto.common.SegmentState - (commonpb.SegmentLevel)(0), // 61: milvus.proto.common.SegmentLevel + (*FileResourceInfo)(nil), // 49: milvus.proto.internal.FileResourceInfo + (*SyncFileResourceRequest)(nil), // 50: milvus.proto.internal.SyncFileResourceRequest + nil, // 51: milvus.proto.internal.SearchResults.ChannelsMvccEntry + (*commonpb.Address)(nil), // 52: milvus.proto.common.Address + (*commonpb.KeyValuePair)(nil), // 53: milvus.proto.common.KeyValuePair + (*commonpb.Status)(nil), // 54: milvus.proto.common.Status + (*commonpb.MsgBase)(nil), // 55: milvus.proto.common.MsgBase + (commonpb.DslType)(0), // 56: milvus.proto.common.DslType + (commonpb.ConsistencyLevel)(0), // 57: milvus.proto.common.ConsistencyLevel + (*schemapb.IDs)(nil), // 58: milvus.proto.schema.IDs + (*schemapb.FieldData)(nil), // 59: milvus.proto.schema.FieldData + (*milvuspb.PrivilegeGroupInfo)(nil), // 60: milvus.proto.milvus.PrivilegeGroupInfo + (*schemapb.CollectionSchema)(nil), // 61: milvus.proto.schema.CollectionSchema + (commonpb.SegmentState)(0), // 62: milvus.proto.common.SegmentState + (commonpb.SegmentLevel)(0), // 63: milvus.proto.common.SegmentLevel } var file_internal_proto_depIdxs = []int32{ - 50, // 0: milvus.proto.internal.NodeInfo.address:type_name -> milvus.proto.common.Address - 51, // 1: milvus.proto.internal.InitParams.start_params:type_name -> milvus.proto.common.KeyValuePair - 52, // 2: milvus.proto.internal.StringList.status:type_name -> milvus.proto.common.Status - 53, // 3: milvus.proto.internal.GetStatisticsRequest.base:type_name -> milvus.proto.common.MsgBase - 53, // 4: milvus.proto.internal.GetStatisticsResponse.base:type_name -> milvus.proto.common.MsgBase - 52, // 5: milvus.proto.internal.GetStatisticsResponse.status:type_name -> milvus.proto.common.Status - 51, // 6: milvus.proto.internal.GetStatisticsResponse.stats:type_name -> milvus.proto.common.KeyValuePair - 53, // 7: milvus.proto.internal.CreateAliasRequest.base:type_name -> milvus.proto.common.MsgBase - 53, // 8: milvus.proto.internal.DropAliasRequest.base:type_name -> milvus.proto.common.MsgBase - 53, // 9: milvus.proto.internal.AlterAliasRequest.base:type_name -> milvus.proto.common.MsgBase - 53, // 10: milvus.proto.internal.CreateIndexRequest.base:type_name -> milvus.proto.common.MsgBase - 51, // 11: milvus.proto.internal.CreateIndexRequest.extra_params:type_name -> milvus.proto.common.KeyValuePair - 54, // 12: milvus.proto.internal.SubSearchRequest.dsl_type:type_name -> milvus.proto.common.DslType - 53, // 13: milvus.proto.internal.SearchRequest.base:type_name -> milvus.proto.common.MsgBase - 54, // 14: milvus.proto.internal.SearchRequest.dsl_type:type_name -> milvus.proto.common.DslType + 52, // 0: milvus.proto.internal.NodeInfo.address:type_name -> milvus.proto.common.Address + 53, // 1: milvus.proto.internal.InitParams.start_params:type_name -> milvus.proto.common.KeyValuePair + 54, // 2: milvus.proto.internal.StringList.status:type_name -> milvus.proto.common.Status + 55, // 3: milvus.proto.internal.GetStatisticsRequest.base:type_name -> milvus.proto.common.MsgBase + 55, // 4: milvus.proto.internal.GetStatisticsResponse.base:type_name -> milvus.proto.common.MsgBase + 54, // 5: milvus.proto.internal.GetStatisticsResponse.status:type_name -> milvus.proto.common.Status + 53, // 6: milvus.proto.internal.GetStatisticsResponse.stats:type_name -> milvus.proto.common.KeyValuePair + 55, // 7: milvus.proto.internal.CreateAliasRequest.base:type_name -> milvus.proto.common.MsgBase + 55, // 8: milvus.proto.internal.DropAliasRequest.base:type_name -> milvus.proto.common.MsgBase + 55, // 9: milvus.proto.internal.AlterAliasRequest.base:type_name -> milvus.proto.common.MsgBase + 55, // 10: milvus.proto.internal.CreateIndexRequest.base:type_name -> milvus.proto.common.MsgBase + 53, // 11: milvus.proto.internal.CreateIndexRequest.extra_params:type_name -> milvus.proto.common.KeyValuePair + 56, // 12: milvus.proto.internal.SubSearchRequest.dsl_type:type_name -> milvus.proto.common.DslType + 55, // 13: milvus.proto.internal.SearchRequest.base:type_name -> milvus.proto.common.MsgBase + 56, // 14: milvus.proto.internal.SearchRequest.dsl_type:type_name -> milvus.proto.common.DslType 15, // 15: milvus.proto.internal.SearchRequest.sub_reqs:type_name -> milvus.proto.internal.SubSearchRequest - 55, // 16: milvus.proto.internal.SearchRequest.consistency_level:type_name -> milvus.proto.common.ConsistencyLevel - 53, // 17: milvus.proto.internal.SearchResults.base:type_name -> milvus.proto.common.MsgBase - 52, // 18: milvus.proto.internal.SearchResults.status:type_name -> milvus.proto.common.Status + 57, // 16: milvus.proto.internal.SearchRequest.consistency_level:type_name -> milvus.proto.common.ConsistencyLevel + 55, // 17: milvus.proto.internal.SearchResults.base:type_name -> milvus.proto.common.MsgBase + 54, // 18: milvus.proto.internal.SearchResults.status:type_name -> milvus.proto.common.Status 19, // 19: milvus.proto.internal.SearchResults.costAggregation:type_name -> milvus.proto.internal.CostAggregation - 49, // 20: milvus.proto.internal.SearchResults.channels_mvcc:type_name -> milvus.proto.internal.SearchResults.ChannelsMvccEntry + 51, // 20: milvus.proto.internal.SearchResults.channels_mvcc:type_name -> milvus.proto.internal.SearchResults.ChannelsMvccEntry 17, // 21: milvus.proto.internal.SearchResults.sub_results:type_name -> milvus.proto.internal.SubSearchResults - 53, // 22: milvus.proto.internal.RetrieveRequest.base:type_name -> milvus.proto.common.MsgBase - 55, // 23: milvus.proto.internal.RetrieveRequest.consistency_level:type_name -> milvus.proto.common.ConsistencyLevel - 53, // 24: milvus.proto.internal.RetrieveResults.base:type_name -> milvus.proto.common.MsgBase - 52, // 25: milvus.proto.internal.RetrieveResults.status:type_name -> milvus.proto.common.Status - 56, // 26: milvus.proto.internal.RetrieveResults.ids:type_name -> milvus.proto.schema.IDs - 57, // 27: milvus.proto.internal.RetrieveResults.fields_data:type_name -> milvus.proto.schema.FieldData + 55, // 22: milvus.proto.internal.RetrieveRequest.base:type_name -> milvus.proto.common.MsgBase + 57, // 23: milvus.proto.internal.RetrieveRequest.consistency_level:type_name -> milvus.proto.common.ConsistencyLevel + 55, // 24: milvus.proto.internal.RetrieveResults.base:type_name -> milvus.proto.common.MsgBase + 54, // 25: milvus.proto.internal.RetrieveResults.status:type_name -> milvus.proto.common.Status + 58, // 26: milvus.proto.internal.RetrieveResults.ids:type_name -> milvus.proto.schema.IDs + 59, // 27: milvus.proto.internal.RetrieveResults.fields_data:type_name -> milvus.proto.schema.FieldData 19, // 28: milvus.proto.internal.RetrieveResults.costAggregation:type_name -> milvus.proto.internal.CostAggregation - 53, // 29: milvus.proto.internal.LoadIndex.base:type_name -> milvus.proto.common.MsgBase - 51, // 30: milvus.proto.internal.LoadIndex.index_params:type_name -> milvus.proto.common.KeyValuePair - 51, // 31: milvus.proto.internal.IndexStats.index_params:type_name -> milvus.proto.common.KeyValuePair + 55, // 29: milvus.proto.internal.LoadIndex.base:type_name -> milvus.proto.common.MsgBase + 53, // 30: milvus.proto.internal.LoadIndex.index_params:type_name -> milvus.proto.common.KeyValuePair + 53, // 31: milvus.proto.internal.IndexStats.index_params:type_name -> milvus.proto.common.KeyValuePair 23, // 32: milvus.proto.internal.FieldStats.index_stats:type_name -> milvus.proto.internal.IndexStats - 53, // 33: milvus.proto.internal.ChannelTimeTickMsg.base:type_name -> milvus.proto.common.MsgBase - 53, // 34: milvus.proto.internal.ListPolicyRequest.base:type_name -> milvus.proto.common.MsgBase - 52, // 35: milvus.proto.internal.ListPolicyResponse.status:type_name -> milvus.proto.common.Status - 58, // 36: milvus.proto.internal.ListPolicyResponse.privilege_groups:type_name -> milvus.proto.milvus.PrivilegeGroupInfo - 53, // 37: milvus.proto.internal.ShowConfigurationsRequest.base:type_name -> milvus.proto.common.MsgBase - 52, // 38: milvus.proto.internal.ShowConfigurationsResponse.status:type_name -> milvus.proto.common.Status - 51, // 39: milvus.proto.internal.ShowConfigurationsResponse.configuations:type_name -> milvus.proto.common.KeyValuePair + 55, // 33: milvus.proto.internal.ChannelTimeTickMsg.base:type_name -> milvus.proto.common.MsgBase + 55, // 34: milvus.proto.internal.ListPolicyRequest.base:type_name -> milvus.proto.common.MsgBase + 54, // 35: milvus.proto.internal.ListPolicyResponse.status:type_name -> milvus.proto.common.Status + 60, // 36: milvus.proto.internal.ListPolicyResponse.privilege_groups:type_name -> milvus.proto.milvus.PrivilegeGroupInfo + 55, // 37: milvus.proto.internal.ShowConfigurationsRequest.base:type_name -> milvus.proto.common.MsgBase + 54, // 38: milvus.proto.internal.ShowConfigurationsResponse.status:type_name -> milvus.proto.common.Status + 53, // 39: milvus.proto.internal.ShowConfigurationsResponse.configuations:type_name -> milvus.proto.common.KeyValuePair 1, // 40: milvus.proto.internal.Rate.rt:type_name -> milvus.proto.internal.RateType - 59, // 41: milvus.proto.internal.ImportRequestInternal.schema:type_name -> milvus.proto.schema.CollectionSchema + 61, // 41: milvus.proto.internal.ImportRequestInternal.schema:type_name -> milvus.proto.schema.CollectionSchema 33, // 42: milvus.proto.internal.ImportRequestInternal.files:type_name -> milvus.proto.internal.ImportFile - 51, // 43: milvus.proto.internal.ImportRequestInternal.options:type_name -> milvus.proto.common.KeyValuePair + 53, // 43: milvus.proto.internal.ImportRequestInternal.options:type_name -> milvus.proto.common.KeyValuePair 33, // 44: milvus.proto.internal.ImportRequest.files:type_name -> milvus.proto.internal.ImportFile - 51, // 45: milvus.proto.internal.ImportRequest.options:type_name -> milvus.proto.common.KeyValuePair - 52, // 46: milvus.proto.internal.ImportResponse.status:type_name -> milvus.proto.common.Status - 52, // 47: milvus.proto.internal.GetImportProgressResponse.status:type_name -> milvus.proto.common.Status + 53, // 45: milvus.proto.internal.ImportRequest.options:type_name -> milvus.proto.common.KeyValuePair + 54, // 46: milvus.proto.internal.ImportResponse.status:type_name -> milvus.proto.common.Status + 54, // 47: milvus.proto.internal.GetImportProgressResponse.status:type_name -> milvus.proto.common.Status 2, // 48: milvus.proto.internal.GetImportProgressResponse.state:type_name -> milvus.proto.internal.ImportJobState 38, // 49: milvus.proto.internal.GetImportProgressResponse.task_progresses:type_name -> milvus.proto.internal.ImportTaskProgress - 52, // 50: milvus.proto.internal.ListImportsResponse.status:type_name -> milvus.proto.common.Status + 54, // 50: milvus.proto.internal.ListImportsResponse.status:type_name -> milvus.proto.common.Status 2, // 51: milvus.proto.internal.ListImportsResponse.states:type_name -> milvus.proto.internal.ImportJobState - 60, // 52: milvus.proto.internal.SegmentInfo.state:type_name -> milvus.proto.common.SegmentState - 61, // 53: milvus.proto.internal.SegmentInfo.level:type_name -> milvus.proto.common.SegmentLevel + 62, // 52: milvus.proto.internal.SegmentInfo.state:type_name -> milvus.proto.common.SegmentState + 63, // 53: milvus.proto.internal.SegmentInfo.level:type_name -> milvus.proto.common.SegmentLevel 44, // 54: milvus.proto.internal.SegmentInfo.insert_logs:type_name -> milvus.proto.internal.FieldBinlog 44, // 55: milvus.proto.internal.SegmentInfo.delta_logs:type_name -> milvus.proto.internal.FieldBinlog 44, // 56: milvus.proto.internal.SegmentInfo.stats_logs:type_name -> milvus.proto.internal.FieldBinlog - 52, // 57: milvus.proto.internal.GetSegmentsInfoResponse.status:type_name -> milvus.proto.common.Status + 54, // 57: milvus.proto.internal.GetSegmentsInfoResponse.status:type_name -> milvus.proto.common.Status 45, // 58: milvus.proto.internal.GetSegmentsInfoResponse.segmentInfos:type_name -> milvus.proto.internal.SegmentInfo - 53, // 59: milvus.proto.internal.GetQuotaMetricsRequest.base:type_name -> milvus.proto.common.MsgBase - 52, // 60: milvus.proto.internal.GetQuotaMetricsResponse.status:type_name -> milvus.proto.common.Status - 61, // [61:61] is the sub-list for method output_type - 61, // [61:61] is the sub-list for method input_type - 61, // [61:61] is the sub-list for extension type_name - 61, // [61:61] is the sub-list for extension extendee - 0, // [0:61] is the sub-list for field type_name + 55, // 59: milvus.proto.internal.GetQuotaMetricsRequest.base:type_name -> milvus.proto.common.MsgBase + 54, // 60: milvus.proto.internal.GetQuotaMetricsResponse.status:type_name -> milvus.proto.common.Status + 49, // 61: milvus.proto.internal.SyncFileResourceRequest.resources:type_name -> milvus.proto.internal.FileResourceInfo + 62, // [62:62] is the sub-list for method output_type + 62, // [62:62] is the sub-list for method input_type + 62, // [62:62] is the sub-list for extension type_name + 62, // [62:62] is the sub-list for extension extendee + 0, // [0:62] is the sub-list for field type_name } func init() { file_internal_proto_init() } @@ -5474,6 +5617,30 @@ func file_internal_proto_init() { return nil } } + file_internal_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*FileResourceInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_internal_proto_msgTypes[47].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SyncFileResourceRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } } type x struct{} out := protoimpl.TypeBuilder{ @@ -5481,7 +5648,7 @@ func file_internal_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_internal_proto_rawDesc, NumEnums: 3, - NumMessages: 47, + NumMessages: 49, NumExtensions: 0, NumServices: 0, }, diff --git a/pkg/proto/query_coord.proto b/pkg/proto/query_coord.proto index d1a09b593f..8258154212 100644 --- a/pkg/proto/query_coord.proto +++ b/pkg/proto/query_coord.proto @@ -178,6 +178,9 @@ service QueryNode { rpc DropIndex(DropIndexRequest) returns (common.Status) {} rpc ValidateAnalyzer(ValidateAnalyzerRequest) returns(common.Status){} + + // file resource + rpc SyncFileResource(internal.SyncFileResourceRequest) returns(common.Status) {} } // --------------------QueryCoord grpc request and response proto------------------ diff --git a/pkg/proto/querypb/query_coord.pb.go b/pkg/proto/querypb/query_coord.pb.go index aba60905b6..c373652b83 100644 --- a/pkg/proto/querypb/query_coord.pb.go +++ b/pkg/proto/querypb/query_coord.pb.go @@ -10396,7 +10396,7 @@ var file_query_coord_proto_rawDesc = []byte{ 0x72, 0x79, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, - 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x32, 0x85, 0x17, 0x0a, 0x09, 0x51, 0x75, + 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x32, 0xe8, 0x17, 0x0a, 0x09, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x6c, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, @@ -10581,10 +10581,16 @@ var file_query_coord_proto_rawDesc = []byte{ 0x61, 0x74, 0x65, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, - 0x00, 0x42, 0x32, 0x5a, 0x30, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x71, 0x75, - 0x65, 0x72, 0x79, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x00, 0x12, 0x61, 0x0a, 0x10, 0x53, 0x79, 0x6e, 0x63, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x79, + 0x6e, 0x63, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x22, 0x00, 0x42, 0x32, 0x5a, 0x30, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, + 0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -10765,17 +10771,18 @@ var file_query_coord_proto_goTypes = []interface{}{ (*milvuspb.GetComponentStatesRequest)(nil), // 160: milvus.proto.milvus.GetComponentStatesRequest (*internalpb.GetTimeTickChannelRequest)(nil), // 161: milvus.proto.internal.GetTimeTickChannelRequest (*internalpb.GetStatisticsChannelRequest)(nil), // 162: milvus.proto.internal.GetStatisticsChannelRequest - (*internalpb.ShowConfigurationsResponse)(nil), // 163: milvus.proto.internal.ShowConfigurationsResponse - (*milvuspb.GetMetricsResponse)(nil), // 164: milvus.proto.milvus.GetMetricsResponse - (*milvuspb.GetReplicasResponse)(nil), // 165: milvus.proto.milvus.GetReplicasResponse - (*milvuspb.CheckHealthResponse)(nil), // 166: milvus.proto.milvus.CheckHealthResponse - (*milvuspb.ListResourceGroupsResponse)(nil), // 167: milvus.proto.milvus.ListResourceGroupsResponse - (*milvuspb.RunAnalyzerResponse)(nil), // 168: milvus.proto.milvus.RunAnalyzerResponse - (*milvuspb.ComponentStates)(nil), // 169: milvus.proto.milvus.ComponentStates - (*milvuspb.StringResponse)(nil), // 170: milvus.proto.milvus.StringResponse - (*internalpb.GetStatisticsResponse)(nil), // 171: milvus.proto.internal.GetStatisticsResponse - (*internalpb.SearchResults)(nil), // 172: milvus.proto.internal.SearchResults - (*internalpb.RetrieveResults)(nil), // 173: milvus.proto.internal.RetrieveResults + (*internalpb.SyncFileResourceRequest)(nil), // 163: milvus.proto.internal.SyncFileResourceRequest + (*internalpb.ShowConfigurationsResponse)(nil), // 164: milvus.proto.internal.ShowConfigurationsResponse + (*milvuspb.GetMetricsResponse)(nil), // 165: milvus.proto.milvus.GetMetricsResponse + (*milvuspb.GetReplicasResponse)(nil), // 166: milvus.proto.milvus.GetReplicasResponse + (*milvuspb.CheckHealthResponse)(nil), // 167: milvus.proto.milvus.CheckHealthResponse + (*milvuspb.ListResourceGroupsResponse)(nil), // 168: milvus.proto.milvus.ListResourceGroupsResponse + (*milvuspb.RunAnalyzerResponse)(nil), // 169: milvus.proto.milvus.RunAnalyzerResponse + (*milvuspb.ComponentStates)(nil), // 170: milvus.proto.milvus.ComponentStates + (*milvuspb.StringResponse)(nil), // 171: milvus.proto.milvus.StringResponse + (*internalpb.GetStatisticsResponse)(nil), // 172: milvus.proto.internal.GetStatisticsResponse + (*internalpb.SearchResults)(nil), // 173: milvus.proto.internal.SearchResults + (*internalpb.RetrieveResults)(nil), // 174: milvus.proto.internal.RetrieveResults } var file_query_coord_proto_depIdxs = []int32{ 131, // 0: milvus.proto.query.ShowCollectionsRequest.base:type_name -> milvus.proto.common.MsgBase @@ -11037,77 +11044,79 @@ var file_query_coord_proto_depIdxs = []int32{ 103, // 256: milvus.proto.query.QueryNode.GetHighlight:input_type -> milvus.proto.query.GetHighlightRequest 109, // 257: milvus.proto.query.QueryNode.DropIndex:input_type -> milvus.proto.query.DropIndexRequest 99, // 258: milvus.proto.query.QueryNode.ValidateAnalyzer:input_type -> milvus.proto.query.ValidateAnalyzerRequest - 9, // 259: milvus.proto.query.QueryCoord.ShowLoadCollections:output_type -> milvus.proto.query.ShowCollectionsResponse - 11, // 260: milvus.proto.query.QueryCoord.ShowLoadPartitions:output_type -> milvus.proto.query.ShowPartitionsResponse - 132, // 261: milvus.proto.query.QueryCoord.LoadPartitions:output_type -> milvus.proto.common.Status - 132, // 262: milvus.proto.query.QueryCoord.ReleasePartitions:output_type -> milvus.proto.common.Status - 132, // 263: milvus.proto.query.QueryCoord.LoadCollection:output_type -> milvus.proto.common.Status - 132, // 264: milvus.proto.query.QueryCoord.ReleaseCollection:output_type -> milvus.proto.common.Status - 132, // 265: milvus.proto.query.QueryCoord.SyncNewCreatedPartition:output_type -> milvus.proto.common.Status - 18, // 266: milvus.proto.query.QueryCoord.GetPartitionStates:output_type -> milvus.proto.query.GetPartitionStatesResponse - 20, // 267: milvus.proto.query.QueryCoord.GetLoadSegmentInfo:output_type -> milvus.proto.query.GetSegmentInfoResponse - 132, // 268: milvus.proto.query.QueryCoord.LoadBalance:output_type -> milvus.proto.common.Status - 163, // 269: milvus.proto.query.QueryCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse - 164, // 270: milvus.proto.query.QueryCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse - 165, // 271: milvus.proto.query.QueryCoord.GetReplicas:output_type -> milvus.proto.milvus.GetReplicasResponse - 22, // 272: milvus.proto.query.QueryCoord.GetShardLeaders:output_type -> milvus.proto.query.GetShardLeadersResponse - 166, // 273: milvus.proto.query.QueryCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse - 132, // 274: milvus.proto.query.QueryCoord.CreateResourceGroup:output_type -> milvus.proto.common.Status - 132, // 275: milvus.proto.query.QueryCoord.UpdateResourceGroups:output_type -> milvus.proto.common.Status - 132, // 276: milvus.proto.query.QueryCoord.DropResourceGroup:output_type -> milvus.proto.common.Status - 132, // 277: milvus.proto.query.QueryCoord.TransferNode:output_type -> milvus.proto.common.Status - 132, // 278: milvus.proto.query.QueryCoord.TransferReplica:output_type -> milvus.proto.common.Status - 167, // 279: milvus.proto.query.QueryCoord.ListResourceGroups:output_type -> milvus.proto.milvus.ListResourceGroupsResponse - 67, // 280: milvus.proto.query.QueryCoord.DescribeResourceGroup:output_type -> milvus.proto.query.DescribeResourceGroupResponse - 108, // 281: milvus.proto.query.QueryCoord.ListLoadedSegments:output_type -> milvus.proto.query.ListLoadedSegmentsResponse - 75, // 282: milvus.proto.query.QueryCoord.ListCheckers:output_type -> milvus.proto.query.ListCheckersResponse - 132, // 283: milvus.proto.query.QueryCoord.ActivateChecker:output_type -> milvus.proto.common.Status - 132, // 284: milvus.proto.query.QueryCoord.DeactivateChecker:output_type -> milvus.proto.common.Status - 83, // 285: milvus.proto.query.QueryCoord.ListQueryNode:output_type -> milvus.proto.query.ListQueryNodeResponse - 85, // 286: milvus.proto.query.QueryCoord.GetQueryNodeDistribution:output_type -> milvus.proto.query.GetQueryNodeDistributionResponse - 132, // 287: milvus.proto.query.QueryCoord.SuspendBalance:output_type -> milvus.proto.common.Status - 132, // 288: milvus.proto.query.QueryCoord.ResumeBalance:output_type -> milvus.proto.common.Status - 89, // 289: milvus.proto.query.QueryCoord.CheckBalanceStatus:output_type -> milvus.proto.query.CheckBalanceStatusResponse - 132, // 290: milvus.proto.query.QueryCoord.SuspendNode:output_type -> milvus.proto.common.Status - 132, // 291: milvus.proto.query.QueryCoord.ResumeNode:output_type -> milvus.proto.common.Status - 132, // 292: milvus.proto.query.QueryCoord.TransferSegment:output_type -> milvus.proto.common.Status - 132, // 293: milvus.proto.query.QueryCoord.TransferChannel:output_type -> milvus.proto.common.Status - 132, // 294: milvus.proto.query.QueryCoord.CheckQueryNodeDistribution:output_type -> milvus.proto.common.Status - 132, // 295: milvus.proto.query.QueryCoord.UpdateLoadConfig:output_type -> milvus.proto.common.Status - 168, // 296: milvus.proto.query.QueryCoord.RunAnalyzer:output_type -> milvus.proto.milvus.RunAnalyzerResponse - 132, // 297: milvus.proto.query.QueryCoord.ValidateAnalyzer:output_type -> milvus.proto.common.Status - 169, // 298: milvus.proto.query.QueryNode.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates - 170, // 299: milvus.proto.query.QueryNode.GetTimeTickChannel:output_type -> milvus.proto.milvus.StringResponse - 170, // 300: milvus.proto.query.QueryNode.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse - 132, // 301: milvus.proto.query.QueryNode.WatchDmChannels:output_type -> milvus.proto.common.Status - 132, // 302: milvus.proto.query.QueryNode.UnsubDmChannel:output_type -> milvus.proto.common.Status - 132, // 303: milvus.proto.query.QueryNode.LoadSegments:output_type -> milvus.proto.common.Status - 132, // 304: milvus.proto.query.QueryNode.ReleaseCollection:output_type -> milvus.proto.common.Status - 132, // 305: milvus.proto.query.QueryNode.LoadPartitions:output_type -> milvus.proto.common.Status - 132, // 306: milvus.proto.query.QueryNode.ReleasePartitions:output_type -> milvus.proto.common.Status - 132, // 307: milvus.proto.query.QueryNode.ReleaseSegments:output_type -> milvus.proto.common.Status - 20, // 308: milvus.proto.query.QueryNode.GetSegmentInfo:output_type -> milvus.proto.query.GetSegmentInfoResponse - 132, // 309: milvus.proto.query.QueryNode.SyncReplicaSegments:output_type -> milvus.proto.common.Status - 171, // 310: milvus.proto.query.QueryNode.GetStatistics:output_type -> milvus.proto.internal.GetStatisticsResponse - 172, // 311: milvus.proto.query.QueryNode.Search:output_type -> milvus.proto.internal.SearchResults - 172, // 312: milvus.proto.query.QueryNode.SearchSegments:output_type -> milvus.proto.internal.SearchResults - 173, // 313: milvus.proto.query.QueryNode.Query:output_type -> milvus.proto.internal.RetrieveResults - 173, // 314: milvus.proto.query.QueryNode.QueryStream:output_type -> milvus.proto.internal.RetrieveResults - 173, // 315: milvus.proto.query.QueryNode.QuerySegments:output_type -> milvus.proto.internal.RetrieveResults - 173, // 316: milvus.proto.query.QueryNode.QueryStreamSegments:output_type -> milvus.proto.internal.RetrieveResults - 163, // 317: milvus.proto.query.QueryNode.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse - 164, // 318: milvus.proto.query.QueryNode.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse - 52, // 319: milvus.proto.query.QueryNode.GetDataDistribution:output_type -> milvus.proto.query.GetDataDistributionResponse - 132, // 320: milvus.proto.query.QueryNode.SyncDistribution:output_type -> milvus.proto.common.Status - 132, // 321: milvus.proto.query.QueryNode.Delete:output_type -> milvus.proto.common.Status - 71, // 322: milvus.proto.query.QueryNode.DeleteBatch:output_type -> milvus.proto.query.DeleteBatchResponse - 132, // 323: milvus.proto.query.QueryNode.UpdateSchema:output_type -> milvus.proto.common.Status - 168, // 324: milvus.proto.query.QueryNode.RunAnalyzer:output_type -> milvus.proto.milvus.RunAnalyzerResponse - 106, // 325: milvus.proto.query.QueryNode.GetHighlight:output_type -> milvus.proto.query.GetHighlightResponse - 132, // 326: milvus.proto.query.QueryNode.DropIndex:output_type -> milvus.proto.common.Status - 132, // 327: milvus.proto.query.QueryNode.ValidateAnalyzer:output_type -> milvus.proto.common.Status - 259, // [259:328] is the sub-list for method output_type - 190, // [190:259] is the sub-list for method input_type + 163, // 259: milvus.proto.query.QueryNode.SyncFileResource:input_type -> milvus.proto.internal.SyncFileResourceRequest + 9, // 260: milvus.proto.query.QueryCoord.ShowLoadCollections:output_type -> milvus.proto.query.ShowCollectionsResponse + 11, // 261: milvus.proto.query.QueryCoord.ShowLoadPartitions:output_type -> milvus.proto.query.ShowPartitionsResponse + 132, // 262: milvus.proto.query.QueryCoord.LoadPartitions:output_type -> milvus.proto.common.Status + 132, // 263: milvus.proto.query.QueryCoord.ReleasePartitions:output_type -> milvus.proto.common.Status + 132, // 264: milvus.proto.query.QueryCoord.LoadCollection:output_type -> milvus.proto.common.Status + 132, // 265: milvus.proto.query.QueryCoord.ReleaseCollection:output_type -> milvus.proto.common.Status + 132, // 266: milvus.proto.query.QueryCoord.SyncNewCreatedPartition:output_type -> milvus.proto.common.Status + 18, // 267: milvus.proto.query.QueryCoord.GetPartitionStates:output_type -> milvus.proto.query.GetPartitionStatesResponse + 20, // 268: milvus.proto.query.QueryCoord.GetLoadSegmentInfo:output_type -> milvus.proto.query.GetSegmentInfoResponse + 132, // 269: milvus.proto.query.QueryCoord.LoadBalance:output_type -> milvus.proto.common.Status + 164, // 270: milvus.proto.query.QueryCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse + 165, // 271: milvus.proto.query.QueryCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse + 166, // 272: milvus.proto.query.QueryCoord.GetReplicas:output_type -> milvus.proto.milvus.GetReplicasResponse + 22, // 273: milvus.proto.query.QueryCoord.GetShardLeaders:output_type -> milvus.proto.query.GetShardLeadersResponse + 167, // 274: milvus.proto.query.QueryCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse + 132, // 275: milvus.proto.query.QueryCoord.CreateResourceGroup:output_type -> milvus.proto.common.Status + 132, // 276: milvus.proto.query.QueryCoord.UpdateResourceGroups:output_type -> milvus.proto.common.Status + 132, // 277: milvus.proto.query.QueryCoord.DropResourceGroup:output_type -> milvus.proto.common.Status + 132, // 278: milvus.proto.query.QueryCoord.TransferNode:output_type -> milvus.proto.common.Status + 132, // 279: milvus.proto.query.QueryCoord.TransferReplica:output_type -> milvus.proto.common.Status + 168, // 280: milvus.proto.query.QueryCoord.ListResourceGroups:output_type -> milvus.proto.milvus.ListResourceGroupsResponse + 67, // 281: milvus.proto.query.QueryCoord.DescribeResourceGroup:output_type -> milvus.proto.query.DescribeResourceGroupResponse + 108, // 282: milvus.proto.query.QueryCoord.ListLoadedSegments:output_type -> milvus.proto.query.ListLoadedSegmentsResponse + 75, // 283: milvus.proto.query.QueryCoord.ListCheckers:output_type -> milvus.proto.query.ListCheckersResponse + 132, // 284: milvus.proto.query.QueryCoord.ActivateChecker:output_type -> milvus.proto.common.Status + 132, // 285: milvus.proto.query.QueryCoord.DeactivateChecker:output_type -> milvus.proto.common.Status + 83, // 286: milvus.proto.query.QueryCoord.ListQueryNode:output_type -> milvus.proto.query.ListQueryNodeResponse + 85, // 287: milvus.proto.query.QueryCoord.GetQueryNodeDistribution:output_type -> milvus.proto.query.GetQueryNodeDistributionResponse + 132, // 288: milvus.proto.query.QueryCoord.SuspendBalance:output_type -> milvus.proto.common.Status + 132, // 289: milvus.proto.query.QueryCoord.ResumeBalance:output_type -> milvus.proto.common.Status + 89, // 290: milvus.proto.query.QueryCoord.CheckBalanceStatus:output_type -> milvus.proto.query.CheckBalanceStatusResponse + 132, // 291: milvus.proto.query.QueryCoord.SuspendNode:output_type -> milvus.proto.common.Status + 132, // 292: milvus.proto.query.QueryCoord.ResumeNode:output_type -> milvus.proto.common.Status + 132, // 293: milvus.proto.query.QueryCoord.TransferSegment:output_type -> milvus.proto.common.Status + 132, // 294: milvus.proto.query.QueryCoord.TransferChannel:output_type -> milvus.proto.common.Status + 132, // 295: milvus.proto.query.QueryCoord.CheckQueryNodeDistribution:output_type -> milvus.proto.common.Status + 132, // 296: milvus.proto.query.QueryCoord.UpdateLoadConfig:output_type -> milvus.proto.common.Status + 169, // 297: milvus.proto.query.QueryCoord.RunAnalyzer:output_type -> milvus.proto.milvus.RunAnalyzerResponse + 132, // 298: milvus.proto.query.QueryCoord.ValidateAnalyzer:output_type -> milvus.proto.common.Status + 170, // 299: milvus.proto.query.QueryNode.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates + 171, // 300: milvus.proto.query.QueryNode.GetTimeTickChannel:output_type -> milvus.proto.milvus.StringResponse + 171, // 301: milvus.proto.query.QueryNode.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse + 132, // 302: milvus.proto.query.QueryNode.WatchDmChannels:output_type -> milvus.proto.common.Status + 132, // 303: milvus.proto.query.QueryNode.UnsubDmChannel:output_type -> milvus.proto.common.Status + 132, // 304: milvus.proto.query.QueryNode.LoadSegments:output_type -> milvus.proto.common.Status + 132, // 305: milvus.proto.query.QueryNode.ReleaseCollection:output_type -> milvus.proto.common.Status + 132, // 306: milvus.proto.query.QueryNode.LoadPartitions:output_type -> milvus.proto.common.Status + 132, // 307: milvus.proto.query.QueryNode.ReleasePartitions:output_type -> milvus.proto.common.Status + 132, // 308: milvus.proto.query.QueryNode.ReleaseSegments:output_type -> milvus.proto.common.Status + 20, // 309: milvus.proto.query.QueryNode.GetSegmentInfo:output_type -> milvus.proto.query.GetSegmentInfoResponse + 132, // 310: milvus.proto.query.QueryNode.SyncReplicaSegments:output_type -> milvus.proto.common.Status + 172, // 311: milvus.proto.query.QueryNode.GetStatistics:output_type -> milvus.proto.internal.GetStatisticsResponse + 173, // 312: milvus.proto.query.QueryNode.Search:output_type -> milvus.proto.internal.SearchResults + 173, // 313: milvus.proto.query.QueryNode.SearchSegments:output_type -> milvus.proto.internal.SearchResults + 174, // 314: milvus.proto.query.QueryNode.Query:output_type -> milvus.proto.internal.RetrieveResults + 174, // 315: milvus.proto.query.QueryNode.QueryStream:output_type -> milvus.proto.internal.RetrieveResults + 174, // 316: milvus.proto.query.QueryNode.QuerySegments:output_type -> milvus.proto.internal.RetrieveResults + 174, // 317: milvus.proto.query.QueryNode.QueryStreamSegments:output_type -> milvus.proto.internal.RetrieveResults + 164, // 318: milvus.proto.query.QueryNode.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse + 165, // 319: milvus.proto.query.QueryNode.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse + 52, // 320: milvus.proto.query.QueryNode.GetDataDistribution:output_type -> milvus.proto.query.GetDataDistributionResponse + 132, // 321: milvus.proto.query.QueryNode.SyncDistribution:output_type -> milvus.proto.common.Status + 132, // 322: milvus.proto.query.QueryNode.Delete:output_type -> milvus.proto.common.Status + 71, // 323: milvus.proto.query.QueryNode.DeleteBatch:output_type -> milvus.proto.query.DeleteBatchResponse + 132, // 324: milvus.proto.query.QueryNode.UpdateSchema:output_type -> milvus.proto.common.Status + 169, // 325: milvus.proto.query.QueryNode.RunAnalyzer:output_type -> milvus.proto.milvus.RunAnalyzerResponse + 106, // 326: milvus.proto.query.QueryNode.GetHighlight:output_type -> milvus.proto.query.GetHighlightResponse + 132, // 327: milvus.proto.query.QueryNode.DropIndex:output_type -> milvus.proto.common.Status + 132, // 328: milvus.proto.query.QueryNode.ValidateAnalyzer:output_type -> milvus.proto.common.Status + 132, // 329: milvus.proto.query.QueryNode.SyncFileResource:output_type -> milvus.proto.common.Status + 260, // [260:330] is the sub-list for method output_type + 190, // [190:260] is the sub-list for method input_type 190, // [190:190] is the sub-list for extension type_name 190, // [190:190] is the sub-list for extension extendee 0, // [0:190] is the sub-list for field type_name diff --git a/pkg/proto/querypb/query_coord_grpc.pb.go b/pkg/proto/querypb/query_coord_grpc.pb.go index 78fcb1cd11..f24b57e075 100644 --- a/pkg/proto/querypb/query_coord_grpc.pb.go +++ b/pkg/proto/querypb/query_coord_grpc.pb.go @@ -1555,6 +1555,7 @@ const ( QueryNode_GetHighlight_FullMethodName = "/milvus.proto.query.QueryNode/GetHighlight" QueryNode_DropIndex_FullMethodName = "/milvus.proto.query.QueryNode/DropIndex" QueryNode_ValidateAnalyzer_FullMethodName = "/milvus.proto.query.QueryNode/ValidateAnalyzer" + QueryNode_SyncFileResource_FullMethodName = "/milvus.proto.query.QueryNode/SyncFileResource" ) // QueryNodeClient is the client API for QueryNode service. @@ -1594,6 +1595,8 @@ type QueryNodeClient interface { GetHighlight(ctx context.Context, in *GetHighlightRequest, opts ...grpc.CallOption) (*GetHighlightResponse, error) DropIndex(ctx context.Context, in *DropIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) ValidateAnalyzer(ctx context.Context, in *ValidateAnalyzerRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + // file resource + SyncFileResource(ctx context.Context, in *internalpb.SyncFileResourceRequest, opts ...grpc.CallOption) (*commonpb.Status, error) } type queryNodeClient struct { @@ -1920,6 +1923,15 @@ func (c *queryNodeClient) ValidateAnalyzer(ctx context.Context, in *ValidateAnal return out, nil } +func (c *queryNodeClient) SyncFileResource(ctx context.Context, in *internalpb.SyncFileResourceRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryNode_SyncFileResource_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + // QueryNodeServer is the server API for QueryNode service. // All implementations should embed UnimplementedQueryNodeServer // for forward compatibility @@ -1957,6 +1969,8 @@ type QueryNodeServer interface { GetHighlight(context.Context, *GetHighlightRequest) (*GetHighlightResponse, error) DropIndex(context.Context, *DropIndexRequest) (*commonpb.Status, error) ValidateAnalyzer(context.Context, *ValidateAnalyzerRequest) (*commonpb.Status, error) + // file resource + SyncFileResource(context.Context, *internalpb.SyncFileResourceRequest) (*commonpb.Status, error) } // UnimplementedQueryNodeServer should be embedded to have forward compatible implementations. @@ -2053,6 +2067,9 @@ func (UnimplementedQueryNodeServer) DropIndex(context.Context, *DropIndexRequest func (UnimplementedQueryNodeServer) ValidateAnalyzer(context.Context, *ValidateAnalyzerRequest) (*commonpb.Status, error) { return nil, status.Errorf(codes.Unimplemented, "method ValidateAnalyzer not implemented") } +func (UnimplementedQueryNodeServer) SyncFileResource(context.Context, *internalpb.SyncFileResourceRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method SyncFileResource not implemented") +} // UnsafeQueryNodeServer may be embedded to opt out of forward compatibility for this service. // Use of this interface is not recommended, as added methods to QueryNodeServer will @@ -2611,6 +2628,24 @@ func _QueryNode_ValidateAnalyzer_Handler(srv interface{}, ctx context.Context, d return interceptor(ctx, in, info, handler) } +func _QueryNode_SyncFileResource_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(internalpb.SyncFileResourceRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryNodeServer).SyncFileResource(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryNode_SyncFileResource_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryNodeServer).SyncFileResource(ctx, req.(*internalpb.SyncFileResourceRequest)) + } + return interceptor(ctx, in, info, handler) +} + // QueryNode_ServiceDesc is the grpc.ServiceDesc for QueryNode service. // It's only intended for direct use with grpc.RegisterService, // and not to be introspected or modified (even as a copy) @@ -2730,6 +2765,10 @@ var QueryNode_ServiceDesc = grpc.ServiceDesc{ MethodName: "ValidateAnalyzer", Handler: _QueryNode_ValidateAnalyzer_Handler, }, + { + MethodName: "SyncFileResource", + Handler: _QueryNode_SyncFileResource_Handler, + }, }, Streams: []grpc.StreamDesc{ { diff --git a/pkg/util/paramtable/component_param.go b/pkg/util/paramtable/component_param.go index f95c83803d..260128f3c0 100644 --- a/pkg/util/paramtable/component_param.go +++ b/pkg/util/paramtable/component_param.go @@ -2497,6 +2497,8 @@ type queryCoordConfig struct { BalanceCheckCollectionMaxCount ParamItem `refreshable:"true"` ResourceExhaustionPenaltyDuration ParamItem `refreshable:"true"` ResourceExhaustionCleanupInterval ParamItem `refreshable:"true"` + + FileResourceMode ParamItem `refreshable:"false"` } func (p *queryCoordConfig) init(base *BaseTable) { @@ -2508,6 +2510,13 @@ func (p *queryCoordConfig) init(base *BaseTable) { } p.RetryNum.Init(base.mgr) + p.FileResourceMode = ParamItem{ + Key: "queryCoord.fileResource.mode", + Version: "2.6.3", + DefaultValue: "sync", + } + p.FileResourceMode.Init(base.mgr) + p.RetryInterval = ParamItem{ Key: "queryCoord.task.retryinterval", Version: "2.2.0", @@ -4631,6 +4640,7 @@ type dataCoordConfig struct { JSONStatsWriteBatchSize ParamItem `refreshable:"true"` RequestTimeoutSeconds ParamItem `refreshable:"true"` + FileResourceMode ParamItem `refreshable:"false"` } func (p *dataCoordConfig) init(base *BaseTable) { @@ -4643,6 +4653,13 @@ func (p *dataCoordConfig) init(base *BaseTable) { } p.WatchTimeoutInterval.Init(base.mgr) + p.FileResourceMode = ParamItem{ + Key: "dataCoord.fileResource.mode", + Version: "2.6.3", + DefaultValue: "sync", + } + p.FileResourceMode.Init(base.mgr) + p.LegacyVersionWithoutRPCWatch = ParamItem{ Key: "dataCoord.channel.legacyVersionWithoutRPCWatch", Version: "2.4.1",