Compare commits

...

6 Commits

Author SHA1 Message Date
congqixia
3bbc5d0825
fix: correct loop logic for timestamptz scalar index output (#46100)
Related to #46098

Fix the ReverseDataFromIndex function where the assignment of raw_data
to scalar_array and the break statement were incorrectly placed inside
the for loop for TIMESTAMPTZ data type. This caused QueryNode to panic
when outputting timestamptz fields from scalar index.

Move the assignment and break statement outside the loop to match the
pattern used by other data types like VARCHAR.

Signed-off-by: Congqi Xia <congqi.xia@zilliz.com>
2025-12-05 00:09:11 +08:00
Yiqing Lu
89b4c58266
enhance: bump etcd version to 3.5.25 (#46093)
As title.

Signed-off-by: AlintaLu <yiqing.lu@zilliz.com>
2025-12-04 17:37:15 +08:00
Xinyi7
59752f216d
fix: add check in batch_score function to prevent query node seg fault (#46025)
previously we saw that when doing reranker with phrase matching, the
query node throws a segmentation fault error.

github issue link: https://github.com/milvus-io/milvus/issues/45990

---------

Signed-off-by: Xinyi Jiang <xinyi.jiang@reddit.com>
Co-authored-by: Xinyi Jiang <xinyi.jiang@reddit.com>
2025-12-04 17:35:17 +08:00
XuanYang-cn
2cb6073bdd
test: Increase PyMilvus version to 2.7.0rc83 for master branch (#46072)
Automated daily bump from pymilvus master branch. Updates
tests/python_client/requirements.txt.

Signed-off-by: XuanYang-cn <xuan.yang@zilliz.com>
2025-12-04 17:33:20 +08:00
Zhen Ye
c22cdbbf9a
enhance: support proxy DQL forward (#46036)
issue: #45812

Signed-off-by: chyezh <chyezh@outlook.com>
2025-12-04 17:11:12 +08:00
aoiasd
354ab2f55e
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 <zhicheng.yue@zilliz.com>
2025-12-04 16:23:11 +08:00
70 changed files with 2952 additions and 460 deletions

View File

@ -5,7 +5,7 @@
- name: etcd
docker_container:
name: etcd
image: quay.io/coreos/etcd:v3.5.18
image: quay.io/coreos/etcd:v3.5.25
volumes:
- etcd_volume:/etcd
command: etcd -advertise-client-urls=http://127.0.0.1:2379 -listen-client-urls http://0.0.0.0:2379 --data-dir /etcd

View File

@ -2,7 +2,7 @@ version: '3.5'
services:
etcd:
image: quay.io/coreos/etcd:v3.5.18
image: quay.io/coreos/etcd:v3.5.25
environment:
- ETCD_AUTO_COMPACTION_MODE=revision
- ETCD_AUTO_COMPACTION_RETENTION=1000

View File

@ -2,7 +2,7 @@ version: '3.5'
services:
etcd:
image: quay.io/coreos/etcd:v3.5.18
image: quay.io/coreos/etcd:v3.5.25
environment:
- ETCD_AUTO_COMPACTION_MODE=revision
- ETCD_AUTO_COMPACTION_RETENTION=1000

View File

@ -3,7 +3,7 @@ version: '3.5'
services:
etcd:
container_name: milvus-etcd
image: quay.io/coreos/etcd:v3.5.18
image: quay.io/coreos/etcd:v3.5.25
environment:
- ETCD_AUTO_COMPACTION_MODE=revision
- ETCD_AUTO_COMPACTION_RETENTION=1000

View File

@ -3,7 +3,7 @@ version: '3.5'
services:
etcd:
container_name: milvus-etcd
image: quay.io/coreos/etcd:v3.5.18
image: quay.io/coreos/etcd:v3.5.25
environment:
- ETCD_AUTO_COMPACTION_MODE=revision
- ETCD_AUTO_COMPACTION_RETENTION=1000

View File

@ -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)

View File

@ -48,10 +48,18 @@ WeightScorer::batch_score(milvus::OpContext* op_ctx,
const FixedVector<int32_t>& offsets,
const TargetBitmap& bitmap,
std::vector<std::optional<float>>& boost_scores) {
auto bitmap_size = bitmap.size();
for (auto i = 0; i < offsets.size(); i++) {
if (bitmap[offsets[i]] > 0) {
set_score(boost_scores[i], mode);
auto offset = offsets[i];
// Bounds check: offset must be within bitmap size.
// Race condition: text index may lag behind vector index,
// causing offsets to reference rows not yet in text index.
if (offset >= 0 && static_cast<size_t>(offset) < bitmap_size) {
if (bitmap[offset] > 0) {
set_score(boost_scores[i], mode);
}
}
// If offset is out of bounds, treat as "no match" (don't apply boost)
}
};
@ -163,9 +171,9 @@ RandomScorer::random_score(milvus::OpContext* op_ctx,
"now only support int64 field as seed");
// TODO: Support varchar and int32 field as random field.
auto datas = array->scalars().long_data();
for (int i = 0; i < datas.data_size(); i++) {
auto a = datas.data()[i];
auto data = array->scalars().long_data();
for (int i = 0; i < data.data_size(); i++) {
auto a = data.data()[i];
auto random_score =
hash_to_double(MurmurHash3_x64_64_Special(a, seed_));
if (idx == nullptr) {

View File

@ -1003,10 +1003,10 @@ ReverseDataFromIndex(const index::IndexBase* index,
valid_data[i] = true;
}
raw_data[i] = raw.value();
auto obj = scalar_array->mutable_timestamptz_data();
*(obj->mutable_data()) = {raw_data.begin(), raw_data.end()};
break;
}
auto obj = scalar_array->mutable_timestamptz_data();
*(obj->mutable_data()) = {raw_data.begin(), raw_data.end()};
break;
}
case DataType::VARCHAR: {
using IndexType = index::ScalarIndex<std::string>;

View File

@ -43,6 +43,7 @@ set(MILVUS_TEST_FILES
test_index_wrapper.cpp
test_integer_overflow.cpp
test_query.cpp
test_scorer.cpp
test_sealed.cpp
test_storage.cpp
test_string_expr.cpp

View File

@ -0,0 +1,84 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#include <gtest/gtest.h>
#include <optional>
#include <vector>
#include "common/Types.h"
#include "rescores/Scorer.h"
#include "pb/plan.pb.h"
using namespace milvus;
using namespace milvus::rescores;
class WeightScorerTest : public ::testing::Test {
protected:
void
SetUp() override {
// Create a WeightScorer with no filter and weight of 2.0
scorer_ = std::make_unique<WeightScorer>(nullptr, 2.0f);
}
std::unique_ptr<WeightScorer> scorer_;
};
// Test: TargetBitmap batch_score with valid offsets (all within bitmap bounds)
TEST_F(WeightScorerTest, BatchScoreTargetBitmapValidOffsets) {
TargetBitmap bitmap(100);
bitmap.set(10);
bitmap.set(50);
bitmap.set(90);
// Offsets that are all within bitmap bounds
FixedVector<int32_t> offsets = {10, 20, 50, 90};
std::vector<std::optional<float>> boost_scores(offsets.size(),
std::nullopt);
proto::plan::FunctionMode mode = proto::plan::FunctionMode::FunctionModeSum;
scorer_->batch_score(nullptr, nullptr, mode, offsets, bitmap, boost_scores);
// Positions 10, 50, 90 should have scores (they are set in bitmap)
EXPECT_TRUE(boost_scores[0].has_value());
EXPECT_FALSE(boost_scores[1].has_value());
EXPECT_TRUE(boost_scores[2].has_value());
EXPECT_TRUE(boost_scores[3].has_value());
}
// Test: TargetBitmap batch_score with out-of-bounds offsets (should NOT crash)
TEST_F(WeightScorerTest, BatchScoreTargetBitmapOutOfBoundsOffsets) {
// Create a small bitmap of size 50
TargetBitmap bitmap(50);
bitmap.set(10); // Set bit at position 10
bitmap.set(40); // Set bit at position 40
// Offsets where some are OUT OF BOUNDS (>= 50)
// This simulates the race condition where text index lags behind vector index
FixedVector<int32_t> offsets = {10, 40, 60, 100, 200};
std::vector<std::optional<float>> boost_scores(offsets.size(),
std::nullopt);
proto::plan::FunctionMode mode = proto::plan::FunctionMode::FunctionModeSum;
// Should NOT crash! Out-of-bounds offsets should be safely skipped
ASSERT_NO_THROW(scorer_->batch_score(
nullptr, nullptr, mode, offsets, bitmap, boost_scores));
// In-bounds offsets should be scored correctly
EXPECT_TRUE(boost_scores[0].has_value());
EXPECT_TRUE(boost_scores[1].has_value());
// Out-of-bounds offsets should NOT have scores (safely skipped)
EXPECT_FALSE(boost_scores[2].has_value());
EXPECT_FALSE(boost_scores[3].has_value());
EXPECT_FALSE(boost_scores[4].has_value());
}

View File

@ -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
}

View File

@ -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))
}

View File

@ -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)

View File

@ -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())

View File

@ -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)

View File

@ -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
}

