diff --git a/core/CMakeLists.txt b/core/CMakeLists.txt new file mode 100644 index 0000000000..79ae64f03b --- /dev/null +++ b/core/CMakeLists.txt @@ -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) diff --git a/core/src/CMakeLists.txt b/core/src/CMakeLists.txt new file mode 100644 index 0000000000..746c72aac8 --- /dev/null +++ b/core/src/CMakeLists.txt @@ -0,0 +1,4 @@ +add_subdirectory(utils) +add_subdirectory(dog_segment) +#add_subdirectory(index) +add_subdirectory(query) diff --git a/core/src/dog_segment/CMakeLists.txt b/core/src/dog_segment/CMakeLists.txt new file mode 100644 index 0000000000..4f0744641e --- /dev/null +++ b/core/src/dog_segment/CMakeLists.txt @@ -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) \ No newline at end of file diff --git a/core/src/dog_segment/Collection.h b/core/src/dog_segment/Collection.h new file mode 100644 index 0000000000..d7973e25cc --- /dev/null +++ b/core/src/dog_segment/Collection.h @@ -0,0 +1,51 @@ +#pragma once + +#include "SegmentDefs.h" + +////////////////////////////////////////////////////////////////// + +class Partition { +public: + const std::deque& segments() const { + return segments_; + } + +private: + std::string name_; + std::deque segments_; +}; + +using PartitionPtr = std::shard_ptr; + +////////////////////////////////////////////////////////////////// + +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 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 partitions_; +}; diff --git a/core/src/dog_segment/SegmentBase.h b/core/src/dog_segment/SegmentBase.h new file mode 100644 index 0000000000..a5f53c68c9 --- /dev/null +++ b/core/src/dog_segment/SegmentBase.h @@ -0,0 +1,90 @@ +#pragma once +#include + +// #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 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& 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 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 CreateSegment(SchemaPtr ptr); + +} // namespace engine +} // namespace milvus diff --git a/core/src/dog_segment/SegmentDefs.h b/core/src/dog_segment/SegmentDefs.h new file mode 100644 index 0000000000..d1b56d4356 --- /dev/null +++ b/core/src/dog_segment/SegmentDefs.h @@ -0,0 +1,173 @@ +#pragma once + +#include +#include + +// #include "db/Types.h" +// #include "knowhere/index/Index.h" +#include "utils/Status.h" +#include "utils/Types.h" +#include + +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 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 fields_; + + private: + // a mapping for random access + std::unordered_map indexes_; + int total_sizeof_; +}; + +using SchemaPtr = std::shared_ptr; + +class IndexData { + public: + virtual std::vector + serilize() = 0; + + static std::shared_ptr + deserialize(int64_t size, const char* blob); +}; + +} // namespace milvus::dog_segment diff --git a/core/src/dog_segment/SegmentNaive.cpp b/core/src/dog_segment/SegmentNaive.cpp new file mode 100644 index 0000000000..cbb9afdc45 --- /dev/null +++ b/core/src/dog_segment/SegmentNaive.cpp @@ -0,0 +1,241 @@ +#include + +#include "dog_segment/SegmentBase.h" +#include "utils/Status.h" +#include +#include +#include + +namespace milvus::dog_segment { + +int +TestABI() { + return 42; +} + +struct ColumnBasedDataChunk { + std::vector> entity_vecs; + static ColumnBasedDataChunk from(const DogDataChunk& source, const Schema& schema){ + ColumnBasedDataChunk dest; + auto count = source.count; + auto raw_data = reinterpret_cast(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 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 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& ids, DataChunkPtr& results) = 0; + + // stop receive insert requests + Status + Close() override { + std::lock_guard 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 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 + // get_index_data(); + + ssize_t + get_deleted_count() const override { + return 0; + } + + public: + friend std::unique_ptr + CreateSegment(SchemaPtr schema); + private: + SchemaPtr schema_; + std::shared_mutex mutex_; + std::atomic state_ = SegmentState::Open; + std::atomic ack_count_ = 0; + tbb::concurrent_vector uids_; + tbb::concurrent_vector timestamps_; + std::vector> entity_vecs_; + tbb::concurrent_unordered_map internal_indexes_; + + tbb::concurrent_unordered_multimap delete_logs_; +}; + +std::unique_ptr +CreateSegment(SchemaPtr schema) { + auto segment = std::make_unique(); + 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 query_vector(dim, 0); + auto& target_vec = entity_vecs_[0]; + int current_index = -1; + float min_diff = std::numeric_limits::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 + + + + + + + + + + diff --git a/core/src/query/BinaryQuery.cpp b/core/src/query/BinaryQuery.cpp new file mode 100644 index 0000000000..4fdd714847 --- /dev/null +++ b/core/src/query/BinaryQuery.cpp @@ -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 +#include +#include +#include +#include +#include + +#include "query/BinaryQuery.h" + +namespace milvus { +namespace query { + +BinaryQueryPtr +ConstructBinTree(std::vector 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(); + bquery->relation = relation; + bquery->left_query = std::make_shared(); + bquery->right_query = std::make_shared(); + bquery->left_query->bin = queries[idx]->getBinaryQuery(); + ++idx; + bquery->right_query->bin = ConstructBinTree(queries, relation, idx); + return bquery; + } +} + +Status +ConstructLeafBinTree(std::vector leaf_queries, BinaryQueryPtr binary_query, uint64_t idx) { + if (idx == leaf_queries.size()) { + return Status::OK(); + } + binary_query->left_query = std::make_shared(); + binary_query->right_query = std::make_shared(); + 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(); + 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 must_queries; + std::vector must_not_queries; + std::vector 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(); + binary_query->right_query = std::make_shared(); + BinaryQueryPtr must_should_query = std::make_shared(); + must_should_query->left_query = std::make_shared(); + must_should_query->right_query = std::make_shared(); + 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& 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 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 diff --git a/core/src/query/BinaryQuery.h b/core/src/query/BinaryQuery.h new file mode 100644 index 0000000000..5912892847 --- /dev/null +++ b/core/src/query/BinaryQuery.h @@ -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 +#include + +#include "BooleanQuery.h" + +namespace milvus { +namespace query { + +BinaryQueryPtr +ConstructBinTree(std::vector clauses, QueryRelation relation, uint64_t idx); + +Status +ConstructLeafBinTree(std::vector 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 diff --git a/core/src/query/BooleanQuery.h b/core/src/query/BooleanQuery.h new file mode 100644 index 0000000000..7b743c7a37 --- /dev/null +++ b/core/src/query/BooleanQuery.h @@ -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 +#include + +#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 boolean_clause) { + boolean_clauses_.emplace_back(boolean_clause); + } + + void + AddLeafQuery(LeafQueryPtr leaf_query) { + leaf_queries_.emplace_back(leaf_query); + } + + void + SetLeafQuery(std::vector leaf_queries) { + leaf_queries_ = leaf_queries; + } + + std::vector> + getBooleanQueries() { + return boolean_clauses_; + } + + BinaryQueryPtr& + getBinaryQuery() { + return binary_query_; + } + + std::vector& + getLeafQueries() { + return leaf_queries_; + } + + private: + Occur occur_ = Occur::INVALID; + std::vector> boolean_clauses_; + std::vector leaf_queries_; + BinaryQueryPtr binary_query_ = std::make_shared(); +}; +using BooleanQueryPtr = std::shared_ptr; + +} // namespace query +} // namespace milvus diff --git a/core/src/query/CMakeLists.txt b/core/src/query/CMakeLists.txt new file mode 100644 index 0000000000..b6dbda7f35 --- /dev/null +++ b/core/src/query/CMakeLists.txt @@ -0,0 +1,2 @@ +# TODO + diff --git a/core/src/query/GeneralQuery.h b/core/src/query/GeneralQuery.h new file mode 100644 index 0000000000..622aa15f55 --- /dev/null +++ b/core/src/query/GeneralQuery.h @@ -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 +#include +#include +#include +#include +#include + +// #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 field_value; +// // float boost; +// }; +// using TermQueryPtr = std::shared_ptr; + +// struct CompareExpr { +// CompareOperator compare_operator; +// std::string operand; +// }; + +// struct RangeQuery { +// milvus::json json_obj; +// // std::string field_name; +// // std::vector compare_expr; +// // float boost; +// }; +// using RangeQueryPtr = std::shared_ptr; + +// struct VectorRecord { +// std::vector float_data; +// std::vector 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; + +// struct LeafQuery; +// using LeafQueryPtr = std::shared_ptr; + +// struct BinaryQuery; +// using BinaryQueryPtr = std::shared_ptr; + +// struct GeneralQuery { +// LeafQueryPtr leaf; +// BinaryQueryPtr bin = std::make_shared(); +// }; +// using GeneralQueryPtr = std::shared_ptr; + +// 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 vectors; + +// std::string collection_id; +// std::vector partitions; +// std::vector field_names; +// std::set index_fields; +// std::unordered_map metric_types; +// }; +struct Query{ + // TODO +}; +using QueryPtr = std::shared_ptr; + +} // namespace query +} // namespace milvus diff --git a/core/src/utils/CMakeLists.txt b/core/src/utils/CMakeLists.txt new file mode 100644 index 0000000000..c932d67b1f --- /dev/null +++ b/core/src/utils/CMakeLists.txt @@ -0,0 +1,5 @@ + +set(UTILS_FILES Status.cpp) +add_library(milvus_utils + ${UTILS_FILES} +) \ No newline at end of file diff --git a/core/src/utils/Error.h b/core/src/utils/Error.h new file mode 100644 index 0000000000..a82b4570db --- /dev/null +++ b/core/src/utils/Error.h @@ -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 +#include +#include + +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 diff --git a/core/src/utils/Status.cpp b/core/src/utils/Status.cpp new file mode 100644 index 0000000000..5cf627ef32 --- /dev/null +++ b/core/src/utils/Status.cpp @@ -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 + +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(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 diff --git a/core/src/utils/Status.h b/core/src/utils/Status.h new file mode 100644 index 0000000000..1a45e7bb73 --- /dev/null +++ b/core/src/utils/Status.h @@ -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 + +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 diff --git a/core/src/utils/Types.h b/core/src/utils/Types.h new file mode 100644 index 0000000000..8da7b795e7 --- /dev/null +++ b/core/src/utils/Types.h @@ -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 + +#include +#include +#include +#include +#include +#include +#include +#include + +// #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; + +// using VectorDistance = faiss::Index::distance_t; +// using VectorDistances = std::vector; + +using ResultIds = std::vector; +using ResultDistances = std::vector; + +/////////////////////////////////////////////////////////////////////////////////////////////////// +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 data_; +//}; +//using BinaryDataPtr = std::shared_ptr; +// +///////////////////////////////////////////////////////////////////////////////////////////////////// +//class VaribleData : public cache::DataObj { +// public: +// int64_t +// Size() { +// return data_.size() + offset_.size() * sizeof(int64_t); +// } +// +// public: +// std::vector data_; +// std::vector offset_; +//}; +//using VaribleDataPtr = std::shared_ptr; +// +///////////////////////////////////////////////////////////////////////////////////////////////////// +//using FIELD_TYPE_MAP = std::unordered_map; +//using FIELD_WIDTH_MAP = std::unordered_map; +//using FIXEDX_FIELD_MAP = std::unordered_map; +//using VARIABLE_FIELD_MAP = std::unordered_map; +//using VECTOR_INDEX_MAP = std::unordered_map; +//using STRUCTURED_INDEX_MAP = std::unordered_map; + +// /////////////////////////////////////////////////////////////////////////////////////////////////// +// struct DataChunk { +// int64_t count_ = 0; +// FIXEDX_FIELD_MAP fixed_fields_; +// VARIABLE_FIELD_MAP variable_fields_; +// }; +// using DataChunkPtr = std::shared_ptr; + +// /////////////////////////////////////////////////////////////////////////////////////////////////// +// 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_data_; + std::vector binary_data_; + IDNumbers id_array_; +}; + +/////////////////////////////////////////////////////////////////////////////////////////////////// +struct AttrsData { + uint64_t attr_count_ = 0; + std::unordered_map attr_type_; + std::unordered_map> 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; + +} // namespace engine +} // namespace milvus diff --git a/core/milvus_core.cpp b/core/unittest/CMakeLists.txt similarity index 100% rename from core/milvus_core.cpp rename to core/unittest/CMakeLists.txt diff --git a/core/unittest/test_dog_segment.cpp b/core/unittest/test_dog_segment.cpp new file mode 100644 index 0000000000..7a89a9f461 --- /dev/null +++ b/core/unittest/test_dog_segment.cpp @@ -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 +// #include +// #include + +#include +#include + +// #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, const std::string& collection_name, const LSN_TYPE& lsn) { +// CreateCollectionContext context; +// context.lsn = lsn; +// auto collection_schema = std::make_shared(collection_name); +// context.collection = collection_schema; + +// int64_t collection_id = 0; +// int64_t field_id = 0; +// /* field uid */ +// auto uid_field = std::make_shared(milvus::engine::FIELD_UID, 0, milvus::engine::DataType::INT64, +// milvus::engine::snapshot::JEmpty, field_id); +// auto uid_field_element_blt = +// std::make_shared(collection_id, field_id, milvus::engine::ELEMENT_BLOOM_FILTER, +// milvus::engine::FieldElementType::FET_BLOOM_FILTER); +// auto uid_field_element_del = +// std::make_shared(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("vector", 0, milvus::engine::DataType::VECTOR_FLOAT, vector_param, field_id); +// auto vector_field_element_index = +// std::make_shared(collection_id, field_id, milvus::knowhere::IndexEnum::INDEX_FAISS_IVFSQ8, +// milvus::engine::FieldElementType::FET_INDEX); +// /* another field*/ +// auto int_field = std::make_shared("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().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(); +// 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 primary_ids; +// } + + + +TEST_F(DogSegmentTest, MockTest) { + using namespace milvus::dog_segment; + using namespace milvus::engine; + auto schema = std::make_shared(); + schema->AddField("fakevec", DataType::VECTOR_FLOAT, 16); + schema->AddField("age", DataType::INT32); + std::vector raw_data; + std::vector timestamps; + std::vector 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 segfile_ctxs; +// SFContextsBuilder(segfile_ctxs, snapshot); +// +// std::cout << snapshot->ToString() << std::endl; +// +// ID_TYPE partition_id; +// { +// auto& partitions = snapshot->GetResources(); +// 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_id); +// +// auto new_seg_op = std::make_shared(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 raw_uids = {123}; +// // std::vector 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()); +//}