mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-06 09:08:43 +08:00
Compare commits
6 Commits
0c63ed95bb
...
3bbc5d0825
| Author | SHA1 | Date | |
|---|---|---|---|
|
|
3bbc5d0825 | ||
|
|
89b4c58266 | ||
|
|
59752f216d | ||
|
|
2cb6073bdd | ||
|
|
c22cdbbf9a | ||
|
|
354ab2f55e |
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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) {
|
||||
|
||||
@ -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>;
|
||||
|
||||
@ -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
|
||||
|
||||
84
internal/core/unittest/test_scorer.cpp
Normal file
84
internal/core/unittest/test_scorer.cpp
Normal 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());
|
||||
}
|
||||
157
internal/datacoord/file_resource_manager.go
Normal file
157
internal/datacoord/file_resource_manager.go
Normal 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
|
||||
}
|
||||
222
internal/datacoord/file_resource_manager_test.go
Normal file
222
internal/datacoord/file_resource_manager_test.go
Normal 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))
|
||||
}
|
||||
@ -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)
|
||||
|
||||
|
||||
@ -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())
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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:
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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,
|
||||
},
|
||||
|
||||
@ -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)
|
||||
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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)
|
||||
})
|
||||
}
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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)
|
||||
})
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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)
|
||||
})
|
||||
}
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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]
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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"
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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,
|
||||
}
|
||||
}
|
||||
@ -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)
|
||||
|
||||
@ -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))
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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))
|
||||
|
||||
@ -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)
|
||||
|
||||
167
internal/querycoordv2/observers/file_resource_observer.go
Normal file
167
internal/querycoordv2/observers/file_resource_observer.go
Normal 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()
|
||||
}
|
||||
261
internal/querycoordv2/observers/file_resource_observer_test.go
Normal file
261
internal/querycoordv2/observers/file_resource_observer_test.go
Normal 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))
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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))
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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))
|
||||
|
||||
@ -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
|
||||
|
||||
288
internal/util/fileresource/manager.go
Normal file
288
internal/util/fileresource/manager.go
Normal 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))
|
||||
}
|
||||
}
|
||||
338
internal/util/fileresource/manager_test.go
Normal file
338
internal/util/fileresource/manager_test.go
Normal 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))
|
||||
}
|
||||
36
internal/util/fileresource/util.go
Normal file
36
internal/util/fileresource/util.go
Normal 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"
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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))
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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",
|
||||
|
||||
@ -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;
|
||||
}
|
||||
@ -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,
|
||||
},
|
||||
|
||||
@ -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------------------
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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{
|
||||
{
|
||||
|
||||
@ -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",
|
||||
|
||||
@ -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
|
||||
|
||||
|
||||
Loading…
x
Reference in New Issue
Block a user