Enable Segment Naive and Base, support Collection

Signed-off-by: FluorineDog <guilin.gou@zilliz.com>
This commit is contained in:
FluorineDog 2020-08-26 16:49:03 +08:00 committed by yefu.chen
parent af1f67b9d2
commit 996c591088
19 changed files with 1910 additions and 0 deletions

9
core/CMakeLists.txt Normal file
View File

@ -0,0 +1,9 @@
project(sulvim_core)
cmake_minimum_required(VERSION 3.16)
set( CMAKE_CXX_STANDARD 17 )
set( CMAKE_CXX_STANDARD_REQUIRED on )
include_directories(src)
add_subdirectory(src)
add_subdirectory(unittest)

4
core/src/CMakeLists.txt Normal file
View File

@ -0,0 +1,4 @@
add_subdirectory(utils)
add_subdirectory(dog_segment)
#add_subdirectory(index)
add_subdirectory(query)

View File

@ -0,0 +1,10 @@
set(DOG_SEGMENT_FILES
SegmentNaive.cpp
)
# Third Party dablooms file
#aux_source_directory( ${MILVUS_THIRDPARTY_SRC}/dablooms THIRDPARTY_DABLOOMS_FILES )
add_library(milvus_dog_segment
${DOG_SEGMENT_FILES}
)
#add_dependencies( segment sqlite mysqlpp )
target_link_libraries(milvus_dog_segment tbb milvus_utils)

View File

@ -0,0 +1,51 @@
#pragma once
#include "SegmentDefs.h"
//////////////////////////////////////////////////////////////////
class Partition {
public:
const std::deque<SegmentBasePtr>& segments() const {
return segments_;
}
private:
std::string name_;
std::deque<SegmentBasePtr> segments_;
};
using PartitionPtr = std::shard_ptr<Partition>;
//////////////////////////////////////////////////////////////////
class Collection {
public:
explicit Collection(std::string name): name_(name){}
// TODO: set index
set_index() {}
set_schema(std::string config) {
// TODO: config to schema
schema_ = null;
}
public:
// std::vector<int64_t> Insert() {
// for (auto partition: partitions_) {
// for (auto segment: partition.segments()) {
// if (segment.Status == Status.open) {
// segment.Insert()
// }
// }
// }
// }
private:
// TODO: Index ptr
IndexPtr index_ = nullptr;
std::string name_;
SchemaPtr schema_;
std::vector<PartitionPtr> partitions_;
};

View File

@ -0,0 +1,90 @@
#pragma once
#include <vector>
// #include "db/Types.h"
#include "dog_segment/SegmentDefs.h"
// #include "knowhere/index/Index.h"
#include "query/GeneralQuery.h"
namespace milvus {
namespace dog_segment {
using engine::QueryResult;
int
TestABI();
class SegmentBase {
public:
// definitions
enum class SegmentState {
Invalid = 0,
Open, // able to insert data
Closed // able to build index
};
public:
virtual ~SegmentBase() = default;
// SegmentBase(std::shared_ptr<FieldsInfo> collection);
virtual Status
Insert(int64_t size, const uint64_t* primary_keys, const Timestamp* timestamps, const DogDataChunk& values) = 0;
// TODO: add id into delete log, possibly bitmap
virtual Status
Delete(int64_t size, const uint64_t* primary_keys, const Timestamp* timestamps) = 0;
// query contains metadata of
virtual Status
Query(const query::QueryPtr& query, Timestamp timestamp, QueryResult& results) = 0;
// // THIS FUNCTION IS REMOVED
// virtual Status
// GetEntityByIds(Timestamp timestamp, const std::vector<Id>& ids, DataChunkPtr& results) = 0;
// stop receive insert requests
virtual Status
Close() = 0;
// // to make all data inserted visible
// // maybe a no-op?
// virtual Status
// Flush(Timestamp timestamp) = 0;
// BuildIndex With Paramaters, must with Frozen State
// This function is atomic
// NOTE: index_params contains serveral policies for several index
virtual Status
BuildIndex(std::shared_ptr<IndexConfig> index_params) = 0;
// Remove Index
virtual Status
DropIndex(std::string_view field_name) = 0;
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;
public:
virtual ssize_t
get_row_count() const = 0;
virtual SegmentState
get_state() const = 0;
virtual ssize_t
get_deleted_count() const = 0;
public:
// getter and setter
private:
Timestamp time_begin_;
Timestamp time_end_;
uint64_t segment_id_;
};
std::unique_ptr<SegmentBase> CreateSegment(SchemaPtr ptr);
} // namespace engine
} // namespace milvus

View File

@ -0,0 +1,173 @@
#pragma once
#include <vector>
#include <unordered_map>
// #include "db/Types.h"
// #include "knowhere/index/Index.h"
#include "utils/Status.h"
#include "utils/Types.h"
#include <cassert>
using Timestamp = uint64_t; // TODO: use TiKV-like timestamp
namespace milvus::dog_segment {
using engine::DataType;
using engine::FieldElementType;
struct DogDataChunk {
void* raw_data; // schema
int sizeof_per_row; // alignment
int64_t count;
};
struct IndexConfig {
// TODO
// std::unordered_map<std::string, knowhere::Config> configs;
};
inline int
field_sizeof(DataType data_type, int dim = 1) {
switch (data_type) {
case DataType::BOOL:
return sizeof(bool);
case DataType::DOUBLE:
return sizeof(double);
case DataType::FLOAT:
return sizeof(float);
case DataType::INT8:
return sizeof(uint8_t);
case DataType::INT16:
return sizeof(uint16_t);
case DataType::INT32:
return sizeof(uint32_t);
case DataType::INT64:
return sizeof(uint64_t);
case DataType::VECTOR_FLOAT:
return sizeof(float) * dim;
case DataType::VECTOR_BINARY: {
assert(dim % 8 == 0);
return dim / 8;
}
default: {
assert(false);
return 0;
}
}
}
struct FieldMeta {
public:
FieldMeta(std::string_view name, DataType type, int dim = 1) : name_(name), type_(type), dim_(dim) {
}
bool
is_vector() const {
assert(type_ != DataType::NONE);
return type_ == DataType::VECTOR_BINARY || type_ == DataType::VECTOR_FLOAT;
}
void
set_dim(int dim) {
dim_ = dim;
}
int
get_dim() const {
return dim_;
}
const std::string&
get_name() const {
return name_;
}
DataType
get_data_type() const {
return type_;
}
int
get_sizeof() const {
return field_sizeof(type_, dim_);
}
private:
std::string name_;
DataType type_ = DataType::NONE;
int dim_ = 1;
};
class Schema {
public:
void
AddField(std::string_view field_name, DataType data_type, int dim = 1) {
auto field_meta = FieldMeta(field_name, data_type, dim);
this->AddField(std::move(field_meta));
}
void
AddField(FieldMeta field_meta) {
auto index = fields_.size();
fields_.emplace_back(field_meta);
indexes_.emplace(field_meta.get_name(), index);
total_sizeof_ = field_meta.get_sizeof();
}
auto
begin() {
return fields_.begin();
}
auto
end() {
return fields_.end();
}
auto
begin() const {
return fields_.begin();
}
auto
end() const {
return fields_.end();
}
int size() const {
return fields_.size();
}
const FieldMeta&
operator[](int field_index) const {
return fields_[field_index];
}
const FieldMeta&
operator[](const std::string& field_name) const {
auto index_iter = indexes_.find(field_name);
assert(index_iter != indexes_.end());
auto index = index_iter->second;
return (*this)[index];
}
private:
// this is where data holds
std::vector<FieldMeta> fields_;
private:
// a mapping for random access
std::unordered_map<std::string, int> indexes_;
int total_sizeof_;
};
using SchemaPtr = std::shared_ptr<Schema>;
class IndexData {
public:
virtual std::vector<char>
serilize() = 0;
static std::shared_ptr<IndexData>
deserialize(int64_t size, const char* blob);
};
} // namespace milvus::dog_segment