View File

@ -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)

View File

@ -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:

View File

@ -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
}

View File

@ -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()

View File

@ -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)
}

View File

@ -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,
},

View File

@ -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)

View File

@ -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
}

View File

@ -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)
})
}

View File

@ -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)
}

View File

@ -383,7 +383,7 @@ func wrapperProxyWithLimit(ctx context.Context, ginCtx *gin.Context, req any, ch
}
forwardHandler := func(reqCtx context.Context, req any) (any, error) {
interceptor := streaming.ForwardDMLToLegacyProxyUnaryServerInterceptor()
interceptor := streaming.ForwardLegacyProxyUnaryServerInterceptor()
return interceptor(reqCtx, req, &grpc.UnaryServerInfo{FullMethod: fullMethod}, func(ctx context.Context, req any) (interface{}, error) {
return handler(ctx, req)
})

View File

@ -240,7 +240,7 @@ func (s *Server) startExternalGrpc(errChan chan error) {
var unaryServerOption grpc.ServerOption
if enableCustomInterceptor {
unaryServerOption = grpc.UnaryInterceptor(grpc_middleware.ChainUnaryServer(
streaming.ForwardDMLToLegacyProxyUnaryServerInterceptor(),
streaming.ForwardLegacyProxyUnaryServerInterceptor(),
proxy.DatabaseInterceptor(),
UnaryRequestStatsInterceptor,
accesslog.UnaryAccessLogInterceptor,

View File

@ -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)
})
}

View File

@ -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)
}

