Update run_go_unittest.sh

Signed-off-by: quicksilver <zhifeng.zhang@zilliz.com>
pull/4973/head^2
quicksilver 2021-01-13 10:40:46 +08:00 committed by yefu.chen
parent 4cd42c553f
commit 8e9d8e36e1
55 changed files with 505 additions and 1260 deletions

View File

@ -84,12 +84,12 @@ datatype_is_vector(DataType datatype) {
struct FieldMeta {
public:
FieldMeta(const FieldName& name, FieldId id, DataType type) : name_(name), id_(id), type_(type) {
FieldMeta(std::string_view name, DataType type) : name_(name), type_(type) {
Assert(!is_vector());
}
FieldMeta(const FieldName& name, FieldId id, DataType type, int64_t dim, MetricType metric_type)
: name_(name), id_(id), type_(type), vector_info_(VectorInfo{dim, metric_type}) {
FieldMeta(std::string_view name, DataType type, int64_t dim, MetricType metric_type)
: name_(name), type_(type), vector_info_(VectorInfo{dim, metric_type}) {
Assert(is_vector());
}
@ -113,16 +113,11 @@ struct FieldMeta {
return vector_info_->metric_type_;
}
const FieldName&
const std::string&
get_name() const {
return name_;
}
const FieldId&
get_id() const {
return id_;
}
DataType
get_data_type() const {
return type_;
@ -142,8 +137,7 @@ struct FieldMeta {
int64_t dim_;
MetricType metric_type_;
};
FieldName name_;
FieldId id_;
std::string name_;
DataType type_ = DataType::NONE;
std::optional<VectorInfo> vector_info_;
};

View File

@ -35,7 +35,6 @@ Schema::ParseFrom(const milvus::proto::schema::CollectionSchema& schema_proto) {
std::set<std::string> system_field_names = {"RowID", "Timestamp"};
for (const milvus::proto::schema::FieldSchema& child : schema_proto.fields()) {
auto field_offset = FieldOffset(schema->size());
auto field_id = child.fieldid();
auto name = child.name();
if (field_id < 100) {
@ -48,7 +47,7 @@ Schema::ParseFrom(const milvus::proto::schema::CollectionSchema& schema_proto) {
if (child.is_primary_key()) {
AssertInfo(!schema->primary_key_offset_opt_.has_value(), "repetitive primary key");
schema->primary_key_offset_opt_ = field_offset;
schema->primary_key_offset_opt_ = schema->size();
}
if (datatype_is_vector(data_type)) {
@ -64,9 +63,9 @@ Schema::ParseFrom(const milvus::proto::schema::CollectionSchema& schema_proto) {
auto dim = boost::lexical_cast<int64_t>(type_map.at("dim"));
AssertInfo(index_map.count("metric_type"), "index not found");
auto metric_type = GetMetricType(index_map.at("metric_type"));
schema->AddField(FieldName(name), FieldId(field_id), data_type, dim, metric_type);
schema->AddField(name, data_type, dim, metric_type);
} else {
schema->AddField(FieldName(name), FieldId(field_id), data_type);
schema->AddField(name, data_type);
}
}
return schema;

View File

@ -24,32 +24,14 @@ namespace milvus {
class Schema {
public:
void
AddDebugField(const std::string& name, DataType data_type) {
static int64_t debug_id = 1000;
this->AddField(FieldName(name), FieldId(debug_id), data_type);
debug_id++;
}
// auto gen field_id for convenience
void
AddDebugField(const std::string& name, DataType data_type, int64_t dim, MetricType metric_type) {
static int64_t debug_id = 2000;
auto field_meta = FieldMeta(FieldName(name), FieldId(debug_id), data_type, dim, metric_type);
debug_id++;
AddField(std::string_view field_name, DataType data_type) {
auto field_meta = FieldMeta(field_name, data_type);
this->AddField(std::move(field_meta));
}
// scalar type
void
AddField(const FieldName& name, const FieldId id, DataType data_type) {
auto field_meta = FieldMeta(name, id, data_type);
this->AddField(std::move(field_meta));
}
// vector type
void
AddField(const FieldName& name, const FieldId id, DataType data_type, int64_t dim, MetricType metric_type) {
auto field_meta = FieldMeta(name, id, data_type, dim, metric_type);
AddField(std::string_view field_name, DataType data_type, int64_t dim, MetricType metric_type) {
auto field_meta = FieldMeta(field_name, data_type, dim, metric_type);
this->AddField(std::move(field_meta));
}
@ -79,10 +61,10 @@ class Schema {
}
const FieldMeta&
operator[](FieldOffset field_offset) const {
Assert(field_offset.get() >= 0);
Assert(field_offset.get() < fields_.size());
return fields_[field_offset.get()];
operator[](int field_index) const {
Assert(field_index >= 0);
Assert(field_index < fields_.size());
return fields_[field_index];
}
auto
@ -90,46 +72,34 @@ class Schema {
return total_sizeof_;
}
const std::vector<int64_t>&
const std::vector<int>&
get_sizeof_infos() const {
return sizeof_infos_;
}
[[deprecated]] std::optional<FieldOffset>
get_offset_opt(const FieldName& field_name) const {
if (!name_offsets_.count(field_name)) {
std::optional<int>
get_offset(const std::string& field_name) const {
if (!offsets_.count(field_name)) {
return std::nullopt;
} else {
return name_offsets_.at(field_name);
return offsets_.at(field_name);
}
}
FieldOffset
get_offset(const FieldName& field_name) const {
Assert(name_offsets_.count(field_name));
return name_offsets_.at(field_name);
}
FieldOffset
get_offset(const FieldId& field_id) const {
Assert(id_offsets_.count(field_id));
return id_offsets_.at(field_id);
}
const std::vector<FieldMeta>&
get_fields() const {
return fields_;
}
const FieldMeta&
operator[](const FieldName& field_name) const {
auto offset_iter = name_offsets_.find(field_name);
AssertInfo(offset_iter != name_offsets_.end(), "Cannot find field_name: " + field_name.get());
operator[](const std::string& field_name) const {
auto offset_iter = offsets_.find(field_name);
AssertInfo(offset_iter != offsets_.end(), "Cannot found field_name: " + field_name);
auto offset = offset_iter->second;
return (*this)[offset];
}
std::optional<FieldOffset>
std::optional<int>
get_primary_key_offset() const {
return primary_key_offset_opt_;
}
@ -142,10 +112,7 @@ class Schema {
AddField(FieldMeta&& field_meta) {
auto offset = fields_.size();
fields_.emplace_back(field_meta);
AssertInfo(!name_offsets_.count(field_meta.get_name()), "duplicated field name");
name_offsets_.emplace(field_meta.get_name(), offset);
AssertInfo(!id_offsets_.count(field_meta.get_id()), "duplicated field id");
id_offsets_.emplace(field_meta.get_id(), offset);
offsets_.emplace(field_meta.get_name(), offset);
auto field_sizeof = field_meta.get_sizeof();
sizeof_infos_.push_back(std::move(field_sizeof));
total_sizeof_ += field_sizeof;
@ -156,12 +123,11 @@ class Schema {
std::vector<FieldMeta> fields_;
// a mapping for random access
std::unordered_map<FieldName, FieldOffset> name_offsets_; // field_name -> offset
std::unordered_map<FieldId, FieldOffset> id_offsets_; // field_id -> offset
std::vector<int64_t> sizeof_infos_;
std::unordered_map<std::string, int> offsets_;
std::vector<int> sizeof_infos_;
int total_sizeof_ = 0;
bool is_auto_id_ = true;
std::optional<FieldOffset> primary_key_offset_opt_;
std::optional<int> primary_key_offset_opt_;
};
using SchemaPtr = std::shared_ptr<Schema>;

View File

@ -16,7 +16,6 @@
#include <boost/align/aligned_allocator.hpp>
#include <memory>
#include <vector>
#include <NamedType/named_type.hpp>
namespace milvus {
using Timestamp = uint64_t; // TODO: use TiKV-like timestamp
@ -71,8 +70,4 @@ struct QueryResult {
using QueryResultPtr = std::shared_ptr<QueryResult>;
using FieldId = fluent::NamedType<int64_t, struct FieldIdTag, fluent::Comparable, fluent::Hashable>;
using FieldName = fluent::NamedType<std::string, struct FieldNameTag, fluent::Comparable, fluent::Hashable>;
using FieldOffset = fluent::NamedType<int64_t, struct FieldOffsetTag, fluent::Comparable, fluent::Hashable>;
} // namespace milvus

View File

@ -61,8 +61,10 @@ struct BoolBinaryExpr : BinaryExpr {
accept(ExprVisitor&) override;
};
using FieldId = std::string;
struct TermExpr : Expr {
FieldOffset field_offset_;
FieldId field_id_;
DataType data_type_ = DataType::NONE;
// std::vector<std::any> terms_;
@ -76,7 +78,7 @@ struct TermExpr : Expr {
};
struct RangeExpr : Expr {
FieldOffset field_offset_;
FieldId field_id_;
DataType data_type_ = DataType::NONE;
enum class OpType { GreaterThan = 0, GreaterEqual = 1, LessThan = 2, LessEqual = 3, Equal, NotEqual };
static const std::map<std::string, OpType> mapping_; // op_name -> op

View File

@ -87,15 +87,15 @@ class Parser {
template <typename T>
ExprPtr
ParseRangeNodeImpl(const FieldName& field_name, const Json& body);
ParseRangeNodeImpl(const std::string& field_name, const Json& body);
template <typename T>
ExprPtr
ParseTermNodeImpl(const FieldName& field_name, const Json& body);
ParseTermNodeImpl(const std::string& field_name, const Json& body);
private:
const Schema& schema;
std::map<std::string, FieldOffset> tag2field_; // PlaceholderName -> field offset
std::map<std::string, FieldId> tag2field_; // PlaceholderName -> FieldId
std::optional<std::unique_ptr<VectorPlanNode>> vector_node_opt_;
};
@ -104,7 +104,7 @@ Parser::ParseRangeNode(const Json& out_body) {
Assert(out_body.is_object());
Assert(out_body.size() == 1);
auto out_iter = out_body.begin();
auto field_name = FieldName(out_iter.key());
auto field_name = out_iter.key();
auto body = out_iter.value();
auto data_type = schema[field_name].get_data_type();
Assert(!datatype_is_vector(data_type));
@ -152,7 +152,7 @@ ExprPtr
Parser::ParseTermNode(const Json& out_body) {
Assert(out_body.size() == 1);
auto out_iter = out_body.begin();
auto field_name = FieldName(out_iter.key());
auto field_name = out_iter.key();
auto body = out_iter.value();
auto data_type = schema[field_name].get_data_type();
Assert(!datatype_is_vector(data_type));
@ -190,7 +190,7 @@ Parser::ParseVecNode(const Json& out_body) {
// TODO add binary info
Assert(out_body.size() == 1);
auto iter = out_body.begin();
auto field_name = FieldName(iter.key());
std::string field_name = iter.key();
auto& vec_info = iter.value();
Assert(vec_info.is_object());
@ -198,7 +198,8 @@ Parser::ParseVecNode(const Json& out_body) {
AssertInfo(topK > 0, "topK must greater than 0");
AssertInfo(topK < 16384, "topK is too large");
auto field_offset = schema.get_offset(field_name);
auto field_offset_opt = schema.get_offset(field_name);
AssertInfo(field_offset_opt.has_value(), "field_name(" + field_name + ") not found");
auto vec_node = [&]() -> std::unique_ptr<VectorPlanNode> {
auto field_meta = schema.operator[](field_name);
@ -212,24 +213,24 @@ Parser::ParseVecNode(const Json& out_body) {
vec_node->query_info_.topK_ = topK;
vec_node->query_info_.metric_type_ = vec_info.at("metric_type");
vec_node->query_info_.search_params_ = vec_info.at("params");
vec_node->query_info_.field_offset_ = field_offset;
vec_node->query_info_.field_id_ = field_name;
vec_node->query_info_.field_offset_ = field_offset_opt.value();
vec_node->placeholder_tag_ = vec_info.at("query");
auto tag = vec_node->placeholder_tag_;
AssertInfo(!tag2field_.count(tag), "duplicated placeholder tag");
tag2field_.emplace(tag, field_offset);
tag2field_.emplace(tag, field_name);
return vec_node;
}
template <typename T>
ExprPtr
Parser::ParseTermNodeImpl(const FieldName& field_name, const Json& body) {
Parser::ParseTermNodeImpl(const std::string& field_name, const Json& body) {
auto expr = std::make_unique<TermExprImpl<T>>();
auto field_offset = schema.get_offset(field_name);
auto data_type = schema[field_name].get_data_type();
Assert(body.is_object());
auto values = body["values"];
expr->field_offset_ = field_offset;
expr->field_id_ = field_name;
expr->data_type_ = data_type;
for (auto& value : values) {
if constexpr (std::is_same_v<T, bool>) {
@ -251,12 +252,11 @@ Parser::ParseTermNodeImpl(const FieldName& field_name, const Json& body) {
template <typename T>
ExprPtr
Parser::ParseRangeNodeImpl(const FieldName& field_name, const Json& body) {
Parser::ParseRangeNodeImpl(const std::string& field_name, const Json& body) {
auto expr = std::make_unique<RangeExprImpl<T>>();
auto field_meta = schema[field_name];
auto data_type = field_meta.get_data_type();
auto data_type = schema[field_name].get_data_type();
expr->data_type_ = data_type;
expr->field_offset_ = schema.get_offset(field_name);
expr->field_id_ = field_name;
Assert(body.is_object());
for (auto& item : body.items()) {
auto op_name = boost::algorithm::to_lower_copy(std::string(item.key()));
@ -291,8 +291,8 @@ ParsePlaceholderGroup(const Plan* plan, const std::string& blob) {
Placeholder element;
element.tag_ = info.tag();
Assert(plan->tag2field_.count(element.tag_));
auto field_offset = plan->tag2field_.at(element.tag_);
auto& field_meta = plan->schema_[field_offset];
auto field_id = plan->tag2field_.at(element.tag_);
auto& field_meta = plan->schema_[field_id];
element.num_of_queries_ = info.values_size();
AssertInfo(element.num_of_queries_, "must have queries");
Assert(element.num_of_queries_ > 0);

View File

@ -33,7 +33,7 @@ struct Plan {
public:
const Schema& schema_;
std::unique_ptr<VectorPlanNode> plan_node_;
std::map<std::string, FieldOffset> tag2field_; // PlaceholderName -> FieldOffset
std::map<std::string, FieldId> tag2field_; // PlaceholderName -> FieldId
std::vector<std::string> target_entries_;
// TODO: add move extra info
};

View File

@ -40,7 +40,8 @@ using PlanNodePtr = std::unique_ptr<PlanNode>;
struct QueryInfo {
int64_t topK_;
FieldOffset field_offset_;
FieldId field_id_;
int64_t field_offset_;
std::string metric_type_; // TODO: use enum
nlohmann::json search_params_;
};

View File

@ -57,7 +57,9 @@ FloatSearch(const segcore::SegmentSmallIndex& segment,
// step 2.1: get meta
// step 2.2: get which vector field to search
auto vecfield_offset = info.field_offset_;
auto vecfield_offset_opt = schema.get_offset(info.field_id_);
Assert(vecfield_offset_opt.has_value());
auto vecfield_offset = vecfield_offset_opt.value();
auto& field = schema[vecfield_offset];
Assert(field.get_data_type() == DataType::VECTOR_FLOAT);
@ -152,7 +154,9 @@ BinarySearch(const segcore::SegmentSmallIndex& segment,
// step 2.1: get meta
// step 2.2: get which vector field to search
auto vecfield_offset = info.field_offset_;
auto vecfield_offset_opt = schema.get_offset(info.field_id_);
Assert(vecfield_offset_opt.has_value());
auto vecfield_offset = vecfield_offset_opt.value();
auto& field = schema[vecfield_offset];
Assert(field.get_data_type() == DataType::VECTOR_BINARY);

View File

@ -61,7 +61,7 @@ SearchOnSealed(const Schema& schema,
// Assert(field.get_data_type() == DataType::VECTOR_FLOAT);
auto dim = field.get_dim();
Assert(record.is_ready(field_offset));
Assert(record.test_readiness(field_offset));
auto indexing_entry = record.get_entry(field_offset);
std::cout << " SearchOnSealed, indexing_entry->metric:" << indexing_entry->metric_type_ << std::endl;
std::cout << " SearchOnSealed, query_info.metric_type_:" << query_info.metric_type_ << std::endl;

View File

@ -121,7 +121,9 @@ ExecExprVisitor::ExecRangeVisitorImpl(RangeExprImpl<T>& expr, IndexFunc index_fu
auto& records = segment_.get_insert_record();
auto data_type = expr.data_type_;
auto& schema = segment_.get_schema();
auto field_offset = expr.field_offset_;
auto field_offset_opt = schema.get_offset(expr.field_id_);
Assert(field_offset_opt);
auto field_offset = field_offset_opt.value();
auto& field_meta = schema[field_offset];
auto vec_ptr = records.get_entity<T>(field_offset);
auto& vec = *vec_ptr;
@ -232,7 +234,7 @@ ExecExprVisitor::ExecRangeVisitorDispatcher(RangeExpr& expr_raw) -> RetType {
void
ExecExprVisitor::visit(RangeExpr& expr) {
auto& field_meta = segment_.get_schema()[expr.field_offset_];
auto& field_meta = segment_.get_schema()[expr.field_id_];
Assert(expr.data_type_ == field_meta.get_data_type());
RetType ret;
switch (expr.data_type_) {
@ -277,8 +279,9 @@ ExecExprVisitor::ExecTermVisitorImpl(TermExpr& expr_raw) -> RetType {
auto& records = segment_.get_insert_record();
auto data_type = expr.data_type_;
auto& schema = segment_.get_schema();
auto field_offset = expr_raw.field_offset_;
auto field_offset_opt = schema.get_offset(expr.field_id_);
Assert(field_offset_opt);
auto field_offset = field_offset_opt.value();
auto& field_meta = schema[field_offset];
auto vec_ptr = records.get_entity<T>(field_offset);
auto& vec = *vec_ptr;
@ -307,7 +310,7 @@ ExecExprVisitor::ExecTermVisitorImpl(TermExpr& expr_raw) -> RetType {
void
ExecExprVisitor::visit(TermExpr& expr) {
auto& field_meta = segment_.get_schema()[expr.field_offset_];
auto& field_meta = segment_.get_schema()[expr.field_id_];
Assert(expr.data_type_ == field_meta.get_data_type());
RetType ret;
switch (expr.data_type_) {

View File

@ -75,7 +75,7 @@ ExecPlanNodeVisitor::visit(FloatVectorANNS& node) {
auto& sealed_indexing = segment->get_sealed_indexing_record();
if (sealed_indexing.is_ready(node.query_info_.field_offset_)) {
if (sealed_indexing.test_readiness(node.query_info_.field_offset_)) {
SearchOnSealed(segment->get_schema(), sealed_indexing, node.query_info_, src_data, num_queries, timestamp_,
bitset_pack, ret);
} else {
@ -105,7 +105,7 @@ ExecPlanNodeVisitor::visit(BinaryVectorANNS& node) {
}
auto& sealed_indexing = segment->get_sealed_indexing_record();
if (sealed_indexing.is_ready(node.query_info_.field_offset_)) {
if (sealed_indexing.test_readiness(node.query_info_.field_offset_)) {
SearchOnSealed(segment->get_schema(), sealed_indexing, node.query_info_, src_data, num_queries, timestamp_,
bitset_pack, ret);
} else {

View File

@ -137,7 +137,7 @@ ShowExprVisitor::visit(TermExpr& expr) {
}();
Json res{{"expr_type", "Term"},
{"field_offset", expr.field_offset_.get()},
{"field_id", expr.field_id_},
{"data_type", datatype_name(expr.data_type_)},
{"terms", std::move(terms)}};
@ -184,7 +184,7 @@ ShowExprVisitor::visit(RangeExpr& expr) {
}();
Json res{{"expr_type", "Range"},
{"field_offset", expr.field_offset_.get()},
{"field_id", expr.field_id_},
{"data_type", datatype_name(expr.data_type_)},
{"conditions", std::move(conditions)}};
ret_ = res;

View File

@ -54,12 +54,12 @@ ShowPlanNodeVisitor::visit(FloatVectorANNS& node) {
assert(!ret_);
auto& info = node.query_info_;
Json json_body{
{"node_type", "FloatVectorANNS"}, //
{"metric_type", info.metric_type_}, //
{"field_offset_", info.field_offset_.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_}, //
{"topK", info.topK_}, //
{"search_params", info.search_params_}, //
{"placeholder_tag", node.placeholder_tag_}, //
};
if (node.predicate_.has_value()) {
ShowExprVisitor expr_show;
@ -76,12 +76,12 @@ ShowPlanNodeVisitor::visit(BinaryVectorANNS& node) {
assert(!ret_);
auto& info = node.query_info_;
Json json_body{
{"node_type", "BinaryVectorANNS"}, //
{"metric_type", info.metric_type_}, //
{"field_offset_", info.field_offset_.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_}, //
{"topK", info.topK_}, //
{"search_params", info.search_params_}, //
{"placeholder_tag", node.placeholder_tag_}, //
};
if (node.predicate_.has_value()) {
ShowExprVisitor expr_show;

View File

@ -1,6 +1,8 @@
set(SEGCORE_FILES
SegmentNaive.cpp
SegmentSmallIndex.cpp
IndexMeta.cpp
Collection.cpp
collection_c.cpp
segment_c.cpp

View File

@ -21,7 +21,104 @@ namespace milvus::segcore {
Collection::Collection(const std::string& collection_proto) : schema_proto_(collection_proto) {
parse();
index_ = nullptr;
}
#if 0
void
Collection::AddIndex(const grpc::IndexParam& index_param) {
auto& index_name = index_param.index_name();
auto& field_name = index_param.field_name();
Assert(!index_name.empty());
Assert(!field_name.empty());
auto index_type = knowhere::IndexEnum::INDEX_FAISS_IVFPQ;
auto index_mode = knowhere::IndexMode::MODE_CPU;
knowhere::Config index_conf;
bool found_index_type = false;
bool found_index_mode = false;
bool found_index_conf = false;
auto extra_params = index_param.extra_params();
for (auto& extra_param : extra_params) {
if (extra_param.key() == "index_type") {
index_type = extra_param.value().data();
found_index_type = true;
continue;
}
if (extra_param.key() == "index_mode") {
auto index_mode_int = stoi(extra_param.value());
if (index_mode_int == 0) {
found_index_mode = true;
continue;
} else if (index_mode_int == 1) {
index_mode = knowhere::IndexMode::MODE_GPU;
found_index_mode = true;
continue;
} else {
throw std::runtime_error("Illegal index mode, only 0 or 1 is supported.");
}
}
if (extra_param.key() == "params") {
index_conf = nlohmann::json::parse(extra_param.value());
found_index_conf = true;
continue;
}
}
if (!found_index_type) {
std::cout << "WARN: Not specify index type, use default index type: INDEX_FAISS_IVFPQ" << std::endl;
}
if (!found_index_mode) {
std::cout << "WARN: Not specify index mode, use default index mode: MODE_CPU" << std::endl;
}
if (!found_index_conf) {
int dim = 0;
for (auto& field : schema_->get_fields()) {
if (field.get_data_type() == DataType::VECTOR_FLOAT) {
dim = field.get_dim();
}
}
Assert(dim != 0);
index_conf = milvus::knowhere::Config{
{knowhere::meta::DIM, dim}, {knowhere::IndexParams::nlist, 100},
{knowhere::IndexParams::nprobe, 4}, {knowhere::IndexParams::m, 4},
{knowhere::IndexParams::nbits, 8}, {knowhere::Metric::TYPE, milvus::knowhere::Metric::L2},
{knowhere::meta::DEVICEID, 0},
};
std::cout << "WARN: Not specify index config, use default index config" << std::endl;
}
index_->AddEntry(index_name, field_name, index_type, index_mode, index_conf);
}
void
Collection::CreateIndex(std::string& index_config) {
if (index_config.empty()) {
index_ = nullptr;
std::cout << "null index config when create index" << std::endl;
return;
}
milvus::proto::etcd::CollectionMeta collection_meta;
auto suc = google::protobuf::TextFormat::ParseFromString(index_config, &collection_meta);
if (!suc) {
std::cerr << "unmarshal index string failed" << std::endl;
}
index_ = std::make_shared<IndexMeta>(schema_);
// for (const auto& index : collection_meta.indexes()) {
// std::cout << "add index, index name =" << index.index_name() << ", field_name = " << index.field_name()
// << std::endl;
// AddIndex(index);
// }
}
#endif
void
Collection::parse() {
@ -29,8 +126,8 @@ Collection::parse() {
// TODO: remove hard code use unittests are ready
std::cout << "WARN: Use default schema" << std::endl;
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddDebugField("age", DataType::INT32);
schema->AddField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddField("age", DataType::INT32);
collection_name_ = "default-collection";
schema_ = schema;
return;

View File

@ -12,6 +12,7 @@
#pragma once
#include "common/Schema.h"
#include "IndexMeta.h"
#include <string>
#include <memory>
@ -30,12 +31,18 @@ class Collection {
return schema_;
}
IndexMetaPtr&
get_index() {
return index_;
}
const std::string&
get_collection_name() {
return collection_name_;
}
private:
IndexMetaPtr index_;
std::string collection_name_;
std::string schema_proto_;
SchemaPtr schema_;

View File

@ -8,7 +8,7 @@
// 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
#if 0
#include "IndexMeta.h"
#include <mutex>
#include <cassert>
@ -16,8 +16,7 @@ namespace milvus::segcore {
Status
IndexMeta::AddEntry(
const std::string& index_name, const std::string& field_name_, IndexType type, IndexMode mode, IndexConfig config) {
auto field_name = FieldName(field_name_);
const std::string& index_name, const std::string& field_name, IndexType type, IndexMode mode, IndexConfig config) {
Entry entry{index_name, field_name, type, mode, std::move(config)};
VerifyEntry(entry);
@ -60,5 +59,3 @@ IndexMeta::VerifyEntry(const Entry& entry) {
}
} // namespace milvus::segcore
#endif

View File

@ -15,7 +15,6 @@
//
//#include "common/Schema.h"
// #include "segcore/SegmentBase.h"
#if 0
#include "common/Schema.h"
#include "knowhere/index/IndexType.h"
#include "knowhere/common/Config.h"
@ -34,7 +33,7 @@ class IndexMeta {
struct Entry {
std::string index_name;
FieldName field_name;
std::string field_name;
IndexType type;
IndexMode mode;
IndexConfig config;
@ -56,8 +55,8 @@ class IndexMeta {
}
const Entry&
lookup_by_field(const FieldName& field_name) {
AssertInfo(lookups_.count(field_name), field_name.get());
lookup_by_field(const std::string& field_name) {
AssertInfo(lookups_.count(field_name), field_name);
auto index_name = lookups_.at(field_name);
AssertInfo(entries_.count(index_name), index_name);
return entries_.at(index_name);
@ -70,10 +69,8 @@ class IndexMeta {
private:
SchemaPtr schema_;
std::map<std::string, Entry> entries_; // index_name => Entry
std::map<FieldName, std::string> lookups_; // field_name => index_name
std::map<std::string, std::string> lookups_; // field_name => index_name
};
using IndexMetaPtr = std::shared_ptr<IndexMeta>;
} // namespace milvus::segcore
#endif

View File

@ -75,8 +75,8 @@ IndexingRecord::UpdateResourceAck(int64_t chunk_ack, const InsertRecord& record)
// std::thread([this, old_ack, chunk_ack, &record] {
for (auto& [field_offset, entry] : entries_) {
auto vec_base = record.get_base_entity(field_offset);
entry->BuildIndexRange(old_ack, chunk_ack, vec_base.get());
auto vec_base = record.entity_vec_[field_offset].get();
entry->BuildIndexRange(old_ack, chunk_ack, vec_base);
}
finished_ack_.AddSegment(old_ack, chunk_ack);
// }).detach();

View File

@ -108,7 +108,7 @@ class IndexingRecord {
int offset = 0;
for (auto& field : schema_) {
if (field.get_data_type() != DataType::VECTOR_BINARY) {
entries_.try_emplace(FieldOffset(offset), CreateIndex(field, chunk_size_));
entries_.try_emplace(offset, CreateIndex(field, chunk_size_));
}
++offset;
}
@ -126,13 +126,13 @@ class IndexingRecord {
}
const IndexingEntry&
get_entry(FieldOffset field_offset) const {
get_entry(int field_offset) const {
assert(entries_.count(field_offset));
return *entries_.at(field_offset);
}
const VecIndexingEntry&
get_vec_entry(FieldOffset field_offset) const {
get_vec_entry(int field_offset) const {
auto& entry = get_entry(field_offset);
auto ptr = dynamic_cast<const VecIndexingEntry*>(&entry);
AssertInfo(ptr, "invalid indexing");
@ -140,7 +140,7 @@ class IndexingRecord {
}
template <typename T>
auto
get_scalar_entry(FieldOffset field_offset) const -> const ScalarIndexingEntry<T>& {
get_scalar_entry(int field_offset) const -> const ScalarIndexingEntry<T>& {
auto& entry = get_entry(field_offset);
auto ptr = dynamic_cast<const ScalarIndexingEntry<T>*>(&entry);
AssertInfo(ptr, "invalid indexing");
@ -160,7 +160,7 @@ class IndexingRecord {
private:
// field_offset => indexing
std::map<FieldOffset, std::unique_ptr<IndexingEntry>> entries_;
std::map<int, std::unique_ptr<IndexingEntry>> entries_;
};
} // namespace milvus::segcore

View File

@ -17,10 +17,10 @@ InsertRecord::InsertRecord(const Schema& schema, int64_t chunk_size) : uids_(1),
for (auto& field : schema) {
if (field.is_vector()) {
if (field.get_data_type() == DataType::VECTOR_FLOAT) {
this->insert_entity<FloatVector>(field.get_dim(), chunk_size);
entity_vec_.emplace_back(std::make_shared<ConcurrentVector<FloatVector>>(field.get_dim(), chunk_size));
continue;
} else if (field.get_data_type() == DataType::VECTOR_BINARY) {
this->insert_entity<BinaryVector>(field.get_dim(), chunk_size);
entity_vec_.emplace_back(std::make_shared<ConcurrentVector<BinaryVector>>(field.get_dim(), chunk_size));
continue;
} else {
PanicInfo("unsupported");
@ -28,34 +28,34 @@ InsertRecord::InsertRecord(const Schema& schema, int64_t chunk_size) : uids_(1),
}
switch (field.get_data_type()) {
case DataType::BOOL: {
this->insert_entity<bool>(chunk_size);
entity_vec_.emplace_back(std::make_shared<ConcurrentVector<bool>>(chunk_size));
break;
}
case DataType::INT8: {
this->insert_entity<int8_t>(chunk_size);
entity_vec_.emplace_back(std::make_shared<ConcurrentVector<int8_t>>(chunk_size));
break;
}
case DataType::INT16: {
this->insert_entity<int16_t>(chunk_size);
entity_vec_.emplace_back(std::make_shared<ConcurrentVector<int16_t>>(chunk_size));
break;
}
case DataType::INT32: {
this->insert_entity<int32_t>(chunk_size);
entity_vec_.emplace_back(std::make_shared<ConcurrentVector<int32_t>>(chunk_size));
break;
}
case DataType::INT64: {
this->insert_entity<int64_t>(chunk_size);
entity_vec_.emplace_back(std::make_shared<ConcurrentVector<int64_t>>(chunk_size));
break;
}
case DataType::FLOAT: {
this->insert_entity<float>(chunk_size);
entity_vec_.emplace_back(std::make_shared<ConcurrentVector<float>>(chunk_size));
break;
}
case DataType::DOUBLE: {
this->insert_entity<double>(chunk_size);
entity_vec_.emplace_back(std::make_shared<ConcurrentVector<double>>(chunk_size));
break;
}
default: {

View File

@ -23,48 +23,23 @@ struct InsertRecord {
AckResponder ack_responder_;
ConcurrentVector<Timestamp> timestamps_;
ConcurrentVector<idx_t> uids_;
std::vector<std::shared_ptr<VectorBase>> entity_vec_;
explicit InsertRecord(const Schema& schema, int64_t chunk_size);
auto
get_base_entity(FieldOffset field_offset) const {
auto ptr = entity_vec_[field_offset.get()];
return ptr;
}
template <typename Type>
auto
get_entity(FieldOffset field_offset) const {
auto base_ptr = get_base_entity(field_offset);
auto ptr = std::dynamic_pointer_cast<const ConcurrentVector<Type>>(base_ptr);
get_entity(int offset) const {
auto ptr = std::dynamic_pointer_cast<const ConcurrentVector<Type>>(entity_vec_[offset]);
Assert(ptr);
return ptr;
}
template <typename Type>
auto
get_entity(FieldOffset field_offset) {
auto base_ptr = get_base_entity(field_offset);
auto ptr = std::dynamic_pointer_cast<ConcurrentVector<Type>>(base_ptr);
get_entity(int offset) {
auto ptr = std::dynamic_pointer_cast<ConcurrentVector<Type>>(entity_vec_[offset]);
Assert(ptr);
return ptr;
}
template <typename Type>
void
insert_entity(int64_t chunk_size) {
static_assert(std::is_fundamental_v<Type>);
entity_vec_.emplace_back(std::make_shared<ConcurrentVector<Type>>(chunk_size));
}
template <typename VectorType>
void
insert_entity(int64_t dim, int64_t chunk_size) {
static_assert(std::is_base_of_v<VectorTrait, VectorType>);
entity_vec_.emplace_back(std::make_shared<ConcurrentVector<VectorType>>(dim, chunk_size));
}
private:
std::vector<std::shared_ptr<VectorBase>> entity_vec_;
};
} // namespace milvus::segcore

View File

@ -31,27 +31,27 @@ using SealedIndexingEntryPtr = std::unique_ptr<SealedIndexingEntry>;
struct SealedIndexingRecord {
void
add_entry(FieldOffset field_offset, SealedIndexingEntryPtr&& ptr) {
add_entry(int64_t field_offset, SealedIndexingEntryPtr&& ptr) {
std::unique_lock lck(mutex_);
entries_[field_offset] = std::move(ptr);
}
const SealedIndexingEntry*
get_entry(FieldOffset field_offset) const {
get_entry(int64_t field_offset) const {
std::shared_lock lck(mutex_);
AssertInfo(entries_.count(field_offset), "field_offset not found");
return entries_.at(field_offset).get();
}
bool
is_ready(FieldOffset field_offset) const {
test_readiness(int64_t field_offset) const {
std::shared_lock lck(mutex_);
return entries_.count(field_offset);
}
private:
// field_offset -> SealedIndexingEntry
std::map<FieldOffset, SealedIndexingEntryPtr> entries_;
std::map<int64_t, SealedIndexingEntryPtr> entries_;
mutable std::shared_mutex mutex_;
};
} // namespace milvus::segcore

View File

@ -12,6 +12,7 @@
#pragma once
#include <vector>
#include "IndexMeta.h"
#include "common/Types.h"
#include "common/Schema.h"
#include <memory>
@ -82,6 +83,23 @@ class SegmentBase {
virtual Status
LoadIndexing(const LoadIndexInfo& info) = 0;
// // to make all data inserted visible
// // maybe a no-op?
// virtual Status
// Flush(Timestamp timestamp) = 0;
// watch changes
// NOTE: Segment will use this ptr as correct
virtual Status
DropRawData(std::string_view field_name) = 0;
virtual Status
LoadRawData(std::string_view field_name, const char* blob, int64_t blob_size) = 0;
virtual Status
BuildIndex(IndexMetaPtr index_meta) = 0;
virtual int64_t
GetMemoryUsageInBytes() = 0;

View File

@ -8,9 +8,8 @@
// 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
#if 0
#include <segcore/deprecated/SegmentNaive.h>
#include <segcore/SegmentNaive.h>
#include <random>
#include <algorithm>
#include <numeric>
@ -240,17 +239,20 @@ SegmentNaive::QueryImpl(query::QueryDeprecatedPtr query_info, Timestamp timestam
Assert(bitmap_holder);
Assert(bitmap_holder->bitmap_ptr->count() == ins_barrier);
auto field_name = FieldName(query_info->field_name);
auto field_offset = schema_->get_offset(field_name);
auto& field = schema_->operator[](field_name);
auto field_offset = schema_->get_offset(query_info->field_name);
auto& field = schema_->operator[](query_info->field_name);
Assert(field.get_data_type() == DataType::VECTOR_FLOAT);
auto dim = field.get_dim();
auto bitmap = bitmap_holder->bitmap_ptr;
auto topK = query_info->topK;
auto num_queries = query_info->num_queries;
auto vec_ptr = std::static_pointer_cast<ConcurrentVector<FloatVector>>(record_.entity_vec_.at(field_offset));
auto index_entry = index_meta_->lookup_by_field(field_name);
auto the_offset_opt = schema_->get_offset(query_info->field_name);
Assert(the_offset_opt.has_value());
Assert(the_offset_opt.value() < record_.entity_vec_.size());
auto vec_ptr =
std::static_pointer_cast<ConcurrentVector<FloatVector>>(record_.entity_vec_.at(the_offset_opt.value()));
auto index_entry = index_meta_->lookup_by_field(query_info->field_name);
auto conf = index_entry.config;
conf[milvus::knowhere::meta::TOPK] = query_info->topK;
@ -294,17 +296,19 @@ SegmentNaive::QuerySlowImpl(query::QueryDeprecatedPtr query_info, Timestamp time
auto del_barrier = get_barrier(deleted_record_, timestamp);
auto bitmap_holder = get_deleted_bitmap(del_barrier, timestamp, ins_barrier);
Assert(bitmap_holder);
auto field_name = FieldName(query_info->field_name);
auto& field = schema_->operator[](field_name);
auto& field = schema_->operator[](query_info->field_name);
Assert(field.get_data_type() == DataType::VECTOR_FLOAT);
auto dim = field.get_dim();
auto bitmap = bitmap_holder->bitmap_ptr;
auto topK = query_info->topK;
auto num_queries = query_info->num_queries;
// TODO: optimize
auto field_offset = schema_->get_offset(field_name);
Assert(field_offset < record_.entity_vec_.size());
auto vec_ptr = std::static_pointer_cast<ConcurrentVector<FloatVector>>(record_.entity_vec_.at(field_offset));
auto the_offset_opt = schema_->get_offset(query_info->field_name);
Assert(the_offset_opt.has_value());
Assert(the_offset_opt.value() < record_.entity_vec_.size());
auto vec_ptr =
std::static_pointer_cast<ConcurrentVector<FloatVector>>(record_.entity_vec_.at(the_offset_opt.value()));
std::vector<std::priority_queue<std::pair<float, int>>> records(num_queries);
auto get_L2_distance = [dim](const float* a, const float* b) {
@ -366,7 +370,7 @@ SegmentNaive::QueryDeprecated(query::QueryDeprecatedPtr query_info, Timestamp ti
query_info->topK = 10;
query_info->num_queries = 1;
auto dim = schema_->operator[](FieldName("fakevec")).get_dim();
auto dim = schema_->operator[]("fakevec").get_dim();
std::default_random_engine e(42);
std::uniform_real_distribution<> dis(0.0, 1.0);
query_info->query_raw_data.resize(query_info->num_queries * dim);
@ -411,7 +415,7 @@ SegmentNaive::BuildIndex(IndexMetaPtr remote_index_meta) {
for (auto& field : schema_->get_fields()) {
if (field.get_data_type() == DataType::VECTOR_FLOAT) {
dim = field.get_dim();
index_field_name = field.get_name().get();
index_field_name = field.get_name();
}
}
@ -460,5 +464,3 @@ SegmentNaive::GetMemoryUsageInBytes() {
}
} // namespace milvus::segcore
#endif

View File

@ -10,7 +10,6 @@
// or implied. See the License for the specific language governing permissions and limitations under the License
#pragma once
#if 0
#include <tbb/concurrent_priority_queue.h>
#include <tbb/concurrent_unordered_map.h>
@ -163,5 +162,3 @@ class SegmentNaive : public SegmentBase {
tbb::concurrent_unordered_multimap<idx_t, int64_t> uid2offset_;
};
} // namespace milvus::segcore
#endif

View File

@ -168,8 +168,7 @@ SegmentSmallIndex::Insert(int64_t reserved_begin,
record_.timestamps_.set_data(reserved_begin, timestamps.data(), size);
record_.uids_.set_data(reserved_begin, uids.data(), size);
for (int fid = 0; fid < schema_->size(); ++fid) {
auto field_offset = FieldOffset(fid);
record_.get_base_entity(field_offset)->set_data_raw(reserved_begin, entities[fid].data(), size);
record_.entity_vec_[fid]->set_data_raw(reserved_begin, entities[fid].data(), size);
}
for (int i = 0; i < uids.size(); ++i) {
@ -227,9 +226,103 @@ SegmentSmallIndex::Close() {
return Status::OK();
}
template <typename Type>
knowhere::IndexPtr
SegmentSmallIndex::BuildVecIndexImpl(const IndexMeta::Entry& entry) {
auto offset_opt = schema_->get_offset(entry.field_name);
Assert(offset_opt.has_value());
auto offset = offset_opt.value();
auto field = (*schema_)[offset];
auto dim = field.get_dim();
auto indexing = knowhere::VecIndexFactory::GetInstance().CreateVecIndex(entry.type, entry.mode);
auto& uids = record_.uids_;
auto entities = record_.get_entity<FloatVector>(offset);
std::vector<knowhere::DatasetPtr> datasets;
for (int chunk_id = 0; chunk_id < uids.num_chunk(); ++chunk_id) {
auto entities_chunk = entities->get_chunk(chunk_id).data();
int64_t count = chunk_id == uids.num_chunk() - 1 ? record_.reserved - chunk_id * chunk_size_ : chunk_size_;
datasets.push_back(knowhere::GenDataset(count, dim, entities_chunk));
}
for (auto& ds : datasets) {
indexing->Train(ds, entry.config);
}
for (auto& ds : datasets) {
indexing->AddWithoutIds(ds, entry.config);
}
return indexing;
}
Status
SegmentSmallIndex::BuildIndex(IndexMetaPtr remote_index_meta) {
if (remote_index_meta == nullptr) {
PanicInfo("deprecated");
std::cout << "WARN: Null index ptr is detected, use default index" << std::endl;
int dim = 0;
std::string index_field_name;
for (auto& field : schema_->get_fields()) {
if (field.get_data_type() == DataType::VECTOR_FLOAT) {
dim = field.get_dim();
index_field_name = field.get_name();
}
}
Assert(dim != 0);
Assert(!index_field_name.empty());
auto index_meta = std::make_shared<IndexMeta>(schema_);
// TODO: this is merge of query conf and insert conf
// TODO: should be split into multiple configs
auto conf = milvus::knowhere::Config{
{milvus::knowhere::meta::DIM, dim}, {milvus::knowhere::IndexParams::nlist, 100},
{milvus::knowhere::IndexParams::nprobe, 4}, {milvus::knowhere::IndexParams::m, 4},
{milvus::knowhere::IndexParams::nbits, 8}, {milvus::knowhere::Metric::TYPE, milvus::knowhere::Metric::L2},
{milvus::knowhere::meta::DEVICEID, 0},
};
index_meta->AddEntry("fakeindex", index_field_name, knowhere::IndexEnum::INDEX_FAISS_IVFPQ,
knowhere::IndexMode::MODE_CPU, conf);
remote_index_meta = index_meta;
}
if (record_.ack_responder_.GetAck() < 1024 * 4) {
return Status(SERVER_BUILD_INDEX_ERROR, "too few elements");
}
PanicInfo("unimplemented");
#if 0
index_meta_ = remote_index_meta;
for (auto& [index_name, entry] : index_meta_->get_entries()) {
Assert(entry.index_name == index_name);
const auto& field = (*schema_)[entry.field_name];
if (field.is_vector()) {
Assert(field.get_data_type() == engine::DataType::VECTOR_FLOAT);
auto index_ptr = BuildVecIndexImpl<float>(entry);
indexings_[index_name] = index_ptr;
} else {
throw std::runtime_error("unimplemented");
}
}
index_ready_ = true;
return Status::OK();
#endif
}
int64_t
SegmentSmallIndex::GetMemoryUsageInBytes() {
int64_t total_bytes = 0;
#if 0
if (index_ready_) {
auto& index_entries = index_meta_->get_entries();
for (auto [index_name, entry] : index_entries) {
Assert(schema_->operator[](entry.field_name).is_vector());
auto vec_ptr = std::static_pointer_cast<knowhere::VecIndex>(indexings_[index_name]);
total_bytes += vec_ptr->IndexSize();
}
}
#endif
int64_t ins_n = upper_align(record_.reserved, chunk_size_);
total_bytes += ins_n * (schema_->get_total_sizeof() + 16 + 1);
int64_t del_n = upper_align(deleted_record_.reserved, chunk_size_);
@ -287,7 +380,8 @@ SegmentSmallIndex::FillTargetEntry(const query::Plan* plan, QueryResult& results
Status
SegmentSmallIndex::LoadIndexing(const LoadIndexInfo& info) {
auto field_offset = schema_->get_offset(FieldName(info.field_name));
auto field_offset_opt = schema_->get_offset(info.field_name);
AssertInfo(field_offset_opt.has_value(), "field name(" + info.field_name + ") not found");
Assert(info.index_params.count("metric_type"));
auto metric_type_str = info.index_params.at("metric_type");
@ -296,7 +390,7 @@ SegmentSmallIndex::LoadIndexing(const LoadIndexInfo& info) {
entry->metric_type_ = GetMetricType(metric_type_str);
entry->indexing_ = info.index;
sealed_indexing_record_.add_entry(field_offset, std::move(entry));
sealed_indexing_record_.add_entry(field_offset_opt.value(), std::move(entry));
return Status::OK();
}

View File

@ -67,6 +67,26 @@ class SegmentSmallIndex : public SegmentBase {
Status
Close() override;
// using IndexType = knowhere::IndexType;
// using IndexMode = knowhere::IndexMode;
// using IndexConfig = knowhere::Config;
// BuildIndex With Paramaters, must with Frozen State
// NOTE: index_params contains several policies for several index
// TODO: currently, index has to be set at startup, and can't be modified
// AddIndex and DropIndex will be added later
Status
BuildIndex(IndexMetaPtr index_meta) override;
Status
DropRawData(std::string_view field_name) override {
PanicInfo("unimplemented");
}
Status
LoadRawData(std::string_view field_name, const char* blob, int64_t blob_size) override {
PanicInfo("unimplemented");
}
int64_t
GetMemoryUsageInBytes() override;
@ -130,6 +150,10 @@ class SegmentSmallIndex : public SegmentBase {
std::shared_ptr<DeletedRecord::TmpBitmap>
get_deleted_bitmap(int64_t del_barrier, Timestamp query_timestamp, int64_t insert_barrier, bool force = false);
template <typename Type>
knowhere::IndexPtr
BuildVecIndexImpl(const IndexMeta::Entry& entry);
Status
FillTargetEntry(const query::Plan* Plan, QueryResult& results) override;
@ -137,6 +161,7 @@ class SegmentSmallIndex : public SegmentBase {
int64_t chunk_size_;
SchemaPtr schema_;
std::atomic<SegmentState> state_ = SegmentState::Open;
IndexMetaPtr index_meta_;
InsertRecord record_;
DeletedRecord deleted_record_;

View File

@ -206,7 +206,11 @@ Close(CSegmentBase c_segment) {
int
BuildIndex(CCollection c_collection, CSegmentBase c_segment) {
PanicInfo("unimplemented");
auto collection = (milvus::segcore::Collection*)c_collection;
auto segment = (milvus::segcore::SegmentBase*)c_segment;
auto status = segment->BuildIndex(collection->get_index());
return status.code();
}
bool

View File

@ -1,110 +0,0 @@
Language: Cpp
AccessModifierOffset: -4
AlignAfterOpenBracket: AlwaysBreak
AlignConsecutiveAssignments: false
AlignConsecutiveDeclarations: false
AlignEscapedNewlines: Right
AlignOperands: false
AlignTrailingComments: true
AllowAllParametersOfDeclarationOnNextLine: false
AllowShortBlocksOnASingleLine: false
AllowShortCaseLabelsOnASingleLine: false
AllowShortFunctionsOnASingleLine: None
AllowShortIfStatementsOnASingleLine: false
AllowShortLoopsOnASingleLine: false
AlwaysBreakAfterDefinitionReturnType: None
AlwaysBreakAfterReturnType: None
AlwaysBreakBeforeMultilineStrings: false
AlwaysBreakTemplateDeclarations: Yes
BinPackArguments: false
BinPackParameters: false
BraceWrapping:
AfterClass: true
AfterControlStatement: true
AfterEnum: true
AfterFunction: true
AfterNamespace: true
AfterObjCDeclaration: true
AfterStruct: true
AfterUnion: false
AfterExternBlock: true
BeforeCatch: true
BeforeElse: true
IndentBraces: false
SplitEmptyFunction: true
SplitEmptyRecord: true
SplitEmptyNamespace: true
BreakBeforeBinaryOperators: None
BreakBeforeBraces: Allman
BreakBeforeInheritanceComma: false
BreakInheritanceList: BeforeComma
BreakBeforeTernaryOperators: true
BreakConstructorInitializersBeforeComma: false
BreakConstructorInitializers: BeforeColon
BreakAfterJavaFieldAnnotations: false
BreakStringLiterals: true
ColumnLimit: 120
CommentPragmas: '^ IWYU pragma:'
CompactNamespaces: false
ConstructorInitializerAllOnOneLineOrOnePerLine: true
ConstructorInitializerIndentWidth: 4
ContinuationIndentWidth: 4
Cpp11BracedListStyle: true
DerivePointerAlignment: false
DisableFormat: false
ExperimentalAutoDetectBinPacking: false
FixNamespaceComments: true
IncludeBlocks: Preserve
IncludeCategories:
- Regex: '^"(llvm|llvm-c|clang|clang-c)/'
Priority: 2
- Regex: '^(<|"(gtest|gmock|isl|json)/)'
Priority: 3
- Regex: '.*'
Priority: 1
IncludeIsMainRegex: '(Test)?$'
IndentCaseLabels: true
IndentPPDirectives: AfterHash
IndentWidth: 4
IndentWrappedFunctionNames: false
JavaScriptQuotes: Leave
JavaScriptWrapImports: true
KeepEmptyLinesAtTheStartOfBlocks: true
MacroBlockBegin: ''
MacroBlockEnd: ''
MaxEmptyLinesToKeep: 1
NamespaceIndentation: None
ObjCBinPackProtocolList: Auto
ObjCBlockIndentWidth: 2
ObjCSpaceAfterProperty: false
ObjCSpaceBeforeProtocolList: true
PenaltyBreakAssignment: 2
PenaltyBreakBeforeFirstCallParameter: 19
PenaltyBreakComment: 300
PenaltyBreakFirstLessLess: 120
PenaltyBreakString: 1000
PenaltyBreakTemplateDeclaration: 10
PenaltyExcessCharacter: 1000000
PenaltyReturnTypeOnItsOwnLine: 60
PointerAlignment: Left
ReflowComments: true
SortIncludes: true
SortUsingDeclarations: true
SpaceAfterCStyleCast: false
SpaceAfterTemplateKeyword: true
SpaceBeforeAssignmentOperators: true
SpaceBeforeCpp11BracedList: false
SpaceBeforeCtorInitializerColon: true
SpaceBeforeInheritanceColon: true
SpaceBeforeParens: ControlStatements
SpaceBeforeRangeBasedForLoopColon: true
SpaceInEmptyParentheses: false
SpacesBeforeTrailingComments: 1
SpacesInAngles: false
SpacesInContainerLiterals: true
SpacesInCStyleCastParentheses: false
SpacesInParentheses: false
SpacesInSquareBrackets: false
Standard: Cpp11
TabWidth: 4
UseTab: Never

View File

@ -1,92 +0,0 @@
language: generic
dist: bionic
matrix:
include:
- env: CXX=g++-10 CC=gcc-10
addons:
apt:
packages:
- g++-10
sources:
- sourceline: 'ppa:ubuntu-toolchain-r/test'
- env: CXX=g++-9 CC=gcc-9
addons:
apt:
packages:
- g++-9
sources:
- sourceline: 'ppa:ubuntu-toolchain-r/test'
- env: CXX=g++-8 CC=gcc-8
addons:
apt:
packages:
- g++-8
- env: CXX=g++-7 CC=gcc-7
addons:
apt:
packages:
- g++-7
- env: CXX=g++-6 CC=gcc-6
addons:
apt:
packages:
- g++-6
- env: CXX=g++-5 CC=gcc-5
addons:
apt:
packages:
- g++-5
- env: CXX=clang++-11 CC=clang-11
addons:
apt:
packages:
- clang-11
- libc++-11-dev
- libc++abi-11-dev
sources:
- sourceline: 'deb http://apt.llvm.org/bionic/ llvm-toolchain-bionic-11 main'
key_url: 'https://apt.llvm.org/llvm-snapshot.gpg.key'
- env: CXX=clang++-10 CC=clang-10
addons:
apt:
packages:
- clang-10
- libc++-10-dev
- libc++abi-10-dev
sources:
- sourceline: 'deb http://apt.llvm.org/bionic/ llvm-toolchain-bionic-10 main'
key_url: 'https://apt.llvm.org/llvm-snapshot.gpg.key'
- env: CXX=clang++-9 CC=clang-9
addons:
apt:
packages:
- clang-9
- libc++-9-dev
- libc++abi-9-dev
sources:
- sourceline: 'deb http://apt.llvm.org/bionic/ llvm-toolchain-bionic-9 main'
key_url: 'https://apt.llvm.org/llvm-snapshot.gpg.key'
- env: CXX=clang++-8 CC=clang-8
addons:
apt:
packages:
- clang-8
- libc++-8-dev
- libc++abi-8-dev
- env: CXX=clang++-7 CC=clang-7
addons:
apt:
packages:
- clang-7
- libc++-7-dev
- libc++abi-7-dev
script:
- if [[ "$CXX" == clang* ]]; then export CXXFLAGS="-stdlib=libc++"; fi
- mkdir build && cd build
- cmake -DENABLE_TEST=ON ..
- make
- test/NamedTypeTest

View File

@ -1,21 +0,0 @@
MIT License
Copyright (c) 2017 Jonathan Boccara
Permission is hereby granted, free of charge, to any person obtaining a copy
of this software and associated documentation files (the "Software"), to deal
in the Software without restriction, including without limitation the rights
to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
copies of the Software, and to permit persons to whom the Software is
furnished to do so, subject to the following conditions:
The above copyright notice and this permission notice shall be included in all
copies or substantial portions of the Software.
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
SOFTWARE.

View File

@ -1,108 +0,0 @@
[![Build Status](https://travis-ci.org/joboccara/NamedType.svg?branch=master)](https://travis-ci.org/joboccara/NamedType)
![GitHub](https://img.shields.io/github/license/joboccara/pipes)
A **strong type** is a type used in place of another type to carry specific **meaning** through its **name**.
This project experiments with strong types in C++. All components are in the namespace `fluent`. You can find a collection of blog posts explaining the rationale of the library and usages for strong types on [Fluent C++](https://www.fluentcpp.com/2016/12/08/strong-types-for-strong-interfaces/).
<a href="https://www.patreon.com/join/fluentcpp?"><img alt="become a patron" src="https://c5.patreon.com/external/logo/become_a_patron_button.png" height="35px"></a>
## Basic usage
It central piece is the templated class NamedType, which can be used to declare a strong type with a typedef-like syntax:
```cpp
using Width = NamedType<double, struct WidthTag>;
using Height = NamedType<double, struct HeightTag>;
```
which can be used to make interfaces more expressive and more robust.
Note how the below constructor shows in which order it expects its parameters:
```cpp
class Rectangle
{
public:
Rectangle(Width width, Height height) : width_(width.get()), height_(height.get()) {}
double getWidth() const { return width_; }
double getHeight() const { return height_; }
private:
double width_;
double height_;
};
```
**Strong types are about better expressing your intentions, both to the compiler and to other human developers.**
## Strong typing over generic types
This implementation of strong types can be used to add strong typing over generic or unknown types such as lambdas:
```cpp
template<typename Function>
using Comparator = NamedType<Function, struct ComparatorTag>;
template <typename Function>
void performAction(Comparator<Function> comp)
{
comp.get()();
}
performAction(make_named<Comparator>([](){ std::cout << "compare\n"; }));
```
## Strong typing over references
The NamedType class is designed so that the following usage:
```cpp
using FamilyNameRef = NamedType<std:string&, struct FamilyNameRefTag>;
```
behaves like a reference on an std::string, strongly typed.
## Inheriting the underlying type functionalities
You can declare which functionalities should be inherited from the underlying type. So far, only basic operators are taken into account.
For instance, to inherit from operator+ and operator<<, you can declare the strong type:
```cpp
using Meter = NamedType<double, MeterTag, Addable, Printable>
```
There is one special skill, `FunctionCallable`, that lets the strong type be converted in the underlying type. This has the effect of removing the need to call .get() to get the underlying value. And `MethodCallable` enables `operator->` on the strong type to invoke methods on the underlying type.
The skill `Callable` is the union of `FunctionCallable` and `MethodCallable`.
## Named arguments
By their nature strong types can play the role of named parameters:
```cpp
using FirstName = NamedType<std::string, struct FirstNameTag>;
using LastName = NamedType<std::string, struct LastNameTag>;
void displayName(FirstName const& theFirstName, LastName const& theLastName);
// Call site
displayName(FirstName("John"), LastName("Doe"));
```
But the nested type `argument` allows to emulate a named argument syntax:
```cpp
using FirstName = NamedType<std::string, struct FirstNameTag>;
using LastName = NamedType<std::string, struct LastNameTag>;
static const FirstName::argument firstName;
static const LastName::argument lastName;
void displayName(FirstName const& theFirstName, LastName const& theLastName);
// Call site
displayName(firstName = "John", lastName = "Doe");
```
You can have a look at tests.cpp for usage examples.
<a href="https://www.patreon.com/join/fluentcpp?"><img alt="become a patron" src="https://c5.patreon.com/external/logo/become_a_patron_button.png" height="35px"></a>

View File

@ -1,22 +0,0 @@
#ifndef CRTP_HPP
#define CRTP_HPP
namespace fluent
{
template <typename T, template <typename> class crtpType>
struct crtp
{
constexpr T& underlying()
{
return static_cast<T&>(*this);
}
constexpr T const& underlying() const
{
return static_cast<T const&>(*this);
}
};
} // namespace fluent
#endif

View File

@ -1,8 +0,0 @@
#ifndef NAMED_TYPE_HPP
#define NAMED_TYPE_HPP
#include "named_type_impl.hpp"
#include "underlying_functionalities.hpp"
#include "version.hpp"
#endif

View File

@ -1,146 +0,0 @@
#ifndef named_type_impl_h
#define named_type_impl_h
#include <tuple>
#include <type_traits>
#include <utility>
// C++17 detection
#if defined(_MSC_VER) && (defined(_HAS_CXX17) && _HAS_CXX17)
# define FLUENT_CPP17_PRESENT 1
#elif __cplusplus >= 201703L
# define FLUENT_CPP17_PRESENT 1
#else
# define FLUENT_CPP17_PRESENT 0
#endif
// Use [[nodiscard]] if available
#ifndef FLUENT_NODISCARD_PRESENT
# define FLUENT_NODISCARD_PRESENT FLUENT_CPP17_PRESENT
#endif
#if FLUENT_NODISCARD_PRESENT
# define FLUENT_NODISCARD [[nodiscard]]
#else
# define FLUENT_NODISCARD
#endif
// Enable empty base class optimization with multiple inheritance on Visual Studio.
#if defined(_MSC_VER) && _MSC_VER >= 1910
# define FLUENT_EBCO __declspec(empty_bases)
#else
# define FLUENT_EBCO
#endif
#if defined(__clang__) || defined(__GNUC__)
# define IGNORE_SHOULD_RETURN_REFERENCE_TO_THIS_BEGIN \
_Pragma("GCC diagnostic push") _Pragma("GCC diagnostic ignored \"-Weffc++\"")
# define IGNORE_SHOULD_RETURN_REFERENCE_TO_THIS_END _Pragma("GCC diagnostic pop")
#else
# define IGNORE_SHOULD_RETURN_REFERENCE_TO_THIS_BEGIN /* Nothing */
# define IGNORE_SHOULD_RETURN_REFERENCE_TO_THIS_END /* Nothing */
#endif
namespace fluent
{
template <typename T>
using IsNotReference = typename std::enable_if<!std::is_reference<T>::value, void>::type;
template <typename T, typename Parameter, template <typename> class... Skills>
class FLUENT_EBCO NamedType : public Skills<NamedType<T, Parameter, Skills...>>...
{
public:
using UnderlyingType = T;
// constructor
NamedType() = default;
explicit constexpr NamedType(T const& value) noexcept(std::is_nothrow_copy_constructible<T>::value) : value_(value)
{
}
template <typename T_ = T, typename = IsNotReference<T_>>
explicit constexpr NamedType(T&& value) noexcept(std::is_nothrow_move_constructible<T>::value)
: value_(std::move(value))
{
}
// get
FLUENT_NODISCARD constexpr T& get() noexcept
{
return value_;
}
FLUENT_NODISCARD constexpr std::remove_reference_t<T> const& get() const noexcept
{
return value_;
}
// conversions
using ref = NamedType<T&, Parameter, Skills...>;
operator ref()
{
return ref(value_);
}
struct argument
{
NamedType operator=(T&& value) const
{
IGNORE_SHOULD_RETURN_REFERENCE_TO_THIS_BEGIN
return NamedType(std::forward<T>(value));
IGNORE_SHOULD_RETURN_REFERENCE_TO_THIS_END
}
template <typename U>
NamedType operator=(U&& value) const
{
IGNORE_SHOULD_RETURN_REFERENCE_TO_THIS_BEGIN
return NamedType(std::forward<U>(value));
IGNORE_SHOULD_RETURN_REFERENCE_TO_THIS_END
}
argument() = default;
argument(argument const&) = delete;
argument(argument&&) = delete;
argument& operator=(argument const&) = delete;
argument& operator=(argument&&) = delete;
};
private:
T value_;
};
template <template <typename T> class StrongType, typename T>
constexpr StrongType<T> make_named(T const& value)
{
return StrongType<T>(value);
}
namespace details {
template <class F, class... Ts>
struct AnyOrderCallable{
F f;
template <class... Us>
auto operator()(Us&&...args) const
{
static_assert(sizeof...(Ts) == sizeof...(Us), "Passing wrong number of arguments");
auto x = std::make_tuple(std::forward<Us>(args)...);
return f(std::move(std::get<Ts>(x))...);
}
};
} //namespace details
// EXPERIMENTAL - CAN BE CHANGED IN THE FUTURE. FEEDBACK WELCOME FOR IMPROVEMENTS!
template <class... Args, class F>
auto make_named_arg_function(F&& f)
{
return details::AnyOrderCallable<F, Args...>{std::forward<F>(f)};
}
} // namespace fluent
#endif /* named_type_impl_h */

View File

@ -1,430 +0,0 @@
#ifndef UNDERLYING_FUNCTIONALITIES_HPP
#define UNDERLYING_FUNCTIONALITIES_HPP
#include "crtp.hpp"
#include "named_type_impl.hpp"
#include <functional>
#include <iostream>
#include <memory>
// C++17 constexpr additions
#if FLUENT_CPP17_PRESENT
# define FLUENT_CONSTEXPR17 constexpr
#else
# define FLUENT_CONSTEXPR17
#endif
namespace fluent
{
template <typename T>
struct PreIncrementable : crtp<T, PreIncrementable>
{
IGNORE_SHOULD_RETURN_REFERENCE_TO_THIS_BEGIN
FLUENT_CONSTEXPR17 T& operator++()
{
++this->underlying().get();
return this->underlying();
}
IGNORE_SHOULD_RETURN_REFERENCE_TO_THIS_END
};
template <typename T>
struct PostIncrementable : crtp<T, PostIncrementable>
{
IGNORE_SHOULD_RETURN_REFERENCE_TO_THIS_BEGIN
FLUENT_CONSTEXPR17 T operator++(int)
{
return T(this->underlying().get()++);
}
IGNORE_SHOULD_RETURN_REFERENCE_TO_THIS_END
};
template <typename T>
struct PreDecrementable : crtp<T, PreDecrementable>
{
IGNORE_SHOULD_RETURN_REFERENCE_TO_THIS_BEGIN
FLUENT_CONSTEXPR17 T& operator--()
{
--this->underlying().get();
return this->underlying();
}
IGNORE_SHOULD_RETURN_REFERENCE_TO_THIS_END
};
template <typename T>
struct PostDecrementable : crtp<T, PostDecrementable>
{
IGNORE_SHOULD_RETURN_REFERENCE_TO_THIS_BEGIN
FLUENT_CONSTEXPR17 T operator--(int)
{
return T( this->underlying().get()-- );
}
IGNORE_SHOULD_RETURN_REFERENCE_TO_THIS_END
};
template <typename T>
struct BinaryAddable : crtp<T, BinaryAddable>
{
FLUENT_NODISCARD constexpr T operator+(T const& other) const
{
return T(this->underlying().get() + other.get());
}
FLUENT_CONSTEXPR17 T& operator+=(T const& other)
{
this->underlying().get() += other.get();
return this->underlying();
}
};
template <typename T>
struct UnaryAddable : crtp<T, UnaryAddable>
{
FLUENT_NODISCARD constexpr T operator+() const
{
return T(+this->underlying().get());
}
};
template <typename T>
struct Addable : BinaryAddable<T>, UnaryAddable<T>
{
using BinaryAddable<T>::operator+;
using UnaryAddable<T>::operator+;
};
template <typename T>
struct BinarySubtractable : crtp<T, BinarySubtractable>
{
FLUENT_NODISCARD constexpr T operator-(T const& other) const
{
return T(this->underlying().get() - other.get());
}
FLUENT_CONSTEXPR17 T& operator-=(T const& other)
{
this->underlying().get() -= other.get();
return this->underlying();
}
};
template <typename T>
struct UnarySubtractable : crtp<T, UnarySubtractable>
{
FLUENT_NODISCARD constexpr T operator-() const
{
return T(-this->underlying().get());
}
};
template <typename T>
struct Subtractable : BinarySubtractable<T>, UnarySubtractable<T>
{
using UnarySubtractable<T>::operator-;
using BinarySubtractable<T>::operator-;
};
template <typename T>
struct Multiplicable : crtp<T, Multiplicable>
{
FLUENT_NODISCARD constexpr T operator*(T const& other) const
{
return T(this->underlying().get() * other.get());
}
FLUENT_CONSTEXPR17 T& operator*=(T const& other)
{
this->underlying().get() *= other.get();
return this->underlying();
}
};
template <typename T>
struct Divisible : crtp<T, Divisible>
{
FLUENT_NODISCARD constexpr T operator/(T const& other) const
{
return T(this->underlying().get() / other.get());
}
FLUENT_CONSTEXPR17 T& operator/=(T const& other)
{
this->underlying().get() /= other.get();
return this->underlying();
}
};
template <typename T>
struct Modulable : crtp<T, Modulable>
{
FLUENT_NODISCARD constexpr T operator%(T const& other) const
{
return T(this->underlying().get() % other.get());
}
FLUENT_CONSTEXPR17 T& operator%=(T const& other)
{
this->underlying().get() %= other.get();
return this->underlying();
}
};
template <typename T>
struct BitWiseInvertable : crtp<T, BitWiseInvertable>
{
FLUENT_NODISCARD constexpr T operator~() const
{
return T(~this->underlying().get());
}
};
template <typename T>
struct BitWiseAndable : crtp<T, BitWiseAndable>
{
FLUENT_NODISCARD constexpr T operator&(T const& other) const
{
return T(this->underlying().get() & other.get());
}
FLUENT_CONSTEXPR17 T& operator&=(T const& other)
{
this->underlying().get() &= other.get();
return this->underlying();
}
};
template <typename T>
struct BitWiseOrable : crtp<T, BitWiseOrable>
{
FLUENT_NODISCARD constexpr T operator|(T const& other) const
{
return T(this->underlying().get() | other.get());
}
FLUENT_CONSTEXPR17 T& operator|=(T const& other)
{
this->underlying().get() |= other.get();
return this->underlying();
}
};
template <typename T>
struct BitWiseXorable : crtp<T, BitWiseXorable>
{
FLUENT_NODISCARD constexpr T operator^(T const& other) const
{
return T(this->underlying().get() ^ other.get());
}
FLUENT_CONSTEXPR17 T& operator^=(T const& other)
{
this->underlying().get() ^= other.get();
return this->underlying();
}
};
template <typename T>
struct BitWiseLeftShiftable : crtp<T, BitWiseLeftShiftable>
{
FLUENT_NODISCARD constexpr T operator<<(T const& other) const
{
return T(this->underlying().get() << other.get());
}
FLUENT_CONSTEXPR17 T& operator<<=(T const& other)
{
this->underlying().get() <<= other.get();
return this->underlying();
}
};
template <typename T>
struct BitWiseRightShiftable : crtp<T, BitWiseRightShiftable>
{
FLUENT_NODISCARD constexpr T operator>>(T const& other) const
{
return T(this->underlying().get() >> other.get());
}
FLUENT_CONSTEXPR17 T& operator>>=(T const& other)
{
this->underlying().get() >>= other.get();
return this->underlying();
}
};
template <typename T>
struct Comparable : crtp<T, Comparable>
{
FLUENT_NODISCARD constexpr bool operator<(T const& other) const
{
return this->underlying().get() < other.get();
}
FLUENT_NODISCARD constexpr bool operator>(T const& other) const
{
return other.get() < this->underlying().get();
}
FLUENT_NODISCARD constexpr bool operator<=(T const& other) const
{
return !(other.get() < this->underlying().get());
}
FLUENT_NODISCARD constexpr bool operator>=(T const& other) const
{
return !(*this < other);
}
// On Visual Studio before 19.22, you cannot define constexpr with friend function
// See: https://stackoverflow.com/a/60400110
#if defined(_MSC_VER) && _MSC_VER < 1922
FLUENT_NODISCARD constexpr bool operator==(T const& other) const
{
return !(*this < other) && !(other.get() < this->underlying().get());
}
#else
FLUENT_NODISCARD friend constexpr bool operator==(Comparable<T> const& self, T const& other)
{
return !(self < other) && !(other.get() < self.underlying().get());
}
#endif
FLUENT_NODISCARD constexpr bool operator!=(T const& other) const
{
return !(*this == other);
}
};
template< typename T >
struct Dereferencable;
template< typename T, typename Parameter, template< typename > class ... Skills >
struct Dereferencable<NamedType<T, Parameter, Skills...>> : crtp<NamedType<T, Parameter, Skills...>, Dereferencable>
{
FLUENT_NODISCARD constexpr T& operator*() &
{
return this->underlying().get();
}
FLUENT_NODISCARD constexpr std::remove_reference_t<T> const& operator*() const &
{
return this->underlying().get();
}
};
template <typename Destination>
struct ImplicitlyConvertibleTo
{
template <typename T>
struct templ : crtp<T, templ>
{
FLUENT_NODISCARD constexpr operator Destination() const
{
return this->underlying().get();
}
};
};
template <typename T>
struct Printable : crtp<T, Printable>
{
static constexpr bool is_printable = true;
void print(std::ostream& os) const
{
os << this->underlying().get();
}
};
template <typename T, typename Parameter, template <typename> class... Skills>
typename std::enable_if<NamedType<T, Parameter, Skills...>::is_printable, std::ostream&>::type
operator<<(std::ostream& os, NamedType<T, Parameter, Skills...> const& object)
{
object.print(os);
return os;
}
template <typename T>
struct Hashable
{
static constexpr bool is_hashable = true;
};
template <typename NamedType_>
struct FunctionCallable;
template <typename T, typename Parameter, template <typename> class... Skills>
struct FunctionCallable<NamedType<T, Parameter, Skills...>> : crtp<NamedType<T, Parameter, Skills...>, FunctionCallable>
{
FLUENT_NODISCARD constexpr operator T const&() const
{
return this->underlying().get();
}
FLUENT_NODISCARD constexpr operator T&()
{
return this->underlying().get();
}
};
template <typename NamedType_>
struct MethodCallable;
template <typename T, typename Parameter, template <typename> class... Skills>
struct MethodCallable<NamedType<T, Parameter, Skills...>> : crtp<NamedType<T, Parameter, Skills...>, MethodCallable>
{
FLUENT_NODISCARD FLUENT_CONSTEXPR17 std::remove_reference_t<T> const* operator->() const
{
return std::addressof(this->underlying().get());
}
FLUENT_NODISCARD FLUENT_CONSTEXPR17 std::remove_reference_t<T>* operator->()
{
return std::addressof(this->underlying().get());
}
};
template <typename NamedType_>
struct Callable
: FunctionCallable<NamedType_>
, MethodCallable<NamedType_>
{
};
template <typename T>
struct Arithmetic
: PreIncrementable<T>
, PostIncrementable<T>
, PreDecrementable<T>
, PostDecrementable<T>
, Addable<T>
, Subtractable<T>
, Multiplicable<T>
, Divisible<T>
, Modulable<T>
, BitWiseInvertable<T>
, BitWiseAndable<T>
, BitWiseOrable<T>
, BitWiseXorable<T>
, BitWiseLeftShiftable<T>
, BitWiseRightShiftable<T>
, Comparable<T>
, Printable<T>
, Hashable<T>
{
};
} // namespace fluent
namespace std
{
template <typename T, typename Parameter, template <typename> class... Skills>
struct hash<fluent::NamedType<T, Parameter, Skills...>>
{
using NamedType = fluent::NamedType<T, Parameter, Skills...>;
using checkIfHashable = typename std::enable_if<NamedType::is_hashable, void>::type;
size_t operator()(fluent::NamedType<T, Parameter, Skills...> const& x) const noexcept
{
static_assert(noexcept(std::hash<T>()(x.get())), "hash fuction should not throw");
return std::hash<T>()(x.get());
}
};
} // namespace std
#endif

View File

@ -1,9 +0,0 @@
#pragma once
namespace fluent
{
#define NAMED_TYPE_VERSION_MAJOR 1
#define NAMED_TYPE_VERSION_MINOR 1
#define NAMED_TYPE_VERSION_PATCH 0
#define NAMED_TYPE_VERSION "1.1.0"
} // namespace fluent

View File

@ -21,8 +21,8 @@ TEST(Binary, Insert) {
int64_t num_queries = 10;
int64_t topK = 5;
auto schema = std::make_shared<Schema>();
schema->AddDebugField("vecbin", DataType::VECTOR_BINARY, 128, MetricType::METRIC_Jaccard);
schema->AddDebugField("age", DataType::INT32);
schema->AddField("vecbin", DataType::VECTOR_BINARY, 128, MetricType::METRIC_Jaccard);
schema->AddField("age", DataType::INT32);
auto dataset = DataGen(schema, N, 10);
auto segment = CreateSegment(schema);
segment->PreInsert(N);

View File

@ -18,7 +18,7 @@ TEST(Bitmap, Naive) {
using namespace milvus::segcore;
using namespace milvus::query;
auto schema = std::make_shared<Schema>();
schema->AddDebugField("height", DataType::FLOAT);
schema->AddField("height", DataType::FLOAT);
int N = 10000;
auto raw_data = DataGen(schema, N);
auto vec = raw_data.get_col<float>(0);

View File

@ -98,11 +98,11 @@ TEST(Expr, Range) {
}
})";
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddDebugField("age", DataType::INT32);
schema->AddField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddField("age", DataType::INT32);
auto plan = CreatePlan(*schema, dsl_string);
ShowPlanNodeVisitor shower;
Assert(plan->tag2field_.at("$0") == schema->get_offset(FieldName("fakevec")));
Assert(plan->tag2field_.at("$0") == "fakevec");
auto out = shower.call_child(*plan->plan_node_);
std::cout << out.dump(4);
}
@ -140,11 +140,11 @@ TEST(Expr, RangeBinary) {
}
})";
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_BINARY, 512, MetricType::METRIC_Jaccard);
schema->AddDebugField("age", DataType::INT32);
schema->AddField("fakevec", DataType::VECTOR_BINARY, 512, MetricType::METRIC_Jaccard);
schema->AddField("age", DataType::INT32);
auto plan = CreatePlan(*schema, dsl_string);
ShowPlanNodeVisitor shower;
Assert(plan->tag2field_.at("$0") == schema->get_offset(FieldName("fakevec")));
Assert(plan->tag2field_.at("$0") == "fakevec");
auto out = shower.call_child(*plan->plan_node_);
std::cout << out.dump(4);
}
@ -182,8 +182,8 @@ TEST(Expr, InvalidRange) {
}
})";
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddDebugField("age", DataType::INT32);
schema->AddField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddField("age", DataType::INT32);
ASSERT_ANY_THROW(CreatePlan(*schema, dsl_string));
}
@ -221,8 +221,8 @@ TEST(Expr, InvalidDSL) {
})";
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddDebugField("age", DataType::INT32);
schema->AddField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddField("age", DataType::INT32);
ASSERT_ANY_THROW(CreatePlan(*schema, dsl_string));
}
@ -231,13 +231,13 @@ TEST(Expr, ShowExecutor) {
using namespace milvus::segcore;
auto node = std::make_unique<FloatVectorANNS>();
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
int64_t num_queries = 100L;
auto raw_data = DataGen(schema, num_queries);
auto& info = node->query_info_;
info.metric_type_ = "L2";
info.topK_ = 20;
info.field_offset_ = FieldOffset(0);
info.field_id_ = "fakevec";
node->predicate_ = std::nullopt;
ShowPlanNodeVisitor show_visitor;
PlanNodePtr base(node.release());
@ -290,8 +290,8 @@ TEST(Expr, TestRange) {
}
})";
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddDebugField("age", DataType::INT32);
schema->AddField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddField("age", DataType::INT32);
auto seg = CreateSegment(schema);
int N = 10000;
@ -374,8 +374,8 @@ TEST(Expr, TestTerm) {
}
})";
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddDebugField("age", DataType::INT32);
schema->AddField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddField("age", DataType::INT32);
auto seg = CreateSegment(schema);
int N = 10000;
@ -477,8 +477,8 @@ TEST(Expr, TestSimpleDsl) {
}
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddDebugField("age", DataType::INT32);
schema->AddField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddField("age", DataType::INT32);
auto seg = CreateSegment(schema);
std::vector<int> age_col;

View File

@ -227,10 +227,10 @@ GenDataset(int64_t N, const milvus::knowhere::MetricType& metric_type, bool is_b
auto schema = std::make_shared<milvus::Schema>();
auto faiss_metric_type = milvus::knowhere::GetMetricType(metric_type);
if (!is_binary) {
schema->AddDebugField("fakevec", milvus::engine::DataType::VECTOR_FLOAT, dim, faiss_metric_type);
schema->AddField("fakevec", milvus::engine::DataType::VECTOR_FLOAT, dim, faiss_metric_type);
return milvus::segcore::DataGen(schema, N);
} else {
schema->AddDebugField("fakebinvec", milvus::engine::DataType::VECTOR_BINARY, dim, faiss_metric_type);
schema->AddField("fakebinvec", milvus::engine::DataType::VECTOR_BINARY, dim, faiss_metric_type);
return milvus::segcore::DataGen(schema, N);
}
}

View File

@ -242,8 +242,8 @@ TEST(Indexing, BinaryBruteForce) {
int64_t dim = 512;
auto result_count = topk * num_queries;
auto schema = std::make_shared<Schema>();
schema->AddDebugField("vecbin", DataType::VECTOR_BINARY, dim, MetricType::METRIC_Jaccard);
schema->AddDebugField("age", DataType::INT64);
schema->AddField("vecbin", DataType::VECTOR_BINARY, dim, MetricType::METRIC_Jaccard);
schema->AddField("age", DataType::INT64);
auto dataset = DataGen(schema, N, 10);
auto bin_vec = dataset.get_col<uint8_t>(0);
auto query_data = 1024 * dim / 8 + bin_vec.data();

View File

@ -72,13 +72,13 @@ TEST(Query, ShowExecutor) {
using namespace milvus;
auto node = std::make_unique<FloatVectorANNS>();
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
int64_t num_queries = 100L;
auto raw_data = DataGen(schema, num_queries);
auto& info = node->query_info_;
info.metric_type_ = "L2";
info.topK_ = 20;
info.field_offset_ = FieldOffset(1000);
info.field_id_ = "fakevec";
node->predicate_ = std::nullopt;
ShowPlanNodeVisitor show_visitor;
PlanNodePtr base(node.release());
@ -113,7 +113,7 @@ TEST(Query, DSL) {
})";
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
auto plan = CreatePlan(*schema, dsl_string);
auto res = shower.call_child(*plan->plan_node_);
@ -159,7 +159,7 @@ TEST(Query, ParsePlaceholderGroup) {
})";
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
auto plan = CreatePlan(*schema, dsl_string);
int64_t num_queries = 100000;
int dim = 16;
@ -172,8 +172,8 @@ 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("age", DataType::FLOAT);
schema->AddField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddField("age", DataType::FLOAT);
std::string dsl = R"({
"bool": {
"must": [
@ -263,8 +263,8 @@ 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("age", DataType::FLOAT);
schema->AddField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddField("age", DataType::FLOAT);
std::string dsl = R"({
"bool": {
"must": [
@ -316,8 +316,8 @@ TEST(Query, ExecEmpty) {
using namespace milvus::query;
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->AddField("age", DataType::FLOAT);
schema->AddField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
std::string dsl = R"({
"bool": {
"must": [
@ -361,8 +361,8 @@ 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("age", DataType::FLOAT);
schema->AddField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddField("age", DataType::FLOAT);
std::string dsl = R"({
"bool": {
"must": [
@ -466,7 +466,7 @@ TEST(Indexing, InnerProduct) {
]
}
})";
schema->AddDebugField("normalized", DataType::VECTOR_FLOAT, dim, MetricType::METRIC_INNER_PRODUCT);
schema->AddField("normalized", DataType::VECTOR_FLOAT, dim, MetricType::METRIC_INNER_PRODUCT);
auto dataset = DataGen(schema, N);
auto segment = CreateSegment(schema);
auto plan = CreatePlan(*schema, dsl);
@ -571,8 +571,8 @@ TEST(Query, ExecWithPredicateBinary) {
using namespace milvus::query;
using namespace milvus::segcore;
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_BINARY, 512, MetricType::METRIC_Jaccard);
schema->AddDebugField("age", DataType::FLOAT);
schema->AddField("fakevec", DataType::VECTOR_BINARY, 512, MetricType::METRIC_Jaccard);
schema->AddField("age", DataType::FLOAT);
std::string dsl = R"({
"bool": {
"must": [

View File

@ -30,8 +30,8 @@ TEST(Sealed, without_predicate) {
auto dim = 16;
auto topK = 5;
auto metric_type = MetricType::METRIC_L2;
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, dim, metric_type);
schema->AddDebugField("age", DataType::FLOAT);
schema->AddField("fakevec", DataType::VECTOR_FLOAT, dim, metric_type);
schema->AddField("age", DataType::FLOAT);
std::string dsl = R"({
"bool": {
"must": [
@ -127,8 +127,8 @@ TEST(Sealed, with_predicate) {
auto dim = 16;
auto topK = 5;
auto metric_type = MetricType::METRIC_L2;
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, dim, metric_type);
schema->AddDebugField("counter", DataType::INT64);
schema->AddField("fakevec", DataType::VECTOR_FLOAT, dim, metric_type);
schema->AddField("counter", DataType::INT64);
std::string dsl = R"({
"bool": {
"must": [

View File

@ -63,8 +63,8 @@ TEST(SegmentCoreTest, NormalDistributionTest) {
using namespace milvus::segcore;
using namespace milvus::engine;
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddDebugField("age", DataType::INT32);
schema->AddField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddField("age", DataType::INT32);
int N = 1000 * 1000;
auto [raw_data, timestamps, uids] = generate_data(N);
auto segment = CreateSegment(schema);
@ -76,8 +76,8 @@ TEST(SegmentCoreTest, MockTest) {
using namespace milvus::segcore;
using namespace milvus::engine;
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddDebugField("age", DataType::INT32);
schema->AddField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddField("age", DataType::INT32);
std::vector<char> raw_data;
std::vector<Timestamp> timestamps;
std::vector<int64_t> uids;
@ -116,8 +116,8 @@ TEST(SegmentCoreTest, SmallIndex) {
using namespace milvus::segcore;
using namespace milvus::engine;
auto schema = std::make_shared<Schema>();
schema->AddDebugField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddDebugField("age", DataType::INT32);
schema->AddField("fakevec", DataType::VECTOR_FLOAT, 16, MetricType::METRIC_L2);
schema->AddField("age", DataType::INT32);
int N = 1024 * 1024;
auto data = DataGen(schema, N);
}

View File

@ -96,7 +96,7 @@ DataGen(SchemaPtr schema, int64_t N, uint64_t seed = 42) {
case engine::DataType::VECTOR_FLOAT: {
auto dim = field.get_dim();
vector<float> final;
bool is_ip = starts_with(field.get_name().get(), "normalized");
bool is_ip = starts_with(field.get_name(), "normalized");
for (int n = 0; n < N; ++n) {
vector<float> data(dim);
float sum = 0;
@ -129,7 +129,7 @@ DataGen(SchemaPtr schema, int64_t N, uint64_t seed = 42) {
case engine::DataType::INT64: {
vector<int64_t> data(N);
// begin with counter
if (starts_with(field.get_name().get(), "counter")) {
if (starts_with(field.get_name(), "counter")) {
int64_t index = 0;
for (auto& x : data) {
x = index++;

View File

@ -64,11 +64,11 @@ type collectionReplica interface {
}
type collectionReplicaImpl struct {
mu sync.RWMutex
tSafe tSafe
mu sync.RWMutex // guards collections and segments
collections []*Collection
segments map[UniqueID]*Segment
tSafe tSafe
}
//----------------------------------------------------------------------------------------------------- tSafe
@ -95,11 +95,10 @@ func (colReplica *collectionReplicaImpl) addCollection(collectionID UniqueID, sc
}
func (colReplica *collectionReplicaImpl) removeCollection(collectionID UniqueID) error {
collection, err := colReplica.getCollectionByID(collectionID)
colReplica.mu.Lock()
defer colReplica.mu.Unlock()
collection, err := colReplica.getCollectionByIDPrivate(collectionID)
if err != nil {
return err
}
@ -128,6 +127,10 @@ func (colReplica *collectionReplicaImpl) getCollectionByID(collectionID UniqueID
colReplica.mu.RLock()
defer colReplica.mu.RUnlock()
return colReplica.getCollectionByIDPrivate(collectionID)
}
func (colReplica *collectionReplicaImpl) getCollectionByIDPrivate(collectionID UniqueID) (*Collection, error) {
for _, collection := range colReplica.collections {
if collection.ID() == collectionID {
return collection, nil
@ -164,26 +167,26 @@ func (colReplica *collectionReplicaImpl) hasCollection(collectionID UniqueID) bo
//----------------------------------------------------------------------------------------------------- partition
func (colReplica *collectionReplicaImpl) getPartitionNum(collectionID UniqueID) (int, error) {
collection, err := colReplica.getCollectionByID(collectionID)
colReplica.mu.RLock()
defer colReplica.mu.RUnlock()
collection, err := colReplica.getCollectionByIDPrivate(collectionID)
if err != nil {
return -1, err
}
colReplica.mu.RLock()
defer colReplica.mu.RUnlock()
return len(collection.partitions), nil
}
func (colReplica *collectionReplicaImpl) addPartition(collectionID UniqueID, partitionTag string) error {
collection, err := colReplica.getCollectionByID(collectionID)
colReplica.mu.Lock()
defer colReplica.mu.Unlock()
collection, err := colReplica.getCollectionByIDPrivate(collectionID)
if err != nil {
return err
}
colReplica.mu.Lock()
defer colReplica.mu.Unlock()
var newPartition = newPartition(partitionTag)
*collection.Partitions() = append(*collection.Partitions(), newPartition)
@ -191,14 +194,18 @@ func (colReplica *collectionReplicaImpl) addPartition(collectionID UniqueID, par
}
func (colReplica *collectionReplicaImpl) removePartition(collectionID UniqueID, partitionTag string) error {
collection, err := colReplica.getCollectionByID(collectionID)
colReplica.mu.Lock()
defer colReplica.mu.Unlock()
return colReplica.removePartitionPrivate(collectionID, partitionTag)
}
func (colReplica *collectionReplicaImpl) removePartitionPrivate(collectionID UniqueID, partitionTag string) error {
collection, err := colReplica.getCollectionByIDPrivate(collectionID)
if err != nil {
return err
}
colReplica.mu.Lock()
defer colReplica.mu.Unlock()
var tmpPartitions = make([]*Partition, 0)
for _, p := range *collection.Partitions() {
if p.Tag() == partitionTag {
@ -215,6 +222,7 @@ func (colReplica *collectionReplicaImpl) removePartition(collectionID UniqueID,
return nil
}
// deprecated
func (colReplica *collectionReplicaImpl) addPartitionsByCollectionMeta(colMeta *etcdpb.CollectionMeta) error {
if !colReplica.hasCollection(colMeta.ID) {
err := errors.New("Cannot find collection, id = " + strconv.FormatInt(colMeta.ID, 10))
@ -239,13 +247,14 @@ func (colReplica *collectionReplicaImpl) addPartitionsByCollectionMeta(colMeta *
}
func (colReplica *collectionReplicaImpl) removePartitionsByCollectionMeta(colMeta *etcdpb.CollectionMeta) error {
col, err := colReplica.getCollectionByID(colMeta.ID)
colReplica.mu.Lock()
defer colReplica.mu.Unlock()
col, err := colReplica.getCollectionByIDPrivate(colMeta.ID)
if err != nil {
return err
}
colReplica.mu.Lock()
pToDel := make([]string, 0)
for _, partition := range col.partitions {
hasPartition := false
@ -259,10 +268,8 @@ func (colReplica *collectionReplicaImpl) removePartitionsByCollectionMeta(colMet
}
}
colReplica.mu.Unlock()
for _, tag := range pToDel {
err := colReplica.removePartition(col.ID(), tag)
err := colReplica.removePartitionPrivate(col.ID(), tag)
if err != nil {
log.Println(err)
}
@ -273,14 +280,18 @@ func (colReplica *collectionReplicaImpl) removePartitionsByCollectionMeta(colMet
}
func (colReplica *collectionReplicaImpl) getPartitionByTag(collectionID UniqueID, partitionTag string) (*Partition, error) {
collection, err := colReplica.getCollectionByID(collectionID)
colReplica.mu.RLock()
defer colReplica.mu.RUnlock()
return colReplica.getPartitionByTagPrivate(collectionID, partitionTag)
}
func (colReplica *collectionReplicaImpl) getPartitionByTagPrivate(collectionID UniqueID, partitionTag string) (*Partition, error) {
collection, err := colReplica.getCollectionByIDPrivate(collectionID)
if err != nil {
return nil, err
}
colReplica.mu.RLock()
defer colReplica.mu.RUnlock()
for _, p := range *collection.Partitions() {
if p.Tag() == partitionTag {
return p, nil
@ -291,15 +302,15 @@ func (colReplica *collectionReplicaImpl) getPartitionByTag(collectionID UniqueID
}
func (colReplica *collectionReplicaImpl) hasPartition(collectionID UniqueID, partitionTag string) bool {
collection, err := colReplica.getCollectionByID(collectionID)
colReplica.mu.RLock()
defer colReplica.mu.RUnlock()
collection, err := colReplica.getCollectionByIDPrivate(collectionID)
if err != nil {
log.Println(err)
return false
}
colReplica.mu.RLock()
defer colReplica.mu.RUnlock()
for _, p := range *collection.Partitions() {
if p.Tag() == partitionTag {
return true
@ -343,19 +354,19 @@ func (colReplica *collectionReplicaImpl) getSegmentStatistics() []*internalpb.Se
}
func (colReplica *collectionReplicaImpl) addSegment(segmentID UniqueID, partitionTag string, collectionID UniqueID) error {
collection, err := colReplica.getCollectionByID(collectionID)
colReplica.mu.Lock()
defer colReplica.mu.Unlock()
collection, err := colReplica.getCollectionByIDPrivate(collectionID)
if err != nil {
return err
}
partition, err2 := colReplica.getPartitionByTag(collectionID, partitionTag)
partition, err2 := colReplica.getPartitionByTagPrivate(collectionID, partitionTag)
if err2 != nil {
return err2
}
colReplica.mu.Lock()
defer colReplica.mu.Unlock()
var newSegment = newSegment(collection, segmentID, partitionTag, collectionID)
colReplica.segments[segmentID] = newSegment

View File

@ -27,8 +27,8 @@ type searchService struct {
replica collectionReplica
tSafeWatcher *tSafeWatcher
serviceableTimeMutex sync.Mutex // guards serviceableTime
serviceableTime Timestamp
serviceableTimeMutex sync.Mutex
msgBuffer chan msgstream.TsMsg
unsolvedMsg []msgstream.TsMsg

View File

@ -25,16 +25,18 @@ import (
type indexParam = map[string]string
type Segment struct {
segmentPtr C.CSegmentBase
segmentID UniqueID
partitionTag string // TODO: use partitionID
collectionID UniqueID
lastMemSize int64
lastRowCount int64
mu sync.Mutex
segmentPtr C.CSegmentBase
segmentID UniqueID
partitionTag string // TODO: use partitionID
collectionID UniqueID
lastMemSize int64
lastRowCount int64
rmMutex sync.Mutex // guards recentlyModified
recentlyModified bool
indexParam map[int64]indexParam
paramMutex sync.RWMutex
paramMutex sync.RWMutex // guards indexParam
indexParam map[int64]indexParam
}
func (s *Segment) ID() UniqueID {
@ -42,14 +44,14 @@ func (s *Segment) ID() UniqueID {
}
func (s *Segment) SetRecentlyModified(modify bool) {
s.mu.Lock()
defer s.mu.Unlock()
s.rmMutex.Lock()
defer s.rmMutex.Unlock()
s.recentlyModified = modify
}
func (s *Segment) GetRecentlyModified() bool {
s.mu.Lock()
defer s.mu.Unlock()
s.rmMutex.Lock()
defer s.rmMutex.Unlock()
return s.recentlyModified
}

View File

@ -31,7 +31,7 @@ type tSafe interface {
}
type tSafeImpl struct {
tSafeMu sync.Mutex
tSafeMu sync.Mutex // guards all fields
tSafe Timestamp
watcherList []*tSafeWatcher
}

View File

@ -18,5 +18,5 @@ go test -race -cover "${MILVUS_DIR}/kv/..." -failfast
go test -race -cover "${MILVUS_DIR}/proxy/..." -failfast
go test -race -cover "${MILVUS_DIR}/writenode/..." -failfast
go test -race -cover "${MILVUS_DIR}/master/..." -failfast
go test -cover "${MILVUS_DIR}/msgstream/..." "${MILVUS_DIR}/querynode/..." "${MILVUS_DIR}/storage" "${MILVUS_DIR}/util/..." -failfast
go test -race -cover "${MILVUS_DIR}/msgstream/..." "${MILVUS_DIR}/querynode/..." "${MILVUS_DIR}/storage" "${MILVUS_DIR}/util/..." -failfast
#go test -race -cover "${MILVUS_DIR}/kv/..." "${MILVUS_DIR}/msgstream/..." "${MILVUS_DIR}/master/..." "${MILVUS_DIR}/querynode/..." -failfast

View File

@ -286,7 +286,6 @@ class TestIndexBase:
res = connect.search(collection, query)
assert len(res) == nq
@pytest.mark.skip("test_create_index_multithread_ip")
@pytest.mark.timeout(BUILD_TIMEOUT)
@pytest.mark.level(2)
def test_create_index_multithread_ip(self, connect, collection, args):