View File

@ -0,0 +1,241 @@
#include <shared_mutex>
#include "dog_segment/SegmentBase.h"
#include "utils/Status.h"
#include <tbb/concurrent_vector.h>
#include <tbb/concurrent_unordered_map.h>
#include <atomic>
namespace milvus::dog_segment {
int
TestABI() {
return 42;
}
struct ColumnBasedDataChunk {
std::vector<std::vector<float>> entity_vecs;
static ColumnBasedDataChunk from(const DogDataChunk& source, const Schema& schema){
ColumnBasedDataChunk dest;
auto count = source.count;
auto raw_data = reinterpret_cast<const char*>(source.raw_data);
auto align = source.sizeof_per_row;
for(auto& field: schema) {
auto len = field.get_sizeof();
assert(len % sizeof(float) == 0);
std::vector<float> new_col(len * count / sizeof(float));
for(int64_t i = 0; i < count; ++i) {
memcpy(new_col.data() + i * len / sizeof(float), raw_data + i * align , len);
}
dest.entity_vecs.push_back(std::move(new_col));
// offset the raw_data
raw_data += len / sizeof(float);
}
return dest;
}
};
class SegmentNaive : public SegmentBase {
public:
virtual ~SegmentNaive() = default;
// SegmentBase(std::shared_ptr<FieldsInfo> collection);
// TODO: originally, id should be put into data_chunk
// TODO: Is it ok to put them the other side?
Status
Insert(int64_t size, const uint64_t* primary_keys, const Timestamp* timestamps,
const DogDataChunk& values) override;
// TODO: add id into delete log, possibly bitmap
Status
Delete(int64_t size, const uint64_t* primary_keys, const Timestamp* timestamps) override;
// query contains metadata of
Status
Query(const query::QueryPtr& query, Timestamp timestamp, QueryResult& results) override;
// // THIS FUNCTION IS REMOVED
// virtual Status
// GetEntityByIds(Timestamp timestamp, const std::vector<Id>& ids, DataChunkPtr& results) = 0;
// stop receive insert requests
Status
Close() override {
std::lock_guard<std::shared_mutex> lck(mutex_);
assert(state_ == SegmentState::Open);
state_ = SegmentState::Closed;
return Status::OK();
}
// // to make all data inserted visible
// // maybe a no-op?
// virtual Status
// Flush(Timestamp timestamp) = 0;
// BuildIndex With Paramaters, must with Frozen State
// This function is atomic
// NOTE: index_params contains serveral policies for several index
Status
BuildIndex(std::shared_ptr<IndexConfig> index_params) override {
throw std::runtime_error("not implemented");
}
// Remove Index
Status
DropIndex(std::string_view field_name) override {
throw std::runtime_error("not implemented");
}
Status
DropRawData(std::string_view field_name) override {
// TODO: NO-OP
return Status::OK();
}
Status
LoadRawData(std::string_view field_name, const char* blob, int64_t blob_size) override {
// TODO: NO-OP
return Status::OK();
}
public:
ssize_t
get_row_count() const override {
return ack_count_.load(std::memory_order_relaxed);
}
// const FieldsInfo&
// get_fields_info() const override {
//
// }
//
// // check is_indexed here
// virtual const IndexConfig&
// get_index_param() const = 0;
//
SegmentState
get_state() const override {
return state_.load(std::memory_order_relaxed);
}
//
// std::shared_ptr<IndexData>
// get_index_data();
ssize_t
get_deleted_count() const override {
return 0;
}
public:
friend std::unique_ptr<SegmentBase>
CreateSegment(SchemaPtr schema);
private:
SchemaPtr schema_;
std::shared_mutex mutex_;
std::atomic<SegmentState> state_ = SegmentState::Open;
std::atomic<int64_t> ack_count_ = 0;
tbb::concurrent_vector<uint64_t> uids_;
tbb::concurrent_vector<Timestamp> timestamps_;
std::vector<tbb::concurrent_vector<float>> entity_vecs_;
tbb::concurrent_unordered_map<uint64_t, int> internal_indexes_;
tbb::concurrent_unordered_multimap<int, Timestamp> delete_logs_;
};
std::unique_ptr<SegmentBase>
CreateSegment(SchemaPtr schema) {
auto segment = std::make_unique<SegmentNaive>();
segment->schema_ = schema;
segment->entity_vecs_.resize(schema->size());
return segment;
}
Status
SegmentNaive::Insert(int64_t size, const uint64_t* primary_keys, const Timestamp* timestamps,
const DogDataChunk& row_values) {
const auto& schema = *schema_;
auto data_chunk = ColumnBasedDataChunk::from(row_values, schema);
// insert datas
// TODO: use shared_lock
std::lock_guard lck(mutex_);
assert(state_ == SegmentState::Open);
auto ack_id = ack_count_.load();
uids_.grow_by(primary_keys, primary_keys + size);
for(int64_t i = 0; i < size; ++i) {
auto key = primary_keys[i];
auto internal_index = i + ack_id;
internal_indexes_[key] = internal_index;
}
timestamps_.grow_by(timestamps, timestamps + size);
for(int fid = 0; fid < schema.size(); ++fid) {
auto field = schema[fid];
auto total_len = field.get_sizeof() * size / sizeof(float);
auto source_vec = data_chunk.entity_vecs[fid];
entity_vecs_[fid].grow_by(source_vec.data(), source_vec.data() + total_len);
}
// finish insert
ack_count_ += size;
return Status::OK();
}
Status SegmentNaive::Delete(int64_t size, const uint64_t *primary_keys, const Timestamp *timestamps) {
for(int i = 0; i < size; ++i) {
auto key = primary_keys[i];
auto time = timestamps[i];
delete_logs_.insert(std::make_pair(key, time));
}
return Status::OK();
}
// TODO: remove mock
Status
SegmentNaive::Query(const query::QueryPtr &query, Timestamp timestamp, QueryResult &result) {
std::shared_lock lck(mutex_);
auto ack_count = ack_count_.load();
assert(query == nullptr);
assert(schema_->size() >= 1);
const auto& field = schema_->operator[](0);
assert(field.get_data_type() == DataType::VECTOR_FLOAT);
assert(field.get_name() == "fakevec");
auto dim = field.get_dim();
// assume query vector is [0, 0, ..., 0]
std::vector<float> query_vector(dim, 0);
auto& target_vec = entity_vecs_[0];
int current_index = -1;
float min_diff = std::numeric_limits<float>::max();
for(int index = 0; index < ack_count; ++index) {
float diff = 0;
int offset = index * dim;
for(auto d = 0; d < dim; ++d) {
auto v = target_vec[offset + d] - query_vector[d];
diff += v * v;
}
if(diff < min_diff) {
min_diff = diff;
current_index = index;
}
}
QueryResult query_result;
query_result.row_num_ = 1;
query_result.result_distances_.push_back(min_diff);
query_result.result_ids_.push_back(uids_[current_index]);
// query_result.data_chunk_ = nullptr;
result = std::move(query_result);
return Status::OK();
}
} // namespace milvus::engine

