Replace faiss::MetricType with knowhere::MetricType (#17891)

Signed-off-by: yudong.cai <yudong.cai@zilliz.com>
This commit is contained in:
Cai Yudong 2022-06-29 14:20:19 +08:00 committed by GitHub
parent 63b4c8621b
commit a001412e12
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
39 changed files with 147 additions and 268 deletions

View File

@ -13,7 +13,6 @@ milvus_add_pkg_config("milvus_common")
set(COMMON_SRC
Schema.cpp
Types.cpp
SystemProperty.cpp
vector_index_c.cpp
memory_c.cpp

View File

@ -146,7 +146,8 @@ class FieldMeta {
Assert(is_string());
}
FieldMeta(const FieldName& name, FieldId id, DataType type, int64_t dim, std::optional<MetricType> metric_type)
FieldMeta(
const FieldName& name, FieldId id, DataType type, int64_t dim, std::optional<knowhere::MetricType> metric_type)
: name_(name), id_(id), type_(type), vector_info_(VectorInfo{dim, metric_type}) {
Assert(is_vector());
}
@ -177,7 +178,7 @@ class FieldMeta {
return string_info_->max_length;
}
std::optional<MetricType>
std::optional<knowhere::MetricType>
get_metric_type() const {
Assert(is_vector());
Assert(vector_info_.has_value());
@ -213,7 +214,7 @@ class FieldMeta {
private:
struct VectorInfo {
int64_t dim_;
std::optional<MetricType> metric_type_;
std::optional<knowhere::MetricType> metric_type_;
};
struct StringInfo {
int64_t max_length;

View File

@ -65,7 +65,7 @@ Schema::ParseFrom(const milvus::proto::schema::CollectionSchema& schema_proto) {
if (!index_map.count("metric_type")) {
schema->AddField(name, field_id, data_type, dim, std::nullopt);
} else {
auto metric_type = GetMetricType(index_map.at("metric_type"));
auto metric_type = index_map.at("metric_type");
schema->AddField(name, field_id, data_type, dim, metric_type);
}
} else if (datatype_is_string(data_type)) {

View File

@ -43,7 +43,10 @@ class Schema {
// auto gen field_id for convenience
FieldId
AddDebugField(const std::string& name, DataType data_type, int64_t dim, std::optional<MetricType> metric_type) {
AddDebugField(const std::string& name,
DataType data_type,
int64_t dim,
std::optional<knowhere::MetricType> metric_type) {
auto field_id = FieldId(debug_id);
debug_id++;
auto field_meta = FieldMeta(FieldName(name), field_id, data_type, dim, metric_type);
@ -71,7 +74,7 @@ class Schema {
const FieldId id,
DataType data_type,
int64_t dim,
std::optional<MetricType> metric_type) {
std::optional<knowhere::MetricType> metric_type) {
auto field_meta = FieldMeta(name, id, data_type, dim, metric_type);
this->AddField(std::move(field_meta));
}

View File

@ -1,64 +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.
#include "common/Types.h"
#include <knowhere/index/vector_index/helpers/IndexParameter.h>
#include "exceptions/EasyAssert.h"
#include <boost/bimap.hpp>
#include <boost/algorithm/string/case_conv.hpp>
#include "common/type_c.h"
#include "pb/schema.pb.h"
#include "CGoHelper.h"
#include "common/Consts.h"
namespace milvus {
using boost::algorithm::to_upper_copy;
namespace metric = knowhere::metric;
static const auto metric_bimap = [] {
boost::bimap<std::string, MetricType> mapping;
using pos = boost::bimap<std::string, MetricType>::value_type;
mapping.insert(pos(std::string(metric::L2), MetricType::METRIC_L2));
mapping.insert(pos(std::string(metric::IP), MetricType::METRIC_INNER_PRODUCT));
mapping.insert(pos(std::string(metric::JACCARD), MetricType::METRIC_Jaccard));
mapping.insert(pos(std::string(metric::TANIMOTO), MetricType::METRIC_Tanimoto));
mapping.insert(pos(std::string(metric::HAMMING), MetricType::METRIC_Hamming));
mapping.insert(pos(std::string(metric::SUBSTRUCTURE), MetricType::METRIC_Substructure));
mapping.insert(pos(std::string(metric::SUPERSTRUCTURE), MetricType::METRIC_Superstructure));
return mapping;
}();
MetricType
GetMetricType(const std::string& type_name) {
// Assume Metric is all upper at Knowhere
auto real_name = to_upper_copy(type_name);
AssertInfo(metric_bimap.left.count(real_name), "metric type not found: (" + type_name + ")");
return metric_bimap.left.at(real_name);
}
std::string
MetricTypeToName(MetricType metric_type) {
AssertInfo(metric_bimap.right.count(metric_type),
"metric_type enum(" + std::to_string((int)metric_type) + ") not found");
return metric_bimap.right.at(metric_type);
}
bool
IsPrimaryKeyDataType(DataType data_type) {
return data_type == DataType::INT64 || data_type == DataType::VARCHAR;
}
} // namespace milvus

View File

@ -28,7 +28,7 @@
#include <NamedType/named_type.hpp>
#include <variant>
#include "knowhere/common/MetricType.h"
#include "knowhere/index/vector_index/helpers/IndexParameter.h"
#include "pb/schema.pb.h"
#include "pb/segcore.pb.h"
#include "pb/plan.pb.h"
@ -68,19 +68,14 @@ using ScalarArray = proto::schema::ScalarField;
using DataArray = proto::schema::FieldData;
using VectorArray = proto::schema::VectorField;
using IdArray = proto::schema::IDs;
using MetricType = faiss::MetricType;
using InsertData = proto::segcore::InsertRecord;
using PkType = std::variant<std::monostate, int64_t, std::string>;
using Pk2OffsetType = tbb::concurrent_unordered_multimap<PkType, int64_t, std::hash<PkType>>;
MetricType
GetMetricType(const std::string& type);
std::string
MetricTypeToName(MetricType metric_type);
bool
IsPrimaryKeyDataType(DataType data_type);
inline bool
IsPrimaryKeyDataType(DataType data_type) {
return data_type == DataType::INT64 || data_type == DataType::VARCHAR;
}
// NOTE: dependent type
// used at meta-template programming

View File

@ -139,7 +139,7 @@ VecIndexCreator::check_parameter(knowhere::Config& conf,
template <typename T>
std::optional<T>
VecIndexCreator::get_config_by_name(std::string_view name) {
VecIndexCreator::get_config_by_name(const std::string& name) {
if (config_.contains(name)) {
return knowhere::GetValueFromConfig<T>(config_, name);
}

View File

@ -74,7 +74,7 @@ class VecIndexCreator : public IndexCreatorBase {
template <typename T>
std::optional<T>
get_config_by_name(std::string_view name);
get_config_by_name(const std::string& name);
void
StoreRawData(const knowhere::DatasetPtr& dataset);

View File

@ -200,7 +200,7 @@ Parser::ParseVecNode(const Json& out_body) {
}
}();
vec_node->search_info_.topk_ = topk;
vec_node->search_info_.metric_type_ = GetMetricType(vec_info.at("metric_type"));
vec_node->search_info_.metric_type_ = vec_info.at("metric_type");
vec_node->search_info_.search_params_ = vec_info.at("params");
vec_node->search_info_.field_id_ = field_id;
vec_node->search_info_.round_decimal_ = vec_info.at("round_decimal");

View File

@ -38,7 +38,7 @@ struct SearchInfo {
int64_t topk_;
int64_t round_decimal_;
FieldId field_id_;
MetricType metric_type_;
knowhere::MetricType metric_type_;
knowhere::Config search_params_;
};

View File

@ -145,7 +145,7 @@ ProtoParser::PlanNodeFromProto(const planpb::PlanNode& plan_node_proto) {
auto field_id = FieldId(anns_proto.field_id());
search_info.field_id_ = field_id;
search_info.metric_type_ = GetMetricType(query_info_proto.metric_type());
search_info.metric_type_ = query_info_proto.metric_type();
search_info.topk_ = query_info_proto.topk();
search_info.round_decimal_ = query_info_proto.round_decimal();
search_info.search_params_ = json::parse(query_info_proto.search_params());

View File

@ -25,7 +25,7 @@ namespace milvus::query {
// copy from faiss/IndexBinaryFlat.cpp::IndexBinaryFlat::search()
// disable lint to make further migration easier
static void
binary_search(MetricType metric_type,
binary_search(const knowhere::MetricType& metric_type,
const uint8_t* xb,
int64_t ntotal,
int code_size,
@ -36,28 +36,28 @@ binary_search(MetricType metric_type,
idx_t* labels,
const BitsetView bitset) {
using namespace faiss; // NOLINT
if (metric_type == METRIC_Jaccard || metric_type == METRIC_Tanimoto) {
if (metric_type == knowhere::metric::JACCARD || metric_type == knowhere::metric::TANIMOTO) {
float_maxheap_array_t res = {size_t(n), size_t(k), labels, D};
binary_distance_knn_hc(METRIC_Jaccard, &res, x, xb, ntotal, code_size, bitset);
if (metric_type == METRIC_Tanimoto) {
if (metric_type == knowhere::metric::TANIMOTO) {
for (int i = 0; i < k * n; i++) {
D[i] = Jaccard_2_Tanimoto(D[i]);
}
}
} else if (metric_type == METRIC_Hamming) {
} else if (metric_type == knowhere::metric::HAMMING) {
std::vector<int32_t> int_distances(n * k);
int_maxheap_array_t res = {size_t(n), size_t(k), labels, int_distances.data()};
binary_distance_knn_hc(METRIC_Hamming, &res, x, xb, ntotal, code_size, bitset);
for (int i = 0; i < n * k; ++i) {
D[i] = int_distances[i];
}
} else if (metric_type == METRIC_Substructure || metric_type == METRIC_Superstructure) {
} else if (metric_type == knowhere::metric::SUBSTRUCTURE || metric_type == knowhere::metric::SUPERSTRUCTURE) {
// only matched ids will be chosen, not to use heap
binary_distance_knn_mc(metric_type, x, xb, n, ntotal, k, code_size, D, labels, bitset);
auto faiss_metric_type = knowhere::GetFaissMetricType(metric_type);
binary_distance_knn_mc(faiss_metric_type, x, xb, n, ntotal, k, code_size, D, labels, bitset);
} else {
std::string msg =
std::string("binary search not support metric type: ") + segcore::MetricTypeToString(metric_type);
std::string msg = "binary search not support metric type: " + metric_type;
PanicInfo(msg);
}
}
@ -97,7 +97,7 @@ FloatSearchBruteForce(const dataset::SearchDataset& dataset,
SubSearchResult sub_qr(num_queries, topk, metric_type, round_decimal);
auto query_data = reinterpret_cast<const float*>(dataset.query_data);
auto chunk_data = reinterpret_cast<const float*>(chunk_data_raw);
if (metric_type == MetricType::METRIC_L2) {
if (metric_type == knowhere::metric::L2) {
faiss::float_maxheap_array_t buf{(size_t)num_queries, (size_t)topk, sub_qr.get_seg_offsets(),
sub_qr.get_distances()};
faiss::knn_L2sqr(query_data, chunk_data, dim, num_queries, size_per_chunk, &buf, nullptr, bitset);

View File

@ -47,7 +47,7 @@ SearchOnSealed(const Schema& schema,
auto conf = search_info.search_params_;
knowhere::SetMetaTopk(conf, search_info.topk_);
knowhere::SetMetaMetricType(conf, MetricTypeToName(field_indexing->metric_type_));
knowhere::SetMetaMetricType(conf, field_indexing->metric_type_);
auto index_type = field_indexing->indexing_->index_type();
auto adapter = knowhere::AdapterMgr::GetInstance().GetAdapter(index_type);
try {

View File

@ -19,7 +19,7 @@ namespace milvus::query {
class SubSearchResult {
public:
SubSearchResult(int64_t num_queries, int64_t topk, MetricType metric_type, int64_t round_decimal)
SubSearchResult(int64_t num_queries, int64_t topk, const knowhere::MetricType& metric_type, int64_t round_decimal)
: metric_type_(metric_type),
num_queries_(num_queries),
topk_(topk),
@ -29,15 +29,15 @@ class SubSearchResult {
}
public:
static constexpr float
init_value(MetricType metric_type) {
static float
init_value(const knowhere::MetricType& metric_type) {
return (is_descending(metric_type) ? -1 : 1) * std::numeric_limits<float>::max();
}
static constexpr bool
is_descending(MetricType metric_type) {
static bool
is_descending(const knowhere::MetricType& metric_type) {
// TODO(dog): more types
if (metric_type == MetricType::METRIC_INNER_PRODUCT) {
if (metric_type == knowhere::metric::IP) {
return true;
} else {
return false;
@ -103,7 +103,7 @@ class SubSearchResult {
int64_t num_queries_;
int64_t topk_;
int64_t round_decimal_;
MetricType metric_type_;
knowhere::MetricType metric_type_;
std::vector<int64_t> seg_offsets_;
std::vector<float> distances_;
};

View File

@ -21,7 +21,7 @@ namespace milvus::query {
namespace dataset {
struct SearchDataset {
MetricType metric_type;
knowhere::MetricType metric_type;
int64_t num_queries;
int64_t topk;
int64_t round_decimal;

View File

@ -53,12 +53,12 @@ ShowPlanNodeVisitor::visit(FloatVectorANNS& node) {
assert(!ret_);
auto& info = node.search_info_;
Json json_body{
{"node_type", "FloatVectorANNS"}, //
{"metric_type", MetricTypeToName(info.metric_type_)}, //
{"field_id_", info.field_id_.get()}, //
{"topk", info.topk_}, //
{"search_params", info.search_params_}, //
{"placeholder_tag", node.placeholder_tag_}, //
{"node_type", "FloatVectorANNS"}, //
{"metric_type", info.metric_type_}, //
{"field_id_", info.field_id_.get()}, //
{"topk", info.topk_}, //
{"search_params", info.search_params_}, //
{"placeholder_tag", node.placeholder_tag_}, //
};
if (node.predicate_.has_value()) {
ShowExprVisitor expr_show;
@ -75,12 +75,12 @@ ShowPlanNodeVisitor::visit(BinaryVectorANNS& node) {
assert(!ret_);
auto& info = node.search_info_;
Json json_body{
{"node_type", "BinaryVectorANNS"}, //
{"metric_type", MetricTypeToName(info.metric_type_)}, //
{"field_id_", info.field_id_.get()}, //
{"topk", info.topk_}, //
{"search_params", info.search_params_}, //
{"placeholder_tag", node.placeholder_tag_}, //
{"node_type", "BinaryVectorANNS"}, //
{"metric_type", info.metric_type_}, //
{"field_id_", info.field_id_.get()}, //
{"topk", info.topk_}, //
{"search_params", info.search_params_}, //
{"placeholder_tag", node.placeholder_tag_}, //
};
if (node.predicate_.has_value()) {
ShowExprVisitor expr_show;

View File

@ -48,14 +48,13 @@ VectorFieldIndexing::get_build_params() const {
// TODO
auto type_opt = field_meta_.get_metric_type();
AssertInfo(type_opt.has_value(), "Metric type of field meta doesn't have value");
auto metric_type = type_opt.value();
auto type_name = MetricTypeToName(metric_type);
auto& metric_type = type_opt.value();
auto& config = segcore_config_.at(metric_type);
auto base_params = config.build_params;
AssertInfo(base_params.count("nlist"), "Can't get nlist from index params");
knowhere::SetMetaDim(base_params, field_meta_.get_dim());
knowhere::SetMetaMetricType(base_params, type_name);
knowhere::SetMetaMetricType(base_params, metric_type);
return base_params;
}
@ -65,14 +64,13 @@ VectorFieldIndexing::get_search_params(int top_K) const {
// TODO
auto type_opt = field_meta_.get_metric_type();
AssertInfo(type_opt.has_value(), "Metric type of field meta doesn't have value");
auto metric_type = type_opt.value();
auto type_name = MetricTypeToName(metric_type);
auto& metric_type = type_opt.value();
auto& config = segcore_config_.at(metric_type);
auto base_params = config.search_params;
AssertInfo(base_params.count("nprobe"), "Can't get nprobe from base params");
knowhere::SetMetaTopk(base_params, top_K);
knowhere::SetMetaMetricType(base_params, type_name);
knowhere::SetMetaMetricType(base_params, metric_type);
return base_params;
}

View File

@ -25,7 +25,7 @@
namespace milvus::segcore {
struct SealedIndexingEntry {
MetricType metric_type_;
knowhere::MetricType metric_type_;
knowhere::VecIndexPtr indexing_;
};
@ -33,7 +33,7 @@ using SealedIndexingEntryPtr = std::unique_ptr<SealedIndexingEntry>;
struct SealedIndexingRecord {
void
append_field_indexing(FieldId field_id, MetricType metric_type, knowhere::VecIndexPtr indexing) {
append_field_indexing(FieldId field_id, const knowhere::MetricType& metric_type, knowhere::VecIndexPtr indexing) {
auto ptr = std::make_unique<SealedIndexingEntry>();
ptr->indexing_ = indexing;
ptr->metric_type_ = metric_type;

View File

@ -34,8 +34,8 @@ class SegcoreConfig {
sub_conf.build_params["nlist"] = nlist_;
sub_conf.search_params["nprobe"] = nprobe_;
sub_conf.index_type = "IVF";
table_[MetricType::METRIC_L2] = sub_conf;
table_[MetricType::METRIC_INNER_PRODUCT] = sub_conf;
table_[knowhere::metric::L2] = sub_conf;
table_[knowhere::metric::IP] = sub_conf;
}
public:
@ -50,7 +50,7 @@ class SegcoreConfig {
parse_from(const std::string& string_path);
const SmallIndexConf&
at(MetricType metric_type) const {
at(const knowhere::MetricType& metric_type) const {
Assert(table_.count(metric_type));
return table_.at(metric_type);
}
@ -76,7 +76,7 @@ class SegcoreConfig {
}
void
set_small_index_config(MetricType metric_type, const SmallIndexConf& small_index_conf) {
set_small_index_config(const knowhere::MetricType& metric_type, const SmallIndexConf& small_index_conf) {
table_[metric_type] = small_index_conf;
}
@ -84,7 +84,7 @@ class SegcoreConfig {
int64_t chunk_rows_ = 32 * 1024;
int64_t nlist_ = 100;
int64_t nprobe_ = 4;
std::map<MetricType, SmallIndexConf> table_;
std::map<knowhere::MetricType, SmallIndexConf> table_;
};
} // namespace milvus::segcore

View File

@ -84,7 +84,7 @@ SegmentSealedImpl::LoadVecIndex(const LoadIndexInfo& info) {
auto index = std::dynamic_pointer_cast<knowhere::VecIndex>(info.index);
AssertInfo(info.index_params.count("metric_type"), "Can't get metric_type in index_params");
auto metric_type_str = info.index_params.at("metric_type");
auto metric_type = info.index_params.at("metric_type");
auto row_count = index->Count();
AssertInfo(row_count > 0, "Index count is 0");
@ -101,7 +101,7 @@ SegmentSealedImpl::LoadVecIndex(const LoadIndexInfo& info) {
std::to_string(row_count_opt_.value()) + ")");
}
AssertInfo(!vector_indexings_.is_ready(field_id), "vec index is not ready");
vector_indexings_.append_field_indexing(field_id, GetMetricType(metric_type_str), index);
vector_indexings_.append_field_indexing(field_id, metric_type, index);
set_bit(index_ready_bitset_, field_id, true);
update_row_count(row_count);

View File

@ -11,11 +11,11 @@
#pragma once
#include <knowhere/common/MetricType.h>
#include <knowhere/index/vector_index/helpers/IndexParameter.h>
namespace milvus::segcore {
static inline bool
PositivelyRelated(faiss::MetricType metric_type) {
return metric_type == faiss::MetricType::METRIC_INNER_PRODUCT;
PositivelyRelated(const knowhere::MetricType& metric_type) {
return metric_type == knowhere::metric::IP;
}
} // namespace milvus::segcore

View File

@ -9,55 +9,21 @@
// 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 <stdlib.h>
#include <string>
#include <exception>
#include <memory>
#include <stdexcept>
#include <stdlib.h>
#include <string>
#include <utility>
#include <vector>
#include <stdexcept>
#include <knowhere/common/MetricType.h>
#include "knowhere/index/Index.h"
#include "common/QueryResult.h"
#include "knowhere/index/Index.h"
#include "segcore/DeletedRecord.h"
#include "segcore/InsertRecord.h"
namespace milvus::segcore {
static inline constexpr const char*
MetricTypeToString(faiss::MetricType metric_type) {
switch (metric_type) {
case faiss::MetricType::METRIC_INNER_PRODUCT:
return "METRIC_INNER_PRODUCT";
case faiss::MetricType::METRIC_L2:
return "METRIC_L2";
case faiss::MetricType::METRIC_L1:
return "METRIC_L1";
case faiss::MetricType::METRIC_Linf:
return "METRIC_Linf";
case faiss::MetricType::METRIC_Lp:
return "METRIC_Lp";
case faiss::MetricType::METRIC_Jaccard:
return "METRIC_Jaccard";
case faiss::MetricType::METRIC_Tanimoto:
return "METRIC_Tanimoto";
case faiss::MetricType::METRIC_Hamming:
return "METRIC_Hamming";
case faiss::MetricType::METRIC_Substructure:
return "METRIC_Substructure";
case faiss::MetricType::METRIC_Superstructure:
return "METRIC_Superstructure";
case faiss::MetricType::METRIC_Canberra:
return "METRIC_Canberra";
case faiss::MetricType::METRIC_BrayCurtis:
return "METRIC_BrayCurtis";
case faiss::MetricType::METRIC_JensenShannon:
return "METRIC_JensenShannon";
default:
return "Unsupported";
}
}
void
ParsePksFromFieldData(std::vector<PkType>& pks, const DataArray& data);

View File

@ -112,7 +112,7 @@ GetTopK(CSearchPlan plan) {
const char*
GetMetricType(CSearchPlan plan) {
auto search_plan = static_cast<milvus::query::Plan*>(plan);
auto metric_str = milvus::MetricTypeToName(search_plan->plan_node_->search_info_.metric_type_);
auto& metric_str = search_plan->plan_node_->search_info_.metric_type_;
return strdup(metric_str.c_str());
}

View File

@ -11,8 +11,8 @@
# or implied. See the License for the specific language governing permissions and limitations under the License.
#-------------------------------------------------------------------------------
set( KNOWHERE_VERSION v1.1.12 )
set( KNOWHERE_SOURCE_MD5 "816dadc4d41fc9a4eab1fb9c9a7c2736")
set( KNOWHERE_VERSION v1.1.13 )
set( KNOWHERE_SOURCE_MD5 "5ea7ce8ae71b4aa496ee3c66ccf56d5a")
if ( DEFINED ENV{MILVUS_KNOWHERE_URL} )
set( KNOWHERE_SOURCE_URL "$ENV{MILVUS_KNOWHERE_URL}" )

View File

@ -24,7 +24,7 @@ static int dim = 768;
const auto schema = []() {
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, dim, MetricType::METRIC_L2);
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, dim, knowhere::metric::L2);
return schema;
}();
@ -108,7 +108,7 @@ Search_Sealed(benchmark::State& state) {
info.field_id = (*schema)[FieldName("fakevec")].get_id().get();
info.index_params["index_type"] = "IVF";
info.index_params["index_mode"] = "CPU";
info.index_params["metric_type"] = MetricTypeToName(MetricType::METRIC_L2);
info.index_params["metric_type"] = knowhere::metric::L2;
segment->LoadIndex(info);
}
Timestamp time = 10000000;

View File

@ -18,7 +18,7 @@ using namespace milvus::segcore;
TEST(Binary, Insert) {
int64_t N = 100000;
auto schema = std::make_shared<Schema>();
auto vec_fid = schema->AddDebugField("vecbin", DataType::VECTOR_BINARY, 128, MetricType::METRIC_Jaccard);
auto vec_fid = schema->AddDebugField("vecbin", DataType::VECTOR_BINARY, 128, knowhere::metric::JACCARD);
auto i64_fid = schema->AddDebugField("age", DataType::INT64);
schema->set_primary_field_id(i64_fid);
auto dataset = DataGen(schema, N, 10);

View File

@ -97,7 +97,7 @@ TEST(Expr, Range) {
}
})";
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
schema->AddDebugField("age", DataType::INT32);
auto plan = CreatePlan(*schema, dsl_string);
ShowPlanNodeVisitor shower;
@ -139,7 +139,7 @@ TEST(Expr, RangeBinary) {
}
})";
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_BINARY, 512, MetricType::METRIC_Jaccard);
schema->AddDebugField("fakevec", DataType::VECTOR_BINARY, 512, knowhere::metric::JACCARD);
schema->AddDebugField("age", DataType::INT32);
auto plan = CreatePlan(*schema, dsl_string);
ShowPlanNodeVisitor shower;
@ -181,7 +181,7 @@ TEST(Expr, InvalidRange) {
}
})";
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
schema->AddDebugField("age", DataType::INT32);
ASSERT_ANY_THROW(CreatePlan(*schema, dsl_string));
}
@ -219,7 +219,7 @@ TEST(Expr, InvalidDSL) {
})";
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
schema->AddDebugField("age", DataType::INT32);
ASSERT_ANY_THROW(CreatePlan(*schema, dsl_string));
}
@ -229,12 +229,13 @@ TEST(Expr, ShowExecutor) {
using namespace milvus::segcore;
auto node = std::make_unique<FloatVectorANNS>();
auto schema = std::make_shared<Schema>();
auto field_id = schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
auto metric_type = knowhere::metric::L2;
auto field_id = schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, metric_type);
int64_t num_queries = 100L;
auto raw_data = DataGen(schema, num_queries);
auto& info = node->search_info_;
info.metric_type_ = MetricType::METRIC_L2;
info.metric_type_ = metric_type;
info.topk_ = 20;
info.field_id_ = field_id;
node->predicate_ = std::nullopt;
@ -289,7 +290,7 @@ TEST(Expr, TestRange) {
}
})";
auto schema = std::make_shared<Schema>();
auto vec_fid = schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
auto vec_fid = schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
auto i64_fid = schema->AddDebugField("age", DataType::INT64);
schema->set_primary_field_id(i64_fid);
@ -372,7 +373,7 @@ TEST(Expr, TestTerm) {
}
})";
auto schema = std::make_shared<Schema>();
auto vec_fid = schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
auto vec_fid = schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
auto i64_fid = schema->AddDebugField("age", DataType::INT64);
schema->set_primary_field_id(i64_fid);
@ -473,7 +474,7 @@ TEST(Expr, TestSimpleDsl) {
}
auto schema = std::make_shared<Schema>();
auto vec_fid = schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
auto vec_fid = schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
auto i64_fid = schema->AddDebugField("age", DataType::INT64);
schema->set_primary_field_id(i64_fid);
@ -544,7 +545,7 @@ TEST(Expr, TestCompare) {
}
})";
auto schema = std::make_shared<Schema>();
auto vec_fid = schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
auto vec_fid = schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
auto i32_fid = schema->AddDebugField("age1", DataType::INT32);
auto i64_fid = schema->AddDebugField("age2", DataType::INT64);
schema->set_primary_field_id(i64_fid);
@ -621,7 +622,7 @@ TEST(Expr, TestCompareWithScalarIndex) {
>)";
auto schema = std::make_shared<Schema>();
auto vec_fid = schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
auto vec_fid = schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
auto i32_fid = schema->AddDebugField("age32", DataType::INT32);
auto i64_fid = schema->AddDebugField("age64", DataType::INT64);
schema->set_primary_field_id(i64_fid);
@ -711,7 +712,7 @@ TEST(Expr, TestCompareWithScalarIndexMaris) {
>)";
auto schema = std::make_shared<Schema>();
auto vec_fid = schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
auto vec_fid = schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
auto str1_fid = schema->AddDebugField("string1", DataType::VARCHAR);
auto str2_fid = schema->AddDebugField("string2", DataType::VARCHAR);
schema->set_primary_field_id(str1_fid);
@ -916,7 +917,7 @@ TEST(Expr, TestBinaryArithOpEvalRange) {
})";
auto schema = std::make_shared<Schema>();
auto vec_fid = schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
auto vec_fid = schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
auto i8_fid = schema->AddDebugField("age8", DataType::INT8);
auto i16_fid = schema->AddDebugField("age16", DataType::INT16);
auto i32_fid = schema->AddDebugField("age32", DataType::INT32);
@ -1106,7 +1107,7 @@ TEST(Expr, TestBinaryArithOpEvalRangeExceptions) {
})";
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
schema->AddDebugField("age", DataType::INT32);
schema->AddDebugField("FloatN", DataType::FLOAT);
schema->AddDebugField("BoolField", DataType::BOOL);
@ -1287,7 +1288,7 @@ TEST(Expr, TestBinaryArithOpEvalRangeWithScalarSortIndex) {
@@@@)";
auto schema = std::make_shared<Schema>();
auto vec_fid = schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
auto vec_fid = schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
auto i8_fid = schema->AddDebugField("age8", DataType::INT8);
auto i16_fid = schema->AddDebugField("age16", DataType::INT16);
auto i32_fid = schema->AddDebugField("age32", DataType::INT32);

View File

@ -307,17 +307,18 @@ TEST(Indexing, BinaryBruteForce) {
int64_t topk = 5;
int64_t round_decimal = 3;
int64_t dim = 8192;
auto metric_type = knowhere::metric::JACCARD;
auto result_count = topk * num_queries;
auto schema = std::make_shared<Schema>();
auto vec_fid = schema->AddDebugField("vecbin", DataType::VECTOR_BINARY, dim, MetricType::METRIC_Jaccard);
auto vec_fid = schema->AddDebugField("vecbin", DataType::VECTOR_BINARY, dim, metric_type);
auto i64_fid = schema->AddDebugField("age", DataType::INT64);
auto dataset = DataGen(schema, N, 10);
auto bin_vec = dataset.get_col<uint8_t>(vec_fid);
auto query_data = 1024 * dim / 8 + bin_vec.data();
query::dataset::SearchDataset search_dataset{
faiss::MetricType::METRIC_Jaccard, //
num_queries, //
topk, //
metric_type, //
num_queries, //
topk, //
round_decimal,
dim, //
query_data //

View File

@ -29,8 +29,8 @@ namespace spb = proto::schema;
static SchemaPtr
getStandardSchema() {
auto schema = std::make_shared<Schema>();
schema->AddDebugField("FloatVectorField", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddDebugField("BinaryVectorField", DataType::VECTOR_BINARY, 16, MetricType::METRIC_Jaccard);
schema->AddDebugField("FloatVectorField", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
schema->AddDebugField("BinaryVectorField", DataType::VECTOR_BINARY, 16, knowhere::metric::JACCARD);
schema->AddDebugField("Int64Field", DataType::INT64);
schema->AddDebugField("Int32Field", DataType::INT32);
schema->AddDebugField("Int16Field", DataType::INT16);

View File

@ -33,13 +33,14 @@ TEST(Query, ShowExecutor) {
using namespace milvus::query;
using namespace milvus::segcore;
using namespace milvus;
auto metric_type = knowhere::metric::L2;
auto node = std::make_unique<FloatVectorANNS>();
auto schema = std::make_shared<Schema>();
auto field_id = schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
auto field_id = schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, metric_type);
int64_t num_queries = 100L;
auto raw_data = DataGen(schema, num_queries);
auto& info = node->search_info_;
info.metric_type_ = MetricType::METRIC_L2;
info.metric_type_ = metric_type;
info.topk_ = 20;
info.field_id_ = field_id;
node->predicate_ = std::nullopt;
@ -77,7 +78,7 @@ TEST(Query, DSL) {
})";
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
auto plan = CreatePlan(*schema, dsl_string);
auto res = shower.call_child(*plan->plan_node_);
@ -124,7 +125,7 @@ TEST(Query, ParsePlaceholderGroup) {
})";
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
auto plan = CreatePlan(*schema, dsl_string);
int64_t num_queries = 100000;
int dim = 16;
@ -137,7 +138,7 @@ TEST(Query, ExecWithPredicateLoader) {
using namespace milvus::query;
using namespace milvus::segcore;
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
schema->AddDebugField("age", DataType::FLOAT);
auto counter_fid = schema->AddDebugField("counter", DataType::INT64);
schema->set_primary_field_id(counter_fid);
@ -215,7 +216,7 @@ TEST(Query, ExecWithPredicateSmallN) {
using namespace milvus::query;
using namespace milvus::segcore;
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 7, MetricType::METRIC_L2);
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 7, knowhere::metric::L2);
schema->AddDebugField("age", DataType::FLOAT);
auto i64_fid = schema->AddDebugField("counter", DataType::INT64);
schema->set_primary_field_id(i64_fid);
@ -269,7 +270,7 @@ TEST(Query, ExecWithPredicate) {
using namespace milvus::query;
using namespace milvus::segcore;
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
schema->AddDebugField("age", DataType::FLOAT);
auto i64_fid = schema->AddDebugField("counter", DataType::INT64);
schema->set_primary_field_id(i64_fid);
@ -347,7 +348,7 @@ TEST(Query, ExecTerm) {
using namespace milvus::query;
using namespace milvus::segcore;
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
schema->AddDebugField("age", DataType::FLOAT);
auto i64_fid = schema->AddDebugField("counter", DataType::INT64);
schema->set_primary_field_id(i64_fid);
@ -403,7 +404,7 @@ TEST(Query, ExecEmpty) {
using namespace milvus::segcore;
auto schema = std::make_shared<Schema>();
schema->AddDebugField("age", DataType::FLOAT);
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
std::string dsl = R"({
"bool": {
"must": [
@ -493,7 +494,7 @@ TEST(Query, ExecWithoutPredicate) {
using namespace milvus::query;
using namespace milvus::segcore;
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
schema->AddDebugField("age", DataType::FLOAT);
auto i64_fid = schema->AddDebugField("counter", DataType::INT64);
schema->set_primary_field_id(i64_fid);
@ -584,7 +585,7 @@ TEST(Indexing, InnerProduct) {
]
}
})";
auto vec_fid = schema->AddDebugField("normalized", DataType::VECTOR_FLOAT, dim, MetricType::METRIC_INNER_PRODUCT);
auto vec_fid = schema->AddDebugField("normalized", DataType::VECTOR_FLOAT, dim, knowhere::metric::IP);
auto i64_fid = schema->AddDebugField("age", DataType::INT64);
schema->set_primary_field_id(i64_fid);
auto dataset = DataGen(schema, N);
@ -755,7 +756,7 @@ TEST(Query, ExecWithPredicateBinary) {
using namespace milvus::query;
using namespace milvus::segcore;
auto schema = std::make_shared<Schema>();
auto vec_fid = schema->AddDebugField("fakevec", DataType::VECTOR_BINARY, 512, MetricType::METRIC_Jaccard);
auto vec_fid = schema->AddDebugField("fakevec", DataType::VECTOR_BINARY, 512, knowhere::metric::JACCARD);
auto float_fid = schema->AddDebugField("age", DataType::FLOAT);
auto i64_fid = schema->AddDebugField("counter", DataType::INT64);
schema->set_primary_field_id(i64_fid);
@ -773,7 +774,7 @@ TEST(Query, ExecWithPredicateBinary) {
{
"vector": {
"fakevec": {
"metric_type": "Jaccard",
"metric_type": "JACCARD",
"params": {
"nprobe": 10
},

View File

@ -25,7 +25,7 @@ TEST(Reduce, SubQueryResult) {
int64_t iteration = 50;
int64_t round_decimal = 3;
constexpr int64_t limit = 100000000L;
auto metric_type = MetricType::METRIC_L2;
auto metric_type = knowhere::metric::L2;
using queue_type = std::priority_queue<int64_t>;
std::vector<queue_type> ref_results(num_queries);
@ -77,7 +77,7 @@ TEST(Reduce, SubSearchResultDesc) {
int64_t round_decimal = 3;
constexpr int64_t limit = 100000000L;
constexpr int64_t init_value = 0;
auto metric_type = MetricType::METRIC_INNER_PRODUCT;
auto metric_type = knowhere::metric::IP;
using queue_type = std::priority_queue<int64_t, std::vector<int64_t>, std::greater<int64_t>>;
std::vector<queue_type> ref_results(num_queries);

View File

@ -48,7 +48,7 @@ TEST(Retrieve, AutoID) {
auto schema = std::make_shared<Schema>();
auto fid_64 = schema->AddDebugField("i64", DataType::INT64);
auto DIM = 16;
auto fid_vec = schema->AddDebugField("vector_64", DataType::VECTOR_FLOAT, DIM, MetricType::METRIC_L2);
auto fid_vec = schema->AddDebugField("vector_64", DataType::VECTOR_FLOAT, DIM, knowhere::metric::L2);
schema->set_primary_field_id(fid_64);
int64_t N = 100;
@ -98,7 +98,7 @@ TEST(Retrieve, AutoID2) {
auto schema = std::make_shared<Schema>();
auto fid_64 = schema->AddDebugField("i64", DataType::INT64);
auto DIM = 16;
auto fid_vec = schema->AddDebugField("vector_64", DataType::VECTOR_FLOAT, DIM, MetricType::METRIC_L2);
auto fid_vec = schema->AddDebugField("vector_64", DataType::VECTOR_FLOAT, DIM, knowhere::metric::L2);
schema->set_primary_field_id(fid_64);
int64_t N = 100;
@ -143,7 +143,7 @@ TEST(Retrieve, NotExist) {
auto schema = std::make_shared<Schema>();
auto fid_64 = schema->AddDebugField("i64", DataType::INT64);
auto DIM = 16;
auto fid_vec = schema->AddDebugField("vector_64", DataType::VECTOR_FLOAT, DIM, MetricType::METRIC_L2);
auto fid_vec = schema->AddDebugField("vector_64", DataType::VECTOR_FLOAT, DIM, knowhere::metric::L2);
schema->set_primary_field_id(fid_64);
int64_t N = 100;
@ -191,7 +191,7 @@ TEST(Retrieve, Empty) {
auto schema = std::make_shared<Schema>();
auto fid_64 = schema->AddDebugField("i64", DataType::INT64);
auto DIM = 16;
auto fid_vec = schema->AddDebugField("vector_64", DataType::VECTOR_FLOAT, DIM, MetricType::METRIC_L2);
auto fid_vec = schema->AddDebugField("vector_64", DataType::VECTOR_FLOAT, DIM, knowhere::metric::L2);
schema->set_primary_field_id(fid_64);
int64_t N = 100;
@ -226,7 +226,7 @@ TEST(Retrieve, LargeTimestamp) {
auto schema = std::make_shared<Schema>();
auto fid_64 = schema->AddDebugField("i64", DataType::INT64);
auto DIM = 16;
auto fid_vec = schema->AddDebugField("vector_64", DataType::VECTOR_FLOAT, DIM, MetricType::METRIC_L2);
auto fid_vec = schema->AddDebugField("vector_64", DataType::VECTOR_FLOAT, DIM, knowhere::metric::L2);
schema->set_primary_field_id(fid_64);
int64_t N = 100;
@ -276,7 +276,7 @@ TEST(Retrieve, Delete) {
auto schema = std::make_shared<Schema>();
auto fid_64 = schema->AddDebugField("i64", DataType::INT64);
auto DIM = 16;
auto fid_vec = schema->AddDebugField("vector_64", DataType::VECTOR_FLOAT, DIM, MetricType::METRIC_L2);
auto fid_vec = schema->AddDebugField("vector_64", DataType::VECTOR_FLOAT, DIM, knowhere::metric::L2);
schema->set_primary_field_id(fid_64);
int64_t N = 10;

View File

@ -29,7 +29,7 @@ TEST(Sealed, without_predicate) {
auto schema = std::make_shared<Schema>();
auto dim = 16;
auto topK = 5;
auto metric_type = MetricType::METRIC_L2;
auto metric_type = knowhere::metric::L2;
auto fake_id = schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, dim, metric_type);
auto float_fid = schema->AddDebugField("age", DataType::FLOAT);
auto i64_fid = schema->AddDebugField("counter", DataType::INT64);
@ -133,7 +133,7 @@ TEST(Sealed, with_predicate) {
auto schema = std::make_shared<Schema>();
auto dim = 16;
auto topK = 5;
auto metric_type = MetricType::METRIC_L2;
auto metric_type = knowhere::metric::L2;
auto fake_id = schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, dim, metric_type);
auto i64_fid = schema->AddDebugField("counter", DataType::INT64);
schema->set_primary_field_id(i64_fid);
@ -228,7 +228,7 @@ TEST(Sealed, LoadFieldData) {
auto dim = 16;
auto topK = 5;
auto N = ROW_COUNT;
auto metric_type = MetricType::METRIC_L2;
auto metric_type = knowhere::metric::L2;
auto schema = std::make_shared<Schema>();
auto fakevec_id = schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, dim, metric_type);
auto counter_id = schema->AddDebugField("counter", DataType::INT64);
@ -349,7 +349,7 @@ TEST(Sealed, LoadFieldData) {
TEST(Sealed, LoadScalarIndex) {
auto dim = 16;
auto N = ROW_COUNT;
auto metric_type = MetricType::METRIC_L2;
auto metric_type = knowhere::metric::L2;
auto schema = std::make_shared<Schema>();
auto fakevec_id = schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, dim, metric_type);
auto counter_id = schema->AddDebugField("counter", DataType::INT64);
@ -454,7 +454,7 @@ TEST(Sealed, Delete) {
auto dim = 16;
auto topK = 5;
auto N = 10;
auto metric_type = MetricType::METRIC_L2;
auto metric_type = knowhere::metric::L2;
auto schema = std::make_shared<Schema>();
auto fakevec_id = schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, dim, metric_type);
auto counter_id = schema->AddDebugField("counter", DataType::INT64);

View File

@ -47,7 +47,7 @@ generate_data(int N) {
TEST(SegmentCoreTest, NormalDistributionTest) {
using namespace milvus::segcore;
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
schema->AddDebugField("age", DataType::INT32);
int N = 100 * 1000;
auto [raw_data, timestamps, uids] = generate_data(N);
@ -62,7 +62,7 @@ TEST(SegmentCoreTest, MockTest2) {
// schema
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
auto i64_fid = schema->AddDebugField("age", DataType::INT64);
schema->set_primary_field_id(i64_fid);
@ -76,6 +76,6 @@ TEST(SegmentCoreTest, MockTest2) {
TEST(SegmentCoreTest, SmallIndex) {
using namespace milvus::segcore;
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
schema->AddDebugField("age", DataType::INT32);
}

View File

@ -14,12 +14,12 @@
#include "segcore/SimilarityCorelation.h"
TEST(SimilarityCorelation, Naive) {
ASSERT_TRUE(milvus::segcore::PositivelyRelated(faiss::METRIC_INNER_PRODUCT));
ASSERT_TRUE(milvus::segcore::PositivelyRelated(knowhere::metric::IP));
ASSERT_FALSE(milvus::segcore::PositivelyRelated(faiss::METRIC_Jaccard));
ASSERT_FALSE(milvus::segcore::PositivelyRelated(faiss::METRIC_Tanimoto));
ASSERT_FALSE(milvus::segcore::PositivelyRelated(faiss::METRIC_L2));
ASSERT_FALSE(milvus::segcore::PositivelyRelated(faiss::METRIC_Hamming));
ASSERT_FALSE(milvus::segcore::PositivelyRelated(faiss::METRIC_Substructure));
ASSERT_FALSE(milvus::segcore::PositivelyRelated(faiss::METRIC_Superstructure));
ASSERT_FALSE(milvus::segcore::PositivelyRelated(knowhere::metric::L2));
ASSERT_FALSE(milvus::segcore::PositivelyRelated(knowhere::metric::HAMMING));
ASSERT_FALSE(milvus::segcore::PositivelyRelated(knowhere::metric::JACCARD));
ASSERT_FALSE(milvus::segcore::PositivelyRelated(knowhere::metric::TANIMOTO));
ASSERT_FALSE(milvus::segcore::PositivelyRelated(knowhere::metric::SUBSTRUCTURE));
ASSERT_FALSE(milvus::segcore::PositivelyRelated(knowhere::metric::SUPERSTRUCTURE));
}

View File

@ -23,9 +23,9 @@ TEST(Span, Naive) {
int64_t N = ROW_COUNT;
constexpr int64_t size_per_chunk = 32 * 1024;
auto schema = std::make_shared<Schema>();
auto bin_vec_fid = schema->AddDebugField("binaryvec", DataType::VECTOR_BINARY, 512, MetricType::METRIC_Jaccard);
auto bin_vec_fid = schema->AddDebugField("binaryvec", DataType::VECTOR_BINARY, 512, knowhere::metric::JACCARD);
auto float_fid = schema->AddDebugField("age", DataType::FLOAT);
auto float_vec_fid = schema->AddDebugField("floatvec", DataType::VECTOR_FLOAT, 32, MetricType::METRIC_L2);
auto float_vec_fid = schema->AddDebugField("floatvec", DataType::VECTOR_FLOAT, 32, knowhere::metric::L2);
auto i64_fid = schema->AddDebugField("counter", DataType::INT64);
schema->set_primary_field_id(i64_fid);

View File

@ -217,7 +217,7 @@ GenTestSchema() {
auto schema = std::make_shared<Schema>();
schema->AddDebugField("str", DataType::VARCHAR);
schema->AddDebugField("another_str", DataType::VARCHAR);
schema->AddDebugField("fvec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddDebugField("fvec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
auto pk = schema->AddDebugField("int64", DataType::INT64);
schema->set_primary_field_id(pk);
return schema;
@ -228,7 +228,7 @@ GenStrPKSchema() {
auto schema = std::make_shared<Schema>();
auto pk = schema->AddDebugField("str", DataType::VARCHAR);
schema->AddDebugField("another_str", DataType::VARCHAR);
schema->AddDebugField("fvec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddDebugField("fvec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
schema->AddDebugField("int64", DataType::INT64);
schema->set_primary_field_id(pk);
return schema;
@ -531,9 +531,9 @@ TEST(AlwaysTrueStringPlan, SearchWithOutputFields) {
std::vector<const PlaceholderGroup*> ph_group_arr = {ph_group.get()};
query::dataset::SearchDataset search_dataset{
faiss::MetricType::METRIC_L2, //
num_queries, //
topk, //
knowhere::metric::L2, //
num_queries, //
topk, //
round_decimal,
dim, //
query_ptr //

View File

@ -11,31 +11,11 @@
#include <gtest/gtest.h>
#include <string.h>
#include <knowhere/common/MetricType.h>
#include "common/Utils.h"
#include "query/Utils.h"
#include "test_utils/DataGen.h"
TEST(Util, FaissMetricTypeToString) {
using namespace milvus::segcore;
using namespace faiss;
ASSERT_EQ(MetricTypeToString(MetricType::METRIC_INNER_PRODUCT), "METRIC_INNER_PRODUCT");
ASSERT_EQ(MetricTypeToString(MetricType::METRIC_L2), "METRIC_L2");
ASSERT_EQ(MetricTypeToString(MetricType::METRIC_L1), "METRIC_L1");
ASSERT_EQ(MetricTypeToString(MetricType::METRIC_Linf), "METRIC_Linf");
ASSERT_EQ(MetricTypeToString(MetricType::METRIC_Lp), "METRIC_Lp");
ASSERT_EQ(MetricTypeToString(MetricType::METRIC_Jaccard), "METRIC_Jaccard");
ASSERT_EQ(MetricTypeToString(MetricType::METRIC_Tanimoto), "METRIC_Tanimoto");
ASSERT_EQ(MetricTypeToString(MetricType::METRIC_Hamming), "METRIC_Hamming");
ASSERT_EQ(MetricTypeToString(MetricType::METRIC_Substructure), "METRIC_Substructure");
ASSERT_EQ(MetricTypeToString(MetricType::METRIC_Superstructure), "METRIC_Superstructure");
ASSERT_EQ(MetricTypeToString(MetricType::METRIC_Canberra), "METRIC_Canberra");
ASSERT_EQ(MetricTypeToString(MetricType::METRIC_BrayCurtis), "METRIC_BrayCurtis");
ASSERT_EQ(MetricTypeToString(MetricType::METRIC_JensenShannon), "METRIC_JensenShannon");
}
TEST(Util, StringMatch) {
using namespace milvus;
using namespace milvus::query;
@ -61,7 +41,7 @@ TEST(Util, GetDeleteBitmap) {
using namespace milvus::segcore;
auto schema = std::make_shared<Schema>();
auto vec_fid = schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
auto vec_fid = schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, knowhere::metric::L2);
auto i64_fid = schema->AddDebugField("age", DataType::INT64);
schema->set_primary_field_id(i64_fid);
auto N = 10;

View File

@ -25,7 +25,6 @@
#include "indexbuilder/helper.h"
#include "indexbuilder/index_c.h"
#include "indexbuilder/utils.h"
#include "knowhere/common/MetricType.h"
#include "knowhere/index/VecIndexFactory.h"
#include "knowhere/index/vector_index/helpers/IndexParameter.h"
#include "knowhere/index/vector_index/adapter/VectorAdapter.h"
@ -245,12 +244,11 @@ generate_params(const knowhere::IndexType& index_type, const knowhere::MetricTyp
auto
GenDataset(int64_t N, const knowhere::MetricType& metric_type, bool is_binary, int64_t dim = DIM) {
auto schema = std::make_shared<milvus::Schema>();
auto faiss_metric_type = knowhere::GetMetricType(std::string(metric_type));
if (!is_binary) {
schema->AddDebugField("fakevec", milvus::DataType::VECTOR_FLOAT, dim, faiss_metric_type);
schema->AddDebugField("fakevec", milvus::DataType::VECTOR_FLOAT, dim, metric_type);
return milvus::segcore::DataGen(schema, N);
} else {
schema->AddDebugField("fakebinvec", milvus::DataType::VECTOR_BINARY, dim, faiss_metric_type);
schema->AddDebugField("fakebinvec", milvus::DataType::VECTOR_BINARY, dim, metric_type);
return milvus::segcore::DataGen(schema, N);
}
}