View File

@ -60,7 +60,7 @@ func newForwardService(streamingCoordClient client.Client) *forwardServiceImpl {
}
type ForwardService interface {
ForwardDMLToLegacyProxy(ctx context.Context, request any) (any, error)
ForwardLegacyProxy(ctx context.Context, request any) (any, error)
}
// forwardServiceImpl is the implementation of FallbackService.
@ -74,13 +74,13 @@ type forwardServiceImpl struct {
rb resolver.Builder
}
// ForwardDMLToLegacyProxy forwards the DML request to the legacy proxy.
func (fs *forwardServiceImpl) ForwardDMLToLegacyProxy(ctx context.Context, request any) (any, error) {
// ForwardLegacyProxy forwards the request to the legacy proxy.
func (fs *forwardServiceImpl) ForwardLegacyProxy(ctx context.Context, request any) (any, error) {
if err := fs.checkIfForwardDisabledWithLock(ctx); err != nil {
return nil, err
}
return fs.forwardDMLToLegacyProxy(ctx, request)
return fs.forwardLegacyProxy(ctx, request)
}
// checkIfForwardDisabledWithLock checks if the forward is disabled with lock.
@ -91,8 +91,8 @@ func (fs *forwardServiceImpl) checkIfForwardDisabledWithLock(ctx context.Context
return fs.checkIfForwardDisabled(ctx)
}
// forwardDMLToLegacyProxy forwards the DML request to the legacy proxy.
func (fs *forwardServiceImpl) forwardDMLToLegacyProxy(ctx context.Context, request any) (any, error) {
// forwardLegacyProxy forwards the request to the legacy proxy.
func (fs *forwardServiceImpl) forwardLegacyProxy(ctx context.Context, request any) (any, error) {
s, err := fs.getLegacyProxyService(ctx)
if err != nil {
return nil, err
@ -106,6 +106,12 @@ func (fs *forwardServiceImpl) forwardDMLToLegacyProxy(ctx context.Context, reque
result, err = s.Delete(ctx, req)
case *milvuspb.UpsertRequest:
result, err = s.Upsert(ctx, req)
case *milvuspb.SearchRequest:
result, err = s.Search(ctx, req)
case *milvuspb.HybridSearchRequest:
result, err = s.HybridSearch(ctx, req)
case *milvuspb.QueryRequest:
result, err = s.Query(ctx, req)
default:
panic(fmt.Sprintf("unsupported request type: %T", request))
}
@ -178,7 +184,7 @@ func (fs *forwardServiceImpl) initLegacyProxy() {
})
fs.legacyProxy = lazygrpc.WithServiceCreator(conn, milvuspb.NewMilvusServiceClient)
fs.rb = rb
fs.Logger().Info("streaming service is not ready, legacy proxy is initiated to forward DML request", zap.Int("proxyPort", port))
fs.Logger().Info("streaming service is not ready, legacy proxy is initiated to forward request", zap.Int("proxyPort", port))
}
// getDialOptions returns the dial options for the legacy proxy.
@ -236,21 +242,24 @@ func (fs *forwardServiceImpl) markForwardDisabled() {
}
}
// ForwardDMLToLegacyProxyUnaryServerInterceptor forwards the DML request to the legacy proxy.
// ForwardLegacyProxyUnaryServerInterceptor forwards the request to the legacy proxy.
// When upgrading from 2.5.x to 2.6.x, the streaming service is not ready yet,
// the dml cannot be executed at new 2.6.x proxy until all 2.5.x proxies are down.
//
// so we need to forward the request to the 2.5.x proxy.
func ForwardDMLToLegacyProxyUnaryServerInterceptor() grpc.UnaryServerInterceptor {
func ForwardLegacyProxyUnaryServerInterceptor() grpc.UnaryServerInterceptor {
return func(ctx context.Context, req any, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
if info.FullMethod != milvuspb.MilvusService_Insert_FullMethodName &&
info.FullMethod != milvuspb.MilvusService_Delete_FullMethodName &&
info.FullMethod != milvuspb.MilvusService_Upsert_FullMethodName {
info.FullMethod != milvuspb.MilvusService_Upsert_FullMethodName &&
info.FullMethod != milvuspb.MilvusService_Search_FullMethodName &&
info.FullMethod != milvuspb.MilvusService_HybridSearch_FullMethodName &&
info.FullMethod != milvuspb.MilvusService_Query_FullMethodName {
return handler(ctx, req)
}
// try to forward the request to the legacy proxy.
resp, err := WAL().ForwardService().ForwardDMLToLegacyProxy(ctx, req)
resp, err := WAL().ForwardService().ForwardLegacyProxy(ctx, req)
if err == nil {
return resp, nil
}

View File

@ -66,13 +66,19 @@ func TestForwardDMLToLegacyProxy(t *testing.T) {
&milvuspb.DeleteRequest{},
&milvuspb.InsertRequest{},
&milvuspb.UpsertRequest{},
&milvuspb.SearchRequest{},
&milvuspb.HybridSearchRequest{},
&milvuspb.QueryRequest{},
}
methods := []string{
milvuspb.MilvusService_Delete_FullMethodName,
milvuspb.MilvusService_Insert_FullMethodName,
milvuspb.MilvusService_Upsert_FullMethodName,
milvuspb.MilvusService_Search_FullMethodName,
milvuspb.MilvusService_HybridSearch_FullMethodName,
milvuspb.MilvusService_Query_FullMethodName,
}
interceptor := ForwardDMLToLegacyProxyUnaryServerInterceptor()
interceptor := ForwardLegacyProxyUnaryServerInterceptor()
for idx, req := range reqs {
method := methods[idx]

View File

@ -244,7 +244,7 @@ func (n *noopWALAccesser) ForwardService() ForwardService {
type noopForwardService struct{}
func (n *noopForwardService) ForwardDMLToLegacyProxy(ctx context.Context, request any) (any, error) {
func (n *noopForwardService) ForwardLegacyProxy(ctx context.Context, request any) (any, error) {
return nil, ErrForwardDisabled
}

View File

@ -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 {

View File

@ -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"

View File

@ -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 {

View File

@ -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
}

View File

@ -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,
}
}

View File

@ -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)

View File

@ -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))

View File

@ -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)

View File

@ -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)

View File

@ -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))

View File

@ -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)

View File

@ -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()
}

View File

@ -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))
}

View File

@ -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
}

View File

@ -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 {

View File

@ -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)

View File

@ -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))

View File

@ -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
}

View File

@ -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))

View File

@ -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

View File

@ -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))
}
}

View File

@ -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))
}

View File

@ -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"
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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))

View File

@ -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,

View File

@ -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 {

View File

@ -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

View File

@ -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",

View File

@ -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;
}

View File

@ -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,
},

View File

@ -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------------------

View File

@ -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

View File

@ -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{
{

View File

@ -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",

View File

@ -28,8 +28,8 @@ pytest-parallel
pytest-random-order
# pymilvus
pymilvus==2.7.0rc82
pymilvus[bulk_writer]==2.7.0rc82
pymilvus==2.7.0rc83
pymilvus[bulk_writer]==2.7.0rc83
# for protobuf
protobuf>=5.29.5