View File

@ -0,0 +1,302 @@
// 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 <memory>
#include <set>
#include <stack>
#include <string>
#include <utility>
#include <vector>
#include "query/BinaryQuery.h"
namespace milvus {
namespace query {
BinaryQueryPtr
ConstructBinTree(std::vector<BooleanQueryPtr> queries, QueryRelation relation, uint64_t idx) {
if (idx == queries.size()) {
return nullptr;
} else if (idx == queries.size() - 1) {
return queries[idx]->getBinaryQuery();
} else {
BinaryQueryPtr bquery = std::make_shared<BinaryQuery>();
bquery->relation = relation;
bquery->left_query = std::make_shared<GeneralQuery>();
bquery->right_query = std::make_shared<GeneralQuery>();
bquery->left_query->bin = queries[idx]->getBinaryQuery();
++idx;
bquery->right_query->bin = ConstructBinTree(queries, relation, idx);
return bquery;
}
}
Status
ConstructLeafBinTree(std::vector<LeafQueryPtr> leaf_queries, BinaryQueryPtr binary_query, uint64_t idx) {
if (idx == leaf_queries.size()) {
return Status::OK();
}
binary_query->left_query = std::make_shared<GeneralQuery>();
binary_query->right_query = std::make_shared<GeneralQuery>();
if (leaf_queries.size() == leaf_queries.size() - 1) {
binary_query->left_query->leaf = leaf_queries[idx];
return Status::OK();
} else if (idx == leaf_queries.size() - 2) {
binary_query->left_query->leaf = leaf_queries[idx];
++idx;
binary_query->right_query->leaf = leaf_queries[idx];
return Status::OK();
} else {
binary_query->left_query->bin->relation = binary_query->relation;
binary_query->right_query->leaf = leaf_queries[idx];
++idx;
return ConstructLeafBinTree(leaf_queries, binary_query->left_query->bin, idx);
}
}
Status
GenBinaryQuery(BooleanQueryPtr query, BinaryQueryPtr& binary_query) {
if (query->getBooleanQueries().size() == 0) {
if (binary_query->relation == QueryRelation::AND || binary_query->relation == QueryRelation::OR) {
// Put VectorQuery to the end of leaf queries
auto query_size = query->getLeafQueries().size();
for (uint64_t i = 0; i < query_size; ++i) {
if (query->getLeafQueries()[i]->vector_placeholder.size() > 0) {
std::swap(query->getLeafQueries()[i], query->getLeafQueries()[0]);
break;
}
}
return ConstructLeafBinTree(query->getLeafQueries(), binary_query, 0);
} else {
switch (query->getOccur()) {
case Occur::MUST: {
binary_query->relation = QueryRelation::AND;
return GenBinaryQuery(query, binary_query);
}
case Occur::MUST_NOT:
case Occur::SHOULD: {
binary_query->relation = QueryRelation::OR;
return GenBinaryQuery(query, binary_query);
}
default:
return Status::OK();
}
}
}
if (query->getBooleanQueries().size() == 1) {
auto bc = query->getBooleanQueries()[0];
binary_query->left_query = std::make_shared<GeneralQuery>();
switch (bc->getOccur()) {
case Occur::MUST: {
binary_query->relation = QueryRelation::AND;
return GenBinaryQuery(bc, binary_query);
}
case Occur::MUST_NOT:
case Occur::SHOULD: {
binary_query->relation = QueryRelation::OR;
return GenBinaryQuery(bc, binary_query);
}
default:
return Status::OK();
}
}
// Construct binary query for every single boolean query
std::vector<BooleanQueryPtr> must_queries;
std::vector<BooleanQueryPtr> must_not_queries;
std::vector<BooleanQueryPtr> should_queries;
Status status;
for (auto& _query : query->getBooleanQueries()) {
status = GenBinaryQuery(_query, _query->getBinaryQuery());
if (!status.ok()) {
return status;
}
if (_query->getOccur() == Occur::MUST) {
must_queries.emplace_back(_query);
} else if (_query->getOccur() == Occur::MUST_NOT) {
must_not_queries.emplace_back(_query);
} else {
should_queries.emplace_back(_query);
}
}
// Construct binary query for combine boolean queries
BinaryQueryPtr must_bquery, should_bquery, must_not_bquery;
uint64_t bquery_num = 0;
if (must_queries.size() > 1) {
// Construct a must binary tree
must_bquery = ConstructBinTree(must_queries, QueryRelation::R1, 0);
++bquery_num;
} else if (must_queries.size() == 1) {
must_bquery = must_queries[0]->getBinaryQuery();
++bquery_num;
}
if (should_queries.size() > 1) {
// Construct a should binary tree
should_bquery = ConstructBinTree(should_queries, QueryRelation::R2, 0);
++bquery_num;
} else if (should_queries.size() == 1) {
should_bquery = should_queries[0]->getBinaryQuery();
++bquery_num;
}
if (must_not_queries.size() > 1) {
// Construct a must_not binary tree
must_not_bquery = ConstructBinTree(must_not_queries, QueryRelation::R1, 0);
++bquery_num;
} else if (must_not_queries.size() == 1) {
must_not_bquery = must_not_queries[0]->getBinaryQuery();
++bquery_num;
}
binary_query->left_query = std::make_shared<GeneralQuery>();
binary_query->right_query = std::make_shared<GeneralQuery>();
BinaryQueryPtr must_should_query = std::make_shared<BinaryQuery>();
must_should_query->left_query = std::make_shared<GeneralQuery>();
must_should_query->right_query = std::make_shared<GeneralQuery>();
if (bquery_num == 3) {
must_should_query->relation = QueryRelation::R3;
must_should_query->left_query->bin = must_bquery;
must_should_query->right_query->bin = should_bquery;
binary_query->relation = QueryRelation::R1;
binary_query->left_query->bin = must_should_query;
binary_query->right_query->bin = must_not_bquery;
} else if (bquery_num == 2) {
if (must_bquery == nullptr) {
binary_query->relation = QueryRelation::R3;
binary_query->left_query->bin = must_not_bquery;
binary_query->right_query->bin = should_bquery;
} else if (should_bquery == nullptr) {
binary_query->relation = QueryRelation::R4;
binary_query->left_query->bin = must_bquery;
binary_query->right_query->bin = must_not_bquery;
} else {
binary_query->relation = QueryRelation::R3;
binary_query->left_query->bin = must_bquery;
binary_query->right_query->bin = should_bquery;
}
} else {
if (must_bquery != nullptr) {
binary_query = must_bquery;
} else if (should_bquery != nullptr) {
binary_query = should_bquery;
} else {
binary_query = must_not_bquery;
}
}
return Status::OK();
}
uint64_t
BinaryQueryHeight(BinaryQueryPtr& binary_query) {
if (binary_query == nullptr) {
return 1;
}
uint64_t left_height = 0, right_height = 0;
if (binary_query->left_query != nullptr) {
left_height = BinaryQueryHeight(binary_query->left_query->bin);
}
if (binary_query->right_query != nullptr) {
right_height = BinaryQueryHeight(binary_query->right_query->bin);
}
return left_height > right_height ? left_height + 1 : right_height + 1;
}
/**
* rules:
* 1. The child node of 'should' and 'must_not' can only be 'term query' and 'range query'.
* 2. One layer cannot include bool query and leaf query.
* 3. The direct child node of 'bool' node cannot be 'should' node or 'must_not' node.
* 4. All filters are pre-filtered(Do structure query first, then use the result to do filtering for vector query).
*
*/
Status
rule_1(BooleanQueryPtr& boolean_query, std::stack<BooleanQueryPtr>& path_stack) {
auto status = Status::OK();
if (boolean_query != nullptr) {
path_stack.push(boolean_query);
for (const auto& leaf_query : boolean_query->getLeafQueries()) {
if (!leaf_query->vector_placeholder.empty()) {
while (!path_stack.empty()) {
auto query = path_stack.top();
if (query->getOccur() == Occur::SHOULD || query->getOccur() == Occur::MUST_NOT) {
std::string msg =
"The child node of 'should' and 'must_not' can only be 'term query' and 'range query'.";
return Status{SERVER_INVALID_DSL_PARAMETER, msg};
}
path_stack.pop();
}
}
}
for (auto query : boolean_query->getBooleanQueries()) {
status = rule_1(query, path_stack);
if (!status.ok()) {
return status;
}
}
}
return status;
}
Status
rule_2(BooleanQueryPtr& boolean_query) {
auto status = Status::OK();
if (boolean_query != nullptr) {
if (!boolean_query->getBooleanQueries().empty() && !boolean_query->getLeafQueries().empty()) {
std::string msg = "One layer cannot include bool query and leaf query.";
return Status{SERVER_INVALID_DSL_PARAMETER, msg};
} else {
for (auto query : boolean_query->getBooleanQueries()) {
status = rule_2(query);
if (!status.ok()) {
return status;
}
}
}
}
return status;
}
Status
ValidateBooleanQuery(BooleanQueryPtr& boolean_query) {
auto status = Status::OK();
if (boolean_query != nullptr) {
for (auto& query : boolean_query->getBooleanQueries()) {
if (query->getOccur() == Occur::SHOULD || query->getOccur() == Occur::MUST_NOT) {
std::string msg = "The direct child node of 'bool' node cannot be 'should' node or 'must_not' node.";
return Status{SERVER_INVALID_DSL_PARAMETER, msg};
}
}
std::stack<BooleanQueryPtr> path_stack;
status = rule_1(boolean_query, path_stack);
if (!status.ok()) {
return status;
}
status = rule_2(boolean_query);
if (!status.ok()) {
return status;
}
}
return status;
}
bool
ValidateBinaryQuery(BinaryQueryPtr& binary_query) {
uint64_t height = BinaryQueryHeight(binary_query);
return height > 1;
}
} // namespace query
} // namespace milvus

