mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-06 09:08:43 +08:00
enhance: sync file resource to querynode and datanode (#44480)
relate:https://github.com/milvus-io/milvus/issues/43687 Support use file resource with sync mode. Auto download or remove file resource to local when user add or remove file resource. Sync file resource to node when find new node session. --------- Signed-off-by: aoiasd <zhicheng.yue@zilliz.com>
This commit is contained in:
parent
0c63ed95bb
commit
354ab2f55e
@ -211,6 +211,10 @@ func (s *mixCoordImpl) initInternal() error {
|
|||||||
log.Error("queryCoord start failed", zap.Error(err))
|
log.Error("queryCoord start failed", zap.Error(err))
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if err := s.datacoordServer.SyncFileResources(s.ctx); err != nil {
|
||||||
|
log.Error("init file resources failed", zap.Error(err))
|
||||||
|
}
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -883,6 +887,10 @@ func (s *mixCoordImpl) GetQcMetrics(ctx context.Context, in *milvuspb.GetMetrics
|
|||||||
return s.queryCoordServer.GetMetrics(ctx, in)
|
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
|
// QueryCoordServer
|
||||||
func (s *mixCoordImpl) ActivateChecker(ctx context.Context, req *querypb.ActivateCheckerRequest) (*commonpb.Status, error) {
|
func (s *mixCoordImpl) ActivateChecker(ctx context.Context, req *querypb.ActivateCheckerRequest) (*commonpb.Status, error) {
|
||||||
return s.queryCoordServer.ActivateChecker(ctx, req)
|
return s.queryCoordServer.ActivateChecker(ctx, req)
|
||||||
|
|||||||
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().ListCompactionTask(mock.Anything).Return(nil, nil)
|
||||||
catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil)
|
catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil)
|
||||||
catalog.EXPECT().ListStatsTasks(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())
|
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().ListCompactionTask(mock.Anything).Return(nil, nil)
|
||||||
catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil)
|
catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil)
|
||||||
catalog.EXPECT().ListStatsTasks(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 := allocator.NewMockAllocator(t)
|
||||||
|
|
||||||
|
|||||||
@ -63,7 +63,7 @@ func (s *ImportInspectorSuite) SetupTest() {
|
|||||||
s.catalog.EXPECT().ListCompactionTask(mock.Anything).Return(nil, nil)
|
s.catalog.EXPECT().ListCompactionTask(mock.Anything).Return(nil, nil)
|
||||||
s.catalog.EXPECT().ListPartitionStatsInfos(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().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())
|
s.alloc = allocator.NewMockAllocator(s.T())
|
||||||
broker := broker.NewMockBroker(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().ListCompactionTask(mock.Anything).Return(nil, nil)
|
||||||
catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil)
|
catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil)
|
||||||
catalog.EXPECT().ListStatsTasks(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 := broker.NewMockBroker(t)
|
||||||
broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(nil, nil)
|
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().ListCompactionTask(mock.Anything).Return(nil, nil)
|
||||||
catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil)
|
catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil)
|
||||||
catalog.EXPECT().ListStatsTasks(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 := broker2.NewMockBroker(t)
|
||||||
broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(&rootcoordpb.ShowCollectionIDsResponse{}, nil)
|
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().ListCompactionTask(mock.Anything).Return(nil, nil)
|
||||||
catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil)
|
catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil)
|
||||||
catalog.EXPECT().ListStatsTasks(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 := allocator.NewMockAllocator(t)
|
||||||
alloc.EXPECT().AllocN(mock.Anything).RunAndReturn(func(n int64) (int64, int64, error) {
|
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().ListCompactionTask(mock.Anything).Return(nil, nil)
|
||||||
catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil)
|
catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil)
|
||||||
catalog.EXPECT().ListStatsTasks(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 := allocator.NewMockAllocator(t)
|
||||||
alloc.EXPECT().AllocN(mock.Anything).RunAndReturn(func(n int64) (int64, int64, error) {
|
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().ListCompactionTask(mock.Anything).Return(nil, nil)
|
||||||
catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil)
|
catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil)
|
||||||
catalog.EXPECT().ListStatsTasks(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)
|
importMeta, err := NewImportMeta(context.TODO(), catalog, nil, nil)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
@ -616,7 +616,7 @@ func TestImportUtil_GetImportProgress(t *testing.T) {
|
|||||||
catalog.EXPECT().ListCompactionTask(mock.Anything).Return(nil, nil)
|
catalog.EXPECT().ListCompactionTask(mock.Anything).Return(nil, nil)
|
||||||
catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil)
|
catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil)
|
||||||
catalog.EXPECT().ListStatsTasks(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)
|
importMeta, err := NewImportMeta(context.TODO(), catalog, nil, nil)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|||||||
@ -36,13 +36,13 @@ import (
|
|||||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||||
"github.com/milvus-io/milvus/internal/datacoord/broker"
|
"github.com/milvus-io/milvus/internal/datacoord/broker"
|
||||||
"github.com/milvus-io/milvus/internal/metastore"
|
"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/storage"
|
||||||
"github.com/milvus-io/milvus/internal/util/segmentutil"
|
"github.com/milvus-io/milvus/internal/util/segmentutil"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/common"
|
"github.com/milvus-io/milvus/pkg/v2/common"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/metrics"
|
"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/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/proto/rootcoordpb"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/util/conc"
|
"github.com/milvus-io/milvus/pkg/v2/util/conc"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/util/funcutil"
|
"github.com/milvus-io/milvus/pkg/v2/util/funcutil"
|
||||||
@ -102,8 +102,9 @@ type meta struct {
|
|||||||
externalCollectionTaskMeta *externalCollectionTaskMeta
|
externalCollectionTaskMeta *externalCollectionTaskMeta
|
||||||
|
|
||||||
// File Resource Meta
|
// File Resource Meta
|
||||||
resourceMeta map[string]*model.FileResource
|
resourceMeta map[string]*internalpb.FileResourceInfo // name -> info
|
||||||
resourceLock lock.RWMutex
|
resourceVersion uint64
|
||||||
|
resourceLock lock.RWMutex
|
||||||
}
|
}
|
||||||
|
|
||||||
func (m *meta) GetIndexMeta() *indexMeta {
|
func (m *meta) GetIndexMeta() *indexMeta {
|
||||||
@ -204,7 +205,7 @@ func newMeta(ctx context.Context, catalog metastore.DataCoordCatalog, chunkManag
|
|||||||
compactionTaskMeta: ctm,
|
compactionTaskMeta: ctm,
|
||||||
statsTaskMeta: stm,
|
statsTaskMeta: stm,
|
||||||
// externalCollectionTaskMeta: ectm,
|
// externalCollectionTaskMeta: ectm,
|
||||||
resourceMeta: make(map[string]*model.FileResource),
|
resourceMeta: make(map[string]*internalpb.FileResourceInfo),
|
||||||
}
|
}
|
||||||
err = mt.reloadFromKV(ctx, broker)
|
err = mt.reloadFromKV(ctx, broker)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -2417,20 +2418,21 @@ func (m *meta) reloadFileResourceMeta(ctx context.Context) error {
|
|||||||
m.resourceLock.Lock()
|
m.resourceLock.Lock()
|
||||||
defer m.resourceLock.Unlock()
|
defer m.resourceLock.Unlock()
|
||||||
|
|
||||||
resources, err := m.catalog.ListFileResource(ctx)
|
resources, version, err := m.catalog.ListFileResource(ctx)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
m.resourceMeta = make(map[string]*model.FileResource)
|
m.resourceMeta = make(map[string]*internalpb.FileResourceInfo)
|
||||||
for _, resource := range resources {
|
for _, resource := range resources {
|
||||||
m.resourceMeta[resource.Name] = resource
|
m.resourceMeta[resource.Name] = resource
|
||||||
}
|
}
|
||||||
|
m.resourceVersion = version
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// AddFileResource add file resource to meta
|
// 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()
|
m.resourceLock.Lock()
|
||||||
defer m.resourceLock.Unlock()
|
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"))
|
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 {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
m.resourceMeta[resource.Name] = resource
|
m.resourceMeta[resource.Name] = resource
|
||||||
|
m.resourceVersion += 1
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -2453,21 +2456,22 @@ func (m *meta) RemoveFileResource(ctx context.Context, name string) error {
|
|||||||
defer m.resourceLock.Unlock()
|
defer m.resourceLock.Unlock()
|
||||||
|
|
||||||
if resource, ok := m.resourceMeta[name]; ok {
|
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 {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
delete(m.resourceMeta, name)
|
delete(m.resourceMeta, name)
|
||||||
|
m.resourceVersion += 1
|
||||||
}
|
}
|
||||||
|
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// ListFileResource list file resources from meta
|
// 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()
|
m.resourceLock.RLock()
|
||||||
defer m.resourceLock.RUnlock()
|
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/kv"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/metrics"
|
"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/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/proto/rootcoordpb"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/util"
|
"github.com/milvus-io/milvus/pkg/v2/util"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
||||||
@ -129,7 +130,7 @@ func (suite *MetaReloadSuite) TestReloadFromKV() {
|
|||||||
},
|
},
|
||||||
}, nil)
|
}, 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().ListIndexes(mock.Anything).Return([]*model.Index{}, nil)
|
||||||
suite.catalog.EXPECT().ListSegmentIndexes(mock.Anything).Return([]*model.SegmentIndex{}, nil)
|
suite.catalog.EXPECT().ListSegmentIndexes(mock.Anything).Return([]*model.SegmentIndex{}, nil)
|
||||||
suite.catalog.EXPECT().ListAnalyzeTasks(mock.Anything).Return(nil, nil)
|
suite.catalog.EXPECT().ListAnalyzeTasks(mock.Anything).Return(nil, nil)
|
||||||
@ -175,7 +176,7 @@ func (suite *MetaReloadSuite) TestReloadFromKV() {
|
|||||||
},
|
},
|
||||||
}, nil)
|
}, 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().ListIndexes(mock.Anything).Return([]*model.Index{}, nil)
|
||||||
suite.catalog.EXPECT().ListSegmentIndexes(mock.Anything).Return([]*model.SegmentIndex{}, nil)
|
suite.catalog.EXPECT().ListSegmentIndexes(mock.Anything).Return([]*model.SegmentIndex{}, nil)
|
||||||
suite.catalog.EXPECT().ListAnalyzeTasks(mock.Anything).Return(nil, 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().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().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().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, "", "")
|
catalog := datacoord.NewCatalog(metakv, "", "")
|
||||||
broker := broker.NewMockBroker(t)
|
broker := broker.NewMockBroker(t)
|
||||||
broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(nil, nil)
|
broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(nil, nil)
|
||||||
@ -714,6 +716,7 @@ func TestMeta_Basic(t *testing.T) {
|
|||||||
|
|
||||||
metakv2 := mockkv.NewMetaKv(t)
|
metakv2 := mockkv.NewMetaKv(t)
|
||||||
metakv2.EXPECT().Save(mock.Anything, mock.Anything, mock.Anything).Return(nil).Maybe()
|
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().MultiSave(mock.Anything, mock.Anything).Return(nil).Maybe()
|
||||||
metakv2.EXPECT().Remove(mock.Anything, mock.Anything).Return(errors.New("failed")).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()
|
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().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().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().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, "", "")
|
catalog := datacoord.NewCatalog(metakv, "", "")
|
||||||
broker := broker.NewMockBroker(t)
|
broker := broker.NewMockBroker(t)
|
||||||
broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(nil, nil)
|
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().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) {
|
mockNodeManager.EXPECT().GetClient(mock.Anything).RunAndReturn(func(nodeID int64) (types.DataNodeClient, error) {
|
||||||
switch nodeID {
|
switch nodeID {
|
||||||
case 100:
|
case 100:
|
||||||
|
|||||||
@ -252,6 +252,10 @@ func (m *mockMixCoord) GetComponentStates(ctx context.Context, req *milvuspb.Get
|
|||||||
}, nil
|
}, 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) {
|
func (m *mockMixCoord) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) {
|
||||||
panic("not implemented") // TODO: Implement
|
panic("not implemented") // TODO: Implement
|
||||||
}
|
}
|
||||||
|
|||||||
@ -166,6 +166,9 @@ type Server struct {
|
|||||||
broker broker.Broker
|
broker broker.Broker
|
||||||
|
|
||||||
metricsRequest *metricsinfo.MetricsRequest
|
metricsRequest *metricsinfo.MetricsRequest
|
||||||
|
|
||||||
|
// file resource
|
||||||
|
fileManager *FileResourceManager
|
||||||
}
|
}
|
||||||
|
|
||||||
type CollectionNameInfo struct {
|
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.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)
|
s.serverLoopCtx, s.serverLoopCancel = context.WithCancel(s.ctx)
|
||||||
|
|
||||||
RegisterDDLCallbacks(s)
|
RegisterDDLCallbacks(s)
|
||||||
@ -432,7 +437,8 @@ func (s *Server) Start() error {
|
|||||||
func (s *Server) startDataCoord() {
|
func (s *Server) startDataCoord() {
|
||||||
s.startTaskScheduler()
|
s.startTaskScheduler()
|
||||||
s.startServerLoop()
|
s.startServerLoop()
|
||||||
|
s.fileManager.Start()
|
||||||
|
s.fileManager.Notify()
|
||||||
s.afterStart()
|
s.afterStart()
|
||||||
s.UpdateStateCode(commonpb.StateCode_Healthy)
|
s.UpdateStateCode(commonpb.StateCode_Healthy)
|
||||||
sessionutil.SaveServerInfo(typeutil.MixCoordRole, s.session.GetServerID())
|
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()))
|
zap.String("event type", event.EventType.String()))
|
||||||
return nil
|
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:
|
case sessionutil.SessionDelEvent:
|
||||||
log.Info("received datanode unregister",
|
log.Info("received datanode unregister",
|
||||||
zap.String("address", info.Address),
|
zap.String("address", info.Address),
|
||||||
@ -1051,6 +1064,7 @@ func (s *Server) Stop() error {
|
|||||||
s.stopServerLoop()
|
s.stopServerLoop()
|
||||||
log.Info("datacoord stopServerLoop stopped")
|
log.Info("datacoord stopServerLoop stopped")
|
||||||
|
|
||||||
|
s.fileManager.Close()
|
||||||
s.globalScheduler.Stop()
|
s.globalScheduler.Stop()
|
||||||
s.importInspector.Close()
|
s.importInspector.Close()
|
||||||
s.importChecker.Close()
|
s.importChecker.Close()
|
||||||
|
|||||||
@ -35,7 +35,6 @@ import (
|
|||||||
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
||||||
"github.com/milvus-io/milvus/internal/coordinator/snmanager"
|
"github.com/milvus-io/milvus/internal/coordinator/snmanager"
|
||||||
"github.com/milvus-io/milvus/internal/metastore/kv/binlog"
|
"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/storage"
|
||||||
"github.com/milvus-io/milvus/internal/util/componentutil"
|
"github.com/milvus-io/milvus/internal/util/componentutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/importutilv2"
|
"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
|
return merr.Status(err), nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// Convert to model.FileResource
|
// Convert to internalpb.FileResourceInfo
|
||||||
resource := &model.FileResource{
|
resource := &internalpb.FileResourceInfo{
|
||||||
ID: id,
|
Id: id,
|
||||||
Name: req.GetName(),
|
Name: req.GetName(),
|
||||||
Path: req.GetPath(),
|
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))
|
log.Ctx(ctx).Warn("AddFileResource fail", zap.Error(err))
|
||||||
return merr.Status(err), nil
|
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")
|
log.Ctx(ctx).Info("AddFileResource success")
|
||||||
return merr.Success(), nil
|
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))
|
log.Ctx(ctx).Warn("RemoveFileResource fail", zap.Error(err))
|
||||||
return merr.Status(err), nil
|
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")
|
log.Ctx(ctx).Info("RemoveFileResource success")
|
||||||
return merr.Success(), nil
|
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")
|
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))
|
fileResources := make([]*milvuspb.FileResourceInfo, 0, len(resources))
|
||||||
for _, resource := range resources {
|
for _, resource := range resources {
|
||||||
fileResources = append(fileResources, &milvuspb.FileResourceInfo{
|
fileResources = append(fileResources, &milvuspb.FileResourceInfo{
|
||||||
@ -2143,3 +2150,9 @@ func (s *Server) CreateExternalCollection(ctx context.Context, req *msgpb.Create
|
|||||||
Status: merr.Success(),
|
Status: merr.Success(),
|
||||||
}, nil
|
}, 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{
|
server := &Server{
|
||||||
idAllocator: globalIDAllocator.NewTestGlobalIDAllocator(mockAllocator),
|
idAllocator: globalIDAllocator.NewTestGlobalIDAllocator(mockAllocator),
|
||||||
|
mixCoord: newMockMixCoord(),
|
||||||
meta: &meta{
|
meta: &meta{
|
||||||
resourceMeta: make(map[string]*model.FileResource),
|
resourceMeta: make(map[string]*internalpb.FileResourceInfo),
|
||||||
catalog: mockCatalog,
|
catalog: mockCatalog,
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
@ -1791,9 +1792,9 @@ func TestServer_AddFileResource(t *testing.T) {
|
|||||||
Path: "/path/to/resource",
|
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 resource.Name == "test_resource" && resource.Path == "/path/to/resource"
|
||||||
})).Return(nil)
|
}), mock.Anything).Return(nil)
|
||||||
|
|
||||||
resp, err := server.AddFileResource(context.Background(), req)
|
resp, err := server.AddFileResource(context.Background(), req)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
@ -1822,7 +1823,7 @@ func TestServer_AddFileResource(t *testing.T) {
|
|||||||
server := &Server{
|
server := &Server{
|
||||||
idAllocator: globalIDAllocator.NewTestGlobalIDAllocator(mockAllocator),
|
idAllocator: globalIDAllocator.NewTestGlobalIDAllocator(mockAllocator),
|
||||||
meta: &meta{
|
meta: &meta{
|
||||||
resourceMeta: make(map[string]*model.FileResource),
|
resourceMeta: make(map[string]*internalpb.FileResourceInfo),
|
||||||
catalog: mockCatalog,
|
catalog: mockCatalog,
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
@ -1846,7 +1847,7 @@ func TestServer_AddFileResource(t *testing.T) {
|
|||||||
server := &Server{
|
server := &Server{
|
||||||
idAllocator: globalIDAllocator.NewTestGlobalIDAllocator(mockAllocator),
|
idAllocator: globalIDAllocator.NewTestGlobalIDAllocator(mockAllocator),
|
||||||
meta: &meta{
|
meta: &meta{
|
||||||
resourceMeta: make(map[string]*model.FileResource),
|
resourceMeta: make(map[string]*internalpb.FileResourceInfo),
|
||||||
catalog: mockCatalog,
|
catalog: mockCatalog,
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
@ -1857,7 +1858,7 @@ func TestServer_AddFileResource(t *testing.T) {
|
|||||||
Path: "/path/to/resource",
|
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)
|
resp, err := server.AddFileResource(context.Background(), req)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
@ -1869,8 +1870,8 @@ func TestServer_AddFileResource(t *testing.T) {
|
|||||||
mockAllocator := tso.NewMockAllocator()
|
mockAllocator := tso.NewMockAllocator()
|
||||||
mockAllocator.GenerateTSOF = func(count uint32) (uint64, error) { return 100, nil }
|
mockAllocator.GenerateTSOF = func(count uint32) (uint64, error) { return 100, nil }
|
||||||
|
|
||||||
existingResource := &model.FileResource{
|
existingResource := &internalpb.FileResourceInfo{
|
||||||
ID: 1,
|
Id: 1,
|
||||||
Name: "test_resource",
|
Name: "test_resource",
|
||||||
Path: "/existing/path",
|
Path: "/existing/path",
|
||||||
}
|
}
|
||||||
@ -1878,7 +1879,7 @@ func TestServer_AddFileResource(t *testing.T) {
|
|||||||
server := &Server{
|
server := &Server{
|
||||||
idAllocator: globalIDAllocator.NewTestGlobalIDAllocator(mockAllocator),
|
idAllocator: globalIDAllocator.NewTestGlobalIDAllocator(mockAllocator),
|
||||||
meta: &meta{
|
meta: &meta{
|
||||||
resourceMeta: map[string]*model.FileResource{
|
resourceMeta: map[string]*internalpb.FileResourceInfo{
|
||||||
"test_resource": existingResource,
|
"test_resource": existingResource,
|
||||||
},
|
},
|
||||||
catalog: mockCatalog,
|
catalog: mockCatalog,
|
||||||
@ -1902,19 +1903,20 @@ func TestServer_RemoveFileResource(t *testing.T) {
|
|||||||
t.Run("success", func(t *testing.T) {
|
t.Run("success", func(t *testing.T) {
|
||||||
mockCatalog := mocks.NewDataCoordCatalog(t)
|
mockCatalog := mocks.NewDataCoordCatalog(t)
|
||||||
|
|
||||||
existingResource := &model.FileResource{
|
existingResource := &internalpb.FileResourceInfo{
|
||||||
ID: 1,
|
Id: 1,
|
||||||
Name: "test_resource",
|
Name: "test_resource",
|
||||||
Path: "/path/to/resource",
|
Path: "/path/to/resource",
|
||||||
}
|
}
|
||||||
|
|
||||||
server := &Server{
|
server := &Server{
|
||||||
meta: &meta{
|
meta: &meta{
|
||||||
resourceMeta: map[string]*model.FileResource{
|
resourceMeta: map[string]*internalpb.FileResourceInfo{
|
||||||
"test_resource": existingResource,
|
"test_resource": existingResource,
|
||||||
},
|
},
|
||||||
catalog: mockCatalog,
|
catalog: mockCatalog,
|
||||||
},
|
},
|
||||||
|
mixCoord: newMockMixCoord(),
|
||||||
}
|
}
|
||||||
server.stateCode.Store(commonpb.StateCode_Healthy)
|
server.stateCode.Store(commonpb.StateCode_Healthy)
|
||||||
|
|
||||||
@ -1923,7 +1925,7 @@ func TestServer_RemoveFileResource(t *testing.T) {
|
|||||||
Name: "test_resource",
|
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)
|
resp, err := server.RemoveFileResource(context.Background(), req)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
@ -1948,9 +1950,10 @@ func TestServer_RemoveFileResource(t *testing.T) {
|
|||||||
|
|
||||||
server := &Server{
|
server := &Server{
|
||||||
meta: &meta{
|
meta: &meta{
|
||||||
resourceMeta: make(map[string]*model.FileResource),
|
resourceMeta: make(map[string]*internalpb.FileResourceInfo),
|
||||||
catalog: mockCatalog,
|
catalog: mockCatalog,
|
||||||
},
|
},
|
||||||
|
mixCoord: newMockMixCoord(),
|
||||||
}
|
}
|
||||||
server.stateCode.Store(commonpb.StateCode_Healthy)
|
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) {
|
t.Run("catalog remove error", func(t *testing.T) {
|
||||||
mockCatalog := mocks.NewDataCoordCatalog(t)
|
mockCatalog := mocks.NewDataCoordCatalog(t)
|
||||||
|
|
||||||
existingResource := &model.FileResource{
|
existingResource := &internalpb.FileResourceInfo{
|
||||||
ID: 1,
|
Id: 1,
|
||||||
Name: "test_resource",
|
Name: "test_resource",
|
||||||
Path: "/path/to/resource",
|
Path: "/path/to/resource",
|
||||||
}
|
}
|
||||||
|
|
||||||
server := &Server{
|
server := &Server{
|
||||||
meta: &meta{
|
meta: &meta{
|
||||||
resourceMeta: map[string]*model.FileResource{
|
resourceMeta: map[string]*internalpb.FileResourceInfo{
|
||||||
"test_resource": existingResource,
|
"test_resource": existingResource,
|
||||||
},
|
},
|
||||||
catalog: mockCatalog,
|
catalog: mockCatalog,
|
||||||
@ -1986,7 +1989,7 @@ func TestServer_RemoveFileResource(t *testing.T) {
|
|||||||
Name: "test_resource",
|
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)
|
resp, err := server.RemoveFileResource(context.Background(), req)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
@ -2000,7 +2003,7 @@ func TestServer_ListFileResources(t *testing.T) {
|
|||||||
|
|
||||||
server := &Server{
|
server := &Server{
|
||||||
meta: &meta{
|
meta: &meta{
|
||||||
resourceMeta: make(map[string]*model.FileResource),
|
resourceMeta: make(map[string]*internalpb.FileResourceInfo),
|
||||||
catalog: mockCatalog,
|
catalog: mockCatalog,
|
||||||
},
|
},
|
||||||
}
|
}
|
||||||
@ -2020,20 +2023,20 @@ func TestServer_ListFileResources(t *testing.T) {
|
|||||||
t.Run("success with resources", func(t *testing.T) {
|
t.Run("success with resources", func(t *testing.T) {
|
||||||
mockCatalog := mocks.NewDataCoordCatalog(t)
|
mockCatalog := mocks.NewDataCoordCatalog(t)
|
||||||
|
|
||||||
resource1 := &model.FileResource{
|
resource1 := &internalpb.FileResourceInfo{
|
||||||
ID: 1,
|
Id: 1,
|
||||||
Name: "resource1",
|
Name: "resource1",
|
||||||
Path: "/path/to/resource1",
|
Path: "/path/to/resource1",
|
||||||
}
|
}
|
||||||
resource2 := &model.FileResource{
|
resource2 := &internalpb.FileResourceInfo{
|
||||||
ID: 2,
|
Id: 2,
|
||||||
Name: "resource2",
|
Name: "resource2",
|
||||||
Path: "/path/to/resource2",
|
Path: "/path/to/resource2",
|
||||||
}
|
}
|
||||||
|
|
||||||
server := &Server{
|
server := &Server{
|
||||||
meta: &meta{
|
meta: &meta{
|
||||||
resourceMeta: map[string]*model.FileResource{
|
resourceMeta: map[string]*internalpb.FileResourceInfo{
|
||||||
"resource1": resource1,
|
"resource1": resource1,
|
||||||
"resource2": resource2,
|
"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/commonpb"
|
||||||
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
|
"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/compactor"
|
||||||
"github.com/milvus-io/milvus/internal/datanode/importv2"
|
"github.com/milvus-io/milvus/internal/datanode/importv2"
|
||||||
"github.com/milvus-io/milvus/internal/datanode/index"
|
"github.com/milvus-io/milvus/internal/datanode/index"
|
||||||
"github.com/milvus-io/milvus/internal/flushcommon/syncmgr"
|
"github.com/milvus-io/milvus/internal/flushcommon/syncmgr"
|
||||||
"github.com/milvus-io/milvus/internal/types"
|
"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/internal/util/sessionutil"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/metrics"
|
"github.com/milvus-io/milvus/pkg/v2/metrics"
|
||||||
@ -194,6 +196,19 @@ func (node *DataNode) Init() error {
|
|||||||
syncMgr := syncmgr.NewSyncManager(nil)
|
syncMgr := syncmgr.NewSyncManager(nil)
|
||||||
node.syncMgr = syncMgr
|
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.importTaskMgr = importv2.NewTaskManager()
|
||||||
node.importScheduler = importv2.NewScheduler(node.importTaskMgr)
|
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/compactor"
|
||||||
"github.com/milvus-io/milvus/internal/datanode/importv2"
|
"github.com/milvus-io/milvus/internal/datanode/importv2"
|
||||||
"github.com/milvus-io/milvus/internal/flushcommon/io"
|
"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/hookutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/importutilv2"
|
"github.com/milvus-io/milvus/internal/util/importutilv2"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/common"
|
"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
|
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)
|
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) {
|
func (s *Server) DropTask(ctx context.Context, request *workerpb.DropTaskRequest) (*commonpb.Status, error) {
|
||||||
return s.datanode.DropTask(ctx, request)
|
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)
|
||||||
|
}
|
||||||
|
|||||||
@ -416,3 +416,10 @@ func (c *Client) GetHighlight(ctx context.Context, req *querypb.GetHighlightRequ
|
|||||||
return client.GetHighlight(ctx, req)
|
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) {
|
func (s *Server) GetHighlight(ctx context.Context, req *querypb.GetHighlightRequest) (*querypb.GetHighlightResponse, error) {
|
||||||
return s.querynode.GetHighlight(ctx, req)
|
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)
|
||||||
|
}
|
||||||
|
|||||||
@ -8,6 +8,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/metastore/model"
|
"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/datapb"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
|
"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/querypb"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
|
||||||
@ -185,9 +186,9 @@ type DataCoordCatalog interface {
|
|||||||
DropUpdateExternalCollectionTask(ctx context.Context, taskID typeutil.UniqueID) error
|
DropUpdateExternalCollectionTask(ctx context.Context, taskID typeutil.UniqueID) error
|
||||||
|
|
||||||
// Analyzer Resource
|
// Analyzer Resource
|
||||||
SaveFileResource(ctx context.Context, resource *model.FileResource) error
|
SaveFileResource(ctx context.Context, resource *internalpb.FileResourceInfo, version uint64) error
|
||||||
RemoveFileResource(ctx context.Context, resourceID int64) error
|
RemoveFileResource(ctx context.Context, resourceID int64, version uint64) error
|
||||||
ListFileResource(ctx context.Context) ([]*model.FileResource, error)
|
ListFileResource(ctx context.Context) ([]*internalpb.FileResourceInfo, uint64, error)
|
||||||
}
|
}
|
||||||
|
|
||||||
type QueryCoordCatalog interface {
|
type QueryCoordCatalog interface {
|
||||||
|
|||||||
@ -34,7 +34,8 @@ const (
|
|||||||
PartitionStatsCurrentVersionPrefix = MetaPrefix + "/current-partition-stats-version"
|
PartitionStatsCurrentVersionPrefix = MetaPrefix + "/current-partition-stats-version"
|
||||||
StatsTaskPrefix = MetaPrefix + "/stats-task"
|
StatsTaskPrefix = MetaPrefix + "/stats-task"
|
||||||
UpdateExternalCollectionTaskPrefix = MetaPrefix + "/update-external-collection-task"
|
UpdateExternalCollectionTaskPrefix = MetaPrefix + "/update-external-collection-task"
|
||||||
FileResourceMetaPrefix = MetaPrefix + "/file_resource"
|
FileResourceMetaPrefix = MetaPrefix + "/file_resource_info"
|
||||||
|
FileResourceVersionKey = MetaPrefix + "/file_resource_version"
|
||||||
|
|
||||||
NonRemoveFlagTomestone = "non-removed"
|
NonRemoveFlagTomestone = "non-removed"
|
||||||
RemoveFlagTomestone = "removed"
|
RemoveFlagTomestone = "removed"
|
||||||
|
|||||||
@ -41,6 +41,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
"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/datapb"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
|
"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"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/util/etcd"
|
"github.com/milvus-io/milvus/pkg/v2/util/etcd"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/util/merr"
|
"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)
|
return kc.MetaKv.Remove(ctx, key)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (kc *Catalog) SaveFileResource(ctx context.Context, resource *model.FileResource) error {
|
func (kc *Catalog) SaveFileResource(ctx context.Context, resource *internalpb.FileResourceInfo, version uint64) error {
|
||||||
k := BuildFileResourceKey(resource.ID)
|
kvs := make(map[string]string)
|
||||||
v, err := proto.Marshal(resource.Marshal())
|
|
||||||
|
k := BuildFileResourceKey(resource.Id)
|
||||||
|
v, err := proto.Marshal(resource)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Ctx(ctx).Error("failed to marshal resource info", zap.Error(err))
|
log.Ctx(ctx).Error("failed to marshal resource info", zap.Error(err))
|
||||||
return 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))
|
log.Ctx(ctx).Warn("fail to save resource info", zap.String("key", k), zap.Error(err))
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
return nil
|
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)
|
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))
|
log.Ctx(ctx).Warn("fail to remove resource info", zap.String("key", k), zap.Error(err))
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
return nil
|
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)
|
_, values, err := kc.MetaKv.LoadWithPrefix(ctx, FileResourceMetaPrefix)
|
||||||
if err != nil {
|
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 {
|
for _, v := range values {
|
||||||
info := &datapb.FileResourceInfo{}
|
info := &internalpb.FileResourceInfo{}
|
||||||
err := proto.Unmarshal([]byte(v), info)
|
err := proto.Unmarshal([]byte(v), info)
|
||||||
if err != nil {
|
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 {
|
func BuildFileResourceKey(resourceID typeutil.UniqueID) string {
|
||||||
|
|||||||
@ -8,6 +8,8 @@ import (
|
|||||||
datapb "github.com/milvus-io/milvus/pkg/v2/proto/datapb"
|
datapb "github.com/milvus-io/milvus/pkg/v2/proto/datapb"
|
||||||
indexpb "github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
|
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"
|
metastore "github.com/milvus-io/milvus/internal/metastore"
|
||||||
|
|
||||||
mock "github.com/stretchr/testify/mock"
|
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
|
// 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)
|
ret := _m.Called(ctx)
|
||||||
|
|
||||||
if len(ret) == 0 {
|
if len(ret) == 0 {
|
||||||
panic("no return value specified for ListFileResource")
|
panic("no return value specified for ListFileResource")
|
||||||
}
|
}
|
||||||
|
|
||||||
var r0 []*model.FileResource
|
var r0 []*internalpb.FileResourceInfo
|
||||||
var r1 error
|
var r1 uint64
|
||||||
if rf, ok := ret.Get(0).(func(context.Context) ([]*model.FileResource, error)); ok {
|
var r2 error
|
||||||
|
if rf, ok := ret.Get(0).(func(context.Context) ([]*internalpb.FileResourceInfo, uint64, error)); ok {
|
||||||
return rf(ctx)
|
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)
|
r0 = rf(ctx)
|
||||||
} else {
|
} else {
|
||||||
if ret.Get(0) != nil {
|
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)
|
r1 = rf(ctx)
|
||||||
} else {
|
} 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'
|
// 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
|
return _c
|
||||||
}
|
}
|
||||||
|
|
||||||
func (_c *DataCoordCatalog_ListFileResource_Call) Return(_a0 []*model.FileResource, _a1 error) *DataCoordCatalog_ListFileResource_Call {
|
func (_c *DataCoordCatalog_ListFileResource_Call) Return(_a0 []*internalpb.FileResourceInfo, _a1 uint64, _a2 error) *DataCoordCatalog_ListFileResource_Call {
|
||||||
_c.Call.Return(_a0, _a1)
|
_c.Call.Return(_a0, _a1, _a2)
|
||||||
return _c
|
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)
|
_c.Call.Return(run)
|
||||||
return _c
|
return _c
|
||||||
}
|
}
|
||||||
@ -1994,17 +2003,17 @@ func (_c *DataCoordCatalog_MarkChannelDeleted_Call) RunAndReturn(run func(contex
|
|||||||
return _c
|
return _c
|
||||||
}
|
}
|
||||||
|
|
||||||
// RemoveFileResource provides a mock function with given fields: ctx, resourceID
|
// RemoveFileResource provides a mock function with given fields: ctx, resourceID, version
|
||||||
func (_m *DataCoordCatalog) RemoveFileResource(ctx context.Context, resourceID int64) error {
|
func (_m *DataCoordCatalog) RemoveFileResource(ctx context.Context, resourceID int64, version uint64) error {
|
||||||
ret := _m.Called(ctx, resourceID)
|
ret := _m.Called(ctx, resourceID, version)
|
||||||
|
|
||||||
if len(ret) == 0 {
|
if len(ret) == 0 {
|
||||||
panic("no return value specified for RemoveFileResource")
|
panic("no return value specified for RemoveFileResource")
|
||||||
}
|
}
|
||||||
|
|
||||||
var r0 error
|
var r0 error
|
||||||
if rf, ok := ret.Get(0).(func(context.Context, int64) error); ok {
|
if rf, ok := ret.Get(0).(func(context.Context, int64, uint64) error); ok {
|
||||||
r0 = rf(ctx, resourceID)
|
r0 = rf(ctx, resourceID, version)
|
||||||
} else {
|
} else {
|
||||||
r0 = ret.Error(0)
|
r0 = ret.Error(0)
|
||||||
}
|
}
|
||||||
@ -2020,13 +2029,14 @@ type DataCoordCatalog_RemoveFileResource_Call struct {
|
|||||||
// RemoveFileResource is a helper method to define mock.On call
|
// RemoveFileResource is a helper method to define mock.On call
|
||||||
// - ctx context.Context
|
// - ctx context.Context
|
||||||
// - resourceID int64
|
// - resourceID int64
|
||||||
func (_e *DataCoordCatalog_Expecter) RemoveFileResource(ctx interface{}, resourceID interface{}) *DataCoordCatalog_RemoveFileResource_Call {
|
// - version uint64
|
||||||
return &DataCoordCatalog_RemoveFileResource_Call{Call: _e.mock.On("RemoveFileResource", ctx, resourceID)}
|
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) {
|
_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
|
return _c
|
||||||
}
|
}
|
||||||
@ -2036,7 +2046,7 @@ func (_c *DataCoordCatalog_RemoveFileResource_Call) Return(_a0 error) *DataCoord
|
|||||||
return _c
|
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)
|
_c.Call.Return(run)
|
||||||
return _c
|
return _c
|
||||||
}
|
}
|
||||||
@ -2327,17 +2337,17 @@ func (_c *DataCoordCatalog_SaveDroppedSegmentsInBatch_Call) RunAndReturn(run fun
|
|||||||
return _c
|
return _c
|
||||||
}
|
}
|
||||||
|
|
||||||
// SaveFileResource provides a mock function with given fields: ctx, resource
|
// SaveFileResource provides a mock function with given fields: ctx, resource, version
|
||||||
func (_m *DataCoordCatalog) SaveFileResource(ctx context.Context, resource *model.FileResource) error {
|
func (_m *DataCoordCatalog) SaveFileResource(ctx context.Context, resource *internalpb.FileResourceInfo, version uint64) error {
|
||||||
ret := _m.Called(ctx, resource)
|
ret := _m.Called(ctx, resource, version)
|
||||||
|
|
||||||
if len(ret) == 0 {
|
if len(ret) == 0 {
|
||||||
panic("no return value specified for SaveFileResource")
|
panic("no return value specified for SaveFileResource")
|
||||||
}
|
}
|
||||||
|
|
||||||
var r0 error
|
var r0 error
|
||||||
if rf, ok := ret.Get(0).(func(context.Context, *model.FileResource) error); ok {
|
if rf, ok := ret.Get(0).(func(context.Context, *internalpb.FileResourceInfo, uint64) error); ok {
|
||||||
r0 = rf(ctx, resource)
|
r0 = rf(ctx, resource, version)
|
||||||
} else {
|
} else {
|
||||||
r0 = ret.Error(0)
|
r0 = ret.Error(0)
|
||||||
}
|
}
|
||||||
@ -2352,14 +2362,15 @@ type DataCoordCatalog_SaveFileResource_Call struct {
|
|||||||
|
|
||||||
// SaveFileResource is a helper method to define mock.On call
|
// SaveFileResource is a helper method to define mock.On call
|
||||||
// - ctx context.Context
|
// - ctx context.Context
|
||||||
// - resource *model.FileResource
|
// - resource *internalpb.FileResourceInfo
|
||||||
func (_e *DataCoordCatalog_Expecter) SaveFileResource(ctx interface{}, resource interface{}) *DataCoordCatalog_SaveFileResource_Call {
|
// - version uint64
|
||||||
return &DataCoordCatalog_SaveFileResource_Call{Call: _e.mock.On("SaveFileResource", ctx, resource)}
|
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) {
|
_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
|
return _c
|
||||||
}
|
}
|
||||||
@ -2369,7 +2380,7 @@ func (_c *DataCoordCatalog_SaveFileResource_Call) Return(_a0 error) *DataCoordCa
|
|||||||
return _c
|
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)
|
_c.Call.Return(run)
|
||||||
return _c
|
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
|
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
|
// SyncSegments provides a mock function with given fields: _a0, _a1
|
||||||
func (_m *MockDataNode) SyncSegments(_a0 context.Context, _a1 *datapb.SyncSegmentsRequest) (*commonpb.Status, error) {
|
func (_m *MockDataNode) SyncSegments(_a0 context.Context, _a1 *datapb.SyncSegmentsRequest) (*commonpb.Status, error) {
|
||||||
ret := _m.Called(_a0, _a1)
|
ret := _m.Called(_a0, _a1)
|
||||||
|
|||||||
@ -2150,6 +2150,80 @@ func (_c *MockDataNodeClient_ShowConfigurations_Call) RunAndReturn(run func(cont
|
|||||||
return _c
|
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
|
// 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) {
|
func (_m *MockDataNodeClient) SyncSegments(ctx context.Context, in *datapb.SyncSegmentsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||||
_va := make([]interface{}, len(opts))
|
_va := make([]interface{}, len(opts))
|
||||||
|
|||||||
@ -8436,6 +8436,54 @@ func (_c *MixCoord_SyncNewCreatedPartition_Call) RunAndReturn(run func(context.C
|
|||||||
return _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
|
// TransferChannel provides a mock function with given fields: _a0, _a1
|
||||||
func (_m *MixCoord) TransferChannel(_a0 context.Context, _a1 *querypb.TransferChannelRequest) (*commonpb.Status, error) {
|
func (_m *MixCoord) TransferChannel(_a0 context.Context, _a1 *querypb.TransferChannelRequest) (*commonpb.Status, error) {
|
||||||
ret := _m.Called(_a0, _a1)
|
ret := _m.Called(_a0, _a1)
|
||||||
|
|||||||
@ -1817,6 +1817,65 @@ func (_c *MockQueryNode_SyncDistribution_Call) RunAndReturn(run func(context.Con
|
|||||||
return _c
|
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
|
// SyncReplicaSegments provides a mock function with given fields: _a0, _a1
|
||||||
func (_m *MockQueryNode) SyncReplicaSegments(_a0 context.Context, _a1 *querypb.SyncReplicaSegmentsRequest) (*commonpb.Status, error) {
|
func (_m *MockQueryNode) SyncReplicaSegments(_a0 context.Context, _a1 *querypb.SyncReplicaSegmentsRequest) (*commonpb.Status, error) {
|
||||||
ret := _m.Called(_a0, _a1)
|
ret := _m.Called(_a0, _a1)
|
||||||
|
|||||||
@ -1926,6 +1926,80 @@ func (_c *MockQueryNodeClient_SyncDistribution_Call) RunAndReturn(run func(conte
|
|||||||
return _c
|
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
|
// 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) {
|
func (_m *MockQueryNodeClient) SyncReplicaSegments(ctx context.Context, in *querypb.SyncReplicaSegmentsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||||
_va := make([]interface{}, len(opts))
|
_va := make([]interface{}, len(opts))
|
||||||
|
|||||||
@ -1480,6 +1480,65 @@ func (_c *MockQueryNodeServer_SyncDistribution_Call) RunAndReturn(run func(conte
|
|||||||
return _c
|
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
|
// SyncReplicaSegments provides a mock function with given fields: _a0, _a1
|
||||||
func (_m *MockQueryNodeServer) SyncReplicaSegments(_a0 context.Context, _a1 *querypb.SyncReplicaSegmentsRequest) (*commonpb.Status, error) {
|
func (_m *MockQueryNodeServer) SyncReplicaSegments(_a0 context.Context, _a1 *querypb.SyncReplicaSegmentsRequest) (*commonpb.Status, error) {
|
||||||
ret := _m.Called(_a0, _a1)
|
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/kv"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
"github.com/milvus-io/milvus/pkg/v2/log"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/metrics"
|
"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"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/util/expr"
|
"github.com/milvus-io/milvus/pkg/v2/util/expr"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/util/metricsinfo"
|
"github.com/milvus-io/milvus/pkg/v2/util/metricsinfo"
|
||||||
@ -111,11 +112,12 @@ type Server struct {
|
|||||||
checkerController *checkers.CheckerController
|
checkerController *checkers.CheckerController
|
||||||
|
|
||||||
// Observers
|
// Observers
|
||||||
collectionObserver *observers.CollectionObserver
|
collectionObserver *observers.CollectionObserver
|
||||||
targetObserver *observers.TargetObserver
|
targetObserver *observers.TargetObserver
|
||||||
replicaObserver *observers.ReplicaObserver
|
replicaObserver *observers.ReplicaObserver
|
||||||
resourceObserver *observers.ResourceObserver
|
resourceObserver *observers.ResourceObserver
|
||||||
leaderCacheObserver *observers.LeaderCacheObserver
|
leaderCacheObserver *observers.LeaderCacheObserver
|
||||||
|
fileResourceObserver *observers.FileResourceObserver
|
||||||
|
|
||||||
getBalancerFunc checkers.GetBalancerFunc
|
getBalancerFunc checkers.GetBalancerFunc
|
||||||
balancerMap map[string]balance.Balance
|
balancerMap map[string]balance.Balance
|
||||||
@ -467,6 +469,8 @@ func (s *Server) initObserver() {
|
|||||||
s.leaderCacheObserver = observers.NewLeaderCacheObserver(
|
s.leaderCacheObserver = observers.NewLeaderCacheObserver(
|
||||||
s.proxyClientManager,
|
s.proxyClientManager,
|
||||||
)
|
)
|
||||||
|
|
||||||
|
s.fileResourceObserver = observers.NewFileResourceObserver(s.ctx, s.nodeMgr, s.cluster)
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *Server) afterStart() {}
|
func (s *Server) afterStart() {}
|
||||||
@ -530,6 +534,7 @@ func (s *Server) startServerLoop() {
|
|||||||
s.targetObserver.Start()
|
s.targetObserver.Start()
|
||||||
s.replicaObserver.Start()
|
s.replicaObserver.Start()
|
||||||
s.resourceObserver.Start()
|
s.resourceObserver.Start()
|
||||||
|
s.fileResourceObserver.Start()
|
||||||
|
|
||||||
log.Info("start task scheduler...")
|
log.Info("start task scheduler...")
|
||||||
s.taskScheduler.Start()
|
s.taskScheduler.Start()
|
||||||
@ -585,6 +590,9 @@ func (s *Server) Stop() error {
|
|||||||
if s.leaderCacheObserver != nil {
|
if s.leaderCacheObserver != nil {
|
||||||
s.leaderCacheObserver.Stop()
|
s.leaderCacheObserver.Stop()
|
||||||
}
|
}
|
||||||
|
if s.fileResourceObserver != nil {
|
||||||
|
s.fileResourceObserver.Stop()
|
||||||
|
}
|
||||||
|
|
||||||
if s.distController != nil {
|
if s.distController != nil {
|
||||||
log.Info("stop dist controller...")
|
log.Info("stop dist controller...")
|
||||||
@ -687,6 +695,7 @@ func (s *Server) watchNodes(revision int64) {
|
|||||||
Labels: event.Session.GetServerLabel(),
|
Labels: event.Session.GetServerLabel(),
|
||||||
}))
|
}))
|
||||||
s.handleNodeUp(nodeID)
|
s.handleNodeUp(nodeID)
|
||||||
|
s.fileResourceObserver.Notify()
|
||||||
|
|
||||||
case sessionutil.SessionUpdateEvent:
|
case sessionutil.SessionUpdateEvent:
|
||||||
log.Info("stopping the node")
|
log.Info("stopping the node")
|
||||||
@ -934,3 +943,8 @@ func (s *Server) watchLoadConfigChanges() {
|
|||||||
})
|
})
|
||||||
paramtable.Get().Watch(paramtable.Get().QueryCoordCfg.ClusterLevelLoadResourceGroups.Key, rgHandler)
|
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"
|
grpcquerynodeclient "github.com/milvus-io/milvus/internal/distributed/querynode/client"
|
||||||
"github.com/milvus-io/milvus/internal/types"
|
"github.com/milvus-io/milvus/internal/types"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
"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/proto/querypb"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
"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)
|
DropIndex(ctx context.Context, nodeID int64, req *querypb.DropIndexRequest) (*commonpb.Status, error)
|
||||||
RunAnalyzer(ctx context.Context, nodeID int64, req *querypb.RunAnalyzerRequest) (*milvuspb.RunAnalyzerResponse, 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)
|
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()
|
Start()
|
||||||
Stop()
|
Stop()
|
||||||
}
|
}
|
||||||
@ -315,6 +317,21 @@ func (c *QueryCluster) ValidateAnalyzer(ctx context.Context, nodeID int64, req *
|
|||||||
return resp, err
|
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 {
|
func (c *QueryCluster) send(ctx context.Context, nodeID int64, fn func(cli types.QueryNodeClient)) error {
|
||||||
node := c.nodeManager.Get(nodeID)
|
node := c.nodeManager.Get(nodeID)
|
||||||
if node == nil {
|
if node == nil {
|
||||||
|
|||||||
@ -7,6 +7,8 @@ import (
|
|||||||
|
|
||||||
commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
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"
|
milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
|
||||||
|
|
||||||
mock "github.com/stretchr/testify/mock"
|
mock "github.com/stretchr/testify/mock"
|
||||||
@ -690,6 +692,66 @@ func (_c *MockCluster_SyncDistribution_Call) RunAndReturn(run func(context.Conte
|
|||||||
return _c
|
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
|
// 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) {
|
func (_m *MockCluster) UnsubDmChannel(ctx context.Context, nodeID int64, req *querypb.UnsubDmChannelRequest) (*commonpb.Status, error) {
|
||||||
ret := _m.Called(ctx, nodeID, req)
|
ret := _m.Called(ctx, nodeID, req)
|
||||||
|
|||||||
@ -56,6 +56,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/types"
|
"github.com/milvus-io/milvus/internal/types"
|
||||||
"github.com/milvus-io/milvus/internal/util/analyzer"
|
"github.com/milvus-io/milvus/internal/util/analyzer"
|
||||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
"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/hookutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/initcore"
|
"github.com/milvus-io/milvus/internal/util/initcore"
|
||||||
"github.com/milvus-io/milvus/internal/util/searchutil/optimizers"
|
"github.com/milvus-io/milvus/internal/util/searchutil/optimizers"
|
||||||
@ -353,6 +354,8 @@ func (node *QueryNode) Init() error {
|
|||||||
// init pipeline manager
|
// init pipeline manager
|
||||||
node.pipelineManager = pipeline.NewManager(node.manager, node.dispClient, node.delegators)
|
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)
|
err = initcore.InitQueryNode(node.ctx)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Error("QueryNode init segcore failed", zap.Error(err))
|
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/querynodev2/tasks"
|
||||||
"github.com/milvus-io/milvus/internal/storage"
|
"github.com/milvus-io/milvus/internal/storage"
|
||||||
"github.com/milvus-io/milvus/internal/util/analyzer"
|
"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/searchutil/scheduler"
|
||||||
"github.com/milvus-io/milvus/internal/util/streamrpc"
|
"github.com/milvus-io/milvus/internal/util/streamrpc"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/common"
|
"github.com/milvus-io/milvus/pkg/v2/common"
|
||||||
@ -1774,3 +1775,20 @@ func (node *QueryNode) GetHighlight(ctx context.Context, req *querypb.GetHighlig
|
|||||||
Results: results,
|
Results: results,
|
||||||
}, nil
|
}, 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/resource"
|
||||||
"github.com/milvus-io/milvus/internal/streamingnode/server/service"
|
"github.com/milvus-io/milvus/internal/streamingnode/server/service"
|
||||||
"github.com/milvus-io/milvus/internal/streamingnode/server/walmanager"
|
"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/initcore"
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/log"
|
"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/kafka"
|
||||||
_ "github.com/milvus-io/milvus/pkg/v2/streaming/walimpls/impls/pulsar"
|
_ "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/streaming/walimpls/impls/rmq"
|
||||||
|
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
|
||||||
)
|
)
|
||||||
|
|
||||||
// Server is the streamingnode server.
|
// Server is the streamingnode server.
|
||||||
@ -42,6 +44,9 @@ func (s *Server) init() {
|
|||||||
// init all service.
|
// init all service.
|
||||||
s.initService()
|
s.initService()
|
||||||
|
|
||||||
|
// init file resource manager
|
||||||
|
fileresource.InitManager(resource.Resource().ChunkManager(), fileresource.ParseMode(paramtable.Get().QueryCoordCfg.FileResourceMode.GetValue()))
|
||||||
|
|
||||||
log.Info("init query segcore...")
|
log.Info("init query segcore...")
|
||||||
if err := initcore.InitQueryNode(context.TODO()); err != nil {
|
if err := initcore.InitQueryNode(context.TODO()); err != nil {
|
||||||
panic(fmt.Sprintf("init query node segcore failed, %+v", err))
|
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
|
// GetMetrics notifies MixCoordComponent to collect metrics for specified component
|
||||||
GetQcMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error)
|
GetQcMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error)
|
||||||
|
|
||||||
// GetMetrics notifies MixCoordComponent to collect metrics for specified component
|
// GetMetrics notifies MixCoordComponent to collect metrics for specified component
|
||||||
NotifyDropPartition(ctx context.Context, channel string, partitionIDs []int64) error
|
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
|
// 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) {
|
func (m *GrpcDataNodeClient) DropCompactionPlan(ctx context.Context, req *datapb.DropCompactionPlanRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||||
return &commonpb.Status{}, m.Err
|
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) {
|
func (m *GrpcQueryNodeClient) DropIndex(ctx context.Context, in *querypb.DropIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||||
return &commonpb.Status{}, m.Err
|
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
|
LocalChunkPath
|
||||||
BM25Path
|
BM25Path
|
||||||
RootCachePath
|
RootCachePath
|
||||||
|
FileResourcePath
|
||||||
)
|
)
|
||||||
|
|
||||||
const (
|
const (
|
||||||
CachePathPrefix = "cache"
|
CachePathPrefix = "cache"
|
||||||
GrowingMMapPathPrefix = "growing_mmap"
|
GrowingMMapPathPrefix = "growing_mmap"
|
||||||
LocalChunkPathPrefix = "local_chunk"
|
LocalChunkPathPrefix = "local_chunk"
|
||||||
BM25PathPrefix = "bm25"
|
BM25PathPrefix = "bm25"
|
||||||
|
FileResourcePathPrefix = "file_resource"
|
||||||
)
|
)
|
||||||
|
|
||||||
func GetPath(pathType PathType, nodeID int64) string {
|
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)
|
path = filepath.Join(path, fmt.Sprintf("%d", nodeID), LocalChunkPathPrefix)
|
||||||
case BM25Path:
|
case BM25Path:
|
||||||
path = filepath.Join(path, fmt.Sprintf("%d", nodeID), BM25PathPrefix)
|
path = filepath.Join(path, fmt.Sprintf("%d", nodeID), BM25PathPrefix)
|
||||||
|
case FileResourcePath:
|
||||||
|
path = filepath.Join(path, fmt.Sprintf("%d", nodeID), FileResourcePathPrefix)
|
||||||
case RootCachePath:
|
case RootCachePath:
|
||||||
}
|
}
|
||||||
log.Info("Get path for", zap.Any("pathType", pathType), zap.Int64("nodeID", nodeID), zap.String("path", path))
|
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)
|
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 {
|
func WrapQueryNodeServerAsClient(qn types.QueryNode) types.QueryNodeClient {
|
||||||
return &qnServerWrapper{
|
return &qnServerWrapper{
|
||||||
QueryNode: qn,
|
QueryNode: qn,
|
||||||
|
|||||||
@ -145,6 +145,9 @@ service DataNode {
|
|||||||
rpc QuerySlot(QuerySlotRequest) returns(QuerySlotResponse) {}
|
rpc QuerySlot(QuerySlotRequest) returns(QuerySlotResponse) {}
|
||||||
|
|
||||||
rpc DropCompactionPlan(DropCompactionPlanRequest) returns(common.Status) {}
|
rpc DropCompactionPlan(DropCompactionPlanRequest) returns(common.Status) {}
|
||||||
|
|
||||||
|
// file resource
|
||||||
|
rpc SyncFileResource(internal.SyncFileResourceRequest) returns(common.Status) {}
|
||||||
}
|
}
|
||||||
|
|
||||||
message FlushRequest {
|
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,
|
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,
|
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,
|
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,
|
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,
|
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,
|
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,
|
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,
|
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,
|
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,
|
0x00, 0x12, 0x61, 0x0a, 0x10, 0x53, 0x79, 0x6e, 0x63, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73,
|
||||||
0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73,
|
0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
|
||||||
0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x64, 0x61,
|
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x79,
|
||||||
0x74, 0x61, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
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 (
|
var (
|
||||||
@ -11673,27 +11679,28 @@ var file_data_coord_proto_goTypes = []interface{}{
|
|||||||
(*milvuspb.ListFileResourcesRequest)(nil), // 166: milvus.proto.milvus.ListFileResourcesRequest
|
(*milvuspb.ListFileResourcesRequest)(nil), // 166: milvus.proto.milvus.ListFileResourcesRequest
|
||||||
(*milvuspb.GetComponentStatesRequest)(nil), // 167: milvus.proto.milvus.GetComponentStatesRequest
|
(*milvuspb.GetComponentStatesRequest)(nil), // 167: milvus.proto.milvus.GetComponentStatesRequest
|
||||||
(*internalpb.GetStatisticsChannelRequest)(nil), // 168: milvus.proto.internal.GetStatisticsChannelRequest
|
(*internalpb.GetStatisticsChannelRequest)(nil), // 168: milvus.proto.internal.GetStatisticsChannelRequest
|
||||||
(*milvuspb.StringResponse)(nil), // 169: milvus.proto.milvus.StringResponse
|
(*internalpb.SyncFileResourceRequest)(nil), // 169: milvus.proto.internal.SyncFileResourceRequest
|
||||||
(*milvuspb.GetFlushAllStateResponse)(nil), // 170: milvus.proto.milvus.GetFlushAllStateResponse
|
(*milvuspb.StringResponse)(nil), // 170: milvus.proto.milvus.StringResponse
|
||||||
(*internalpb.ShowConfigurationsResponse)(nil), // 171: milvus.proto.internal.ShowConfigurationsResponse
|
(*milvuspb.GetFlushAllStateResponse)(nil), // 171: milvus.proto.milvus.GetFlushAllStateResponse
|
||||||
(*milvuspb.GetMetricsResponse)(nil), // 172: milvus.proto.milvus.GetMetricsResponse
|
(*internalpb.ShowConfigurationsResponse)(nil), // 172: milvus.proto.internal.ShowConfigurationsResponse
|
||||||
(*milvuspb.ManualCompactionResponse)(nil), // 173: milvus.proto.milvus.ManualCompactionResponse
|
(*milvuspb.GetMetricsResponse)(nil), // 173: milvus.proto.milvus.GetMetricsResponse
|
||||||
(*milvuspb.GetCompactionStateResponse)(nil), // 174: milvus.proto.milvus.GetCompactionStateResponse
|
(*milvuspb.ManualCompactionResponse)(nil), // 174: milvus.proto.milvus.ManualCompactionResponse
|
||||||
(*milvuspb.GetCompactionPlansResponse)(nil), // 175: milvus.proto.milvus.GetCompactionPlansResponse
|
(*milvuspb.GetCompactionStateResponse)(nil), // 175: milvus.proto.milvus.GetCompactionStateResponse
|
||||||
(*milvuspb.GetFlushStateResponse)(nil), // 176: milvus.proto.milvus.GetFlushStateResponse
|
(*milvuspb.GetCompactionPlansResponse)(nil), // 176: milvus.proto.milvus.GetCompactionPlansResponse
|
||||||
(*milvuspb.CheckHealthResponse)(nil), // 177: milvus.proto.milvus.CheckHealthResponse
|
(*milvuspb.GetFlushStateResponse)(nil), // 177: milvus.proto.milvus.GetFlushStateResponse
|
||||||
(*indexpb.GetIndexStateResponse)(nil), // 178: milvus.proto.index.GetIndexStateResponse
|
(*milvuspb.CheckHealthResponse)(nil), // 178: milvus.proto.milvus.CheckHealthResponse
|
||||||
(*indexpb.GetSegmentIndexStateResponse)(nil), // 179: milvus.proto.index.GetSegmentIndexStateResponse
|
(*indexpb.GetIndexStateResponse)(nil), // 179: milvus.proto.index.GetIndexStateResponse
|
||||||
(*indexpb.GetIndexInfoResponse)(nil), // 180: milvus.proto.index.GetIndexInfoResponse
|
(*indexpb.GetSegmentIndexStateResponse)(nil), // 180: milvus.proto.index.GetSegmentIndexStateResponse
|
||||||
(*indexpb.DescribeIndexResponse)(nil), // 181: milvus.proto.index.DescribeIndexResponse
|
(*indexpb.GetIndexInfoResponse)(nil), // 181: milvus.proto.index.GetIndexInfoResponse
|
||||||
(*indexpb.GetIndexStatisticsResponse)(nil), // 182: milvus.proto.index.GetIndexStatisticsResponse
|
(*indexpb.DescribeIndexResponse)(nil), // 182: milvus.proto.index.DescribeIndexResponse
|
||||||
(*indexpb.GetIndexBuildProgressResponse)(nil), // 183: milvus.proto.index.GetIndexBuildProgressResponse
|
(*indexpb.GetIndexStatisticsResponse)(nil), // 183: milvus.proto.index.GetIndexStatisticsResponse
|
||||||
(*indexpb.ListIndexesResponse)(nil), // 184: milvus.proto.index.ListIndexesResponse
|
(*indexpb.GetIndexBuildProgressResponse)(nil), // 184: milvus.proto.index.GetIndexBuildProgressResponse
|
||||||
(*internalpb.ImportResponse)(nil), // 185: milvus.proto.internal.ImportResponse
|
(*indexpb.ListIndexesResponse)(nil), // 185: milvus.proto.index.ListIndexesResponse
|
||||||
(*internalpb.GetImportProgressResponse)(nil), // 186: milvus.proto.internal.GetImportProgressResponse
|
(*internalpb.ImportResponse)(nil), // 186: milvus.proto.internal.ImportResponse
|
||||||
(*internalpb.ListImportsResponse)(nil), // 187: milvus.proto.internal.ListImportsResponse
|
(*internalpb.GetImportProgressResponse)(nil), // 187: milvus.proto.internal.GetImportProgressResponse
|
||||||
(*milvuspb.ListFileResourcesResponse)(nil), // 188: milvus.proto.milvus.ListFileResourcesResponse
|
(*internalpb.ListImportsResponse)(nil), // 188: milvus.proto.internal.ListImportsResponse
|
||||||
(*milvuspb.ComponentStates)(nil), // 189: milvus.proto.milvus.ComponentStates
|
(*milvuspb.ListFileResourcesResponse)(nil), // 189: milvus.proto.milvus.ListFileResourcesResponse
|
||||||
|
(*milvuspb.ComponentStates)(nil), // 190: milvus.proto.milvus.ComponentStates
|
||||||
}
|
}
|
||||||
var file_data_coord_proto_depIdxs = []int32{
|
var file_data_coord_proto_depIdxs = []int32{
|
||||||
129, // 0: milvus.proto.data.FlushRequest.base:type_name -> milvus.proto.common.MsgBase
|
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
|
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
|
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
|
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
|
169, // 288: milvus.proto.data.DataNode.SyncFileResource:input_type -> milvus.proto.internal.SyncFileResourceRequest
|
||||||
14, // 289: milvus.proto.data.DataCoord.FlushAll:output_type -> milvus.proto.data.FlushAllResponse
|
10, // 289: milvus.proto.data.DataCoord.Flush:output_type -> milvus.proto.data.FlushResponse
|
||||||
116, // 290: milvus.proto.data.DataCoord.CreateExternalCollection:output_type -> milvus.proto.data.CreateExternalCollectionResponse
|
14, // 290: milvus.proto.data.DataCoord.FlushAll:output_type -> milvus.proto.data.FlushAllResponse
|
||||||
18, // 291: milvus.proto.data.DataCoord.AllocSegment:output_type -> milvus.proto.data.AllocSegmentResponse
|
116, // 291: milvus.proto.data.DataCoord.CreateExternalCollection:output_type -> milvus.proto.data.CreateExternalCollectionResponse
|
||||||
21, // 292: milvus.proto.data.DataCoord.AssignSegmentID:output_type -> milvus.proto.data.AssignSegmentIDResponse
|
18, // 292: milvus.proto.data.DataCoord.AllocSegment:output_type -> milvus.proto.data.AllocSegmentResponse
|
||||||
26, // 293: milvus.proto.data.DataCoord.GetSegmentInfo:output_type -> milvus.proto.data.GetSegmentInfoResponse
|
21, // 293: milvus.proto.data.DataCoord.AssignSegmentID:output_type -> milvus.proto.data.AssignSegmentIDResponse
|
||||||
24, // 294: milvus.proto.data.DataCoord.GetSegmentStates:output_type -> milvus.proto.data.GetSegmentStatesResponse
|
26, // 294: milvus.proto.data.DataCoord.GetSegmentInfo:output_type -> milvus.proto.data.GetSegmentInfoResponse
|
||||||
28, // 295: milvus.proto.data.DataCoord.GetInsertBinlogPaths:output_type -> milvus.proto.data.GetInsertBinlogPathsResponse
|
24, // 295: milvus.proto.data.DataCoord.GetSegmentStates:output_type -> milvus.proto.data.GetSegmentStatesResponse
|
||||||
30, // 296: milvus.proto.data.DataCoord.GetCollectionStatistics:output_type -> milvus.proto.data.GetCollectionStatisticsResponse
|
28, // 296: milvus.proto.data.DataCoord.GetInsertBinlogPaths:output_type -> milvus.proto.data.GetInsertBinlogPathsResponse
|
||||||
32, // 297: milvus.proto.data.DataCoord.GetPartitionStatistics:output_type -> milvus.proto.data.GetPartitionStatisticsResponse
|
30, // 297: milvus.proto.data.DataCoord.GetCollectionStatistics:output_type -> milvus.proto.data.GetCollectionStatisticsResponse
|
||||||
169, // 298: milvus.proto.data.DataCoord.GetSegmentInfoChannel:output_type -> milvus.proto.milvus.StringResponse
|
32, // 298: milvus.proto.data.DataCoord.GetPartitionStatistics:output_type -> milvus.proto.data.GetPartitionStatisticsResponse
|
||||||
130, // 299: milvus.proto.data.DataCoord.SaveBinlogPaths:output_type -> milvus.proto.common.Status
|
170, // 299: milvus.proto.data.DataCoord.GetSegmentInfoChannel:output_type -> milvus.proto.milvus.StringResponse
|
||||||
50, // 300: milvus.proto.data.DataCoord.GetRecoveryInfo:output_type -> milvus.proto.data.GetRecoveryInfoResponse
|
130, // 300: milvus.proto.data.DataCoord.SaveBinlogPaths:output_type -> milvus.proto.common.Status
|
||||||
52, // 301: milvus.proto.data.DataCoord.GetRecoveryInfoV2:output_type -> milvus.proto.data.GetRecoveryInfoResponseV2
|
50, // 301: milvus.proto.data.DataCoord.GetRecoveryInfo:output_type -> milvus.proto.data.GetRecoveryInfoResponse
|
||||||
55, // 302: milvus.proto.data.DataCoord.GetChannelRecoveryInfo:output_type -> milvus.proto.data.GetChannelRecoveryInfoResponse
|
52, // 302: milvus.proto.data.DataCoord.GetRecoveryInfoV2:output_type -> milvus.proto.data.GetRecoveryInfoResponseV2
|
||||||
60, // 303: milvus.proto.data.DataCoord.GetFlushedSegments:output_type -> milvus.proto.data.GetFlushedSegmentsResponse
|
55, // 303: milvus.proto.data.DataCoord.GetChannelRecoveryInfo:output_type -> milvus.proto.data.GetChannelRecoveryInfoResponse
|
||||||
58, // 304: milvus.proto.data.DataCoord.GetSegmentsByStates:output_type -> milvus.proto.data.GetSegmentsByStatesResponse
|
60, // 304: milvus.proto.data.DataCoord.GetFlushedSegments:output_type -> milvus.proto.data.GetFlushedSegmentsResponse
|
||||||
170, // 305: milvus.proto.data.DataCoord.GetFlushAllState:output_type -> milvus.proto.milvus.GetFlushAllStateResponse
|
58, // 305: milvus.proto.data.DataCoord.GetSegmentsByStates:output_type -> milvus.proto.data.GetSegmentsByStatesResponse
|
||||||
171, // 306: milvus.proto.data.DataCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse
|
171, // 306: milvus.proto.data.DataCoord.GetFlushAllState:output_type -> milvus.proto.milvus.GetFlushAllStateResponse
|
||||||
172, // 307: milvus.proto.data.DataCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse
|
172, // 307: milvus.proto.data.DataCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse
|
||||||
173, // 308: milvus.proto.data.DataCoord.ManualCompaction:output_type -> milvus.proto.milvus.ManualCompactionResponse
|
173, // 308: milvus.proto.data.DataCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse
|
||||||
174, // 309: milvus.proto.data.DataCoord.GetCompactionState:output_type -> milvus.proto.milvus.GetCompactionStateResponse
|
174, // 309: milvus.proto.data.DataCoord.ManualCompaction:output_type -> milvus.proto.milvus.ManualCompactionResponse
|
||||||
175, // 310: milvus.proto.data.DataCoord.GetCompactionStateWithPlans:output_type -> milvus.proto.milvus.GetCompactionPlansResponse
|
175, // 310: milvus.proto.data.DataCoord.GetCompactionState:output_type -> milvus.proto.milvus.GetCompactionStateResponse
|
||||||
73, // 311: milvus.proto.data.DataCoord.WatchChannels:output_type -> milvus.proto.data.WatchChannelsResponse
|
176, // 311: milvus.proto.data.DataCoord.GetCompactionStateWithPlans:output_type -> milvus.proto.milvus.GetCompactionPlansResponse
|
||||||
176, // 312: milvus.proto.data.DataCoord.GetFlushState:output_type -> milvus.proto.milvus.GetFlushStateResponse
|
73, // 312: milvus.proto.data.DataCoord.WatchChannels:output_type -> milvus.proto.data.WatchChannelsResponse
|
||||||
78, // 313: milvus.proto.data.DataCoord.DropVirtualChannel:output_type -> milvus.proto.data.DropVirtualChannelResponse
|
177, // 313: milvus.proto.data.DataCoord.GetFlushState:output_type -> milvus.proto.milvus.GetFlushStateResponse
|
||||||
75, // 314: milvus.proto.data.DataCoord.SetSegmentState:output_type -> milvus.proto.data.SetSegmentStateResponse
|
78, // 314: milvus.proto.data.DataCoord.DropVirtualChannel:output_type -> milvus.proto.data.DropVirtualChannelResponse
|
||||||
130, // 315: milvus.proto.data.DataCoord.UpdateSegmentStatistics:output_type -> milvus.proto.common.Status
|
75, // 315: milvus.proto.data.DataCoord.SetSegmentState:output_type -> milvus.proto.data.SetSegmentStateResponse
|
||||||
130, // 316: milvus.proto.data.DataCoord.UpdateChannelCheckpoint:output_type -> milvus.proto.common.Status
|
130, // 316: milvus.proto.data.DataCoord.UpdateSegmentStatistics:output_type -> milvus.proto.common.Status
|
||||||
130, // 317: milvus.proto.data.DataCoord.MarkSegmentsDropped: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.BroadcastAlteredCollection:output_type -> milvus.proto.common.Status
|
130, // 318: milvus.proto.data.DataCoord.MarkSegmentsDropped:output_type -> milvus.proto.common.Status
|
||||||
177, // 319: milvus.proto.data.DataCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse
|
130, // 319: milvus.proto.data.DataCoord.BroadcastAlteredCollection:output_type -> milvus.proto.common.Status
|
||||||
130, // 320: milvus.proto.data.DataCoord.CreateIndex: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.AlterIndex:output_type -> milvus.proto.common.Status
|
130, // 321: milvus.proto.data.DataCoord.CreateIndex:output_type -> milvus.proto.common.Status
|
||||||
178, // 322: milvus.proto.data.DataCoord.GetIndexState:output_type -> milvus.proto.index.GetIndexStateResponse
|
130, // 322: milvus.proto.data.DataCoord.AlterIndex:output_type -> milvus.proto.common.Status
|
||||||
179, // 323: milvus.proto.data.DataCoord.GetSegmentIndexState:output_type -> milvus.proto.index.GetSegmentIndexStateResponse
|
179, // 323: milvus.proto.data.DataCoord.GetIndexState:output_type -> milvus.proto.index.GetIndexStateResponse
|
||||||
180, // 324: milvus.proto.data.DataCoord.GetIndexInfos:output_type -> milvus.proto.index.GetIndexInfoResponse
|
180, // 324: milvus.proto.data.DataCoord.GetSegmentIndexState:output_type -> milvus.proto.index.GetSegmentIndexStateResponse
|
||||||
130, // 325: milvus.proto.data.DataCoord.DropIndex:output_type -> milvus.proto.common.Status
|
181, // 325: milvus.proto.data.DataCoord.GetIndexInfos:output_type -> milvus.proto.index.GetIndexInfoResponse
|
||||||
181, // 326: milvus.proto.data.DataCoord.DescribeIndex:output_type -> milvus.proto.index.DescribeIndexResponse
|
130, // 326: milvus.proto.data.DataCoord.DropIndex:output_type -> milvus.proto.common.Status
|
||||||
182, // 327: milvus.proto.data.DataCoord.GetIndexStatistics:output_type -> milvus.proto.index.GetIndexStatisticsResponse
|
182, // 327: milvus.proto.data.DataCoord.DescribeIndex:output_type -> milvus.proto.index.DescribeIndexResponse
|
||||||
183, // 328: milvus.proto.data.DataCoord.GetIndexBuildProgress:output_type -> milvus.proto.index.GetIndexBuildProgressResponse
|
183, // 328: milvus.proto.data.DataCoord.GetIndexStatistics:output_type -> milvus.proto.index.GetIndexStatisticsResponse
|
||||||
184, // 329: milvus.proto.data.DataCoord.ListIndexes:output_type -> milvus.proto.index.ListIndexesResponse
|
184, // 329: milvus.proto.data.DataCoord.GetIndexBuildProgress:output_type -> milvus.proto.index.GetIndexBuildProgressResponse
|
||||||
88, // 330: milvus.proto.data.DataCoord.GcConfirm:output_type -> milvus.proto.data.GcConfirmResponse
|
185, // 330: milvus.proto.data.DataCoord.ListIndexes:output_type -> milvus.proto.index.ListIndexesResponse
|
||||||
130, // 331: milvus.proto.data.DataCoord.ReportDataNodeTtMsgs:output_type -> milvus.proto.common.Status
|
88, // 331: milvus.proto.data.DataCoord.GcConfirm:output_type -> milvus.proto.data.GcConfirmResponse
|
||||||
130, // 332: milvus.proto.data.DataCoord.GcControl:output_type -> milvus.proto.common.Status
|
130, // 332: milvus.proto.data.DataCoord.ReportDataNodeTtMsgs:output_type -> milvus.proto.common.Status
|
||||||
185, // 333: milvus.proto.data.DataCoord.ImportV2:output_type -> milvus.proto.internal.ImportResponse
|
130, // 333: milvus.proto.data.DataCoord.GcControl:output_type -> milvus.proto.common.Status
|
||||||
186, // 334: milvus.proto.data.DataCoord.GetImportProgress:output_type -> milvus.proto.internal.GetImportProgressResponse
|
186, // 334: milvus.proto.data.DataCoord.ImportV2:output_type -> milvus.proto.internal.ImportResponse
|
||||||
187, // 335: milvus.proto.data.DataCoord.ListImports:output_type -> milvus.proto.internal.ListImportsResponse
|
187, // 335: milvus.proto.data.DataCoord.GetImportProgress:output_type -> milvus.proto.internal.GetImportProgressResponse
|
||||||
130, // 336: milvus.proto.data.DataCoord.AddFileResource:output_type -> milvus.proto.common.Status
|
188, // 336: milvus.proto.data.DataCoord.ListImports:output_type -> milvus.proto.internal.ListImportsResponse
|
||||||
130, // 337: milvus.proto.data.DataCoord.RemoveFileResource:output_type -> milvus.proto.common.Status
|
130, // 337: milvus.proto.data.DataCoord.AddFileResource:output_type -> milvus.proto.common.Status
|
||||||
188, // 338: milvus.proto.data.DataCoord.ListFileResources:output_type -> milvus.proto.milvus.ListFileResourcesResponse
|
130, // 338: milvus.proto.data.DataCoord.RemoveFileResource:output_type -> milvus.proto.common.Status
|
||||||
189, // 339: milvus.proto.data.DataNode.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates
|
189, // 339: milvus.proto.data.DataCoord.ListFileResources:output_type -> milvus.proto.milvus.ListFileResourcesResponse
|
||||||
169, // 340: milvus.proto.data.DataNode.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse
|
190, // 340: milvus.proto.data.DataNode.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates
|
||||||
130, // 341: milvus.proto.data.DataNode.WatchDmChannels:output_type -> milvus.proto.common.Status
|
170, // 341: milvus.proto.data.DataNode.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse
|
||||||
130, // 342: milvus.proto.data.DataNode.FlushSegments:output_type -> milvus.proto.common.Status
|
130, // 342: milvus.proto.data.DataNode.WatchDmChannels:output_type -> milvus.proto.common.Status
|
||||||
171, // 343: milvus.proto.data.DataNode.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse
|
130, // 343: milvus.proto.data.DataNode.FlushSegments:output_type -> milvus.proto.common.Status
|
||||||
172, // 344: milvus.proto.data.DataNode.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse
|
172, // 344: milvus.proto.data.DataNode.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse
|
||||||
130, // 345: milvus.proto.data.DataNode.CompactionV2:output_type -> milvus.proto.common.Status
|
173, // 345: milvus.proto.data.DataNode.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse
|
||||||
70, // 346: milvus.proto.data.DataNode.GetCompactionState:output_type -> milvus.proto.data.CompactionStateResponse
|
130, // 346: milvus.proto.data.DataNode.CompactionV2:output_type -> milvus.proto.common.Status
|
||||||
130, // 347: milvus.proto.data.DataNode.SyncSegments:output_type -> milvus.proto.common.Status
|
70, // 347: milvus.proto.data.DataNode.GetCompactionState:output_type -> milvus.proto.data.CompactionStateResponse
|
||||||
82, // 348: milvus.proto.data.DataNode.ResendSegmentStats:output_type -> milvus.proto.data.ResendSegmentStatsResponse
|
130, // 348: milvus.proto.data.DataNode.SyncSegments:output_type -> milvus.proto.common.Status
|
||||||
130, // 349: milvus.proto.data.DataNode.FlushChannels: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.NotifyChannelOperation:output_type -> milvus.proto.common.Status
|
130, // 350: milvus.proto.data.DataNode.FlushChannels:output_type -> milvus.proto.common.Status
|
||||||
92, // 351: milvus.proto.data.DataNode.CheckChannelOperationProgress:output_type -> milvus.proto.data.ChannelOperationProgressResponse
|
130, // 351: milvus.proto.data.DataNode.NotifyChannelOperation:output_type -> milvus.proto.common.Status
|
||||||
130, // 352: milvus.proto.data.DataNode.PreImport: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.ImportV2:output_type -> milvus.proto.common.Status
|
130, // 353: milvus.proto.data.DataNode.PreImport:output_type -> milvus.proto.common.Status
|
||||||
100, // 354: milvus.proto.data.DataNode.QueryPreImport:output_type -> milvus.proto.data.QueryPreImportResponse
|
130, // 354: milvus.proto.data.DataNode.ImportV2:output_type -> milvus.proto.common.Status
|
||||||
103, // 355: milvus.proto.data.DataNode.QueryImport:output_type -> milvus.proto.data.QueryImportResponse
|
100, // 355: milvus.proto.data.DataNode.QueryPreImport:output_type -> milvus.proto.data.QueryPreImportResponse
|
||||||
130, // 356: milvus.proto.data.DataNode.DropImport:output_type -> milvus.proto.common.Status
|
103, // 356: milvus.proto.data.DataNode.QueryImport:output_type -> milvus.proto.data.QueryImportResponse
|
||||||
111, // 357: milvus.proto.data.DataNode.QuerySlot:output_type -> milvus.proto.data.QuerySlotResponse
|
130, // 357: milvus.proto.data.DataNode.DropImport:output_type -> milvus.proto.common.Status
|
||||||
130, // 358: milvus.proto.data.DataNode.DropCompactionPlan:output_type -> milvus.proto.common.Status
|
111, // 358: milvus.proto.data.DataNode.QuerySlot:output_type -> milvus.proto.data.QuerySlotResponse
|
||||||
288, // [288:359] is the sub-list for method output_type
|
130, // 359: milvus.proto.data.DataNode.DropCompactionPlan:output_type -> milvus.proto.common.Status
|
||||||
217, // [217:288] is the sub-list for method input_type
|
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 type_name
|
||||||
217, // [217:217] is the sub-list for extension extendee
|
217, // [217:217] is the sub-list for extension extendee
|
||||||
0, // [0:217] is the sub-list for field type_name
|
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_DropImport_FullMethodName = "/milvus.proto.data.DataNode/DropImport"
|
||||||
DataNode_QuerySlot_FullMethodName = "/milvus.proto.data.DataNode/QuerySlot"
|
DataNode_QuerySlot_FullMethodName = "/milvus.proto.data.DataNode/QuerySlot"
|
||||||
DataNode_DropCompactionPlan_FullMethodName = "/milvus.proto.data.DataNode/DropCompactionPlan"
|
DataNode_DropCompactionPlan_FullMethodName = "/milvus.proto.data.DataNode/DropCompactionPlan"
|
||||||
|
DataNode_SyncFileResource_FullMethodName = "/milvus.proto.data.DataNode/SyncFileResource"
|
||||||
)
|
)
|
||||||
|
|
||||||
// DataNodeClient is the client API for DataNode service.
|
// 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)
|
DropImport(ctx context.Context, in *DropImportRequest, opts ...grpc.CallOption) (*commonpb.Status, error)
|
||||||
QuerySlot(ctx context.Context, in *QuerySlotRequest, opts ...grpc.CallOption) (*QuerySlotResponse, error)
|
QuerySlot(ctx context.Context, in *QuerySlotRequest, opts ...grpc.CallOption) (*QuerySlotResponse, error)
|
||||||
DropCompactionPlan(ctx context.Context, in *DropCompactionPlanRequest, opts ...grpc.CallOption) (*commonpb.Status, 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 {
|
type dataNodeClient struct {
|
||||||
@ -2221,6 +2224,15 @@ func (c *dataNodeClient) DropCompactionPlan(ctx context.Context, in *DropCompact
|
|||||||
return out, nil
|
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.
|
// DataNodeServer is the server API for DataNode service.
|
||||||
// All implementations should embed UnimplementedDataNodeServer
|
// All implementations should embed UnimplementedDataNodeServer
|
||||||
// for forward compatibility
|
// for forward compatibility
|
||||||
@ -2248,6 +2260,8 @@ type DataNodeServer interface {
|
|||||||
DropImport(context.Context, *DropImportRequest) (*commonpb.Status, error)
|
DropImport(context.Context, *DropImportRequest) (*commonpb.Status, error)
|
||||||
QuerySlot(context.Context, *QuerySlotRequest) (*QuerySlotResponse, error)
|
QuerySlot(context.Context, *QuerySlotRequest) (*QuerySlotResponse, error)
|
||||||
DropCompactionPlan(context.Context, *DropCompactionPlanRequest) (*commonpb.Status, 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.
|
// 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) {
|
func (UnimplementedDataNodeServer) DropCompactionPlan(context.Context, *DropCompactionPlanRequest) (*commonpb.Status, error) {
|
||||||
return nil, status.Errorf(codes.Unimplemented, "method DropCompactionPlan not implemented")
|
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.
|
// 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
|
// 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)
|
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.
|
// DataNode_ServiceDesc is the grpc.ServiceDesc for DataNode service.
|
||||||
// It's only intended for direct use with grpc.RegisterService,
|
// It's only intended for direct use with grpc.RegisterService,
|
||||||
// and not to be introspected or modified (even as a copy)
|
// and not to be introspected or modified (even as a copy)
|
||||||
@ -2773,6 +2808,10 @@ var DataNode_ServiceDesc = grpc.ServiceDesc{
|
|||||||
MethodName: "DropCompactionPlan",
|
MethodName: "DropCompactionPlan",
|
||||||
Handler: _DataNode_DropCompactionPlan_Handler,
|
Handler: _DataNode_DropCompactionPlan_Handler,
|
||||||
},
|
},
|
||||||
|
{
|
||||||
|
MethodName: "SyncFileResource",
|
||||||
|
Handler: _DataNode_SyncFileResource_Handler,
|
||||||
|
},
|
||||||
},
|
},
|
||||||
Streams: []grpc.StreamDesc{},
|
Streams: []grpc.StreamDesc{},
|
||||||
Metadata: "data_coord.proto",
|
Metadata: "data_coord.proto",
|
||||||
|
|||||||
@ -448,3 +448,15 @@ message GetQuotaMetricsResponse {
|
|||||||
common.Status status = 1;
|
common.Status status = 1;
|
||||||
string metrics_info = 2;
|
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 ""
|
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 protoreflect.FileDescriptor
|
||||||
|
|
||||||
var file_internal_proto_rawDesc = []byte{
|
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,
|
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,
|
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,
|
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,
|
0x6f, 0x22, 0x6d, 0x0a, 0x10, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63,
|
||||||
0x0a, 0x07, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x44,
|
0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20,
|
||||||
0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x10, 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x43, 0x6f, 0x6c,
|
0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x61, 0x74,
|
||||||
0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x50, 0x61, 0x72,
|
0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x70, 0x61, 0x74, 0x68, 0x12, 0x0e, 0x0a,
|
||||||
0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x03, 0x2a, 0xc4, 0x01, 0x0a, 0x08, 0x52, 0x61, 0x74,
|
0x02, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, 0x69, 0x64, 0x12, 0x21, 0x0a,
|
||||||
0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x44, 0x4c, 0x43, 0x6f, 0x6c, 0x6c,
|
0x0c, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20,
|
||||||
0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x00, 0x12, 0x10, 0x0a, 0x0c, 0x44, 0x44, 0x4c, 0x50,
|
0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x4e, 0x61, 0x6d, 0x65,
|
||||||
0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x01, 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x44,
|
0x22, 0x7a, 0x0a, 0x17, 0x53, 0x79, 0x6e, 0x63, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x6f,
|
||||||
0x4c, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x10, 0x02, 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x44, 0x4c, 0x46,
|
0x75, 0x72, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x45, 0x0a, 0x09, 0x72,
|
||||||
0x6c, 0x75, 0x73, 0x68, 0x10, 0x03, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x44, 0x4c, 0x43, 0x6f, 0x6d,
|
0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27,
|
||||||
0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x04, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x4d, 0x4c,
|
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e,
|
||||||
0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x10, 0x05, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x4d, 0x4c, 0x44,
|
0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75,
|
||||||
0x65, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x06, 0x12, 0x0f, 0x0a, 0x0b, 0x44, 0x4d, 0x4c, 0x42, 0x75,
|
0x72, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x09, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63,
|
||||||
0x6c, 0x6b, 0x4c, 0x6f, 0x61, 0x64, 0x10, 0x07, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x51, 0x4c, 0x53,
|
0x65, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20,
|
||||||
0x65, 0x61, 0x72, 0x63, 0x68, 0x10, 0x08, 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x51, 0x4c, 0x51, 0x75,
|
0x01, 0x28, 0x04, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x2a, 0x45, 0x0a, 0x09,
|
||||||
0x65, 0x72, 0x79, 0x10, 0x09, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x4d, 0x4c, 0x55, 0x70, 0x73, 0x65,
|
0x52, 0x61, 0x74, 0x65, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x43, 0x6c, 0x75,
|
||||||
0x72, 0x74, 0x10, 0x0a, 0x12, 0x09, 0x0a, 0x05, 0x44, 0x44, 0x4c, 0x44, 0x42, 0x10, 0x0b, 0x2a,
|
0x73, 0x74, 0x65, 0x72, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61,
|
||||||
0x83, 0x01, 0x0a, 0x0e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61,
|
0x73, 0x65, 0x10, 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69,
|
||||||
0x74, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07,
|
0x6f, 0x6e, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f,
|
||||||
0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x10, 0x01, 0x12, 0x10, 0x0a, 0x0c, 0x50, 0x72, 0x65,
|
0x6e, 0x10, 0x03, 0x2a, 0xc4, 0x01, 0x0a, 0x08, 0x52, 0x61, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65,
|
||||||
0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x49,
|
0x12, 0x11, 0x0a, 0x0d, 0x44, 0x44, 0x4c, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f,
|
||||||
0x6d, 0x70, 0x6f, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x10, 0x03, 0x12, 0x0a, 0x0a, 0x06, 0x46, 0x61,
|
0x6e, 0x10, 0x00, 0x12, 0x10, 0x0a, 0x0c, 0x44, 0x44, 0x4c, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74,
|
||||||
0x69, 0x6c, 0x65, 0x64, 0x10, 0x04, 0x12, 0x0d, 0x0a, 0x09, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65,
|
0x69, 0x6f, 0x6e, 0x10, 0x01, 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x44, 0x4c, 0x49, 0x6e, 0x64, 0x65,
|
||||||
0x74, 0x65, 0x64, 0x10, 0x05, 0x12, 0x11, 0x0a, 0x0d, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75,
|
0x78, 0x10, 0x02, 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x44, 0x4c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x10,
|
||||||
0x69, 0x6c, 0x64, 0x69, 0x6e, 0x67, 0x10, 0x06, 0x12, 0x0b, 0x0a, 0x07, 0x53, 0x6f, 0x72, 0x74,
|
0x03, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x44, 0x4c, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69,
|
||||||
0x69, 0x6e, 0x67, 0x10, 0x07, 0x42, 0x35, 0x5a, 0x33, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e,
|
0x6f, 0x6e, 0x10, 0x04, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x4d, 0x4c, 0x49, 0x6e, 0x73, 0x65, 0x72,
|
||||||
0x63, 0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69,
|
0x74, 0x10, 0x05, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x4d, 0x4c, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65,
|
||||||
0x6c, 0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74,
|
0x10, 0x06, 0x12, 0x0f, 0x0a, 0x0b, 0x44, 0x4d, 0x4c, 0x42, 0x75, 0x6c, 0x6b, 0x4c, 0x6f, 0x61,
|
||||||
0x6f, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72,
|
0x64, 0x10, 0x07, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x51, 0x4c, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68,
|
||||||
0x6f, 0x74, 0x6f, 0x33,
|
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 (
|
var (
|
||||||
@ -4782,7 +4922,7 @@ func file_internal_proto_rawDescGZIP() []byte {
|
|||||||
}
|
}
|
||||||
|
|
||||||
var file_internal_proto_enumTypes = make([]protoimpl.EnumInfo, 3)
|
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{}{
|
var file_internal_proto_goTypes = []interface{}{
|
||||||
(RateScope)(0), // 0: milvus.proto.internal.RateScope
|
(RateScope)(0), // 0: milvus.proto.internal.RateScope
|
||||||
(RateType)(0), // 1: milvus.proto.internal.RateType
|
(RateType)(0), // 1: milvus.proto.internal.RateType
|
||||||
@ -4833,87 +4973,90 @@ var file_internal_proto_goTypes = []interface{}{
|
|||||||
(*GetSegmentsInfoResponse)(nil), // 46: milvus.proto.internal.GetSegmentsInfoResponse
|
(*GetSegmentsInfoResponse)(nil), // 46: milvus.proto.internal.GetSegmentsInfoResponse
|
||||||
(*GetQuotaMetricsRequest)(nil), // 47: milvus.proto.internal.GetQuotaMetricsRequest
|
(*GetQuotaMetricsRequest)(nil), // 47: milvus.proto.internal.GetQuotaMetricsRequest
|
||||||
(*GetQuotaMetricsResponse)(nil), // 48: milvus.proto.internal.GetQuotaMetricsResponse
|
(*GetQuotaMetricsResponse)(nil), // 48: milvus.proto.internal.GetQuotaMetricsResponse
|
||||||
nil, // 49: milvus.proto.internal.SearchResults.ChannelsMvccEntry
|
(*FileResourceInfo)(nil), // 49: milvus.proto.internal.FileResourceInfo
|
||||||
(*commonpb.Address)(nil), // 50: milvus.proto.common.Address
|
(*SyncFileResourceRequest)(nil), // 50: milvus.proto.internal.SyncFileResourceRequest
|
||||||
(*commonpb.KeyValuePair)(nil), // 51: milvus.proto.common.KeyValuePair
|
nil, // 51: milvus.proto.internal.SearchResults.ChannelsMvccEntry
|
||||||
(*commonpb.Status)(nil), // 52: milvus.proto.common.Status
|
(*commonpb.Address)(nil), // 52: milvus.proto.common.Address
|
||||||
(*commonpb.MsgBase)(nil), // 53: milvus.proto.common.MsgBase
|
(*commonpb.KeyValuePair)(nil), // 53: milvus.proto.common.KeyValuePair
|
||||||
(commonpb.DslType)(0), // 54: milvus.proto.common.DslType
|
(*commonpb.Status)(nil), // 54: milvus.proto.common.Status
|
||||||
(commonpb.ConsistencyLevel)(0), // 55: milvus.proto.common.ConsistencyLevel
|
(*commonpb.MsgBase)(nil), // 55: milvus.proto.common.MsgBase
|
||||||
(*schemapb.IDs)(nil), // 56: milvus.proto.schema.IDs
|
(commonpb.DslType)(0), // 56: milvus.proto.common.DslType
|
||||||
(*schemapb.FieldData)(nil), // 57: milvus.proto.schema.FieldData
|
(commonpb.ConsistencyLevel)(0), // 57: milvus.proto.common.ConsistencyLevel
|
||||||
(*milvuspb.PrivilegeGroupInfo)(nil), // 58: milvus.proto.milvus.PrivilegeGroupInfo
|
(*schemapb.IDs)(nil), // 58: milvus.proto.schema.IDs
|
||||||
(*schemapb.CollectionSchema)(nil), // 59: milvus.proto.schema.CollectionSchema
|
(*schemapb.FieldData)(nil), // 59: milvus.proto.schema.FieldData
|
||||||
(commonpb.SegmentState)(0), // 60: milvus.proto.common.SegmentState
|
(*milvuspb.PrivilegeGroupInfo)(nil), // 60: milvus.proto.milvus.PrivilegeGroupInfo
|
||||||
(commonpb.SegmentLevel)(0), // 61: milvus.proto.common.SegmentLevel
|
(*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{
|
var file_internal_proto_depIdxs = []int32{
|
||||||
50, // 0: milvus.proto.internal.NodeInfo.address:type_name -> milvus.proto.common.Address
|
52, // 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
|
53, // 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
|
54, // 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
|
55, // 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
|
55, // 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
|
54, // 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, // 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
|
55, // 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
|
55, // 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
|
55, // 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
|
55, // 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
|
53, // 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
|
56, // 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
|
55, // 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
|
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
|
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
|
57, // 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
|
55, // 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
|
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
|
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
|
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, // 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
|
57, // 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
|
55, // 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
|
54, // 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
|
58, // 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
|
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
|
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
|
55, // 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
|
53, // 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
|
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
|
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
|
55, // 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
|
55, // 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
|
54, // 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
|
60, // 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
|
55, // 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
|
54, // 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
|
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
|
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
|
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
|
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
|
53, // 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
|
54, // 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
|
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
|
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
|
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
|
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
|
62, // 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
|
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, // 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, // 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
|
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
|
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
|
55, // 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
|
54, // 60: milvus.proto.internal.GetQuotaMetricsResponse.status:type_name -> milvus.proto.common.Status
|
||||||
61, // [61:61] is the sub-list for method output_type
|
49, // 61: milvus.proto.internal.SyncFileResourceRequest.resources:type_name -> milvus.proto.internal.FileResourceInfo
|
||||||
61, // [61:61] is the sub-list for method input_type
|
62, // [62:62] is the sub-list for method output_type
|
||||||
61, // [61:61] is the sub-list for extension type_name
|
62, // [62:62] is the sub-list for method input_type
|
||||||
61, // [61:61] is the sub-list for extension extendee
|
62, // [62:62] is the sub-list for extension type_name
|
||||||
0, // [0:61] is the sub-list for field 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() }
|
func init() { file_internal_proto_init() }
|
||||||
@ -5474,6 +5617,30 @@ func file_internal_proto_init() {
|
|||||||
return nil
|
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{}
|
type x struct{}
|
||||||
out := protoimpl.TypeBuilder{
|
out := protoimpl.TypeBuilder{
|
||||||
@ -5481,7 +5648,7 @@ func file_internal_proto_init() {
|
|||||||
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
|
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
|
||||||
RawDescriptor: file_internal_proto_rawDesc,
|
RawDescriptor: file_internal_proto_rawDesc,
|
||||||
NumEnums: 3,
|
NumEnums: 3,
|
||||||
NumMessages: 47,
|
NumMessages: 49,
|
||||||
NumExtensions: 0,
|
NumExtensions: 0,
|
||||||
NumServices: 0,
|
NumServices: 0,
|
||||||
},
|
},
|
||||||
|
|||||||
@ -178,6 +178,9 @@ service QueryNode {
|
|||||||
|
|
||||||
rpc DropIndex(DropIndexRequest) returns (common.Status) {}
|
rpc DropIndex(DropIndexRequest) returns (common.Status) {}
|
||||||
rpc ValidateAnalyzer(ValidateAnalyzerRequest) returns(common.Status){}
|
rpc ValidateAnalyzer(ValidateAnalyzerRequest) returns(common.Status){}
|
||||||
|
|
||||||
|
// file resource
|
||||||
|
rpc SyncFileResource(internal.SyncFileResourceRequest) returns(common.Status) {}
|
||||||
}
|
}
|
||||||
|
|
||||||
// --------------------QueryCoord grpc request and response proto------------------
|
// --------------------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,
|
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,
|
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,
|
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,
|
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, 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,
|
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,
|
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,
|
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,
|
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,
|
0x00, 0x12, 0x61, 0x0a, 0x10, 0x53, 0x79, 0x6e, 0x63, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73,
|
||||||
0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73,
|
0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
|
||||||
0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x71, 0x75,
|
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x79,
|
||||||
0x65, 0x72, 0x79, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
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 (
|
var (
|
||||||
@ -10765,17 +10771,18 @@ var file_query_coord_proto_goTypes = []interface{}{
|
|||||||
(*milvuspb.GetComponentStatesRequest)(nil), // 160: milvus.proto.milvus.GetComponentStatesRequest
|
(*milvuspb.GetComponentStatesRequest)(nil), // 160: milvus.proto.milvus.GetComponentStatesRequest
|
||||||
(*internalpb.GetTimeTickChannelRequest)(nil), // 161: milvus.proto.internal.GetTimeTickChannelRequest
|
(*internalpb.GetTimeTickChannelRequest)(nil), // 161: milvus.proto.internal.GetTimeTickChannelRequest
|
||||||
(*internalpb.GetStatisticsChannelRequest)(nil), // 162: milvus.proto.internal.GetStatisticsChannelRequest
|
(*internalpb.GetStatisticsChannelRequest)(nil), // 162: milvus.proto.internal.GetStatisticsChannelRequest
|
||||||
(*internalpb.ShowConfigurationsResponse)(nil), // 163: milvus.proto.internal.ShowConfigurationsResponse
|
(*internalpb.SyncFileResourceRequest)(nil), // 163: milvus.proto.internal.SyncFileResourceRequest
|
||||||
(*milvuspb.GetMetricsResponse)(nil), // 164: milvus.proto.milvus.GetMetricsResponse
|
(*internalpb.ShowConfigurationsResponse)(nil), // 164: milvus.proto.internal.ShowConfigurationsResponse
|
||||||
(*milvuspb.GetReplicasResponse)(nil), // 165: milvus.proto.milvus.GetReplicasResponse
|
(*milvuspb.GetMetricsResponse)(nil), // 165: milvus.proto.milvus.GetMetricsResponse
|
||||||
(*milvuspb.CheckHealthResponse)(nil), // 166: milvus.proto.milvus.CheckHealthResponse
|
(*milvuspb.GetReplicasResponse)(nil), // 166: milvus.proto.milvus.GetReplicasResponse
|
||||||
(*milvuspb.ListResourceGroupsResponse)(nil), // 167: milvus.proto.milvus.ListResourceGroupsResponse
|
(*milvuspb.CheckHealthResponse)(nil), // 167: milvus.proto.milvus.CheckHealthResponse
|
||||||
(*milvuspb.RunAnalyzerResponse)(nil), // 168: milvus.proto.milvus.RunAnalyzerResponse
|
(*milvuspb.ListResourceGroupsResponse)(nil), // 168: milvus.proto.milvus.ListResourceGroupsResponse
|
||||||
(*milvuspb.ComponentStates)(nil), // 169: milvus.proto.milvus.ComponentStates
|
(*milvuspb.RunAnalyzerResponse)(nil), // 169: milvus.proto.milvus.RunAnalyzerResponse
|
||||||
(*milvuspb.StringResponse)(nil), // 170: milvus.proto.milvus.StringResponse
|
(*milvuspb.ComponentStates)(nil), // 170: milvus.proto.milvus.ComponentStates
|
||||||
(*internalpb.GetStatisticsResponse)(nil), // 171: milvus.proto.internal.GetStatisticsResponse
|
(*milvuspb.StringResponse)(nil), // 171: milvus.proto.milvus.StringResponse
|
||||||
(*internalpb.SearchResults)(nil), // 172: milvus.proto.internal.SearchResults
|
(*internalpb.GetStatisticsResponse)(nil), // 172: milvus.proto.internal.GetStatisticsResponse
|
||||||
(*internalpb.RetrieveResults)(nil), // 173: milvus.proto.internal.RetrieveResults
|
(*internalpb.SearchResults)(nil), // 173: milvus.proto.internal.SearchResults
|
||||||
|
(*internalpb.RetrieveResults)(nil), // 174: milvus.proto.internal.RetrieveResults
|
||||||
}
|
}
|
||||||
var file_query_coord_proto_depIdxs = []int32{
|
var file_query_coord_proto_depIdxs = []int32{
|
||||||
131, // 0: milvus.proto.query.ShowCollectionsRequest.base:type_name -> milvus.proto.common.MsgBase
|
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
|
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
|
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
|
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
|
163, // 259: milvus.proto.query.QueryNode.SyncFileResource:input_type -> milvus.proto.internal.SyncFileResourceRequest
|
||||||
11, // 260: milvus.proto.query.QueryCoord.ShowLoadPartitions:output_type -> milvus.proto.query.ShowPartitionsResponse
|
9, // 260: milvus.proto.query.QueryCoord.ShowLoadCollections:output_type -> milvus.proto.query.ShowCollectionsResponse
|
||||||
132, // 261: milvus.proto.query.QueryCoord.LoadPartitions:output_type -> milvus.proto.common.Status
|
11, // 261: milvus.proto.query.QueryCoord.ShowLoadPartitions:output_type -> milvus.proto.query.ShowPartitionsResponse
|
||||||
132, // 262: milvus.proto.query.QueryCoord.ReleasePartitions:output_type -> milvus.proto.common.Status
|
132, // 262: milvus.proto.query.QueryCoord.LoadPartitions:output_type -> milvus.proto.common.Status
|
||||||
132, // 263: milvus.proto.query.QueryCoord.LoadCollection: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.ReleaseCollection: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.SyncNewCreatedPartition:output_type -> milvus.proto.common.Status
|
132, // 265: milvus.proto.query.QueryCoord.ReleaseCollection:output_type -> milvus.proto.common.Status
|
||||||
18, // 266: milvus.proto.query.QueryCoord.GetPartitionStates:output_type -> milvus.proto.query.GetPartitionStatesResponse
|
132, // 266: milvus.proto.query.QueryCoord.SyncNewCreatedPartition:output_type -> milvus.proto.common.Status
|
||||||
20, // 267: milvus.proto.query.QueryCoord.GetLoadSegmentInfo:output_type -> milvus.proto.query.GetSegmentInfoResponse
|
18, // 267: milvus.proto.query.QueryCoord.GetPartitionStates:output_type -> milvus.proto.query.GetPartitionStatesResponse
|
||||||
132, // 268: milvus.proto.query.QueryCoord.LoadBalance:output_type -> milvus.proto.common.Status
|
20, // 268: milvus.proto.query.QueryCoord.GetLoadSegmentInfo:output_type -> milvus.proto.query.GetSegmentInfoResponse
|
||||||
163, // 269: milvus.proto.query.QueryCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse
|
132, // 269: milvus.proto.query.QueryCoord.LoadBalance:output_type -> milvus.proto.common.Status
|
||||||
164, // 270: milvus.proto.query.QueryCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse
|
164, // 270: milvus.proto.query.QueryCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse
|
||||||
165, // 271: milvus.proto.query.QueryCoord.GetReplicas:output_type -> milvus.proto.milvus.GetReplicasResponse
|
165, // 271: milvus.proto.query.QueryCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse
|
||||||
22, // 272: milvus.proto.query.QueryCoord.GetShardLeaders:output_type -> milvus.proto.query.GetShardLeadersResponse
|
166, // 272: milvus.proto.query.QueryCoord.GetReplicas:output_type -> milvus.proto.milvus.GetReplicasResponse
|
||||||
166, // 273: milvus.proto.query.QueryCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse
|
22, // 273: milvus.proto.query.QueryCoord.GetShardLeaders:output_type -> milvus.proto.query.GetShardLeadersResponse
|
||||||
132, // 274: milvus.proto.query.QueryCoord.CreateResourceGroup:output_type -> milvus.proto.common.Status
|
167, // 274: milvus.proto.query.QueryCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse
|
||||||
132, // 275: milvus.proto.query.QueryCoord.UpdateResourceGroups:output_type -> milvus.proto.common.Status
|
132, // 275: milvus.proto.query.QueryCoord.CreateResourceGroup:output_type -> milvus.proto.common.Status
|
||||||
132, // 276: milvus.proto.query.QueryCoord.DropResourceGroup: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.TransferNode: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.TransferReplica:output_type -> milvus.proto.common.Status
|
132, // 278: milvus.proto.query.QueryCoord.TransferNode:output_type -> milvus.proto.common.Status
|
||||||
167, // 279: milvus.proto.query.QueryCoord.ListResourceGroups:output_type -> milvus.proto.milvus.ListResourceGroupsResponse
|
132, // 279: milvus.proto.query.QueryCoord.TransferReplica:output_type -> milvus.proto.common.Status
|
||||||
67, // 280: milvus.proto.query.QueryCoord.DescribeResourceGroup:output_type -> milvus.proto.query.DescribeResourceGroupResponse
|
168, // 280: milvus.proto.query.QueryCoord.ListResourceGroups:output_type -> milvus.proto.milvus.ListResourceGroupsResponse
|
||||||
108, // 281: milvus.proto.query.QueryCoord.ListLoadedSegments:output_type -> milvus.proto.query.ListLoadedSegmentsResponse
|
67, // 281: milvus.proto.query.QueryCoord.DescribeResourceGroup:output_type -> milvus.proto.query.DescribeResourceGroupResponse
|
||||||
75, // 282: milvus.proto.query.QueryCoord.ListCheckers:output_type -> milvus.proto.query.ListCheckersResponse
|
108, // 282: milvus.proto.query.QueryCoord.ListLoadedSegments:output_type -> milvus.proto.query.ListLoadedSegmentsResponse
|
||||||
132, // 283: milvus.proto.query.QueryCoord.ActivateChecker:output_type -> milvus.proto.common.Status
|
75, // 283: milvus.proto.query.QueryCoord.ListCheckers:output_type -> milvus.proto.query.ListCheckersResponse
|
||||||
132, // 284: milvus.proto.query.QueryCoord.DeactivateChecker:output_type -> milvus.proto.common.Status
|
132, // 284: milvus.proto.query.QueryCoord.ActivateChecker:output_type -> milvus.proto.common.Status
|
||||||
83, // 285: milvus.proto.query.QueryCoord.ListQueryNode:output_type -> milvus.proto.query.ListQueryNodeResponse
|
132, // 285: milvus.proto.query.QueryCoord.DeactivateChecker:output_type -> milvus.proto.common.Status
|
||||||
85, // 286: milvus.proto.query.QueryCoord.GetQueryNodeDistribution:output_type -> milvus.proto.query.GetQueryNodeDistributionResponse
|
83, // 286: milvus.proto.query.QueryCoord.ListQueryNode:output_type -> milvus.proto.query.ListQueryNodeResponse
|
||||||
132, // 287: milvus.proto.query.QueryCoord.SuspendBalance:output_type -> milvus.proto.common.Status
|
85, // 287: milvus.proto.query.QueryCoord.GetQueryNodeDistribution:output_type -> milvus.proto.query.GetQueryNodeDistributionResponse
|
||||||
132, // 288: milvus.proto.query.QueryCoord.ResumeBalance:output_type -> milvus.proto.common.Status
|
132, // 288: milvus.proto.query.QueryCoord.SuspendBalance:output_type -> milvus.proto.common.Status
|
||||||
89, // 289: milvus.proto.query.QueryCoord.CheckBalanceStatus:output_type -> milvus.proto.query.CheckBalanceStatusResponse
|
132, // 289: milvus.proto.query.QueryCoord.ResumeBalance:output_type -> milvus.proto.common.Status
|
||||||
132, // 290: milvus.proto.query.QueryCoord.SuspendNode: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.ResumeNode:output_type -> milvus.proto.common.Status
|
132, // 291: milvus.proto.query.QueryCoord.SuspendNode:output_type -> milvus.proto.common.Status
|
||||||
132, // 292: milvus.proto.query.QueryCoord.TransferSegment: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.TransferChannel: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.CheckQueryNodeDistribution: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.UpdateLoadConfig:output_type -> milvus.proto.common.Status
|
132, // 295: milvus.proto.query.QueryCoord.CheckQueryNodeDistribution:output_type -> milvus.proto.common.Status
|
||||||
168, // 296: milvus.proto.query.QueryCoord.RunAnalyzer:output_type -> milvus.proto.milvus.RunAnalyzerResponse
|
132, // 296: milvus.proto.query.QueryCoord.UpdateLoadConfig:output_type -> milvus.proto.common.Status
|
||||||
132, // 297: milvus.proto.query.QueryCoord.ValidateAnalyzer:output_type -> milvus.proto.common.Status
|
169, // 297: milvus.proto.query.QueryCoord.RunAnalyzer:output_type -> milvus.proto.milvus.RunAnalyzerResponse
|
||||||
169, // 298: milvus.proto.query.QueryNode.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates
|
132, // 298: milvus.proto.query.QueryCoord.ValidateAnalyzer:output_type -> milvus.proto.common.Status
|
||||||
170, // 299: milvus.proto.query.QueryNode.GetTimeTickChannel:output_type -> milvus.proto.milvus.StringResponse
|
170, // 299: milvus.proto.query.QueryNode.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates
|
||||||
170, // 300: milvus.proto.query.QueryNode.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse
|
171, // 300: milvus.proto.query.QueryNode.GetTimeTickChannel:output_type -> milvus.proto.milvus.StringResponse
|
||||||
132, // 301: milvus.proto.query.QueryNode.WatchDmChannels:output_type -> milvus.proto.common.Status
|
171, // 301: milvus.proto.query.QueryNode.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse
|
||||||
132, // 302: milvus.proto.query.QueryNode.UnsubDmChannel:output_type -> milvus.proto.common.Status
|
132, // 302: milvus.proto.query.QueryNode.WatchDmChannels:output_type -> milvus.proto.common.Status
|
||||||
132, // 303: milvus.proto.query.QueryNode.LoadSegments: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.ReleaseCollection: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.LoadPartitions: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.ReleasePartitions: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.ReleaseSegments:output_type -> milvus.proto.common.Status
|
132, // 307: milvus.proto.query.QueryNode.ReleasePartitions:output_type -> milvus.proto.common.Status
|
||||||
20, // 308: milvus.proto.query.QueryNode.GetSegmentInfo:output_type -> milvus.proto.query.GetSegmentInfoResponse
|
132, // 308: milvus.proto.query.QueryNode.ReleaseSegments:output_type -> milvus.proto.common.Status
|
||||||
132, // 309: milvus.proto.query.QueryNode.SyncReplicaSegments:output_type -> milvus.proto.common.Status
|
20, // 309: milvus.proto.query.QueryNode.GetSegmentInfo:output_type -> milvus.proto.query.GetSegmentInfoResponse
|
||||||
171, // 310: milvus.proto.query.QueryNode.GetStatistics:output_type -> milvus.proto.internal.GetStatisticsResponse
|
132, // 310: milvus.proto.query.QueryNode.SyncReplicaSegments:output_type -> milvus.proto.common.Status
|
||||||
172, // 311: milvus.proto.query.QueryNode.Search:output_type -> milvus.proto.internal.SearchResults
|
172, // 311: milvus.proto.query.QueryNode.GetStatistics:output_type -> milvus.proto.internal.GetStatisticsResponse
|
||||||
172, // 312: milvus.proto.query.QueryNode.SearchSegments:output_type -> milvus.proto.internal.SearchResults
|
173, // 312: milvus.proto.query.QueryNode.Search:output_type -> milvus.proto.internal.SearchResults
|
||||||
173, // 313: milvus.proto.query.QueryNode.Query:output_type -> milvus.proto.internal.RetrieveResults
|
173, // 313: milvus.proto.query.QueryNode.SearchSegments:output_type -> milvus.proto.internal.SearchResults
|
||||||
173, // 314: milvus.proto.query.QueryNode.QueryStream:output_type -> milvus.proto.internal.RetrieveResults
|
174, // 314: milvus.proto.query.QueryNode.Query:output_type -> milvus.proto.internal.RetrieveResults
|
||||||
173, // 315: milvus.proto.query.QueryNode.QuerySegments:output_type -> milvus.proto.internal.RetrieveResults
|
174, // 315: milvus.proto.query.QueryNode.QueryStream:output_type -> milvus.proto.internal.RetrieveResults
|
||||||
173, // 316: milvus.proto.query.QueryNode.QueryStreamSegments:output_type -> milvus.proto.internal.RetrieveResults
|
174, // 316: milvus.proto.query.QueryNode.QuerySegments:output_type -> milvus.proto.internal.RetrieveResults
|
||||||
163, // 317: milvus.proto.query.QueryNode.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse
|
174, // 317: milvus.proto.query.QueryNode.QueryStreamSegments:output_type -> milvus.proto.internal.RetrieveResults
|
||||||
164, // 318: milvus.proto.query.QueryNode.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse
|
164, // 318: milvus.proto.query.QueryNode.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse
|
||||||
52, // 319: milvus.proto.query.QueryNode.GetDataDistribution:output_type -> milvus.proto.query.GetDataDistributionResponse
|
165, // 319: milvus.proto.query.QueryNode.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse
|
||||||
132, // 320: milvus.proto.query.QueryNode.SyncDistribution:output_type -> milvus.proto.common.Status
|
52, // 320: milvus.proto.query.QueryNode.GetDataDistribution:output_type -> milvus.proto.query.GetDataDistributionResponse
|
||||||
132, // 321: milvus.proto.query.QueryNode.Delete:output_type -> milvus.proto.common.Status
|
132, // 321: milvus.proto.query.QueryNode.SyncDistribution:output_type -> milvus.proto.common.Status
|
||||||
71, // 322: milvus.proto.query.QueryNode.DeleteBatch:output_type -> milvus.proto.query.DeleteBatchResponse
|
132, // 322: milvus.proto.query.QueryNode.Delete:output_type -> milvus.proto.common.Status
|
||||||
132, // 323: milvus.proto.query.QueryNode.UpdateSchema:output_type -> milvus.proto.common.Status
|
71, // 323: milvus.proto.query.QueryNode.DeleteBatch:output_type -> milvus.proto.query.DeleteBatchResponse
|
||||||
168, // 324: milvus.proto.query.QueryNode.RunAnalyzer:output_type -> milvus.proto.milvus.RunAnalyzerResponse
|
132, // 324: milvus.proto.query.QueryNode.UpdateSchema:output_type -> milvus.proto.common.Status
|
||||||
106, // 325: milvus.proto.query.QueryNode.GetHighlight:output_type -> milvus.proto.query.GetHighlightResponse
|
169, // 325: milvus.proto.query.QueryNode.RunAnalyzer:output_type -> milvus.proto.milvus.RunAnalyzerResponse
|
||||||
132, // 326: milvus.proto.query.QueryNode.DropIndex:output_type -> milvus.proto.common.Status
|
106, // 326: milvus.proto.query.QueryNode.GetHighlight:output_type -> milvus.proto.query.GetHighlightResponse
|
||||||
132, // 327: milvus.proto.query.QueryNode.ValidateAnalyzer:output_type -> milvus.proto.common.Status
|
132, // 327: milvus.proto.query.QueryNode.DropIndex:output_type -> milvus.proto.common.Status
|
||||||
259, // [259:328] is the sub-list for method output_type
|
132, // 328: milvus.proto.query.QueryNode.ValidateAnalyzer:output_type -> milvus.proto.common.Status
|
||||||
190, // [190:259] is the sub-list for method input_type
|
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 type_name
|
||||||
190, // [190:190] is the sub-list for extension extendee
|
190, // [190:190] is the sub-list for extension extendee
|
||||||
0, // [0:190] is the sub-list for field type_name
|
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_GetHighlight_FullMethodName = "/milvus.proto.query.QueryNode/GetHighlight"
|
||||||
QueryNode_DropIndex_FullMethodName = "/milvus.proto.query.QueryNode/DropIndex"
|
QueryNode_DropIndex_FullMethodName = "/milvus.proto.query.QueryNode/DropIndex"
|
||||||
QueryNode_ValidateAnalyzer_FullMethodName = "/milvus.proto.query.QueryNode/ValidateAnalyzer"
|
QueryNode_ValidateAnalyzer_FullMethodName = "/milvus.proto.query.QueryNode/ValidateAnalyzer"
|
||||||
|
QueryNode_SyncFileResource_FullMethodName = "/milvus.proto.query.QueryNode/SyncFileResource"
|
||||||
)
|
)
|
||||||
|
|
||||||
// QueryNodeClient is the client API for QueryNode service.
|
// 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)
|
GetHighlight(ctx context.Context, in *GetHighlightRequest, opts ...grpc.CallOption) (*GetHighlightResponse, error)
|
||||||
DropIndex(ctx context.Context, in *DropIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, 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)
|
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 {
|
type queryNodeClient struct {
|
||||||
@ -1920,6 +1923,15 @@ func (c *queryNodeClient) ValidateAnalyzer(ctx context.Context, in *ValidateAnal
|
|||||||
return out, nil
|
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.
|
// QueryNodeServer is the server API for QueryNode service.
|
||||||
// All implementations should embed UnimplementedQueryNodeServer
|
// All implementations should embed UnimplementedQueryNodeServer
|
||||||
// for forward compatibility
|
// for forward compatibility
|
||||||
@ -1957,6 +1969,8 @@ type QueryNodeServer interface {
|
|||||||
GetHighlight(context.Context, *GetHighlightRequest) (*GetHighlightResponse, error)
|
GetHighlight(context.Context, *GetHighlightRequest) (*GetHighlightResponse, error)
|
||||||
DropIndex(context.Context, *DropIndexRequest) (*commonpb.Status, error)
|
DropIndex(context.Context, *DropIndexRequest) (*commonpb.Status, error)
|
||||||
ValidateAnalyzer(context.Context, *ValidateAnalyzerRequest) (*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.
|
// 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) {
|
func (UnimplementedQueryNodeServer) ValidateAnalyzer(context.Context, *ValidateAnalyzerRequest) (*commonpb.Status, error) {
|
||||||
return nil, status.Errorf(codes.Unimplemented, "method ValidateAnalyzer not implemented")
|
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.
|
// 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
|
// 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)
|
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.
|
// QueryNode_ServiceDesc is the grpc.ServiceDesc for QueryNode service.
|
||||||
// It's only intended for direct use with grpc.RegisterService,
|
// It's only intended for direct use with grpc.RegisterService,
|
||||||
// and not to be introspected or modified (even as a copy)
|
// and not to be introspected or modified (even as a copy)
|
||||||
@ -2730,6 +2765,10 @@ var QueryNode_ServiceDesc = grpc.ServiceDesc{
|
|||||||
MethodName: "ValidateAnalyzer",
|
MethodName: "ValidateAnalyzer",
|
||||||
Handler: _QueryNode_ValidateAnalyzer_Handler,
|
Handler: _QueryNode_ValidateAnalyzer_Handler,
|
||||||
},
|
},
|
||||||
|
{
|
||||||
|
MethodName: "SyncFileResource",
|
||||||
|
Handler: _QueryNode_SyncFileResource_Handler,
|
||||||
|
},
|
||||||
},
|
},
|
||||||
Streams: []grpc.StreamDesc{
|
Streams: []grpc.StreamDesc{
|
||||||
{
|
{
|
||||||
|
|||||||
@ -2497,6 +2497,8 @@ type queryCoordConfig struct {
|
|||||||
BalanceCheckCollectionMaxCount ParamItem `refreshable:"true"`
|
BalanceCheckCollectionMaxCount ParamItem `refreshable:"true"`
|
||||||
ResourceExhaustionPenaltyDuration ParamItem `refreshable:"true"`
|
ResourceExhaustionPenaltyDuration ParamItem `refreshable:"true"`
|
||||||
ResourceExhaustionCleanupInterval ParamItem `refreshable:"true"`
|
ResourceExhaustionCleanupInterval ParamItem `refreshable:"true"`
|
||||||
|
|
||||||
|
FileResourceMode ParamItem `refreshable:"false"`
|
||||||
}
|
}
|
||||||
|
|
||||||
func (p *queryCoordConfig) init(base *BaseTable) {
|
func (p *queryCoordConfig) init(base *BaseTable) {
|
||||||
@ -2508,6 +2510,13 @@ func (p *queryCoordConfig) init(base *BaseTable) {
|
|||||||
}
|
}
|
||||||
p.RetryNum.Init(base.mgr)
|
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{
|
p.RetryInterval = ParamItem{
|
||||||
Key: "queryCoord.task.retryinterval",
|
Key: "queryCoord.task.retryinterval",
|
||||||
Version: "2.2.0",
|
Version: "2.2.0",
|
||||||
@ -4631,6 +4640,7 @@ type dataCoordConfig struct {
|
|||||||
JSONStatsWriteBatchSize ParamItem `refreshable:"true"`
|
JSONStatsWriteBatchSize ParamItem `refreshable:"true"`
|
||||||
|
|
||||||
RequestTimeoutSeconds ParamItem `refreshable:"true"`
|
RequestTimeoutSeconds ParamItem `refreshable:"true"`
|
||||||
|
FileResourceMode ParamItem `refreshable:"false"`
|
||||||
}
|
}
|
||||||
|
|
||||||
func (p *dataCoordConfig) init(base *BaseTable) {
|
func (p *dataCoordConfig) init(base *BaseTable) {
|
||||||
@ -4643,6 +4653,13 @@ func (p *dataCoordConfig) init(base *BaseTable) {
|
|||||||
}
|
}
|
||||||
p.WatchTimeoutInterval.Init(base.mgr)
|
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{
|
p.LegacyVersionWithoutRPCWatch = ParamItem{
|
||||||
Key: "dataCoord.channel.legacyVersionWithoutRPCWatch",
|
Key: "dataCoord.channel.legacyVersionWithoutRPCWatch",
|
||||||
Version: "2.4.1",
|
Version: "2.4.1",
|
||||||
|
|||||||
Loading…
x
Reference in New Issue
Block a user