mirror of https://github.com/milvus-io/milvus.git
enhance: add vector index mgr to remove vector index type dependency (#36843)
issue: #34298 Signed-off-by: xianliang.li <xianliang.li@zilliz.com>pull/36819/head
parent
eb3e4583ec
commit
3de57ec4fa
|
@ -9,8 +9,8 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/metastore/model"
|
||||
"github.com/milvus-io/milvus/internal/rootcoord"
|
||||
"github.com/milvus-io/milvus/internal/tso"
|
||||
"github.com/milvus-io/milvus/internal/util/vecindexmgr"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/util/indexparamcheck"
|
||||
)
|
||||
|
||||
// In Milvus 2.3.x, querynode.MmapDirPath is used to enable mmap and save mmap files.
|
||||
|
@ -84,7 +84,7 @@ func (m *MmapMigration) MigrateIndexCoordCollection(ctx context.Context) {
|
|||
|
||||
alteredIndexes := make([]*model.Index, 0)
|
||||
for _, index := range fieldIndexes {
|
||||
if !indexparamcheck.IsVectorMmapIndex(getIndexType(index.IndexParams)) {
|
||||
if !vecindexmgr.GetVecIndexMgrInstance().IsMMapSupported(getIndexType(index.IndexParams)) {
|
||||
continue
|
||||
}
|
||||
fmt.Printf("migrate index, collection:%v, indexId: %v, indexName: %s\n", index.CollectionID, index.IndexID, index.IndexName)
|
||||
|
|
|
@ -22,6 +22,7 @@
|
|||
#include "common/EasyAssert.h"
|
||||
#include "knowhere/comp/index_param.h"
|
||||
#include "knowhere/dataset.h"
|
||||
#include "knowhere/index/index_factory.h"
|
||||
#include "common/Tracer.h"
|
||||
#include "common/Types.h"
|
||||
#include "index/Meta.h"
|
||||
|
@ -62,23 +63,8 @@ class IndexBase {
|
|||
virtual const bool
|
||||
HasRawData() const = 0;
|
||||
|
||||
bool
|
||||
IsMmapSupported() const {
|
||||
return index_type_ == knowhere::IndexEnum::INDEX_HNSW ||
|
||||
index_type_ == knowhere::IndexEnum::INDEX_FAISS_IVFFLAT ||
|
||||
index_type_ == knowhere::IndexEnum::INDEX_FAISS_IVFFLAT_CC ||
|
||||
index_type_ == knowhere::IndexEnum::INDEX_FAISS_IVFPQ ||
|
||||
index_type_ == knowhere::IndexEnum::INDEX_FAISS_IVFSQ8 ||
|
||||
index_type_ == knowhere::IndexEnum::INDEX_FAISS_BIN_IVFFLAT ||
|
||||
index_type_ == knowhere::IndexEnum::INDEX_FAISS_IDMAP ||
|
||||
index_type_ == knowhere::IndexEnum::INDEX_FAISS_BIN_IDMAP ||
|
||||
index_type_ ==
|
||||
knowhere::IndexEnum::INDEX_SPARSE_INVERTED_INDEX ||
|
||||
index_type_ == knowhere::IndexEnum::INDEX_SPARSE_WAND ||
|
||||
// support mmap for bitmap/hybrid index
|
||||
index_type_ == milvus::index::BITMAP_INDEX_TYPE ||
|
||||
index_type_ == milvus::index::HYBRID_INDEX_TYPE;
|
||||
}
|
||||
virtual bool
|
||||
IsMmapSupported() const = 0;
|
||||
|
||||
const IndexType&
|
||||
Type() const {
|
||||
|
|
|
@ -133,6 +133,12 @@ class ScalarIndex : public IndexBase {
|
|||
PanicInfo(Unsupported, "pattern match is not supported");
|
||||
}
|
||||
|
||||
virtual bool
|
||||
IsMmapSupported() const {
|
||||
return index_type_ == milvus::index::BITMAP_INDEX_TYPE ||
|
||||
index_type_ == milvus::index::HYBRID_INDEX_TYPE;
|
||||
}
|
||||
|
||||
virtual int64_t
|
||||
Size() = 0;
|
||||
|
||||
|
|
|
@ -115,6 +115,12 @@ class VectorIndex : public IndexBase {
|
|||
err_msg);
|
||||
}
|
||||
|
||||
virtual bool
|
||||
IsMmapSupported() const {
|
||||
return knowhere::IndexFactory::Instance().FeatureCheck(
|
||||
index_type_, knowhere::feature::MMAP);
|
||||
}
|
||||
|
||||
knowhere::Json
|
||||
PrepareSearchParams(const SearchInfo& search_info) const {
|
||||
knowhere::Json search_cfg = search_info.search_params_;
|
||||
|
|
|
@ -0,0 +1,40 @@
|
|||
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
// or implied. See the License for the specific language governing permissions and limitations under the License
|
||||
|
||||
#include "segcore/vector_index_c.h"
|
||||
|
||||
#include "common/Types.h"
|
||||
#include "common/EasyAssert.h"
|
||||
#include "knowhere/utils.h"
|
||||
#include "knowhere/config.h"
|
||||
#include "knowhere/version.h"
|
||||
#include "index/Meta.h"
|
||||
#include "index/IndexFactory.h"
|
||||
#include "pb/index_cgo_msg.pb.h"
|
||||
|
||||
int
|
||||
GetIndexListSize() {
|
||||
return knowhere::IndexFactory::Instance().GetIndexFeatures().size();
|
||||
}
|
||||
|
||||
void
|
||||
GetIndexFeatures(void* index_key_list, uint64_t* index_feature_list) {
|
||||
auto features = knowhere::IndexFactory::Instance().GetIndexFeatures();
|
||||
int idx = 0;
|
||||
|
||||
const char** index_keys = (const char**)index_key_list;
|
||||
uint64_t* index_features = (uint64_t*)index_feature_list;
|
||||
for (auto it = features.begin(); it != features.end(); ++it) {
|
||||
index_keys[idx] = it->first.c_str();
|
||||
index_features[idx] = it->second;
|
||||
idx++;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,28 @@
|
|||
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software distributed under the License
|
||||
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
||||
// or implied. See the License for the specific language governing permissions and limitations under the License
|
||||
|
||||
#include <stdint.h>
|
||||
#ifdef __cplusplus
|
||||
extern "C" {
|
||||
#endif
|
||||
|
||||
#include <stdbool.h>
|
||||
#include "common/type_c.h"
|
||||
|
||||
int
|
||||
GetIndexListSize();
|
||||
|
||||
void
|
||||
GetIndexFeatures(void* index_key_list, uint64_t* index_feature_list);
|
||||
|
||||
#ifdef __cplusplus
|
||||
}
|
||||
#endif
|
|
@ -14,7 +14,7 @@
|
|||
# Update KNOWHERE_VERSION for the first occurrence
|
||||
milvus_add_pkg_config("knowhere")
|
||||
set_property(DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR} PROPERTY INCLUDE_DIRECTORIES "")
|
||||
set( KNOWHERE_VERSION d0d7eefb )
|
||||
set( KNOWHERE_VERSION be5e77c )
|
||||
set( GIT_REPOSITORY "https://github.com/zilliztech/knowhere.git")
|
||||
message(STATUS "Knowhere repo: ${GIT_REPOSITORY}")
|
||||
message(STATUS "Knowhere version: ${KNOWHERE_VERSION}")
|
||||
|
|
|
@ -45,6 +45,7 @@
|
|||
#include "exec/expression/Expr.h"
|
||||
#include "segcore/load_index_c.h"
|
||||
#include "test_utils/c_api_test_utils.h"
|
||||
#include "segcore/vector_index_c.h"
|
||||
|
||||
namespace chrono = std::chrono;
|
||||
|
||||
|
@ -712,6 +713,22 @@ TEST(CApiTest, MultiDeleteGrowingSegment) {
|
|||
DeleteSegment(segment);
|
||||
}
|
||||
|
||||
TEST(CApiTest, GetIndexListSizeAndFeatures) {
|
||||
int size = GetIndexListSize();
|
||||
ASSERT_GT(size, 0);
|
||||
|
||||
std::vector<const char*> index_keys(size);
|
||||
std::vector<uint64_t> index_features(size);
|
||||
|
||||
GetIndexFeatures(index_keys.data(), index_features.data());
|
||||
|
||||
for (int i = 0; i < size; i++) {
|
||||
ASSERT_NE(index_keys[i], nullptr);
|
||||
ASSERT_GT(strlen(index_keys[i]), 0);
|
||||
ASSERT_GT(index_features[i], 0);
|
||||
}
|
||||
}
|
||||
|
||||
TEST(CApiTest, MultiDeleteSealedSegment) {
|
||||
auto collection = NewCollection(get_default_schema_config());
|
||||
CSegmentInterface segment;
|
||||
|
|
|
@ -34,6 +34,7 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/metastore/model"
|
||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/workerpb"
|
||||
"github.com/milvus-io/milvus/internal/util/vecindexmgr"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/metrics"
|
||||
|
@ -980,7 +981,7 @@ func (m *indexMeta) AreAllDiskIndex(collectionID int64, schema *schemapb.Collect
|
|||
})
|
||||
vectorFieldsWithDiskIndex := lo.Filter(vectorFields, func(field *schemapb.FieldSchema, _ int) bool {
|
||||
if indexType, ok := fieldIndexTypes[field.FieldID]; ok {
|
||||
return indexparamcheck.IsDiskIndex(indexType)
|
||||
return vecindexmgr.GetVecIndexMgrInstance().IsDiskVecIndex(indexType)
|
||||
}
|
||||
return false
|
||||
})
|
||||
|
|
|
@ -28,6 +28,7 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/metastore/model"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/internal/util/vecindexmgr"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/metrics"
|
||||
"github.com/milvus-io/milvus/pkg/util/funcutil"
|
||||
|
@ -231,10 +232,10 @@ func (s *Server) CreateIndex(ctx context.Context, req *indexpb.CreateIndexReques
|
|||
metrics.IndexRequestCounter.WithLabelValues(metrics.FailLabel).Inc()
|
||||
return merr.Status(err), nil
|
||||
}
|
||||
if GetIndexType(req.GetIndexParams()) == indexparamcheck.IndexDISKANN && !s.indexNodeManager.ClientSupportDisk() {
|
||||
if vecindexmgr.GetVecIndexMgrInstance().IsDiskANN(GetIndexType(req.IndexParams)) && !s.indexNodeManager.ClientSupportDisk() {
|
||||
errMsg := "all IndexNodes do not support disk indexes, please verify"
|
||||
log.Warn(errMsg)
|
||||
err = merr.WrapErrIndexNotSupported(indexparamcheck.IndexDISKANN)
|
||||
err = merr.WrapErrIndexNotSupported(GetIndexType(req.IndexParams))
|
||||
metrics.IndexRequestCounter.WithLabelValues(metrics.FailLabel).Inc()
|
||||
return merr.Status(err), nil
|
||||
}
|
||||
|
|
|
@ -146,7 +146,7 @@ func (it *indexBuildTask) PreCheck(ctx context.Context, dependency *taskSchedule
|
|||
}
|
||||
indexParams := dependency.meta.indexMeta.GetIndexParams(segIndex.CollectionID, segIndex.IndexID)
|
||||
indexType := GetIndexType(indexParams)
|
||||
if isFlatIndex(indexType) || segIndex.NumRows < Params.DataCoordCfg.MinSegmentNumRowsToEnableIndex.GetAsInt64() {
|
||||
if isNoTrainIndex(indexType) || segIndex.NumRows < Params.DataCoordCfg.MinSegmentNumRowsToEnableIndex.GetAsInt64() {
|
||||
log.Ctx(ctx).Info("segment does not need index really", zap.Int64("taskID", it.taskID),
|
||||
zap.Int64("segmentID", segIndex.SegmentID), zap.Int64("num rows", segIndex.NumRows))
|
||||
it.SetStartTime(time.Now())
|
||||
|
|
|
@ -39,7 +39,6 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/workerpb"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/util/indexparamcheck"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
)
|
||||
|
@ -1470,7 +1469,7 @@ func (s *taskSchedulerSuite) Test_indexTaskWithMvOptionalScalarField() {
|
|||
},
|
||||
{
|
||||
Key: common.IndexTypeKey,
|
||||
Value: indexparamcheck.IndexHNSW,
|
||||
Value: "HNSW",
|
||||
},
|
||||
},
|
||||
},
|
||||
|
@ -1523,7 +1522,7 @@ func (s *taskSchedulerSuite) Test_indexTaskWithMvOptionalScalarField() {
|
|||
},
|
||||
{
|
||||
Key: common.IndexTypeKey,
|
||||
Value: indexparamcheck.IndexHNSW,
|
||||
Value: "HNSW",
|
||||
},
|
||||
},
|
||||
},
|
||||
|
@ -1624,7 +1623,7 @@ func (s *taskSchedulerSuite) Test_indexTaskWithMvOptionalScalarField() {
|
|||
resetMetaFunc := func() {
|
||||
mt.indexMeta.buildID2SegmentIndex[buildID].IndexState = commonpb.IndexState_Unissued
|
||||
mt.indexMeta.segmentIndexes[segID][indexID].IndexState = commonpb.IndexState_Unissued
|
||||
mt.indexMeta.indexes[collID][indexID].IndexParams[1].Value = indexparamcheck.IndexHNSW
|
||||
mt.indexMeta.indexes[collID][indexID].IndexParams[1].Value = "HNSW"
|
||||
mt.collections[collID].Schema.Fields[0].DataType = schemapb.DataType_FloatVector
|
||||
mt.collections[collID].Schema.Fields[1].IsPartitionKey = true
|
||||
mt.collections[collID].Schema.Fields[1].DataType = schemapb.DataType_VarChar
|
||||
|
|
|
@ -30,11 +30,11 @@ import (
|
|||
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/internal/util/vecindexmgr"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/metrics"
|
||||
"github.com/milvus-io/milvus/pkg/util/funcutil"
|
||||
"github.com/milvus-io/milvus/pkg/util/indexparamcheck"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/util/tsoutil"
|
||||
|
@ -203,16 +203,16 @@ func GetIndexType(indexParams []*commonpb.KeyValuePair) string {
|
|||
return invalidIndex
|
||||
}
|
||||
|
||||
func isFlatIndex(indexType string) bool {
|
||||
return indexType == indexparamcheck.IndexFaissIDMap || indexType == indexparamcheck.IndexFaissBinIDMap
|
||||
func isNoTrainIndex(indexType string) bool {
|
||||
return vecindexmgr.GetVecIndexMgrInstance().IsNoTrainIndex(indexType)
|
||||
}
|
||||
|
||||
func isOptionalScalarFieldSupported(indexType string) bool {
|
||||
return indexType == indexparamcheck.IndexHNSW
|
||||
return vecindexmgr.GetVecIndexMgrInstance().IsMvSupported(indexType)
|
||||
}
|
||||
|
||||
func isDiskANNIndex(indexType string) bool {
|
||||
return indexType == indexparamcheck.IndexDISKANN
|
||||
return vecindexmgr.GetVecIndexMgrInstance().IsDiskANN(indexType)
|
||||
}
|
||||
|
||||
func parseBuildIDFromFilePath(key string) (UniqueID, error) {
|
||||
|
|
|
@ -33,10 +33,10 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/proto/workerpb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/util/indexcgowrapper"
|
||||
"github.com/milvus-io/milvus/internal/util/vecindexmgr"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/metrics"
|
||||
"github.com/milvus-io/milvus/pkg/util/indexparamcheck"
|
||||
"github.com/milvus-io/milvus/pkg/util/indexparams"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/util/metautil"
|
||||
|
@ -210,7 +210,7 @@ func (it *indexBuildTask) Execute(ctx context.Context) error {
|
|||
zap.Int32("currentIndexVersion", it.req.GetCurrentIndexVersion()))
|
||||
|
||||
indexType := it.newIndexParams[common.IndexTypeKey]
|
||||
if indexType == indexparamcheck.IndexDISKANN {
|
||||
if vecindexmgr.GetVecIndexMgrInstance().IsDiskANN(indexType) {
|
||||
// check index node support disk index
|
||||
if !Params.IndexNodeCfg.EnableDisk.GetAsBool() {
|
||||
log.Warn("IndexNode don't support build disk index",
|
||||
|
|
|
@ -20,7 +20,6 @@ import (
|
|||
"testing"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/pkg/util/indexparamcheck"
|
||||
)
|
||||
|
||||
func Test_CheckVecIndexWithDataTypeExist(t *testing.T) {
|
||||
|
@ -29,25 +28,25 @@ func Test_CheckVecIndexWithDataTypeExist(t *testing.T) {
|
|||
dataType schemapb.DataType
|
||||
want bool
|
||||
}{
|
||||
{indexparamcheck.IndexHNSW, schemapb.DataType_FloatVector, true},
|
||||
{indexparamcheck.IndexHNSW, schemapb.DataType_BinaryVector, false},
|
||||
{indexparamcheck.IndexHNSW, schemapb.DataType_Float16Vector, true},
|
||||
{"HNSW", schemapb.DataType_FloatVector, true},
|
||||
{"HNSW", schemapb.DataType_BinaryVector, false},
|
||||
{"HNSW", schemapb.DataType_Float16Vector, true},
|
||||
|
||||
{indexparamcheck.IndexSparseWand, schemapb.DataType_SparseFloatVector, true},
|
||||
{indexparamcheck.IndexSparseWand, schemapb.DataType_FloatVector, false},
|
||||
{indexparamcheck.IndexSparseWand, schemapb.DataType_Float16Vector, false},
|
||||
{"SPARSE_WAND", schemapb.DataType_SparseFloatVector, true},
|
||||
{"SPARSE_WAND", schemapb.DataType_FloatVector, false},
|
||||
{"SPARSE_WAND", schemapb.DataType_Float16Vector, false},
|
||||
|
||||
{indexparamcheck.IndexGpuBF, schemapb.DataType_FloatVector, true},
|
||||
{indexparamcheck.IndexGpuBF, schemapb.DataType_Float16Vector, false},
|
||||
{indexparamcheck.IndexGpuBF, schemapb.DataType_BinaryVector, false},
|
||||
{"GPU_BRUTE_FORCE", schemapb.DataType_FloatVector, true},
|
||||
{"GPU_BRUTE_FORCE", schemapb.DataType_Float16Vector, false},
|
||||
{"GPU_BRUTE_FORCE", schemapb.DataType_BinaryVector, false},
|
||||
|
||||
{indexparamcheck.IndexFaissBinIvfFlat, schemapb.DataType_BinaryVector, true},
|
||||
{indexparamcheck.IndexFaissBinIvfFlat, schemapb.DataType_FloatVector, false},
|
||||
{"BIN_IVF_FLAT", schemapb.DataType_BinaryVector, true},
|
||||
{"BIN_IVF_FLAT", schemapb.DataType_FloatVector, false},
|
||||
|
||||
{indexparamcheck.IndexDISKANN, schemapb.DataType_FloatVector, true},
|
||||
{indexparamcheck.IndexDISKANN, schemapb.DataType_Float16Vector, true},
|
||||
{indexparamcheck.IndexDISKANN, schemapb.DataType_BFloat16Vector, true},
|
||||
{indexparamcheck.IndexDISKANN, schemapb.DataType_BinaryVector, false},
|
||||
{"DISKANN", schemapb.DataType_FloatVector, true},
|
||||
{"DISKANN", schemapb.DataType_Float16Vector, true},
|
||||
{"DISKANN", schemapb.DataType_BFloat16Vector, true},
|
||||
{"DISKANN", schemapb.DataType_BinaryVector, false},
|
||||
}
|
||||
|
||||
for _, test := range cases {
|
||||
|
|
|
@ -28,6 +28,7 @@ import (
|
|||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/vecindexmgr"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/mq/msgstream"
|
||||
|
@ -340,7 +341,7 @@ func (cit *createIndexTask) parseIndexParams() error {
|
|||
if !exist {
|
||||
return fmt.Errorf("IndexType not specified")
|
||||
}
|
||||
if indexType == indexparamcheck.IndexDISKANN {
|
||||
if vecindexmgr.GetVecIndexMgrInstance().IsDiskANN(indexType) {
|
||||
err := indexparams.FillDiskIndexParams(Params, indexParamsMap)
|
||||
if err != nil {
|
||||
return err
|
||||
|
|
|
@ -37,10 +37,10 @@ import (
|
|||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||
"github.com/milvus-io/milvus/internal/proto/segcorepb"
|
||||
"github.com/milvus-io/milvus/internal/util/vecindexmgr"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/metrics"
|
||||
"github.com/milvus-io/milvus/pkg/util/indexparamcheck"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
|
@ -263,7 +263,7 @@ func NewCollection(collectionID int64, schema *schemapb.CollectionSchema, indexM
|
|||
|
||||
for _, indexMeta := range indexMeta.GetIndexMetas() {
|
||||
isGpuIndex = lo.ContainsBy(indexMeta.GetIndexParams(), func(param *commonpb.KeyValuePair) bool {
|
||||
return param.Key == common.IndexTypeKey && indexparamcheck.IsGpuIndex(param.Value)
|
||||
return param.Key == common.IndexTypeKey && vecindexmgr.GetVecIndexMgrInstance().IsGPUVecIndex(param.Value)
|
||||
})
|
||||
if isGpuIndex {
|
||||
break
|
||||
|
|
|
@ -29,6 +29,7 @@ import (
|
|||
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||
"github.com/milvus-io/milvus/internal/util/vecindexmgr"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/util/conc"
|
||||
"github.com/milvus-io/milvus/pkg/util/funcutil"
|
||||
|
@ -60,7 +61,7 @@ func (c *IndexAttrCache) GetIndexResourceUsage(indexInfo *querypb.FieldIndexInfo
|
|||
if err != nil {
|
||||
return 0, 0, fmt.Errorf("index type not exist in index params")
|
||||
}
|
||||
if indexType == indexparamcheck.IndexDISKANN {
|
||||
if vecindexmgr.GetVecIndexMgrInstance().IsDiskANN(indexType) {
|
||||
neededMemSize := indexInfo.IndexSize / UsedDiskMemoryRatio
|
||||
neededDiskSize := indexInfo.IndexSize - neededMemSize
|
||||
return uint64(neededMemSize), uint64(neededDiskSize), nil
|
||||
|
|
|
@ -62,7 +62,7 @@ func (s *IndexAttrCacheSuite) TestCacheMissing() {
|
|||
func (s *IndexAttrCacheSuite) TestDiskANN() {
|
||||
info := &querypb.FieldIndexInfo{
|
||||
IndexParams: []*commonpb.KeyValuePair{
|
||||
{Key: common.IndexTypeKey, Value: indexparamcheck.IndexDISKANN},
|
||||
{Key: common.IndexTypeKey, Value: "DISKANN"},
|
||||
},
|
||||
CurrentIndexVersion: 0,
|
||||
IndexSize: 100,
|
||||
|
@ -71,7 +71,7 @@ func (s *IndexAttrCacheSuite) TestDiskANN() {
|
|||
memory, disk, err := s.c.GetIndexResourceUsage(info, paramtable.Get().QueryNodeCfg.MemoryIndexLoadPredictMemoryUsageFactor.GetAsFloat(), nil)
|
||||
s.Require().NoError(err)
|
||||
|
||||
_, has := s.c.loadWithDisk.Get(typeutil.NewPair[string, int32](indexparamcheck.IndexDISKANN, 0))
|
||||
_, has := s.c.loadWithDisk.Get(typeutil.NewPair[string, int32]("DISKANN", 0))
|
||||
s.False(has, "DiskANN shall never be checked load with disk")
|
||||
|
||||
s.EqualValues(25, memory)
|
||||
|
|
|
@ -52,6 +52,7 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/querynodev2/segments/state"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/util/cgo"
|
||||
"github.com/milvus-io/milvus/internal/util/vecindexmgr"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/metrics"
|
||||
|
@ -1114,7 +1115,7 @@ func GetCLoadInfoWithFunc(ctx context.Context,
|
|||
delete(indexParams, common.MmapEnabledKey)
|
||||
|
||||
// some build params also exist in indexParams, which are useless during loading process
|
||||
if indexParams["index_type"] == indexparamcheck.IndexDISKANN {
|
||||
if vecindexmgr.GetVecIndexMgrInstance().IsDiskANN(indexParams["index_type"]) {
|
||||
if err := indexparams.SetDiskIndexLoadParams(paramtable.Get(), indexParams, indexInfo.GetNumRows()); err != nil {
|
||||
return err
|
||||
}
|
||||
|
|
|
@ -29,6 +29,7 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/querycoordv2/params"
|
||||
"github.com/milvus-io/milvus/internal/querynodev2/segments/metricsutil"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/util/vecindexmgr"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/mq/msgstream"
|
||||
|
@ -265,7 +266,7 @@ func isIndexMmapEnable(fieldSchema *schemapb.FieldSchema, indexInfo *querypb.Fie
|
|||
var indexSupportMmap bool
|
||||
var defaultEnableMmap bool
|
||||
if typeutil.IsVectorType(fieldSchema.GetDataType()) {
|
||||
indexSupportMmap = indexparamcheck.IsVectorMmapIndex(indexType)
|
||||
indexSupportMmap = vecindexmgr.GetVecIndexMgrInstance().IsMMapSupported(indexType)
|
||||
defaultEnableMmap = params.Params.QueryNodeCfg.MmapVectorIndex.GetAsBool()
|
||||
} else {
|
||||
indexSupportMmap = indexparamcheck.IsScalarMmapIndex(indexType)
|
||||
|
|
|
@ -10,7 +10,6 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/util/indexparamcheck"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
)
|
||||
|
||||
|
@ -131,7 +130,7 @@ func TestIsIndexMmapEnable(t *testing.T) {
|
|||
IndexParams: []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: common.IndexTypeKey,
|
||||
Value: indexparamcheck.IndexFaissIvfFlat,
|
||||
Value: "IVF_FLAT",
|
||||
},
|
||||
},
|
||||
})
|
||||
|
@ -147,7 +146,7 @@ func TestIsIndexMmapEnable(t *testing.T) {
|
|||
IndexParams: []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: common.IndexTypeKey,
|
||||
Value: indexparamcheck.IndexINVERTED,
|
||||
Value: "INVERTED",
|
||||
},
|
||||
},
|
||||
})
|
||||
|
|
|
@ -0,0 +1,244 @@
|
|||
// 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 vecindexmgr
|
||||
|
||||
/*
|
||||
#cgo pkg-config: milvus_core
|
||||
|
||||
#include <stdlib.h> // free
|
||||
#include "segcore/vector_index_c.h"
|
||||
*/
|
||||
import "C"
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"fmt"
|
||||
"sync"
|
||||
"unsafe"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
)
|
||||
|
||||
const (
|
||||
BinaryFlag uint64 = 1 << 0
|
||||
Float32Flag uint64 = 1 << 1
|
||||
Float16Flag uint64 = 1 << 2
|
||||
BFloat16Flag uint64 = 1 << 3
|
||||
SparseFloat32Flag uint64 = 1 << 4
|
||||
|
||||
// NOTrainFlag This flag indicates that there is no need to create any index structure
|
||||
NOTrainFlag uint64 = 1 << 16
|
||||
// KNNFlag This flag indicates that the index defaults to KNN search, meaning the recall rate is 100%
|
||||
KNNFlag uint64 = 1 << 17
|
||||
// GpuFlag This flag indicates that the index is deployed on GPU (need GPU devices)
|
||||
GpuFlag uint64 = 1 << 18
|
||||
// MmapFlag This flag indicates that the index support using mmap manage its mainly memory, which can significant improve the capacity
|
||||
MmapFlag uint64 = 1 << 19
|
||||
// MvFlag This flag indicates that the index support using materialized view to accelerate filtering search
|
||||
MvFlag uint64 = 1 << 20
|
||||
// DiskFlag This flag indicates that the index need disk
|
||||
DiskFlag uint64 = 1 << 21
|
||||
)
|
||||
|
||||
type IndexType = string
|
||||
|
||||
type VecIndexMgr interface {
|
||||
init()
|
||||
|
||||
GetFeature(indexType IndexType) (uint64, bool)
|
||||
|
||||
IsBinarySupport(indexType IndexType) bool
|
||||
IsFlat32Support(indexType IndexType) bool
|
||||
IsFlat16Support(indexType IndexType) bool
|
||||
IsBFlat16Support(indexType IndexType) bool
|
||||
IsSparseFloat32Support(indexType IndexType) bool
|
||||
IsDataTypeSupport(indexType IndexType, dataType schemapb.DataType) bool
|
||||
|
||||
IsFlatVecIndex(indexType IndexType) bool
|
||||
IsNoTrainIndex(indexType IndexType) bool
|
||||
IsVecIndex(indexType IndexType) bool
|
||||
IsDiskANN(indexType IndexType) bool
|
||||
IsGPUVecIndex(indexType IndexType) bool
|
||||
IsDiskVecIndex(indexType IndexType) bool
|
||||
IsMMapSupported(indexType IndexType) bool
|
||||
IsMvSupported(indexType IndexType) bool
|
||||
}
|
||||
|
||||
type vecIndexMgrImpl struct {
|
||||
features map[string]uint64
|
||||
once sync.Once
|
||||
}
|
||||
|
||||
func (mgr *vecIndexMgrImpl) GetFeature(indexType IndexType) (uint64, bool) {
|
||||
feature, ok := mgr.features[indexType]
|
||||
if !ok {
|
||||
log.Error("can not find index " + indexType + " in the index feature map")
|
||||
return 0, false
|
||||
}
|
||||
return feature, true
|
||||
}
|
||||
|
||||
func (mgr *vecIndexMgrImpl) IsNoTrainIndex(indexType IndexType) bool {
|
||||
feature, ok := mgr.GetFeature(indexType)
|
||||
if !ok {
|
||||
return false
|
||||
}
|
||||
return (feature & NOTrainFlag) == NOTrainFlag
|
||||
}
|
||||
|
||||
func (mgr *vecIndexMgrImpl) IsDiskANN(indexType IndexType) bool {
|
||||
return indexType == "DISKANN"
|
||||
}
|
||||
|
||||
func (mgr *vecIndexMgrImpl) init() {
|
||||
size := int(C.GetIndexListSize())
|
||||
if size == 0 {
|
||||
log.Error("get empty vector index features from vector index engine")
|
||||
return
|
||||
}
|
||||
vecIndexList := make([]unsafe.Pointer, size)
|
||||
vecIndexFeatures := make([]uint64, size)
|
||||
|
||||
C.GetIndexFeatures(unsafe.Pointer(&vecIndexList[0]), (*C.uint64_t)(unsafe.Pointer(&vecIndexFeatures[0])))
|
||||
mgr.features = make(map[string]uint64)
|
||||
var featureLog bytes.Buffer
|
||||
for i := 0; i < size; i++ {
|
||||
key := C.GoString((*C.char)(vecIndexList[i]))
|
||||
mgr.features[key] = vecIndexFeatures[i]
|
||||
featureLog.WriteString(key + " : " + fmt.Sprintf("%d", vecIndexFeatures[i]) + ",")
|
||||
}
|
||||
log.Info("init vector indexes with features : " + featureLog.String())
|
||||
}
|
||||
|
||||
func (mgr *vecIndexMgrImpl) IsBinarySupport(indexType IndexType) bool {
|
||||
feature, ok := mgr.GetFeature(indexType)
|
||||
if !ok {
|
||||
return false
|
||||
}
|
||||
return (feature & BinaryFlag) == BinaryFlag
|
||||
}
|
||||
|
||||
func (mgr *vecIndexMgrImpl) IsFlat32Support(indexType IndexType) bool {
|
||||
feature, ok := mgr.GetFeature(indexType)
|
||||
if !ok {
|
||||
return false
|
||||
}
|
||||
return (feature & Float32Flag) == Float32Flag
|
||||
}
|
||||
|
||||
func (mgr *vecIndexMgrImpl) IsFlat16Support(indexType IndexType) bool {
|
||||
feature, ok := mgr.GetFeature(indexType)
|
||||
if !ok {
|
||||
return false
|
||||
}
|
||||
return (feature & Float16Flag) == Float16Flag
|
||||
}
|
||||
|
||||
func (mgr *vecIndexMgrImpl) IsBFlat16Support(indexType IndexType) bool {
|
||||
feature, ok := mgr.GetFeature(indexType)
|
||||
if !ok {
|
||||
return false
|
||||
}
|
||||
return (feature & BFloat16Flag) == BFloat16Flag
|
||||
}
|
||||
|
||||
func (mgr *vecIndexMgrImpl) IsSparseFloat32Support(indexType IndexType) bool {
|
||||
feature, ok := mgr.GetFeature(indexType)
|
||||
if !ok {
|
||||
return false
|
||||
}
|
||||
return (feature & SparseFloat32Flag) == SparseFloat32Flag
|
||||
}
|
||||
|
||||
func (mgr *vecIndexMgrImpl) IsDataTypeSupport(indexType IndexType, dataType schemapb.DataType) bool {
|
||||
if dataType == schemapb.DataType_BinaryVector {
|
||||
return mgr.IsBinarySupport(indexType)
|
||||
} else if dataType == schemapb.DataType_FloatVector {
|
||||
return mgr.IsFlat32Support(indexType)
|
||||
} else if dataType == schemapb.DataType_BFloat16Vector {
|
||||
return mgr.IsBFlat16Support(indexType)
|
||||
} else if dataType == schemapb.DataType_Float16Vector {
|
||||
return mgr.IsFlat16Support(indexType)
|
||||
} else if dataType == schemapb.DataType_SparseFloatVector {
|
||||
return mgr.IsSparseFloat32Support(indexType)
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func (mgr *vecIndexMgrImpl) IsFlatVecIndex(indexType IndexType) bool {
|
||||
feature, ok := mgr.features[indexType]
|
||||
if !ok {
|
||||
return false
|
||||
}
|
||||
return (feature & KNNFlag) == KNNFlag
|
||||
}
|
||||
|
||||
func (mgr *vecIndexMgrImpl) IsMvSupported(indexType IndexType) bool {
|
||||
feature, ok := mgr.GetFeature(indexType)
|
||||
if !ok {
|
||||
return false
|
||||
}
|
||||
return (feature & MvFlag) == MvFlag
|
||||
}
|
||||
|
||||
func (mgr *vecIndexMgrImpl) IsGPUVecIndex(indexType IndexType) bool {
|
||||
feature, ok := mgr.GetFeature(indexType)
|
||||
if !ok {
|
||||
return false
|
||||
}
|
||||
return (feature & GpuFlag) == GpuFlag
|
||||
}
|
||||
|
||||
func (mgr *vecIndexMgrImpl) IsMMapSupported(indexType IndexType) bool {
|
||||
feature, ok := mgr.GetFeature(indexType)
|
||||
if !ok {
|
||||
return false
|
||||
}
|
||||
return (feature & MmapFlag) == MmapFlag
|
||||
}
|
||||
|
||||
func (mgr *vecIndexMgrImpl) IsVecIndex(indexType IndexType) bool {
|
||||
_, ok := mgr.GetFeature(indexType)
|
||||
return ok
|
||||
}
|
||||
|
||||
func (mgr *vecIndexMgrImpl) IsDiskVecIndex(indexType IndexType) bool {
|
||||
feature, ok := mgr.GetFeature(indexType)
|
||||
if !ok {
|
||||
return false
|
||||
}
|
||||
return (feature & DiskFlag) == DiskFlag
|
||||
}
|
||||
|
||||
func newVecIndexMgr() *vecIndexMgrImpl {
|
||||
mgr := &vecIndexMgrImpl{}
|
||||
mgr.once.Do(mgr.init)
|
||||
return mgr
|
||||
}
|
||||
|
||||
var vecIndexMgr VecIndexMgr
|
||||
|
||||
var getVecIndexMgrOnce sync.Once
|
||||
|
||||
// GetVecIndexMgrInstance gets the instance of VecIndexMgrInstance.
|
||||
func GetVecIndexMgrInstance() VecIndexMgr {
|
||||
getVecIndexMgrOnce.Do(func() {
|
||||
vecIndexMgr = newVecIndexMgr()
|
||||
})
|
||||
return vecIndexMgr
|
||||
}
|
|
@ -0,0 +1,232 @@
|
|||
package vecindexmgr
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
)
|
||||
|
||||
func Test_VecIndex_DataType_Support(t *testing.T) {
|
||||
type testCase struct {
|
||||
indexType IndexType
|
||||
dataTypes []schemapb.DataType
|
||||
wants []bool
|
||||
}
|
||||
|
||||
tests := []testCase{
|
||||
{
|
||||
indexType: "FLAT",
|
||||
dataTypes: []schemapb.DataType{
|
||||
schemapb.DataType_FloatVector,
|
||||
schemapb.DataType_BinaryVector,
|
||||
schemapb.DataType_BFloat16Vector,
|
||||
schemapb.DataType_Float16Vector,
|
||||
schemapb.DataType_SparseFloatVector,
|
||||
},
|
||||
wants: []bool{true, false, true, true, false},
|
||||
},
|
||||
{
|
||||
indexType: "BIN_FLAT",
|
||||
dataTypes: []schemapb.DataType{
|
||||
schemapb.DataType_FloatVector,
|
||||
schemapb.DataType_BinaryVector,
|
||||
schemapb.DataType_BFloat16Vector,
|
||||
schemapb.DataType_Float16Vector,
|
||||
schemapb.DataType_SparseFloatVector,
|
||||
},
|
||||
wants: []bool{false, true, false, false, false},
|
||||
},
|
||||
{
|
||||
indexType: "IVF_FLAT",
|
||||
dataTypes: []schemapb.DataType{
|
||||
schemapb.DataType_FloatVector,
|
||||
schemapb.DataType_BinaryVector,
|
||||
schemapb.DataType_BFloat16Vector,
|
||||
schemapb.DataType_Float16Vector,
|
||||
schemapb.DataType_SparseFloatVector,
|
||||
},
|
||||
wants: []bool{true, false, true, true, false},
|
||||
},
|
||||
{
|
||||
indexType: "IVF_PQ",
|
||||
dataTypes: []schemapb.DataType{
|
||||
schemapb.DataType_FloatVector,
|
||||
schemapb.DataType_BinaryVector,
|
||||
schemapb.DataType_BFloat16Vector,
|
||||
schemapb.DataType_Float16Vector,
|
||||
schemapb.DataType_SparseFloatVector,
|
||||
},
|
||||
wants: []bool{true, false, true, true, false},
|
||||
},
|
||||
{
|
||||
indexType: "HNSW",
|
||||
dataTypes: []schemapb.DataType{
|
||||
schemapb.DataType_FloatVector,
|
||||
schemapb.DataType_BinaryVector,
|
||||
schemapb.DataType_BFloat16Vector,
|
||||
schemapb.DataType_Float16Vector,
|
||||
schemapb.DataType_SparseFloatVector,
|
||||
},
|
||||
wants: []bool{true, true, true, true, false},
|
||||
},
|
||||
{
|
||||
indexType: "DISKANN",
|
||||
dataTypes: []schemapb.DataType{
|
||||
schemapb.DataType_FloatVector,
|
||||
schemapb.DataType_BinaryVector,
|
||||
schemapb.DataType_BFloat16Vector,
|
||||
schemapb.DataType_Float16Vector,
|
||||
schemapb.DataType_SparseFloatVector,
|
||||
},
|
||||
wants: []bool{true, false, true, true, false},
|
||||
},
|
||||
{
|
||||
indexType: "UNKNOWN",
|
||||
dataTypes: []schemapb.DataType{
|
||||
schemapb.DataType_FloatVector,
|
||||
schemapb.DataType_BinaryVector,
|
||||
schemapb.DataType_BFloat16Vector,
|
||||
schemapb.DataType_Float16Vector,
|
||||
schemapb.DataType_SparseFloatVector,
|
||||
},
|
||||
wants: []bool{false, false, false, false, false},
|
||||
},
|
||||
}
|
||||
|
||||
mgr := GetVecIndexMgrInstance()
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(string(tt.indexType), func(t *testing.T) {
|
||||
for i, dataType := range tt.dataTypes {
|
||||
got := mgr.IsDataTypeSupport(tt.indexType, dataType)
|
||||
if got != tt.wants[i] {
|
||||
t.Errorf("IsDataTypeSupport(%v, %v) = %v, want %v", tt.indexType, dataType, got, tt.wants[i])
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func Test_VecIndex_IsNoTrainIndex(t *testing.T) {
|
||||
mgr := GetVecIndexMgrInstance()
|
||||
tests := []struct {
|
||||
indexType IndexType
|
||||
want bool
|
||||
}{
|
||||
{
|
||||
indexType: "FLAT",
|
||||
want: true,
|
||||
},
|
||||
{
|
||||
indexType: "BIN_FLAT",
|
||||
want: true,
|
||||
},
|
||||
{
|
||||
indexType: "IVF_FLAT",
|
||||
want: false,
|
||||
},
|
||||
{
|
||||
indexType: "IVF_SQ8",
|
||||
want: false,
|
||||
},
|
||||
{
|
||||
indexType: "IVF_PQ",
|
||||
want: false,
|
||||
},
|
||||
{
|
||||
indexType: "HNSW",
|
||||
want: false,
|
||||
},
|
||||
{
|
||||
indexType: "DISKANN",
|
||||
want: false,
|
||||
},
|
||||
{
|
||||
indexType: "UNKNOWN",
|
||||
want: false,
|
||||
},
|
||||
}
|
||||
|
||||
for _, test := range tests {
|
||||
got := mgr.IsNoTrainIndex(test.indexType)
|
||||
if got != test.want {
|
||||
t.Errorf("IsNoTrainIndex(%v) = %v, 期望 %v", test.indexType, got, test.want)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func Test_VecIndex_IsDiskVecIndex(t *testing.T) {
|
||||
mgr := GetVecIndexMgrInstance()
|
||||
tests := []struct {
|
||||
indexType IndexType
|
||||
want bool
|
||||
}{
|
||||
{
|
||||
indexType: "FLAT",
|
||||
want: false,
|
||||
},
|
||||
{
|
||||
indexType: "BIN_FLAT",
|
||||
want: false,
|
||||
},
|
||||
{
|
||||
indexType: "IVF_FLAT",
|
||||
want: false,
|
||||
},
|
||||
{
|
||||
indexType: "DISKANN",
|
||||
want: true,
|
||||
},
|
||||
{
|
||||
indexType: "HNSW",
|
||||
want: false,
|
||||
},
|
||||
{
|
||||
indexType: "UNKNOWN",
|
||||
want: false,
|
||||
},
|
||||
}
|
||||
|
||||
for _, test := range tests {
|
||||
got := mgr.IsDiskVecIndex(test.indexType)
|
||||
if got != test.want {
|
||||
t.Errorf("IsDiskVecIndex(%v) = %v, want %v", test.indexType, got, test.want)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func Test_VecIndex_IsMvSupported(t *testing.T) {
|
||||
mgr := GetVecIndexMgrInstance()
|
||||
tests := []struct {
|
||||
indexType IndexType
|
||||
want bool
|
||||
}{
|
||||
{
|
||||
indexType: "FLAT",
|
||||
want: false,
|
||||
},
|
||||
{
|
||||
indexType: "IVF_FLAT",
|
||||
want: false,
|
||||
},
|
||||
{
|
||||
indexType: "DISKANN",
|
||||
want: false,
|
||||
},
|
||||
{
|
||||
indexType: "HNSW",
|
||||
want: true,
|
||||
},
|
||||
{
|
||||
indexType: "UNKNOWN",
|
||||
want: false,
|
||||
},
|
||||
}
|
||||
|
||||
for _, test := range tests {
|
||||
got := mgr.IsMvSupported(test.indexType)
|
||||
if got != test.want {
|
||||
t.Errorf("IsMvSupported(%v) = %v, want %v", test.indexType, got, test.want)
|
||||
}
|
||||
}
|
||||
}
|
|
@ -34,7 +34,7 @@ func TestIsScalarMmapIndex(t *testing.T) {
|
|||
|
||||
func TestIsVectorMmapIndex(t *testing.T) {
|
||||
t.Run("vector index", func(t *testing.T) {
|
||||
assert.True(t, IsVectorMmapIndex(IndexFaissIDMap))
|
||||
assert.True(t, IsVectorMmapIndex("FLAT"))
|
||||
assert.False(t, IsVectorMmapIndex(IndexINVERTED))
|
||||
})
|
||||
}
|
||||
|
@ -60,7 +60,7 @@ func TestValidateMmapTypeParams(t *testing.T) {
|
|||
})
|
||||
|
||||
t.Run("invalid mmap enable type", func(t *testing.T) {
|
||||
err := ValidateMmapIndexParams(IndexGpuBF, map[string]string{
|
||||
err := ValidateMmapIndexParams("GPU_BRUTE_FORCE", map[string]string{
|
||||
common.MmapEnabledKey: "true",
|
||||
})
|
||||
assert.Error(t, err)
|
||||
|
|
Loading…
Reference in New Issue