View File

@ -0,0 +1,41 @@
// 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 <memory>
#include <vector>
#include "BooleanQuery.h"
namespace milvus {
namespace query {
BinaryQueryPtr
ConstructBinTree(std::vector<BooleanQueryPtr> clauses, QueryRelation relation, uint64_t idx);
Status
ConstructLeafBinTree(std::vector<LeafQueryPtr> leaf_clauses, BinaryQueryPtr binary_query, uint64_t idx);
Status
GenBinaryQuery(BooleanQueryPtr clause, BinaryQueryPtr& binary_query);
uint64_t
BinaryQueryHeight(BinaryQueryPtr& binary_query);
Status
ValidateBooleanQuery(BooleanQueryPtr& boolean_query);
bool
ValidateBinaryQuery(BinaryQueryPtr& binary_query);
} // namespace query
} // namespace milvus

View File

@ -0,0 +1,87 @@
// 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 <memory>
#include <vector>
#include "GeneralQuery.h"
#include "utils/Status.h"
namespace milvus {
namespace query {
enum class Occur {
INVALID = 0,
MUST,
MUST_NOT,
SHOULD,
};
class BooleanQuery {
public:
BooleanQuery() {
}
explicit BooleanQuery(Occur occur) : occur_(occur) {
}
Occur
getOccur() {
return occur_;
}
void
SetOccur(Occur occur) {
occur_ = occur;
}
void
AddBooleanQuery(std::shared_ptr<BooleanQuery> boolean_clause) {
boolean_clauses_.emplace_back(boolean_clause);
}
void
AddLeafQuery(LeafQueryPtr leaf_query) {
leaf_queries_.emplace_back(leaf_query);
}
void
SetLeafQuery(std::vector<LeafQueryPtr> leaf_queries) {
leaf_queries_ = leaf_queries;
}
std::vector<std::shared_ptr<BooleanQuery>>
getBooleanQueries() {
return boolean_clauses_;
}
BinaryQueryPtr&
getBinaryQuery() {
return binary_query_;
}
std::vector<LeafQueryPtr>&
getLeafQueries() {
return leaf_queries_;
}
private:
Occur occur_ = Occur::INVALID;
std::vector<std::shared_ptr<BooleanQuery>> boolean_clauses_;
std::vector<LeafQueryPtr> leaf_queries_;
BinaryQueryPtr binary_query_ = std::make_shared<BinaryQuery>();
};
using BooleanQueryPtr = std::shared_ptr<BooleanQuery>;
} // namespace query
} // namespace milvus

View File

@ -0,0 +1,2 @@
# TODO

View File

