mirror of https://github.com/milvus-io/milvus.git
enhance: Remove vector chunk manager (#28569)
We have implemented the chunkcache (in cpp) to retrieve vectors, hence rendering the vectorchunkcache (in golang) obsolete. issue: https://github.com/milvus-io/milvus/issues/28568 --------- Signed-off-by: bigsheeper <yihao.dai@zilliz.com>pull/28876/head
parent
adce6aab4f
commit
b4353ca4ce
|
@ -121,8 +121,7 @@ type QueryNode struct {
|
|||
session *sessionutil.Session
|
||||
eventCh <-chan *sessionutil.SessionEvent
|
||||
|
||||
cacheChunkManager storage.ChunkManager
|
||||
vectorStorage storage.ChunkManager
|
||||
chunkManager storage.ChunkManager
|
||||
|
||||
/*
|
||||
// Pool for search/query
|
||||
|
@ -283,7 +282,6 @@ func (node *QueryNode) Init() error {
|
|||
node.factory.Init(paramtable.Get())
|
||||
|
||||
localRootPath := paramtable.Get().LocalStorageCfg.Path.GetValue()
|
||||
localChunkManager := storage.NewLocalChunkManager(storage.RootPath(localRootPath))
|
||||
localUsedSize, err := segments.GetLocalUsedSize(localRootPath)
|
||||
if err != nil {
|
||||
log.Warn("get local used size failed", zap.Error(err))
|
||||
|
@ -291,33 +289,14 @@ func (node *QueryNode) Init() error {
|
|||
return
|
||||
}
|
||||
metrics.QueryNodeDiskUsedSize.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Set(float64(localUsedSize / 1024 / 1024))
|
||||
remoteChunkManager, err := node.factory.NewPersistentStorageChunkManager(node.ctx)
|
||||
if err != nil {
|
||||
log.Warn("failed to init remote chunk manager", zap.Error(err))
|
||||
initError = err
|
||||
return
|
||||
}
|
||||
node.cacheChunkManager, err = storage.NewVectorChunkManager(node.ctx,
|
||||
localChunkManager,
|
||||
remoteChunkManager,
|
||||
paramtable.Get().QueryNodeCfg.CacheMemoryLimit.GetAsInt64(),
|
||||
paramtable.Get().QueryNodeCfg.CacheEnabled.GetAsBool(),
|
||||
)
|
||||
if err != nil {
|
||||
log.Error("failed to init cache chunk manager", zap.Error(err))
|
||||
initError = err
|
||||
return
|
||||
}
|
||||
|
||||
node.vectorStorage, err = node.factory.NewPersistentStorageChunkManager(node.ctx)
|
||||
node.chunkManager, err = node.factory.NewPersistentStorageChunkManager(node.ctx)
|
||||
if err != nil {
|
||||
log.Error("QueryNode init vector storage failed", zap.Error(err))
|
||||
initError = err
|
||||
return
|
||||
}
|
||||
|
||||
log.Info("queryNode try to connect etcd success", zap.String("MetaRootPath", paramtable.Get().EtcdCfg.MetaRootPath.GetValue()))
|
||||
|
||||
schedulePolicy := paramtable.Get().QueryNodeCfg.SchedulePolicyName.GetValue()
|
||||
node.scheduler = tasks.NewScheduler(
|
||||
schedulePolicy,
|
||||
|
@ -353,7 +332,7 @@ func (node *QueryNode) Init() error {
|
|||
node.subscribingChannels = typeutil.NewConcurrentSet[string]()
|
||||
node.unsubscribingChannels = typeutil.NewConcurrentSet[string]()
|
||||
node.manager = segments.NewManager()
|
||||
node.loader = segments.NewLoader(node.manager, node.vectorStorage)
|
||||
node.loader = segments.NewLoader(node.manager, node.chunkManager)
|
||||
node.dispClient = msgdispatcher.NewClient(node.factory, typeutil.QueryNodeRole, paramtable.GetNodeID())
|
||||
// init pipeline manager
|
||||
node.pipelineManager = pipeline.NewManager(node.manager, node.tSafeManager, node.dispClient, node.delegators)
|
||||
|
|
|
@ -143,7 +143,6 @@ func (suite *QueryNodeSuite) TestInit_VactorChunkManagerFailed() {
|
|||
|
||||
// init vactor chunk manager failed
|
||||
suite.factory.EXPECT().Init(mock.Anything).Return()
|
||||
suite.factory.EXPECT().NewPersistentStorageChunkManager(mock.Anything).Return(suite.chunkManagerFactory.NewPersistentStorageChunkManager(context.Background())).Once()
|
||||
suite.factory.EXPECT().NewPersistentStorageChunkManager(mock.Anything).Return(nil, errors.New("mock error")).Once()
|
||||
err = suite.node.Init()
|
||||
suite.Error(err)
|
||||
|
|
|
@ -160,7 +160,7 @@ func (suite *ServiceSuite) TearDownTest() {
|
|||
})
|
||||
suite.NoError(err)
|
||||
suite.Equal(commonpb.ErrorCode_Success, resp.ErrorCode)
|
||||
suite.node.vectorStorage.RemoveWithPrefix(ctx, suite.rootPath)
|
||||
suite.node.chunkManager.RemoveWithPrefix(ctx, suite.rootPath)
|
||||
|
||||
suite.node.Stop()
|
||||
suite.etcdClient.Close()
|
||||
|
@ -260,7 +260,7 @@ func (suite *ServiceSuite) TestWatchDmChannelsInt64() {
|
|||
deltaLogs, err := segments.SaveDeltaLog(suite.collectionID,
|
||||
suite.partitionIDs[0],
|
||||
suite.flushedSegmentIDs[0],
|
||||
suite.node.cacheChunkManager,
|
||||
suite.node.chunkManager,
|
||||
)
|
||||
suite.NoError(err)
|
||||
|
||||
|
@ -530,7 +530,7 @@ func (suite *ServiceSuite) genSegmentLoadInfos(schema *schemapb.CollectionSchema
|
|||
suite.validSegmentIDs[i],
|
||||
1000,
|
||||
schema,
|
||||
suite.node.vectorStorage,
|
||||
suite.node.chunkManager,
|
||||
)
|
||||
suite.Require().NoError(err)
|
||||
|
||||
|
@ -543,7 +543,7 @@ func (suite *ServiceSuite) genSegmentLoadInfos(schema *schemapb.CollectionSchema
|
|||
1000,
|
||||
segments.IndexFaissIVFFlat,
|
||||
metric.L2,
|
||||
suite.node.vectorStorage,
|
||||
suite.node.chunkManager,
|
||||
)
|
||||
suite.Require().NoError(err)
|
||||
|
||||
|
|
|
@ -26,6 +26,10 @@ import (
|
|||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
const (
|
||||
localPath = "/tmp/milvus_test/chunkmanager/"
|
||||
)
|
||||
|
||||
func TestLocalCM(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
|
||||
|
|
|
@ -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 storage
|
||||
|
||||
import (
|
||||
"os"
|
||||
"testing"
|
||||
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
)
|
||||
|
||||
var Params = paramtable.Get()
|
||||
|
||||
func TestMain(m *testing.M) {
|
||||
paramtable.Init()
|
||||
exitCode := m.Run()
|
||||
err := os.RemoveAll(localPath)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
os.Exit(exitCode)
|
||||
}
|
|
@ -1,337 +0,0 @@
|
|||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package storage
|
||||
|
||||
import (
|
||||
"context"
|
||||
"io"
|
||||
"path"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"go.uber.org/zap"
|
||||
"golang.org/x/exp/mmap"
|
||||
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/util/cache"
|
||||
)
|
||||
|
||||
var defaultLocalCacheSize = 64
|
||||
|
||||
// VectorChunkManager is responsible for read and write vector data.
|
||||
type VectorChunkManager struct {
|
||||
cacheStorage ChunkManager
|
||||
vectorStorage ChunkManager
|
||||
cache cache.LoadingCache[string, *mmap.ReaderAt]
|
||||
|
||||
insertCodec *InsertCodec
|
||||
|
||||
cacheEnable bool
|
||||
cacheLimit int64
|
||||
cacheSize int64
|
||||
cacheSizeMutex sync.Mutex
|
||||
}
|
||||
|
||||
var _ ChunkManager = (*VectorChunkManager)(nil)
|
||||
|
||||
// NewVectorChunkManager create a new vector manager object.
|
||||
func NewVectorChunkManager(ctx context.Context, cacheStorage ChunkManager, vectorStorage ChunkManager, cacheLimit int64, cacheEnable bool) (*VectorChunkManager, error) {
|
||||
insertCodec := NewInsertCodec()
|
||||
vcm := &VectorChunkManager{
|
||||
cacheStorage: cacheStorage,
|
||||
vectorStorage: vectorStorage,
|
||||
|
||||
insertCodec: insertCodec,
|
||||
cacheEnable: cacheEnable,
|
||||
cacheLimit: cacheLimit,
|
||||
}
|
||||
|
||||
err := vcm.initCache(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return vcm, nil
|
||||
}
|
||||
|
||||
func (vcm *VectorChunkManager) initCache(ctx context.Context) error {
|
||||
if !vcm.cacheEnable {
|
||||
return nil
|
||||
}
|
||||
|
||||
if vcm.cacheLimit <= 0 {
|
||||
return errors.New("cache limit must be positive if cacheEnable")
|
||||
}
|
||||
|
||||
loader := func(filePath string) (*mmap.ReaderAt, error) {
|
||||
return vcm.readFile(ctx, filePath)
|
||||
}
|
||||
|
||||
onRemoveFn := func(filePath string, v *mmap.ReaderAt) {
|
||||
size := v.Len()
|
||||
err := v.Close()
|
||||
if err != nil {
|
||||
log.Error("close mmap file failed", zap.Any("file", filePath))
|
||||
}
|
||||
localPath := path.Join(vcm.cacheStorage.RootPath(), filePath)
|
||||
err = vcm.cacheStorage.Remove(ctx, localPath)
|
||||
if err != nil {
|
||||
log.Error("cache storage remove file failed", zap.Any("file", localPath))
|
||||
}
|
||||
|
||||
vcm.cacheSizeMutex.Lock()
|
||||
vcm.cacheSize -= int64(size)
|
||||
vcm.cacheSizeMutex.Unlock()
|
||||
}
|
||||
|
||||
vcm.cache = cache.NewLoadingCache(loader,
|
||||
cache.WithRemovalListener[string, *mmap.ReaderAt](onRemoveFn),
|
||||
cache.WithMaximumSize[string, *mmap.ReaderAt](vcm.cacheLimit),
|
||||
)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// For vector data, we will download vector file from storage. And we will
|
||||
// deserialize the file for it has binlog style. At last we store pure vector
|
||||
// data to local storage as cache.
|
||||
func (vcm *VectorChunkManager) deserializeVectorFile(filePath string, content []byte) ([]byte, error) {
|
||||
blob := &Blob{
|
||||
Key: filePath,
|
||||
Value: content,
|
||||
}
|
||||
|
||||
_, _, data, err := vcm.insertCodec.Deserialize([]*Blob{blob})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Note: here we assume that only one field in the binlog.
|
||||
var results []byte
|
||||
for _, singleData := range data.Data {
|
||||
bs, err := FieldDataToBytes(common.Endian, singleData)
|
||||
if err == nil {
|
||||
results = bs
|
||||
}
|
||||
}
|
||||
return results, nil
|
||||
}
|
||||
|
||||
// RootPath returns vector root path
|
||||
func (vcm *VectorChunkManager) RootPath() string {
|
||||
return vcm.vectorStorage.RootPath()
|
||||
}
|
||||
|
||||
// Path returns the path of vector data. If cached, return local path.
|
||||
// If not cached return remote path.
|
||||
func (vcm *VectorChunkManager) Path(ctx context.Context, filePath string) (string, error) {
|
||||
return vcm.vectorStorage.Path(ctx, filePath)
|
||||
}
|
||||
|
||||
func (vcm *VectorChunkManager) Size(ctx context.Context, filePath string) (int64, error) {
|
||||
return vcm.vectorStorage.Size(ctx, filePath)
|
||||
}
|
||||
|
||||
// Write writes the vector data to local cache if cache enabled.
|
||||
func (vcm *VectorChunkManager) Write(ctx context.Context, filePath string, content []byte) error {
|
||||
return vcm.vectorStorage.Write(ctx, filePath, content)
|
||||
}
|
||||
|
||||
// MultiWrite writes the vector data to local cache if cache enabled.
|
||||
func (vcm *VectorChunkManager) MultiWrite(ctx context.Context, contents map[string][]byte) error {
|
||||
return vcm.vectorStorage.MultiWrite(ctx, contents)
|
||||
}
|
||||
|
||||
// Exist checks whether vector data is saved to local cache.
|
||||
func (vcm *VectorChunkManager) Exist(ctx context.Context, filePath string) (bool, error) {
|
||||
return vcm.vectorStorage.Exist(ctx, filePath)
|
||||
}
|
||||
|
||||
func (vcm *VectorChunkManager) readFile(ctx context.Context, filePath string) (*mmap.ReaderAt, error) {
|
||||
contents, err := vcm.vectorStorage.Read(ctx, filePath)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
results, err := vcm.deserializeVectorFile(filePath, contents)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
localPath := path.Join(vcm.cacheStorage.RootPath(), filePath)
|
||||
err = vcm.cacheStorage.Write(ctx, localPath, results)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
r, err := vcm.cacheStorage.Mmap(ctx, localPath)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
vcm.cacheSizeMutex.Lock()
|
||||
vcm.cacheSize += int64(r.Len())
|
||||
vcm.cacheSizeMutex.Unlock()
|
||||
return r, nil
|
||||
}
|
||||
|
||||
// Read reads the pure vector data. If cached, it reads from local.
|
||||
func (vcm *VectorChunkManager) Read(ctx context.Context, filePath string) ([]byte, error) {
|
||||
if vcm.cacheEnable {
|
||||
r, err := vcm.cache.Get(filePath)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
p := make([]byte, r.Len())
|
||||
_, err = r.ReadAt(p, 0)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return p, nil
|
||||
}
|
||||
contents, err := vcm.vectorStorage.Read(ctx, filePath)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return vcm.deserializeVectorFile(filePath, contents)
|
||||
}
|
||||
|
||||
// MultiRead reads the pure vector data. If cached, it reads from local.
|
||||
func (vcm *VectorChunkManager) MultiRead(ctx context.Context, filePaths []string) ([][]byte, error) {
|
||||
results := make([][]byte, len(filePaths))
|
||||
for i, filePath := range filePaths {
|
||||
content, err := vcm.Read(ctx, filePath)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
results[i] = content
|
||||
}
|
||||
|
||||
return results, nil
|
||||
}
|
||||
|
||||
func (vcm *VectorChunkManager) ReadWithPrefix(ctx context.Context, prefix string) ([]string, [][]byte, error) {
|
||||
filePaths, _, err := vcm.ListWithPrefix(ctx, prefix, true)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
results, err := vcm.MultiRead(ctx, filePaths)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
return filePaths, results, nil
|
||||
}
|
||||
|
||||
func (vcm *VectorChunkManager) ListWithPrefix(ctx context.Context, prefix string, recursive bool) ([]string, []time.Time, error) {
|
||||
return vcm.vectorStorage.ListWithPrefix(ctx, prefix, recursive)
|
||||
}
|
||||
|
||||
func (vcm *VectorChunkManager) Mmap(ctx context.Context, filePath string) (*mmap.ReaderAt, error) {
|
||||
if vcm.cacheEnable && vcm.cache != nil {
|
||||
if r, err := vcm.cache.Get(filePath); err == nil {
|
||||
return r, nil
|
||||
}
|
||||
}
|
||||
return nil, errors.New("the file mmap has not been cached")
|
||||
}
|
||||
|
||||
func (vcm *VectorChunkManager) Reader(ctx context.Context, filePath string) (FileReader, error) {
|
||||
return nil, errors.New("this method has not been implemented")
|
||||
}
|
||||
|
||||
// ReadAt reads specific position data of vector. If cached, it reads from local.
|
||||
func (vcm *VectorChunkManager) ReadAt(ctx context.Context, filePath string, off int64, length int64) ([]byte, error) {
|
||||
if vcm.cacheEnable {
|
||||
r, err := vcm.cache.Get(filePath)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
p := make([]byte, length)
|
||||
_, err = r.ReadAt(p, off)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return p, nil
|
||||
}
|
||||
contents, err := vcm.vectorStorage.Read(ctx, filePath)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
results, err := vcm.deserializeVectorFile(filePath, contents)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if off < 0 || int64(len(results)) < off {
|
||||
return nil, errors.New("vectorChunkManager: invalid offset")
|
||||
}
|
||||
|
||||
p := make([]byte, length)
|
||||
n := copy(p, results[off:])
|
||||
if n < len(p) {
|
||||
return nil, io.EOF
|
||||
}
|
||||
return p, nil
|
||||
}
|
||||
|
||||
func (vcm *VectorChunkManager) Remove(ctx context.Context, filePath string) error {
|
||||
err := vcm.vectorStorage.Remove(ctx, filePath)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if vcm.cacheEnable {
|
||||
vcm.cache.Invalidate(filePath)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (vcm *VectorChunkManager) MultiRemove(ctx context.Context, filePaths []string) error {
|
||||
err := vcm.vectorStorage.MultiRemove(ctx, filePaths)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if vcm.cacheEnable {
|
||||
for _, p := range filePaths {
|
||||
vcm.cache.Invalidate(p)
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (vcm *VectorChunkManager) RemoveWithPrefix(ctx context.Context, prefix string) error {
|
||||
err := vcm.vectorStorage.RemoveWithPrefix(ctx, prefix)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if vcm.cacheEnable {
|
||||
filePaths, _, err := vcm.ListWithPrefix(ctx, prefix, true)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
for _, p := range filePaths {
|
||||
vcm.cache.Invalidate(p)
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (vcm *VectorChunkManager) Close() {
|
||||
if vcm.cache != nil && vcm.cacheEnable {
|
||||
vcm.cache.Close()
|
||||
}
|
||||
}
|
|
@ -1,486 +0,0 @@
|
|||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package storage
|
||||
|
||||
import (
|
||||
"context"
|
||||
"os"
|
||||
"path"
|
||||
"testing"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/etcdpb"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
|
||||
func initMeta() *etcdpb.CollectionMeta {
|
||||
meta := &etcdpb.CollectionMeta{
|
||||
ID: 1,
|
||||
CreateTime: 1,
|
||||
SegmentIDs: []int64{0, 1},
|
||||
PartitionTags: []string{"partition_0", "partition_1"},
|
||||
Schema: &schemapb.CollectionSchema{
|
||||
Name: "schema",
|
||||
Description: "schema",
|
||||
AutoID: true,
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: 0,
|
||||
Name: "row_id",
|
||||
IsPrimaryKey: false,
|
||||
Description: "row_id",
|
||||
DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: 1,
|
||||
Name: "Ts",
|
||||
IsPrimaryKey: false,
|
||||
Description: "Ts",
|
||||
DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: 101,
|
||||
Name: "field_int8",
|
||||
IsPrimaryKey: false,
|
||||
Description: "description_3",
|
||||
DataType: schemapb.DataType_Int8,
|
||||
},
|
||||
{
|
||||
FieldID: 108,
|
||||
Name: "field_binary_vector",
|
||||
IsPrimaryKey: false,
|
||||
Description: "description_10",
|
||||
DataType: schemapb.DataType_BinaryVector,
|
||||
},
|
||||
{
|
||||
FieldID: 109,
|
||||
Name: "field_float_vector",
|
||||
IsPrimaryKey: false,
|
||||
Description: "description_11",
|
||||
DataType: schemapb.DataType_FloatVector,
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
return meta
|
||||
}
|
||||
|
||||
func initBinlogFile(schema *etcdpb.CollectionMeta) []*Blob {
|
||||
insertCodec := NewInsertCodecWithSchema(schema)
|
||||
insertData := &InsertData{
|
||||
Data: map[int64]FieldData{
|
||||
0: &Int64FieldData{
|
||||
Data: []int64{3, 4},
|
||||
},
|
||||
1: &Int64FieldData{
|
||||
Data: []int64{3, 4},
|
||||
},
|
||||
101: &Int8FieldData{
|
||||
Data: []int8{3, 4},
|
||||
},
|
||||
108: &BinaryVectorFieldData{
|
||||
Data: []byte{0, 255},
|
||||
Dim: 8,
|
||||
},
|
||||
109: &FloatVectorFieldData{
|
||||
Data: []float32{0, 1, 2, 3, 4, 5, 6, 7, 0, 111, 222, 333, 444, 555, 777, 666},
|
||||
Dim: 8,
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
blobs, err := insertCodec.Serialize(1, 1, insertData)
|
||||
if err != nil {
|
||||
return nil
|
||||
}
|
||||
return blobs
|
||||
}
|
||||
|
||||
func buildVectorChunkManager(ctx context.Context, localPath string, localCacheEnable bool) (*VectorChunkManager, error) {
|
||||
bucketName := "vector-chunk-manager"
|
||||
|
||||
rcm, err := newMinIOChunkManager(ctx, bucketName, "")
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
lcm := NewLocalChunkManager(RootPath(localPath))
|
||||
|
||||
vcm, err := NewVectorChunkManager(ctx, lcm, rcm, 16, localCacheEnable)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return vcm, nil
|
||||
}
|
||||
|
||||
var (
|
||||
Params = paramtable.Get()
|
||||
localPath = "/tmp/milvus_test/chunkmanager/"
|
||||
)
|
||||
|
||||
func TestMain(m *testing.M) {
|
||||
paramtable.Init()
|
||||
exitCode := m.Run()
|
||||
err := os.RemoveAll(localPath)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
os.Exit(exitCode)
|
||||
}
|
||||
|
||||
func TestNewVectorChunkManager(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
bucketName := "vector-chunk-manager"
|
||||
|
||||
rcm, err := newMinIOChunkManager(ctx, bucketName, "")
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, rcm)
|
||||
lcm := NewLocalChunkManager(RootPath(localPath))
|
||||
|
||||
vcm, err := NewVectorChunkManager(ctx, lcm, rcm, 16, true)
|
||||
assert.Equal(t, "", vcm.RootPath())
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, vcm)
|
||||
|
||||
vcm, err = NewVectorChunkManager(ctx, lcm, rcm, -1, true)
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, vcm)
|
||||
}
|
||||
|
||||
func TestVectorChunkManager_GetPath(t *testing.T) {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
localCaches := []bool{true, false}
|
||||
for _, localCache := range localCaches {
|
||||
vcm, err := buildVectorChunkManager(ctx, localPath, localCache)
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, vcm)
|
||||
|
||||
key := path.Join(localPath, "1")
|
||||
err = vcm.Write(ctx, key, []byte{1})
|
||||
assert.NoError(t, err)
|
||||
pathGet, err := vcm.Path(ctx, key)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, pathGet, key)
|
||||
|
||||
err = vcm.cacheStorage.Write(ctx, key, []byte{1})
|
||||
assert.NoError(t, err)
|
||||
pathGet, err = vcm.Path(ctx, key)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, pathGet, key)
|
||||
|
||||
err = vcm.RemoveWithPrefix(ctx, localPath)
|
||||
assert.NoError(t, err)
|
||||
vcm.Close()
|
||||
}
|
||||
}
|
||||
|
||||
func TestVectorChunkManager_GetSize(t *testing.T) {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
localCaches := []bool{true, false}
|
||||
for _, localCache := range localCaches {
|
||||
vcm, err := buildVectorChunkManager(ctx, localPath, localCache)
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, vcm)
|
||||
|
||||
key := path.Join(localPath, "1")
|
||||
err = vcm.Write(ctx, key, []byte{1})
|
||||
assert.NoError(t, err)
|
||||
sizeGet, err := vcm.Size(ctx, key)
|
||||
assert.NoError(t, err)
|
||||
assert.EqualValues(t, sizeGet, 1)
|
||||
|
||||
err = vcm.cacheStorage.Write(ctx, key, []byte{1})
|
||||
assert.NoError(t, err)
|
||||
sizeGet, err = vcm.Size(ctx, key)
|
||||
assert.NoError(t, err)
|
||||
assert.EqualValues(t, sizeGet, 1)
|
||||
|
||||
err = vcm.RemoveWithPrefix(ctx, localPath)
|
||||
assert.NoError(t, err)
|
||||
vcm.Close()
|
||||
}
|
||||
}
|
||||
|
||||
func TestVectorChunkManager_Write(t *testing.T) {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
localCaches := []bool{true, false}
|
||||
for _, localCache := range localCaches {
|
||||
vcm, err := buildVectorChunkManager(ctx, localPath, localCache)
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, vcm)
|
||||
|
||||
key1 := path.Join(localPath, "key_1")
|
||||
key2 := path.Join(localPath, "key_2")
|
||||
err = vcm.Write(ctx, key1, []byte{1})
|
||||
assert.NoError(t, err)
|
||||
|
||||
exist, err := vcm.Exist(ctx, key1)
|
||||
assert.True(t, exist)
|
||||
assert.NoError(t, err)
|
||||
|
||||
contents := map[string][]byte{
|
||||
key1: {111},
|
||||
key2: {222},
|
||||
}
|
||||
err = vcm.MultiWrite(ctx, contents)
|
||||
assert.NoError(t, err)
|
||||
|
||||
exist, err = vcm.Exist(ctx, key1)
|
||||
assert.True(t, exist)
|
||||
assert.NoError(t, err)
|
||||
exist, err = vcm.Exist(ctx, key2)
|
||||
assert.True(t, exist)
|
||||
assert.NoError(t, err)
|
||||
|
||||
err = vcm.RemoveWithPrefix(ctx, localPath)
|
||||
assert.NoError(t, err)
|
||||
vcm.Close()
|
||||
}
|
||||
}
|
||||
|
||||
func TestVectorChunkManager_Remove(t *testing.T) {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
localCaches := []bool{true, false}
|
||||
for _, localCache := range localCaches {
|
||||
vcm, err := buildVectorChunkManager(ctx, localPath, localCache)
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, vcm)
|
||||
|
||||
key1 := path.Join(localPath, "key_1")
|
||||
key2 := path.Join(localPath, "key_2")
|
||||
err = vcm.cacheStorage.Write(ctx, key1, []byte{1})
|
||||
assert.NoError(t, err)
|
||||
|
||||
err = vcm.Remove(ctx, key1)
|
||||
assert.NoError(t, err)
|
||||
|
||||
exist, err := vcm.Exist(ctx, key1)
|
||||
assert.False(t, exist)
|
||||
assert.NoError(t, err)
|
||||
|
||||
contents := map[string][]byte{
|
||||
key1: {111},
|
||||
key2: {222},
|
||||
}
|
||||
err = vcm.cacheStorage.MultiWrite(ctx, contents)
|
||||
assert.NoError(t, err)
|
||||
|
||||
err = vcm.MultiRemove(ctx, []string{key1, key2})
|
||||
assert.NoError(t, err)
|
||||
|
||||
exist, err = vcm.Exist(ctx, key1)
|
||||
assert.False(t, exist)
|
||||
assert.NoError(t, err)
|
||||
exist, err = vcm.Exist(ctx, key2)
|
||||
assert.False(t, exist)
|
||||
assert.NoError(t, err)
|
||||
|
||||
err = vcm.RemoveWithPrefix(ctx, localPath)
|
||||
assert.NoError(t, err)
|
||||
vcm.Close()
|
||||
}
|
||||
}
|
||||
|
||||
type mockFailedChunkManager struct {
|
||||
fail bool
|
||||
ChunkManager
|
||||
}
|
||||
|
||||
func (m *mockFailedChunkManager) Remove(ctx context.Context, key string) error {
|
||||
if m.fail {
|
||||
return errors.New("remove error")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *mockFailedChunkManager) RemoveWithPrefix(ctx context.Context, prefix string) error {
|
||||
if m.fail {
|
||||
return errors.New("remove with prefix error")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *mockFailedChunkManager) MultiRemove(ctx context.Context, key []string) error {
|
||||
if m.fail {
|
||||
return errors.New("multi remove error")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func TestVectorChunkManager_Remove_Fail(t *testing.T) {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
vcm := &VectorChunkManager{
|
||||
vectorStorage: &mockFailedChunkManager{fail: true},
|
||||
cacheStorage: &mockFailedChunkManager{fail: true},
|
||||
}
|
||||
assert.Error(t, vcm.Remove(ctx, "test"))
|
||||
assert.Error(t, vcm.MultiRemove(ctx, []string{"test"}))
|
||||
assert.Error(t, vcm.RemoveWithPrefix(ctx, "test"))
|
||||
}
|
||||
|
||||
func TestVectorChunkManager_LocalPath(t *testing.T) {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
vcm, err := buildVectorChunkManager(ctx, localPath, true)
|
||||
assert.NotNil(t, vcm)
|
||||
assert.NoError(t, err)
|
||||
|
||||
meta := initMeta()
|
||||
binlogs := initBinlogFile(meta)
|
||||
keys := make([]string, len(binlogs))
|
||||
for i, binlog := range binlogs {
|
||||
err = vcm.vectorStorage.Write(ctx, binlog.Key, binlog.Value)
|
||||
assert.Nil(t, err)
|
||||
keys[i] = binlog.Key
|
||||
}
|
||||
|
||||
// cache file to cacheStorage
|
||||
_, err = vcm.Read(ctx, keys[0])
|
||||
assert.NoError(t, err)
|
||||
|
||||
// check file under cacheStorage.rootPath
|
||||
absLocalPath := path.Join(vcm.cacheStorage.RootPath(), keys[0])
|
||||
exit, err := vcm.cacheStorage.Exist(ctx, absLocalPath)
|
||||
assert.NoError(t, err)
|
||||
assert.True(t, exit)
|
||||
|
||||
err = vcm.RemoveWithPrefix(ctx, localPath)
|
||||
assert.NoError(t, err)
|
||||
|
||||
vcm.Close()
|
||||
}
|
||||
|
||||
func TestVectorChunkManager_Read(t *testing.T) {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
localCaches := []bool{true, false}
|
||||
for _, localCache := range localCaches {
|
||||
vcm, err := buildVectorChunkManager(ctx, localPath, localCache)
|
||||
assert.NotNil(t, vcm)
|
||||
assert.NoError(t, err)
|
||||
|
||||
content, err := vcm.Read(ctx, "9999")
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, content)
|
||||
|
||||
meta := initMeta()
|
||||
binlogs := initBinlogFile(meta)
|
||||
assert.NotNil(t, binlogs)
|
||||
for _, binlog := range binlogs {
|
||||
err := vcm.vectorStorage.Write(ctx, binlog.Key, binlog.Value)
|
||||
assert.NoError(t, err)
|
||||
}
|
||||
|
||||
content, err = vcm.Read(ctx, "108")
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, []byte{0, 255}, content)
|
||||
|
||||
content, err = vcm.Read(ctx, "109")
|
||||
assert.NoError(t, err)
|
||||
floatResult := make([]float32, 0)
|
||||
for i := 0; i < len(content)/4; i++ {
|
||||
singleData := typeutil.BytesToFloat32(content[i*4 : i*4+4])
|
||||
floatResult = append(floatResult, singleData)
|
||||
}
|
||||
assert.Equal(t, []float32{0, 1, 2, 3, 4, 5, 6, 7, 0, 111, 222, 333, 444, 555, 777, 666}, floatResult)
|
||||
|
||||
contents, err := vcm.MultiRead(ctx, []string{"108", "109"})
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, []byte{0, 255}, contents[0])
|
||||
|
||||
floatResult = make([]float32, 0)
|
||||
for i := 0; i < len(content)/4; i++ {
|
||||
singleData := typeutil.BytesToFloat32(contents[1][i*4 : i*4+4])
|
||||
floatResult = append(floatResult, singleData)
|
||||
}
|
||||
assert.Equal(t, []float32{0, 1, 2, 3, 4, 5, 6, 7, 0, 111, 222, 333, 444, 555, 777, 666}, floatResult)
|
||||
|
||||
keys, contents, err := vcm.ReadWithPrefix(ctx, "10")
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, "101", keys[0])
|
||||
assert.Equal(t, []byte{3, 4}, contents[0])
|
||||
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, "108", keys[1])
|
||||
assert.Equal(t, []byte{0, 255}, contents[1])
|
||||
|
||||
floatResult = make([]float32, 0)
|
||||
for i := 0; i < len(content)/4; i++ {
|
||||
singleData := typeutil.BytesToFloat32(contents[2][i*4 : i*4+4])
|
||||
floatResult = append(floatResult, singleData)
|
||||
}
|
||||
assert.Equal(t, "109", keys[2])
|
||||
assert.Equal(t, []float32{0, 1, 2, 3, 4, 5, 6, 7, 0, 111, 222, 333, 444, 555, 777, 666}, floatResult)
|
||||
|
||||
content, err = vcm.ReadAt(ctx, "109", 8*4, 8*4)
|
||||
assert.NoError(t, err)
|
||||
|
||||
floatResult = make([]float32, 0)
|
||||
for i := 0; i < len(content)/4; i++ {
|
||||
singleData := typeutil.BytesToFloat32(content[i*4 : i*4+4])
|
||||
floatResult = append(floatResult, singleData)
|
||||
}
|
||||
assert.Equal(t, []float32{0, 111, 222, 333, 444, 555, 777, 666}, floatResult)
|
||||
|
||||
content, err = vcm.ReadAt(ctx, "9999", 0, 8*4)
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, content)
|
||||
|
||||
content, err = vcm.ReadAt(ctx, "109", 8*4, 8*4)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 32, len(content))
|
||||
|
||||
if localCache {
|
||||
r, err := vcm.Mmap(ctx, "109")
|
||||
assert.NoError(t, err)
|
||||
p := make([]byte, 32)
|
||||
n, err := r.ReadAt(p, 32)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, n, 32)
|
||||
|
||||
r, err = vcm.Mmap(ctx, "not exist")
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, r)
|
||||
}
|
||||
|
||||
content, err = vcm.ReadAt(ctx, "109", 9999, 8*4)
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, content)
|
||||
|
||||
content, err = vcm.ReadAt(ctx, "9999", 0, 8*4)
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, content)
|
||||
|
||||
err = vcm.RemoveWithPrefix(ctx, localPath)
|
||||
assert.NoError(t, err)
|
||||
|
||||
vcm.Close()
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue