提交 ef98dab2 编写于 作者: F FluorineDog 提交者: yefu.chen

Support segcore config

Signed-off-by: NFluorineDog <guilin.gou@zilliz.com>
上级 22063c6d
......@@ -69,7 +69,9 @@ Search_SmallIndex(benchmark::State& state) {
auto is_small_index = state.range(0);
auto chunk_size = state.range(1) * 1024;
auto segment = CreateGrowingSegment(schema, chunk_size);
auto segconf = SegcoreConfig::default_config();
segconf.set_size_per_chunk(chunk_size);
auto segment = CreateGrowingSegment(schema, segconf);
if (!is_small_index) {
segment->debug_disable_small_index();
}
......
......@@ -7,4 +7,4 @@ set(COMMON_SRC
add_library(milvus_common
${COMMON_SRC}
)
target_link_libraries(milvus_common milvus_proto)
target_link_libraries(milvus_common milvus_proto yaml-cpp)
......@@ -11,6 +11,8 @@
#include "common/SystemProperty.h"
#include "utils/EasyAssert.h"
#include <yaml-cpp/yaml.h>
#include "common/Types.h"
namespace milvus {
class SystemPropertyImpl : public SystemProperty {
......@@ -50,19 +52,20 @@ class SystemPropertyImpl : public SystemProperty {
return name_to_types_.count(field_name);
}
friend const SystemProperty&
SystemProperty::Instance();
friend SystemPropertyImpl&
InstanceImpl();
private:
std::map<FieldName, SystemFieldType> name_to_types_;
std::map<FieldId, SystemFieldType> id_to_types_;
};
const SystemProperty&
SystemProperty::Instance() {
SystemPropertyImpl&
InstanceImpl() {
static auto impl = [] {
SystemPropertyImpl impl;
using Type = SystemFieldType;
impl.name_to_types_.emplace(FieldName("RowID"), Type::RowId);
impl.id_to_types_.emplace(FieldId(0), Type::RowId);
......@@ -73,4 +76,10 @@ SystemProperty::Instance() {
}();
return impl;
}
const SystemProperty&
SystemProperty::Instance() {
return InstanceImpl();
}
}; // namespace milvus
......@@ -11,6 +11,8 @@
#pragma once
#include "common/Types.h"
#include "utils/Json.h"
#include <string>
namespace milvus {
......@@ -22,8 +24,20 @@ enum class SystemFieldType {
class SystemProperty {
public:
SystemProperty() = default;
~SystemProperty() = default;
static const SystemProperty&
Instance();
SystemProperty(const SystemProperty&) = delete;
SystemProperty&
operator=(const SystemProperty&) = delete;
protected:
SystemProperty(SystemProperty&&) = default;
SystemProperty&
operator=(SystemProperty&&) = default;
public:
virtual bool
......@@ -42,4 +56,7 @@ class SystemProperty {
IsSystem(FieldName field_name) const = 0;
};
void
InitalizeSegcore(const std::string& segcore_yaml_path);
} // namespace milvus
......@@ -77,7 +77,7 @@ FloatSearch(const segcore::SegmentGrowingImpl& segment,
if (indexing_record.is_in(vecfield_offset)) {
auto max_indexed_id = indexing_record.get_finished_ack();
const auto& field_indexing = indexing_record.get_vec_field_indexing(vecfield_offset);
auto search_conf = field_indexing.get_search_conf(topK);
auto search_conf = field_indexing.get_search_params(topK);
Assert(vec_ptr->get_size_per_chunk() == field_indexing.get_size_per_chunk());
for (int chunk_id = current_chunk_id; chunk_id < max_indexed_id; ++chunk_id) {
......
......@@ -14,7 +14,7 @@ set(SEGCORE_FILES
load_index_c.cpp
SealedIndexingRecord.cpp
SegmentInterface.cpp
)
SegcoreConfig.h SegcoreConfig.cpp)
add_library(milvus_segcore SHARED
${SEGCORE_FILES}
)
......
......@@ -13,6 +13,8 @@
#include <thread>
#include <knowhere/index/vector_index/IndexIVF.h>
#include <knowhere/index/vector_index/adapter/VectorAdapter.h>
#include <string>
#include "common/SystemProperty.h"
namespace milvus::segcore {
void
......@@ -24,7 +26,7 @@ VectorFieldIndexing::BuildIndexRange(int64_t ack_beg, int64_t ack_end, const Vec
Assert(source);
auto num_chunk = source->num_chunk();
assert(ack_end <= num_chunk);
auto conf = get_build_conf();
auto conf = get_build_params();
data_.grow_to_at_least(ack_end);
for (int chunk_id = ack_beg; chunk_id < ack_end; chunk_id++) {
const auto& chunk = source->get_chunk(chunk_id);
......@@ -38,27 +40,37 @@ VectorFieldIndexing::BuildIndexRange(int64_t ack_beg, int64_t ack_end, const Vec
}
knowhere::Config
VectorFieldIndexing::get_build_conf() const {
VectorFieldIndexing::get_build_params() const {
// TODO
auto type_opt = field_meta_.get_metric_type();
Assert(type_opt.has_value());
auto type_name = MetricTypeToName(type_opt.value());
return knowhere::Config{{knowhere::meta::DIM, field_meta_.get_dim()},
{knowhere::IndexParams::nlist, 100},
{knowhere::IndexParams::nprobe, 4},
{knowhere::Metric::TYPE, type_name},
{knowhere::meta::DEVICEID, 0}};
auto metric_type = type_opt.value();
auto type_name = MetricTypeToName(metric_type);
auto& config = segcore_config_.at(metric_type);
auto base_params = config.build_params;
Assert(base_params.count("nlist"));
base_params[knowhere::meta::DIM] = field_meta_.get_dim();
base_params[knowhere::Metric::TYPE] = type_name;
return base_params;
}
knowhere::Config
VectorFieldIndexing::get_search_conf(int top_K) const {
VectorFieldIndexing::get_search_params(int top_K) const {
// TODO
auto type_opt = field_meta_.get_metric_type();
Assert(type_opt.has_value());
auto type_name = MetricTypeToName(type_opt.value());
return knowhere::Config{{knowhere::meta::DIM, field_meta_.get_dim()}, {knowhere::meta::TOPK, top_K},
{knowhere::IndexParams::nlist, 100}, {knowhere::IndexParams::nprobe, 4},
{knowhere::Metric::TYPE, type_name}, {knowhere::meta::DEVICEID, 0}};
auto metric_type = type_opt.value();
auto type_name = MetricTypeToName(metric_type);
auto& config = segcore_config_.at(metric_type);
auto base_params = config.search_params;
Assert(base_params.count("nprobe"));
base_params[knowhere::meta::TOPK] = top_K;
base_params[knowhere::Metric::TYPE] = type_name;
return base_params;
}
void
......@@ -103,10 +115,10 @@ ScalarFieldIndexing<T>::BuildIndexRange(int64_t ack_beg, int64_t ack_end, const
}
std::unique_ptr<FieldIndexing>
CreateIndex(const FieldMeta& field_meta, int64_t size_per_chunk) {
CreateIndex(const FieldMeta& field_meta, const SegcoreConfig& segcore_config) {
if (field_meta.is_vector()) {
if (field_meta.get_data_type() == DataType::VECTOR_FLOAT) {
return std::make_unique<VectorFieldIndexing>(field_meta, size_per_chunk);
return std::make_unique<VectorFieldIndexing>(field_meta, segcore_config);
} else {
// TODO
PanicInfo("unsupported");
......@@ -114,19 +126,19 @@ CreateIndex(const FieldMeta& field_meta, int64_t size_per_chunk) {
}
switch (field_meta.get_data_type()) {
case DataType::BOOL:
return std::make_unique<ScalarFieldIndexing<bool>>(field_meta, size_per_chunk);
return std::make_unique<ScalarFieldIndexing<bool>>(field_meta, segcore_config);
case DataType::INT8:
return std::make_unique<ScalarFieldIndexing<int8_t>>(field_meta, size_per_chunk);
return std::make_unique<ScalarFieldIndexing<int8_t>>(field_meta, segcore_config);
case DataType::INT16:
return std::make_unique<ScalarFieldIndexing<int16_t>>(field_meta, size_per_chunk);
return std::make_unique<ScalarFieldIndexing<int16_t>>(field_meta, segcore_config);
case DataType::INT32:
return std::make_unique<ScalarFieldIndexing<int32_t>>(field_meta, size_per_chunk);
return std::make_unique<ScalarFieldIndexing<int32_t>>(field_meta, segcore_config);
case DataType::INT64:
return std::make_unique<ScalarFieldIndexing<int64_t>>(field_meta, size_per_chunk);
return std::make_unique<ScalarFieldIndexing<int64_t>>(field_meta, segcore_config);
case DataType::FLOAT:
return std::make_unique<ScalarFieldIndexing<float>>(field_meta, size_per_chunk);
return std::make_unique<ScalarFieldIndexing<float>>(field_meta, segcore_config);
case DataType::DOUBLE:
return std::make_unique<ScalarFieldIndexing<double>>(field_meta, size_per_chunk);
return std::make_unique<ScalarFieldIndexing<double>>(field_meta, segcore_config);
default:
PanicInfo("unsupported");
}
......
......@@ -19,6 +19,7 @@
#include "InsertRecord.h"
#include <knowhere/index/vector_index/IndexIVF.h>
#include <knowhere/index/structured_index_simple/StructuredIndexSort.h>
#include "segcore/SegcoreConfig.h"
namespace milvus::segcore {
......@@ -26,8 +27,8 @@ namespace milvus::segcore {
// All concurrent
class FieldIndexing {
public:
explicit FieldIndexing(const FieldMeta& field_meta, int64_t size_per_chunk)
: field_meta_(field_meta), size_per_chunk_(size_per_chunk) {
explicit FieldIndexing(const FieldMeta& field_meta, const SegcoreConfig& segcore_config)
: field_meta_(field_meta), segcore_config_(segcore_config) {
}
FieldIndexing(const FieldIndexing&) = delete;
FieldIndexing&
......@@ -44,7 +45,7 @@ class FieldIndexing {
int64_t
get_size_per_chunk() const {
return size_per_chunk_;
return segcore_config_.get_size_per_chunk();
}
virtual knowhere::Index*
......@@ -53,7 +54,7 @@ class FieldIndexing {
protected:
// additional info
const FieldMeta& field_meta_;
const int64_t size_per_chunk_;
const SegcoreConfig& segcore_config_;
};
template <typename T>
class ScalarFieldIndexing : public FieldIndexing {
......@@ -89,21 +90,22 @@ class VectorFieldIndexing : public FieldIndexing {
}
knowhere::Config
get_build_conf() const;
get_build_params() const;
knowhere::Config
get_search_conf(int top_k) const;
get_search_params(int top_k) const;
private:
tbb::concurrent_vector<std::unique_ptr<knowhere::VecIndex>> data_;
};
std::unique_ptr<FieldIndexing>
CreateIndex(const FieldMeta& field_meta, int64_t size_per_chunk);
CreateIndex(const FieldMeta& field_meta, const SegcoreConfig& segcore_config);
class IndexingRecord {
public:
explicit IndexingRecord(const Schema& schema, int64_t size_per_chunk)
: schema_(schema), size_per_chunk_(size_per_chunk) {
explicit IndexingRecord(const Schema& schema, const SegcoreConfig& segcore_config)
: schema_(schema), segcore_config_(segcore_config) {
Initialize();
}
......@@ -125,7 +127,7 @@ class IndexingRecord {
}
}
field_indexings_.try_emplace(offset, CreateIndex(field, size_per_chunk_));
field_indexings_.try_emplace(offset, CreateIndex(field, segcore_config_));
}
assert(offset_id == schema_.size());
}
......@@ -170,6 +172,7 @@ class IndexingRecord {
private:
const Schema& schema_;
const SegcoreConfig& segcore_config_;
private:
// control info
......@@ -177,7 +180,6 @@ class IndexingRecord {
// std::atomic<int64_t> finished_ack_ = 0;
AckResponder finished_ack_;
std::mutex mutex_;
int64_t size_per_chunk_;
private:
// field_offset => indexing
......
// 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 "common/Schema.h"
#include "SegcoreConfig.h"
#include "utils/Json.h"
#include "yaml-cpp/yaml.h"
namespace milvus::segcore {
static YAML::Node
subnode(const YAML::Node& parent, const std::string& key) {
AssertInfo(parent.IsMap(), "wrong type node when getting key[" + key + "]");
auto& node = parent[key];
AssertInfo(node.IsDefined(), "key[" + key + "] not found in sub-node");
return node;
}
template <typename T, typename Func>
std::vector<T>
apply_parser(const YAML::Node& node, Func func) {
std::vector<T> results;
Assert(node.IsDefined());
if (node.IsScalar()) {
results.emplace_back(func(node));
} else if (node.IsSequence()) {
for (auto& element : node) {
Assert(element.IsScalar());
results.emplace_back(func(element));
}
} else {
PanicInfo("node should be scalar or sequence");
}
return results;
}
SegcoreConfig
SegcoreConfig::parse_from(const std::string& config_path) {
try {
SegcoreConfig result;
YAML::Node top_config = YAML::LoadFile(config_path);
Assert(top_config.IsMap());
auto seg_config = subnode(top_config, "segcore");
auto chunk_size = subnode(seg_config, "chunk_size").as<int64_t>();
result.size_per_chunk_ = chunk_size;
auto index_list = subnode(seg_config, "small_index");
Assert(index_list.IsSequence());
for (auto index : index_list) {
Assert(index.IsMap());
auto metric_types = apply_parser<MetricType>(subnode(index, "metric_type"), [](const YAML::Node& node) {
return GetMetricType(node.as<std::string>());
});
{
std::sort(metric_types.begin(), metric_types.end());
auto end_iter = std::unique(metric_types.begin(), metric_types.end());
metric_types.resize(end_iter - metric_types.begin());
}
auto index_type = index["index_type"].as<std::string>();
AssertInfo(index_type == "IVF", "only ivf is supported now");
SmallIndexConf conf;
conf.index_type = index_type;
// parse build config
for (auto node : index["build_params"]) {
// TODO: currently support IVF only
auto key = node.first.as<std::string>();
Assert(key == "nlist");
auto value = node.second.as<int64_t>();
conf.build_params[key] = value;
}
// parse search config
for (auto node : index["search_params"]) {
// TODO: currently support IVF only
auto key = node.first.as<std::string>();
Assert(key == "nprobe");
auto value = node.second.as<int64_t>();
conf.search_params[key] = value;
}
for (auto metric_type : metric_types) {
Assert(result.table_.count(metric_type));
result.table_[metric_type] = conf;
}
}
} catch (const WrappedRuntimeError& e) {
// re-throw
throw e;
} catch (std::exception& e) {
PanicInfo(std::string("Invalid Yaml:\n") + e.what());
}
}
} // namespace milvus::segcore
// 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
#pragma once
#include "common/Types.h"
#include "utils/Json.h"
#include <map>
#include <string>
namespace milvus::segcore {
struct SmallIndexConf {
std::string index_type;
nlohmann::json build_params;
nlohmann::json search_params;
};
class SegcoreConfig {
public:
static SegcoreConfig
parse_from(const std::string& string_path);
static SegcoreConfig
default_config() {
// TODO: remove this when go side is ready
SegcoreConfig config;
config.set_size_per_chunk(32 * 1024);
SmallIndexConf sub_conf;
sub_conf.build_params["nlist"] = 100;
sub_conf.search_params["nprobe"] = 4;
sub_conf.index_type = "IVF";
config.table_[MetricType::METRIC_L2] = sub_conf;
config.table_[MetricType::METRIC_INNER_PRODUCT] = sub_conf;
return config;
}
const SmallIndexConf&
at(MetricType metric_type) const {
Assert(table_.count(metric_type));
return table_.at(metric_type);
}
int64_t
get_size_per_chunk() const {
return size_per_chunk_;
}
void
set_size_per_chunk(int64_t size_per_chunk) {
size_per_chunk_ = size_per_chunk;
}
void
set_small_index_config(MetricType metric_type, const SmallIndexConf& small_index_conf) {
table_[metric_type] = small_index_conf;
}
protected:
SegcoreConfig() = default;
private:
int64_t size_per_chunk_ = -1;
std::map<MetricType, SmallIndexConf> table_;
};
} // namespace milvus::segcore
......@@ -20,8 +20,8 @@ TestABI() {
}
std::unique_ptr<SegmentGrowing>
CreateGrowingSegment(SchemaPtr schema, int64_t size_per_chunk) {
auto segment = std::make_unique<SegmentGrowingImpl>(schema, size_per_chunk);
CreateGrowingSegment(SchemaPtr schema, const SegcoreConfig& segcore_config) {
auto segment = std::make_unique<SegmentGrowingImpl>(schema, segcore_config);
return segment;
}
......
......@@ -95,7 +95,13 @@ class SegmentGrowing : public SegmentInternalInterface {
using SegmentGrowingPtr = std::unique_ptr<SegmentGrowing>;
SegmentGrowingPtr
CreateGrowingSegment(SchemaPtr schema, int64_t size_per_chunk = 32 * 1024);
CreateGrowingSegment(SchemaPtr schema, const SegcoreConfig& segcore_config);
inline SegmentGrowingPtr
CreateGrowingSegment(SchemaPtr schema) {
auto seg_conf = SegcoreConfig::default_config();
return CreateGrowingSegment(schema, seg_conf);
}
} // namespace segcore
} // namespace milvus
......@@ -190,7 +190,8 @@ SegmentGrowingImpl::do_insert(int64_t reserved_begin,
}
record_.ack_responder_.AddSegment(reserved_begin, reserved_begin + size);
if (!debug_disable_small_index_) {
indexing_record_.UpdateResourceAck(record_.ack_responder_.GetAck() / size_per_chunk_, record_);
indexing_record_.UpdateResourceAck(record_.ack_responder_.GetAck() / segcore_config_.get_size_per_chunk(),
record_);
}
}
......@@ -241,9 +242,10 @@ SegmentGrowingImpl::Close() {
int64_t
SegmentGrowingImpl::GetMemoryUsageInBytes() const {
int64_t total_bytes = 0;
int64_t ins_n = upper_align(record_.reserved, size_per_chunk_);
auto size_per_chunk = segcore_config_.get_size_per_chunk();
int64_t ins_n = upper_align(record_.reserved, size_per_chunk);
total_bytes += ins_n * (schema_->get_total_sizeof() + 16 + 1);
int64_t del_n = upper_align(deleted_record_.reserved, size_per_chunk_);
int64_t del_n = upper_align(deleted_record_.reserved, size_per_chunk);
total_bytes += del_n * (16 * 2);
return total_bytes;
}
......@@ -268,7 +270,7 @@ SegmentGrowingImpl::chunk_data_impl(FieldOffset field_offset, int64_t chunk_id)
int64_t
SegmentGrowingImpl::num_chunk() const {
auto size = get_insert_record().ack_responder_.GetAck();
return upper_div(size, size_per_chunk_);
return upper_div(size, segcore_config_.get_size_per_chunk());
}
void
SegmentGrowingImpl::vector_search(int64_t vec_count,
......
......@@ -108,7 +108,7 @@ class SegmentGrowingImpl : public SegmentGrowing {
int64_t
size_per_chunk() const final {
return size_per_chunk_;
return segcore_config_.get_size_per_chunk();
}
public:
......@@ -157,13 +157,13 @@ class SegmentGrowingImpl : public SegmentGrowing {
public:
friend std::unique_ptr<SegmentGrowing>
CreateGrowingSegment(SchemaPtr schema, int64_t size_per_chunk);
CreateGrowingSegment(SchemaPtr schema, const SegcoreConfig& segcore_config);
explicit SegmentGrowingImpl(SchemaPtr schema, int64_t size_per_chunk)
: size_per_chunk_(size_per_chunk),
explicit SegmentGrowingImpl(SchemaPtr schema, const SegcoreConfig& segcore_config)
: segcore_config_(segcore_config),
schema_(std::move(schema)),
record_(*schema_, size_per_chunk),
indexing_record_(*schema_, size_per_chunk) {
record_(*schema_, segcore_config.get_size_per_chunk()),
indexing_record_(*schema_, segcore_config_) {
}
void
......@@ -199,7 +199,7 @@ class SegmentGrowingImpl : public SegmentGrowing {
const std::vector<aligned_vector<uint8_t>>& columns_data);
private:
int64_t size_per_chunk_;
SegcoreConfig segcore_config_;
SchemaPtr schema_;
std::atomic<SegmentState> state_ = SegmentState::Open;
......
......@@ -35,6 +35,6 @@ class SegmentSealed : public SegmentInternalInterface {
using SegmentSealedPtr = std::unique_ptr<SegmentSealed>;
SegmentSealedPtr
CreateSealedSegment(SchemaPtr schema, int64_t size_per_chunk = 32 * 1024);
CreateSealedSegment(SchemaPtr schema);
} // namespace milvus::segcore
......@@ -384,7 +384,7 @@ SegmentSealedImpl::HasFieldData(FieldId field_id) const {
}
SegmentSealedPtr
CreateSealedSegment(SchemaPtr schema, int64_t size_per_chunk) {
CreateSealedSegment(SchemaPtr schema) {
return std::make_unique<SegmentSealedImpl>(schema);
}
......
......@@ -46,11 +46,11 @@ EasyAssertInfo(
[[noreturn]] void
ThrowWithTrace(const std::exception& exception) {
if (typeid(exception) == typeid(WrappedRuntimError)) {
if (typeid(exception) == typeid(WrappedRuntimeError)) {
throw exception;
}
auto err_msg = exception.what() + std::string("\n") + EasyStackTrace();
throw WrappedRuntimError(err_msg);
throw WrappedRuntimeError(err_msg);
}
} // namespace milvus::impl
......@@ -18,19 +18,22 @@
/* Paste this on the file you want to debug. */
namespace milvus::impl {
namespace milvus {
namespace impl {
void
EasyAssertInfo(
bool value, std::string_view expr_str, std::string_view filename, int lineno, std::string_view extra_info);
class WrappedRuntimError : public std::runtime_error {
using std::runtime_error::runtime_error;
};
[[noreturn]] void
ThrowWithTrace(const std::exception& exception);
} // namespace milvus::impl
} // namespace impl
class WrappedRuntimeError : public std::runtime_error {
using std::runtime_error::runtime_error;
};
} // namespace milvus
#define AssertInfo(expr, info) \
do { \
......
......@@ -46,9 +46,7 @@ endif()
# ****************************** Thirdparty yaml ***************************************
if ( MILVUS_WITH_YAMLCPP )
add_subdirectory( yaml-cpp )
endif()
add_subdirectory( yaml-cpp )
# ****************************** Thirdparty opentracing ***************************************
if ( MILVUS_WITH_OPENTRACING )
......
include_directories(${CMAKE_HOME_DIRECTORY}/src)
include_directories(${CMAKE_HOME_DIRECTORY}/src/index/knowhere)
include_directories(${CMAKE_HOME_DIRECTORY}/src/thirdparty)
add_definitions(-DMILVUS_TEST_SEGCORE_YAML_PATH="${CMAKE_SOURCE_DIR}/unittest/test_utils/test_segcore.yaml")
set(MILVUS_TEST_FILES
test_naive.cpp
test_segcore.cpp
......@@ -17,14 +21,18 @@ set(MILVUS_TEST_FILES
test_interface.cpp
test_span.cpp
test_load.cpp
init_gtest.cpp
)
add_executable(all_tests
${MILVUS_TEST_FILES}
)
# check if memory leak exists in index builder
set(INDEX_BUILDER_TEST_FILES
test_index_wrapper.cpp)
test_index_wrapper.cpp
)
add_executable(index_builder_test
${INDEX_BUILDER_TEST_FILES}
)
......@@ -39,7 +47,6 @@ target_link_libraries(index_builder_test
target_link_libraries(all_tests
gtest
gtest_main
milvus_segcore
milvus_indexbuilder
log
......
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#include <gtest/gtest.h>
#include "common/SystemProperty.h"
#ifndef MILVUS_TEST_SEGCORE_YAML_PATH
#error MILVUS_TEST_SEGCORE_YAML_PATH is not defined
#define MILVUS_TEST_SEGCORE_YAML_PATH ""
#endif
int
main(int argc, char** argv) {
::testing::InitGoogleTest(&argc, argv);
return RUN_ALL_TESTS();
}
\ No newline at end of file
......@@ -26,7 +26,8 @@ TEST(Span, Naive) {
schema->AddDebugField("floatvec", DataType::VECTOR_FLOAT, 32, MetricType::METRIC_L2);
auto dataset = DataGen(schema, N);
auto segment = CreateGrowingSegment(schema, size_per_chunk);
auto seg_conf = SegcoreConfig::default_config();
auto segment = CreateGrowingSegment(schema, seg_conf);
segment->PreInsert(N);
segment->Insert(0, N, dataset.row_ids_.data(), dataset.timestamps_.data(), dataset.raw_);
auto vec_ptr = dataset.get_col<uint8_t>(0);
......
segcore:
chunk_size: 32768
small_index:
- metric_type: ["L2", "IP"]
index_type: "IVF"
build_params:
nlist: 100
search_params:
nprobe: 4
# binary is not ready yet
- metric_type: ["jaccard"]
index_type: "IVF"
build_params:
nlist: 100
search_params:
nprobe: 4
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册