@ -0,0 +1,130 @@
// 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 <iostream>
#include <memory>
#include <set>
#include <string>
#include <unordered_map>
#include <vector>
// #include "db/Types.h"
// #include "utils/Json.h"
namespace milvus {
namespace query {
// enum class CompareOperator {
// LT = 0,
// LTE,
// EQ,
// GT,
// GTE,
// NE,
// };
// enum class QueryRelation {
// INVALID = 0,
// R1,
// R2,
// R3,
// R4,
// AND,
// OR,
// };
// struct QueryColumn {
// std::string name;
// std::string column_value;
// };
// struct TermQuery {
// milvus::json json_obj;
// // std::string field_name;
// // std::vector<uint8_t> field_value;
// // float boost;
// };
// using TermQueryPtr = std::shared_ptr<TermQuery>;
// struct CompareExpr {
// CompareOperator compare_operator;
// std::string operand;
// };
// struct RangeQuery {
// milvus::json json_obj;
// // std::string field_name;
// // std::vector<CompareExpr> compare_expr;
// // float boost;
// };
// using RangeQueryPtr = std::shared_ptr<RangeQuery>;
// struct VectorRecord {
// std::vector<float> float_data;
// std::vector<uint8_t> binary_data;
// };
// struct VectorQuery {
// std::string field_name;
// milvus::json extra_params = {};
// int64_t topk;
// int64_t nq;
// std::string metric_type = "";
// float boost;
// VectorRecord query_vector;
// };
// using VectorQueryPtr = std::shared_ptr<VectorQuery>;
// struct LeafQuery;
// using LeafQueryPtr = std::shared_ptr<LeafQuery>;
// struct BinaryQuery;
// using BinaryQueryPtr = std::shared_ptr<BinaryQuery>;
// struct GeneralQuery {
// LeafQueryPtr leaf;
// BinaryQueryPtr bin = std::make_shared<BinaryQuery>();
// };
// using GeneralQueryPtr = std::shared_ptr<GeneralQuery>;
// struct LeafQuery {
// TermQueryPtr term_query;
// RangeQueryPtr range_query;
// std::string vector_placeholder;
// float query_boost;
// };
// struct BinaryQuery {
// GeneralQueryPtr left_query;
// GeneralQueryPtr right_query;
// QueryRelation relation;
// float query_boost;
// };
// struct Query {
// GeneralQueryPtr root;
// std::unordered_map<std::string, VectorQueryPtr> vectors;
// std::string collection_id;
// std::vector<std::string> partitions;
// std::vector<std::string> field_names;
// std::set<std::string> index_fields;
// std::unordered_map<std::string, std::string> metric_types;
// };
struct Query{
// TODO
};
using QueryPtr = std::shared_ptr<Query>;
} // namespace query
} // namespace milvus

View File

@ -0,0 +1,5 @@
set(UTILS_FILES Status.cpp)
add_library(milvus_utils
${UTILS_FILES}
)

141
core/src/utils/Error.h Normal file
View File

@ -0,0 +1,141 @@
// 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 <cstdint>
#include <exception>
#include <string>
namespace milvus {
using ErrorCode = int32_t;
constexpr ErrorCode SERVER_SUCCESS = 0;
constexpr ErrorCode SERVER_ERROR_CODE_BASE = 30000;
constexpr ErrorCode
ToServerErrorCode(const ErrorCode error_code) {
return SERVER_ERROR_CODE_BASE + error_code;
}
constexpr ErrorCode DB_SUCCESS = 0;
constexpr ErrorCode DB_ERROR_CODE_BASE = 40000;
constexpr ErrorCode
ToDbErrorCode(const ErrorCode error_code) {
return DB_ERROR_CODE_BASE + error_code;
}
constexpr ErrorCode KNOWHERE_SUCCESS = 0;
constexpr ErrorCode KNOWHERE_ERROR_CODE_BASE = 50000;
constexpr ErrorCode
ToKnowhereErrorCode(const ErrorCode error_code) {
return KNOWHERE_ERROR_CODE_BASE + error_code;
}
constexpr ErrorCode WAL_SUCCESS = 0;
constexpr ErrorCode WAL_ERROR_CODE_BASE = 60000;
constexpr ErrorCode
ToWalErrorCode(const ErrorCode error_code) {
return WAL_ERROR_CODE_BASE + error_code;
}
constexpr ErrorCode SS_SUCCESS = 0;
constexpr ErrorCode SS_ERROR_CODE_BASE = 70000;
constexpr ErrorCode
ToSSErrorCode(const ErrorCode error_code) {
return SS_ERROR_CODE_BASE + error_code;
}
// server error code
constexpr ErrorCode SERVER_UNEXPECTED_ERROR = ToServerErrorCode(1);
constexpr ErrorCode SERVER_UNSUPPORTED_ERROR = ToServerErrorCode(2);
constexpr ErrorCode SERVER_NULL_POINTER = ToServerErrorCode(3);
constexpr ErrorCode SERVER_INVALID_ARGUMENT = ToServerErrorCode(4);
constexpr ErrorCode SERVER_FILE_NOT_FOUND = ToServerErrorCode(5);
constexpr ErrorCode SERVER_NOT_IMPLEMENT = ToServerErrorCode(6);
constexpr ErrorCode SERVER_CANNOT_CREATE_FOLDER = ToServerErrorCode(8);
constexpr ErrorCode SERVER_CANNOT_CREATE_FILE = ToServerErrorCode(9);
constexpr ErrorCode SERVER_CANNOT_DELETE_FOLDER = ToServerErrorCode(10);
constexpr ErrorCode SERVER_CANNOT_DELETE_FILE = ToServerErrorCode(11);
constexpr ErrorCode SERVER_BUILD_INDEX_ERROR = ToServerErrorCode(12);
constexpr ErrorCode SERVER_CANNOT_OPEN_FILE = ToServerErrorCode(13);
constexpr ErrorCode SERVER_FILE_MAGIC_BYTES_ERROR = ToServerErrorCode(14);
constexpr ErrorCode SERVER_FILE_SUM_BYTES_ERROR = ToServerErrorCode(15);
constexpr ErrorCode SERVER_CANNOT_READ_FILE = ToServerErrorCode(16);
constexpr ErrorCode SERVER_COLLECTION_NOT_EXIST = ToServerErrorCode(100);
constexpr ErrorCode SERVER_INVALID_COLLECTION_NAME = ToServerErrorCode(101);
constexpr ErrorCode SERVER_INVALID_COLLECTION_DIMENSION = ToServerErrorCode(102);
constexpr ErrorCode SERVER_INVALID_VECTOR_DIMENSION = ToServerErrorCode(104);
constexpr ErrorCode SERVER_INVALID_INDEX_TYPE = ToServerErrorCode(105);
constexpr ErrorCode SERVER_INVALID_ROWRECORD = ToServerErrorCode(106);
constexpr ErrorCode SERVER_INVALID_ROWRECORD_ARRAY = ToServerErrorCode(107);
constexpr ErrorCode SERVER_INVALID_TOPK = ToServerErrorCode(108);
constexpr ErrorCode SERVER_ILLEGAL_VECTOR_ID = ToServerErrorCode(109);
constexpr ErrorCode SERVER_ILLEGAL_SEARCH_RESULT = ToServerErrorCode(110);
constexpr ErrorCode SERVER_CACHE_FULL = ToServerErrorCode(111);
constexpr ErrorCode SERVER_WRITE_ERROR = ToServerErrorCode(112);
constexpr ErrorCode SERVER_INVALID_NPROBE = ToServerErrorCode(113);
constexpr ErrorCode SERVER_INVALID_INDEX_NLIST = ToServerErrorCode(114);
constexpr ErrorCode SERVER_INVALID_INDEX_METRIC_TYPE = ToServerErrorCode(115);
constexpr ErrorCode SERVER_INVALID_SEGMENT_ROW_COUNT = ToServerErrorCode(116);
constexpr ErrorCode SERVER_OUT_OF_MEMORY = ToServerErrorCode(117);
constexpr ErrorCode SERVER_INVALID_PARTITION_TAG = ToServerErrorCode(118);
constexpr ErrorCode SERVER_INVALID_BINARY_QUERY = ToServerErrorCode(119);
constexpr ErrorCode SERVER_INVALID_DSL_PARAMETER = ToServerErrorCode(120);
constexpr ErrorCode SERVER_INVALID_FIELD_NAME = ToServerErrorCode(121);
constexpr ErrorCode SERVER_INVALID_FIELD_NUM = ToServerErrorCode(122);
// db error code
constexpr ErrorCode DB_META_TRANSACTION_FAILED = ToDbErrorCode(1);
constexpr ErrorCode DB_ERROR = ToDbErrorCode(2);
constexpr ErrorCode DB_NOT_FOUND = ToDbErrorCode(3);
constexpr ErrorCode DB_ALREADY_EXIST = ToDbErrorCode(4);
constexpr ErrorCode DB_INVALID_PATH = ToDbErrorCode(5);
constexpr ErrorCode DB_INCOMPATIB_META = ToDbErrorCode(6);
constexpr ErrorCode DB_INVALID_META_URI = ToDbErrorCode(7);
constexpr ErrorCode DB_EMPTY_COLLECTION = ToDbErrorCode(8);
constexpr ErrorCode DB_BLOOM_FILTER_ERROR = ToDbErrorCode(9);
constexpr ErrorCode DB_PARTITION_NOT_FOUND = ToDbErrorCode(10);
constexpr ErrorCode DB_OUT_OF_STORAGE = ToDbErrorCode(11);
constexpr ErrorCode DB_META_QUERY_FAILED = ToDbErrorCode(12);
// knowhere error code
constexpr ErrorCode KNOWHERE_ERROR = ToKnowhereErrorCode(1);
constexpr ErrorCode KNOWHERE_INVALID_ARGUMENT = ToKnowhereErrorCode(2);
constexpr ErrorCode KNOWHERE_UNEXPECTED_ERROR = ToKnowhereErrorCode(3);
constexpr ErrorCode KNOWHERE_NO_SPACE = ToKnowhereErrorCode(4);
// knowhere error code
constexpr ErrorCode WAL_ERROR = ToWalErrorCode(1);
constexpr ErrorCode WAL_META_ERROR = ToWalErrorCode(2);
constexpr ErrorCode WAL_FILE_ERROR = ToWalErrorCode(3);
constexpr ErrorCode WAL_PATH_ERROR = ToWalErrorCode(4);
// Snapshot error code
constexpr ErrorCode SS_ERROR = ToSSErrorCode(1);
constexpr ErrorCode SS_STALE_ERROR = ToSSErrorCode(2);
constexpr ErrorCode SS_NOT_FOUND_ERROR = ToSSErrorCode(3);
constexpr ErrorCode SS_INVALID_CONTEX_ERROR = ToSSErrorCode(4);
constexpr ErrorCode SS_DUPLICATED_ERROR = ToSSErrorCode(5);
constexpr ErrorCode SS_NOT_ACTIVE_ERROR = ToSSErrorCode(6);
constexpr ErrorCode SS_CONSTRAINT_CHECK_ERROR = ToSSErrorCode(7);
constexpr ErrorCode SS_INVALID_ARGUMENT_ERROR = ToSSErrorCode(8);
constexpr ErrorCode SS_OPERATION_PENDING = ToSSErrorCode(9);
constexpr ErrorCode SS_TIMEOUT = ToSSErrorCode(10);
constexpr ErrorCode SS_NOT_COMMITED = ToSSErrorCode(11);
} // namespace milvus

