diff --git a/core/formats/columnstore2.cpp b/core/formats/columnstore2.cpp index bff5943e7..828f4afc0 100644 --- a/core/formats/columnstore2.cpp +++ b/core/formats/columnstore2.cpp @@ -1349,6 +1349,9 @@ void column::flush_block() { math::ceil64(docs_count, packed::BLOCK_SIZE_64); auto* begin = addr_table_.begin(); auto* end = begin + addr_table_size; + if (auto* it = addr_table_.current(); it != end) { + std::memset(it, 0, (end - it) * sizeof(*it)); + } bool all_equal = !data_.file.length(); if (!all_equal) { @@ -1431,6 +1434,7 @@ column::column(const context& ctx, field_id id, const type_info& compression, blocks_{{resource_manager}}, data_{resource_manager}, docs_{resource_manager}, + addr_table_{{resource_manager}}, id_{id} { IRS_ASSERT(field_limits::valid(id_)); } diff --git a/core/formats/columnstore2.hpp b/core/formats/columnstore2.hpp index 27146644c..125de1f58 100644 --- a/core/formats/columnstore2.hpp +++ b/core/formats/columnstore2.hpp @@ -82,45 +82,46 @@ class column final : public irs::column_output { class address_table { public: + address_table(ManagedTypedAllocator alloc) : alloc_{alloc} { + offsets_ = alloc_.allocate(kBlockSize); + offset_ = offsets_; + } + + ~address_table() { alloc_.deallocate(offsets_, kBlockSize); } + uint64_t back() const noexcept { - IRS_ASSERT(offset_ > offsets_); - return *(offset_ - 1); + IRS_ASSERT(offsets_ < offset_); + return offset_[-1]; } void push_back(uint64_t offset) noexcept { - IRS_ASSERT(offset_ >= offsets_); IRS_ASSERT(offset_ < offsets_ + kBlockSize); *offset_++ = offset; - IRS_ASSERT(offset >= offset_[-1]); } void pop_back() noexcept { - IRS_ASSERT(offset_ > offsets_); - *--offset_ = 0; + IRS_ASSERT(offsets_ < offset_); + --offset_; } - // returns number of items to be flushed uint32_t size() const noexcept { - IRS_ASSERT(offset_ >= offsets_); - return uint32_t(offset_ - offsets_); + return static_cast(offset_ - offsets_); } bool empty() const noexcept { return offset_ == offsets_; } - bool full() const noexcept { return offset_ == std::end(offsets_); } + bool full() const noexcept { return offset_ == offsets_ + kBlockSize; } - void reset() noexcept { - std::memset(offsets_, 0, sizeof offsets_); - offset_ = std::begin(offsets_); - } + void reset() noexcept { offset_ = offsets_; } - uint64_t* begin() noexcept { return std::begin(offsets_); } + uint64_t* begin() noexcept { return offsets_; } uint64_t* current() noexcept { return offset_; } - uint64_t* end() noexcept { return std::end(offsets_); } + uint64_t* end() noexcept { return offsets_ + kBlockSize; } private: - uint64_t offsets_[kBlockSize]{}; - uint64_t* offset_{offsets_}; + ManagedTypedAllocator alloc_; + uint64_t* offsets_{nullptr}; + uint64_t* offset_{nullptr}; }; void Prepare(doc_id_t key) final; diff --git a/core/formats/formats_burst_trie.cpp b/core/formats/formats_burst_trie.cpp index 41416c4c4..a8ee24bb5 100644 --- a/core/formats/formats_burst_trie.cpp +++ b/core/formats/formats_burst_trie.cpp @@ -22,23 +22,12 @@ #include "formats_burst_trie.hpp" -#include #include #include "utils/assert.hpp" // clang-format off -#if (defined(__clang__) || defined(_MSC_VER) || \ - defined(__GNUC__) && \ - (__GNUC__ > 8)) // GCCs <= don't have "" header -#include -#endif - -#ifdef __cpp_lib_memory_resource -#include -#endif - #include "utils/fstext/fst_utils.hpp" #if defined(_MSC_VER) @@ -175,11 +164,81 @@ using volatile_byte_ref = volatile_ref; using feature_map_t = std::vector; +template +struct Node { + T* next = nullptr; +}; + +template +struct IntrusiveList { + public: + IntrusiveList& operator=(const IntrusiveList&) = delete; + IntrusiveList(const IntrusiveList&) = delete; + + IntrusiveList() noexcept = default; + + IntrusiveList(IntrusiveList&& other) noexcept + : tail_{std::exchange(other.tail_, nullptr)} {} + + IntrusiveList& operator=(IntrusiveList&& other) noexcept { + std::swap(tail_, other.tail_); + return *this; + } + + void Append(IntrusiveList&& rhs) noexcept { + IRS_ASSERT(this != &rhs); + if (rhs.tail_ == nullptr) { + return; + } + if (tail_ == nullptr) { + tail_ = rhs.tail_; + rhs.tail_ = nullptr; + return; + } + // h1->t1->h1 h2->t2->h2 + auto* head = tail_->next; + // h1->t1->h2->t2->h2 + tail_->next = rhs.tail_->next; + // h1->t1->h2->t2->h1 + rhs.tail_->next = head; + // h1->**->h2->t2/t1->h1 + tail_ = rhs.tail_; + // h1->**->h2->t1->h1 + rhs.tail_ = nullptr; + } + + void PushFront(T& front) noexcept { + IRS_ASSERT(front.next == &front); + if (IRS_LIKELY(tail_ == nullptr)) { + tail_ = &front; + return; + } + front.next = tail_->next; + tail_->next = &front; + } + + template + IRS_FORCE_INLINE void Visit(Func&& func) const { + if (IRS_LIKELY(tail_ == nullptr)) { + return; + } + auto* head = tail_->next; + auto* it = head; + do { + func(*std::exchange(it, it->next)); + } while (it != head); + } + + T* tail_ = nullptr; +}; + // Block of terms struct block_t : private util::noncopyable { - struct prefixed_output final : data_output, private util::noncopyable { + struct prefixed_output final : data_output, + Node, + private util::noncopyable { explicit prefixed_output(volatile_byte_ref&& prefix) noexcept - : prefix(std::move(prefix)) {} + : Node{this}, prefix(std::move(prefix)) {} void write_byte(byte_type b) final { weight.PushBack(b); } @@ -193,21 +252,11 @@ struct block_t : private util::noncopyable { static constexpr uint16_t INVALID_LABEL{std::numeric_limits::max()}; -#ifdef __cpp_lib_memory_resource - using block_index_t = - std::list>; + using block_index_t = IntrusiveList; - block_t(IResourceManager& rm, std::pmr::memory_resource& mrc, - uint64_t block_start, byte_type meta, uint16_t label) noexcept - : index({rm, &mrc}), start(block_start), label(label), meta(meta) {} -#else - using block_index_t = - std::list>; - - block_t(IResourceManager& rm, uint64_t block_start, byte_type meta, + block_t(block_index_t&& other, uint64_t block_start, byte_type meta, uint16_t label) noexcept - : index({rm}), start(block_start), label(label), meta(meta) {} -#endif + : index{std::move(other)}, start{block_start}, label{label}, meta{meta} {} block_t(block_t&& rhs) noexcept : index(std::move(rhs.index)), @@ -225,16 +274,118 @@ struct block_t : private util::noncopyable { return *this; } + ~block_t() { + index.Visit([](prefixed_output& output) { // + output.~prefixed_output(); + }); + } + block_index_t index; // fst index data uint64_t start; // file pointer uint16_t label; // block lead label byte_type meta; // block metadata -}; // block_t +}; + +template +class MonotonicBuffer { + static constexpr size_t kAlign = + (alignof(T) * alignof(void*)) / std::gcd(alignof(T), alignof(void*)); + + struct alignas(kAlign) Block { + Block* prev = nullptr; + }; + + static_assert(std::is_trivially_destructible_v); + + public: + MonotonicBuffer(IResourceManager& resource_manager, + size_t initial_size) noexcept + : resource_manager_{resource_manager}, next_size_{initial_size} { + IRS_ASSERT(initial_size > 1); + } + + MonotonicBuffer(const MonotonicBuffer&) = delete; + MonotonicBuffer& operator=(const MonotonicBuffer&) = delete; + ~MonotonicBuffer() { Reset(); } + + template + T* Allocate(Args&&... args) { + if (IRS_UNLIKELY(available_ == 0)) { + AllocateMemory(); + } + IRS_ASSERT(reinterpret_cast(current_) % alignof(T) == 0); + auto* p = new (current_) T{std::forward(args)...}; + current_ += sizeof(T); + --available_; + return p; + } + + // Release all memory, except current_ and keep next_size_ + void Clear() noexcept { + if (IRS_UNLIKELY(head_ == nullptr)) { + return; + } + + Release(std::exchange(head_->prev, nullptr)); + // TODO(MBkkt) Don't be lazy, call Decrease eager + // resource_manager_.Decrease(blocks_memory_ - size of head block); + + auto* initial_current = reinterpret_cast(head_) + sizeof(Block); + available_ += (current_ - initial_current) / sizeof(T); + current_ = initial_current; + } + + // Release all memory, but keep next_size_ + void Reset() noexcept { + if (IRS_UNLIKELY(head_ == nullptr)) { + return; + } + + Release(std::exchange(head_, nullptr)); + resource_manager_.Decrease(std::exchange(blocks_memory_, 0)); + + // otherwise we always increasing size! + // TODO(MBkkt) we could compute current size, but it's + next_size_ = (next_size_ * 2 + 2) / 3; + + available_ = 0; + current_ = nullptr; + } + + private: + void Release(Block* it) noexcept { + while (it != nullptr) { + operator delete(std::exchange(it, it->prev), std::align_val_t{kAlign}); + } + } + + void AllocateMemory() { + const auto size = sizeof(Block) + next_size_ * sizeof(T); + // TODO(MBkkt) use allocate_at_least but it's only C++23 :( + resource_manager_.Increase(size); + blocks_memory_ += size; + auto* p = + static_cast(operator new(size, std::align_val_t{kAlign})); + head_ = new (p) Block{head_}; + IRS_ASSERT(p == reinterpret_cast(head_)); + current_ = p + sizeof(Block); + available_ = next_size_; + next_size_ = (next_size_ * 3) / 2; + IRS_ASSERT(available_ < next_size_); + } + + // TODO(MBkkt) Do we really want to measure this? + IResourceManager& resource_manager_; + size_t blocks_memory_ = 0; -// FIXME std::is_nothrow_move_constructible_v> == false -static_assert(std::is_nothrow_move_constructible_v); -// FIXME std::is_nothrow_move_assignable_v> == false -static_assert(std::is_nothrow_move_assignable_v); + size_t next_size_; + Block* head_ = nullptr; + + size_t available_ = 0; + byte_type* current_ = nullptr; +}; + +using OutputBuffer = MonotonicBuffer; enum EntryType : byte_type { ET_TERM = 0, ET_BLOCK, ET_INVALID }; @@ -244,10 +395,7 @@ class entry : private util::noncopyable { entry(irs::bytes_view term, irs::postings_writer::state&& attrs, bool volatile_term); - entry(irs::bytes_view prefix, IResourceManager& rm, -#ifdef __cpp_lib_memory_resource - std::pmr::memory_resource& mrc, -#endif + entry(irs::bytes_view prefix, block_t::block_index_t&& index, uint64_t block_start, byte_type meta, uint16_t label, bool volatile_term); entry(entry&& rhs) noexcept; @@ -287,10 +435,7 @@ entry::entry(irs::bytes_view term, irs::postings_writer::state&& attrs, mem_.construct(std::move(attrs)); } -entry::entry(irs::bytes_view prefix, IResourceManager& rm, -#ifdef __cpp_lib_memory_resource - std::pmr::memory_resource& mrc, -#endif +entry::entry(irs::bytes_view prefix, block_t::block_index_t&& index, uint64_t block_start, byte_type meta, uint16_t label, bool volatile_term) : type_(ET_BLOCK) { @@ -299,12 +444,7 @@ entry::entry(irs::bytes_view prefix, IResourceManager& rm, } else { data_.assign(prefix, volatile_term); } - -#ifdef __cpp_lib_memory_resource - mem_.construct(rm, mrc, block_start, meta, label); -#else - mem_.construct(rm, block_start, meta, label); -#endif + mem_.construct(std::move(index), block_start, meta, label); } entry::entry(entry&& rhs) noexcept : data_{std::move(rhs.data_)} { @@ -730,8 +870,9 @@ const fst::FstReadOptions& fst_read_options() { // mininum size of string weight we store in FST [[maybe_unused]] constexpr const size_t MIN_WEIGHT_SIZE = 2; -template -void merge_blocks(Blocks& blocks) { +using Blocks = std::vector>; + +void MergeBlocks(Blocks& blocks, OutputBuffer& buffer) { IRS_ASSERT(!blocks.empty()); auto it = blocks.begin(); @@ -740,7 +881,8 @@ void merge_blocks(Blocks& blocks) { auto& root_block = root.block(); auto& root_index = root_block.index; - root_index.emplace_front(std::move(root.data())); + auto& out = *buffer.Allocate(std::move(root.data())); + root_index.PushFront(out); // First byte in block header must not be equal to fst::kStringInfinity // Consider the following: @@ -753,7 +895,6 @@ void merge_blocks(Blocks& blocks) { IRS_ASSERT(char(root_block.meta) != fst::kStringInfinity); // will store just several bytes here - auto& out = *root_index.begin(); out.write_byte(static_cast(root_block.meta)); // block metadata out.write_vlong(root_block.start); // start pointer of the block @@ -770,11 +911,11 @@ void merge_blocks(Blocks& blocks) { out.write_vlong(start_delta); out.write_byte(static_cast(block->meta)); - root_index.splice(root_index.end(), it->block().index); + root_index.Append(std::move(it->block().index)); } } else { for (++it; it != blocks.end(); ++it) { - root_index.splice(root_index.end(), it->block().index); + root_index.Append(std::move(it->block().index)); } } @@ -811,13 +952,13 @@ class fst_buffer : public vector_byte_fst { using fst_byte_builder = fst_builder; - template - fst_stats reset(const Data& data) { + fst_stats reset(const block_t::block_index_t& index) { builder.reset(); - for (auto& fst_data : data) { - builder.add(fst_data.prefix, fst_data.weight); - } + index.Visit([&](block_t::prefixed_output& output) { + builder.add(output.prefix, output.weight); + // TODO(MBkkt) Call dtor here? + }); return builder.finish(); } @@ -878,10 +1019,8 @@ class field_writer final : public irs::field_writer { void Push(bytes_view term); absl::flat_hash_map feature_map_; -#ifdef __cpp_lib_memory_resource - std::pmr::monotonic_buffer_resource block_index_buf_; -#endif - std::vector> blocks_; + OutputBuffer output_buffer_; + Blocks blocks_; memory_output suffix_; // term suffix column memory_output stats_; // term stats column encryption::stream::ptr terms_out_cipher_; @@ -914,12 +1053,7 @@ void field_writer::WriteBlock(size_t prefix, size_t begin, size_t end, // write block entries const bool leaf = !block_meta::blocks(meta); -#ifdef __cpp_lib_memory_resource - block_t::block_index_t index( - {blocks_.get_allocator().ResourceManager(), &block_index_buf_}); -#else - block_t::block_index_t index({blocks_.get_allocator().ResourceManager()}); -#endif + block_t::block_index_t index; pw_->begin_block(); @@ -945,7 +1079,7 @@ void field_writer::WriteBlock(size_t prefix, size_t begin, size_t end, // current block start pointer should be greater IRS_ASSERT(block_start > e.block().start); suffix_.stream.write_vlong(block_start - e.block().start); - index.splice(index.end(), e.block().index); + index.Append(std::move(e.block().index)); } } @@ -992,15 +1126,8 @@ void field_writer::WriteBlock(size_t prefix, size_t begin, size_t end, // add new block to the list of created blocks blocks_.emplace_back(bytes_view{last_term_.view().data(), prefix}, - blocks_.get_allocator().ResourceManager(), -#ifdef __cpp_lib_memory_resource - block_index_buf_, -#endif - block_start, meta, label, consolidation_); - - if (!index.empty()) { - blocks_.back().block().index = std::move(index); - } + std::move(index), block_start, meta, label, + consolidation_); } void field_writer::WriteBlocks(size_t prefix, size_t count) { @@ -1059,7 +1186,7 @@ void field_writer::WriteBlocks(size_t prefix, size_t count) { } // merge blocks into 1st block - ::merge_blocks(blocks_); + ::MergeBlocks(blocks_, output_buffer_); // remove processed entries from the // top of the stack @@ -1102,10 +1229,7 @@ field_writer::field_writer( burst_trie::Version version /* = Format::MAX */, uint32_t min_block_size /* = DEFAULT_MIN_BLOCK_SIZE */, uint32_t max_block_size /* = DEFAULT_MAX_BLOCK_SIZE */) - : -#ifdef __cpp_lib_memory_resource - block_index_buf_{sizeof(block_t::prefixed_output) * 32}, -#endif + : output_buffer_(rm, 32), blocks_(ManagedTypedAllocator(rm)), suffix_(rm), stats_(rm), @@ -1284,6 +1408,9 @@ void field_writer::EndField(std::string_view name, IndexFeatures index_features, const entry& root = *stack_.begin(); IRS_ASSERT(fst_buf_); [[maybe_unused]] const auto fst_stats = fst_buf_->reset(root.block().index); + stack_.clear(); + output_buffer_.Clear(); + const vector_byte_fst& fst = *fst_buf_; #ifdef IRESEARCH_DEBUG @@ -1319,11 +1446,12 @@ void field_writer::EndField(std::string_view name, IndexFeatures index_features, absl::StrCat("Failed to write term index for field: ", name)}; } - stack_.clear(); ++fields_count_; } void field_writer::end() { + output_buffer_.Reset(); + IRS_ASSERT(terms_out_); IRS_ASSERT(index_out_); @@ -3320,9 +3448,7 @@ void field_reader::prepare(const ReaderState& state) { if (term_index_version != term_dict_version) { throw index_error(absl::StrCat("Term index version '", term_index_version, "' mismatches term dictionary version '", - term_dict_version, - "' in " - "segment '", + term_dict_version, "' in segment '", meta.name, "'")); } @@ -3453,6 +3579,7 @@ class term_reader_visitor { encryption::stream* terms_in_cipher_; }; +/* // "Dumper" visitor for term_reader_visitor. Suitable for debugging needs. class dumper : util::noncopyable { public: @@ -3470,7 +3597,7 @@ class dumper : util::noncopyable { prefix_ = block.prefix(); } - void sub_block(const block_iterator& /*block*/) { + void sub_block(const block_iterator&) { indent(); out_ << "|\n"; indent(); @@ -3498,6 +3625,8 @@ class dumper : util::noncopyable { size_t indent_ = 0; size_t prefix_ = 0; }; +*/ + } // namespace namespace irs { diff --git a/core/resource_manager.hpp b/core/resource_manager.hpp index f96a7b76d..840774b5a 100644 --- a/core/resource_manager.hpp +++ b/core/resource_manager.hpp @@ -23,15 +23,6 @@ #pragma once #include "shared.hpp" - -#if (defined(__clang__) || defined(_MSC_VER) || defined(__GNUC__)) -#include -#endif - -#ifdef __cpp_lib_memory_resource -#include -#endif - #include "utils/managed_allocator.hpp" namespace irs { @@ -91,13 +82,4 @@ struct ManagedTypedAllocator using Base::Base; }; -#ifdef __cpp_lib_polymorphic_allocator -template -struct ManagedTypedPmrAllocator - : ManagedAllocator, IResourceManager> { - using ManagedAllocator, - IResourceManager>::ManagedAllocator; -}; -#endif - } // namespace irs diff --git a/core/utils/bitset.hpp b/core/utils/bitset.hpp index 8728c2e24..d9d938080 100644 --- a/core/utils/bitset.hpp +++ b/core/utils/bitset.hpp @@ -198,7 +198,20 @@ class dynamic_bitset { bool none() const noexcept { return !any(); } - bool all() const noexcept { return count() == bits_; } + bool all() const noexcept { + if (words_ == 0) { + return true; + } + auto* begin = data_.get(); + for (auto* end = begin + words_ - 1; begin != end; ++begin) { + static_assert(std::is_unsigned_v); + if (*begin != std::numeric_limits::max()) { + return false; + } + } + return std::popcount(*begin) == + bits_ - (words_ - 1) * bits_required(); + } void clear() noexcept { clear_offset(0); } diff --git a/microbench/CMakeLists.txt b/microbench/CMakeLists.txt index c4075cbad..08224f0dd 100644 --- a/microbench/CMakeLists.txt +++ b/microbench/CMakeLists.txt @@ -61,7 +61,7 @@ include_directories(iresearch-microbench add_executable(iresearch-memory-file-bench memory_file_benchmark.cpp -) + ) set_ipo(iresearch-memory-file-bench) @@ -76,3 +76,21 @@ target_include_directories(iresearch-memory-file-bench target_link_libraries(iresearch-memory-file-bench iresearch-static ) + +add_executable(iresearch-column-writer + column_writer_benchmark.cpp + ) + +set_ipo(iresearch-column-writer) + +add_dependencies(iresearch-column-writer + iresearch-static + ) + +target_include_directories(iresearch-column-writer + PRIVATE ${PROJECT_BINARY_DIR}/core + ) + +target_link_libraries(iresearch-column-writer + iresearch-static + ) diff --git a/microbench/column_writer_benchmark.cpp b/microbench/column_writer_benchmark.cpp new file mode 100644 index 000000000..44d03787d --- /dev/null +++ b/microbench/column_writer_benchmark.cpp @@ -0,0 +1,57 @@ +//////////////////////////////////////////////////////////////////////////////// +/// DISCLAIMER +/// +/// Copyright 2021 ArangoDB GmbH, Cologne, Germany +/// +/// 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. +/// +/// Copyright holder is ArangoDB GmbH, Cologne, Germany +/// +/// @author Valerii Mironov +//////////////////////////////////////////////////////////////////////////////// + +#include +#include +#include + +#include "formats/columnstore2.hpp" + +static constexpr size_t kThreads = 1; +static constexpr size_t kColumns = 1000; +static constexpr size_t kColumnsIter = kColumns * 10; + +static void WriteFile(std::mt19937_64& rng) { + auto writer = irs::columnstore2::make_writer( + irs::columnstore2::Version::kMax, false, irs::IResourceManager::kNoop); + // const auto size = size_t{1} << kFileSizePower(rng); + for (size_t i = 0; i != kColumnsIter; ++i) { + if (i % kColumns == 0) { + writer->rollback(); + } + writer->push_column({}, {}); + } +} + +int main() { + std::vector threads; + threads.reserve(kThreads); + for (size_t i = 0; i != kThreads; ++i) { + threads.emplace_back([i] { + std::mt19937_64 rng(43 * i); + WriteFile(rng); + }); + } + for (auto& thread : threads) { + thread.join(); + } +} diff --git a/microbench/crc_benchmark.cpp b/microbench/crc_benchmark.cpp index 03a754442..c99c9ddd4 100644 --- a/microbench/crc_benchmark.cpp +++ b/microbench/crc_benchmark.cpp @@ -1,3 +1,25 @@ +//////////////////////////////////////////////////////////////////////////////// +/// DISCLAIMER +/// +/// Copyright 2021 ArangoDB GmbH, Cologne, Germany +/// +/// 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. +/// +/// Copyright holder is ArangoDB GmbH, Cologne, Germany +/// +/// @author Valerii Mironov +//////////////////////////////////////////////////////////////////////////////// + #include #include diff --git a/microbench/memory_file_benchmark.cpp b/microbench/memory_file_benchmark.cpp index caad2e31c..aebbabca1 100644 --- a/microbench/memory_file_benchmark.cpp +++ b/microbench/memory_file_benchmark.cpp @@ -1,3 +1,25 @@ +//////////////////////////////////////////////////////////////////////////////// +/// DISCLAIMER +/// +/// Copyright 2021 ArangoDB GmbH, Cologne, Germany +/// +/// 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. +/// +/// Copyright holder is ArangoDB GmbH, Cologne, Germany +/// +/// @author Valerii Mironov +//////////////////////////////////////////////////////////////////////////////// + #include #include #include