130
core/src/utils/Status.cpp Normal file
View File

@ -0,0 +1,130 @@
// 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 "utils/Status.h"
#include <cstring>
namespace milvus {
constexpr int CODE_WIDTH = sizeof(StatusCode);
Status::Status(StatusCode code, const std::string& msg) {
// 4 bytes store code
// 4 bytes store message length
// the left bytes store message string
auto length = static_cast<uint32_t>(msg.size());
auto result = new char[length + sizeof(length) + CODE_WIDTH];
std::memcpy(result, &code, CODE_WIDTH);
std::memcpy(result + CODE_WIDTH, &length, sizeof(length));
memcpy(result + sizeof(length) + CODE_WIDTH, msg.data(), length);
state_ = result;
}
Status::~Status() {
delete state_;
}
Status::Status(const Status& s) {
CopyFrom(s);
}
Status::Status(Status&& s) noexcept {
MoveFrom(s);
}
Status&
Status::operator=(const Status& s) {
CopyFrom(s);
return *this;
}
Status&
Status::operator=(Status&& s) noexcept {
MoveFrom(s);
return *this;
}
void
Status::CopyFrom(const Status& s) {
delete state_;
state_ = nullptr;
if (s.state_ == nullptr) {
return;
}
uint32_t length = 0;
memcpy(&length, s.state_ + CODE_WIDTH, sizeof(length));
int buff_len = length + sizeof(length) + CODE_WIDTH;
state_ = new char[buff_len];
memcpy(state_, s.state_, buff_len);
}
void
Status::MoveFrom(Status& s) {
delete state_;
state_ = s.state_;
s.state_ = nullptr;
}
std::string
Status::message() const {
if (state_ == nullptr) {
return "OK";
}
std::string msg;
uint32_t length = 0;
memcpy(&length, state_ + CODE_WIDTH, sizeof(length));
if (length > 0) {
msg.append(state_ + sizeof(length) + CODE_WIDTH, length);
}
return msg;
}
std::string
Status::ToString() const {
if (state_ == nullptr) {
return "OK";
}
std::string result;
switch (code()) {
case DB_SUCCESS:
result = "OK ";
break;
case DB_ERROR:
result = "Error: ";
break;
case DB_META_TRANSACTION_FAILED:
result = "Database error: ";
break;
case DB_NOT_FOUND:
result = "Not found: ";
break;
case DB_ALREADY_EXIST:
result = "Already exist: ";
break;
case DB_INVALID_PATH:
result = "Invalid path: ";
break;
default:
result = "Error code(" + std::to_string(code()) + "): ";
break;
}
result += message();
return result;
}
} // namespace milvus

79
core/src/utils/Status.h Normal file
View File

@ -0,0 +1,79 @@
// 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 "utils/Error.h"
#include <string>
namespace milvus {
class Status;
#define STATUS_CHECK(func) \
do { \
Status s = func; \
if (!s.ok()) { \
return s; \
} \
} while (false)
using StatusCode = ErrorCode;
class Status {
public:
Status(StatusCode code, const std::string& msg);
Status() = default;
virtual ~Status();
Status(const Status& s);
Status(Status&& s) noexcept;
Status&
operator=(const Status& s);
Status&
operator=(Status&& s) noexcept;
static Status
OK() {
return Status();
}
bool
ok() const {
return state_ == nullptr || code() == 0;
}
StatusCode
code() const {
return (state_ == nullptr) ? 0 : *(StatusCode*)(state_);
}
std::string
message() const;
std::string
ToString() const;
private:
inline void
CopyFrom(const Status& s);
inline void
MoveFrom(Status& s);
private:
char* state_ = nullptr;
}; // Status
} // namespace milvus

148
core/src/utils/Types.h Normal file
View File

@ -0,0 +1,148 @@
// 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 <faiss/Index.h>
#include <cstdint>
#include <map>
#include <memory>
#include <set>
#include <string>
#include <unordered_map>
#include <utility>
#include <vector>
// #include "utils/Json.h"
namespace milvus {
namespace engine {
///////////////////////////////////////////////////////////////////////////////////////////////////
using idx_t = int64_t;
using offset_t = int32_t;
using date_t = int32_t;
using distance_t = float;
using IDNumbers = std::vector<idx_t>;
// using VectorDistance = faiss::Index::distance_t;
// using VectorDistances = std::vector<VectorDistance>;
using ResultIds = std::vector<idx_t>;
using ResultDistances = std::vector<distance_t>;
///////////////////////////////////////////////////////////////////////////////////////////////////
enum class DataType {
NONE = 0,
BOOL = 1,
INT8 = 2,
INT16 = 3,
INT32 = 4,
INT64 = 5,
FLOAT = 10,
DOUBLE = 11,
STRING = 20,
VECTOR_BINARY = 100,
VECTOR_FLOAT = 101,
};
///////////////////////////////////////////////////////////////////////////////////////////////////
enum class FieldElementType {
FET_NONE = 0,
FET_RAW = 1,
FET_BLOOM_FILTER = 2,
FET_DELETED_DOCS = 3,
FET_INDEX = 4,
FET_COMPRESS_SQ8 = 5,
};
/////////////////////////////////////////////////////////////////////////////////////////////////////
//class BinaryData : public cache::DataObj {
// public:
// int64_t
// Size() {
// return data_.size();
// }
//
// public:
// std::vector<uint8_t> data_;
//};
//using BinaryDataPtr = std::shared_ptr<BinaryData>;
//
/////////////////////////////////////////////////////////////////////////////////////////////////////
//class VaribleData : public cache::DataObj {
// public:
// int64_t
// Size() {
// return data_.size() + offset_.size() * sizeof(int64_t);
// }
//
// public:
// std::vector<uint8_t> data_;
// std::vector<int64_t> offset_;
//};
//using VaribleDataPtr = std::shared_ptr<VaribleData>;
//
/////////////////////////////////////////////////////////////////////////////////////////////////////
//using FIELD_TYPE_MAP = std::unordered_map<std::string, DataType>;
//using FIELD_WIDTH_MAP = std::unordered_map<std::string, int64_t>;
//using FIXEDX_FIELD_MAP = std::unordered_map<std::string, BinaryDataPtr>;
//using VARIABLE_FIELD_MAP = std::unordered_map<std::string, VaribleDataPtr>;
//using VECTOR_INDEX_MAP = std::unordered_map<std::string, knowhere::VecIndexPtr>;
//using STRUCTURED_INDEX_MAP = std::unordered_map<std::string, knowhere::IndexPtr>;
// ///////////////////////////////////////////////////////////////////////////////////////////////////
// struct DataChunk {
// int64_t count_ = 0;
// FIXEDX_FIELD_MAP fixed_fields_;
// VARIABLE_FIELD_MAP variable_fields_;
// };
// using DataChunkPtr = std::shared_ptr<DataChunk>;
// ///////////////////////////////////////////////////////////////////////////////////////////////////
// struct CollectionIndex {
// std::string index_name_;
// std::string index_type_;
// std::string metric_name_;
// milvus::json extra_params_ = {{"nlist", 2048}};
// };
///////////////////////////////////////////////////////////////////////////////////////////////////
struct VectorsData {
uint64_t vector_count_ = 0;
std::vector<float> float_data_;
std::vector<uint8_t> binary_data_;
IDNumbers id_array_;
};
///////////////////////////////////////////////////////////////////////////////////////////////////
struct AttrsData {
uint64_t attr_count_ = 0;
std::unordered_map<std::string, engine::DataType> attr_type_;
std::unordered_map<std::string, std::vector<uint8_t>> attr_data_;
IDNumbers id_array_;
};
///////////////////////////////////////////////////////////////////////////////////////////////////
struct QueryResult {
uint64_t row_num_;
engine::ResultIds result_ids_;
engine::ResultDistances result_distances_;
// engine::DataChunkPtr data_chunk_;
};
using QueryResultPtr = std::shared_ptr<QueryResult>;
} // namespace engine
} // namespace milvus

View File

@ -0,0 +1,267 @@
// 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 <fiu-control.h>
// #include <fiu-local.h>
// #include <gtest/gtest.h>
#include <iostream>
#include <string>
// #include "db/SnapshotVisitor.h"
// #include "db/Types.h"
// #include "db/snapshot/IterateHandler.h"
// #include "db/snapshot/Resources.h"
// #include "db/utils.h"
// #include "knowhere/index/vector_index/helpers/IndexParameter.h"
// #include "segment/SegmentReader.h"
// #include "segment/SegmentWriter.h"
// #include "src/dog_segment/SegmentBase.h"
// #include "utils/Json.h"
#include "dog_segment/SegmentBase.h"
using std::cin;
using std::cout;
using std::endl;
using SegmentVisitor = milvus::engine::SegmentVisitor;
namespace {
// milvus::Status
// CreateCollection(std::shared_ptr<DB> db, const std::string& collection_name, const LSN_TYPE& lsn) {
// CreateCollectionContext context;
// context.lsn = lsn;
// auto collection_schema = std::make_shared<Collection>(collection_name);
// context.collection = collection_schema;
// int64_t collection_id = 0;
// int64_t field_id = 0;
// /* field uid */
// auto uid_field = std::make_shared<Field>(milvus::engine::FIELD_UID, 0, milvus::engine::DataType::INT64,
// milvus::engine::snapshot::JEmpty, field_id);
// auto uid_field_element_blt =
// std::make_shared<FieldElement>(collection_id, field_id, milvus::engine::ELEMENT_BLOOM_FILTER,
// milvus::engine::FieldElementType::FET_BLOOM_FILTER);
// auto uid_field_element_del =
// std::make_shared<FieldElement>(collection_id, field_id, milvus::engine::ELEMENT_DELETED_DOCS,
// milvus::engine::FieldElementType::FET_DELETED_DOCS);
// field_id++;
// /* field vector */
// milvus::json vector_param = {{milvus::knowhere::meta::DIM, 4}};
// auto vector_field =
// std::make_shared<Field>("vector", 0, milvus::engine::DataType::VECTOR_FLOAT, vector_param, field_id);
// auto vector_field_element_index =
// std::make_shared<FieldElement>(collection_id, field_id, milvus::knowhere::IndexEnum::INDEX_FAISS_IVFSQ8,
// milvus::engine::FieldElementType::FET_INDEX);
// /* another field*/
// auto int_field = std::make_shared<Field>("int", 0, milvus::engine::DataType::INT32,
// milvus::engine::snapshot::JEmpty, field_id++);
// context.fields_schema[uid_field] = {uid_field_element_blt, uid_field_element_del};
// context.fields_schema[vector_field] = {vector_field_element_index};
// context.fields_schema[int_field] = {};
// return db->CreateCollection(context);
// }
// } // namespace
TEST_F(DogSegmentTest, TestABI) {
using namespace milvus::engine;
using namespace milvus::dog_segment;
ASSERT_EQ(TestABI(), 42);
assert(true);
}
// TEST_F(DogSegmentTest, TestCreateAndSchema) {
// using namespace milvus::engine;
// using namespace milvus::dog_segment;
// // step1: create segment from current snapshot.
// LSN_TYPE lsn = 0;
// auto next_lsn = [&]() -> decltype(lsn) { return ++lsn; };
// // step 1.1: create collection
// std::string db_root = "/tmp/milvus_test/db/table";
// std::string collection_name = "c1";
// auto status = CreateCollection(db_, collection_name, next_lsn());
// ASSERT_TRUE(status.ok());
// // step 1.2: get snapshot
// ScopedSnapshotT snapshot;
// status = Snapshots::GetInstance().GetSnapshot(snapshot, collection_name);
// ASSERT_TRUE(status.ok());
// ASSERT_TRUE(snapshot);
// ASSERT_EQ(snapshot->GetName(), collection_name);
// // step 1.3: get partition_id
// cout << endl;
// cout << endl;
// ID_TYPE partition_id = snapshot->GetResources<Partition>().begin()->first;
// cout << partition_id;
// // step 1.5 create schema from ids
// auto collection = snapshot->GetCollection();
// auto field_names = snapshot->GetFieldNames();
// auto schema = std::make_shared<Schema>();
// for (const auto& field_name : field_names) {
// auto the_field = snapshot->GetField(field_name);
// auto param = the_field->GetParams();
// auto type = the_field->GetFtype();
// cout << field_name //
// << " " << (int)type //
// << " " << param //
// << endl;
// FieldMeta field(field_name, type);
// int dim = 1;
// if(field.is_vector()) {
// field.set_dim(dim);
// }
// schema->AddField(field);
// }
// // step 1.6 create a segment from ids
// auto segment = CreateSegment(schema);
// std::vector<id_t> primary_ids;
// }
TEST_F(DogSegmentTest, MockTest) {
using namespace milvus::dog_segment;
using namespace milvus::engine;
auto schema = std::make_shared<Schema>();
schema->AddField("fakevec", DataType::VECTOR_FLOAT, 16);
schema->AddField("age", DataType::INT32);
std::vector<char> raw_data;
std::vector<Timestamp> timestamps;
std::vector<uint64_t> uids;
int N = 10000;
std::default_random_engine e(67);
for(int i = 0; i < N; ++i) {
uids.push_back(100000 + i);
timestamps.push_back(0);
// append vec
float vec[16];
for(auto &x: vec) {
x = e() % 2000 * 0.001 - 1.0;
}
raw_data.insert(raw_data.end(), (const char*)std::begin(vec), (const char*)std::end(vec));
int age = e() % 100;
raw_data.insert(raw_data.end(), (const char*)&age, ((const char*)&age) + sizeof(age));
}
auto line_sizeof = (sizeof(int) + sizeof(float) * 16);
assert(raw_data.size() == line_sizeof * N);
auto segment = CreateSegment(schema);
DogDataChunk data_chunk{raw_data.data(), (int)line_sizeof, N};
segment->Insert(N, uids.data(), timestamps.data(), data_chunk);
QueryResult query_result;
segment->Query(nullptr, 0, query_result);
int i = 0;
i++;
}
//TEST_F(DogSegmentTest, DogSegmentTest) {
// LSN_TYPE lsn = 0;
// auto next_lsn = [&]() -> decltype(lsn) { return ++lsn; };
//
// std::string db_root = "/tmp/milvus_test/db/table";
// std::string c1 = "c1";
// auto status = CreateCollection(db_, c1, next_lsn());
// ASSERT_TRUE(status.ok());
//
// ScopedSnapshotT snapshot;
// status = Snapshots::GetInstance().GetSnapshot(snapshot, c1);
// ASSERT_TRUE(status.ok());
// ASSERT_TRUE(snapshot);
// ASSERT_EQ(snapshot->GetName(), c1);
// {
// SegmentFileContext sf_context;
// SFContextBuilder(sf_context, snapshot);
// }
// std::vector<SegmentFileContext> segfile_ctxs;
// SFContextsBuilder(segfile_ctxs, snapshot);
//
// std::cout << snapshot->ToString() << std::endl;
//
// ID_TYPE partition_id;
// {
// auto& partitions = snapshot->GetResources<Partition>();
// partition_id = partitions.begin()->first;
// }
//
// [&next_lsn, //
// &segfile_ctxs, //
// &partition_id, //
// &snapshot, //
// &db_root] {
// /* commit new segment */
// OperationContext op_ctx;
// op_ctx.lsn = next_lsn();
// op_ctx.prev_partition = snapshot->GetResource<Partition>(partition_id);
//
// auto new_seg_op = std::make_shared<NewSegmentOperation>(op_ctx, snapshot);
// SegmentPtr new_seg;
// auto status = new_seg_op->CommitNewSegment(new_seg);
// ASSERT_TRUE(status.ok());
//
// /* commit new segment file */
// for (auto& cctx : segfile_ctxs) {
// SegmentFilePtr seg_file;
// auto nsf_context = cctx;
// nsf_context.segment_id = new_seg->GetID();
// nsf_context.partition_id = new_seg->GetPartitionId();
// status = new_seg_op->CommitNewSegmentFile(nsf_context, seg_file);
// }
//
// /* build segment visitor */
// auto ctx = new_seg_op->GetContext();
// ASSERT_TRUE(ctx.new_segment);
// auto visitor = SegmentVisitor::Build(snapshot, ctx.new_segment, ctx.new_segment_files);
// ASSERT_TRUE(visitor);
// ASSERT_EQ(visitor->GetSegment(), new_seg);
// ASSERT_FALSE(visitor->GetSegment()->IsActive());
// // std::cout << visitor->ToString() << std::endl;
// // std::cout << snapshot->ToString() << std::endl;
//
// /* write data */
// milvus::segment::SegmentWriter segment_writer(db_root, visitor);
//
// // std::vector<milvus::segment::doc_id_t> raw_uids = {123};
// // std::vector<uint8_t> raw_vectors = {1, 2, 3, 4};
// // status = segment_writer.AddChunk("test", raw_vectors, raw_uids);
// // ASSERT_TRUE(status.ok())
// //
// // status = segment_writer.Serialize();
// // ASSERT_TRUE(status.ok());
//
// /* read data */
// // milvus::segment::SSSegmentReader segment_reader(db_root, visitor);
// //
// // status = segment_reader.Load();
// // ASSERT_TRUE(status.ok());
// //
// // milvus::segment::SegmentPtr segment_ptr;
// // status = segment_reader.GetSegment(segment_ptr);
// // ASSERT_TRUE(status.ok());
// //
// // auto& out_uids = segment_ptr->vectors_ptr_->GetUids();
// // ASSERT_EQ(raw_uids.size(), out_uids.size());
// // ASSERT_EQ(raw_uids[0], out_uids[0]);
// // auto& out_vectors = segment_ptr->vectors_ptr_->GetData();
// // ASSERT_EQ(raw_vectors.size(), out_vectors.size());
// // ASSERT_EQ(raw_vectors[0], out_vectors[0]);
// }();
//
// status = db_->DropCollection(c1);
// ASSERT_TRUE(status.ok());
//}