From 167cb2fb6c8f1be284a0149c4255040a49261070 Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Thu, 16 Oct 2025 15:01:38 +0300 Subject: [PATCH 01/46] Add basic structure for read_modify_write --- .../version/local_versioned_engine.cpp | 9 +++++++ .../version/local_versioned_engine.hpp | 5 ++++ cpp/arcticdb/version/python_bindings.cpp | 4 +++ cpp/arcticdb/version/version_store_api.cpp | 7 +++++ cpp/arcticdb/version/version_store_api.hpp | 5 ++++ cpp/arcticdb/version/versioned_engine.hpp | 6 +++++ python/arcticdb/version_store/_store.py | 26 +++++++++++++++++++ .../version_store/test_read_modify_write.py | 11 ++++++++ 8 files changed, 73 insertions(+) create mode 100644 python/tests/unit/arcticdb/version_store/test_read_modify_write.py diff --git a/cpp/arcticdb/version/local_versioned_engine.cpp b/cpp/arcticdb/version/local_versioned_engine.cpp index ceddd7dcf7e..5604a9913ea 100644 --- a/cpp/arcticdb/version/local_versioned_engine.cpp +++ b/cpp/arcticdb/version/local_versioned_engine.cpp @@ -418,6 +418,15 @@ ReadVersionOutput LocalVersionedEngine::read_dataframe_version_internal( return read_frame_for_version(store(), identifier, read_query, read_options, handler_data).get(); } +VersionedItem LocalVersionedEngine::read_modify_write_internal( + [[maybe_unused]] const StreamId& stream_id, [[maybe_unused]] const StreamId& target_stream, + [[maybe_unused]] const VersionQuery& version_query, + [[maybe_unused]] const std::shared_ptr& read_query, [[maybe_unused]] const ReadOptions& read_options +) { + py::gil_scoped_release release_gil; + return {}; +} + folly::Future LocalVersionedEngine::get_descriptor(AtomKey&& k) { const auto key = std::move(k); return store()->read(key).thenValue([](auto&& key_seg_pair) -> DescriptorItem { diff --git a/cpp/arcticdb/version/local_versioned_engine.hpp b/cpp/arcticdb/version/local_versioned_engine.hpp index 6ee80470150..863529f02ee 100644 --- a/cpp/arcticdb/version/local_versioned_engine.hpp +++ b/cpp/arcticdb/version/local_versioned_engine.hpp @@ -116,6 +116,11 @@ class LocalVersionedEngine : public VersionedEngine { const ReadOptions& read_options, std::any& handler_data ) override; + VersionedItem read_modify_write_internal( + const StreamId& stream_id, const StreamId& target_stream, const VersionQuery& version_query, + const std::shared_ptr& read_query, const ReadOptions& read_options + ) override; + DescriptorItem read_descriptor_internal(const StreamId& stream_id, const VersionQuery& version_query); StageResult write_parallel_frame( diff --git a/cpp/arcticdb/version/python_bindings.cpp b/cpp/arcticdb/version/python_bindings.cpp index 51addcc0761..1227815f2dc 100644 --- a/cpp/arcticdb/version/python_bindings.cpp +++ b/cpp/arcticdb/version/python_bindings.cpp @@ -825,6 +825,10 @@ void register_bindings(py::module& version, py::exception(), "Read the specified version of the dataframe from the store" ) + .def("read_modify_write", + &PythonVersionStore::read_modify_write, + py::call_guard(), + "Read, modify and write the specified version for the dataframe (experimental)") .def( "read_index", [&](PythonVersionStore& v, StreamId sid, const VersionQuery& version_query) { diff --git a/cpp/arcticdb/version/version_store_api.cpp b/cpp/arcticdb/version/version_store_api.cpp index 8c3882e029b..dca4c56707a 100644 --- a/cpp/arcticdb/version/version_store_api.cpp +++ b/cpp/arcticdb/version/version_store_api.cpp @@ -918,6 +918,13 @@ ReadResult PythonVersionStore::read_dataframe_version( ); } +VersionedItem PythonVersionStore::read_modify_write( + const StreamId& source_stream, const StreamId& target_stream, const VersionQuery& version_query, + const std::shared_ptr& read_query, const ReadOptions& read_options +) { + return read_modify_write_internal(source_stream, target_stream, version_query, read_query, read_options); +} + namespace { std::vector ARCTICDB_UNUSED iterate_snapshot_tombstones( diff --git a/cpp/arcticdb/version/version_store_api.hpp b/cpp/arcticdb/version/version_store_api.hpp index 7ceb5b74705..0cf1eec9e25 100644 --- a/cpp/arcticdb/version/version_store_api.hpp +++ b/cpp/arcticdb/version/version_store_api.hpp @@ -124,6 +124,11 @@ class PythonVersionStore : public LocalVersionedEngine { const ReadOptions& read_options, std::any& handler_data ); + VersionedItem read_modify_write( + const StreamId& stream_id, const StreamId& target_stream, const VersionQuery& version_query, + const std::shared_ptr& read_query, const ReadOptions& read_options + ); + std::variant sort_merge( const StreamId& stream_id, const py::object& user_meta, bool append, bool convert_int_to_float, bool via_iteration, bool sparsify, bool prune_previous_versions, bool delete_staged_data_on_failure, diff --git a/cpp/arcticdb/version/versioned_engine.hpp b/cpp/arcticdb/version/versioned_engine.hpp index ab0cc5f070f..5aa6fbd5742 100644 --- a/cpp/arcticdb/version/versioned_engine.hpp +++ b/cpp/arcticdb/version/versioned_engine.hpp @@ -89,6 +89,12 @@ class VersionedEngine { const ReadOptions& read_options, std::any& handler_data ) = 0; + // New API mirroring read path: read-modify-write operation + virtual VersionedItem read_modify_write_internal( + const StreamId& stream_id, const StreamId& target_id, const VersionQuery& version_query, + const std::shared_ptr& read_query, const ReadOptions& read_options + ) = 0; + virtual VersionedItem write_versioned_dataframe_internal( const StreamId& stream_id, const std::shared_ptr& frame, bool prune_previous_versions, bool allow_sparse, bool validate_index diff --git a/python/arcticdb/version_store/_store.py b/python/arcticdb/version_store/_store.py index 428ae8d5502..32773a6c3e9 100644 --- a/python/arcticdb/version_store/_store.py +++ b/python/arcticdb/version_store/_store.py @@ -2243,6 +2243,32 @@ def tail( def _read_dataframe(self, symbol, version_query, read_query, read_options): return ReadResult(*self.version_store.read_dataframe_version(symbol, version_query, read_query, read_options)) + def _read_modify_write( + self, + source_symbol: str, + query_builder: QueryBuilder, + target_symbol: Optional[str] = None, + as_of: Optional[VersionQueryInput] = None, + date_range: Optional[DateRangeInput] = None, + row_range: Optional[Tuple[int, int]] = None, + columns: Optional[List[str]] = None, + **kwargs, + ): + if target_symbol is None: + target_symbol = source_symbol + query_builder = copy.deepcopy(query_builder) + version_query, read_options, read_query = self._get_queries( + as_of=as_of, + date_range=date_range, + row_range=row_range, + columns=columns, + query_builder=query_builder, + **kwargs, + ) + return self.version_store.read_modify_write( + source_symbol, target_symbol, version_query, read_query, read_options + ) + def _post_process_dataframe( self, read_result, read_query, read_options, implement_read_index=False, head=None, tail=None ): diff --git a/python/tests/unit/arcticdb/version_store/test_read_modify_write.py b/python/tests/unit/arcticdb/version_store/test_read_modify_write.py new file mode 100644 index 00000000000..e4bff451d45 --- /dev/null +++ b/python/tests/unit/arcticdb/version_store/test_read_modify_write.py @@ -0,0 +1,11 @@ +import pandas as pd +from arcticdb.version_store.processing import QueryBuilder + + +def test_read_modify_write(lmdb_version_store_v1): + lib = lmdb_version_store_v1 + q = QueryBuilder() + q = q[q["col"] < 5] + lib.write("sym", pd.DataFrame({"col": [1, 10, 3]})) + lib._read_modify_write("sym", q) + lib.read("sym") From 580cf145762c6cbb5a47ce62e3b5fef9b03f90ef Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Tue, 21 Oct 2025 13:50:22 +0300 Subject: [PATCH 02/46] Working version of read modify write --- cpp/arcticdb/pipeline/frame_slice.cpp | 4 + cpp/arcticdb/pipeline/frame_slice.hpp | 10 +- cpp/arcticdb/pipeline/frame_utils.cpp | 17 ++- cpp/arcticdb/pipeline/frame_utils.hpp | 1 + cpp/arcticdb/pipeline/write_options.hpp | 8 +- cpp/arcticdb/util/error_code.hpp | 1 + .../version/local_versioned_engine.cpp | 45 ++++++- .../version/local_versioned_engine.hpp | 5 +- cpp/arcticdb/version/version_core.cpp | 124 +++++++++++++++++- cpp/arcticdb/version/version_core.hpp | 7 + cpp/arcticdb/version/version_store_api.cpp | 9 +- cpp/arcticdb/version/version_store_api.hpp | 5 +- cpp/arcticdb/version/versioned_engine.hpp | 5 +- python/arcticdb/version_store/_store.py | 5 +- .../version_store/test_read_modify_write.py | 22 +++- 15 files changed, 234 insertions(+), 34 deletions(-) diff --git a/cpp/arcticdb/pipeline/frame_slice.cpp b/cpp/arcticdb/pipeline/frame_slice.cpp index c24e8817096..f173d571d2f 100644 --- a/cpp/arcticdb/pipeline/frame_slice.cpp +++ b/cpp/arcticdb/pipeline/frame_slice.cpp @@ -27,6 +27,10 @@ SegmentInMemory& SliceAndKey::segment(const std::shared_ptr& store) { return *segment_; } +const SegmentInMemory& SliceAndKey::segment() const& { return *segment_; } + +SegmentInMemory&& SliceAndKey::segment() && { return std::move(*segment_); } + SegmentInMemory&& SliceAndKey::release_segment(const std::shared_ptr& store) const { ensure_segment(store); return std::move(*segment_); diff --git a/cpp/arcticdb/pipeline/frame_slice.hpp b/cpp/arcticdb/pipeline/frame_slice.hpp index 7f9ae3768ec..5dd4bdc5a6b 100644 --- a/cpp/arcticdb/pipeline/frame_slice.hpp +++ b/cpp/arcticdb/pipeline/frame_slice.hpp @@ -240,6 +240,10 @@ struct SliceAndKey { SegmentInMemory& segment(const std::shared_ptr& store); + const SegmentInMemory& segment() const&; + + SegmentInMemory&& segment() &&; + SegmentInMemory&& release_segment(const std::shared_ptr& store) const; const SegmentInMemory& segment(const std::shared_ptr& store) const; @@ -250,9 +254,11 @@ struct SliceAndKey { return c(*segment_, slice_, key_); } - const FrameSlice& slice() const { return slice_; } + const FrameSlice& slice() const& { return slice_; } + + FrameSlice& slice() & { return slice_; } - FrameSlice& slice() { return slice_; } + FrameSlice&& slice() && { return std::move(slice_); } bool invalid() const { return (!segment_ && !key_) || (segment_ && segment_->is_null()); } diff --git a/cpp/arcticdb/pipeline/frame_utils.cpp b/cpp/arcticdb/pipeline/frame_utils.cpp index 6c700a2bf1b..f1ccd0dbfd8 100644 --- a/cpp/arcticdb/pipeline/frame_utils.cpp +++ b/cpp/arcticdb/pipeline/frame_utils.cpp @@ -79,11 +79,9 @@ TimeseriesDescriptor index_descriptor_from_frame( ); } -void adjust_slice_ranges(const std::shared_ptr& pipeline_context) { - using namespace arcticdb::pipelines; - auto& slice_and_keys = pipeline_context->slice_and_keys_; +size_t adjust_slice_ranges(std::span slice_and_keys) { if (slice_and_keys.empty()) - return; + return 0; // Row and Col ranges input can be disjoint, "compress" them into the top left corner // e.g. // 1 3 6 9 @@ -119,16 +117,21 @@ void adjust_slice_ranges(const std::shared_ptr& pipe } else { increment_row_slice = true; } - slice.row_range = RowRange(row_offset, row_offset + slice.rows().diff()); - slice.col_range = ColRange(col_offset, col_offset + slice.columns().diff()); + slice.row_range = pipelines::RowRange(row_offset, row_offset + slice.rows().diff()); + slice.col_range = pipelines::ColRange(col_offset, col_offset + slice.columns().diff()); col_offset += slice.columns().diff(); if (increment_row_slice) { row_offset += slice.rows().diff(); col_offset = 0; } } + return row_offset; +} - pipeline_context->total_rows_ = row_offset; +void adjust_slice_ranges(const std::shared_ptr& pipeline_context) { + using namespace arcticdb::pipelines; + auto& slice_and_keys = pipeline_context->slice_and_keys_; + pipeline_context->total_rows_ = adjust_slice_ranges(slice_and_keys); } size_t adjust_slice_rowcounts(std::vector& slice_and_keys) { diff --git a/cpp/arcticdb/pipeline/frame_utils.hpp b/cpp/arcticdb/pipeline/frame_utils.hpp index dc958b25afa..2f4f110b96c 100644 --- a/cpp/arcticdb/pipeline/frame_utils.hpp +++ b/cpp/arcticdb/pipeline/frame_utils.hpp @@ -353,6 +353,7 @@ std::optional aggregator_set_data( size_t adjust_slice_rowcounts(std::vector& slice_and_keys); +size_t adjust_slice_ranges(std::span slice_and_keys); void adjust_slice_ranges(const std::shared_ptr& pipeline_context); size_t get_slice_rowcounts(std::vector& slice_and_keys); diff --git a/cpp/arcticdb/pipeline/write_options.hpp b/cpp/arcticdb/pipeline/write_options.hpp index 6ae5acf10fc..e21a29f95ae 100644 --- a/cpp/arcticdb/pipeline/write_options.hpp +++ b/cpp/arcticdb/pipeline/write_options.hpp @@ -13,29 +13,29 @@ namespace arcticdb { struct WriteOptions { static WriteOptions from_proto(const arcticdb::proto::storage::VersionStoreConfig::WriteOptions& opt) { - WriteOptions def; + constexpr static WriteOptions def; return {opt.dynamic_schema() && !opt.bucketize_dynamic() ? std::numeric_limits::max() : (opt.column_group_size() > 0 ? size_t(opt.column_group_size()) : def.column_group_size), opt.segment_row_size() > 0 ? size_t(opt.segment_row_size()) : def.segment_row_size, + opt.max_num_buckets() > 0 ? size_t(opt.max_num_buckets()) : def.max_num_buckets, opt.prune_previous_version(), opt.de_duplication(), opt.snapshot_dedup(), opt.dynamic_schema(), opt.ignore_sort_order(), - opt.bucketize_dynamic(), - opt.max_num_buckets() > 0 ? size_t(opt.max_num_buckets()) : def.max_num_buckets}; + opt.bucketize_dynamic()}; } size_t column_group_size = 127; size_t segment_row_size = 100'000; + size_t max_num_buckets = 150; bool prune_previous_version = false; bool de_duplication = false; bool snapshot_dedup = false; bool dynamic_schema = false; bool ignore_sort_order = false; bool bucketize_dynamic = false; - size_t max_num_buckets = 150; bool sparsify_floats = false; }; } // namespace arcticdb \ No newline at end of file diff --git a/cpp/arcticdb/util/error_code.hpp b/cpp/arcticdb/util/error_code.hpp index 3f832f240ec..7d4675728c3 100644 --- a/cpp/arcticdb/util/error_code.hpp +++ b/cpp/arcticdb/util/error_code.hpp @@ -59,6 +59,7 @@ inline std::unordered_map get_error_category_names() ERROR_CODE(1002, E_ASSERTION_FAILURE) \ ERROR_CODE(1003, E_RUNTIME_ERROR) \ ERROR_CODE(1004, E_STORED_CONFIG_ERROR) \ + ERROR_CODE(1005, E_NOT_SUPPORTED) \ ERROR_CODE(2000, E_INCOMPATIBLE_OBJECTS) \ ERROR_CODE(2001, E_UNIMPLEMENTED_INPUT_TYPE) \ ERROR_CODE(2002, E_UPDATE_NOT_SUPPORTED) \ diff --git a/cpp/arcticdb/version/local_versioned_engine.cpp b/cpp/arcticdb/version/local_versioned_engine.cpp index 5604a9913ea..f620d8e09b5 100644 --- a/cpp/arcticdb/version/local_versioned_engine.cpp +++ b/cpp/arcticdb/version/local_versioned_engine.cpp @@ -419,12 +419,49 @@ ReadVersionOutput LocalVersionedEngine::read_dataframe_version_internal( } VersionedItem LocalVersionedEngine::read_modify_write_internal( - [[maybe_unused]] const StreamId& stream_id, [[maybe_unused]] const StreamId& target_stream, - [[maybe_unused]] const VersionQuery& version_query, - [[maybe_unused]] const std::shared_ptr& read_query, [[maybe_unused]] const ReadOptions& read_options + const StreamId& source_stream, const StreamId& target_stream, [[maybe_unused]] const py::object& user_meta, + const VersionQuery& version_query, const std::shared_ptr& read_query, + const ReadOptions& read_options, [[maybe_unused]] bool prune_previous_versions ) { py::gil_scoped_release release_gil; - return {}; + + std::unique_ptr user_meta_proto{ + [](const py::object& user_meta) -> proto::descriptors::UserDefinedMetadata* { + if (user_meta.is_none()) { + return nullptr; + } + proto::descriptors::UserDefinedMetadata* user_meta_proto = + new proto::descriptors::UserDefinedMetadata(); + python_util::pb_from_python(user_meta, *user_meta_proto); + return user_meta_proto; + }(user_meta) + }; + + const auto source_version = get_version_to_read(source_stream, version_query); + const auto identifier = get_version_identifier(source_stream, version_query, read_options, source_version); + + const WriteOptions write_options = get_write_options(); + auto [maybe_prev, deleted] = ::arcticdb::get_latest_version(store(), version_map(), target_stream); + const auto target_version = get_next_version_from_key(maybe_prev); + if (target_version == 0) { + if (auto check_outcome = verify_symbol_key(target_stream); std::holds_alternative(check_outcome)) { + std::get(check_outcome).throw_error(); + } + } + VersionedItem versioned_item = read_modify_write_impl( + store(), + identifier, + std::move(user_meta_proto), + read_query, + read_options, + write_options, + IndexPartialKey{target_stream, target_version} + ); + if (cfg().symbol_list()) + symbol_list().add_symbol(store(), target_stream, versioned_item.key_.version_id()); + + write_version_and_prune_previous(prune_previous_versions, versioned_item.key_, deleted ? std::nullopt : maybe_prev); + return versioned_item; } folly::Future LocalVersionedEngine::get_descriptor(AtomKey&& k) { diff --git a/cpp/arcticdb/version/local_versioned_engine.hpp b/cpp/arcticdb/version/local_versioned_engine.hpp index 863529f02ee..45da344afac 100644 --- a/cpp/arcticdb/version/local_versioned_engine.hpp +++ b/cpp/arcticdb/version/local_versioned_engine.hpp @@ -117,8 +117,9 @@ class LocalVersionedEngine : public VersionedEngine { ) override; VersionedItem read_modify_write_internal( - const StreamId& stream_id, const StreamId& target_stream, const VersionQuery& version_query, - const std::shared_ptr& read_query, const ReadOptions& read_options + const StreamId& stream_id, const StreamId& target_stream, const py::object& user_meta, + const VersionQuery& version_query, const std::shared_ptr& read_query, + const ReadOptions& read_options, bool prune_previous_versions ) override; DescriptorItem read_descriptor_internal(const StreamId& stream_id, const VersionQuery& version_query); diff --git a/cpp/arcticdb/version/version_core.cpp b/cpp/arcticdb/version/version_core.cpp index a647c9ab1c4..a11b21b3f53 100644 --- a/cpp/arcticdb/version/version_core.cpp +++ b/cpp/arcticdb/version/version_core.cpp @@ -135,6 +135,72 @@ IndexDescriptorImpl check_index_match(const arcticdb::stream::Index& index, cons return desc; } +template +StreamSink::PartialKey generate_partial_key(const SliceAndKey& slice, const StreamId& id, const VersionId& version_id) { + if constexpr (index_type == IndexDescriptor::Type::ROWCOUNT) { + return StreamSink::PartialKey{ + .key_type = KeyType::TABLE_DATA, + .version_id = version_id, + .stream_id = id, + .start_index = safe_convert_to_numeric_index(slice.slice().row_range.first, "Rows"), + .end_index = safe_convert_to_numeric_index(slice.slice().row_range.second, "Rows"), + }; + } else if constexpr (index_type == IndexDescriptor::Type::TIMESTAMP) { + using IndexType = ScalarTagType>; + return StreamSink::PartialKey{ + .key_type = KeyType::TABLE_DATA, + .version_id = version_id, + .stream_id = id, + .start_index = *slice.segment().column_data(0).begin(), + .end_index = end_index_generator(*(slice.segment().column_data(0).last()->end() - 1)) + }; + } else { + internal::raise("Read modify write supports only row range and date time indexes"); + } +} + +folly::Future> write_slices( + std::vector&& slices, const StreamId& id, const VersionId& version_id, + const IndexDescriptor::Type index_type, const std::shared_ptr& de_dup_map, + std::shared_ptr store +) { + std::vector> write_input; + write_input.reserve(slices.size()); + + switch (index_type) { + case IndexDescriptor::Type::ROWCOUNT: { + ranges::transform(slices, std::back_inserter(write_input), [&](SliceAndKey& slice) { + return std::make_tuple( + generate_partial_key(slice, id, version_id), + std::move(slice).segment(), + std::move(slice).slice() + ); + }); + } break; + case IndexDescriptor::Type::TIMESTAMP: { + ranges::transform(slices, std::back_inserter(write_input), [&](SliceAndKey& slice) { + return std::make_tuple( + generate_partial_key(slice, id, version_id), + std::move(slice).segment(), + std::move(slice).slice() + ); + }); + } break; + default: { + internal::raise("Read modify write supports only row range and date time indexes"); + } + } + const size_t write_batch_size = write_window_size(); + return folly::collect(folly::window( + std::move(write_input), + [de_dup_map, + store](std::tuple&& input) { + return store->async_write(std::move(input), de_dup_map); + }, + write_batch_size + )) + .via(&async::io_executor()); +} } // namespace void sorted_data_check_append(const InputFrame& frame, index::IndexSegmentReader& index_segment_reader) { @@ -2703,6 +2769,60 @@ folly::Future read_frame_for_version( }); } +VersionedItem read_modify_write_impl( + const std::shared_ptr& store, const std::variant& source_version_info, + [[maybe_unused]] std::unique_ptr&& user_meta, + const std::shared_ptr& read_query, const ReadOptions& read_options, + [[maybe_unused]] const WriteOptions& write_options, + [[maybe_unused]] const IndexPartialKey& target_partial_index_key +) { + const auto pipeline_context = setup_pipeline_context(store, source_version_info, *read_query, read_options); + internal::check( + !pipeline_context->multi_key_, + "Performing read modify write is not supported for data using recursive or custom normalizers" + ); + return read_process_and_collect(store, pipeline_context, read_query, read_options) + .thenValue([&](std::vector&& slices) { + adjust_slice_ranges(slices); + return write_slices( + std::move(slices), + target_partial_index_key.id, + target_partial_index_key.version_id, + pipeline_context->descriptor().index().type(), + std::make_shared(), + store + ); + }) + .thenValue([&](std::vector&& slices) { + ranges::sort(slices, [](const SliceAndKey& a, const SliceAndKey& b) { + if (a.slice().col_range < b.slice().col_range) { + return true; + } else if (a.slice().col_range == b.slice().col_range) { + return a.slice().row_range < b.slice().row_range; + } + return false; + }); + const size_t row_count = slices.back().slice().row_range.second - slices[0].slice().row_range.first; + const TimeseriesDescriptor tsd = make_timeseries_descriptor( + row_count, + pipeline_context->descriptor(), + std::move(*pipeline_context->norm_meta_), + user_meta ? std::make_optional(*std::move(user_meta)) : std::nullopt, + std::nullopt, + std::nullopt, + write_options.bucketize_dynamic + ); + return index::write_index( + index_type_from_descriptor(pipeline_context->descriptor()), + tsd, + std::move(slices), + target_partial_index_key, + store + ); + }) + .get(); +} + folly::Future read_and_process( const std::shared_ptr& store, const std::variant& version_info, const std::shared_ptr& read_query, const ReadOptions& read_options, @@ -2730,8 +2850,8 @@ folly::Future read_and_process( .thenValueInline([res_versioned_item = std::move(res_versioned_item), pipeline_context, output_schema = std::move(output_schema)](auto&& entity_ids) mutable { - // Pipeline context user metadata is not populated in the case that only incomplete segments exist for a - // symbol, no indexed versions + // Pipeline context user metadata is not populated in the case that only incomplete segments exist + // for a symbol, no indexed versions return SymbolProcessingResult{ std::move(res_versioned_item), pipeline_context->user_meta_ ? std::move(*pipeline_context->user_meta_) diff --git a/cpp/arcticdb/version/version_core.hpp b/cpp/arcticdb/version/version_core.hpp index a050593e030..978ad4304e1 100644 --- a/cpp/arcticdb/version/version_core.hpp +++ b/cpp/arcticdb/version/version_core.hpp @@ -213,6 +213,13 @@ folly::Future prepare_output_frame( const std::shared_ptr& store, const ReadOptions& read_options, std::any& handler_data ); +VersionedItem read_modify_write_impl( + const std::shared_ptr& store, const std::variant& version_info, + std::unique_ptr&& user_meta, + const std::shared_ptr& read_query, const ReadOptions& read_options, + const WriteOptions& write_options, const IndexPartialKey& target_partial_index_key +); + } // namespace arcticdb::version_store namespace arcticdb { diff --git a/cpp/arcticdb/version/version_store_api.cpp b/cpp/arcticdb/version/version_store_api.cpp index dca4c56707a..b05db548ff7 100644 --- a/cpp/arcticdb/version/version_store_api.cpp +++ b/cpp/arcticdb/version/version_store_api.cpp @@ -919,10 +919,13 @@ ReadResult PythonVersionStore::read_dataframe_version( } VersionedItem PythonVersionStore::read_modify_write( - const StreamId& source_stream, const StreamId& target_stream, const VersionQuery& version_query, - const std::shared_ptr& read_query, const ReadOptions& read_options + const StreamId& source_stream, const StreamId& target_stream, const py::object& user_meta, + const VersionQuery& version_query, const std::shared_ptr& read_query, + const ReadOptions& read_options, bool prune_previous_versions ) { - return read_modify_write_internal(source_stream, target_stream, version_query, read_query, read_options); + return read_modify_write_internal( + source_stream, target_stream, user_meta, version_query, read_query, read_options, prune_previous_versions + ); } namespace { diff --git a/cpp/arcticdb/version/version_store_api.hpp b/cpp/arcticdb/version/version_store_api.hpp index 0cf1eec9e25..85f75a4c5f0 100644 --- a/cpp/arcticdb/version/version_store_api.hpp +++ b/cpp/arcticdb/version/version_store_api.hpp @@ -125,8 +125,9 @@ class PythonVersionStore : public LocalVersionedEngine { ); VersionedItem read_modify_write( - const StreamId& stream_id, const StreamId& target_stream, const VersionQuery& version_query, - const std::shared_ptr& read_query, const ReadOptions& read_options + const StreamId& stream_id, const StreamId& target_stream, const py::object& user_meta, + const VersionQuery& version_query, const std::shared_ptr& read_query, + const ReadOptions& read_options, bool prune_previous_versions ); std::variant sort_merge( diff --git a/cpp/arcticdb/version/versioned_engine.hpp b/cpp/arcticdb/version/versioned_engine.hpp index 5aa6fbd5742..8dede4eff3c 100644 --- a/cpp/arcticdb/version/versioned_engine.hpp +++ b/cpp/arcticdb/version/versioned_engine.hpp @@ -91,8 +91,9 @@ class VersionedEngine { // New API mirroring read path: read-modify-write operation virtual VersionedItem read_modify_write_internal( - const StreamId& stream_id, const StreamId& target_id, const VersionQuery& version_query, - const std::shared_ptr& read_query, const ReadOptions& read_options + const StreamId& stream_id, const StreamId& target_id, const py::object& user_meta, + const VersionQuery& version_query, const std::shared_ptr& read_query, + const ReadOptions& read_options, bool prune_previous_versions ) = 0; virtual VersionedItem write_versioned_dataframe_internal( diff --git a/python/arcticdb/version_store/_store.py b/python/arcticdb/version_store/_store.py index 32773a6c3e9..68305da36cb 100644 --- a/python/arcticdb/version_store/_store.py +++ b/python/arcticdb/version_store/_store.py @@ -2252,10 +2252,13 @@ def _read_modify_write( date_range: Optional[DateRangeInput] = None, row_range: Optional[Tuple[int, int]] = None, columns: Optional[List[str]] = None, + metadata: Optional[Any] = None, + prune_previous_versions: bool = False, **kwargs, ): if target_symbol is None: target_symbol = source_symbol + udm = normalize_metadata(metadata) query_builder = copy.deepcopy(query_builder) version_query, read_options, read_query = self._get_queries( as_of=as_of, @@ -2266,7 +2269,7 @@ def _read_modify_write( **kwargs, ) return self.version_store.read_modify_write( - source_symbol, target_symbol, version_query, read_query, read_options + source_symbol, target_symbol, udm, version_query, read_query, read_options, prune_previous_versions ) def _post_process_dataframe( diff --git a/python/tests/unit/arcticdb/version_store/test_read_modify_write.py b/python/tests/unit/arcticdb/version_store/test_read_modify_write.py index e4bff451d45..ce0177e663e 100644 --- a/python/tests/unit/arcticdb/version_store/test_read_modify_write.py +++ b/python/tests/unit/arcticdb/version_store/test_read_modify_write.py @@ -1,11 +1,23 @@ import pandas as pd +import pytest from arcticdb.version_store.processing import QueryBuilder +from arcticdb.util.test import assert_frame_equal -def test_read_modify_write(lmdb_version_store_v1): - lib = lmdb_version_store_v1 +def test_read_modify_write_filter(version_store_factory): + lmdb_version_store = version_store_factory(col_per_group=2, row_per_segment=2) + lib = lmdb_version_store q = QueryBuilder() q = q[q["col"] < 5] - lib.write("sym", pd.DataFrame({"col": [1, 10, 3]})) - lib._read_modify_write("sym", q) - lib.read("sym") + lib.write( + "sym", + pd.DataFrame( + { + "col": [1, 10, 10, 10, 3, 5, 5, 5, 5, 1, 2], + "col_2": [1, 10, 10, 10, 3, 5, 5, 5, 5, 1, 2], + "col_3": [1, 10, 10, 10, 3, 5, 5, 5, 5, 1, 2], + } + ), + ) + lib._read_modify_write("sym", q, "sym1") + assert_frame_equal(lib.read("sym1").data, lib.read("sym", query_builder=q).data) From abe52e3a46d55756dab2484e38846fae2a62f5e2 Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Tue, 21 Oct 2025 19:42:55 +0300 Subject: [PATCH 03/46] Implement write clause --- cpp/arcticdb/async/async_store.hpp | 8 +-- cpp/arcticdb/async/tasks.hpp | 2 +- cpp/arcticdb/pipeline/slicing.hpp | 6 +- cpp/arcticdb/pipeline/write_frame.cpp | 6 +- cpp/arcticdb/pipeline/write_frame.hpp | 8 +-- cpp/arcticdb/processing/clause.cpp | 86 ++++++++++++++++++++++++ cpp/arcticdb/processing/clause.hpp | 47 +++++++++++++ cpp/arcticdb/storage/file/file_store.hpp | 2 +- cpp/arcticdb/stream/incompletes.cpp | 17 ++--- cpp/arcticdb/stream/stream_sink.hpp | 40 +++++------ cpp/arcticdb/version/version_core.cpp | 13 ++-- cpp/arcticdb/version/version_core.hpp | 2 +- cpp/arcticdb/version/version_map.hpp | 4 +- 13 files changed, 184 insertions(+), 57 deletions(-) diff --git a/cpp/arcticdb/async/async_store.hpp b/cpp/arcticdb/async/async_store.hpp index c2f69edf158..79fc256fb33 100644 --- a/cpp/arcticdb/async/async_store.hpp +++ b/cpp/arcticdb/async/async_store.hpp @@ -116,7 +116,7 @@ class AsyncStore : public Store { .thenValue(WriteSegmentTask{library_}); } - folly::Future write(PartialKey pk, SegmentInMemory&& segment) override { + folly::Future write(stream::PartialKey pk, SegmentInMemory&& segment) override { return write(pk.key_type, pk.version_id, pk.stream_id, pk.start_index, pk.end_index, std::move(segment)); } @@ -129,7 +129,7 @@ class AsyncStore : public Store { } folly::Future write_maybe_blocking( - PartialKey pk, SegmentInMemory&& segment, std::shared_ptr semaphore + stream::PartialKey pk, SegmentInMemory&& segment, std::shared_ptr semaphore ) override { log::version().debug("Waiting for semaphore for write_maybe_blocking {}", pk); semaphore->wait(); @@ -168,7 +168,7 @@ class AsyncStore : public Store { return WriteSegmentTask{library_}(std::move(encoded)); } - entity::VariantKey write_sync(PartialKey pk, SegmentInMemory&& segment) override { + entity::VariantKey write_sync(stream::PartialKey pk, SegmentInMemory&& segment) override { return write_sync(pk.key_type, pk.version_id, pk.stream_id, pk.start_index, pk.end_index, std::move(segment)); } @@ -427,7 +427,7 @@ class AsyncStore : public Store { } folly::Future async_write( - folly::Future>&& input_fut, + folly::Future>&& input_fut, const std::shared_ptr& de_dup_map ) override { return std::move(input_fut) diff --git a/cpp/arcticdb/async/tasks.hpp b/cpp/arcticdb/async/tasks.hpp index 9f4bc6ae429..0f71602336a 100644 --- a/cpp/arcticdb/async/tasks.hpp +++ b/cpp/arcticdb/async/tasks.hpp @@ -34,7 +34,7 @@ using AtomKey = entity::AtomKey; using IndexValue = entity::IndexValue; struct EncodeAtomTask : BaseTask { - using PartialKey = stream::StreamSink::PartialKey; + using PartialKey = stream::PartialKey; PartialKey partial_key_; timestamp creation_ts_; SegmentInMemory segment_; diff --git a/cpp/arcticdb/pipeline/slicing.hpp b/cpp/arcticdb/pipeline/slicing.hpp index a4ba7aeab54..ff1d4a530c7 100644 --- a/cpp/arcticdb/pipeline/slicing.hpp +++ b/cpp/arcticdb/pipeline/slicing.hpp @@ -88,7 +88,7 @@ inline auto end_index_generator(T end_index) { // works for both rawtype and raw } inline auto get_partial_key_gen(std::shared_ptr frame, const TypedStreamVersion& key) { - using PartialKey = stream::StreamSink::PartialKey; + using PartialKey = stream::PartialKey; return [frame = std::move(frame), &key](const FrameSlice& s) { if (frame->has_index()) { @@ -110,10 +110,10 @@ inline auto get_partial_key_gen(std::shared_ptr frame, const TypedSt }; } -inline stream::StreamSink::PartialKey get_partial_key_for_segment_slice( +inline stream::PartialKey get_partial_key_for_segment_slice( const IndexDescriptorImpl& index, const TypedStreamVersion& key, const SegmentInMemory& slice ) { - using PartialKey = stream::StreamSink::PartialKey; + using PartialKey = stream::PartialKey; if (index.field_count() != 0) { util::check( diff --git a/cpp/arcticdb/pipeline/write_frame.cpp b/cpp/arcticdb/pipeline/write_frame.cpp index 9802ff19b90..19797248919 100644 --- a/cpp/arcticdb/pipeline/write_frame.cpp +++ b/cpp/arcticdb/pipeline/write_frame.cpp @@ -31,8 +31,8 @@ namespace ranges = std::ranges; WriteToSegmentTask::WriteToSegmentTask( std::shared_ptr frame, FrameSlice slice, const SlicingPolicy& slicing, - folly::Function&& partial_key_gen, - size_t slice_num_for_column, Index index, bool sparsify_floats + folly::Function&& partial_key_gen, size_t slice_num_for_column, Index index, + bool sparsify_floats ) : frame_(std::move(frame)), slice_(std::move(slice)), @@ -44,7 +44,7 @@ WriteToSegmentTask::WriteToSegmentTask( slice_.check_magic(); } -std::tuple WriteToSegmentTask::operator()() { +std::tuple WriteToSegmentTask::operator()() { ARCTICDB_SUBSAMPLE_AGG(WriteSliceCopyToSegment) slice_.check_magic(); magic_.check(); diff --git a/cpp/arcticdb/pipeline/write_frame.hpp b/cpp/arcticdb/pipeline/write_frame.hpp index 0eb5e82918d..a65128022c5 100644 --- a/cpp/arcticdb/pipeline/write_frame.hpp +++ b/cpp/arcticdb/pipeline/write_frame.hpp @@ -28,7 +28,7 @@ struct WriteToSegmentTask : public async::BaseTask { std::shared_ptr frame_; const FrameSlice slice_; const SlicingPolicy slicing_; - folly::Function partial_key_gen_; + folly::Function partial_key_gen_; size_t slice_num_for_column_; Index index_; bool sparsify_floats_; @@ -36,11 +36,11 @@ struct WriteToSegmentTask : public async::BaseTask { WriteToSegmentTask( std::shared_ptr frame, FrameSlice slice, const SlicingPolicy& slicing, - folly::Function&& partial_key_gen, - size_t slice_num_for_column, Index index, bool sparsify_floats + folly::Function&& partial_key_gen, size_t slice_num_for_column, Index index, + bool sparsify_floats ); - std::tuple operator()(); + std::tuple operator()(); private: SegmentInMemory slice_tensors() const; diff --git a/cpp/arcticdb/processing/clause.cpp b/cpp/arcticdb/processing/clause.cpp index 6cdf4730784..b00589a9229 100644 --- a/cpp/arcticdb/processing/clause.cpp +++ b/cpp/arcticdb/processing/clause.cpp @@ -23,6 +23,9 @@ #include #include #include +#include +#include +#include #include @@ -1645,4 +1648,87 @@ OutputSchema ConcatClause::join_schemas(std::vector&& input_schema std::string ConcatClause::to_string() const { return "CONCAT"; } +WriteClause::WriteClause( + const WriteOptions& write_options, const IndexPartialKey& index_partial_key, + std::shared_ptr dedup_map, std::shared_ptr store +) : + write_options_(write_options), + index_partial_key_(index_partial_key), + dedup_map_(std::move(dedup_map)), + store_(std::move(store)) {} + +std::vector> WriteClause::structure_for_processing(std::vector&) { + internal::raise("WriteClause should never be first in the pipeline"); +} + +std::vector> WriteClause::structure_for_processing( + std::vector>&& entity_ids_vec +) { + return entity_ids_vec; +} + +std::vector WriteClause::process(std::vector&& entity_ids) const { + if (entity_ids.empty()) { + return {}; + } + auto proc = gather_entities>(*component_manager_, std::move(entity_ids)); + const StreamDescriptor& stream_desc = ((*proc.segments_)[0])->descriptor(); + if (stream_desc.index().type() == IndexDescriptor::Type::EMPTY) { + return {}; + } + const stream::PartialKey& partial_key = create_partial_key(stream_desc, *((*proc.segments_)[0])); + std::vector> data_segments_to_write; + data_segments_to_write.reserve(proc.segments_->size()); + ranges::transform( + *proc.segments_, + std::back_inserter(data_segments_to_write), + [&partial_key, this](const std::shared_ptr& segment) { + return store_->async_write(std::make_tuple(partial_key, *segment, FrameSlice{}), dedup_map_); + } + ); + folly::collectAll(data_segments_to_write).get(); + return entity_ids; +} + +stream::PartialKey WriteClause::create_partial_key(const StreamDescriptor& stream_desc, const SegmentInMemory& segment) + const { + if (stream_desc.index().type() == IndexDescriptor::Type::ROWCOUNT) { + return stream::PartialKey{ + .key_type = KeyType::TABLE_DATA, + .version_id = index_partial_key_.version_id, + .stream_id = index_partial_key_.id, + .start_index = 0, + .end_index = 0 + }; + } else if (stream_desc.index().type() == IndexDescriptor::Type::TIMESTAMP) { + const timestamp start_ts = std::get(stream::TimeseriesIndex::start_value_for_segment(segment)); + const timestamp end_ts = + std::get(end_index_generator(stream::TimeseriesIndex::end_value_for_segment(segment))); + return stream::PartialKey{ + .key_type = KeyType::TABLE_DATA, + .version_id = index_partial_key_.version_id, + .stream_id = index_partial_key_.id, + .start_index = start_ts, + .end_index = end_ts + }; + } + internal::raise("Unknown index encountered in WriteClause"); +} + +const ClauseInfo& WriteClause::clause_info() const { return clause_info_; } + +void WriteClause::set_processing_config(const ProcessingConfig&) {} + +void WriteClause::set_component_manager(std::shared_ptr component_manager) { + component_manager_ = std::move(component_manager); +} + +OutputSchema WriteClause::modify_schema(OutputSchema&& output_schema) const { return output_schema; } + +OutputSchema WriteClause::join_schemas(std::vector&&) const { + util::raise_rte("WriteClause::join_schemas should never be called"); +} + +std::string WriteClause::to_string() const { return "Write"; } + } // namespace arcticdb diff --git a/cpp/arcticdb/processing/clause.hpp b/cpp/arcticdb/processing/clause.hpp index 26dcb5324ff..0c7c0e2f67b 100644 --- a/cpp/arcticdb/processing/clause.hpp +++ b/cpp/arcticdb/processing/clause.hpp @@ -8,6 +8,8 @@ #pragma once +#include + #include #include #include @@ -19,6 +21,7 @@ #include #include #include +#include #include #include #include @@ -31,6 +34,12 @@ using ResampleOrigin = std::variant; using RangesAndKey = pipelines::RangesAndKey; using SliceAndKey = pipelines::SliceAndKey; +namespace stream { +struct PartialKey; +} // namespace stream + +class DeDupMap; + struct IClause { template struct Interface : Base { @@ -791,4 +800,42 @@ struct ConcatClause { [[nodiscard]] std::string to_string() const; }; +struct WriteClause { + ClauseInfo clause_info_; + std::shared_ptr component_manager_; + WriteOptions write_options_; + IndexPartialKey index_partial_key_; + std::shared_ptr dedup_map_; + std::shared_ptr store_; + + WriteClause( + const WriteOptions& write_options, const IndexPartialKey& index_partial_key, + std::shared_ptr dedup_map, std::shared_ptr store + ); + ARCTICDB_MOVE_COPY_DEFAULT(WriteClause) + + [[nodiscard]] std::vector> structure_for_processing(std::vector&); + + [[nodiscard]] std::vector> structure_for_processing( + std::vector>&& entity_ids_vec + ); + + [[nodiscard]] std::vector process(std::vector&& entity_ids) const; + + [[nodiscard]] const ClauseInfo& clause_info() const; + + void set_processing_config(const ProcessingConfig&); + + void set_component_manager(std::shared_ptr component_manager); + + OutputSchema modify_schema(OutputSchema&& output_schema) const; + + OutputSchema join_schemas(std::vector&&) const; + + [[nodiscard]] std::string to_string() const; + + private: + stream::PartialKey create_partial_key(const StreamDescriptor& descriptor, const SegmentInMemory& segment) const; +}; + } // namespace arcticdb diff --git a/cpp/arcticdb/storage/file/file_store.hpp b/cpp/arcticdb/storage/file/file_store.hpp index 79a62f78242..1f031a9fbdf 100644 --- a/cpp/arcticdb/storage/file/file_store.hpp +++ b/cpp/arcticdb/storage/file/file_store.hpp @@ -27,7 +27,7 @@ namespace arcticdb { size_t max_data_size( - const std::vector>& items, + const std::vector>& items, const arcticdb::proto::encoding::VariantCodec& codec_opts, EncodingVersion encoding_version ) { auto max_file_size = 0UL; diff --git a/cpp/arcticdb/stream/incompletes.cpp b/cpp/arcticdb/stream/incompletes.cpp index ef5eda5c2ca..52d8cd44a41 100644 --- a/cpp/arcticdb/stream/incompletes.cpp +++ b/cpp/arcticdb/stream/incompletes.cpp @@ -333,9 +333,7 @@ void do_sort(SegmentInMemory& mutable_seg, const std::vector sort_c } const auto local_index_start = IdxType::start_value_for_segment(seg); const auto local_index_end = IdxType::end_value_for_segment(seg); - stream::StreamSink::PartialKey pk{ - KeyType::APPEND_DATA, 0, stream_id, local_index_start, local_index_end - }; + const PartialKey pk{KeyType::APPEND_DATA, 0, stream_id, local_index_start, local_index_end}; return store->write(pk, std::move(seg)).thenValueInline([](VariantKey&& res) { return to_atom(std::move(res)); }); @@ -441,10 +439,7 @@ void do_sort(SegmentInMemory& mutable_seg, const std::vector sort_c sparsify_floats )) .thenValue([store, de_dup_map, bucketize_dynamic, desc, norm_meta, user_meta]( - std::tuple< - stream::StreamSink::PartialKey, - SegmentInMemory, - pipelines::FrameSlice>&& ks + std::tuple&& ks ) { auto& seg = std::get(ks); auto norm_meta_copy = norm_meta; @@ -461,10 +456,10 @@ void do_sort(SegmentInMemory& mutable_seg, const std::vector sort_c ); seg.set_timeseries_descriptor(tsd); - // Just inherit sortedness from the overall frame for now. This is not - // mathematically correct when our slicing happens to break an unordered df - // up in to ordered chunks, but should be OK in practice since the user did - // stage unordered data. + // Just inherit sortedness from the overall frame for now. This is + // not mathematically correct when our slicing happens to break an + // unordered df up in to ordered chunks, but should be OK in practice + // since the user did stage unordered data. seg.descriptor().set_sorted(tsd.sorted()); return std::move(ks); diff --git a/cpp/arcticdb/stream/stream_sink.hpp b/cpp/arcticdb/stream/stream_sink.hpp index fa1f2fcf3f3..eb5e73eb39c 100644 --- a/cpp/arcticdb/stream/stream_sink.hpp +++ b/cpp/arcticdb/stream/stream_sink.hpp @@ -28,6 +28,24 @@ namespace arcticdb::stream { using KeyType = entity::KeyType; using IndexValue = entity::IndexValue; +struct PartialKey { + KeyType key_type; + VersionId version_id; + StreamId stream_id; + IndexValue start_index; + IndexValue end_index; + + [[nodiscard]] AtomKey build_key(timestamp creation_ts, ContentHash content_hash) const { + return entity::atom_key_builder() + .gen_id(version_id) + .start_index(start_index) + .end_index(end_index) + .content_hash(content_hash) + .creation_ts(creation_ts) + .build(stream_id, key_type); + } +}; + struct StreamSink { /** The remove_key{,s,sync} methods used to return the key to indicate success/not. However, most implementations @@ -61,24 +79,6 @@ struct StreamSink { const VariantKey& key, SegmentInMemory&& segment, storage::UpdateOpts = storage::UpdateOpts{} ) = 0; - struct PartialKey { - KeyType key_type; - VersionId version_id; - StreamId stream_id; - IndexValue start_index; - IndexValue end_index; - - [[nodiscard]] AtomKey build_key(timestamp creation_ts, ContentHash content_hash) const { - return entity::atom_key_builder() - .gen_id(version_id) - .start_index(start_index) - .end_index(end_index) - .content_hash(content_hash) - .creation_ts(creation_ts) - .build(stream_id, key_type); - } - }; - [[nodiscard]] virtual folly::Future write(PartialKey pk, SegmentInMemory&& segment) = 0; // shared_ptr for semaphore as executing futures need guarantees it is in a valid state, so need to participate @@ -146,14 +146,14 @@ namespace fmt { using namespace arcticdb::stream; template<> -struct formatter { +struct formatter { template constexpr auto parse(ParseContext& ctx) { return ctx.begin(); } template - auto format(const StreamSink::PartialKey& pk, FormatContext& ctx) const { + auto format(const PartialKey& pk, FormatContext& ctx) const { return fmt::format_to( ctx.out(), "'{}:{}:{}:{}:{}", pk.key_type, pk.stream_id, pk.version_id, pk.start_index, pk.end_index ); diff --git a/cpp/arcticdb/version/version_core.cpp b/cpp/arcticdb/version/version_core.cpp index a11b21b3f53..c113467d3e8 100644 --- a/cpp/arcticdb/version/version_core.cpp +++ b/cpp/arcticdb/version/version_core.cpp @@ -136,9 +136,9 @@ IndexDescriptorImpl check_index_match(const arcticdb::stream::Index& index, cons return desc; } template -StreamSink::PartialKey generate_partial_key(const SliceAndKey& slice, const StreamId& id, const VersionId& version_id) { +PartialKey generate_partial_key(const SliceAndKey& slice, const StreamId& id, const VersionId& version_id) { if constexpr (index_type == IndexDescriptor::Type::ROWCOUNT) { - return StreamSink::PartialKey{ + return PartialKey{ .key_type = KeyType::TABLE_DATA, .version_id = version_id, .stream_id = id, @@ -147,7 +147,7 @@ StreamSink::PartialKey generate_partial_key(const SliceAndKey& slice, const Stre }; } else if constexpr (index_type == IndexDescriptor::Type::TIMESTAMP) { using IndexType = ScalarTagType>; - return StreamSink::PartialKey{ + return PartialKey{ .key_type = KeyType::TABLE_DATA, .version_id = version_id, .stream_id = id, @@ -164,7 +164,7 @@ folly::Future> write_slices( const IndexDescriptor::Type index_type, const std::shared_ptr& de_dup_map, std::shared_ptr store ) { - std::vector> write_input; + std::vector> write_input; write_input.reserve(slices.size()); switch (index_type) { @@ -193,8 +193,7 @@ folly::Future> write_slices( const size_t write_batch_size = write_window_size(); return folly::collect(folly::window( std::move(write_input), - [de_dup_map, - store](std::tuple&& input) { + [de_dup_map, store](std::tuple&& input) { return store->async_write(std::move(input), de_dup_map); }, write_batch_size @@ -2348,7 +2347,7 @@ std::variant sort_merge_impl( [pipeline_context, &fut_vec, &store, &semaphore](SegmentInMemory&& segment) { const auto local_index_start = TimeseriesIndex::start_value_for_segment(segment); const auto local_index_end = TimeseriesIndex::end_value_for_segment(segment); - stream::StreamSink::PartialKey pk{ + const PartialKey pk{ KeyType::TABLE_DATA, pipeline_context->version_id_, pipeline_context->stream_id_, diff --git a/cpp/arcticdb/version/version_core.hpp b/cpp/arcticdb/version/version_core.hpp index 978ad4304e1..608aae7191a 100644 --- a/cpp/arcticdb/version/version_core.hpp +++ b/cpp/arcticdb/version/version_core.hpp @@ -264,7 +264,7 @@ template< [&write_futures, &store, &pipeline_context, &semaphore](SegmentInMemory&& segment) { auto local_index_start = IndexType::start_value_for_segment(segment); auto local_index_end = pipelines::end_index_generator(IndexType::end_value_for_segment(segment)); - stream::StreamSink::PartialKey pk{ + const PartialKey pk{ KeyType::TABLE_DATA, pipeline_context->version_id_, pipeline_context->stream_id_, diff --git a/cpp/arcticdb/version/version_map.hpp b/cpp/arcticdb/version/version_map.hpp index 7a28ed2358e..f9bd3fe0844 100644 --- a/cpp/arcticdb/version/version_map.hpp +++ b/cpp/arcticdb/version/version_map.hpp @@ -404,7 +404,7 @@ class VersionMapImpl { IndexAggregator journal_agg( stream_id, [&store, &journal_key, &version_id, &stream_id](auto&& segment) { - stream::StreamSink::PartialKey pk{ + const PartialKey pk{ KeyType::VERSION, version_id, stream_id, @@ -900,7 +900,7 @@ class VersionMapImpl { IndexAggregator version_agg( stream_id, [&store, &journal_key, &version_id, &stream_id](auto&& segment) { - stream::StreamSink::PartialKey pk{ + const PartialKey pk{ KeyType::VERSION, version_id, stream_id, From 59af3466c65dba56e70af0a345ad37892a89a8b2 Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Wed, 22 Oct 2025 19:12:19 +0300 Subject: [PATCH 04/46] Make the write clause store future components so that it does not deadlock --- cpp/arcticdb/pipeline/read_query.cpp | 2 +- cpp/arcticdb/pipeline/read_query.hpp | 2 +- cpp/arcticdb/processing/clause.cpp | 33 ++--- .../version/local_versioned_engine.cpp | 4 +- cpp/arcticdb/version/python_bindings.cpp | 2 +- cpp/arcticdb/version/version_core.cpp | 130 ++++++------------ 6 files changed, 63 insertions(+), 110 deletions(-) diff --git a/cpp/arcticdb/pipeline/read_query.cpp b/cpp/arcticdb/pipeline/read_query.cpp index a0078028271..23d73c41763 100644 --- a/cpp/arcticdb/pipeline/read_query.cpp +++ b/cpp/arcticdb/pipeline/read_query.cpp @@ -5,7 +5,7 @@ namespace arcticdb::pipelines { ReadQuery::ReadQuery(std::vector>&& clauses) : clauses_(std::move(clauses)) {} -void ReadQuery::add_clauses(std::vector>& clauses) { clauses_ = clauses; } +void ReadQuery::add_clauses(std::vector>&& clauses) { clauses_ = std::move(clauses); } void ReadQuery::convert_to_positive_row_filter(int64_t total_rows) { if (!row_range) { diff --git a/cpp/arcticdb/pipeline/read_query.hpp b/cpp/arcticdb/pipeline/read_query.hpp index f1cdae811f5..a42a9dc9f80 100644 --- a/cpp/arcticdb/pipeline/read_query.hpp +++ b/cpp/arcticdb/pipeline/read_query.hpp @@ -28,7 +28,7 @@ struct ReadQuery { explicit ReadQuery(std::vector>&& clauses); - void add_clauses(std::vector>& clauses); + void add_clauses(std::vector>&& clauses); void convert_to_positive_row_filter(int64_t total_rows); }; diff --git a/cpp/arcticdb/processing/clause.cpp b/cpp/arcticdb/processing/clause.cpp index b00589a9229..e8ee2976a2a 100644 --- a/cpp/arcticdb/processing/clause.cpp +++ b/cpp/arcticdb/processing/clause.cpp @@ -1671,23 +1671,24 @@ std::vector WriteClause::process(std::vector&& entity_ids) c if (entity_ids.empty()) { return {}; } - auto proc = gather_entities>(*component_manager_, std::move(entity_ids)); - const StreamDescriptor& stream_desc = ((*proc.segments_)[0])->descriptor(); - if (stream_desc.index().type() == IndexDescriptor::Type::EMPTY) { - return {}; - } - const stream::PartialKey& partial_key = create_partial_key(stream_desc, *((*proc.segments_)[0])); - std::vector> data_segments_to_write; + const auto proc = + gather_entities, std::shared_ptr, std::shared_ptr>( + *component_manager_, std::move(entity_ids) + ); + + std::vector>> data_segments_to_write; data_segments_to_write.reserve(proc.segments_->size()); - ranges::transform( - *proc.segments_, - std::back_inserter(data_segments_to_write), - [&partial_key, this](const std::shared_ptr& segment) { - return store_->async_write(std::make_tuple(partial_key, *segment, FrameSlice{}), dedup_map_); - } - ); - folly::collectAll(data_segments_to_write).get(); - return entity_ids; + + for (size_t i = 0; i < proc.segments_->size(); ++i) { + const SegmentInMemory& segment = *(*proc.segments_)[i]; + const RowRange& row_range = *(*proc.row_ranges_)[i]; + const ColRange& col_range = *(*proc.col_ranges_)[i]; + stream::PartialKey partial_key = create_partial_key(segment.descriptor(), segment); + data_segments_to_write.push_back(std::make_shared>(store_->async_write( + std::make_tuple(std::move(partial_key), segment, FrameSlice(col_range, row_range)), dedup_map_ + ))); + } + return component_manager_->add_entities(std::move(data_segments_to_write)); } stream::PartialKey WriteClause::create_partial_key(const StreamDescriptor& stream_desc, const SegmentInMemory& segment) diff --git a/cpp/arcticdb/version/local_versioned_engine.cpp b/cpp/arcticdb/version/local_versioned_engine.cpp index f620d8e09b5..81259d81292 100644 --- a/cpp/arcticdb/version/local_versioned_engine.cpp +++ b/cpp/arcticdb/version/local_versioned_engine.cpp @@ -419,9 +419,9 @@ ReadVersionOutput LocalVersionedEngine::read_dataframe_version_internal( } VersionedItem LocalVersionedEngine::read_modify_write_internal( - const StreamId& source_stream, const StreamId& target_stream, [[maybe_unused]] const py::object& user_meta, + const StreamId& source_stream, const StreamId& target_stream, const py::object& user_meta, const VersionQuery& version_query, const std::shared_ptr& read_query, - const ReadOptions& read_options, [[maybe_unused]] bool prune_previous_versions + const ReadOptions& read_options, bool prune_previous_versions ) { py::gil_scoped_release release_gil; diff --git a/cpp/arcticdb/version/python_bindings.cpp b/cpp/arcticdb/version/python_bindings.cpp index 1227815f2dc..abd0e9df01f 100644 --- a/cpp/arcticdb/version/python_bindings.cpp +++ b/cpp/arcticdb/version/python_bindings.cpp @@ -485,7 +485,7 @@ void register_bindings(py::module& version, py::exception(*clause)); }); } - self.add_clauses(_clauses); + self.add_clauses(std::move(_clauses)); }); py::enum_(version, "OperationType") diff --git a/cpp/arcticdb/version/version_core.cpp b/cpp/arcticdb/version/version_core.cpp index c113467d3e8..3c85fb82655 100644 --- a/cpp/arcticdb/version/version_core.cpp +++ b/cpp/arcticdb/version/version_core.cpp @@ -135,71 +135,6 @@ IndexDescriptorImpl check_index_match(const arcticdb::stream::Index& index, cons return desc; } -template -PartialKey generate_partial_key(const SliceAndKey& slice, const StreamId& id, const VersionId& version_id) { - if constexpr (index_type == IndexDescriptor::Type::ROWCOUNT) { - return PartialKey{ - .key_type = KeyType::TABLE_DATA, - .version_id = version_id, - .stream_id = id, - .start_index = safe_convert_to_numeric_index(slice.slice().row_range.first, "Rows"), - .end_index = safe_convert_to_numeric_index(slice.slice().row_range.second, "Rows"), - }; - } else if constexpr (index_type == IndexDescriptor::Type::TIMESTAMP) { - using IndexType = ScalarTagType>; - return PartialKey{ - .key_type = KeyType::TABLE_DATA, - .version_id = version_id, - .stream_id = id, - .start_index = *slice.segment().column_data(0).begin(), - .end_index = end_index_generator(*(slice.segment().column_data(0).last()->end() - 1)) - }; - } else { - internal::raise("Read modify write supports only row range and date time indexes"); - } -} - -folly::Future> write_slices( - std::vector&& slices, const StreamId& id, const VersionId& version_id, - const IndexDescriptor::Type index_type, const std::shared_ptr& de_dup_map, - std::shared_ptr store -) { - std::vector> write_input; - write_input.reserve(slices.size()); - - switch (index_type) { - case IndexDescriptor::Type::ROWCOUNT: { - ranges::transform(slices, std::back_inserter(write_input), [&](SliceAndKey& slice) { - return std::make_tuple( - generate_partial_key(slice, id, version_id), - std::move(slice).segment(), - std::move(slice).slice() - ); - }); - } break; - case IndexDescriptor::Type::TIMESTAMP: { - ranges::transform(slices, std::back_inserter(write_input), [&](SliceAndKey& slice) { - return std::make_tuple( - generate_partial_key(slice, id, version_id), - std::move(slice).segment(), - std::move(slice).slice() - ); - }); - } break; - default: { - internal::raise("Read modify write supports only row range and date time indexes"); - } - } - const size_t write_batch_size = write_window_size(); - return folly::collect(folly::window( - std::move(write_input), - [de_dup_map, store](std::tuple&& input) { - return store->async_write(std::move(input), de_dup_map); - }, - write_batch_size - )) - .via(&async::io_executor()); -} } // namespace void sorted_data_check_append(const InputFrame& frame, index::IndexSegmentReader& index_segment_reader) { @@ -1143,13 +1078,13 @@ static OutputSchema create_initial_output_schema(PipelineContext& pipeline_conte return OutputSchema{generate_initial_output_schema_descriptor(pipeline_context), *pipeline_context.norm_meta_}; } -static OutputSchema generate_output_schema(PipelineContext& pipeline_context, std::shared_ptr read_query) { +static OutputSchema generate_output_schema(PipelineContext& pipeline_context, const ReadQuery& read_query) { OutputSchema output_schema = create_initial_output_schema(pipeline_context); - for (const auto& clause : read_query->clauses_) { + for (const auto& clause : read_query.clauses_) { output_schema = clause->modify_schema(std::move(output_schema)); } - if (read_query->columns) { - std::unordered_set selected_columns(read_query->columns->begin(), read_query->columns->end()); + if (read_query.columns) { + std::unordered_set selected_columns(read_query.columns->begin(), read_query.columns->end()); FieldCollection fields_to_use; if (!pipeline_context.filter_columns_) { pipeline_context.filter_columns_ = std::make_shared(); @@ -1171,6 +1106,18 @@ static OutputSchema generate_output_schema(PipelineContext& pipeline_context, st return output_schema; } +static void generate_output_schema_and_save_to_pipeline( + PipelineContext& pipeline_context, const ReadQuery& read_query +) { + OutputSchema schema = generate_output_schema(pipeline_context, read_query); + auto&& [descriptor, norm_meta, default_values] = schema.release(); + pipeline_context.set_descriptor(std::forward(descriptor)); + pipeline_context.norm_meta_ = std::make_shared( + std::forward(norm_meta) + ); + pipeline_context.default_values_ = std::forward(default_values); +} + folly::Future> read_and_schedule_processing( const std::shared_ptr& store, const std::shared_ptr& pipeline_context, const std::shared_ptr& read_query, const ReadOptions& read_options, @@ -1229,13 +1176,7 @@ folly::Future> read_process_and_collect( auto component_manager = std::make_shared(); return read_and_schedule_processing(store, pipeline_context, read_query, read_options, component_manager) .thenValue([component_manager, pipeline_context, read_query](std::vector&& processed_entity_ids) { - OutputSchema schema = generate_output_schema(*pipeline_context, std::move(read_query)); - auto&& [descriptor, norm_meta, default_values] = schema.release(); - pipeline_context->set_descriptor(std::forward(descriptor)); - pipeline_context->norm_meta_ = std::make_shared( - std::forward(norm_meta) - ); - pipeline_context->default_values_ = std::forward(default_values); + generate_output_schema_and_save_to_pipeline(*pipeline_context, *read_query); auto proc = gather_entities< std::shared_ptr, std::shared_ptr, @@ -2772,27 +2713,38 @@ VersionedItem read_modify_write_impl( const std::shared_ptr& store, const std::variant& source_version_info, [[maybe_unused]] std::unique_ptr&& user_meta, const std::shared_ptr& read_query, const ReadOptions& read_options, - [[maybe_unused]] const WriteOptions& write_options, - [[maybe_unused]] const IndexPartialKey& target_partial_index_key + const WriteOptions& write_options, const IndexPartialKey& target_partial_index_key ) { + + read_query->clauses_.push_back(std::make_shared( + WriteClause(write_options, target_partial_index_key, std::make_shared(), store) + )); const auto pipeline_context = setup_pipeline_context(store, source_version_info, *read_query, read_options); internal::check( !pipeline_context->multi_key_, "Performing read modify write is not supported for data using recursive or custom normalizers" ); - return read_process_and_collect(store, pipeline_context, read_query, read_options) - .thenValue([&](std::vector&& slices) { - adjust_slice_ranges(slices); - return write_slices( - std::move(slices), - target_partial_index_key.id, - target_partial_index_key.version_id, - pipeline_context->descriptor().index().type(), - std::make_shared(), - store + + auto component_manager = std::make_shared(); + return read_and_schedule_processing(store, pipeline_context, read_query, read_options, component_manager) + .thenValue([component_manager, pipeline_context, read_query](std::vector&& processed_entity_ids) { + generate_output_schema_and_save_to_pipeline(*pipeline_context, *read_query); + std::vector> write_segments_futures; + ranges::transform( + std::get<0>(component_manager->get_entities>>( + processed_entity_ids + )), + std::back_inserter(write_segments_futures), + [](std::shared_ptr>& fut) { + folly::Future res = std::move(*fut); + fut.reset(); + return res; + } ); + return folly::collect(std::move(write_segments_futures)); }) .thenValue([&](std::vector&& slices) { + adjust_slice_ranges(slices); ranges::sort(slices, [](const SliceAndKey& a, const SliceAndKey& b) { if (a.slice().col_range < b.slice().col_range) { return true; @@ -2842,7 +2794,7 @@ folly::Future read_and_process( !pipeline_context->is_pickled(), "Cannot perform multi-symbol join on pickled data" ); - OutputSchema output_schema = generate_output_schema(*pipeline_context, read_query); + OutputSchema output_schema = generate_output_schema(*pipeline_context, *read_query); ARCTICDB_DEBUG(log::version(), "Fetching data to frame"); return read_and_schedule_processing(store, pipeline_context, read_query, read_options, std::move(component_manager)) From 97549b6f67cf03b7370a19e4db4ec093a4da3d58 Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Thu, 23 Oct 2025 16:20:53 +0300 Subject: [PATCH 05/46] Fix a bug with empty dataframes. Test all filtering --- cpp/arcticdb/version/version_core.cpp | 3 +- python/arcticdb/util/test.py | 32 ++++++++++++++----- .../version_store/test_read_modify_write.py | 6 ++-- 3 files changed, 29 insertions(+), 12 deletions(-) diff --git a/cpp/arcticdb/version/version_core.cpp b/cpp/arcticdb/version/version_core.cpp index 3c85fb82655..77c7892b923 100644 --- a/cpp/arcticdb/version/version_core.cpp +++ b/cpp/arcticdb/version/version_core.cpp @@ -2753,7 +2753,8 @@ VersionedItem read_modify_write_impl( } return false; }); - const size_t row_count = slices.back().slice().row_range.second - slices[0].slice().row_range.first; + const size_t row_count = + slices.empty() ? 0 : slices.back().slice().row_range.second - slices[0].slice().row_range.first; const TimeseriesDescriptor tsd = make_timeseries_descriptor( row_count, pipeline_context->descriptor(), diff --git a/python/arcticdb/util/test.py b/python/arcticdb/util/test.py index 00d882b7861..653202f7430 100644 --- a/python/arcticdb/util/test.py +++ b/python/arcticdb/util/test.py @@ -746,15 +746,31 @@ def random_seed_context(): FIXED_STRINGS_SUFFIX = "fixed_strings" +def read_modify_write_data(lib, symbol, query): + dest_symbol = f"{symbol}_modified" + lib._read_modify_write(symbol, query, dest_symbol) + return lib.read(dest_symbol).data + + +def get_query_processing_functions(lib, symbol, arctic_query): + test_read_modify_write = os.getenv("ARCTICDB_TEST_READ_MODIFY_WRITE", "0") == "1" + processing_functions = [lambda: lib.read(symbol, query_builder=arctic_query).data] + if test_read_modify_write: + processing_functions.append(lambda: read_modify_write_data(lib, symbol, arctic_query)) + return processing_functions + + def generic_filter_test(lib, symbol, arctic_query, expected): - received = lib.read(symbol, query_builder=arctic_query).data - if not np.array_equal(expected, received): - original_df = lib.read(symbol).data - print( - f"""Original df:\n{original_df}\nwith dtypes:\n{original_df.dtypes}\nquery:\n{arctic_query}""" - f"""\nPandas result:\n{expected}\nArcticDB result:\n{received}""" - ) - assert False + query_processing_functions = get_query_processing_functions(lib, symbol, arctic_query) + for processing in query_processing_functions: + received = processing() + if not np.array_equal(expected, received): + original_df = lib.read(symbol).data + print( + f"""Original df:\n{original_df}\nwith dtypes:\n{original_df.dtypes}\nquery:\n{arctic_query}""" + f"""\nPandas result:\n{expected}\nArcticDB result:\n{received}""" + ) + assert False # For string queries, test both with and without dynamic strings, and with the query both optimised for speed and memory diff --git a/python/tests/unit/arcticdb/version_store/test_read_modify_write.py b/python/tests/unit/arcticdb/version_store/test_read_modify_write.py index ce0177e663e..143e2570b36 100644 --- a/python/tests/unit/arcticdb/version_store/test_read_modify_write.py +++ b/python/tests/unit/arcticdb/version_store/test_read_modify_write.py @@ -13,9 +13,9 @@ def test_read_modify_write_filter(version_store_factory): "sym", pd.DataFrame( { - "col": [1, 10, 10, 10, 3, 5, 5, 5, 5, 1, 2], - "col_2": [1, 10, 10, 10, 3, 5, 5, 5, 5, 1, 2], - "col_3": [1, 10, 10, 10, 3, 5, 5, 5, 5, 1, 2], + "col": [10, 10, 10, 10, 3, 5, 5, 5, 5, 1, 2], + "col_2": [10, 10, 10, 10, 3, 5, 5, 5, 5, 1, 2], + "col_3": [10, 10, 10, 10, 3, 5, 5, 5, 5, 1, 2], } ), ) From dfda400ef4edb05f0ee2c5bc5261640cd343df2e Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Thu, 23 Oct 2025 18:18:59 +0300 Subject: [PATCH 06/46] Fix col range for multiindex --- cpp/arcticdb/version/version_core.cpp | 23 ++++++++--- python/arcticdb/util/test.py | 38 ++++++++++--------- .../version_store/test_read_modify_write.py | 30 +++++++-------- 3 files changed, 52 insertions(+), 39 deletions(-) diff --git a/cpp/arcticdb/version/version_core.cpp b/cpp/arcticdb/version/version_core.cpp index 77c7892b923..27ca02471d7 100644 --- a/cpp/arcticdb/version/version_core.cpp +++ b/cpp/arcticdb/version/version_core.cpp @@ -2711,7 +2711,7 @@ folly::Future read_frame_for_version( VersionedItem read_modify_write_impl( const std::shared_ptr& store, const std::variant& source_version_info, - [[maybe_unused]] std::unique_ptr&& user_meta, + std::unique_ptr&& user_meta, const std::shared_ptr& read_query, const ReadOptions& read_options, const WriteOptions& write_options, const IndexPartialKey& target_partial_index_key ) { @@ -2744,15 +2744,28 @@ VersionedItem read_modify_write_impl( return folly::collect(std::move(write_segments_futures)); }) .thenValue([&](std::vector&& slices) { - adjust_slice_ranges(slices); ranges::sort(slices, [](const SliceAndKey& a, const SliceAndKey& b) { - if (a.slice().col_range < b.slice().col_range) { + if (a.slice().col_range.first < b.slice().col_range.first) { return true; - } else if (a.slice().col_range == b.slice().col_range) { - return a.slice().row_range < b.slice().row_range; + } else if (a.slice().col_range.first == b.slice().col_range.first) { + return a.slice().row_range.first < b.slice().row_range.first; } return false; }); + + size_t current_compacted_row = 0; + size_t previous_uncompacted_end = slices.empty() ? 0 : slices.front().slice().row_range.end(); + for (SliceAndKey& slice : slices) { + if (slice.slice().row_range.start() < previous_uncompacted_end) { + current_compacted_row = 0; + } + previous_uncompacted_end = slice.slice().row_range.end(); + const size_t rows_in_slice = slice.slice().row_range.diff(); + slice.slice().row_range.first = current_compacted_row; + slice.slice().row_range.second = current_compacted_row + rows_in_slice; + current_compacted_row += rows_in_slice; + } + const size_t row_count = slices.empty() ? 0 : slices.back().slice().row_range.second - slices[0].slice().row_range.first; const TimeseriesDescriptor tsd = make_timeseries_descriptor( diff --git a/python/arcticdb/util/test.py b/python/arcticdb/util/test.py index 653202f7430..503ebab4db9 100644 --- a/python/arcticdb/util/test.py +++ b/python/arcticdb/util/test.py @@ -783,24 +783,26 @@ def generic_filter_test_strings(lib, base_symbol, arctic_query, expected): def generic_filter_test_dynamic(lib, symbol, arctic_query, queried_slices): - received = lib.read(symbol, query_builder=arctic_query).data - assert len(received) == sum([len(queried_slice) for queried_slice in queried_slices]) - start_row = 0 - arrays_equal = True - for queried_slice in queried_slices: - for col_name in queried_slice.columns: - if not np.array_equal( - queried_slice[col_name], received[col_name].iloc[start_row : start_row + len(queried_slice)] - ): - arrays_equal = False - start_row += len(queried_slice) - if not arrays_equal: - original_df = lib.read(symbol).data - print( - f"""Original df (in ArcticDB, backfilled):\n{original_df}\nwith dtypes:\n{original_df.dtypes}\nquery:\n{arctic_query}""" - f"""\nPandas result:\n{queried_slices}\nArcticDB result:\n{received}""" - ) - assert False + query_processing_functions = get_query_processing_functions(lib, symbol, arctic_query) + for proccessing_function in query_processing_functions: + received = proccessing_function() + assert len(received) == sum([len(queried_slice) for queried_slice in queried_slices]) + start_row = 0 + arrays_equal = True + for queried_slice in queried_slices: + for col_name in queried_slice.columns: + if not np.array_equal( + queried_slice[col_name], received[col_name].iloc[start_row : start_row + len(queried_slice)] + ): + arrays_equal = False + start_row += len(queried_slice) + if not arrays_equal: + original_df = lib.read(symbol).data + print( + f"""Original df (in ArcticDB, backfilled):\n{original_df}\nwith dtypes:\n{original_df.dtypes}\nquery:\n{arctic_query}""" + f"""\nPandas result:\n{queried_slices}\nArcticDB result:\n{received}""" + ) + assert False # For string queries, test both with and without dynamic strings, and with the query both optimised for speed and memory diff --git a/python/tests/unit/arcticdb/version_store/test_read_modify_write.py b/python/tests/unit/arcticdb/version_store/test_read_modify_write.py index 143e2570b36..5ae8cf3d089 100644 --- a/python/tests/unit/arcticdb/version_store/test_read_modify_write.py +++ b/python/tests/unit/arcticdb/version_store/test_read_modify_write.py @@ -4,20 +4,18 @@ from arcticdb.util.test import assert_frame_equal -def test_read_modify_write_filter(version_store_factory): - lmdb_version_store = version_store_factory(col_per_group=2, row_per_segment=2) - lib = lmdb_version_store +def test_read_modify_write_filter(lmdb_version_store_dynamic_schema_v1): + lib = lmdb_version_store_dynamic_schema_v1 + sym = "sym" + df1 = pd.DataFrame({"a": [1.0, 2.0, 3.0], "b": [4.0, 5.0, 6.0]}) + lib.append(sym, df1) + + df2 = pd.DataFrame({"b": [4.0, 5.0, 6.0]}) + lib.append(sym, df2) + + df3 = pd.DataFrame({"a": [4.0, 5.0, 6.0]}) + lib.append(sym, df3) + q = QueryBuilder() - q = q[q["col"] < 5] - lib.write( - "sym", - pd.DataFrame( - { - "col": [10, 10, 10, 10, 3, 5, 5, 5, 5, 1, 2], - "col_2": [10, 10, 10, 10, 3, 5, 5, 5, 5, 1, 2], - "col_3": [10, 10, 10, 10, 3, 5, 5, 5, 5, 1, 2], - } - ), - ) - lib._read_modify_write("sym", q, "sym1") - assert_frame_equal(lib.read("sym1").data, lib.read("sym", query_builder=q).data) + q = q[q["a"] > 2.0] + lib._read_modify_write(sym, q, "sym2") From 0473e65b325bac736d98b5efed35bdc047a43515 Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Fri, 24 Oct 2025 10:40:01 +0300 Subject: [PATCH 07/46] wip --- cpp/arcticdb/processing/clause.cpp | 9 +- .../processing/unsorted_aggregation.cpp | 2 +- cpp/arcticdb/version/version_core.cpp | 12 ++ python/arcticdb/util/test.py | 156 ++++++++++-------- 4 files changed, 104 insertions(+), 75 deletions(-) diff --git a/cpp/arcticdb/processing/clause.cpp b/cpp/arcticdb/processing/clause.cpp index e8ee2976a2a..f4f99f37384 100644 --- a/cpp/arcticdb/processing/clause.cpp +++ b/cpp/arcticdb/processing/clause.cpp @@ -547,12 +547,9 @@ std::vector AggregationClause::process(std::vector&& entity_ ); auto column_data = index_col->data(); - std::transform( - elements.cbegin(), - elements.cend(), - column_data.begin(), - [](const auto& element) { return element.first; } - ); + ranges::transform(elements, column_data.begin(), [](const auto& element) { + return element.first; + }); }); index_col->set_row_data(grouping_map.size() - 1); diff --git a/cpp/arcticdb/processing/unsorted_aggregation.cpp b/cpp/arcticdb/processing/unsorted_aggregation.cpp index 865b74878c6..aa7a9e085ca 100644 --- a/cpp/arcticdb/processing/unsorted_aggregation.cpp +++ b/cpp/arcticdb/processing/unsorted_aggregation.cpp @@ -544,7 +544,7 @@ SegmentInMemory CountAggregatorData::finalize(const ColumnName& output_column_na create_output_column(make_scalar_type(get_output_data_type()), std::move(sparse_map_), unique_values); if (!col->opt_sparse_map().has_value()) { // If all values are set we use memcpy for efficiency - auto ptr = reinterpret_cast(col->ptr()); + const auto ptr = reinterpret_cast(col->ptr()); memcpy(ptr, aggregated_.data(), sizeof(uint64_t) * unique_values); } else { using OutputTypeDescriptor = typename ScalarTypeInfo>::TDT; diff --git a/cpp/arcticdb/version/version_core.cpp b/cpp/arcticdb/version/version_core.cpp index 27ca02471d7..732aa619955 100644 --- a/cpp/arcticdb/version/version_core.cpp +++ b/cpp/arcticdb/version/version_core.cpp @@ -2744,6 +2744,12 @@ VersionedItem read_modify_write_impl( return folly::collect(std::move(write_segments_futures)); }) .thenValue([&](std::vector&& slices) { + for (auto& slice : slices) { + std::cout << fmt::format( + "Row slice: {}, Col slice: {}\n", slice.slice().row_range, slice.slice().col_range + ); + } + std::cout << "==========================\n"; ranges::sort(slices, [](const SliceAndKey& a, const SliceAndKey& b) { if (a.slice().col_range.first < b.slice().col_range.first) { return true; @@ -2766,6 +2772,12 @@ VersionedItem read_modify_write_impl( current_compacted_row += rows_in_slice; } + for (auto& slice : slices) { + std::cout << fmt::format( + "Row slice: {}, Col slice: {}\n", slice.slice().row_range, slice.slice().col_range + ); + } + const size_t row_count = slices.empty() ? 0 : slices.back().slice().row_range.second - slices[0].slice().row_range.first; const TimeseriesDescriptor tsd = make_timeseries_descriptor( diff --git a/python/arcticdb/util/test.py b/python/arcticdb/util/test.py index 503ebab4db9..dd885b149e2 100644 --- a/python/arcticdb/util/test.py +++ b/python/arcticdb/util/test.py @@ -752,11 +752,14 @@ def read_modify_write_data(lib, symbol, query): return lib.read(dest_symbol).data -def get_query_processing_functions(lib, symbol, arctic_query): +def get_query_processing_functions(lib, symbol, arctic_query, date_range=None): + qb = copy.deepcopy(arctic_query) test_read_modify_write = os.getenv("ARCTICDB_TEST_READ_MODIFY_WRITE", "0") == "1" - processing_functions = [lambda: lib.read(symbol, query_builder=arctic_query).data] + processing_functions = [] # [lambda: lib.read(symbol, query_builder=qb).data] if test_read_modify_write: - processing_functions.append(lambda: read_modify_write_data(lib, symbol, arctic_query)) + if date_range is not None: + qb.prepend(QueryBuilder().date_range(date_range)) + processing_functions.append(lambda: read_modify_write_data(lib, symbol, qb)) return processing_functions @@ -823,34 +826,39 @@ def generic_filter_test_strings_dynamic(lib, base_symbol, slices, arctic_query, # TODO: Replace with np.array_equal with equal_nan argument (added in 1.19.0) def generic_filter_test_nans(lib, symbol, arctic_query, expected, output_format=OutputFormat.PANDAS): - received = lib.read(symbol, query_builder=arctic_query).data - assert expected.shape == received.shape - for col in expected.columns: - expected_col = expected.loc[:, col] - received_col = received.loc[:, col] - for idx, expected_val in expected_col.items(): - received_val = received_col[idx] - if isinstance(expected_val, str): - assert isinstance(received_val, str) and expected_val == received_val - elif expected_val is None: - assert received_val is None - elif np.isnan(expected_val): - if output_format == OutputFormat.PANDAS: - assert np.isnan(received_val) - else: - # When reading as arrow `None` vs `NaN` information is lost. It's all stored as arrow `null`s - # which then is converted to pandas `None`s + query_processing_functions = get_query_processing_functions(lib, symbol, arctic_query) + for proccessing_function in query_processing_functions: + received = proccessing_function() + assert expected.shape == received.shape + for col in expected.columns: + expected_col = expected.loc[:, col] + received_col = received.loc[:, col] + for idx, expected_val in expected_col.items(): + received_val = received_col[idx] + if isinstance(expected_val, str): + assert isinstance(received_val, str) and expected_val == received_val + elif expected_val is None: assert received_val is None + elif np.isnan(expected_val): + if output_format == OutputFormat.PANDAS: + assert np.isnan(received_val) + else: + # When reading as arrow `None` vs `NaN` information is lost. It's all stored as arrow `null`s + # which then is converted to pandas `None`s + assert received_val is None def generic_aggregation_test(lib, symbol, df, grouping_column, aggs_dict): expected = df.groupby(grouping_column).agg(aggs_dict) expected = expected.reindex(columns=sorted(expected.columns)) q = QueryBuilder().groupby(grouping_column).agg(aggs_dict) - received = lib.read(symbol, query_builder=q).data - received = received.reindex(columns=sorted(received.columns)) - received.sort_index(inplace=True) - assert_frame_equal(expected, received, check_dtype=False) + query_processing_functions = get_query_processing_functions(lib, symbol, q) + for proccessing_function in query_processing_functions: + print(expected) + received = proccessing_function() + received = received.reindex(columns=sorted(received.columns)) + received.sort_index(inplace=True) + assert_frame_equal(expected, received, check_dtype=False) def generic_named_aggregation_test(lib, symbol, df, grouping_column, aggs_dict, agg_dtypes=None): @@ -865,21 +873,23 @@ def generic_named_aggregation_test(lib, symbol, df, grouping_column, aggs_dict, expected[name] = expected[name].fillna(0) expected = expected.astype(agg_dtypes) q = QueryBuilder().groupby(grouping_column).agg(aggs_dict) - received = lib.read(symbol, query_builder=q).data - received = received.reindex(columns=sorted(received.columns)) - received.sort_index(inplace=True) - try: - assert_frame_equal(expected, received, check_dtype=agg_dtypes is not None) - except AssertionError as e: - print( - f"""Original df:\n{df}\nwith dtypes:\n{df.dtypes}\naggs dict:\n{aggs_dict}""" - f"""\nPandas result:\n{expected}\n"ArcticDB result:\n{received}""" - f"""\n{df.dtypes}""" - f"""\n{expected.dtypes}""" - f"""\n{received.dtypes}""" - f"""\n{agg_dtypes}""" - ) - raise e + query_processing_functions = get_query_processing_functions(lib, symbol, q) + for proccessing_function in query_processing_functions: + received = proccessing_function() + received = received.reindex(columns=sorted(received.columns)) + received.sort_index(inplace=True) + try: + assert_frame_equal(expected, received, check_dtype=agg_dtypes is not None) + except AssertionError as e: + print( + f"""Original df:\n{df}\nwith dtypes:\n{df.dtypes}\naggs dict:\n{aggs_dict}""" + f"""\nPandas result:\n{expected}\n"ArcticDB result:\n{received}""" + f"""\n{df.dtypes}""" + f"""\n{expected.dtypes}""" + f"""\n{received.dtypes}""" + f"""\n{agg_dtypes}""" + ) + raise e def drop_inf_and_nan(df: pd.DataFrame) -> pd.DataFrame: @@ -1029,38 +1039,48 @@ def generic_resample_test( q = q.resample(rule, closed=closed, label=label, offset=offset, origin=origin).agg(aggregations) else: q = q.resample(rule, closed=closed, label=label, offset=offset).agg(aggregations) - received = lib.read(sym, date_range=date_range, query_builder=q).data - received = received.reindex(columns=sorted(received.columns)) + query_processing_functions = get_query_processing_functions(lib, sym, q, date_range) + for proccessing_function in query_processing_functions: + received = proccessing_function() + received = received.reindex(columns=sorted(received.columns)) - expected = expected_pandas_resample_generic( - original_data, rule, aggregations, closed, label, offset, origin, drop_empty_buckets_for, expected_types - ) + expected = expected_pandas_resample_generic( + original_data, rule, aggregations, closed, label, offset, origin, drop_empty_buckets_for, expected_types + ) - check_dtype = expected_types is not None - try: - assert_resampled_dataframes_are_equal(received, expected, check_dtype=check_dtype) - except AssertionError: - if origin in ["end_day", "end"] and closed == "right": - # Pandas has a bug https://github.com/pandas-dev/pandas/issues/62154 - # When end_day or end is used with right closed, and the first rows lie on the bucket, Pandas includes them in the - # bucket, which is wrong since closed is right. We remove the first rows from the dataframe and run the - # resampling again. - - resampler = create_resampler(original_data, rule, closed, "left", offset, origin) - first_bin = list(resampler.groups.keys())[0] - rows_to_pop = 0 - while rows_to_pop < len(original_data) and original_data.index[rows_to_pop] == first_bin: - rows_to_pop += 1 - if rows_to_pop == 0: - # If there are no rows to be removed, then this is not the same issue. - raise - original_data = original_data.tail(len(original_data) - rows_to_pop) - expected = expected_pandas_resample_generic( - original_data, rule, aggregations, closed, label, offset, origin, drop_empty_buckets_for, expected_types - ) + check_dtype = expected_types is not None + try: assert_resampled_dataframes_are_equal(received, expected, check_dtype=check_dtype) - else: - raise + except AssertionError: + if origin in ["end_day", "end"] and closed == "right": + # Pandas has a bug https://github.com/pandas-dev/pandas/issues/62154 + # When end_day or end is used with right closed, and the first rows lie on the bucket, Pandas includes them in the + # bucket, which is wrong since closed is right. We remove the first rows from the dataframe and run the + # resampling again. + + resampler = create_resampler(original_data, rule, closed, "left", offset, origin) + first_bin = list(resampler.groups.keys())[0] + rows_to_pop = 0 + while rows_to_pop < len(original_data) and original_data.index[rows_to_pop] == first_bin: + rows_to_pop += 1 + if rows_to_pop == 0: + # If there are no rows to be removed, then this is not the same issue. + raise + original_data = original_data.tail(len(original_data) - rows_to_pop) + expected = expected_pandas_resample_generic( + original_data, + rule, + aggregations, + closed, + label, + offset, + origin, + drop_empty_buckets_for, + expected_types, + ) + assert_resampled_dataframes_are_equal(received, expected, check_dtype=check_dtype) + else: + raise def equals(x, y): From 3490e74df2dba739aa8ac4248cc1fd801a590a4f Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Fri, 24 Oct 2025 16:40:00 +0300 Subject: [PATCH 08/46] Compact rows function --- cpp/arcticdb/processing/clause.cpp | 1 - cpp/arcticdb/version/version_core.cpp | 59 +++++++++---------- .../arcticdb/version_store/test_resample.py | 2 +- 3 files changed, 29 insertions(+), 33 deletions(-) diff --git a/cpp/arcticdb/processing/clause.cpp b/cpp/arcticdb/processing/clause.cpp index f4f99f37384..f1d85e6341e 100644 --- a/cpp/arcticdb/processing/clause.cpp +++ b/cpp/arcticdb/processing/clause.cpp @@ -561,7 +561,6 @@ std::vector AggregationClause::process(std::vector&& entity_ seg.set_string_pool(string_pool); seg.set_row_id(num_unique - 1); - return push_entities(*component_manager_, ProcessingUnit(std::move(seg))); } diff --git a/cpp/arcticdb/version/version_core.cpp b/cpp/arcticdb/version/version_core.cpp index 732aa619955..466ce7e4f14 100644 --- a/cpp/arcticdb/version/version_core.cpp +++ b/cpp/arcticdb/version/version_core.cpp @@ -135,6 +135,32 @@ IndexDescriptorImpl check_index_match(const arcticdb::stream::Index& index, cons return desc; } + +/// When segments are produced by the processing pipeline some rows might be missing. Imagine a filter with the middle +/// rows filtered out. These slices cannot be put in an index key as the row slices in the index key must be contiguous. +/// This function adjusts the row slices so that there are no gaps. +/// @note This expects the slices to be sorted by colum slice i.e. first are all row in the first column slice, then +/// are the row slices in the second column slice, etc... +void compact_row_slices(std::span slices) { + size_t current_compacted_row = 0; + size_t previous_uncompacted_end = slices.empty() ? 0 : slices.front().slice().row_range.end(); + size_t previous_col_slice_end = slices.empty() ? 0 : slices.front().slice().col_range.end(); + for (SliceAndKey& slice : slices) { + // Aggregation clause performs aggregations in parallel for each group. Thus it can produce several slices with + // the exact row_range and column_range. The ordering doesnt matter but this must be taken into account so that + // the row slices are always increasing. The second condition in the if takes care of this scenario. + if (slice.slice().row_range.start() < previous_uncompacted_end && + slice.slice().col_range.start() > previous_col_slice_end) { + current_compacted_row = 0; + } + previous_uncompacted_end = slice.slice().row_range.end(); + const size_t rows_in_slice = slice.slice().row_range.diff(); + slice.slice().row_range.first = current_compacted_row; + slice.slice().row_range.second = current_compacted_row + rows_in_slice; + current_compacted_row += rows_in_slice; + previous_col_slice_end = slice.slice().col_range.end(); + } +} } // namespace void sorted_data_check_append(const InputFrame& frame, index::IndexSegmentReader& index_segment_reader) { @@ -2735,21 +2761,11 @@ VersionedItem read_modify_write_impl( processed_entity_ids )), std::back_inserter(write_segments_futures), - [](std::shared_ptr>& fut) { - folly::Future res = std::move(*fut); - fut.reset(); - return res; - } + [](const std::shared_ptr>& fut) { return std::move(*fut); } ); return folly::collect(std::move(write_segments_futures)); }) .thenValue([&](std::vector&& slices) { - for (auto& slice : slices) { - std::cout << fmt::format( - "Row slice: {}, Col slice: {}\n", slice.slice().row_range, slice.slice().col_range - ); - } - std::cout << "==========================\n"; ranges::sort(slices, [](const SliceAndKey& a, const SliceAndKey& b) { if (a.slice().col_range.first < b.slice().col_range.first) { return true; @@ -2758,26 +2774,7 @@ VersionedItem read_modify_write_impl( } return false; }); - - size_t current_compacted_row = 0; - size_t previous_uncompacted_end = slices.empty() ? 0 : slices.front().slice().row_range.end(); - for (SliceAndKey& slice : slices) { - if (slice.slice().row_range.start() < previous_uncompacted_end) { - current_compacted_row = 0; - } - previous_uncompacted_end = slice.slice().row_range.end(); - const size_t rows_in_slice = slice.slice().row_range.diff(); - slice.slice().row_range.first = current_compacted_row; - slice.slice().row_range.second = current_compacted_row + rows_in_slice; - current_compacted_row += rows_in_slice; - } - - for (auto& slice : slices) { - std::cout << fmt::format( - "Row slice: {}, Col slice: {}\n", slice.slice().row_range, slice.slice().col_range - ); - } - + compact_row_slices(slices); const size_t row_count = slices.empty() ? 0 : slices.back().slice().row_range.second - slices[0].slice().row_range.first; const TimeseriesDescriptor tsd = make_timeseries_descriptor( diff --git a/python/tests/unit/arcticdb/version_store/test_resample.py b/python/tests/unit/arcticdb/version_store/test_resample.py index 6d88fb6b5a3..1a99729fd2b 100644 --- a/python/tests/unit/arcticdb/version_store/test_resample.py +++ b/python/tests/unit/arcticdb/version_store/test_resample.py @@ -30,7 +30,7 @@ pytestmark = pytest.mark.pipeline -ALL_AGGREGATIONS = ["sum", "mean", "min", "max", "first", "last", "count"] +ALL_AGGREGATIONS = ["sum"] def all_aggregations_dict(col): From b61326493af21ea781cfc162ed9430f57788f947 Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Fri, 24 Oct 2025 17:47:19 +0300 Subject: [PATCH 09/46] Add comments --- python/arcticdb/util/test.py | 9 ++++++-- .../version_store/test_read_modify_write.py | 21 ------------------- .../arcticdb/version_store/test_resample.py | 2 +- 3 files changed, 8 insertions(+), 24 deletions(-) delete mode 100644 python/tests/unit/arcticdb/version_store/test_read_modify_write.py diff --git a/python/arcticdb/util/test.py b/python/arcticdb/util/test.py index dd885b149e2..f49c16c9ba3 100644 --- a/python/arcticdb/util/test.py +++ b/python/arcticdb/util/test.py @@ -753,9 +753,15 @@ def read_modify_write_data(lib, symbol, query): def get_query_processing_functions(lib, symbol, arctic_query, date_range=None): + """ + This added to ease testing of read_modify_write. It returns a list of functions which perform processing + One is the standard read with a query builder, the other is a read_modify_write followed by a read of the newly + created symbol. Functions that test the QueryBuilder such as generic_filter_test, generic_aggregation_test, etc... + can iterate on the result array and test both methods for processing. + """ qb = copy.deepcopy(arctic_query) test_read_modify_write = os.getenv("ARCTICDB_TEST_READ_MODIFY_WRITE", "0") == "1" - processing_functions = [] # [lambda: lib.read(symbol, query_builder=qb).data] + processing_functions = [lambda: lib.read(symbol, query_builder=qb).data] if test_read_modify_write: if date_range is not None: qb.prepend(QueryBuilder().date_range(date_range)) @@ -854,7 +860,6 @@ def generic_aggregation_test(lib, symbol, df, grouping_column, aggs_dict): q = QueryBuilder().groupby(grouping_column).agg(aggs_dict) query_processing_functions = get_query_processing_functions(lib, symbol, q) for proccessing_function in query_processing_functions: - print(expected) received = proccessing_function() received = received.reindex(columns=sorted(received.columns)) received.sort_index(inplace=True) diff --git a/python/tests/unit/arcticdb/version_store/test_read_modify_write.py b/python/tests/unit/arcticdb/version_store/test_read_modify_write.py deleted file mode 100644 index 5ae8cf3d089..00000000000 --- a/python/tests/unit/arcticdb/version_store/test_read_modify_write.py +++ /dev/null @@ -1,21 +0,0 @@ -import pandas as pd -import pytest -from arcticdb.version_store.processing import QueryBuilder -from arcticdb.util.test import assert_frame_equal - - -def test_read_modify_write_filter(lmdb_version_store_dynamic_schema_v1): - lib = lmdb_version_store_dynamic_schema_v1 - sym = "sym" - df1 = pd.DataFrame({"a": [1.0, 2.0, 3.0], "b": [4.0, 5.0, 6.0]}) - lib.append(sym, df1) - - df2 = pd.DataFrame({"b": [4.0, 5.0, 6.0]}) - lib.append(sym, df2) - - df3 = pd.DataFrame({"a": [4.0, 5.0, 6.0]}) - lib.append(sym, df3) - - q = QueryBuilder() - q = q[q["a"] > 2.0] - lib._read_modify_write(sym, q, "sym2") diff --git a/python/tests/unit/arcticdb/version_store/test_resample.py b/python/tests/unit/arcticdb/version_store/test_resample.py index 1a99729fd2b..6d88fb6b5a3 100644 --- a/python/tests/unit/arcticdb/version_store/test_resample.py +++ b/python/tests/unit/arcticdb/version_store/test_resample.py @@ -30,7 +30,7 @@ pytestmark = pytest.mark.pipeline -ALL_AGGREGATIONS = ["sum"] +ALL_AGGREGATIONS = ["sum", "mean", "min", "max", "first", "last", "count"] def all_aggregations_dict(col): From 769e271a0dffbf36ccd6f0aac641a1af7ff661ab Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Fri, 24 Oct 2025 18:07:06 +0300 Subject: [PATCH 10/46] Fix compilation errors --- cpp/arcticdb/async/test/test_async.cpp | 20 ++++---------------- 1 file changed, 4 insertions(+), 16 deletions(-) diff --git a/cpp/arcticdb/async/test/test_async.cpp b/cpp/arcticdb/async/test/test_async.cpp index 23d653a9343..fa26813efed 100644 --- a/cpp/arcticdb/async/test/test_async.cpp +++ b/cpp/arcticdb/async/test/test_async.cpp @@ -91,20 +91,10 @@ TEST(Async, DeDupTest) { aa::AsyncStore store(lib, *codec_opt, EncodingVersion::V2); auto seg = SegmentInMemory(); - std::vector> key_segments; + std::vector> key_segments; - key_segments.emplace_back( - ast::StreamSink::PartialKey{ - entity::KeyType::TABLE_DATA, 1, "", entity::NumericIndex{0}, entity::NumericIndex{1} - }, - seg - ); - key_segments.emplace_back( - ast::StreamSink::PartialKey{ - entity::KeyType::TABLE_DATA, 2, "", entity::NumericIndex{1}, entity::NumericIndex{2} - }, - seg - ); + key_segments.emplace_back(PartialKey{KeyType::TABLE_DATA, 1, "", NumericIndex{0}, NumericIndex{1}}, seg); + key_segments.emplace_back(PartialKey{KeyType::TABLE_DATA, 2, "", NumericIndex{1}, NumericIndex{2}}, seg); HashAccum h; auto default_content_hash = h.digest(); @@ -121,9 +111,7 @@ TEST(Async, DeDupTest) { std::vector> slice_key_futures; for (auto& [key, segment] : key_segments) { - auto input = std::make_tuple( - std::move(key), std::move(segment), {} - ); + auto input = std::make_tuple(std::move(key), std::move(segment), {}); auto fut = folly::makeFuture(std::move(input)); slice_key_futures.emplace_back(store.async_write(std::move(fut), de_dup_map)); } From 83a8065ca6dd339b493210c251bb3b3c53f17394 Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Fri, 24 Oct 2025 18:25:56 +0300 Subject: [PATCH 11/46] Fix compilation errors --- cpp/arcticdb/processing/clause.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/cpp/arcticdb/processing/clause.cpp b/cpp/arcticdb/processing/clause.cpp index f1d85e6341e..12a21598bcc 100644 --- a/cpp/arcticdb/processing/clause.cpp +++ b/cpp/arcticdb/processing/clause.cpp @@ -547,9 +547,12 @@ std::vector AggregationClause::process(std::vector&& entity_ ); auto column_data = index_col->data(); - ranges::transform(elements, column_data.begin(), [](const auto& element) { - return element.first; - }); + std::transform( + elements.begin(), + elements.begin(), + column_data.begin(), + [](const auto& element) { return element.first; } + ); }); index_col->set_row_data(grouping_map.size() - 1); From c57fdc0efff3abc980bbb96eac67f2e7a9fc05fa Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Fri, 24 Oct 2025 18:26:21 +0300 Subject: [PATCH 12/46] Fix compilation errors --- cpp/arcticdb/processing/clause.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cpp/arcticdb/processing/clause.cpp b/cpp/arcticdb/processing/clause.cpp index 12a21598bcc..9d3a6680839 100644 --- a/cpp/arcticdb/processing/clause.cpp +++ b/cpp/arcticdb/processing/clause.cpp @@ -548,8 +548,8 @@ std::vector AggregationClause::process(std::vector&& entity_ auto column_data = index_col->data(); std::transform( - elements.begin(), - elements.begin(), + elements.cbegin(), + elements.cend(), column_data.begin(), [](const auto& element) { return element.first; } ); From 27cbe29f92003fd8406e9fba56bc0b3009156196 Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Thu, 30 Oct 2025 18:06:15 +0200 Subject: [PATCH 13/46] Apply comments --- cpp/arcticdb/pipeline/frame_slice.hpp | 12 ++--- cpp/arcticdb/pipeline/slicing.hpp | 2 +- cpp/arcticdb/processing/clause.cpp | 9 ++-- cpp/arcticdb/processing/clause.hpp | 2 +- cpp/arcticdb/version/python_bindings.cpp | 2 +- cpp/arcticdb/version/version_core.cpp | 64 ++++++++++++------------ python/arcticdb/version_store/_store.py | 2 +- 7 files changed, 45 insertions(+), 48 deletions(-) diff --git a/cpp/arcticdb/pipeline/frame_slice.hpp b/cpp/arcticdb/pipeline/frame_slice.hpp index 5dd4bdc5a6b..2cb3337c7bd 100644 --- a/cpp/arcticdb/pipeline/frame_slice.hpp +++ b/cpp/arcticdb/pipeline/frame_slice.hpp @@ -106,9 +106,6 @@ struct FrameSlice { [[nodiscard]] const ColRange& columns() const { return col_range; } [[nodiscard]] const RowRange& rows() const { return row_range; } - ColRange col_range; - RowRange row_range; - [[nodiscard]] std::size_t absolute_field_col(std::size_t col) const { if (indices_) return indices_->at(col) - desc()->index().field_count(); @@ -131,8 +128,8 @@ struct FrameSlice { return static_cast(row_range.second); } - friend bool operator<(const FrameSlice& a, const FrameSlice& b) { - return std::tie(a.col_range.first, a.row_range.first) < std::tie(b.col_range.first, b.row_range.first); + friend std::weak_ordering operator<=>(const FrameSlice& a, const FrameSlice& b) { + return std::tie(a.col_range.first, a.row_range.first) <=> std::tie(b.col_range.first, b.row_range.first); } friend bool operator==(const FrameSlice& a, const FrameSlice& b) { @@ -141,6 +138,9 @@ struct FrameSlice { void check_magic() const { magic_.check(); } + ColRange col_range; + RowRange row_range; + private: // never contains index field std::shared_ptr desc_; @@ -281,7 +281,7 @@ struct SliceAndKey { std::optional key_; }; -inline bool operator<(const SliceAndKey& a, const SliceAndKey& b) { return a.slice_ < b.slice_; } +inline std::weak_ordering operator<=>(const SliceAndKey& a, const SliceAndKey& b) { return a.slice_ <=> b.slice_; } } // namespace arcticdb::pipelines diff --git a/cpp/arcticdb/pipeline/slicing.hpp b/cpp/arcticdb/pipeline/slicing.hpp index ff1d4a530c7..f0a812e5727 100644 --- a/cpp/arcticdb/pipeline/slicing.hpp +++ b/cpp/arcticdb/pipeline/slicing.hpp @@ -96,7 +96,7 @@ inline auto get_partial_key_gen(std::shared_ptr frame, const TypedSt // search for the relevant block. An alternative would be to look at the segment that was just generated in // WriteToSegmentTask and similar methods, but this is unlikely to be a bottleneck auto start = frame->index_value_at(slice_begin_pos(s, *frame)); - auto end = frame->index_value_at(slice_end_pos(s, *frame)); + const auto end = frame->index_value_at(slice_end_pos(s, *frame)); return PartialKey{key.type, key.version_id, key.id, start, end_index_generator(end)}; } else { return PartialKey{ diff --git a/cpp/arcticdb/processing/clause.cpp b/cpp/arcticdb/processing/clause.cpp index 9d3a6680839..731655bf586 100644 --- a/cpp/arcticdb/processing/clause.cpp +++ b/cpp/arcticdb/processing/clause.cpp @@ -1682,7 +1682,7 @@ std::vector WriteClause::process(std::vector&& entity_ids) c const SegmentInMemory& segment = *(*proc.segments_)[i]; const RowRange& row_range = *(*proc.row_ranges_)[i]; const ColRange& col_range = *(*proc.col_ranges_)[i]; - stream::PartialKey partial_key = create_partial_key(segment.descriptor(), segment); + stream::PartialKey partial_key = create_partial_key(segment); data_segments_to_write.push_back(std::make_shared>(store_->async_write( std::make_tuple(std::move(partial_key), segment, FrameSlice(col_range, row_range)), dedup_map_ ))); @@ -1690,9 +1690,8 @@ std::vector WriteClause::process(std::vector&& entity_ids) c return component_manager_->add_entities(std::move(data_segments_to_write)); } -stream::PartialKey WriteClause::create_partial_key(const StreamDescriptor& stream_desc, const SegmentInMemory& segment) - const { - if (stream_desc.index().type() == IndexDescriptor::Type::ROWCOUNT) { +stream::PartialKey WriteClause::create_partial_key(const SegmentInMemory& segment) const { + if (segment.descriptor().index().type() == IndexDescriptor::Type::ROWCOUNT) { return stream::PartialKey{ .key_type = KeyType::TABLE_DATA, .version_id = index_partial_key_.version_id, @@ -1700,7 +1699,7 @@ stream::PartialKey WriteClause::create_partial_key(const StreamDescriptor& strea .start_index = 0, .end_index = 0 }; - } else if (stream_desc.index().type() == IndexDescriptor::Type::TIMESTAMP) { + } else if (segment.descriptor().index().type() == IndexDescriptor::Type::TIMESTAMP) { const timestamp start_ts = std::get(stream::TimeseriesIndex::start_value_for_segment(segment)); const timestamp end_ts = std::get(end_index_generator(stream::TimeseriesIndex::end_value_for_segment(segment))); diff --git a/cpp/arcticdb/processing/clause.hpp b/cpp/arcticdb/processing/clause.hpp index 0c7c0e2f67b..598719b958c 100644 --- a/cpp/arcticdb/processing/clause.hpp +++ b/cpp/arcticdb/processing/clause.hpp @@ -835,7 +835,7 @@ struct WriteClause { [[nodiscard]] std::string to_string() const; private: - stream::PartialKey create_partial_key(const StreamDescriptor& descriptor, const SegmentInMemory& segment) const; + stream::PartialKey create_partial_key(const SegmentInMemory& segment) const; }; } // namespace arcticdb diff --git a/cpp/arcticdb/version/python_bindings.cpp b/cpp/arcticdb/version/python_bindings.cpp index abd0e9df01f..060282ec26f 100644 --- a/cpp/arcticdb/version/python_bindings.cpp +++ b/cpp/arcticdb/version/python_bindings.cpp @@ -825,7 +825,7 @@ void register_bindings(py::module& version, py::exception(), "Read the specified version of the dataframe from the store" ) - .def("read_modify_write", + .def("_read_modify_write", &PythonVersionStore::read_modify_write, py::call_guard(), "Read, modify and write the specified version for the dataframe (experimental)") diff --git a/cpp/arcticdb/version/version_core.cpp b/cpp/arcticdb/version/version_core.cpp index 466ce7e4f14..d9631dd408e 100644 --- a/cpp/arcticdb/version/version_core.cpp +++ b/cpp/arcticdb/version/version_core.cpp @@ -136,19 +136,22 @@ IndexDescriptorImpl check_index_match(const arcticdb::stream::Index& index, cons return desc; } -/// When segments are produced by the processing pipeline some rows might be missing. Imagine a filter with the middle +/// When segments are produced by the processing pipeline, some rows might be missing. Imagine a filter with the middle /// rows filtered out. These slices cannot be put in an index key as the row slices in the index key must be contiguous. /// This function adjusts the row slices so that there are no gaps. -/// @note This expects the slices to be sorted by colum slice i.e. first are all row in the first column slice, then -/// are the row slices in the second column slice, etc... -void compact_row_slices(std::span slices) { +/// @note This expects the slices to be sorted by colum slice i.e., first are all row in the first column slice, then +/// are the row slices in the second column slice, etc... It also expects that there are no missing columns. The +/// difference between this and adjust_slice_ranges is that this will not change the column slices. While +/// adjust_slice_ranges will change the column slices, making the first slice start from 0 even for timestamp-index +/// dataframes (which should have column range starting from 1 when being written to disk). +[[maybe_unused]] void compact_row_slices(std::span slices) { size_t current_compacted_row = 0; size_t previous_uncompacted_end = slices.empty() ? 0 : slices.front().slice().row_range.end(); size_t previous_col_slice_end = slices.empty() ? 0 : slices.front().slice().col_range.end(); for (SliceAndKey& slice : slices) { - // Aggregation clause performs aggregations in parallel for each group. Thus it can produce several slices with - // the exact row_range and column_range. The ordering doesnt matter but this must be taken into account so that - // the row slices are always increasing. The second condition in the if takes care of this scenario. + // Aggregation clause performs aggregations in parallel for each group. Thus, it can produce several slices with + // the exact row_range and column_range. The ordering doesn't matter, but this must be taken into account so + // that the row slices are always increasing. The second condition in the "if" takes care of this scenario. if (slice.slice().row_range.start() < previous_uncompacted_end && slice.slice().col_range.start() > previous_col_slice_end) { current_compacted_row = 0; @@ -1149,7 +1152,7 @@ folly::Future> read_and_schedule_processing( const std::shared_ptr& read_query, const ReadOptions& read_options, std::shared_ptr component_manager ) { - ProcessingConfig processing_config{ + const ProcessingConfig processing_config{ opt_false(read_options.dynamic_schema()), pipeline_context->rows_, pipeline_context->descriptor().index().type() @@ -1283,15 +1286,17 @@ void check_multi_key_is_not_index_only(const PipelineContext& pipeline_context, ); } -void check_can_be_filtered(const std::shared_ptr& pipeline_context, const ReadQuery& read_query) { +void check_can_perform_processing( + const std::shared_ptr& pipeline_context, const ReadQuery& read_query +) { // To remain backward compatibility, pending new major release to merge into below section // Ticket: 18038782559 - bool is_pickled = pipeline_context->norm_meta_ && pipeline_context->is_pickled(); + const bool is_pickled = pipeline_context->norm_meta_ && pipeline_context->is_pickled(); util::check( !is_pickled || (!read_query.columns.has_value() && std::holds_alternative(read_query.row_filter)), - "The data for this symbol is pickled and does not support column stats, date_range, row_range, or column " - "queries" + "Cannot perform processing such as row/column filtering, projection, aggregation, resampling, " + "etc.. on pickled data" ); if (pipeline_context->multi_key_) { check_multi_key_is_not_index_only(*pipeline_context, read_query); @@ -1304,27 +1309,30 @@ void check_can_be_filtered(const std::shared_ptr& pipeline_cont !std::holds_alternative(read_query.row_filter), "Cannot apply date range filter to symbol with non-timestamp index" ); - auto sorted_value = pipeline_context->descriptor().sorted(); + const auto sorted_value = pipeline_context->descriptor().sorted(); sorting::check( sorted_value == SortedValue::UNKNOWN || sorted_value == SortedValue::ASCENDING || !std::holds_alternative(read_query.row_filter), "When filtering data using date_range, the symbol must be sorted in ascending order. ArcticDB believes " - "it " - "is not sorted in ascending order and cannot therefore filter the data using date_range." + "it is not sorted in ascending order and cannot therefore filter the data using date_range." ); } - bool is_query_empty = + const bool is_query_empty = (!read_query.columns && !read_query.row_range && std::holds_alternative(read_query.row_filter) && read_query.clauses_.empty()); - bool is_numpy_array = pipeline_context->norm_meta_ && pipeline_context->norm_meta_->has_np(); + const bool is_numpy_array = pipeline_context->norm_meta_ && pipeline_context->norm_meta_->has_np(); if (!is_query_empty) { // Exception for filterig pickled data is skipped for now for backward compatibility if (pipeline_context->multi_key_) { schema::raise( - "Cannot filter recursively normalized data" + "Cannot perform processing such as row/column filtering, projection, aggregation, resampling, " + "etc.. on recursively normalized data" ); } else if (is_numpy_array) { - schema::raise("Cannot filter numpy array"); + schema::raise( + "Cannot perform processing such as row/column filtering, projection, aggregation, resampling, " + "etc.. on recursively numpy array" + ); } } } @@ -1362,7 +1370,7 @@ static void read_indexed_keys_to_pipeline( std::move(*index_segment_reader.mutable_tsd().mutable_proto().mutable_user_meta()) ); pipeline_context->bucketize_dynamic_ = bucketize_dynamic; - check_can_be_filtered(pipeline_context, read_query); + check_can_perform_processing(pipeline_context, read_query); ARCTICDB_DEBUG( log::version(), "read_indexed_keys_to_pipeline: Symbol {} Found {} keys with {} total rows", @@ -2704,7 +2712,7 @@ folly::Future read_frame_for_version( if (pipeline_context->multi_key_) { if (read_query) { - check_can_be_filtered(pipeline_context, *read_query); + check_can_perform_processing(pipeline_context, *read_query); } return read_multi_key( store, read_options, *pipeline_context->multi_key_, handler_data, std::move(res_versioned_item.key_) @@ -2746,10 +2754,7 @@ VersionedItem read_modify_write_impl( WriteClause(write_options, target_partial_index_key, std::make_shared(), store) )); const auto pipeline_context = setup_pipeline_context(store, source_version_info, *read_query, read_options); - internal::check( - !pipeline_context->multi_key_, - "Performing read modify write is not supported for data using recursive or custom normalizers" - ); + check_can_perform_processing(pipeline_context, *read_query); auto component_manager = std::make_shared(); return read_and_schedule_processing(store, pipeline_context, read_query, read_options, component_manager) @@ -2766,14 +2771,7 @@ VersionedItem read_modify_write_impl( return folly::collect(std::move(write_segments_futures)); }) .thenValue([&](std::vector&& slices) { - ranges::sort(slices, [](const SliceAndKey& a, const SliceAndKey& b) { - if (a.slice().col_range.first < b.slice().col_range.first) { - return true; - } else if (a.slice().col_range.first == b.slice().col_range.first) { - return a.slice().row_range.first < b.slice().row_range.first; - } - return false; - }); + ranges::sort(slices); compact_row_slices(slices); const size_t row_count = slices.empty() ? 0 : slices.back().slice().row_range.second - slices[0].slice().row_range.first; diff --git a/python/arcticdb/version_store/_store.py b/python/arcticdb/version_store/_store.py index 68305da36cb..acd45cd726f 100644 --- a/python/arcticdb/version_store/_store.py +++ b/python/arcticdb/version_store/_store.py @@ -2268,7 +2268,7 @@ def _read_modify_write( query_builder=query_builder, **kwargs, ) - return self.version_store.read_modify_write( + return self.version_store._read_modify_write( source_symbol, target_symbol, udm, version_query, read_query, read_options, prune_previous_versions ) From 5f9421662b6f0dac7a4d30b17920d3d375d93758 Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Fri, 31 Oct 2025 19:21:18 +0200 Subject: [PATCH 14/46] Add C++ stress test to check for deadlocks in write clause --- cpp/arcticdb/processing/clause_utils.cpp | 12 +- cpp/arcticdb/processing/clause_utils.hpp | 2 +- .../test/test_parallel_processing.cpp | 182 +++++++++++++----- 3 files changed, 145 insertions(+), 51 deletions(-) diff --git a/cpp/arcticdb/processing/clause_utils.cpp b/cpp/arcticdb/processing/clause_utils.cpp index 3ad949d2828..69a3e2b4219 100644 --- a/cpp/arcticdb/processing/clause_utils.cpp +++ b/cpp/arcticdb/processing/clause_utils.cpp @@ -113,19 +113,19 @@ std::vector split_futures( } std::shared_ptr> generate_segment_fetch_counts( - const std::vector>& processing_unit_indexes, size_t num_segments + const std::span> processing_unit_indexes, const size_t num_segments ) { - auto res = std::make_shared>(num_segments, 0); + auto res = std::vector(num_segments, 0); for (const auto& list : processing_unit_indexes) { - for (auto idx : list) { - res->at(idx)++; + for (const auto idx : list) { + res[idx]++; } } debug::check( - ranges::none_of(*res, [](size_t val) { return val == 0; }), + ranges::none_of(res, [](const size_t val) { return val == 0; }), "All segments should be needed by at least one ProcessingUnit" ); - return res; + return std::make_shared>(std::move(res)); } template diff --git a/cpp/arcticdb/processing/clause_utils.hpp b/cpp/arcticdb/processing/clause_utils.hpp index 92aba5cd346..51fb1eb1b55 100644 --- a/cpp/arcticdb/processing/clause_utils.hpp +++ b/cpp/arcticdb/processing/clause_utils.hpp @@ -209,7 +209,7 @@ std::vector push_entities( ); std::shared_ptr> generate_segment_fetch_counts( - const std::vector>& processing_unit_indexes, size_t num_segments + std::span> processing_unit_indexes, size_t num_segments ); // Multi-symbol join utilities diff --git a/cpp/arcticdb/processing/test/test_parallel_processing.cpp b/cpp/arcticdb/processing/test/test_parallel_processing.cpp index 747b25e2fc4..fa085e06fa9 100644 --- a/cpp/arcticdb/processing/test/test_parallel_processing.cpp +++ b/cpp/arcticdb/processing/test/test_parallel_processing.cpp @@ -11,10 +11,78 @@ #include #include #include +#include +#include using namespace arcticdb; using namespace arcticdb::pipelines; +// Initialising devices is costly. Create one and reuse it. +std::random_device device{}; + +std::shared_ptr set_component_manager(const std::span> clauses) { + auto component_manager = std::make_shared(); + for (auto& clause : clauses) { + clause->set_component_manager(component_manager); + } + return component_manager; +} + +void push_segments(const std::span> segment_and_slice_promises) { + for (size_t idx = 0; idx < segment_and_slice_promises.size(); ++idx) { + SegmentInMemory segment; + segment.descriptor().set_id(static_cast(idx)); + segment.descriptor().set_index({IndexDescriptorImpl::Type::ROWCOUNT, 0}); + segment_and_slice_promises[idx].setValue( + SegmentAndSlice(RangesAndKey({idx, idx + 1}, {0, 1}, {}), std::move(segment)) + ); + } +} + +template +std::shared_ptr create_clause(const size_t n) { + if (index == n) { + return std::make_shared(CurrentClause{}); + } else if constexpr (sizeof...(RestClauses) > 0) { + return create_clause(n); + } else { + util::raise_rte("Cannot create a clause with type index: {}", n); + } +} + +template +std::shared_ptr create_clause(const size_t n) { + util::check( + n < sizeof...(Clauses), + "Trying to create clause with index {} but only {} clause types provided", + n, + sizeof...(Clauses) + ); + return create_clause<0, Clauses...>(n); +} + +/// Given the type of clauses as a template parameter pack, generate a vector composed of random selection of the +/// clauses +template +std::shared_ptr>> generate_random_clauses(int clause_count) { + if constexpr (sizeof...(ClauseTypes) == 1) { + using ClauseToCreate = std::tuple_element_t<0, std::tuple>; + return std::make_shared>>( + clause_count, std::make_shared(ClauseToCreate{}) + ); + } else { + std::vector> result; + result.reserve(clause_count); + std::mt19937_64 eng{device()}; + std::uniform_int_distribution dist{size_t{0}, sizeof...(ClauseTypes) - 1}; + for (int i = 0; i < clause_count; ++i) { + const int clause_to_generate = dist(eng); + result.emplace_back(create_clause(clause_to_generate)); + } + return std::make_shared>>(std::move(result)); + } +} + struct RowSliceClause { // Simple clause that accepts and produces segments partitioned by row-slice, which is representative of a lot of // the real clauses we support. In place of doing any processing, the process method just sleeps for a random amount @@ -38,9 +106,9 @@ struct RowSliceClause { } [[nodiscard]] std::vector process(std::vector&& entity_ids) const { - std::mt19937_64 eng{std::random_device{}()}; - std::uniform_int_distribution<> dist{10, 100}; - auto sleep_ms = dist(eng); + std::mt19937_64 eng{device()}; + std::uniform_int_distribution dist{10, 100}; + const auto sleep_ms = dist(eng); log::version().warn("RowSliceClause::process sleeping for {}ms", sleep_ms); std::this_thread::sleep_for(std::chrono::milliseconds{sleep_ms}); if (entity_ids.empty()) { @@ -60,10 +128,10 @@ struct RowSliceClause { [[nodiscard]] const ClauseInfo& clause_info() const { return clause_info_; } - void set_processing_config(ARCTICDB_UNUSED const ProcessingConfig&) {} + void set_processing_config(const ProcessingConfig&) {} void set_component_manager(std::shared_ptr component_manager) { - component_manager_ = component_manager; + component_manager_ = std::move(component_manager); } OutputSchema modify_schema(OutputSchema&& output_schema) const { return output_schema; } @@ -93,9 +161,9 @@ struct RestructuringClause { } [[nodiscard]] std::vector process(std::vector&& entity_ids) const { - std::mt19937_64 eng{std::random_device{}()}; - std::uniform_int_distribution<> dist{10, 100}; - auto sleep_ms = dist(eng); + std::mt19937_64 eng{device()}; + std::uniform_int_distribution dist{10, 100}; + const auto sleep_ms = dist(eng); log::version().warn("RestructuringClause::process sleeping for {}ms", sleep_ms); std::this_thread::sleep_for(std::chrono::milliseconds{sleep_ms}); if (entity_ids.empty()) { @@ -115,10 +183,10 @@ struct RestructuringClause { [[nodiscard]] const ClauseInfo& clause_info() const { return clause_info_; } - void set_processing_config(ARCTICDB_UNUSED const ProcessingConfig&) {} + void set_processing_config(const ProcessingConfig&) {} void set_component_manager(std::shared_ptr component_manager) { - component_manager_ = component_manager; + component_manager_ = std::move(component_manager); } OutputSchema modify_schema(OutputSchema&& output_schema) const { return output_schema; } @@ -128,30 +196,18 @@ struct RestructuringClause { TEST(Clause, ScheduleClauseProcessingStress) { // Extensible stress test of schedule_clause_processing. Useful for ensuring a lack of deadlock when running on - // threadpools with 1 or multiple cores. Dummy clauses provided above used to stress the fan-in/fan-out behaviour. - // Could be extended to profile and compare different scheduling algorithms and threadpool implementations if we - // want to move away from folly. + // threadpools with 1 or multiple cores. Dummy clauses provided above are used to stress the fan-in/fan-out + // behaviour. Could be extended to profile and compare different scheduling algorithms and threadpool + // implementations if we want to move away from folly. using namespace arcticdb::version_store; - auto num_clauses = 5; - std::mt19937_64 eng{std::random_device{}()}; - std::uniform_int_distribution<> dist{0, 1}; - - auto clauses = std::make_shared>>(); - for (auto unused = 0; unused < num_clauses; ++unused) { - if (dist(eng) == 0) { - clauses->emplace_back(std::make_shared(RowSliceClause())); - } else { - clauses->emplace_back(std::make_shared(RestructuringClause())); - } - } + constexpr static auto num_clauses = 5; - auto component_manager = std::make_shared(); - for (auto& clause : *clauses) { - clause->set_component_manager(component_manager); - } + auto clauses = generate_random_clauses(num_clauses); - size_t num_segments{2}; - std::vector> segment_and_slice_promises(num_segments); + const auto component_manager = set_component_manager(*clauses); + + constexpr static size_t num_segments{2}; + std::array, num_segments> segment_and_slice_promises; std::vector> segment_and_slice_futures; std::vector> processing_unit_indexes; for (size_t idx = 0; idx < num_segments; ++idx) { @@ -164,21 +220,17 @@ TEST(Clause, ScheduleClauseProcessingStress) { auto segment_fetch_counts = generate_segment_fetch_counts(processing_unit_indexes, num_segments); auto processed_entity_ids_fut = schedule_clause_processing( - component_manager, std::move(segment_and_slice_futures), std::move(processing_unit_indexes), clauses + component_manager, + std::move(segment_and_slice_futures), + std::move(processing_unit_indexes), + std::move(clauses) ); - - for (size_t idx = 0; idx < segment_and_slice_promises.size(); ++idx) { - SegmentInMemory segment; - segment.descriptor().set_id(static_cast(idx)); - segment_and_slice_promises[idx].setValue( - SegmentAndSlice(RangesAndKey({idx, idx + 1}, {0, 1}, {}), std::move(segment)) - ); - } - + push_segments(segment_and_slice_promises); auto processed_entity_ids = std::move(processed_entity_ids_fut).get(); - auto proc = gather_entities, std::shared_ptr, std::shared_ptr>( - *component_manager, std::move(processed_entity_ids) - ); + const auto proc = + gather_entities, std::shared_ptr, std::shared_ptr>( + *component_manager, std::move(processed_entity_ids) + ); ASSERT_EQ(proc.segments_.value().size(), num_segments); NumericId start_id{0}; for (const auto& segment : proc.segments_.value()) { @@ -186,3 +238,45 @@ TEST(Clause, ScheduleClauseProcessingStress) { ASSERT_EQ(id, start_id++ + num_clauses); } } + +TEST(Clause, ScheduleRowSliceProcessingAndWrite) { + using namespace arcticdb::version_store; + constexpr static auto num_clauses = 5; + + auto clauses = generate_random_clauses(num_clauses); + auto store = std::make_shared(); + constexpr static WriteOptions write_options; + clauses->push_back(std::make_shared( + WriteClause(write_options, IndexPartialKey{"target", 0}, std::make_shared(), store) + )); + + const auto component_manager = set_component_manager(*clauses); + + constexpr static size_t num_segments{2}; + std::array, num_segments> segment_and_slice_promises; + std::vector> segment_and_slice_futures; + std::vector> processing_unit_indexes; + for (size_t idx = 0; idx < num_segments; ++idx) { + segment_and_slice_futures.emplace_back(segment_and_slice_promises[idx].getFuture()); + processing_unit_indexes.emplace_back(std::vector{idx}); + } + + auto processed_entity_ids_fut = schedule_clause_processing( + component_manager, + std::move(segment_and_slice_futures), + std::move(processing_unit_indexes), + std::move(clauses) + ); + push_segments(segment_and_slice_promises); + const auto processed_entity_ids = std::move(processed_entity_ids_fut).get(); + std::vector> write_segments_futures; + std::ranges::transform( + std::get<0>( + component_manager->get_entities>>(processed_entity_ids) + ), + std::back_inserter(write_segments_futures), + [](const std::shared_ptr>& fut) { return std::move(*fut); } + ); + const std::vector slices = folly::collect(std::move(write_segments_futures)).get(); + ASSERT_EQ(slices.size(), num_segments); +} From 12e2c28301612386e68926dd9f2924879f0ba70f Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Mon, 3 Nov 2025 17:56:17 +0200 Subject: [PATCH 15/46] Add overloat for async_write that performs the encoding in the current thread --- cpp/arcticdb/async/async_store.hpp | 31 +++++++++++++++++-- cpp/arcticdb/pipeline/write_frame.cpp | 2 +- cpp/arcticdb/stream/incompletes.cpp | 4 ++- cpp/arcticdb/stream/stream_sink.hpp | 5 +++ .../version/local_versioned_engine.cpp | 2 +- 5 files changed, 39 insertions(+), 5 deletions(-) diff --git a/cpp/arcticdb/async/async_store.hpp b/cpp/arcticdb/async/async_store.hpp index 79fc256fb33..47749d7bc50 100644 --- a/cpp/arcticdb/async/async_store.hpp +++ b/cpp/arcticdb/async/async_store.hpp @@ -449,13 +449,40 @@ class AsyncStore : public Store { dedup_lookup, [&](NewObject& obj) { lib->write(obj); - return SliceAndKey{slice, obj.atom_key()}; + return SliceAndKey{std::move(slice), obj.atom_key()}; }, - [&](ExistingObject& obj) { return SliceAndKey{slice, to_atom(std::move(obj))}; } + [&](ExistingObject& obj) { return SliceAndKey{std::move(slice), to_atom(std::move(obj))}; } ); }); } + folly::Future async_write( + std::tuple&& input, + const std::shared_ptr& de_dup_map + ) override { + storage::KeySegmentPair key_seg = EncodeAtomTask{ + std::get<0>(std::move(input)), + ClockType::nanos_since_epoch(), + std::get<1>(std::move(input)), + codec_, + encoding_version_ + }(); + DeDupLookupResult dedup_lookup = lookup_match_in_dedup_map(de_dup_map, key_seg); + return folly::via( + &io_executor(), + [dedup_lookup = std::move(dedup_lookup), slice = std::get<2>(std::move(input)), this]() mutable { + return util::variant_match( + std::move(dedup_lookup), + [&](NewObject&& obj) { + library_->write(obj); + return SliceAndKey{std::move(slice), obj.atom_key()}; + }, + [&](ExistingObject&& obj) { return SliceAndKey{slice, to_atom(std::move(obj))}; } + ); + } + ); + } + void set_failure_sim(const arcticdb::proto::storage::VersionStoreConfig::StorageFailureSimulator& cfg) override { library_->set_failure_sim(cfg); } diff --git a/cpp/arcticdb/pipeline/write_frame.cpp b/cpp/arcticdb/pipeline/write_frame.cpp index 19797248919..26062157843 100644 --- a/cpp/arcticdb/pipeline/write_frame.cpp +++ b/cpp/arcticdb/pipeline/write_frame.cpp @@ -264,7 +264,7 @@ folly::SemiFuture>> write_slices( sparsify_floats )) .then([sink, de_dup_map](auto&& ks) { - return sink->async_write(std::forward(ks), de_dup_map); + return sink->async_write(folly::makeFuture(std::forward(ks)), de_dup_map); }); }, write_window diff --git a/cpp/arcticdb/stream/incompletes.cpp b/cpp/arcticdb/stream/incompletes.cpp index 52d8cd44a41..ff8b34fae2b 100644 --- a/cpp/arcticdb/stream/incompletes.cpp +++ b/cpp/arcticdb/stream/incompletes.cpp @@ -465,7 +465,9 @@ void do_sort(SegmentInMemory& mutable_seg, const std::vector sort_c return std::move(ks); }) .thenValue([store, de_dup_map](auto&& ks) { - return store->async_write(ks, de_dup_map); + return store->async_write( + folly::makeFuture(std::forward(ks)), de_dup_map + ); }) .thenValueInline([](SliceAndKey&& sk) { return sk.key(); }); }, diff --git a/cpp/arcticdb/stream/stream_sink.hpp b/cpp/arcticdb/stream/stream_sink.hpp index eb5e73eb39c..a93dc1288f4 100644 --- a/cpp/arcticdb/stream/stream_sink.hpp +++ b/cpp/arcticdb/stream/stream_sink.hpp @@ -108,6 +108,11 @@ struct StreamSink { const std::shared_ptr& de_dup_map ) = 0; + [[nodiscard]] virtual folly::Future async_write( + std::tuple&& input, + const std::shared_ptr& de_dup_map + ) = 0; + virtual bool is_path_valid(const std::string_view path) const = 0; [[nodiscard]] virtual folly::Future batch_write_compressed(std::vector kvs diff --git a/cpp/arcticdb/version/local_versioned_engine.cpp b/cpp/arcticdb/version/local_versioned_engine.cpp index 81259d81292..b830a37040e 100644 --- a/cpp/arcticdb/version/local_versioned_engine.cpp +++ b/cpp/arcticdb/version/local_versioned_engine.cpp @@ -884,7 +884,7 @@ VersionedItem LocalVersionedEngine::write_segment( auto frame_slice = FrameSlice{descriptor, column_slice, row_slice}; auto pkey = get_partial_key_for_segment_slice(index_desc, tsv, slice); auto ks = std::make_tuple(std::move(pkey), std::move(slice), std::move(frame_slice)); - return sink->async_write(std::move(ks), de_dup_map); + return sink->async_write(folly::makeFuture(std::move(ks)), de_dup_map); }, write_window ) From 658e2dc0dadf28fe0e8a2626e51c1d4cc10c5f4a Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Mon, 3 Nov 2025 18:09:40 +0200 Subject: [PATCH 16/46] Structure write clause by row slice --- cpp/arcticdb/processing/clause.cpp | 2 +- python/arcticdb/util/test.py | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/cpp/arcticdb/processing/clause.cpp b/cpp/arcticdb/processing/clause.cpp index 731655bf586..4fe49f28fc0 100644 --- a/cpp/arcticdb/processing/clause.cpp +++ b/cpp/arcticdb/processing/clause.cpp @@ -1663,7 +1663,7 @@ std::vector> WriteClause::structure_for_processing(std::vect std::vector> WriteClause::structure_for_processing( std::vector>&& entity_ids_vec ) { - return entity_ids_vec; + return structure_by_row_slice(*component_manager_, std::move(entity_ids_vec)); } std::vector WriteClause::process(std::vector&& entity_ids) const { diff --git a/python/arcticdb/util/test.py b/python/arcticdb/util/test.py index f49c16c9ba3..65ae19ca3ae 100644 --- a/python/arcticdb/util/test.py +++ b/python/arcticdb/util/test.py @@ -759,13 +759,13 @@ def get_query_processing_functions(lib, symbol, arctic_query, date_range=None): created symbol. Functions that test the QueryBuilder such as generic_filter_test, generic_aggregation_test, etc... can iterate on the result array and test both methods for processing. """ - qb = copy.deepcopy(arctic_query) + read_modify_write_qb = copy.deepcopy(arctic_query) test_read_modify_write = os.getenv("ARCTICDB_TEST_READ_MODIFY_WRITE", "0") == "1" - processing_functions = [lambda: lib.read(symbol, query_builder=qb).data] + processing_functions = [lambda: lib.read(symbol, query_builder=arctic_query).data] if test_read_modify_write: if date_range is not None: - qb.prepend(QueryBuilder().date_range(date_range)) - processing_functions.append(lambda: read_modify_write_data(lib, symbol, qb)) + read_modify_write_qb.prepend(QueryBuilder().date_range(date_range)) + processing_functions.append(lambda: read_modify_write_data(lib, symbol, read_modify_write_qb)) return processing_functions From 056b8d8a40ae1b042ef5b6685101d81dbea3d580 Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Mon, 3 Nov 2025 18:55:50 +0200 Subject: [PATCH 17/46] Fix resampling tests --- cpp/arcticdb/storage/test/in_memory_store.hpp | 9 +++++++++ python/arcticdb/util/test.py | 17 +++++++++-------- 2 files changed, 18 insertions(+), 8 deletions(-) diff --git a/cpp/arcticdb/storage/test/in_memory_store.hpp b/cpp/arcticdb/storage/test/in_memory_store.hpp index 33228339e8c..49a2220111f 100644 --- a/cpp/arcticdb/storage/test/in_memory_store.hpp +++ b/cpp/arcticdb/storage/test/in_memory_store.hpp @@ -364,6 +364,15 @@ class InMemoryStore : public Store { }); } + folly::Future + async_write(std::tuple&& input, const std::shared_ptr&) + override { + auto [pk, seg, slice] = std::move(input); + auto key = get_key(pk.key_type, 0, pk.stream_id, pk.start_index, pk.end_index); + add_segment(key, std::move(seg)); + return SliceAndKey{std::move(slice), std::move(key)}; + } + std::vector> batch_key_exists(const std::vector& keys) override { auto failure_sim = StorageFailureSimulator::instance(); failure_sim->go(FailureType::READ); diff --git a/python/arcticdb/util/test.py b/python/arcticdb/util/test.py index 65ae19ca3ae..6901319c782 100644 --- a/python/arcticdb/util/test.py +++ b/python/arcticdb/util/test.py @@ -761,7 +761,7 @@ def get_query_processing_functions(lib, symbol, arctic_query, date_range=None): """ read_modify_write_qb = copy.deepcopy(arctic_query) test_read_modify_write = os.getenv("ARCTICDB_TEST_READ_MODIFY_WRITE", "0") == "1" - processing_functions = [lambda: lib.read(symbol, query_builder=arctic_query).data] + processing_functions = [lambda: lib.read(symbol, query_builder=arctic_query, date_range=date_range).data] if test_read_modify_write: if date_range is not None: read_modify_write_qb.prepend(QueryBuilder().date_range(date_range)) @@ -1044,16 +1044,17 @@ def generic_resample_test( q = q.resample(rule, closed=closed, label=label, offset=offset, origin=origin).agg(aggregations) else: q = q.resample(rule, closed=closed, label=label, offset=offset).agg(aggregations) - query_processing_functions = get_query_processing_functions(lib, sym, q, date_range) + + expected = expected_pandas_resample_generic( + original_data, rule, aggregations, closed, label, offset, origin, drop_empty_buckets_for, expected_types + ) + + check_dtype = expected_types is not None + + query_processing_functions = get_query_processing_functions(lib, sym, q, date_range=date_range) for proccessing_function in query_processing_functions: received = proccessing_function() received = received.reindex(columns=sorted(received.columns)) - - expected = expected_pandas_resample_generic( - original_data, rule, aggregations, closed, label, offset, origin, drop_empty_buckets_for, expected_types - ) - - check_dtype = expected_types is not None try: assert_resampled_dataframes_are_equal(received, expected, check_dtype=check_dtype) except AssertionError: From c28903b19112fd53d27a632993eac7e3109da4b3 Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Tue, 4 Nov 2025 18:07:20 +0200 Subject: [PATCH 18/46] Address review comments --- cpp/arcticdb/async/async_store.hpp | 11 ++++------- cpp/arcticdb/pipeline/write_frame.cpp | 2 +- cpp/arcticdb/processing/clause.cpp | 8 +++++--- cpp/arcticdb/storage/test/in_memory_store.hpp | 2 +- cpp/arcticdb/stream/incompletes.cpp | 4 +--- cpp/arcticdb/stream/stream_sink.hpp | 2 +- cpp/arcticdb/version/local_versioned_engine.cpp | 2 +- python/arcticdb/util/test.py | 10 ++++------ 8 files changed, 18 insertions(+), 23 deletions(-) diff --git a/cpp/arcticdb/async/async_store.hpp b/cpp/arcticdb/async/async_store.hpp index 47749d7bc50..dcd23f71800 100644 --- a/cpp/arcticdb/async/async_store.hpp +++ b/cpp/arcticdb/async/async_store.hpp @@ -456,21 +456,18 @@ class AsyncStore : public Store { }); } - folly::Future async_write( + folly::Future compress_and_schedule_async_write( std::tuple&& input, const std::shared_ptr& de_dup_map ) override { + auto [partial_key, seg, input_slice] = std::move(input); storage::KeySegmentPair key_seg = EncodeAtomTask{ - std::get<0>(std::move(input)), - ClockType::nanos_since_epoch(), - std::get<1>(std::move(input)), - codec_, - encoding_version_ + std::move(partial_key), ClockType::nanos_since_epoch(), std::move(seg), codec_, encoding_version_ }(); DeDupLookupResult dedup_lookup = lookup_match_in_dedup_map(de_dup_map, key_seg); return folly::via( &io_executor(), - [dedup_lookup = std::move(dedup_lookup), slice = std::get<2>(std::move(input)), this]() mutable { + [dedup_lookup = std::move(dedup_lookup), slice = std::move(input_slice), this]() mutable { return util::variant_match( std::move(dedup_lookup), [&](NewObject&& obj) { diff --git a/cpp/arcticdb/pipeline/write_frame.cpp b/cpp/arcticdb/pipeline/write_frame.cpp index 26062157843..19797248919 100644 --- a/cpp/arcticdb/pipeline/write_frame.cpp +++ b/cpp/arcticdb/pipeline/write_frame.cpp @@ -264,7 +264,7 @@ folly::SemiFuture>> write_slices( sparsify_floats )) .then([sink, de_dup_map](auto&& ks) { - return sink->async_write(folly::makeFuture(std::forward(ks)), de_dup_map); + return sink->async_write(std::forward(ks), de_dup_map); }); }, write_window diff --git a/cpp/arcticdb/processing/clause.cpp b/cpp/arcticdb/processing/clause.cpp index 4fe49f28fc0..26ee832e52a 100644 --- a/cpp/arcticdb/processing/clause.cpp +++ b/cpp/arcticdb/processing/clause.cpp @@ -1683,9 +1683,11 @@ std::vector WriteClause::process(std::vector&& entity_ids) c const RowRange& row_range = *(*proc.row_ranges_)[i]; const ColRange& col_range = *(*proc.col_ranges_)[i]; stream::PartialKey partial_key = create_partial_key(segment); - data_segments_to_write.push_back(std::make_shared>(store_->async_write( - std::make_tuple(std::move(partial_key), segment, FrameSlice(col_range, row_range)), dedup_map_ - ))); + data_segments_to_write.push_back( + std::make_shared>(store_->compress_and_schedule_async_write( + std::make_tuple(std::move(partial_key), segment, FrameSlice(col_range, row_range)), dedup_map_ + )) + ); } return component_manager_->add_entities(std::move(data_segments_to_write)); } diff --git a/cpp/arcticdb/storage/test/in_memory_store.hpp b/cpp/arcticdb/storage/test/in_memory_store.hpp index 49a2220111f..e4310c07b98 100644 --- a/cpp/arcticdb/storage/test/in_memory_store.hpp +++ b/cpp/arcticdb/storage/test/in_memory_store.hpp @@ -365,7 +365,7 @@ class InMemoryStore : public Store { } folly::Future - async_write(std::tuple&& input, const std::shared_ptr&) + compress_and_schedule_async_write(std::tuple&& input, const std::shared_ptr&) override { auto [pk, seg, slice] = std::move(input); auto key = get_key(pk.key_type, 0, pk.stream_id, pk.start_index, pk.end_index); diff --git a/cpp/arcticdb/stream/incompletes.cpp b/cpp/arcticdb/stream/incompletes.cpp index ff8b34fae2b..e5273eb4a7d 100644 --- a/cpp/arcticdb/stream/incompletes.cpp +++ b/cpp/arcticdb/stream/incompletes.cpp @@ -465,9 +465,7 @@ void do_sort(SegmentInMemory& mutable_seg, const std::vector sort_c return std::move(ks); }) .thenValue([store, de_dup_map](auto&& ks) { - return store->async_write( - folly::makeFuture(std::forward(ks)), de_dup_map - ); + return store->async_write(std::forward(ks), de_dup_map); }) .thenValueInline([](SliceAndKey&& sk) { return sk.key(); }); }, diff --git a/cpp/arcticdb/stream/stream_sink.hpp b/cpp/arcticdb/stream/stream_sink.hpp index a93dc1288f4..67b7aba33f4 100644 --- a/cpp/arcticdb/stream/stream_sink.hpp +++ b/cpp/arcticdb/stream/stream_sink.hpp @@ -108,7 +108,7 @@ struct StreamSink { const std::shared_ptr& de_dup_map ) = 0; - [[nodiscard]] virtual folly::Future async_write( + [[nodiscard]] virtual folly::Future compress_and_schedule_async_write( std::tuple&& input, const std::shared_ptr& de_dup_map ) = 0; diff --git a/cpp/arcticdb/version/local_versioned_engine.cpp b/cpp/arcticdb/version/local_versioned_engine.cpp index b830a37040e..81259d81292 100644 --- a/cpp/arcticdb/version/local_versioned_engine.cpp +++ b/cpp/arcticdb/version/local_versioned_engine.cpp @@ -884,7 +884,7 @@ VersionedItem LocalVersionedEngine::write_segment( auto frame_slice = FrameSlice{descriptor, column_slice, row_slice}; auto pkey = get_partial_key_for_segment_slice(index_desc, tsv, slice); auto ks = std::make_tuple(std::move(pkey), std::move(slice), std::move(frame_slice)); - return sink->async_write(folly::makeFuture(std::move(ks)), de_dup_map); + return sink->async_write(std::move(ks), de_dup_map); }, write_window ) diff --git a/python/arcticdb/util/test.py b/python/arcticdb/util/test.py index 6901319c782..9cfab1900b4 100644 --- a/python/arcticdb/util/test.py +++ b/python/arcticdb/util/test.py @@ -13,6 +13,7 @@ import numpy as np import pandas as pd from pandas import DateOffset, Timedelta + from pandas._typing import Scalar import datetime as dt import string @@ -746,9 +747,9 @@ def random_seed_context(): FIXED_STRINGS_SUFFIX = "fixed_strings" -def read_modify_write_data(lib, symbol, query): +def read_modify_write_data(lib, symbol, query, date_range=None): dest_symbol = f"{symbol}_modified" - lib._read_modify_write(symbol, query, dest_symbol) + lib._read_modify_write(symbol, query, dest_symbol, date_range=date_range) return lib.read(dest_symbol).data @@ -759,13 +760,10 @@ def get_query_processing_functions(lib, symbol, arctic_query, date_range=None): created symbol. Functions that test the QueryBuilder such as generic_filter_test, generic_aggregation_test, etc... can iterate on the result array and test both methods for processing. """ - read_modify_write_qb = copy.deepcopy(arctic_query) test_read_modify_write = os.getenv("ARCTICDB_TEST_READ_MODIFY_WRITE", "0") == "1" processing_functions = [lambda: lib.read(symbol, query_builder=arctic_query, date_range=date_range).data] if test_read_modify_write: - if date_range is not None: - read_modify_write_qb.prepend(QueryBuilder().date_range(date_range)) - processing_functions.append(lambda: read_modify_write_data(lib, symbol, read_modify_write_qb)) + processing_functions.append(lambda: read_modify_write_data(lib, symbol, arctic_query, date_range=date_range)) return processing_functions From 6c9438afbf1b2bc8893728dfb4ea928fb4f8e006 Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Wed, 5 Nov 2025 11:19:58 +0200 Subject: [PATCH 19/46] Extract the variant match in async write to a separate function --- cpp/arcticdb/async/async_store.hpp | 33 ++++++++++------------- cpp/arcticdb/storage/key_segment_pair.hpp | 14 ++++++++-- 2 files changed, 26 insertions(+), 21 deletions(-) diff --git a/cpp/arcticdb/async/async_store.hpp b/cpp/arcticdb/async/async_store.hpp index dcd23f71800..a6f3ba62ff1 100644 --- a/cpp/arcticdb/async/async_store.hpp +++ b/cpp/arcticdb/async/async_store.hpp @@ -426,6 +426,18 @@ class AsyncStore : public Store { return res; } + SliceAndKey write_if_new(std::pair&& item) { + auto [dedup_lookup, slice] = std::move(item); + return util::variant_match( + std::move(dedup_lookup), + [&, this](NewObject&& obj) { + library_->write(obj); + return SliceAndKey{std::move(slice), std::move(obj).atom_key()}; + }, + [&](ExistingObject&& obj) { return SliceAndKey{std::move(slice), to_atom(std::move(obj))}; } + ); + } + folly::Future async_write( folly::Future>&& input_fut, const std::shared_ptr& de_dup_map @@ -443,17 +455,7 @@ class AsyncStore : public Store { return std::pair{lookup_match_in_dedup_map(de_dup_map, key_seg), std::move(slice)}; }) .via(&async::io_executor()) - .thenValue([lib = library_](auto&& item) { - auto& [dedup_lookup, slice] = item; - return util::variant_match( - dedup_lookup, - [&](NewObject& obj) { - lib->write(obj); - return SliceAndKey{std::move(slice), obj.atom_key()}; - }, - [&](ExistingObject& obj) { return SliceAndKey{std::move(slice), to_atom(std::move(obj))}; } - ); - }); + .thenValue([this](auto&& item) { return write_if_new(std::move(item)); }); } folly::Future compress_and_schedule_async_write( @@ -468,14 +470,7 @@ class AsyncStore : public Store { return folly::via( &io_executor(), [dedup_lookup = std::move(dedup_lookup), slice = std::move(input_slice), this]() mutable { - return util::variant_match( - std::move(dedup_lookup), - [&](NewObject&& obj) { - library_->write(obj); - return SliceAndKey{std::move(slice), obj.atom_key()}; - }, - [&](ExistingObject&& obj) { return SliceAndKey{slice, to_atom(std::move(obj))}; } - ); + return write_if_new(std::pair{std::move(dedup_lookup), std::move(slice)}); } ); } diff --git a/cpp/arcticdb/storage/key_segment_pair.hpp b/cpp/arcticdb/storage/key_segment_pair.hpp index 0e60c8b48ae..3bf6aaace8f 100644 --- a/cpp/arcticdb/storage/key_segment_pair.hpp +++ b/cpp/arcticdb/storage/key_segment_pair.hpp @@ -43,21 +43,31 @@ class KeySegmentPair { key_ = std::make_shared(std::forward(key)); } - [[nodiscard]] const AtomKey& atom_key() const { + [[nodiscard]] const AtomKey& atom_key() const& { util::check(std::holds_alternative(variant_key()), "Expected atom key access"); return std::get(variant_key()); } + [[nodiscard]] AtomKey&& atom_key() && { + util::check(std::holds_alternative(variant_key()), "Expected atom key access"); + return std::get(std::move(*key_)); + } + [[nodiscard]] const RefKey& ref_key() const { util::check(std::holds_alternative(variant_key()), "Expected ref key access"); return std::get(variant_key()); } - [[nodiscard]] const VariantKey& variant_key() const { + [[nodiscard]] const VariantKey& variant_key() const& { util::check(key_, "Attempting to access key_ but it has not been set"); return *key_; } + [[nodiscard]] VariantKey&& variant_key() && { + util::check(key_, "Attempting to access key_ but it has not been set"); + return std::move(*key_); + } + [[nodiscard]] const Segment& segment() const { util::check(segment_, "Attempting to access segment_ (const) but it has not been set"); return *segment_; From 809d9612a2147186a0cf05699c21c8611d10b7f2 Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Mon, 3 Nov 2025 18:55:50 +0200 Subject: [PATCH 20/46] Fix resampling tests --- python/arcticdb/util/test.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/python/arcticdb/util/test.py b/python/arcticdb/util/test.py index 9cfab1900b4..c3bdad53ff5 100644 --- a/python/arcticdb/util/test.py +++ b/python/arcticdb/util/test.py @@ -1085,6 +1085,13 @@ def generic_resample_test( assert_resampled_dataframes_are_equal(received, expected, check_dtype=check_dtype) else: raise + original_data = original_data.tail(len(original_data) - rows_to_pop) + expected = expected_pandas_resample_generic( + original_data, rule, aggregations, closed, label, offset, origin, drop_empty_buckets_for, expected_types + ) + assert_resampled_dataframes_are_equal(received, expected, check_dtype=check_dtype) + else: + raise def equals(x, y): From 6615c97941bf4966f96cc48d2ca3d0af212f2bf0 Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Mon, 3 Nov 2025 14:52:22 +0200 Subject: [PATCH 21/46] Prepare for merge update --- .../version/local_versioned_engine.cpp | 25 + .../version/local_versioned_engine.hpp | 5 + cpp/arcticdb/version/merge_options.hpp | 18 + cpp/arcticdb/version/python_bindings.cpp | 4 + cpp/arcticdb/version/version_core.cpp | 9 + cpp/arcticdb/version/version_core.hpp | 7 + cpp/arcticdb/version/version_store_api.cpp | 19 + cpp/arcticdb/version/version_store_api.hpp | 6 + cpp/arcticdb/version/versioned_engine.hpp | 6 + .../unit/arcticdb/version_store/test_merge.py | 1147 +++++++++++++++++ 10 files changed, 1246 insertions(+) create mode 100644 cpp/arcticdb/version/merge_options.hpp create mode 100644 python/tests/unit/arcticdb/version_store/test_merge.py diff --git a/cpp/arcticdb/version/local_versioned_engine.cpp b/cpp/arcticdb/version/local_versioned_engine.cpp index 81259d81292..355f8487862 100644 --- a/cpp/arcticdb/version/local_versioned_engine.cpp +++ b/cpp/arcticdb/version/local_versioned_engine.cpp @@ -2278,4 +2278,29 @@ std::shared_ptr LocalVersionedEngine::_test_get_version_map() { retu void LocalVersionedEngine::_test_set_store(std::shared_ptr store) { set_store(std::move(store)); } +VersionedItem LocalVersionedEngine::merge_internal( + const StreamId& stream_id, const std::shared_ptr& source, bool prune_previous_versions, + const MergeStrategy& strategy, std::span on, bool match_on_timeseries_index +) { + ARCTICDB_RUNTIME_DEBUG(log::version(), "Command: merge"); + py::gil_scoped_release release_gil; + const UpdateInfo update_info = get_latest_undeleted_version_and_next_version_id(store(), version_map(), stream_id); + if (update_info.previous_index_key_.has_value()) { + if (source->empty()) { + ARCTICDB_DEBUG( + log::version(), + "Merging into existing data with an empty source has no effect. \n No new version is being created " + "for symbol='{}', and the last version is returned", + stream_id + ); + return VersionedItem{*std::move(update_info.previous_index_key_)}; + } + auto versioned_item = + merge_impl(store(), source, update_info, get_write_options(), strategy, on, match_on_timeseries_index); + write_version_and_prune_previous(prune_previous_versions, versioned_item.key_, update_info.previous_index_key_); + return versioned_item; + } + storage::raise("Cannot merge into non-existent symbol \"{}\".", stream_id); +} + } // namespace arcticdb::version_store diff --git a/cpp/arcticdb/version/local_versioned_engine.hpp b/cpp/arcticdb/version/local_versioned_engine.hpp index 45da344afac..ceb3032aefd 100644 --- a/cpp/arcticdb/version/local_versioned_engine.hpp +++ b/cpp/arcticdb/version/local_versioned_engine.hpp @@ -352,6 +352,11 @@ class LocalVersionedEngine : public VersionedEngine { const arcticdb::proto::storage::VersionStoreConfig& cfg() const override { return cfg_; } + VersionedItem merge_internal( + const StreamId& stream_id, const std::shared_ptr& source, bool prune_previous_versions, + const MergeStrategy& strategy, std::span on, bool match_on_timeseries_index + ) override; + protected: template explicit LocalVersionedEngine(const std::shared_ptr& store, const ClockType& = ClockType{}); diff --git a/cpp/arcticdb/version/merge_options.hpp b/cpp/arcticdb/version/merge_options.hpp new file mode 100644 index 00000000000..54e9e2e9f9a --- /dev/null +++ b/cpp/arcticdb/version/merge_options.hpp @@ -0,0 +1,18 @@ +/* Copyright 2025 Man Group Operations Limited + * + * Use of this software is governed by the Business Source License 1.1 included in the file licenses/BSL.txt. + * + * As of the Change Date specified in that file, in accordance with the Business Source License, use of this software + * will be governed by the Apache License, version 2.0. + */ +#pragma once +#include + +namespace arcticdb { +enum class MergeAction : uint8_t { DO_NOTHING, UPDATE, INSERT }; +struct MergeStrategy { + MergeAction matched; + MergeAction not_matched_by_target; +}; + +} // namespace arcticdb \ No newline at end of file diff --git a/cpp/arcticdb/version/python_bindings.cpp b/cpp/arcticdb/version/python_bindings.cpp index 060282ec26f..b0574c0ec70 100644 --- a/cpp/arcticdb/version/python_bindings.cpp +++ b/cpp/arcticdb/version/python_bindings.cpp @@ -633,6 +633,10 @@ void register_bindings(py::module& version, py::exception(), "Append a dataframe to the most recent version") + .def("merge", + &PythonVersionStore::merge, + py::call_guard(), + "Merge a dataframe into the most recent version") .def("append_incomplete", &PythonVersionStore::append_incomplete, py::call_guard(), diff --git a/cpp/arcticdb/version/version_core.cpp b/cpp/arcticdb/version/version_core.cpp index d9631dd408e..8e0483c98a1 100644 --- a/cpp/arcticdb/version/version_core.cpp +++ b/cpp/arcticdb/version/version_core.cpp @@ -2795,6 +2795,15 @@ VersionedItem read_modify_write_impl( .get(); } +VersionedItem merge_impl( + [[maybe_unused]] const std::shared_ptr& store, [[maybe_unused]] const std::shared_ptr& frame, + [[maybe_unused]] const UpdateInfo& update_info, [[maybe_unused]] WriteOptions&& options, + [[maybe_unused]] const MergeStrategy& strategy, [[maybe_unused]] std::span on, + [[maybe_unused]] bool match_on_timeseries_index +) { + return VersionedItem{}; +} + folly::Future read_and_process( const std::shared_ptr& store, const std::variant& version_info, const std::shared_ptr& read_query, const ReadOptions& read_options, diff --git a/cpp/arcticdb/version/version_core.hpp b/cpp/arcticdb/version/version_core.hpp index 608aae7191a..d3ede8f9cb2 100644 --- a/cpp/arcticdb/version/version_core.hpp +++ b/cpp/arcticdb/version/version_core.hpp @@ -21,6 +21,7 @@ #include #include #include +#include #include namespace arcticdb::version_store { @@ -220,6 +221,12 @@ VersionedItem read_modify_write_impl( const WriteOptions& write_options, const IndexPartialKey& target_partial_index_key ); +VersionedItem merge_impl( + const std::shared_ptr& store, const std::shared_ptr& frame, const UpdateInfo& update_info, + WriteOptions&& options, const MergeStrategy& strategy, std::span on, + bool match_on_timeseries_index +); + } // namespace arcticdb::version_store namespace arcticdb { diff --git a/cpp/arcticdb/version/version_store_api.cpp b/cpp/arcticdb/version/version_store_api.cpp index b05db548ff7..0e71aeed4d0 100644 --- a/cpp/arcticdb/version/version_store_api.cpp +++ b/cpp/arcticdb/version/version_store_api.cpp @@ -1332,4 +1332,23 @@ void PythonVersionStore::force_delete_symbol(const StreamId& stream_id) { delete_all_for_stream(store(), stream_id, true); version_map()->flush(); } + +VersionedItem PythonVersionStore::merge( + const StreamId& stream_id, const py::tuple& source, const py::object& norm, const py::object& user_meta, + const bool prune_previous_versions, const py::tuple& py_strategy, const std::vector& on, + const bool match_on_timeseries_index +) { + const MergeStrategy strategy{ + .matched = py_strategy[0].cast(), .not_matched_by_target = py_strategy[1].cast() + }; + return merge_internal( + stream_id, + convert::py_ndf_to_frame(stream_id, source, norm, user_meta, cfg().write_options().empty_types()), + prune_previous_versions, + strategy, + on, + match_on_timeseries_index + ); +} + } // namespace arcticdb::version_store diff --git a/cpp/arcticdb/version/version_store_api.hpp b/cpp/arcticdb/version/version_store_api.hpp index 85f75a4c5f0..95805e4f404 100644 --- a/cpp/arcticdb/version/version_store_api.hpp +++ b/cpp/arcticdb/version/version_store_api.hpp @@ -283,6 +283,12 @@ class PythonVersionStore : public LocalVersionedEngine { std::vector get_version_history(const StreamId& stream_id); + VersionedItem merge( + const StreamId& stream_id, const py::tuple& source, const py::object& norm, const py::object& user_meta, + const bool prune_previous_versions, const py::tuple& py_strategy, const std::vector& on, + const bool match_on_timeseries_index + ); + private: void delete_snapshot_sync(const SnapshotId& snap_name, const VariantKey& snap_key); }; diff --git a/cpp/arcticdb/version/versioned_engine.hpp b/cpp/arcticdb/version/versioned_engine.hpp index 8dede4eff3c..1500918f27c 100644 --- a/cpp/arcticdb/version/versioned_engine.hpp +++ b/cpp/arcticdb/version/versioned_engine.hpp @@ -20,6 +20,7 @@ #include #include #include +#include namespace arcticdb::version_store { @@ -144,6 +145,11 @@ class VersionedEngine { virtual void set_store(std::shared_ptr store) = 0; virtual timestamp latest_timestamp(const std::string& symbol) = 0; virtual void flush_version_map() = 0; + + virtual VersionedItem merge_internal( + const StreamId& stream_id, const std::shared_ptr& source, bool prune_previous_versions, + const MergeStrategy& strategy, std::span on, bool match_on_timeseries_index + ) = 0; }; } // namespace arcticdb::version_store diff --git a/python/tests/unit/arcticdb/version_store/test_merge.py b/python/tests/unit/arcticdb/version_store/test_merge.py new file mode 100644 index 00000000000..37e3895a81d --- /dev/null +++ b/python/tests/unit/arcticdb/version_store/test_merge.py @@ -0,0 +1,1147 @@ +""" +Copyright 2025 Man Group Operations Limited + +Use of this software is governed by the Business Source License 1.1 included in the file licenses/BSL.txt. + +As of the Change Date specified in that file, in accordance with the Business Source License, use of this software will be governed by the Apache License, version 2.0. +""" + +import pytest +import pandas as pd +from arcticdb.util.test import assert_frame_equal, assert_vit_equals_except_data +import arcticdb +from arcticdb.version_store import VersionedItem +from arcticdb_ext.storage import KeyType +import numpy as np +from arcticdb.exceptions import StreamDescriptorMismatch, UserInputException, SortingException, StorageException +from arcticdb.version_store.library import MergeAction, MergeStrategy + + +def mock_find_keys_for_symbol(key_types): + keys = {kt: [f"{kt}_{i}" for i in range(key_types[kt])] for kt in key_types} + return lambda key_type, symbol: keys[key_type] + + +def raise_wrapper(exception, message=None): + def _raise(*args, **kwargs): + raise exception(message) + + return _raise + + +class TestMergeTimeseries: + + # ================================================================================================ + # ================================= APPLIES TO ALL STRATEGIES ==================================== + # ================================================================================================ + @pytest.mark.parametrize( + "strategy", + ( + MergeStrategy(MergeAction.UPDATE, MergeAction.DO_NOTHING), + pytest.param( + MergeStrategy(MergeAction.DO_NOTHING, MergeAction.INSERT), + marks=pytest.mark.xfail(reason="Insert is not implemented"), + ), + pytest.param( + MergeStrategy(MergeAction.UPDATE, MergeAction.INSERT), + marks=pytest.mark.xfail(reason="Insert is not implemented"), + ), + ), + ) + def test_merge_matched_update_with_metadata(self, lmdb_library, strategy): + lib = lmdb_library + + target = pd.DataFrame({"a": [1, 2, 3], "b": [1.0, 2.0, 3.0]}, index=pd.date_range("2024-01-01", periods=3)) + write_vit = lib.write("sym", target) + + source = pd.DataFrame( + {"a": [1, 20, 3], "b": [1.0, 20.0, 3.0]}, + index=pd.DatetimeIndex(["2024-01-01 10:00:00", "2024-01-02", "2024-01-04"]), + ) + + metadata = {"meta": "data"} + + merge_vit = lib.merge("sym", source, metadata=metadata, strategy=strategy) + assert merge_vit.version == 1 + assert merge_vit.symbol == write_vit.symbol + assert merge_vit.timestamp > write_vit.timestamp + assert merge_vit.metadata == metadata + assert merge_vit.library == write_vit.library + assert merge_vit.host == write_vit.host + assert merge_vit.data is None + + read_vit = lib.read("sym") + assert_vit_equals_except_data(merge_vit, read_vit) + + lt = lib._dev_tools.library_tool() + + assert len(lt.find_keys_for_symbol(KeyType.TABLE_DATA, "sym")) == 2 + assert len(lt.find_keys_for_symbol(KeyType.TABLE_INDEX, "sym")) == 2 + assert len(lt.find_keys_for_symbol(KeyType.VERSION, "sym")) == 2 + + @pytest.mark.parametrize("metadata", ({"meta": "data"}, None)) + @pytest.mark.parametrize( + "strategy", + ( + MergeStrategy(MergeAction.UPDATE, MergeAction.DO_NOTHING), + pytest.param( + MergeStrategy(MergeAction.DO_NOTHING, MergeAction.INSERT), + marks=pytest.mark.xfail(reason="Insert is not implemented"), + ), + pytest.param( + MergeStrategy(MergeAction.UPDATE, MergeAction.INSERT), + marks=pytest.mark.xfail(reason="Insert is not implemented"), + ), + ), + ) + def test_merge_does_not_write_new_version_with_empty_source(self, lmdb_library, metadata, strategy): + lib = lmdb_library + target = pd.DataFrame({"a": [1, 2, 3], "b": [1.0, 2.0, 3.0]}, index=pd.date_range("2024-01-01", periods=3)) + write_vit = lib.write("sym", target) + merge_vit = lib.merge("sym", pd.DataFrame(), metadata=metadata, strategy=strategy) + # There's a bug in append, update, and merge when there's an empty source. All of them return the passed + # metadata even though it's not used. + merge_vit.metadata = write_vit.metadata + assert_vit_equals_except_data(write_vit, merge_vit) + assert merge_vit.data is None and write_vit.data is None + lt = lib._dev_tools.library_tool() + assert len(lt.find_keys_for_symbol(KeyType.TABLE_DATA, "sym")) == 1 + assert len(lt.find_keys_for_symbol(KeyType.TABLE_INDEX, "sym")) == 1 + assert len(lt.find_keys_for_symbol(KeyType.VERSION, "sym")) == 1 + + @pytest.mark.parametrize( + "strategy", + ( + pytest.param(None, marks=pytest.mark.xfail(reason="Insert is not implemented")), + MergeStrategy(MergeAction.UPDATE, MergeAction.DO_NOTHING), + pytest.param( + MergeStrategy(MergeAction.DO_NOTHING, MergeAction.INSERT), + marks=pytest.mark.xfail(reason="Insert is not implemented"), + ), + pytest.param( + MergeStrategy(MergeAction.UPDATE, MergeAction.INSERT), + marks=pytest.mark.xfail(reason="Insert is not implemented"), + ), + ), + ) + @pytest.mark.parametrize( + "source", + [ + pd.DataFrame( + {"a": np.array([1, 2, 3], dtype=np.int8), "b": [1.0, 2.0, 3.0]}, + index=pd.DatetimeIndex( + [pd.Timestamp("2024-01-01"), pd.Timestamp("2024-01-01 10:00:00"), pd.Timestamp("2024-01-02")] + ), + ), + pd.DataFrame( + {"c": np.array([1, 2, 3], dtype=np.int8), "b": [1.0, 2.0, 3.0]}, + index=pd.DatetimeIndex( + [pd.Timestamp("2024-01-01"), pd.Timestamp("2024-01-01 10:00:00"), pd.Timestamp("2024-01-02")] + ), + ), + pd.DataFrame( + {"b": [1.0, 2.0, 3.0]}, + index=pd.DatetimeIndex( + [pd.Timestamp("2024-01-01"), pd.Timestamp("2024-01-01 10:00:00"), pd.Timestamp("2024-01-02")] + ), + ), + ], + ) + def test_static_schema_merge_throws_when_schemas_differ(self, lmdb_library, strategy, source): + lib = lmdb_library + target = pd.DataFrame({"a": [1, 2, 3], "b": [1.0, 2.0, 3.0]}, index=pd.date_range("2024-01-01", periods=3)) + lib.write("sym", target) + with pytest.raises(StreamDescriptorMismatch): + lib.merge("sym", source, strategy=strategy) + + @pytest.mark.parametrize( + "strategy", + ( + MergeStrategy(MergeAction.UPDATE, MergeAction.DO_NOTHING), + pytest.param( + MergeStrategy(MergeAction.DO_NOTHING, MergeAction.INSERT), + marks=pytest.mark.xfail(reason="Insert is not implemented"), + ), + pytest.param( + MergeStrategy(MergeAction.UPDATE, MergeAction.INSERT), + marks=pytest.mark.xfail(reason="Insert is not implemented"), + ), + ), + ) + def test_throws_if_source_is_not_sorted(self, lmdb_library, strategy): + # This requirement can be lifted, however, passing a sorted source will be faster. We can start with it and + # extend if needed. + lib = lmdb_library + target = pd.DataFrame({"a": [1, 2, 3], "b": [1.0, 2.0, 3.0]}, index=pd.date_range("2024-01-01", periods=3)) + lib.write("sym", target) + + source = pd.DataFrame( + {"a": [10, 20, 30], "b": [10.1, 20.1, 30.1]}, + index=pd.DatetimeIndex( + [pd.Timestamp("2024-01-02"), pd.Timestamp("2024-01-03"), pd.Timestamp("2024-01-01")] + ), + ) + + with pytest.raises(UserInputException): + lib.merge("sym", source, strategy=strategy) + + # ================================================================================================ + # ==================================== TEST UPDATE ON MATCH ====================================== + # ================================================================================================ + @pytest.mark.parametrize( + "strategy", + ( + MergeStrategy(MergeAction.UPDATE, MergeAction.DO_NOTHING), + pytest.param(MergeStrategy(not_matched_by_target=MergeAction.DO_NOTHING), marks=pytest.mark.skip), + pytest.param(MergeStrategy(not_matched_by_target="do_nothing"), marks=pytest.mark.skip), + pytest.param(MergeStrategy("update", "do_nothing"), marks=pytest.mark.skip), + pytest.param(MergeStrategy("UPDATE", "DO_NOTHING"), marks=pytest.mark.skip), + pytest.param(MergeStrategy(MergeAction.UPDATE, "do_nothing"), marks=pytest.mark.skip), + pytest.param(MergeStrategy("update", MergeAction.DO_NOTHING), marks=pytest.mark.skip), + ), + ) + def test_merge_update(self, lmdb_library, strategy): + lib = lmdb_library + + target = pd.DataFrame({"a": [1, 2, 3], "b": [1.0, 2.0, 3.0]}, index=pd.date_range("2024-01-01", periods=3)) + write_vit = lib.write("sym", target) + + source = pd.DataFrame( + {"a": [4, 5, 6], "b": [7.0, 8.0, 9.0]}, + # Only the second row: "2024-01-02" matches + index=pd.DatetimeIndex(["2024-01-01 10:00:00", "2024-01-02", "2024-01-04"]), + ) + + merge_vit = lib.merge("sym", source, strategy=strategy) + assert merge_vit.version == 1 + assert merge_vit.symbol == write_vit.symbol + assert merge_vit.timestamp > write_vit.timestamp + assert merge_vit.metadata == write_vit.metadata + assert merge_vit.library == write_vit.library + assert merge_vit.host == write_vit.host + assert merge_vit.data is None + + # Only the second row: "2024-01-02" is updated + expected = pd.DataFrame({"a": [1, 5, 3], "b": [1.0, 8.0, 3.0]}, index=pd.date_range("2024-01-01", periods=3)) + + read_vit = lib.read("sym") + assert_vit_equals_except_data(merge_vit, read_vit) + assert_frame_equal(read_vit.data, expected) + + lt = lib._dev_tools.library_tool() + + assert len(lt.find_keys_for_symbol(KeyType.TABLE_DATA, "sym")) == 2 + assert len(lt.find_keys_for_symbol(KeyType.TABLE_INDEX, "sym")) == 2 + assert len(lt.find_keys_for_symbol(KeyType.VERSION, "sym")) == 2 + + def test_merge_update_writes_new_version_even_if_nothing_is_changed(self, lmdb_library): + # In theory, it's possible to make so that it doesn't write a new version when nothing is matched, but the source + # is not empty. This has lots of edge cases and will burden the implementation for almost no gain. If nothing is + # changed, we'll keep the same index and data keys and just write a new version key which is cheap. + lib = lmdb_library + target = pd.DataFrame({"a": [1, 2, 3], "b": [1.0, 2.0, 3.0]}, index=pd.date_range("2024-01-01", periods=3)) + write_vit = lib.write("sym", target) + + source = pd.DataFrame({"a": [4, 5], "b": [4.0, 5.0]}, index=pd.date_range("2023-01-01", periods=2)) + merge_vit = lib.merge("sym", source, strategy=MergeStrategy(not_matched_by_target=MergeAction.DO_NOTHING)) + assert merge_vit.version == 1 + assert merge_vit.timestamp > write_vit.timestamp + + read_vit = lib.read("sym") + assert_vit_equals_except_data(merge_vit, read_vit) + assert_frame_equal(read_vit.data, target) + + lt = lib._dev_tools.library_tool() + + assert len(lt.find_keys_for_symbol(KeyType.TABLE_DATA, "sym")) == 1 + assert len(lt.find_keys_for_symbol(KeyType.TABLE_INDEX, "sym")) == 2 + assert len(lt.find_keys_for_symbol(KeyType.VERSION, "sym")) == 2 + + @pytest.mark.parametrize( + "slicing_policy", + [{"rows_per_segment": 2}, {"columns_per_segment": 2}, {"rows_per_segment": 2, "columns_per_segment": 2}], + ) + def test_merge_update_row_slicing(self, lmdb_library_factory, slicing_policy): + lib = lmdb_library_factory(arcticdb.LibraryOptions(**slicing_policy)) + target = pd.DataFrame( + {"a": [1, 2, 3, 4, 5], "b": [1.0, 2.0, 3.0, 4.0, 5.0], "c": ["a", "b", "c", "d", "e"]}, + index=pd.date_range("2024-01-01", periods=5), + ) + lib.write("sym", target) + + source = pd.DataFrame( + {"a": [30, 50], "b": [30.1, 50.1], "c": ["C", "E"]}, + index=pd.DatetimeIndex([pd.Timestamp("2024-01-03"), pd.Timestamp("2024-01-05")]), + ) + lib.merge("sym", source, strategy=MergeStrategy(not_matched_by_target=MergeAction.DO_NOTHING)) + + expected = pd.DataFrame( + {"a": [1, 2, 30, 4, 50], "b": [1.0, 2.0, 30.1, 4.0, 50.1], "c": ["a", "b", "C", "d", "E"]}, + index=pd.date_range("2024-01-01", periods=5), + ) + received = lib.read("sym").data + assert_frame_equal(received, expected) + + lt = lib._dev_tools.library_tool() + if "rows_per_segment" in slicing_policy and "columns_per_segment" in slicing_policy: + # Start with 3 row slices and 2 column slices = 6 data keys + # The second segment is overwritten with column slicing = 2 data keys + # The third segment is overwritten with column slicing = 2 data keys + expected_data_keys = 10 + elif "rows_per_segment" in slicing_policy: + # Start with 3 row slices and no column slices -> 3 data keys + # The second segment is overwritten with column slicing = 1 data key + # The third segment is overwritten with column slicing = 1 data key + expected_data_keys = 5 + elif "columns_per_segment" in slicing_policy: + # Start with one row slice and 2 column slices -> 2 data keys + # The segment is overwritten with column slicing = 2 data key + expected_data_keys = 4 + assert len(lt.find_keys_for_symbol(KeyType.TABLE_DATA, "sym")) == expected_data_keys + assert len(lt.find_keys_for_symbol(KeyType.TABLE_INDEX, "sym")) == 2 + assert len(lt.find_keys_for_symbol(KeyType.VERSION, "sym")) == 2 + + def test_merge_update_on_index_and_column(self, lmdb_library, monkeypatch): + lib = lmdb_library + target = pd.DataFrame({"a": [1, 2, 3], "b": [1.0, 2.0, 3.0]}, index=pd.date_range("2024-01-01", periods=3)) + lib.write("sym", target) + + source = pd.DataFrame( + {"a": [1, 2, 30, 40], "b": [10.0, 20.0, 30.0, 40.0]}, + index=pd.DatetimeIndex( + [ + "2024-01-01", # Matches index and column + "2024-01-02 01:00:00", # Matches column, but not index + "2024-01-03", # Matches index, but not column + "2024-01-04", # Does not match either + ] + ), + ) + monkeypatch.setattr(lib.__class__, "merge", lambda *args, **kwargs: None, raising=False) + lib.merge("sym", source, on=["a"], strategy=MergeStrategy(not_matched_by_target=MergeAction.DO_NOTHING)) + + expected = pd.DataFrame({"a": [1, 2, 3], "b": [10.0, 2.0, 3.0]}, index=pd.date_range("2024-01-01", periods=3)) + monkeypatch.setattr(lib, "read", lambda *args, **kwargs: VersionedItem("sym", "lib", expected, 2)) + received = lib.read("sym").data + + assert_frame_equal(received, expected) + + def test_merge_update_on_multiple_columns(self, lmdb_library, monkeypatch): + lib = lmdb_library + target = pd.DataFrame( + { + "a": [1, 2, 3, 4], + "b": ["a", "b", "c", "d"], + "c": ["A", "B", "A", "C"], + "d": [10.1, 20.2, 30.3, 40.4], + "e": [100, 200, 300, 400], + }, + index=pd.date_range("2024-01-01", periods=4), + ) + lib.write("sym", target) + + source = pd.DataFrame( + { + "a": [10, 20, 30, 40, 50], + "b": ["a", "b", "c", "d", "e"], + "c": ["A", "D", "A", "B", "C"], + "d": [10.1, 50.5, 30.3, 40.4, 70.7], + "e": [100, 500, 600, 400, 800], + }, + index=pd.DatetimeIndex( + [ + "2024-01-01", # index + b + d + e match (should update) + "2024-01-02", # index + b match, but d + e differ (do nothing) + "2024-01-03", # index + b + d match, e differs (do nothing) + "2024-01-05", # new index, b+d+e match (do nothing) + "2024-01-06", # new index, new b, new d, new e (do nothing) + ] + ), + ) + + monkeypatch.setattr(lib.__class__, "merge", lambda *args, **kwargs: None, raising=False) + lib.merge( + "sym", source, on=["b", "d", "e"], strategy=MergeStrategy(not_matched_by_target=MergeAction.DO_NOTHING) + ) + + expected = pd.DataFrame( + { + "a": [10, 2, 3, 4], + "b": ["a", "b", "c", "d"], + "c": ["A", "B", "A", "C"], + "d": [10.1, 20.2, 30.3, 40.4], + "e": [100, 200, 300, 400], + }, + index=pd.date_range("2024-01-01", periods=4), + ) + + monkeypatch.setattr(lib, "read", lambda *args, **kwargs: VersionedItem("sym", "lib", expected, 2)) + received = lib.read("sym").data + assert_frame_equal(received, expected) + + def test_merge_update_row_from_source_matches_multiple_rows_from_target(self, lmdb_library): + lib = lmdb_library + target = pd.DataFrame( + {"a": [1, 2, 3], "b": [1.0, 2.0, 3.0]}, + index=pd.DatetimeIndex( + [pd.Timestamp("2024-01-01"), pd.Timestamp("2024-01-01"), pd.Timestamp("2024-01-02")] + ), + ) + lib.write("sym", target) + source = pd.DataFrame({"a": [5], "b": [20.0]}, index=pd.DatetimeIndex([pd.Timestamp("2024-01-01")])) + lib.merge("sym", source, strategy=MergeStrategy(not_matched_by_target=MergeAction.DO_NOTHING)) + expected = pd.DataFrame( + {"a": [5, 5, 3], "b": [20.0, 20.0, 3.0]}, + index=pd.DatetimeIndex( + [pd.Timestamp("2024-01-01"), pd.Timestamp("2024-01-01"), pd.Timestamp("2024-01-02")] + ), + ) + received = lib.read("sym").data + assert_frame_equal(received, expected) + + def test_merge_update_row_from_source_matches_multiple_rows_from_target_in_separate_slices( + self, lmdb_library_factory + ): + lib = lmdb_library_factory(arcticdb.LibraryOptions(rows_per_segment=2)) + target = pd.DataFrame( + {"a": [1, 2, 3], "b": [1.0, 2.0, 3.0]}, + index=pd.DatetimeIndex( + [pd.Timestamp("2024-01-01"), pd.Timestamp("2024-01-02"), pd.Timestamp("2024-01-02")] + ), + ) + lib.write("sym", target) + source = pd.DataFrame({"a": [5], "b": [20.0]}, index=pd.DatetimeIndex([pd.Timestamp("2024-01-02")])) + lib.merge("sym", source, strategy=MergeStrategy(not_matched_by_target=MergeAction.DO_NOTHING)) + expected = pd.DataFrame( + {"a": [1, 5, 5], "b": [1.0, 20.0, 20.0]}, + index=pd.DatetimeIndex( + [pd.Timestamp("2024-01-01"), pd.Timestamp("2024-01-02"), pd.Timestamp("2024-01-02")] + ), + ) + received = lib.read("sym").data + assert_frame_equal(received, expected) + + @pytest.mark.parametrize( + "strategy", + ( + pytest.param( + MergeStrategy(MergeAction.UPDATE, MergeAction.DO_NOTHING), + marks=pytest.mark.xfail(reason="Not implemented"), + ), + pytest.param(MergeStrategy(MergeAction.UPDATE, MergeAction.INSERT), marks=pytest.mark.skip), + ), + ) + def test_merge_update_throws_when_target_row_is_matched_more_than_once(self, lmdb_library, strategy): + lib = lmdb_library + target = pd.DataFrame({"a": [1, 2, 3], "b": [1.0, 2.0, 3.0]}, index=pd.date_range("2024-01-01", periods=3)) + lib.write("sym", target) + + source = pd.DataFrame( + {"a": [1, 2, 3], "b": [1.0, 2.0, 3.0]}, + index=pd.DatetimeIndex( + [ + pd.Timestamp("2024-01-01"), # Matches first row of target + pd.Timestamp("2024-01-01"), # Also matches first row of target + pd.Timestamp("2024-01-03"), + ] + ), + ) + + with pytest.raises(UserInputException): + lib.merge("sym", source, strategy=strategy) + + @pytest.mark.parametrize("merge_metadata", (None, "meta")) + def test_merge_update_target_is_empty(self, lmdb_library, merge_metadata): + lib = lmdb_library + target = pd.DataFrame({"a": np.array([], dtype=np.int64)}, index=pd.DatetimeIndex([])) + lib.write("sym", target) + + source = pd.DataFrame({"a": np.array([1, 2], dtype=np.int64)}, index=pd.date_range("2024-01-01", periods=2)) + # NOTE: detecting that nothing will be changed is easier when the target is empty. It will be easy to not + # increase the version number. However, this will create two different behaviors because we currently increase + # the version if nothing is updated. I think having too many different behaviors will be confusing. IMO this + # must have the same behavior as test_merge_update_writes_new_version_even_if_nothing_is_changed. + merge_vit = lib.merge( + "sym", source, strategy=MergeStrategy(not_matched_by_target=MergeAction.DO_NOTHING), metadata=merge_metadata + ) + expected = target + read_vit = lib.read("sym") + assert_vit_equals_except_data(merge_vit, read_vit) + assert_frame_equal(read_vit.data, expected) + + @pytest.mark.parametrize( + "source", + ( + pd.DataFrame([], index=pd.DatetimeIndex([])), + pd.DataFrame({"a": []}, index=pd.DatetimeIndex([])), + pd.DataFrame({"a": [1]}, index=pd.DatetimeIndex([pd.Timestamp("2024-01-01")])), + ), + ) + @pytest.mark.parametrize("upsert", (pytest.param(True, marks=pytest.mark.xfail(reason="Not implemented")), False)) + def test_update_on_match_target_symbol_does_not_exist(self, lmdb_library, source, upsert): + # Model non-existing target after Library.update + # There is an upsert parameter to control whether to create the index. If upsert=False exception is thrown. + # Since we're doing a merge on non-existing data, I think it's logical to assume that nothing matches. If + # upsert=True, we will create an empty dataframe with the same schema of the source. + lib = lmdb_library + + if not upsert: + with pytest.raises(StorageException): + lib.merge( + "sym", source, strategy=MergeStrategy(MergeAction.UPDATE, MergeAction.DO_NOTHING), upsert=upsert + ) + else: + merge_vit = lib.merge( + "sym", source, strategy=MergeStrategy(MergeAction.UPDATE, MergeAction.DO_NOTHING), upsert=upsert + ) + expected = pd.DataFrame({"a": []}, index=pd.DatetimeIndex([])) + read_vit = lib.read("sym") + assert_vit_equals_except_data(merge_vit, read_vit) + assert_frame_equal(read_vit.data, expected) + + # ================================================================================================ + # ================================= TEST INSERT NOT MATCHED ====================================== + # ================================================================================================ + + @pytest.mark.parametrize( + "strategy", + ( + MergeStrategy(MergeAction.DO_NOTHING, MergeAction.INSERT), + MergeStrategy("do_nothing", "insert"), + MergeStrategy("DO_NOTHING", "INSERT"), + MergeStrategy(MergeAction.DO_NOTHING, "insert"), + MergeStrategy("do_nothing", MergeAction.INSERT), + ), + ) + def test_merge_insert_not_matched(self, lmdb_library, monkeypatch, strategy): + lib = lmdb_library + + target = pd.DataFrame({"a": [1, 2, 3], "b": [1.0, 2.0, 3.0]}, index=pd.date_range("2024-01-01", periods=3)) + write_vit = lib.write("sym", target) + + source = pd.DataFrame( + {"a": [-1, -2, -3], "b": [-1.0, -2.0, -3.0]}, + index=pd.DatetimeIndex(["2023-01-01", "2024-01-01 10:00:00", "2025-01-04"]), + ) + monkeypatch.setattr( + lib.__class__, + "merge", + lambda *args, **kwargs: VersionedItem( + symbol=write_vit.symbol, + library=write_vit.library, + data=None, + version=1, + metadata=write_vit.metadata, + host=write_vit.host, + timestamp=write_vit.timestamp + 1, + ), + raising=False, + ) + + merge_vit = lib.merge("sym", source, strategy=strategy) + assert merge_vit.version == 1 + assert merge_vit.symbol == write_vit.symbol + assert merge_vit.timestamp > write_vit.timestamp + assert merge_vit.metadata == write_vit.metadata + assert merge_vit.library == write_vit.library + assert merge_vit.host == write_vit.host + assert merge_vit.data is None + + expected = pd.concat([target, source]).sort_index() + monkeypatch.setattr(lib, "read", lambda *args, **kwargs: VersionedItem("sym", "lib", expected, 1)) + assert_frame_equal(lib.read("sym").data, expected) + + @pytest.mark.parametrize( + "date", + ( + pd.Timestamp("2023-01-01"), # Before first value + pd.Timestamp("2024-01-01 10:00:00"), # After first value and before second value + pd.Timestamp("2024-01-02 10:00:00"), # After second value + ), + ) + def test_merge_insert_not_matched_in_full_segment(self, lmdb_library_factory, date, monkeypatch): + lib = lmdb_library_factory(arcticdb.LibraryOptions(rows_per_segment=2, columns_per_segment=2)) + target = pd.DataFrame( + {"a": [1, 2, 3], "b": [1.0, 2.0, 3.0], "c": ["a", "b", "c"]}, index=pd.date_range("2024-01-01", periods=3) + ) + lib.write("sym", target) + lt = lib._dev_tools.library_tool() + assert len(lt.find_keys_for_symbol(KeyType.TABLE_DATA, "sym")) == 4 + + source = pd.DataFrame({"a": [10], "b": [20.0], "c": ["A"]}, index=pd.DatetimeIndex([date])) + monkeypatch.setattr(lib.__class__, "merge", lambda *args, **kwargs: None, raising=False) + lib.merge("sym", source, strategy=MergeStrategy(MergeAction.DO_NOTHING, MergeAction.INSERT)) + expected = pd.concat([target, source]).sort_index() + monkeypatch.setattr(lib, "read", lambda *args, **kwargs: VersionedItem("sym", "lib", expected, 2)) + received = lib.read("sym").data + assert_frame_equal(received, expected) + + lt = lib._dev_tools.library_tool() + # The first segment is changed + # A new segment is created because after the insert the first segment overflows the rows_per_segment setting + # Each new segment is column sliced -> 2 * 2 = 4 new data keys + monkeypatch.setattr( + lt, + "find_keys_for_symbol", + mock_find_keys_for_symbol({KeyType.TABLE_DATA: 8, KeyType.TABLE_INDEX: 2, KeyType.VERSION: 2}), + ) + assert len(lt.find_keys_for_symbol(KeyType.TABLE_DATA, "sym")) == 8 + assert len(lt.find_keys_for_symbol(KeyType.TABLE_INDEX, "sym")) == 2 + assert len(lt.find_keys_for_symbol(KeyType.VERSION, "sym")) == 2 + + def test_merge_insert_not_matched_writes_new_version_even_if_nothing_is_changed(self, lmdb_library, monkeypatch): + lib = lmdb_library + target = pd.DataFrame({"a": [1, 2, 3], "b": [1.0, 2.0, 3.0]}, index=pd.date_range("2024-01-01", periods=3)) + write_vit = lib.write("sym", target) + monkeypatch.setattr( + lib, + "merge", + lambda *args, **kwargs: VersionedItem( + symbol=write_vit.symbol, + library=write_vit.library, + data=None, + version=write_vit.version + 1, + metadata=write_vit.metadata, + host=write_vit.host, + timestamp=write_vit.timestamp + 1, + ), + raising=False, + ) + source = pd.DataFrame( + {"a": [10, 20, 30], "b": [10.0, 20.0, 30.0]}, index=pd.date_range("2024-01-01", periods=3) + ) + merge_vit = lib.merge("sym", source, strategy=MergeStrategy(MergeAction.DO_NOTHING, MergeAction.INSERT)) + assert merge_vit.version == 1 + assert merge_vit.timestamp > write_vit.timestamp + + monkeypatch.setattr( + lib, + "read", + lambda *args, **kwargs: VersionedItem( + symbol=merge_vit.symbol, + library=merge_vit.library, + data=target, + version=merge_vit.version, + metadata=merge_vit.metadata, + host=merge_vit.host, + timestamp=merge_vit.timestamp, + ), + ) + read_vit = lib.read("sym") + assert_vit_equals_except_data(read_vit, merge_vit) + assert_frame_equal(read_vit.data, target) + + lt = lib._dev_tools.library_tool() + monkeypatch.setattr( + lt, + "find_keys_for_symbol", + mock_find_keys_for_symbol({KeyType.TABLE_DATA: 1, KeyType.TABLE_INDEX: 1, KeyType.VERSION: 2}), + ) + assert len(lt.find_keys_for_symbol(KeyType.TABLE_DATA, "sym")) == 1 + assert len(lt.find_keys_for_symbol(KeyType.TABLE_INDEX, "sym")) == 1 + assert len(lt.find_keys_for_symbol(KeyType.VERSION, "sym")) == 2 + + def test_merge_insert_not_matched_on_index_and_column(self, lmdb_library, monkeypatch): + lib = lmdb_library + target = pd.DataFrame({"a": [1, 2, 3], "b": [1.0, 2.0, 3.0]}, index=pd.date_range("2024-01-01", periods=3)) + lib.write("sym", target) + + source = pd.DataFrame( + {"a": [1, 2, 30, 40], "b": [10.0, 20.0, 30.0, 40.0]}, + index=pd.DatetimeIndex( + [ + "2024-01-01", # Matches index and column + "2024-01-02 01:00:00", # Matches column, but not index + "2024-01-03", # Matches index, but not column + "2024-01-04", # Does not match either + ] + ), + ) + monkeypatch.setattr(lib.__class__, "merge", lambda *args, **kwargs: None, raising=False) + lib.merge("sym", source, on=["a"], strategy=MergeStrategy(MergeAction.DO_NOTHING, MergeAction.INSERT)) + + # The first row is matched, but the strategy for matched rows is DO_NOTHING, so no update + # the rest rows are inserted + expected = pd.concat([target, source.tail(len(source) - 1)]).sort_index() + monkeypatch.setattr(lib, "read", lambda *args, **kwargs: VersionedItem("sym", "lib", expected, 2)) + received = lib.read("sym").data + assert_frame_equal(received, expected) + + def test_merge_insert_not_matched_on_multiple_columns(self, lmdb_library, monkeypatch): + lib = lmdb_library + target = pd.DataFrame( + { + "a": [1, 2, 3, 4], + "b": ["a", "b", "c", "d"], + "c": ["A", "B", "A", "C"], + "d": [10.1, 20.2, 30.3, 40.4], + "e": [100, 200, 300, 400], + }, + index=pd.date_range("2024-01-01", periods=4), + ) + lib.write("sym", target) + + source = pd.DataFrame( + { + "a": [10, 20, 30, 40, 50], + "b": ["a", "b", "c", "d", "e"], + "c": ["A", "D", "A", "B", "C"], + "d": [10.1, 50.5, 30.3, 40.4, 70.7], + "e": [100, 500, 600, 400, 800], + }, + index=pd.DatetimeIndex( + [ + "2024-01-01", # MATCH: index + b + d + e match + "2024-01-02", # NOT_MATCH: index + b match, but d + e differ + "2024-01-03", # NOT_MATCH: index + b + d match, e differs + "2024-01-05", # NOT_MATCH: new index, b+d+e match + "2024-01-06", # NOT_MATCH: new index, new b, new d, new e + ] + ), + ) + + monkeypatch.setattr(lib.__class__, "merge", lambda *args, **kwargs: None, raising=False) + lib.merge("sym", source, on=["b", "d", "e"], strategy=MergeStrategy(MergeAction.DO_NOTHING, MergeAction.INSERT)) + expected = pd.concat([target, source.tail(len(source) - 1)]).sort_index() + monkeypatch.setattr(lib, "read", lambda *args, **kwargs: VersionedItem("sym", "lib", expected, 2)) + received = lib.read("sym").data + assert_frame_equal(received, expected) + + def test_merge_insert_does_not_throw_when_target_row_is_matched_more_than_once_when_matched_is_do_nothing( + self, lmdb_library, monkeypatch + ): + lib = lmdb_library + target = pd.DataFrame({"a": [1, 2, 3], "b": [1.0, 2.0, 3.0]}, index=pd.date_range("2024-01-01", periods=3)) + lib.write("sym", target) + + source = pd.DataFrame( + {"a": [1, 2, 4], "b": [1.0, 2.0, 4.0]}, + index=pd.DatetimeIndex( + [ + pd.Timestamp("2024-01-01"), # Matches first row of target + pd.Timestamp("2024-01-01"), # Also matches first row of target + pd.Timestamp("2024-01-04"), + ] + ), + ) + + monkeypatch.setattr(lib.__class__, "merge", lambda *args, **kwargs: None, raising=False) + lib.merge("sym", source, strategy=MergeStrategy(MergeAction.DO_NOTHING, MergeAction.INSERT)) + expected = pd.DataFrame( + {"a": [1, 2, 3, 4], "b": [1.0, 2.0, 3.0, 4.0]}, index=pd.date_range("2024-01-01", periods=4) + ) + monkeypatch.setattr(lib, "read", lambda *args, **kwargs: VersionedItem("sym", "lib", expected, 2)) + received = lib.read("sym").data + assert_frame_equal(received, expected) + + def test_merge_insert_target_is_empty(self, lmdb_library, monkeypatch): + lib = lmdb_library + target = pd.DataFrame({"a": np.array([], dtype=np.int64)}, index=pd.DatetimeIndex([])) + write_vit = lib.write("sym", target) + + source = pd.DataFrame({"a": np.array([1, 2], dtype=np.int64)}, index=pd.date_range("2024-01-01", periods=2)) + monkeypatch.setattr( + lib.__class__, + "merge", + lambda *args, **kwargs: VersionedItem( + symbol=write_vit.symbol, + library=write_vit.library, + data=None, + version=1, + metadata=None, + host=write_vit.host, + timestamp=write_vit.timestamp + 1, + ), + raising=False, + ) + merge_vit = lib.merge("sym", source, strategy=MergeStrategy("do_nothing", "insert")) + expected = source + monkeypatch.setattr( + lib, + "read", + lambda *args, **kwargs: VersionedItem( + symbol=merge_vit.symbol, + library=merge_vit.library, + data=expected, + version=merge_vit.version, + metadata=merge_vit.metadata, + host=merge_vit.host, + timestamp=merge_vit.timestamp, + ), + ) + read_vit = lib.read("sym") + assert_vit_equals_except_data(merge_vit, read_vit) + assert_frame_equal(read_vit.data, expected) + + # ================================================================================================ + # =================================== TEST UPDATE AND INSERT ===================================== + # ================================================================================================ + + @pytest.mark.parametrize( + "strategy", + ( + None, + MergeStrategy(MergeAction.UPDATE, MergeAction.INSERT), + MergeStrategy("update", "insert"), + MergeStrategy("UPDATE", "INSERT"), + MergeStrategy("update", MergeAction.INSERT), + MergeStrategy(MergeAction.UPDATE, "insert"), + ), + ) + def test_update_and_insert(self, lmdb_library, monkeypatch, strategy): + lib = lmdb_library + target = pd.DataFrame( + {"a": [1, 2, 3], "b": [1.0, 2.0, 3.0], "c": ["a", "b", "c"]}, index=pd.date_range("2024-01-01", periods=3) + ) + write_vit = lib.write("sym", target) + + source = pd.DataFrame( + {"a": [-1, 20, -2], "b": [-1.1, 20.1, -3.1], "c": ["a", "c", "d"]}, + index=pd.DatetimeIndex( + [pd.Timestamp("2024-01-01 15:00:00"), pd.Timestamp("2024-01-02"), pd.Timestamp("2024-01-02 01:00:00")] + ), + ) + + monkeypatch.setattr( + lib.__class__, + "merge", + lambda *args, **kwargs: VersionedItem( + symbol=write_vit.symbol, + library=write_vit.library, + data=None, + version=1, + metadata=write_vit.metadata, + host=write_vit.host, + timestamp=write_vit.timestamp + 1, + ), + raising=False, + ) + + merge_vit = lib.merge("sym", source, strategy=strategy) if strategy else lib.merge("sym", source) + assert merge_vit.version == 1 + assert merge_vit.symbol == write_vit.symbol + assert merge_vit.timestamp > write_vit.timestamp + assert merge_vit.metadata == write_vit.metadata + assert merge_vit.library == write_vit.library + assert merge_vit.host == write_vit.host + assert merge_vit.data is None + + expected = pd.DataFrame( + {"a": [1, -1, 20, -2, 3], "b": [1.0, -1.1, 20.1, -3.1, 3.0], "c": ["a", "a", "c", "d", "c"]}, + index=pd.DatetimeIndex( + [ + pd.Timestamp("2024-01-01"), + pd.Timestamp("2024-01-01 15:00:00"), + pd.Timestamp("2024-01-02"), + pd.Timestamp("2024-01-02 01:00:00"), + pd.Timestamp("2024-01-03"), + ] + ), + ) + + monkeypatch.setattr( + lib, + "read", + lambda *args, **kwargs: VersionedItem( + symbol=merge_vit.symbol, + library=merge_vit.library, + data=expected, + version=merge_vit.version, + metadata=merge_vit.metadata, + host=merge_vit.host, + timestamp=merge_vit.timestamp, + ), + ) + + read_vit = lib.read("sym") + assert_vit_equals_except_data(read_vit, merge_vit) + assert_frame_equal(read_vit.data, expected) + + lt = lib._dev_tools.library_tool() + monkeypatch.setattr( + lt, + "find_keys_for_symbol", + mock_find_keys_for_symbol({KeyType.TABLE_DATA: 2, KeyType.TABLE_INDEX: 2, KeyType.VERSION: 2}), + ) + assert len(lt.find_keys_for_symbol(KeyType.TABLE_DATA, "sym")) == 2 + assert len(lt.find_keys_for_symbol(KeyType.TABLE_INDEX, "sym")) == 2 + assert len(lt.find_keys_for_symbol(KeyType.VERSION, "sym")) == 2 + + @pytest.mark.parametrize( + "slicing_policy", + [{"rows_per_segment": 2}, {"columns_per_segment": 2}, {"rows_per_segment": 2, "columns_per_segment": 2}], + ) + def test_update_and_insert_with_slicing(self, lmdb_library_factory, monkeypatch, slicing_policy): + lib = lmdb_library_factory(arcticdb.LibraryOptions(**slicing_policy)) + target = pd.DataFrame( + {"a": [1, 2, 3, 4, 5], "b": [1.0, 2.0, 3.0, 4.0, 5.0], "c": ["a", "b", "c", "d", "e"]}, + index=pd.date_range("2024-01-01", periods=5), + ) + lib.write("sym", target) + + # Insertion pushes the row to be updated into a new segment + source = pd.DataFrame( + {"a": [-1, 40], "b": [-1.1, 40.4], "c": ["a", "f"]}, + index=pd.DatetimeIndex([pd.Timestamp("2024-01-03 15:00:00"), pd.Timestamp("2024-01-04")]), + ) + monkeypatch.setattr(lib.__class__, "merge", lambda *args, **kwargs: None, raising=False) + lib.merge("sym", source) + + expected = pd.DataFrame( + {"a": [1, 2, 3, -1, 40, 5], "b": [1.0, 2.0, 3.0, -1.1, 40.4, 5.0], "c": ["a", "b", "c", "a", "f", "e"]}, + index=pd.DatetimeIndex( + [ + pd.Timestamp("2024-01-01"), + pd.Timestamp("2024-01-02"), + pd.Timestamp("2024-01-03"), + pd.Timestamp("2024-01-03 15:00:00"), + pd.Timestamp("2024-01-04"), + pd.Timestamp("2024-01-05"), + ] + ), + ) + monkeypatch.setattr(lib, "read", lambda *args, **kwargs: VersionedItem("sym", "lib", expected, 2)) + assert_frame_equal(lib.read("sym").data, expected) + + lt = lib._dev_tools.library_tool() + if "rows_per_segment" in slicing_policy and "columns_per_segment" in slicing_policy: + expected_data_keys = 10 + elif "rows_per_segment" in slicing_policy: + expected_data_keys = 5 + elif "columns_per_segment" in slicing_policy: + expected_data_keys = 4 + monkeypatch.setattr( + lt, + "find_keys_for_symbol", + mock_find_keys_for_symbol( + {KeyType.TABLE_DATA: expected_data_keys, KeyType.TABLE_INDEX: 2, KeyType.VERSION: 2} + ), + ) + assert len(lt.find_keys_for_symbol(KeyType.TABLE_DATA, "sym")) == expected_data_keys + assert len(lt.find_keys_for_symbol(KeyType.TABLE_INDEX, "sym")) == 2 + assert len(lt.find_keys_for_symbol(KeyType.VERSION, "sym")) == 2 + + @pytest.mark.parametrize( + "source", + ( + pd.DataFrame([], index=pd.DatetimeIndex([])), + pd.DataFrame({"a": []}, index=pd.DatetimeIndex([])), + pd.DataFrame({"a": [1]}, index=pd.DatetimeIndex([pd.Timestamp("2024-01-01")])), + ), + ) + @pytest.mark.parametrize("upsert", (True, False)) + @pytest.mark.parametrize("strategy", (MergeStrategy("update", "insert"), MergeStrategy("do_nothing", "insert"))) + def test_insert_when_target_symbol_does_not_exist(self, lmdb_library, monkeypatch, source, upsert, strategy): + # Model non-existing target after Library.update + # There is an upsert parameter to control whether to create the index. If upsert=False exception is thrown. + # Since we're doing a merge on non-existing data, I think it's logical to assume that nothing matches. No + # updates are performed. The insert operation will insert the data into the newly created target symbol. + lib = lmdb_library + + if not upsert: + monkeypatch.setattr(lib.__class__, "merge", raise_wrapper(StorageException), raising=False) + with pytest.raises(StorageException): + lib.merge("sym", source, strategy=strategy, upsert=upsert) + else: + import datetime + + monkeypatch.setattr( + lib.__class__, + "merge", + lambda *args, **kwargs: VersionedItem( + symbol="sym", + library=lmdb_library.name, + data=None, + version=0, + metadata=None, + host=lmdb_library._nvs.env, + timestamp=pd.Timestamp(datetime.datetime.now()), + ), + raising=False, + ) + merge_vit = lib.merge("sym", source, strategy=strategy, upsert=upsert) + expected = source + monkeypatch.setattr( + lib, + "read", + lambda *args, **kwargs: VersionedItem( + symbol=merge_vit.symbol, + library=merge_vit.library, + data=expected, + version=merge_vit.version, + metadata=merge_vit.metadata, + host=merge_vit.host, + timestamp=merge_vit.timestamp, + ), + ) + read_vit = lib.read("sym") + assert_vit_equals_except_data(merge_vit, read_vit) + assert_frame_equal(read_vit.data, expected) + + def test_update_and_insert_on_index_and_column(self, lmdb_library, monkeypatch): + lib = lmdb_library + target = pd.DataFrame( + {"a": [1, 2, 3], "b": [1.0, 2.0, 3.0], "c": ["a", "b", "c"]}, index=pd.date_range("2024-01-01", periods=3) + ) + lib.write("sym", target) + + source = pd.DataFrame( + {"a": [1, 2, 30, 40], "b": [10.0, 20.0, 30.0, 40.0], "c": ["A", "B", "A", "C"]}, + index=pd.DatetimeIndex( + [ + pd.Timestamp("2024-01-01"), # Matches index and column + pd.Timestamp("2024-01-02 01:00:00"), # Matches column, but not index + pd.Timestamp("2024-01-03"), # Matches index, but not column + pd.Timestamp("2024-01-04"), # Does not match either + ] + ), + ) + monkeypatch.setattr(lib.__class__, "merge", lambda *args, **kwargs: None, raising=False) + lib.merge("sym", source, on=["a"]) + + expected = pd.DataFrame( + {"a": [1, 2, 2, 3, 30, 40], "b": [10.0, 2.0, 20.0, 3.0, 30.0, 40.0], "c": ["A", "b", "B", "c", "A", "C"]}, + index=pd.DatetimeIndex( + [ + pd.Timestamp("2024-01-01"), + pd.Timestamp("2024-01-02"), + pd.Timestamp("2024-01-02 01:00:00"), + pd.Timestamp("2024-01-03"), + pd.Timestamp("2024-01-03"), + pd.Timestamp("2024-01-04"), + ] + ), + ) + monkeypatch.setattr(lib, "read", lambda *args, **kwargs: VersionedItem("sym", "lib", expected, 2)) + received = lib.read("sym").data + + assert_frame_equal(received, expected) + + def test_update_and_insert_multiple_columns(self, lmdb_library, monkeypatch): + lib = lmdb_library + target = pd.DataFrame( + { + "a": [1, 2, 3, 4], + "b": ["a", "b", "c", "d"], + "c": ["A", "B", "A", "C"], + "d": [10.1, 20.2, 30.3, 40.4], + "e": [100, 200, 300, 400], + }, + index=pd.date_range("2024-01-01", periods=4), + ) + lib.write("sym", target) + + source = pd.DataFrame( + { + "a": [10, 20, 30, 40, 50], + "b": ["a", "b", "c", "d", "e"], + "c": ["A", "D", "A", "B", "C"], + "d": [10.1, 50.5, 30.3, 40.4, 70.7], + "e": [100, 500, 600, 400, 800], + }, + index=pd.DatetimeIndex( + [ + "2024-01-01", # index + b + d + e match (update) + "2024-01-02", # index + b match, but d + e differ (insert) + "2024-01-03", # index + b + d match, e differs (insert) + "2024-01-05", # new index, b+d+e match (insert) + "2024-01-06", # new index, new b, new d, new e (insert) + ] + ), + ) + + monkeypatch.setattr(lib.__class__, "merge", lambda *args, **kwargs: None, raising=False) + lib.merge( + "sym", source, on=["b", "d", "e"], strategy=MergeStrategy(not_matched_by_target=MergeAction.DO_NOTHING) + ) + expected = pd.DataFrame( + { + "a": [10, 2, 20, 3, 30, 4, 40, 50], + "b": ["a", "b", "b", "c", "c", "d", "d", "e"], + "c": ["A", "B", "D", "A", "A", "C", "B", "C"], + "d": [10.1, 20.2, 50.5, 30.3, 30.3, 40.4, 40.4, 70.7], + "e": [100, 200, 500, 300, 600, 400, 400, 800], + }, + index=pd.DatetimeIndex( + [ + "2024-01-01", # Updated: matched on b="a", d=10.1, e=100 + "2024-01-02", # Original target row + "2024-01-02", # Inserted: no match for b="b", d=50.5, e=500 + "2024-01-03", # Original target row + "2024-01-03", # Inserted: no match for b="c", d=30.3, e=600 + "2024-01-04", # Original target row + "2024-01-05", # Inserted: no match for b="d", d=40.4, e=400 (different index) + "2024-01-06", # Inserted: completely new b="e", d=70.7, e=800 + ] + ), + ) + + monkeypatch.setattr(lib, "read", lambda *args, **kwargs: VersionedItem("sym", "lib", expected, 2)) + received = lib.read("sym").data + assert_frame_equal(received, expected) + + def test_merge_update_and_insert_target_is_empty(self, lmdb_library, monkeypatch): + lib = lmdb_library + target = pd.DataFrame({"a": np.array([], dtype=np.int64)}, index=pd.DatetimeIndex([])) + write_vit = lib.write("sym", target) + + source = pd.DataFrame({"a": np.array([1, 2], dtype=np.int64)}, index=pd.date_range("2024-01-01", periods=2)) + monkeypatch.setattr( + lib.__class__, + "merge", + lambda *args, **kwargs: VersionedItem( + symbol=write_vit.symbol, + library=write_vit.library, + data=None, + version=1, + metadata=None, + host=write_vit.host, + timestamp=write_vit.timestamp + 1, + ), + raising=False, + ) + merge_vit = lib.merge("sym", source) + expected = source + monkeypatch.setattr( + lib, + "read", + lambda *args, **kwargs: VersionedItem( + symbol=merge_vit.symbol, + library=merge_vit.library, + data=expected, + version=merge_vit.version, + metadata=merge_vit.metadata, + host=merge_vit.host, + timestamp=merge_vit.timestamp, + ), + ) + read_vit = lib.read("sym") + assert_vit_equals_except_data(merge_vit, read_vit) + assert_frame_equal(read_vit.data, expected) + + class TestMergeRowRange: + """Not implemented yet""" + + def test_merge_not_implemented_with_row_range_yet(self, lmdb_library, monkeypatch): + lib = lmdb_library + target = pd.DataFrame({"a": [1, 2, 3], "b": [1.0, 2.0, 3.0]}) + lib.write("sym", target) + + source = pd.DataFrame({"a": [1], "b": [2]}) + monkeypatch.setattr(lib.__class__, "merge", raise_wrapper(UserInputException), raising=False) + with pytest.raises(UserInputException): + lib.merge("sym", source) + + class TestMergeMultiindex: + """Not implemented yet""" + + def test_merge_not_implemented_with_multiindex_yet(self, lmdb_library, monkeypatch): + lib = lmdb_library + target = pd.DataFrame( + {"a": [1, 2, 3], "b": [1.0, 2.0, 3.0]}, index=pd.MultiIndex.from_tuples([("A", 1), ("B", 2), ("C", 3)]) + ) + lib.write("sym", target) + source = pd.DataFrame({"a": [2], "b": [3.0]}, index=pd.MultiIndex.from_tuples([("A", 1)])) + monkeypatch.setattr(lib.__class__, "merge", raise_wrapper(UserInputException), raising=False) + with pytest.raises(UserInputException): + lib.merge("sym", source) From 556fdce534db4da82ad78af0e9d27d35a3b56f3f Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Wed, 5 Nov 2025 12:34:41 +0200 Subject: [PATCH 22/46] Add merge clause skeleton --- cpp/arcticdb/processing/clause.cpp | 80 +++++++++++++------ cpp/arcticdb/processing/clause.hpp | 39 +++++++++ cpp/arcticdb/processing/clause_utils.hpp | 4 +- .../version/local_versioned_engine.cpp | 42 +++++++++- .../version/local_versioned_engine.hpp | 5 +- cpp/arcticdb/version/version_core.cpp | 20 +++-- cpp/arcticdb/version/version_core.hpp | 8 +- cpp/arcticdb/version/version_store_api.cpp | 5 +- cpp/arcticdb/version/version_store_api.hpp | 2 +- cpp/arcticdb/version/versioned_engine.hpp | 5 +- python/arcticdb/version_store/library.py | 10 +++ 11 files changed, 176 insertions(+), 44 deletions(-) diff --git a/cpp/arcticdb/processing/clause.cpp b/cpp/arcticdb/processing/clause.cpp index 26ee832e52a..63504977400 100644 --- a/cpp/arcticdb/processing/clause.cpp +++ b/cpp/arcticdb/processing/clause.cpp @@ -1374,17 +1374,10 @@ std::vector ColumnStatsGenerationClause::process(std::vector } std::vector> RowRangeClause::structure_for_processing(std::vector& ranges_and_keys) { - auto row_range_filter = RowRange{start_, end_}; - ranges_and_keys.erase( - std::remove_if( - ranges_and_keys.begin(), - ranges_and_keys.end(), - [&](const RangesAndKey& ranges_and_key) { - return !is_slice_in_row_range(ranges_and_key.row_range(), row_range_filter); - } - ), - ranges_and_keys.end() - ); + const auto row_range_filter = RowRange{start_, end_}; + std::erase_if(ranges_and_keys, [&](const RangesAndKey& ranges_and_key) { + return !is_slice_in_row_range(ranges_and_key.row_range(), row_range_filter); + }); return structure_by_row_slice(ranges_and_keys); } @@ -1531,18 +1524,11 @@ std::vector> DateRangeClause::structure_for_processing(std:: processing_config_.index_type_ == IndexDescriptor::Type::TIMESTAMP, "Cannot use date range with non-timestamp indexed data" ); - auto index_filter = IndexRange(start_, end_); - ranges_and_keys.erase( - std::remove_if( - ranges_and_keys.begin(), - ranges_and_keys.end(), - [&](const RangesAndKey& ranges_and_key) { - auto slice_index_range = IndexRange(ranges_and_key.key_.time_range()); - return !is_slice_in_index_range(slice_index_range, index_filter, true); - } - ), - ranges_and_keys.end() - ); + const auto index_filter = IndexRange(start_, end_); + std::erase_if(ranges_and_keys, [&](const RangesAndKey& ranges_and_key) { + const auto slice_index_range = IndexRange(ranges_and_key.key_.time_range()); + return !is_slice_in_index_range(slice_index_range, index_filter, true); + }); return structure_by_row_slice(ranges_and_keys); } @@ -1732,4 +1718,52 @@ OutputSchema WriteClause::join_schemas(std::vector&&) const { std::string WriteClause::to_string() const { return "Write"; } +MergeUpdateClause::MergeUpdateClause( + std::vector&& on, MergeStrategy strategy, std::shared_ptr source, + bool match_on_timeseries_index +) : + on_(std::move(on)), + strategy_(strategy), + source_(std::move(source)), + match_on_timeseries_index_(match_on_timeseries_index) {} + +std::vector> MergeUpdateClause::structure_for_processing(std::vector& ranges_and_keys +) { + return structure_by_row_slice(ranges_and_keys); +} + +std::vector> MergeUpdateClause::structure_for_processing( + std::vector>&& entity_ids_vec +) { + return structure_by_row_slice(*component_manager_, std::move(entity_ids_vec)); +} + +std::vector MergeUpdateClause::process(std::vector&& entity_ids) const { + if (entity_ids.empty()) { + return {}; + } + const auto proc = + gather_entities, std::shared_ptr, std::shared_ptr>( + *component_manager_, std::move(entity_ids) + ); + + return {}; +} + +const ClauseInfo& MergeUpdateClause::clause_info() const { return clause_info_; } + +void MergeUpdateClause::set_processing_config(const ProcessingConfig&) {} + +void MergeUpdateClause::set_component_manager(std::shared_ptr component_manager) { + component_manager_ = std::move(component_manager); +} + +OutputSchema MergeUpdateClause::modify_schema(OutputSchema&& output_schema) const { return output_schema; } + +OutputSchema MergeUpdateClause::join_schemas(std::vector&&) const { + util::raise_rte("MergeUpdateClause::join_schemas should never be called"); +} + +std::string MergeUpdateClause::to_string() const { return "MERGE_UPDATE"; } + } // namespace arcticdb diff --git a/cpp/arcticdb/processing/clause.hpp b/cpp/arcticdb/processing/clause.hpp index 598719b958c..2c3f09735d5 100644 --- a/cpp/arcticdb/processing/clause.hpp +++ b/cpp/arcticdb/processing/clause.hpp @@ -22,6 +22,7 @@ #include #include #include +#include #include #include #include @@ -38,6 +39,10 @@ namespace stream { struct PartialKey; } // namespace stream +namespace pipelines { +struct InputFrame; +} + class DeDupMap; struct IClause { @@ -838,4 +843,38 @@ struct WriteClause { stream::PartialKey create_partial_key(const SegmentInMemory& segment) const; }; +struct MergeUpdateClause { + ClauseInfo clause_info_; + std::shared_ptr component_manager_; + std::vector on_; + MergeStrategy strategy_; + std::shared_ptr source_; + bool match_on_timeseries_index_; + MergeUpdateClause( + std::vector&& on, MergeStrategy strategy, std::shared_ptr source, + bool match_on_timeseries_index + ); + ARCTICDB_MOVE_COPY_DEFAULT(MergeUpdateClause) + + [[nodiscard]] std::vector> structure_for_processing(std::vector&); + + [[nodiscard]] std::vector> structure_for_processing( + std::vector>&& entity_ids_vec + ); + + [[nodiscard]] std::vector process(std::vector&& entity_ids) const; + + [[nodiscard]] const ClauseInfo& clause_info() const; + + void set_processing_config(const ProcessingConfig&); + + void set_component_manager(std::shared_ptr component_manager); + + OutputSchema modify_schema(OutputSchema&& output_schema) const; + + OutputSchema join_schemas(std::vector&&) const; + + [[nodiscard]] std::string to_string() const; +}; + } // namespace arcticdb diff --git a/cpp/arcticdb/processing/clause_utils.hpp b/cpp/arcticdb/processing/clause_utils.hpp index 51fb1eb1b55..50ccbf3c154 100644 --- a/cpp/arcticdb/processing/clause_utils.hpp +++ b/cpp/arcticdb/processing/clause_utils.hpp @@ -92,9 +92,9 @@ struct RangesAndEntity { }; template -requires std::is_same_v || std::is_same_v +requires util::any_of std::vector> structure_by_row_slice(std::vector& ranges) { - std::sort(std::begin(ranges), std::end(ranges), [](const T& left, const T& right) { + std::ranges::sort(ranges, [](const T& left, const T& right) { return std::tie(left.row_range().first, left.col_range().first) < std::tie(right.row_range().first, right.col_range().first); }); diff --git a/cpp/arcticdb/version/local_versioned_engine.cpp b/cpp/arcticdb/version/local_versioned_engine.cpp index 355f8487862..ef9a02b2cb1 100644 --- a/cpp/arcticdb/version/local_versioned_engine.cpp +++ b/cpp/arcticdb/version/local_versioned_engine.cpp @@ -2279,8 +2279,9 @@ std::shared_ptr LocalVersionedEngine::_test_get_version_map() { retu void LocalVersionedEngine::_test_set_store(std::shared_ptr store) { set_store(std::move(store)); } VersionedItem LocalVersionedEngine::merge_internal( - const StreamId& stream_id, const std::shared_ptr& source, bool prune_previous_versions, - const MergeStrategy& strategy, std::span on, bool match_on_timeseries_index + const StreamId& stream_id, const std::shared_ptr& source, const py::object& user_meta, + const bool prune_previous_versions, const MergeStrategy& strategy, std::vector&& on, + const bool match_on_timeseries_index ) { ARCTICDB_RUNTIME_DEBUG(log::version(), "Command: merge"); py::gil_scoped_release release_gil; @@ -2295,8 +2296,41 @@ VersionedItem LocalVersionedEngine::merge_internal( ); return VersionedItem{*std::move(update_info.previous_index_key_)}; } - auto versioned_item = - merge_impl(store(), source, update_info, get_write_options(), strategy, on, match_on_timeseries_index); + constexpr static VersionQuery version_query; + const ReadOptions read_options; + const auto source_version = get_version_to_read(stream_id, version_query); + const auto identifier = get_version_identifier(stream_id, version_query, read_options, source_version); + // TODO: read_modify_write uses the same piece of code. Move it to a function. + std::unique_ptr user_meta_proto{ + [](const py::object& user_meta) -> proto::descriptors::UserDefinedMetadata* { + if (user_meta.is_none()) { + return nullptr; + } + const auto user_meta_proto = new proto::descriptors::UserDefinedMetadata(); + python_util::pb_from_python(user_meta, *user_meta_proto); + return user_meta_proto; + }(user_meta) + }; + auto [maybe_prev, deleted] = ::arcticdb::get_latest_version(store(), version_map(), stream_id); + const auto target_version = get_next_version_from_key(maybe_prev); + if (target_version == 0) { + if (auto check_outcome = verify_symbol_key(stream_id); std::holds_alternative(check_outcome)) { + std::get(check_outcome).throw_error(); + } + } + const WriteOptions write_options = get_write_options(); + auto versioned_item = merge_impl( + store(), + identifier, + std::move(user_meta_proto), + read_options, + write_options, + IndexPartialKey{stream_id, target_version}, + std::move(on), + match_on_timeseries_index, + strategy, + source + ); write_version_and_prune_previous(prune_previous_versions, versioned_item.key_, update_info.previous_index_key_); return versioned_item; } diff --git a/cpp/arcticdb/version/local_versioned_engine.hpp b/cpp/arcticdb/version/local_versioned_engine.hpp index ceb3032aefd..9e610f74588 100644 --- a/cpp/arcticdb/version/local_versioned_engine.hpp +++ b/cpp/arcticdb/version/local_versioned_engine.hpp @@ -353,8 +353,9 @@ class LocalVersionedEngine : public VersionedEngine { const arcticdb::proto::storage::VersionStoreConfig& cfg() const override { return cfg_; } VersionedItem merge_internal( - const StreamId& stream_id, const std::shared_ptr& source, bool prune_previous_versions, - const MergeStrategy& strategy, std::span on, bool match_on_timeseries_index + const StreamId& stream_id, const std::shared_ptr& source, const py::object& user_meta, + bool prune_previous_versions, const MergeStrategy& strategy, std::vector&& on, + bool match_on_timeseries_index ) override; protected: diff --git a/cpp/arcticdb/version/version_core.cpp b/cpp/arcticdb/version/version_core.cpp index 8e0483c98a1..6769a9714bf 100644 --- a/cpp/arcticdb/version/version_core.cpp +++ b/cpp/arcticdb/version/version_core.cpp @@ -2796,12 +2796,22 @@ VersionedItem read_modify_write_impl( } VersionedItem merge_impl( - [[maybe_unused]] const std::shared_ptr& store, [[maybe_unused]] const std::shared_ptr& frame, - [[maybe_unused]] const UpdateInfo& update_info, [[maybe_unused]] WriteOptions&& options, - [[maybe_unused]] const MergeStrategy& strategy, [[maybe_unused]] std::span on, - [[maybe_unused]] bool match_on_timeseries_index + const std::shared_ptr& store, const std::variant& version_info, + std::unique_ptr&& user_meta, const ReadOptions& read_options, + const WriteOptions& write_options, const IndexPartialKey& target_partial_index_key, + std::vector&& on, const bool match_on_timeseries_index, const MergeStrategy& strategy, + const std::shared_ptr& source ) { - return VersionedItem{}; + auto read_query = std::make_shared(); + read_query->clauses_.push_back( + std::make_shared(MergeUpdateClause(std::move(on), strategy, source, match_on_timeseries_index)) + ); + if (source->has_index()) { + read_query->row_filter = source->index_range; + } + return read_modify_write_impl( + store, version_info, std::move(user_meta), read_query, read_options, write_options, target_partial_index_key + ); } folly::Future read_and_process( diff --git a/cpp/arcticdb/version/version_core.hpp b/cpp/arcticdb/version/version_core.hpp index d3ede8f9cb2..bc9e88e3b1e 100644 --- a/cpp/arcticdb/version/version_core.hpp +++ b/cpp/arcticdb/version/version_core.hpp @@ -222,9 +222,11 @@ VersionedItem read_modify_write_impl( ); VersionedItem merge_impl( - const std::shared_ptr& store, const std::shared_ptr& frame, const UpdateInfo& update_info, - WriteOptions&& options, const MergeStrategy& strategy, std::span on, - bool match_on_timeseries_index + const std::shared_ptr& store, const std::variant& version_info, + std::unique_ptr&& user_meta, const ReadOptions& read_options, + const WriteOptions& write_options, const IndexPartialKey& target_partial_index_key, + std::vector&& on, bool match_on_timeseries_index, const MergeStrategy& strategy, + const std::shared_ptr& source ); } // namespace arcticdb::version_store diff --git a/cpp/arcticdb/version/version_store_api.cpp b/cpp/arcticdb/version/version_store_api.cpp index 0e71aeed4d0..fb52a7ff3ac 100644 --- a/cpp/arcticdb/version/version_store_api.cpp +++ b/cpp/arcticdb/version/version_store_api.cpp @@ -1335,7 +1335,7 @@ void PythonVersionStore::force_delete_symbol(const StreamId& stream_id) { VersionedItem PythonVersionStore::merge( const StreamId& stream_id, const py::tuple& source, const py::object& norm, const py::object& user_meta, - const bool prune_previous_versions, const py::tuple& py_strategy, const std::vector& on, + const bool prune_previous_versions, const py::tuple& py_strategy, std::vector on, const bool match_on_timeseries_index ) { const MergeStrategy strategy{ @@ -1344,9 +1344,10 @@ VersionedItem PythonVersionStore::merge( return merge_internal( stream_id, convert::py_ndf_to_frame(stream_id, source, norm, user_meta, cfg().write_options().empty_types()), + user_meta, prune_previous_versions, strategy, - on, + std::move(on), match_on_timeseries_index ); } diff --git a/cpp/arcticdb/version/version_store_api.hpp b/cpp/arcticdb/version/version_store_api.hpp index 95805e4f404..aede841b7aa 100644 --- a/cpp/arcticdb/version/version_store_api.hpp +++ b/cpp/arcticdb/version/version_store_api.hpp @@ -285,7 +285,7 @@ class PythonVersionStore : public LocalVersionedEngine { VersionedItem merge( const StreamId& stream_id, const py::tuple& source, const py::object& norm, const py::object& user_meta, - const bool prune_previous_versions, const py::tuple& py_strategy, const std::vector& on, + const bool prune_previous_versions, const py::tuple& py_strategy, std::vector on, const bool match_on_timeseries_index ); diff --git a/cpp/arcticdb/version/versioned_engine.hpp b/cpp/arcticdb/version/versioned_engine.hpp index 1500918f27c..5efca375747 100644 --- a/cpp/arcticdb/version/versioned_engine.hpp +++ b/cpp/arcticdb/version/versioned_engine.hpp @@ -147,8 +147,9 @@ class VersionedEngine { virtual void flush_version_map() = 0; virtual VersionedItem merge_internal( - const StreamId& stream_id, const std::shared_ptr& source, bool prune_previous_versions, - const MergeStrategy& strategy, std::span on, bool match_on_timeseries_index + const StreamId& stream_id, const std::shared_ptr& source, const py::object& user_meta, + bool prune_previous_versions, const MergeStrategy& strategy, std::vector&& on, + bool match_on_timeseries_index ) = 0; }; diff --git a/python/arcticdb/version_store/library.py b/python/arcticdb/version_store/library.py index b3d5e8252df..eea1e5d8600 100644 --- a/python/arcticdb/version_store/library.py +++ b/python/arcticdb/version_store/library.py @@ -3066,6 +3066,16 @@ def defragment_symbol_data( """ return self._nvs.defragment_symbol_data(symbol, segment_size, prune_previous_versions) + def merge( + self, + symbol: str, + data: NormalizableType, + on: Optional[List[str]] = None, + metadata: Any = None, + prune_previous_versions: bool = False, + ): + pass + @property def name(self) -> str: """The name of this library.""" From 7c861b24a10fd3f711d68aee89e85cb1c8d952a2 Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Thu, 6 Nov 2025 17:17:45 +0200 Subject: [PATCH 23/46] Implement structure for processing --- cpp/arcticdb/pipeline/frame_slice.hpp | 3 +- cpp/arcticdb/pipeline/input_frame.cpp | 4 +- cpp/arcticdb/processing/clause.cpp | 60 ++++++++++++++++++++++++--- cpp/arcticdb/processing/clause.hpp | 3 ++ 4 files changed, 62 insertions(+), 8 deletions(-) diff --git a/cpp/arcticdb/pipeline/frame_slice.hpp b/cpp/arcticdb/pipeline/frame_slice.hpp index 2cb3337c7bd..bd66d085e13 100644 --- a/cpp/arcticdb/pipeline/frame_slice.hpp +++ b/cpp/arcticdb/pipeline/frame_slice.hpp @@ -34,7 +34,8 @@ struct AxisRange : std::pair { struct Hasher { template - std::enable_if_t>, std::size_t> operator()(const T& r) const { + requires std::derived_from + std::size_t operator()(const T& r) const { // try to make better use of msb lsb given how F14 is implemented #ifdef _WIN32 return r.first ^ _byteswap_uint64(r.second); diff --git a/cpp/arcticdb/pipeline/input_frame.cpp b/cpp/arcticdb/pipeline/input_frame.cpp index e1a98c015f0..b26fa085a2f 100644 --- a/cpp/arcticdb/pipeline/input_frame.cpp +++ b/cpp/arcticdb/pipeline/input_frame.cpp @@ -74,7 +74,7 @@ timestamp InputFrame::index_value_at(size_t row) { util::check(has_index(), "InputFrame::index_value_at should only be called on timeseries data"); return util::variant_match( input_data, - [row](InputSegment& input_segment) { + [row](const InputSegment& input_segment) { const auto& seg = input_segment.seg; util::check( row < seg.row_count(), @@ -87,7 +87,7 @@ timestamp InputFrame::index_value_at(size_t row) { // equal to the number of input record batches for Arrow return *index_column.scalar_at(row); }, - [row](InputTensors& input_tensors) { + [row](const InputTensors& input_tensors) { util::check( input_tensors.index_tensor.has_value(), "InputFrame::index_value_at call with null index tensor" ); diff --git a/cpp/arcticdb/processing/clause.cpp b/cpp/arcticdb/processing/clause.cpp index 63504977400..ec3bc6f50a2 100644 --- a/cpp/arcticdb/processing/clause.cpp +++ b/cpp/arcticdb/processing/clause.cpp @@ -1729,13 +1729,63 @@ MergeUpdateClause::MergeUpdateClause( std::vector> MergeUpdateClause::structure_for_processing(std::vector& ranges_and_keys ) { - return structure_by_row_slice(ranges_and_keys); + if (!source_->has_index()) { + return structure_by_row_slice(ranges_and_keys); + } + std::vector> entities = structure_by_row_slice(ranges_and_keys); + util::BitSet row_slices_to_keep(entities.size()); + size_t source_row = 0; + auto first_col_slice_in_row = ranges_and_keys.begin(); + for (size_t row_slice_idx = 0; row_slice_idx < entities.size() && source_row < source_->num_rows; ++row_slice_idx) { + const TimestampRange time_range = first_col_slice_in_row->key_.time_range(); + bool keep_row_slice = source_->num_rows && strategy_.not_matched_by_target == MergeAction::INSERT && + source_->index_value_at(source_row) < time_range.first; + if (keep_row_slice) { + source_start_for_row_range_[first_col_slice_in_row->row_range()] = source_row; + } + timestamp source_ts = source_->index_value_at(source_row); + // TODO: If there are values to be inserted before the first segment this will read it and prepend the values + // There is no need to read the segment. We can just create a new segment. + while (source_row < source_->num_rows && strategy_.not_matched_by_target == MergeAction::INSERT && + source_ts < time_range.first) { + source_ts = source_->index_value_at(++source_row); + } + const bool source_ts_in_segment_range = source_ts >= time_range.first && source_ts < time_range.second; + if (!keep_row_slice && source_ts_in_segment_range) { + source_start_for_row_range_[first_col_slice_in_row->row_range()] = source_row; + } + keep_row_slice |= source_ts_in_segment_range; + while (source_row < source_->num_rows && source_ts >= time_range.first && source_ts < time_range.second) { + source_ts = source_->index_value_at(++source_row); + } + row_slices_to_keep[row_slice_idx] = keep_row_slice; + const size_t col_slice_count = entities[row_slice_idx].size(); + first_col_slice_in_row += col_slice_count; + } + // TODO: If there are values to be inserted after the last segment this will read it and append the values and split + // the segment if needed. There is no need to do this as we can just create a new segment. + if (source_row < source_->num_rows && strategy_.not_matched_by_target == MergeAction::INSERT) { + row_slices_to_keep[entities.size() - 1] = true; + } + const size_t num_row_slices_to_keep = row_slices_to_keep.count(); + if (num_row_slices_to_keep == entities.size()) { + return entities; + } + size_t entity_pos = 0; + for (size_t i = 0; i < entities.size(); ++i) { + if (row_slices_to_keep[i]) { + if (entity_pos != i) { + entities[entity_pos] = std::move(entities[i]); + } + ++entity_pos; + } + } + entities.resize(num_row_slices_to_keep); + return entities; } -std::vector> MergeUpdateClause::structure_for_processing( - std::vector>&& entity_ids_vec -) { - return structure_by_row_slice(*component_manager_, std::move(entity_ids_vec)); +std::vector> MergeUpdateClause::structure_for_processing(std::vector>&&) { + internal::raise("MergeUpdate clause should be the first clause in the pipeline"); } std::vector MergeUpdateClause::process(std::vector&& entity_ids) const { diff --git a/cpp/arcticdb/processing/clause.hpp b/cpp/arcticdb/processing/clause.hpp index 2c3f09735d5..5b507f8db10 100644 --- a/cpp/arcticdb/processing/clause.hpp +++ b/cpp/arcticdb/processing/clause.hpp @@ -875,6 +875,9 @@ struct MergeUpdateClause { OutputSchema join_schemas(std::vector&&) const; [[nodiscard]] std::string to_string() const; + + private: + ankerl::unordered_dense::map source_start_for_row_range_; }; } // namespace arcticdb From fca60b928424c1776dde06b0f6887dad30096014 Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Mon, 10 Nov 2025 10:37:57 +0200 Subject: [PATCH 24/46] WIP --- cpp/arcticdb/column_store/column_data.hpp | 12 ++ cpp/arcticdb/entity/native_tensor.hpp | 2 +- cpp/arcticdb/pipeline/frame_slice.hpp | 2 +- cpp/arcticdb/processing/clause.cpp | 168 +++++++++++++++++++++- cpp/arcticdb/processing/clause.hpp | 12 +- 5 files changed, 188 insertions(+), 8 deletions(-) diff --git a/cpp/arcticdb/column_store/column_data.hpp b/cpp/arcticdb/column_store/column_data.hpp index a958bb16bb9..bf22983849b 100644 --- a/cpp/arcticdb/column_store/column_data.hpp +++ b/cpp/arcticdb/column_store/column_data.hpp @@ -453,4 +453,16 @@ struct ColumnData { const util::BitMagic* bit_vector_; }; +template +struct is_column_data_iterator : std::false_type {}; + +template +struct is_column_data_iterator> : std::true_type {}; + +template +inline constexpr bool is_column_data_iterator_v = is_column_data_iterator>::value; + +template +concept column_data_iterator = is_column_data_iterator_v; + } // namespace arcticdb diff --git a/cpp/arcticdb/entity/native_tensor.hpp b/cpp/arcticdb/entity/native_tensor.hpp index eff76ff68e0..c43a7f4b279 100644 --- a/cpp/arcticdb/entity/native_tensor.hpp +++ b/cpp/arcticdb/entity/native_tensor.hpp @@ -171,7 +171,7 @@ ssize_t byte_offset_impl(const stride_t* strides, ssize_t i, Ix... index) { // TODO is the conversion to a typed tensor really necessary for the codec part? template struct TypedTensor : public NativeTensor { - static size_t itemsize() { return sizeof(T); } + constexpr static size_t itemsize() { return sizeof(T); } std::array f_style_strides() { std::array strides = {}; diff --git a/cpp/arcticdb/pipeline/frame_slice.hpp b/cpp/arcticdb/pipeline/frame_slice.hpp index bd66d085e13..44e74efe570 100644 --- a/cpp/arcticdb/pipeline/frame_slice.hpp +++ b/cpp/arcticdb/pipeline/frame_slice.hpp @@ -34,7 +34,7 @@ struct AxisRange : std::pair { struct Hasher { template - requires std::derived_from + requires std::is_base_of_v std::size_t operator()(const T& r) const { // try to make better use of msb lsb given how F14 is implemented #ifdef _WIN32 diff --git a/cpp/arcticdb/processing/clause.cpp b/cpp/arcticdb/processing/clause.cpp index ec3bc6f50a2..ca958696dd4 100644 --- a/cpp/arcticdb/processing/clause.cpp +++ b/cpp/arcticdb/processing/clause.cpp @@ -1725,7 +1725,18 @@ MergeUpdateClause::MergeUpdateClause( on_(std::move(on)), strategy_(strategy), source_(std::move(source)), - match_on_timeseries_index_(match_on_timeseries_index) {} + match_on_timeseries_index_(match_on_timeseries_index) { + + user_input::check( + on.empty(), "Matching on multiple columns is not supported yet" + ); + user_input::check( + source_->has_index(), "Merge can be perfomed only on timestamp indexed dataframes at the moment" + ); + user_input::check( + strategy_.not_matched_by_target == MergeAction::DO_NOTHING, "Merge cannot perform insertion at the moment" + ); +} std::vector> MergeUpdateClause::structure_for_processing(std::vector& ranges_and_keys ) { @@ -1741,7 +1752,7 @@ std::vector> MergeUpdateClause::structure_for_processing(std bool keep_row_slice = source_->num_rows && strategy_.not_matched_by_target == MergeAction::INSERT && source_->index_value_at(source_row) < time_range.first; if (keep_row_slice) { - source_start_for_row_range_[first_col_slice_in_row->row_range()] = source_row; + source_start_for_row_range_[first_col_slice_in_row->row_range()] = std::pair{source_row, source_row + 1}; } timestamp source_ts = source_->index_value_at(source_row); // TODO: If there are values to be inserted before the first segment this will read it and prepend the values @@ -1752,12 +1763,15 @@ std::vector> MergeUpdateClause::structure_for_processing(std } const bool source_ts_in_segment_range = source_ts >= time_range.first && source_ts < time_range.second; if (!keep_row_slice && source_ts_in_segment_range) { - source_start_for_row_range_[first_col_slice_in_row->row_range()] = source_row; + source_start_for_row_range_[first_col_slice_in_row->row_range()] = std::pair{source_row, source_row + 1}; } keep_row_slice |= source_ts_in_segment_range; while (source_row < source_->num_rows && source_ts >= time_range.first && source_ts < time_range.second) { source_ts = source_->index_value_at(++source_row); } + if (keep_row_slice) { + source_start_for_row_range_.at(first_col_slice_in_row->row_range()).second = source_row; + } row_slices_to_keep[row_slice_idx] = keep_row_slice; const size_t col_slice_count = entities[row_slice_idx].size(); first_col_slice_in_row += col_slice_count; @@ -1796,10 +1810,145 @@ std::vector MergeUpdateClause::process(std::vector&& entity_ gather_entities, std::shared_ptr, std::shared_ptr>( *component_manager_, std::move(entity_ids) ); - + std::vector> matched = filter_index_match(proc); + if (source_->has_segment()) { + update_and_insert(source_->segment(), source_->desc(), proc, matched); + } else { + internal::check( + source_->has_tensors(), "Input frame does not contain neither a segment nor tensors" + ); + update_and_insert>(source_->field_tensors(), source_->desc(), proc, matched); + } return {}; } +template +requires column_data_iterator +class SourceColumnIterator { + public: + SourceColumnIterator(T&& source_data, size_t source_data_row) : + iterator_(std::move(source_data)), + source_data_row_(source_data_row) {} + + const auto& operator*() const { return *iterator_; } + auto operator+=(size_t n) { + std::advance(iterator_, n); + source_data_row_ += n; + return *this; + } + + private: + T iterator_; + size_t source_data_row_; +}; + +template +void MergeUpdateClause::update_and_insert( + const T& source, const StreamDescriptor& source_descriptor, const ProcessingUnit& proc, + std::span> rows_to_update +) const { + const std::vector> target_segments = *proc.segments_; + const std::vector> row_ranges = *proc.row_ranges_; + const std::vector> col_ranges = *proc.col_ranges_; + const auto [source_row_start, source_row_end] = source_start_for_row_range_.at(*row_ranges[0]); + for (size_t segment_idx = 0; segment_idx < target_segments.size(); ++segment_idx) { + const ColRange& col_range = *col_ranges[segment_idx]; + for (size_t column_idx = col_range.first; column_idx < col_range.second; ++column_idx) { + entity::visit_field(source_descriptor.field(column_idx), [&](auto tdt) { + internal::check( + !is_sequence_type(tdt.data_type()), "String columns are not supported in merge update yet" + ); + using TDT = decltype(tdt); + + size_t source_row = source_row_start; + while (source_row < source_row_end && rows_to_update[source_row].empty()) { + ++source_row; + }; + // For each row in the source rows to update contains a list of rows in the target that match this + // row and must be updated. If all rows in the row slice are empty this means that the current row + // slice should not be updated + // TODO: Do not return in case of MergeAction::INSERT + if (source_row >= source_row_end) { + return; + } + size_t target_data_row = rows_to_update[source_row].front(); + // TODO: Implement operator+= because std::advance is linear + auto target_data_it = target_segments[segment_idx]->column(column_idx).data().begin(); + std::advance(target_data_it, target_data_row); + if constexpr (std::is_same_v) { + size_t source_data_row = source_row_start; + // TODO: Implement operator+= because std::advance is linear + auto source_data_it = source.column(column_idx).data().template begin(); + std::advance(source_data_it, source_data_row); + while (source_row < source_row_end) { + std::span rows_to_update_for_source_row = rows_to_update[source_row]; + if (rows_to_update_for_source_row.empty()) { + ++source_row; + continue; + } + std::advance(source_data_it, source_row - source_data_row); + source_data_row = source_row; + const auto& source_row_value = *source_data_it; + for (const size_t target_row_to_update : rows_to_update_for_source_row) { + // TODO: Implement operator+= because std::advance is linear + std::advance(target_data_it, target_row_to_update - target_data_row); + *target_data_it = source_row_value; + target_data_row = target_row_to_update; + } + } + } else if constexpr (std::is_same_v>) { + using RawType = typename TDT::DataTypeTag::raw_type; + TypedTensor source_data{source[column_idx]}; + while (source_row < source_row_end) { + std::span rows_to_update_for_source_row = rows_to_update[source_row]; + if (rows_to_update_for_source_row.empty()) { + ++source_row; + continue; + } + const auto& source_row_value = source_data.at(source_row); + for (const size_t target_row_to_update : rows_to_update_for_source_row) { + // TODO: Implement operator+= because std::advance is linear + std::advance(target_data_it, target_row_to_update - target_data_row); + *target_data_it = source_row_value; + target_data_row = target_row_to_update; + } + } + } + }); + } + } +} +template void MergeUpdateClause:: + update_and_insert(const SegmentInMemory&, const StreamDescriptor&, const ProcessingUnit&, std::span>) + const; +template void MergeUpdateClause:: + update_and_insert(const std::vector&, const StreamDescriptor&, const ProcessingUnit&, std::span>) + const; + +std::vector> MergeUpdateClause::filter_index_match(const ProcessingUnit& proc) const { + using IndexType = ScalarTagType>; + const std::vector> target_segments = *proc.segments_; + const std::vector> row_ranges = *proc.row_ranges_; + auto [source_row, source_row_end] = source_start_for_row_range_.at(*row_ranges[0]); + const size_t source_rows_in_row_slice = source_row_end - source_row; + std::vector> matched_rows(source_rows_in_row_slice); + ColumnData target_index = target_segments[0]->column(0).data(); + util::BitSet matched(target_segments[0]->row_count()); + auto target_index_it = target_index.cbegin(); + const auto target_index_end = target_index.cend(); + while (target_index_it != target_index_end && source_row < source_row_end) { + const timestamp source_ts = source_->index_value_at(source_row); + target_index_it = std::lower_bound(target_index_it, target_index_end, source_ts); + while (*target_index_it == source_ts) { + const size_t target_row = std::distance(target_index.cbegin(), target_index_it); + matched_rows[source_row].push_back(target_row); + ++target_index_it; + } + source_row++; + } + return matched_rows; +} + const ClauseInfo& MergeUpdateClause::clause_info() const { return clause_info_; } void MergeUpdateClause::set_processing_config(const ProcessingConfig&) {} @@ -1808,7 +1957,16 @@ void MergeUpdateClause::set_component_manager(std::shared_ptr component_manager_ = std::move(component_manager); } -OutputSchema MergeUpdateClause::modify_schema(OutputSchema&& output_schema) const { return output_schema; } +OutputSchema MergeUpdateClause::modify_schema(OutputSchema&& output_schema) const { + internal::check( + output_schema.stream_descriptor() == source_->desc(), + "Cannot perform merge update when the source and target schema are not the same. Source schema: {}, target " + "schema: {}", + source_->desc(), + output_schema.stream_descriptor() + ); + return output_schema; +} OutputSchema MergeUpdateClause::join_schemas(std::vector&&) const { util::raise_rte("MergeUpdateClause::join_schemas should never be called"); diff --git a/cpp/arcticdb/processing/clause.hpp b/cpp/arcticdb/processing/clause.hpp index 5b507f8db10..2f754a5aabb 100644 --- a/cpp/arcticdb/processing/clause.hpp +++ b/cpp/arcticdb/processing/clause.hpp @@ -877,7 +877,17 @@ struct MergeUpdateClause { [[nodiscard]] std::string to_string() const; private: - ankerl::unordered_dense::map source_start_for_row_range_; + template + void + update_and_insert(const T&, const StreamDescriptor&, const ProcessingUnit&, std::span>) + const; + + /// @return Vector of size equal to the number of source data rows that are withing the rows lice being processed. + /// Each element is a vector of the rows from the target data that has the same index as the corresponding source + /// row + std::vector> filter_index_match(const ProcessingUnit& proc) const; + + ankerl::unordered_dense::map, RowRange::Hasher> source_start_for_row_range_; }; } // namespace arcticdb From 38ac5b59561679f01eef2098a36a11e56a3e7c1c Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Mon, 10 Nov 2025 16:57:36 +0200 Subject: [PATCH 25/46] Unify iteration over source columns --- cpp/arcticdb/column_store/column_data.hpp | 12 -- cpp/arcticdb/entity/native_tensor.hpp | 2 + cpp/arcticdb/processing/clause.cpp | 142 ++++++++++++---------- env.sh | 18 +++ 4 files changed, 99 insertions(+), 75 deletions(-) create mode 100644 env.sh diff --git a/cpp/arcticdb/column_store/column_data.hpp b/cpp/arcticdb/column_store/column_data.hpp index bf22983849b..a958bb16bb9 100644 --- a/cpp/arcticdb/column_store/column_data.hpp +++ b/cpp/arcticdb/column_store/column_data.hpp @@ -453,16 +453,4 @@ struct ColumnData { const util::BitMagic* bit_vector_; }; -template -struct is_column_data_iterator : std::false_type {}; - -template -struct is_column_data_iterator> : std::true_type {}; - -template -inline constexpr bool is_column_data_iterator_v = is_column_data_iterator>::value; - -template -concept column_data_iterator = is_column_data_iterator_v; - } // namespace arcticdb diff --git a/cpp/arcticdb/entity/native_tensor.hpp b/cpp/arcticdb/entity/native_tensor.hpp index c43a7f4b279..8e720d8ac8c 100644 --- a/cpp/arcticdb/entity/native_tensor.hpp +++ b/cpp/arcticdb/entity/native_tensor.hpp @@ -171,6 +171,8 @@ ssize_t byte_offset_impl(const stride_t* strides, ssize_t i, Ix... index) { // TODO is the conversion to a typed tensor really necessary for the codec part? template struct TypedTensor : public NativeTensor { + using raw_type = T; + constexpr static size_t itemsize() { return sizeof(T); } std::array f_style_strides() { diff --git a/cpp/arcticdb/processing/clause.cpp b/cpp/arcticdb/processing/clause.cpp index ca958696dd4..b3b94eb0abf 100644 --- a/cpp/arcticdb/processing/clause.cpp +++ b/cpp/arcticdb/processing/clause.cpp @@ -1718,6 +1718,66 @@ OutputSchema WriteClause::join_schemas(std::vector&&) const { std::string WriteClause::to_string() const { return "Write"; } +template +class SourceView {}; + +template +requires(std::same_as && util::instantiation_of) +class SourceView { + using Iterator = decltype(std::declval().data().template begin()); + + public: + SourceView(const T& source, size_t source_data_row) : + it_(source.data().template begin()), + source_data_row_(source_data_row) { + std::advance(it_, source_data_row); + } + + void set_row(const size_t new_row) { + debug::check( + new_row >= source_data_row_, "Cannot move SourceColumnIterator backwards" + ); + // TODO: Implement operator+= because std::advance is linear + std::advance(it_, new_row - source_data_row_); + source_data_row_ = new_row; + } + + const TDT::DataTypeTag::raw_type& operator*() const { return *it_; } + + private: + Iterator it_; + size_t source_data_row_; +}; + +template +requires std::same_as && util::instantiation_of +class SourceView { + using Iterator = const TDT::DataTypeTag::raw_type*; + + public: + SourceView(const T& source, const size_t source_data_row) : source_(source), source_data_row_(source_data_row) {} + + void set_row(const size_t new_row) { source_data_row_ = new_row; } + + const TDT::DataTypeTag::raw_type& operator*() const { return source_.at(source_data_row_); } + + private: + TypedTensor source_; + size_t source_data_row_; +}; + +template +requires util::any_of> && util::instantiation_of +auto get_source_column_iterator(const T& source, size_t column_index) { + if constexpr (std::is_same_v) { + return SourceView(source.column(column_index), 0); + } else if constexpr (std::is_same_v>) { + return SourceView(source[column_index], 0); + } else { + static_assert(sizeof(T) == 0, "Invalid type"); + } +} + MergeUpdateClause::MergeUpdateClause( std::vector&& on, MergeStrategy strategy, std::shared_ptr source, bool match_on_timeseries_index @@ -1806,10 +1866,9 @@ std::vector MergeUpdateClause::process(std::vector&& entity_ if (entity_ids.empty()) { return {}; } - const auto proc = - gather_entities, std::shared_ptr, std::shared_ptr>( - *component_manager_, std::move(entity_ids) - ); + auto proc = gather_entities, std::shared_ptr, std::shared_ptr>( + *component_manager_, std::move(entity_ids) + ); std::vector> matched = filter_index_match(proc); if (source_->has_segment()) { update_and_insert(source_->segment(), source_->desc(), proc, matched); @@ -1819,29 +1878,9 @@ std::vector MergeUpdateClause::process(std::vector&& entity_ ); update_and_insert>(source_->field_tensors(), source_->desc(), proc, matched); } - return {}; + return push_entities(*component_manager_, std::move(proc)); } -template -requires column_data_iterator -class SourceColumnIterator { - public: - SourceColumnIterator(T&& source_data, size_t source_data_row) : - iterator_(std::move(source_data)), - source_data_row_(source_data_row) {} - - const auto& operator*() const { return *iterator_; } - auto operator+=(size_t n) { - std::advance(iterator_, n); - source_data_row_ += n; - return *this; - } - - private: - T iterator_; - size_t source_data_row_; -}; - template void MergeUpdateClause::update_and_insert( const T& source, const StreamDescriptor& source_descriptor, const ProcessingUnit& proc, @@ -1872,46 +1911,24 @@ void MergeUpdateClause::update_and_insert( return; } size_t target_data_row = rows_to_update[source_row].front(); - // TODO: Implement operator+= because std::advance is linear + auto target_data_it = target_segments[segment_idx]->column(column_idx).data().begin(); + // TODO: Implement operator+= because std::advance is linear std::advance(target_data_it, target_data_row); - if constexpr (std::is_same_v) { - size_t source_data_row = source_row_start; - // TODO: Implement operator+= because std::advance is linear - auto source_data_it = source.column(column_idx).data().template begin(); - std::advance(source_data_it, source_data_row); - while (source_row < source_row_end) { - std::span rows_to_update_for_source_row = rows_to_update[source_row]; - if (rows_to_update_for_source_row.empty()) { - ++source_row; - continue; - } - std::advance(source_data_it, source_row - source_data_row); - source_data_row = source_row; - const auto& source_row_value = *source_data_it; - for (const size_t target_row_to_update : rows_to_update_for_source_row) { - // TODO: Implement operator+= because std::advance is linear - std::advance(target_data_it, target_row_to_update - target_data_row); - *target_data_it = source_row_value; - target_data_row = target_row_to_update; - } + SourceView source_column_view = get_source_column_iterator(source, column_idx); + while (source_row < source_row_end) { + std::span rows_to_update_for_source_row = rows_to_update[source_row]; + if (rows_to_update_for_source_row.empty()) { + ++source_row; + continue; } - } else if constexpr (std::is_same_v>) { - using RawType = typename TDT::DataTypeTag::raw_type; - TypedTensor source_data{source[column_idx]}; - while (source_row < source_row_end) { - std::span rows_to_update_for_source_row = rows_to_update[source_row]; - if (rows_to_update_for_source_row.empty()) { - ++source_row; - continue; - } - const auto& source_row_value = source_data.at(source_row); - for (const size_t target_row_to_update : rows_to_update_for_source_row) { - // TODO: Implement operator+= because std::advance is linear - std::advance(target_data_it, target_row_to_update - target_data_row); - *target_data_it = source_row_value; - target_data_row = target_row_to_update; - } + source_column_view.set_row(source_row); + const auto& source_row_value = *source_column_view; + for (const size_t target_row_to_update : rows_to_update_for_source_row) { + // TODO: Implement operator+= because std::advance is linear + std::advance(target_data_it, target_row_to_update - target_data_row); + *target_data_it = source_row_value; + target_data_row = target_row_to_update; } } }); @@ -1933,7 +1950,6 @@ std::vector> MergeUpdateClause::filter_index_match(const Pro const size_t source_rows_in_row_slice = source_row_end - source_row; std::vector> matched_rows(source_rows_in_row_slice); ColumnData target_index = target_segments[0]->column(0).data(); - util::BitSet matched(target_segments[0]->row_count()); auto target_index_it = target_index.cbegin(); const auto target_index_end = target_index.cend(); while (target_index_it != target_index_end && source_row < source_row_end) { diff --git a/env.sh b/env.sh new file mode 100644 index 00000000000..35bae234239 --- /dev/null +++ b/env.sh @@ -0,0 +1,18 @@ +export ARCTICDB_PERSISTENT_STORAGE_TESTS=1 +export ARCTICDB_PERSISTENT_STORAGE_STRATEGY_BRANCH=blah +export ARCTICDB_PERSISTENT_STORAGE_SHARED_PATH_PREFIX=blah +export ARCTICDB_PERSISTENT_STORAGE_UNIQUE_PATH_PREFIX=blah +export ARCTICDB_REAL_S3_BUCKET=blah +export ARCTICDB_REAL_S3_ENDPOINT=https://s3.eu-west-1.amazonaws.com/ +export ARCTICDB_REAL_S3_REGION=eu-west-1 +export ARCTICDB_REAL_S3_CLEAR=0 +export ARCTICDB_REAL_S3_ACCESS_KEY=blah +export ARCTICDB_REAL_S3_SECRET_KEY=blah + +export ARCTICDB_REAL_AZURE_CONNECTION_STRING=blah +export ARCTICDB_REAL_AZURE_CONTAINER=blah + + +python -m asv run -v --show-stderr master^! --bench .*Resample.peakmem.*mean.* +python -m asv run -v --show-stderr HEAD^! --bench .*Resample.peakmem.*mean.* +python -m asv compare master HEAD From ab71903d908112a477ad71e7bf9f7cbb9d53353a Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Tue, 11 Nov 2025 15:13:53 +0200 Subject: [PATCH 26/46] Add utility functions for generating columns, native tensors and segments --- cpp/arcticdb/CMakeLists.txt | 1 + cpp/arcticdb/column_store/column.hpp | 10 ++ .../column_store/memory_segment_impl.hpp | 37 ++++++ cpp/arcticdb/entity/native_tensor.hpp | 39 ++++++ cpp/arcticdb/entity/types.hpp | 2 + cpp/arcticdb/pipeline/input_frame.cpp | 10 ++ cpp/arcticdb/pipeline/input_frame.hpp | 5 + .../processing/test/test_merge_update.cpp | 52 ++++++++ .../processing/test/test_resample.cpp | 118 ++++++++++++------ .../test/test_unsorted_aggregation.cpp | 4 +- cpp/arcticdb/util/test/test_utils.hpp | 16 --- cpp/arcticdb/util/type_traits.hpp | 9 ++ 12 files changed, 249 insertions(+), 54 deletions(-) create mode 100644 cpp/arcticdb/processing/test/test_merge_update.cpp diff --git a/cpp/arcticdb/CMakeLists.txt b/cpp/arcticdb/CMakeLists.txt index 47d3d9e852a..327c5b3ad1e 100644 --- a/cpp/arcticdb/CMakeLists.txt +++ b/cpp/arcticdb/CMakeLists.txt @@ -982,6 +982,7 @@ if(${TEST}) processing/test/test_signed_unsigned_comparison.cpp processing/test/test_type_comparison.cpp processing/test/test_unsorted_aggregation.cpp + processing/test/test_merge_update.cpp storage/test/test_local_storages.cpp storage/test/test_memory_storage.cpp storage/test/test_s3_storage.cpp diff --git a/cpp/arcticdb/column_store/column.hpp b/cpp/arcticdb/column_store/column.hpp index 42c77d3ee13..d8149c16c35 100644 --- a/cpp/arcticdb/column_store/column.hpp +++ b/cpp/arcticdb/column_store/column.hpp @@ -230,6 +230,16 @@ class Column { Column(TypeDescriptor type, size_t expected_rows, AllocationType presize, Sparsity allow_sparse, OutputFormat output_format, DataTypeMode mode); + template + requires std::ranges::contiguous_range + static Column create_dense_column(const Input& data, const TypeDescriptor& type) { + constexpr size_t element_size = sizeof(std::ranges::range_value_t); + Column result(type, data.size(), AllocationType::PRESIZED, Sparsity::NOT_PERMITTED); + std::memcpy(result.ptr(), data.data(), data.size() * element_size); + result.set_row_data(data.size()); + return result; + } + ARCTICDB_MOVE_ONLY_DEFAULT(Column) friend bool operator==(const Column& left, const Column& right); diff --git a/cpp/arcticdb/column_store/memory_segment_impl.hpp b/cpp/arcticdb/column_store/memory_segment_impl.hpp index 023e7294b92..5080fa0b934 100644 --- a/cpp/arcticdb/column_store/memory_segment_impl.hpp +++ b/cpp/arcticdb/column_store/memory_segment_impl.hpp @@ -16,6 +16,8 @@ #include #include #include +#include +#include namespace arcticdb { @@ -278,6 +280,41 @@ class SegmentInMemoryImpl { OutputFormat output_format, DataTypeMode mode ); + template + requires std::ranges::range && std::ranges::contiguous_range> + SegmentInMemoryImpl create_dense_segment(const T& column_data, const StreamDescriptor& descriptor) { + if (std::ranges::begin(column_data) == std::ranges::end(column_data)) { + return SegmentInMemoryImpl(); + } + + const size_t expected_column_size = column_data.begin()->first.size(); + constexpr static AllocationType allocation_type = AllocationType::PRESIZED; + constexpr static Sparsity sparsity = Sparsity::NOT_PERMITTED; + SegmentInMemoryImpl result(descriptor, expected_column_size, allocation_type, sparsity); + for (auto const& [column_index, contiguous_column_data] : folly::enumerate(column_data)) { + if (is_sequence_type(descriptor.field(column_index).type().data_type())) { + for (std::string_view str : contiguous_column_data) { + result.set_string_at(column_index, str); + } + } else { + using ValueType = std::decay_t>; + Column& column = result.column(column_index); + const size_t row_count = std::ranges::size(contiguous_column_data); + internal::check( + row_count == expected_column_size, + "When creating a dense segment from a range of dense column data, all columns must have " + "the same size. Column[0] has size {} Column[{}] has size: {}", + expected_column_size, + column_index, + row_count + ); + std::memcpy(column.ptr(), row_count, row_count * sizeof(ValueType)); + result.set_row_data(row_count); + } + } + return result; + } + ~SegmentInMemoryImpl(); iterator begin(); diff --git a/cpp/arcticdb/entity/native_tensor.hpp b/cpp/arcticdb/entity/native_tensor.hpp index 8e720d8ac8c..6d53f1c6a5d 100644 --- a/cpp/arcticdb/entity/native_tensor.hpp +++ b/cpp/arcticdb/entity/native_tensor.hpp @@ -11,6 +11,7 @@ #include #include #include +#include // for std::accumulate #include @@ -66,6 +67,17 @@ struct NativeTensor { } } + template + requires std::ranges::contiguous_range + static NativeTensor one_dimensional_tensor(const T& data, const DataType data_type) { + using ValueType = std::ranges::range_value_t; + constexpr static size_t element_size = sizeof(ValueType); + constexpr shape_t shapes = 1; + constexpr stride_t strides = element_size; + const int64_t byte_size = data.size() * element_size; + return NativeTensor{byte_size, 1, &shapes, &strides, data_type, element_size, std::ranges::data(data), 1}; + } + NativeTensor(const NativeTensor& other) : nbytes_(other.nbytes_), ndim_(other.ndim_), @@ -159,6 +171,33 @@ struct NativeTensor { int expanded_dim_; }; +/// Creates a vector of NativeTensors from pairs of 1D contiguous data and DataType. +/// Each input must be a pair-like object where `first` is a contiguous range and `second` is a DataType. +/// The resulting tensors can be passed to InputFrame. +/// @tparam Input Parameter pack of pair-like types satisfying the contiguous_range requirement +/// @param arrays Variadic pairs of (contiguous_range, DataType) +/// @return Vector of one-dimensional NativeTensors +template +requires(... && util::contiguous_type_tagged_data) +std::vector create_one_dimensional_tensors(const Input&... arrays) { + std::vector tensors; + tensors.reserve(sizeof...(arrays)); + (tensors.emplace_back(NativeTensor::one_dimensional_tensor(arrays.first, arrays.second.data_type())), ...); + return tensors; +} + +template +requires util::contiguous_type_tagged_data +std::vector create_one_dimensional_tensors(std::span contiguous_columns) { + std::vector tensors; + for (const T& contiguous_column_data : contiguous_columns) { + tensors.emplace_back(NativeTensor::one_dimensional_tensor( + contiguous_column_data.first, contiguous_column_data.second.data_type() + )); + } + return tensors; +} + template ssize_t byte_offset_impl(const stride_t*) { return 0; diff --git a/cpp/arcticdb/entity/types.hpp b/cpp/arcticdb/entity/types.hpp index 872ef3140c0..c06aeaf9c0c 100644 --- a/cpp/arcticdb/entity/types.hpp +++ b/cpp/arcticdb/entity/types.hpp @@ -416,6 +416,8 @@ struct TypeDescriptor { TypeDescriptor() : data_type_(DataType::UINT8), dimension_(Dimension::Dim0) {} + static TypeDescriptor scalar_type(DataType type) { return TypeDescriptor(type, Dimension::Dim0); } + ARCTICDB_MOVE_COPY_DEFAULT(TypeDescriptor) template diff --git a/cpp/arcticdb/pipeline/input_frame.cpp b/cpp/arcticdb/pipeline/input_frame.cpp index b26fa085a2f..da21056dbae 100644 --- a/cpp/arcticdb/pipeline/input_frame.cpp +++ b/cpp/arcticdb/pipeline/input_frame.cpp @@ -16,6 +16,16 @@ namespace arcticdb::pipelines { InputFrame::InputFrame() : index(stream::empty_index()) {} +InputFrame::InputFrame(SegmentInMemory&& seg) : index(stream::empty_index()) { set_segment(std::move(seg)); } + +InputFrame::InputFrame( + StreamDescriptor&& desc, std::vector&& field_tensors, + std::optional&& index_tensor +) : + index(stream::empty_index()) { + set_from_tensors(std::move(desc), std::move(field_tensors), std::move(index_tensor)); +} + void InputFrame::set_segment(SegmentInMemory&& seg) { num_rows = seg.row_count(); util::check(norm_meta.has_experimental_arrow(), "Unexpected non-Arrow norm metadata provided with Arrow data"); diff --git a/cpp/arcticdb/pipeline/input_frame.hpp b/cpp/arcticdb/pipeline/input_frame.hpp index c36abc0528a..f369678ee16 100644 --- a/cpp/arcticdb/pipeline/input_frame.hpp +++ b/cpp/arcticdb/pipeline/input_frame.hpp @@ -29,6 +29,11 @@ concept ValidIndex = util::any_of< struct InputFrame { public: InputFrame(); + InputFrame(SegmentInMemory&& seg); + InputFrame( + StreamDescriptor&& desc, std::vector&& field_tensors, + std::optional&& index_tensor = std::nullopt + ); void set_segment(SegmentInMemory&& seg); void set_from_tensors( StreamDescriptor&& desc, std::vector&& field_tensors, diff --git a/cpp/arcticdb/processing/test/test_merge_update.cpp b/cpp/arcticdb/processing/test/test_merge_update.cpp new file mode 100644 index 00000000000..79914386409 --- /dev/null +++ b/cpp/arcticdb/processing/test/test_merge_update.cpp @@ -0,0 +1,52 @@ +/* Copyright 2025 Man Group Operations Limited + * + * Use of this software is governed by the Business Source License 1.1 included in the file licenses/BSL.txt. + * + * As of the Change Date specified in that file, in accordance with the Business Source License, use of this software + * will be governed by the Apache License, version 2.0. + */ + +#include +#include +#include +#include + +using namespace arcticdb; + +constexpr static std::array non_string_fields = { + FieldRef(TypeDescriptor(DataType::INT8, Dimension::Dim0), "int8"), + FieldRef(TypeDescriptor(DataType::UINT32, Dimension::Dim0), "uint32"), + FieldRef(TypeDescriptor(DataType::BOOL8, Dimension::Dim0), "bool8"), + FieldRef(TypeDescriptor(DataType::FLOAT32, Dimension::Dim0), "float32"), + FieldRef(TypeDescriptor(DataType::NANOSECONDS_UTC64, Dimension::Dim0), "timestamp") +}; + +TEST(MergeUpdateUpdateTimeseries, SourceIndexMatchesAllSegments) { + using stream::TimeseriesIndex; + + std::vector on{}; + constexpr static auto strategy = + MergeStrategy{.matched = MergeAction::UPDATE, .not_matched_by_target = MergeAction::DO_NOTHING}; + StreamDescriptor source_descriptor = + TimeseriesIndex::default_index().create_stream_descriptor("Source", non_string_fields); + constexpr std::array source_index{1, 12, 25}; + constexpr std::array source_int8{10, 20, 30}; + constexpr std::array source_uint32{100, 200, 300}; + constexpr std::array source_bool{true, false, true}; + constexpr std::array source_float{11.1f, 22.2f, 33.3f}; + constexpr std::array source_timestamp{1000, 2000, 3000}; + InputFrame source( + std::move(source_descriptor), + create_one_dimensional_tensors( + std::pair{source_int8, TypeDescriptor::scalar_type(DataType::INT8)}, + std::pair{source_uint32, TypeDescriptor::scalar_type(DataType::UINT32)}, + std::pair{source_bool, TypeDescriptor::scalar_type(DataType::BOOL8)}, + std::pair{source_float, TypeDescriptor::scalar_type(DataType::FLOAT32)}, + std::pair{source_timestamp, TypeDescriptor::scalar_type(DataType::NANOSECONDS_UTC64)} + ), + NativeTensor::one_dimensional_tensor(source_index, DataType::NANOSECONDS_UTC64) + ); + [[maybe_unused]] MergeUpdateClause clause( + std::move(on), strategy, std::make_shared(std::move(source)), true + ); +} \ No newline at end of file diff --git a/cpp/arcticdb/processing/test/test_resample.cpp b/cpp/arcticdb/processing/test/test_resample.cpp index 1d335cd4a92..9cf66c58c2c 100644 --- a/cpp/arcticdb/processing/test/test_resample.cpp +++ b/cpp/arcticdb/processing/test/test_resample.cpp @@ -476,19 +476,25 @@ TYPED_TEST(SortedAggregatorSparseStructure, NoMissingInputColumnsProducesDenseCo constexpr ResampleBoundary label = TypeParam::label; constexpr static std::array bucket_boundaries{0, 10, 20, 30, 40, 50}; constexpr static std::array output_index = generate_labels(bucket_boundaries, label); - Column output_index_column = create_dense_column(output_index); + Column output_index_column = Column::create_dense_column(output_index, IndexTDT::type_descriptor()); const std::array input_index_columns{ - std::make_shared(create_dense_column(std::array{1, 2, 3})), - std::make_shared(create_dense_column(std::array{11, 21, 31, 41})) + std::make_shared( + Column::create_dense_column(std::array{1, 2, 3}, IndexTDT::type_descriptor()) + ), + std::make_shared( + Column::create_dense_column(std::array{11, 21, 31, 41}, IndexTDT::type_descriptor()) + ) }; const std::array input_agg_columns{ std::make_optional(ColumnWithStrings{ - create_dense_column>>(std::array{0, 5, 6}), + Column::create_dense_column(std::array{0, 5, 6}, TypeDescriptor::scalar_type(DataType::INT32)), nullptr, "col1" }), std::make_optional(ColumnWithStrings{ - create_dense_column>>(std::array{10, 35, 56, 1, 2}), + Column::create_dense_column( + std::array{10, 35, 56, 1, 2}, TypeDescriptor::scalar_type(DataType::INT32) + ), nullptr, "col1" }) @@ -526,14 +532,18 @@ TYPED_TEST(SortedAggregatorSparseStructure, FirstColumnExistSecondIsMissing) { constexpr ResampleBoundary label = TypeParam::label; constexpr static std::array bucket_boundaries{0, 10, 20}; constexpr static std::array output_index = generate_labels(bucket_boundaries, label); - const Column output_index_column = create_dense_column(output_index); + const Column output_index_column = Column::create_dense_column(output_index, IndexTDT::type_descriptor()); const std::array input_index_columns{ - std::make_shared(create_dense_column(std::array{0, 2, 3})), - std::make_shared(create_dense_column(std::array{11, 21, 22, 24})) + std::make_shared( + Column::create_dense_column(std::array{0, 2, 3}, IndexTDT::type_descriptor()) + ), + std::make_shared( + Column::create_dense_column(std::array{11, 21, 22, 24}, IndexTDT::type_descriptor()) + ) }; const std::array input_agg_columns{ std::make_optional(ColumnWithStrings{ - create_dense_column>>(std::array{0, 5, 6}), + Column::create_dense_column(std::array{0, 5, 6}, TypeDescriptor::scalar_type(DataType::INT32)), nullptr, "col1" }), @@ -562,18 +572,22 @@ TYPED_TEST(SortedAggregatorSparseStructure, FirstColumnExistWithValueOnRightBoun constexpr ResampleBoundary closed = TypeParam::SortedAggregator::closed; const Column output_index_column = []() { if constexpr (label == ResampleBoundary::LEFT) { - return create_dense_column(std::array{0, 10, 30}); + return Column::create_dense_column(std::array{0, 10, 30}, IndexTDT::type_descriptor()); } else { - return create_dense_column(std::array{10, 20, 40}); + return Column::create_dense_column(std::array{10, 20, 40}, IndexTDT::type_descriptor()); } }(); const std::array input_index_columns{ - std::make_shared(create_dense_column(std::array{0, 2, 10})), - std::make_shared(create_dense_column(std::array{35, 36})) + std::make_shared( + Column::create_dense_column(std::array{0, 2, 10}, IndexTDT::type_descriptor()) + ), + std::make_shared( + Column::create_dense_column(std::array{35, 36}, IndexTDT::type_descriptor()) + ) }; const std::array input_agg_columns{ std::make_optional(ColumnWithStrings{ - create_dense_column>>(std::array{0, 5, 6}), + Column::create_dense_column(std::array{0, 5, 6}, TypeDescriptor::scalar_type(DataType::INT32)), nullptr, "col1" }), @@ -609,14 +623,18 @@ TYPED_TEST(SortedAggregatorSparseStructure, ReturnDenseInCaseOutputIndexIsFilled constexpr ResampleBoundary label = TypeParam::label; constexpr static std::array bucket_boundaries{0, 10, 20}; constexpr static std::array output_index = generate_labels(bucket_boundaries, label); - const Column output_index_column = create_dense_column(output_index); + const Column output_index_column = Column::create_dense_column(output_index, IndexTDT::type_descriptor()); const std::array input_index_columns{ - std::make_shared(create_dense_column(std::array{0, 2, 12})), - std::make_shared(create_dense_column(std::array{15, 16, 18, 20})) + std::make_shared( + Column::create_dense_column(std::array{0, 2, 12}, IndexTDT::type_descriptor()) + ), + std::make_shared( + Column::create_dense_column(std::array{15, 16, 18, 20}, IndexTDT::type_descriptor()) + ) }; const std::array input_agg_columns{ std::make_optional(ColumnWithStrings{ - create_dense_column>>(std::array{0, 5, 6}), + Column::create_dense_column(std::array{0, 5, 6}, TypeDescriptor::scalar_type(DataType::INT32)), nullptr, "col1" }), @@ -640,15 +658,19 @@ TYPED_TEST(SortedAggregatorSparseStructure, ReturnDenseInCaseOutputIndexIsFilled constexpr ResampleBoundary label = TypeParam::label; constexpr static std::array bucket_boundaries{0, 10, 20}; constexpr static std::array output_index = generate_labels(bucket_boundaries, label); - const Column output_index_column = create_dense_column(output_index); + const Column output_index_column = Column::create_dense_column(output_index, IndexTDT::type_descriptor()); const std::array input_index_columns{ - std::make_shared(create_dense_column(std::array{0, 2, 5})), - std::make_shared(create_dense_column(std::array{7, 8, 9, 15})) + std::make_shared( + Column::create_dense_column(std::array{0, 2, 5}, IndexTDT::type_descriptor()) + ), + std::make_shared( + Column::create_dense_column(std::array{7, 8, 9, 15}, IndexTDT::type_descriptor()) + ) }; const std::array input_agg_columns{ std::optional{}, std::make_optional(ColumnWithStrings{ - create_dense_column>>(std::array{0, 5, 6, 5}), + Column::create_dense_column(std::array{0, 5, 6, 5}, TypeDescriptor::scalar_type(DataType::INT32)), nullptr, "col1" }), @@ -669,15 +691,19 @@ TYPED_TEST(SortedAggregatorSparseStructure, FirstColumnIsMissing) { constexpr ResampleBoundary label = TypeParam::label; constexpr static std::array bucket_boundaries{0, 10, 20}; constexpr static std::array output_index = generate_labels(bucket_boundaries, label); - const Column output_index_column = create_dense_column(output_index); + const Column output_index_column = Column::create_dense_column(output_index, IndexTDT::type_descriptor()); const std::array input_index_columns{ - std::make_shared(create_dense_column(std::array{0, 2, 3})), - std::make_shared(create_dense_column(std::array{11, 15, 16, 17})) + std::make_shared( + Column::create_dense_column(std::array{0, 2, 3}, IndexTDT::type_descriptor()) + ), + std::make_shared( + Column::create_dense_column(std::array{11, 15, 16, 17}, IndexTDT::type_descriptor()) + ) }; const std::array input_agg_columns{ std::optional{}, std::make_optional(ColumnWithStrings{ - create_dense_column>>(std::array{0, 5, 6, 5}), + Column::create_dense_column(std::array{0, 5, 6, 5}, TypeDescriptor::scalar_type(DataType::INT32)), nullptr, "col1" }), @@ -705,20 +731,30 @@ TYPED_TEST(SortedAggregatorSparseStructure, ThreeSegmentsInABucketMiddleIsMissin constexpr ResampleBoundary label = TypeParam::label; constexpr static std::array bucket_boundaries{0, 10}; constexpr static std::array output_index = generate_labels(bucket_boundaries, label); - const Column output_index_column = create_dense_column(output_index); + const Column output_index_column = Column::create_dense_column(output_index, IndexTDT::type_descriptor()); const std::array input_index_columns{ - std::make_shared(create_dense_column(std::array{0, 1})), - std::make_shared(create_dense_column(std::array{2})), - std::make_shared(create_dense_column(std::array{3})) + std::make_shared( + Column::create_dense_column(std::array{0, 1}, IndexTDT::type_descriptor()) + ), + std::make_shared( + Column::create_dense_column(std::array{2}, IndexTDT::type_descriptor()) + ), + std::make_shared( + Column::create_dense_column(std::array{3}, IndexTDT::type_descriptor()) + ) }; const std::array input_agg_columns{ std::make_optional(ColumnWithStrings{ - create_dense_column>>(std::array{1, 2}), nullptr, "col1" + Column::create_dense_column(std::array{1, 2}, TypeDescriptor::scalar_type(DataType::INT32)), + nullptr, + "col1" }), std::optional{}, std::make_optional(ColumnWithStrings{ - create_dense_column>>(std::array{3, 4}), nullptr, "col1" + Column::create_dense_column(std::array{3, 4}, TypeDescriptor::scalar_type(DataType::INT32)), + nullptr, + "col1" }), }; const std::optional output = aggregator.generate_resampling_output_column( @@ -737,17 +773,25 @@ TYPED_TEST(SortedAggregatorSparseStructure, ThreeSegmentsInABuckeOnlyMiddleIsPre constexpr ResampleBoundary label = TypeParam::label; constexpr static std::array bucket_boundaries{0, 10}; constexpr static std::array output_index = generate_labels(bucket_boundaries, label); - const Column output_index_column = create_dense_column(output_index); + const Column output_index_column = Column::create_dense_column(output_index, IndexTDT::type_descriptor()); const std::array input_index_columns{ - std::make_shared(create_dense_column(std::array{0, 1})), - std::make_shared(create_dense_column(std::array{2})), - std::make_shared(create_dense_column(std::array{3})) + std::make_shared( + Column::create_dense_column(std::array{0, 1}, IndexTDT::type_descriptor()) + ), + std::make_shared( + Column::create_dense_column(std::array{2}, IndexTDT::type_descriptor()) + ), + std::make_shared( + Column::create_dense_column(std::array{3}, IndexTDT::type_descriptor()) + ) }; const std::array input_agg_columns{ std::optional{}, std::make_optional(ColumnWithStrings{ - create_dense_column>>(std::array{1}), nullptr, "col1" + Column::create_dense_column(std::array{1}, TypeDescriptor::scalar_type(DataType::INT32)), + nullptr, + "col1" }), std::optional{} }; diff --git a/cpp/arcticdb/processing/test/test_unsorted_aggregation.cpp b/cpp/arcticdb/processing/test/test_unsorted_aggregation.cpp index 704aa93d141..6f40effa17e 100644 --- a/cpp/arcticdb/processing/test/test_unsorted_aggregation.cpp +++ b/cpp/arcticdb/processing/test/test_unsorted_aggregation.cpp @@ -142,7 +142,9 @@ TEST_P(AggregationResult, Mean) { } else { ASSERT_EQ(data.size(), 0); } - const ColumnWithStrings input(create_dense_column(data), nullptr, "input"); + const ColumnWithStrings input( + Column::create_dense_column(data, InputDataTypeTag::type_descriptor()), nullptr, "input" + ); aggregator_data.aggregate(input, groups, group_count); const SegmentInMemory result = aggregator_data.finalize(ColumnName{"output"}, false, group_count); ASSERT_EQ(result.field(0).type(), make_scalar_type(OutputDataTypeTag::data_type())); diff --git a/cpp/arcticdb/util/test/test_utils.hpp b/cpp/arcticdb/util/test/test_utils.hpp index 9c5c6aba62a..9f89630fe35 100644 --- a/cpp/arcticdb/util/test/test_utils.hpp +++ b/cpp/arcticdb/util/test/test_utils.hpp @@ -8,7 +8,6 @@ #pragma once -#include #include #include #include @@ -16,7 +15,6 @@ #include #include -#include #include #include #include @@ -261,17 +259,3 @@ class StorageGenerator { const std::string storage_; inline static const fs::path TEST_DATABASES_PATH = "./test_databases"; }; - -template -requires requires(Input in) { - requires util::instantiation_of; - requires std::ranges::contiguous_range; - requires std::same_as>; -} -Column create_dense_column(const Input& data) { - constexpr size_t element_size = sizeof(std::ranges::range_value_t); - Column result(TagType::type_descriptor(), data.size(), AllocationType::PRESIZED, Sparsity::NOT_PERMITTED); - std::memcpy(result.ptr(), data.data(), data.size() * element_size); - result.set_row_data(data.size()); - return result; -} \ No newline at end of file diff --git a/cpp/arcticdb/util/type_traits.hpp b/cpp/arcticdb/util/type_traits.hpp index 272c279d528..ddab32f2019 100644 --- a/cpp/arcticdb/util/type_traits.hpp +++ b/cpp/arcticdb/util/type_traits.hpp @@ -9,6 +9,7 @@ #pragma once #include +#include namespace arcticdb::util { @@ -26,4 +27,12 @@ concept instantiation_of = is_instantiation_of_v; template concept any_of = std::disjunction_v...>; + +template +concept contiguous_type_tagged_data = requires(T t) { + instantiation_of; + std::ranges::contiguous_range; + std::same_as; +}; + } // namespace arcticdb::util From 9b80b7ae007a8bb95dbfc2f42b522773caef7e5f Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Tue, 11 Nov 2025 17:27:48 +0200 Subject: [PATCH 27/46] Add one more utility function for generating a dense segment in memory --- .../column_store/memory_segment_impl.hpp | 105 ++++++++++++++---- .../processing/test/test_merge_update.cpp | 18 ++- cpp/arcticdb/util/variant.hpp | 7 ++ 3 files changed, 96 insertions(+), 34 deletions(-) diff --git a/cpp/arcticdb/column_store/memory_segment_impl.hpp b/cpp/arcticdb/column_store/memory_segment_impl.hpp index 5080fa0b934..1c1dc6af8b6 100644 --- a/cpp/arcticdb/column_store/memory_segment_impl.hpp +++ b/cpp/arcticdb/column_store/memory_segment_impl.hpp @@ -281,40 +281,77 @@ class SegmentInMemoryImpl { ); template - requires std::ranges::range && std::ranges::contiguous_range> - SegmentInMemoryImpl create_dense_segment(const T& column_data, const StreamDescriptor& descriptor) { - if (std::ranges::begin(column_data) == std::ranges::end(column_data)) { + requires std::ranges::sized_range && std::ranges::sized_range> + static SegmentInMemoryImpl create_dense_segment(const StreamDescriptor& descriptor, const T& columns) { + const size_t input_column_count = std::ranges::size(columns); + internal::check( + input_column_count == descriptor.fields().size(), + "When creating a dense segment in memory the number of columns ({}) must match the number of fields in " + "the stream descriptor ({}}", + input_column_count, + descriptor.fields().size() + ); + if (input_column_count == 0) { return SegmentInMemoryImpl(); } - const size_t expected_column_size = column_data.begin()->first.size(); + const size_t expected_column_size = columns.begin()->first.size(); constexpr static AllocationType allocation_type = AllocationType::PRESIZED; constexpr static Sparsity sparsity = Sparsity::NOT_PERMITTED; SegmentInMemoryImpl result(descriptor, expected_column_size, allocation_type, sparsity); - for (auto const& [column_index, contiguous_column_data] : folly::enumerate(column_data)) { - if (is_sequence_type(descriptor.field(column_index).type().data_type())) { - for (std::string_view str : contiguous_column_data) { - result.set_string_at(column_index, str); - } - } else { - using ValueType = std::decay_t>; - Column& column = result.column(column_index); - const size_t row_count = std::ranges::size(contiguous_column_data); - internal::check( - row_count == expected_column_size, - "When creating a dense segment from a range of dense column data, all columns must have " - "the same size. Column[0] has size {} Column[{}] has size: {}", - expected_column_size, - column_index, - row_count - ); - std::memcpy(column.ptr(), row_count, row_count * sizeof(ValueType)); - result.set_row_data(row_count); - } + for (auto const& [column_index, column_data] : folly::enumerate(columns)) { + const size_t row_count = std::ranges::size(column_data); + internal::check( + row_count == expected_column_size, + "When creating a dense segment all columns must have the same size. Column[0] has {} rows, " + "Column[{}] has {} rows", + expected_column_size, + column_index, + row_count + ); + result.fill_dense_column_data(column_index, column_data); } return result; } + template + requires(... && std::ranges::sized_range) + static SegmentInMemoryImpl create_dense_segment(const StreamDescriptor& descriptor, const T&... columns) { + constexpr size_t input_column_count = sizeof...(T); + internal::check( + input_column_count == descriptor.fields().size(), + "When creating a dense segment in memory the number of columns ({}) must match the number of fields in " + "the stream descriptor ({}}", + input_column_count, + descriptor.fields().size() + ); + if (input_column_count == 0) { + return SegmentInMemoryImpl(); + } + const size_t expected_column_size = [](const H& head, const T&...) { + return std::ranges::size(head); + }(columns...); + constexpr static AllocationType allocation_type = AllocationType::PRESIZED; + constexpr static Sparsity sparsity = Sparsity::NOT_PERMITTED; + SegmentInMemoryImpl result(descriptor, expected_column_size, allocation_type, sparsity); + util::enumerate( + [&result, expected_column_size](size_t column_index, auto&& column_data) { + const size_t row_count = std::ranges::size(column_data); + internal::check( + row_count == expected_column_size, + "When creating a dense segment all columns must have the same size. Column[0] has {} rows, " + "Column[{}] has {} rows", + expected_column_size, + column_index, + row_count + ); + result.fill_dense_column_data(column_index, column_data); + }, + columns... + ); + return result; + } + ~SegmentInMemoryImpl(); iterator begin(); @@ -599,6 +636,26 @@ class SegmentInMemoryImpl { void drop_empty_columns(); private: + template + requires std::ranges::sized_range + void fill_dense_column_data(const T& input_data, const size_t column_index) { + const size_t row_count = std::ranges::size(input_data); + if (is_sequence_type(descriptor().field(column_index).type().data_type())) { + for (const std::string_view str : input_data) { + set_string(column_index, str); + } + } else { + using ValueType = std::decay_t>; + Column& column_to_fill = column(column_index); + if constexpr (std::ranges::contiguous_range>) { + std::memcpy(column_to_fill.ptr(), std::ranges::data(input_data), row_count * sizeof(ValueType)); + } else { + std::ranges::copy(input_data, column_to_fill.ptr()); + } + } + set_row_data(row_count); + } + ssize_t row_id_ = -1; std::shared_ptr descriptor_; std::vector> columns_; diff --git a/cpp/arcticdb/processing/test/test_merge_update.cpp b/cpp/arcticdb/processing/test/test_merge_update.cpp index 79914386409..b5c9e51fd1c 100644 --- a/cpp/arcticdb/processing/test/test_merge_update.cpp +++ b/cpp/arcticdb/processing/test/test_merge_update.cpp @@ -30,19 +30,17 @@ TEST(MergeUpdateUpdateTimeseries, SourceIndexMatchesAllSegments) { StreamDescriptor source_descriptor = TimeseriesIndex::default_index().create_stream_descriptor("Source", non_string_fields); constexpr std::array source_index{1, 12, 25}; - constexpr std::array source_int8{10, 20, 30}; - constexpr std::array source_uint32{100, 200, 300}; - constexpr std::array source_bool{true, false, true}; - constexpr std::array source_float{11.1f, 22.2f, 33.3f}; - constexpr std::array source_timestamp{1000, 2000, 3000}; InputFrame source( std::move(source_descriptor), create_one_dimensional_tensors( - std::pair{source_int8, TypeDescriptor::scalar_type(DataType::INT8)}, - std::pair{source_uint32, TypeDescriptor::scalar_type(DataType::UINT32)}, - std::pair{source_bool, TypeDescriptor::scalar_type(DataType::BOOL8)}, - std::pair{source_float, TypeDescriptor::scalar_type(DataType::FLOAT32)}, - std::pair{source_timestamp, TypeDescriptor::scalar_type(DataType::NANOSECONDS_UTC64)} + std::pair{std::array{10, 20, 30}, TypeDescriptor::scalar_type(DataType::INT8)}, + std::pair{std::array{100, 200, 300}, TypeDescriptor::scalar_type(DataType::UINT32)}, + std::pair{std::array{true, false, true}, TypeDescriptor::scalar_type(DataType::BOOL8)}, + std::pair{std::array{11.1f, 22.2f, 33.3f}, TypeDescriptor::scalar_type(DataType::FLOAT32)}, + std::pair{ + std::array{1000, 2000, 3000}, + TypeDescriptor::scalar_type(DataType::NANOSECONDS_UTC64) + } ), NativeTensor::one_dimensional_tensor(source_index, DataType::NANOSECONDS_UTC64) ); diff --git a/cpp/arcticdb/util/variant.hpp b/cpp/arcticdb/util/variant.hpp index 8563c8e7806..1fd2c0a716c 100644 --- a/cpp/arcticdb/util/variant.hpp +++ b/cpp/arcticdb/util/variant.hpp @@ -47,4 +47,11 @@ auto variant_match(Variant&& v, Ts&&... ts) { return std::visit(overload{std::forward(ts)...}, std::forward(v)); } +template +void enumerate(InputFun&& callback, Elements&&... elements) { + [](Fn_&& cb, Ts_&&... ts, std::index_sequence) { + (cb(Is_, std::forward(ts)), ...); + }(std::forward(callback), std::forward(elements)..., std::index_sequence_for{}); +} + } // namespace arcticdb::util From 0e1921cb2694e47f3e0824547a02f96d878269ed Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Thu, 13 Nov 2025 02:49:40 +0200 Subject: [PATCH 28/46] Add more testing C++ utils --- .../column_store/memory_segment_impl.hpp | 65 ++++--- cpp/arcticdb/pipeline/write_frame.cpp | 1 - .../processing/test/test_merge_update.cpp | 160 +++++++++++++++++- cpp/arcticdb/util/variant.hpp | 6 +- 4 files changed, 202 insertions(+), 30 deletions(-) diff --git a/cpp/arcticdb/column_store/memory_segment_impl.hpp b/cpp/arcticdb/column_store/memory_segment_impl.hpp index 1c1dc6af8b6..ed0e9b4bf65 100644 --- a/cpp/arcticdb/column_store/memory_segment_impl.hpp +++ b/cpp/arcticdb/column_store/memory_segment_impl.hpp @@ -280,14 +280,14 @@ class SegmentInMemoryImpl { OutputFormat output_format, DataTypeMode mode ); - template - requires std::ranges::sized_range && std::ranges::sized_range> + template + requires std::ranges::sized_range> static SegmentInMemoryImpl create_dense_segment(const StreamDescriptor& descriptor, const T& columns) { const size_t input_column_count = std::ranges::size(columns); internal::check( input_column_count == descriptor.fields().size(), "When creating a dense segment in memory the number of columns ({}) must match the number of fields in " - "the stream descriptor ({}}", + "the stream descriptor ({})", input_column_count, descriptor.fields().size() ); @@ -321,21 +321,21 @@ class SegmentInMemoryImpl { internal::check( input_column_count == descriptor.fields().size(), "When creating a dense segment in memory the number of columns ({}) must match the number of fields in " - "the stream descriptor ({}}", + "the stream descriptor ({})", input_column_count, descriptor.fields().size() ); if (input_column_count == 0) { return SegmentInMemoryImpl(); } - const size_t expected_column_size = [](const H& head, const T&...) { + const size_t expected_column_size = [](const H& head, const Tail&...) { return std::ranges::size(head); }(columns...); constexpr static AllocationType allocation_type = AllocationType::PRESIZED; constexpr static Sparsity sparsity = Sparsity::NOT_PERMITTED; SegmentInMemoryImpl result(descriptor, expected_column_size, allocation_type, sparsity); util::enumerate( - [&result, expected_column_size](size_t column_index, auto&& column_data) { + [&result, expected_column_size](size_t column_index, const auto& column_data) { const size_t row_count = std::ranges::size(column_data); internal::check( row_count == expected_column_size, @@ -636,24 +636,47 @@ class SegmentInMemoryImpl { void drop_empty_columns(); private: - template - requires std::ranges::sized_range - void fill_dense_column_data(const T& input_data, const size_t column_index) { + template + void fill_dense_column_data(const size_t column_index, const T& input_data) { + using InputValueType = std::decay_t>; + constexpr static bool is_input_string_like = std::is_convertible_v; const size_t row_count = std::ranges::size(input_data); - if (is_sequence_type(descriptor().field(column_index).type().data_type())) { - for (const std::string_view str : input_data) { - set_string(column_index, str); - } - } else { - using ValueType = std::decay_t>; - Column& column_to_fill = column(column_index); - if constexpr (std::ranges::contiguous_range>) { - std::memcpy(column_to_fill.ptr(), std::ranges::data(input_data), row_count * sizeof(ValueType)); + details::visit_type(descriptor().field(column_index).type().data_type(), [&, this](auto tdt) { + using col_type_info = ScalarTypeInfo; + constexpr static bool is_sequence = is_sequence_type(col_type_info::data_type); + if constexpr (is_input_string_like && is_sequence) { + // Clang has a bug where it the for_each is just regular range-based for the constexpr if will not + // the body of the if even if the condition is false. This leads to compile time errors because it tries + // to call set_string with non-string values. + // https://stackoverflow.com/questions/79817660/discarded-branch-of-c-constexpr-if-fails-compilation-because-it-calls-non-matc + std::ranges::for_each(input_data, [&, this](const std::string_view& str) { + set_string(column_index, str); + }); + } else if constexpr (!is_sequence && !is_input_string_like) { + internal::check( + std::is_same_v, + "Type mismatch when setting data for Column[{}]. Column data type is {}.", + column_index, + col_type_info::data_type + ); + Column& column_to_fill = column(column_index); + if constexpr (std::ranges::contiguous_range>) { + std::memcpy( + column_to_fill.ptr(), std::ranges::data(input_data), row_count * sizeof(InputValueType) + ); + } else { + std::ranges::copy(input_data, column_to_fill.ptr_cast(0, row_count)); + } } else { - std::ranges::copy(input_data, column_to_fill.ptr()); + internal::check( + std::is_same_v, + "Type mismatch when setting data for Column[{}]. Column data type is {}.", + column_index, + col_type_info::data_type + ); } - } - set_row_data(row_count); + set_row_data(row_count); + }); } ssize_t row_id_ = -1; diff --git a/cpp/arcticdb/pipeline/write_frame.cpp b/cpp/arcticdb/pipeline/write_frame.cpp index 19797248919..6f86a98126b 100644 --- a/cpp/arcticdb/pipeline/write_frame.cpp +++ b/cpp/arcticdb/pipeline/write_frame.cpp @@ -281,7 +281,6 @@ folly::Future> slice_and_write( auto slices = slice(*frame, slicing); if (slices.empty()) return folly::makeFuture(std::vector{}); - ARCTICDB_SUBSAMPLE_DEFAULT(SliceAndWrite) TypedStreamVersion tsv{std::move(key.id), key.version_id, KeyType::TABLE_DATA}; return write_slices(frame, std::move(slices), slicing, std::move(tsv), sink, de_dup_map, sparsify_floats) diff --git a/cpp/arcticdb/processing/test/test_merge_update.cpp b/cpp/arcticdb/processing/test/test_merge_update.cpp index b5c9e51fd1c..41f23673384 100644 --- a/cpp/arcticdb/processing/test/test_merge_update.cpp +++ b/cpp/arcticdb/processing/test/test_merge_update.cpp @@ -21,14 +21,166 @@ constexpr static std::array non_string_fields = { FieldRef(TypeDescriptor(DataType::NANOSECONDS_UTC64, Dimension::Dim0), "timestamp") }; +std::vector split_descriptor(const StreamDescriptor& descriptor, const size_t cols_per_segment) { + if (descriptor.fields().size() <= cols_per_segment) { + return std::vector{descriptor}; + } + const size_t num_segments = descriptor.fields().size() / cols_per_segment; + std::vector res; + res.reserve(num_segments); + + const unsigned field_count = descriptor.field_count(); + for (size_t i = 0, source_field = descriptor.index().field_count(); i < num_segments; ++i) { + StreamDescriptor partial(descriptor.id()); + if (descriptor.index().field_count() > 0) { + partial.set_index(descriptor.index()); + for (unsigned index_field = 0; index_field < descriptor.index().field_count(); ++index_field) { + partial.add_field(descriptor.field(index_field)); + } + } + for (size_t field = 0; field < cols_per_segment && source_field < field_count; ++field) { + partial.add_field(descriptor.field(source_field++)); + } + res.push_back(std::move(partial)); + } + return res; +} + +template +auto take(Data&&... data) { + constexpr size_t to_take = std::min(N, sizeof...(Data)); + return std::pair{ + [&](std::index_sequence) { + return std::tuple{std::get(std::forward_as_tuple(std::forward(data)...))...}; + }(std::make_index_sequence{}), + [&](std::index_sequence) { + return std::tuple{std::get(std::forward_as_tuple(std::forward(data)...))...}; + }(std::make_index_sequence{}) + }; +} + +template +requires(util::instantiation_of) +auto take(TupleT&& t) { + constexpr size_t to_take = std::min(N, std::tuple_size_v); + return std::pair{ + [&](std::index_sequence) { + return std::tuple{std::get(t)...}; + }(std::make_index_sequence{}), + [&](std::index_sequence) { + return std::tuple{std::get(t)...}; + }(std::make_index_sequence - to_take>{}), + }; +} + +template +requires(cols_per_segment > 0) +void slice_data_into_segments( + const std::span descriptors, const size_t col_slice, const size_t rows_per_segment, + std::vector& segments, std::vector& col_ranges, + std::vector& row_ranges, IndexCols&& index, Data&&... data +) { + constexpr size_t index_columns_count = std::tuple_size_v>; + if constexpr (sizeof...(Data) > 0) { + const StreamDescriptor& desc = descriptors[col_slice]; + constexpr static size_t current_columns_count = std::min(cols_per_segment, sizeof...(Data)); + auto [current, rest] = take(std::forward(data)...); + const size_t total_rows = std::ranges::size(std::get<0>(current)); + auto current_with_index = + std::tuple_cat(std::forward(index), std::forward(current)); + for (size_t start_row = 0; start_row < total_rows; start_row += rows_per_segment) { + std::apply( + [&](const Cols&... cols) { + const size_t rows_to_take = std::min(rows_per_segment, total_rows - start_row); + segments.push_back(SegmentInMemoryImpl::create_dense_segment( + desc, std::ranges::take_view(std::ranges::drop_view(cols, start_row), rows_to_take)... + )); + row_ranges.emplace_back(start_row, start_row + rows_to_take); + col_ranges.emplace_back( + index_columns_count + col_slice * cols_per_segment, + index_columns_count + col_slice * cols_per_segment + + std::min(cols_per_segment, current_columns_count) + ); + }, + current_with_index + ); + } + auto [next_index, _] = + take(std::forward(current_with_index)); + std::apply( + [&](auto&&... cols) { + slice_data_into_segments( + descriptors, + col_slice + 1, + rows_per_segment, + segments, + col_ranges, + row_ranges, + std::forward(next_index), + cols... + ); + }, + std::forward(rest) + ); + } +} + +template +std::tuple, std::vector, std::vector> slice_data_into_segments( + size_t rows_per_segment, const StreamDescriptor& descriptor, Data&&... data +) { + std::vector descriptors = split_descriptor(descriptor, cols_per_segment); + auto [index_data, non_index] = take(std::forward(data)...); + std::vector segments; + std::vector col_ranges; + std::vector row_ranges; + std::apply( + [&](NonIndexCols&&... cols) { + slice_data_into_segments( + descriptors, + 0, + rows_per_segment, + segments, + col_ranges, + row_ranges, + std::forward(index_data), + std::forward(cols)... + ); + }, + std::forward(non_index) + ); + return {std::move(segments), std::move(col_ranges), std::move(row_ranges)}; +} + +template +std::vector slice_data_into_segments( + size_t rows_per_segment, const StreamDescriptor& descriptor, Data&&... data +) { + return slice_data_into_segments(rows_per_segment, descriptor, std::forward(data)...); +} + TEST(MergeUpdateUpdateTimeseries, SourceIndexMatchesAllSegments) { + using namespace std::ranges; using stream::TimeseriesIndex; - - std::vector on{}; constexpr static auto strategy = MergeStrategy{.matched = MergeAction::UPDATE, .not_matched_by_target = MergeAction::DO_NOTHING}; + constexpr static size_t columns_per_segment = 3; + constexpr static size_t rows_per_segment = 10; StreamDescriptor source_descriptor = TimeseriesIndex::default_index().create_stream_descriptor("Source", non_string_fields); + auto [target_segments, col_ranges, row_ranges] = slice_data_into_segments( + rows_per_segment, + source_descriptor, + iota_view(timestamp{0}, timestamp{30}), + iota_view(static_cast(0), static_cast(30)), + iota_view(static_cast(0), static_cast(30)), + std::array{true, false, true, true, false, false, true, false, true, false, + true, true, false, true, false, false, true, true, false, true, + false, true, false, false, true, true, false, true, false, true}, + iota_view(0, 30) | views::transform([](auto x) { return static_cast(x); }), + iota_view(timestamp{0}, timestamp{30}) + ); + constexpr std::array source_index{1, 12, 25}; InputFrame source( std::move(source_descriptor), @@ -44,7 +196,5 @@ TEST(MergeUpdateUpdateTimeseries, SourceIndexMatchesAllSegments) { ), NativeTensor::one_dimensional_tensor(source_index, DataType::NANOSECONDS_UTC64) ); - [[maybe_unused]] MergeUpdateClause clause( - std::move(on), strategy, std::make_shared(std::move(source)), true - ); + MergeUpdateClause clause({}, strategy, std::make_shared(std::move(source)), true); } \ No newline at end of file diff --git a/cpp/arcticdb/util/variant.hpp b/cpp/arcticdb/util/variant.hpp index 1fd2c0a716c..7970cc3ca95 100644 --- a/cpp/arcticdb/util/variant.hpp +++ b/cpp/arcticdb/util/variant.hpp @@ -49,9 +49,9 @@ auto variant_match(Variant&& v, Ts&&... ts) { template void enumerate(InputFun&& callback, Elements&&... elements) { - [](Fn_&& cb, Ts_&&... ts, std::index_sequence) { - (cb(Is_, std::forward(ts)), ...); - }(std::forward(callback), std::forward(elements)..., std::index_sequence_for{}); + [&callback, &elements...](std::index_sequence) { + (callback(Is_, std::forward(elements)), ...); + }(std::index_sequence_for{}); } } // namespace arcticdb::util From 8691d88886e54f124be7fda70cf3032ccf8e5103 Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Thu, 13 Nov 2025 19:34:56 +0200 Subject: [PATCH 29/46] WIP on test --- cpp/arcticdb/column_store/memory_segment.hpp | 119 +++++++++++++++++- .../column_store/memory_segment_impl.hpp | 115 ----------------- cpp/arcticdb/entity/native_tensor.hpp | 8 +- cpp/arcticdb/pipeline/input_frame.cpp | 15 --- cpp/arcticdb/pipeline/input_frame.hpp | 25 ++-- cpp/arcticdb/processing/clause_utils.cpp | 6 + cpp/arcticdb/processing/clause_utils.hpp | 4 + .../processing/test/test_merge_update.cpp | 117 +++++++++++++++-- cpp/arcticdb/util/type_traits.hpp | 3 + cpp/arcticdb/version/version_core.hpp | 1 - python/test_index.py | 28 +++++ 11 files changed, 286 insertions(+), 155 deletions(-) create mode 100644 python/test_index.py diff --git a/cpp/arcticdb/column_store/memory_segment.hpp b/cpp/arcticdb/column_store/memory_segment.hpp index 3537c2b0d60..4056da2a1db 100644 --- a/cpp/arcticdb/column_store/memory_segment.hpp +++ b/cpp/arcticdb/column_store/memory_segment.hpp @@ -44,6 +44,81 @@ class SegmentInMemory { ARCTICDB_MOVE_COPY_DEFAULT(SegmentInMemory) + template + requires std::ranges::sized_range> + static SegmentInMemory create_dense_segment(const StreamDescriptor& descriptor, const T& columns) { + const size_t input_column_count = std::ranges::size(columns); + internal::check( + input_column_count == descriptor.fields().size(), + "When creating a dense segment in memory the number of columns ({}) must match the number of fields in " + "the stream descriptor ({})", + input_column_count, + descriptor.fields().size() + ); + if (input_column_count == 0) { + return SegmentInMemory{}; + } + + const size_t expected_column_size = columns.begin()->first.size(); + constexpr static AllocationType allocation_type = AllocationType::PRESIZED; + constexpr static Sparsity sparsity = Sparsity::NOT_PERMITTED; + auto result = SegmentInMemory{ + std::make_shared(descriptor, expected_column_size, allocation_type, sparsity) + }; + for (auto const& [column_index, column_data] : folly::enumerate(columns)) { + const size_t row_count = std::ranges::size(column_data); + internal::check( + row_count == expected_column_size, + "When creating a dense segment all columns must have the same size. Column[0] has {} rows, " + "Column[{}] has {} rows", + expected_column_size, + column_index, + row_count + ); + result.fill_dense_column_data(column_index, column_data); + } + return result; + } + + template + static SegmentInMemory create_dense_segment(const StreamDescriptor& descriptor, const T&... columns) { + constexpr size_t input_column_count = sizeof...(T); + internal::check( + input_column_count == descriptor.fields().size(), + "When creating a dense segment in memory the number of columns ({}) must match the number of fields in " + "the stream descriptor ({})", + input_column_count, + descriptor.fields().size() + ); + if (input_column_count == 0) { + return SegmentInMemory{}; + } + const size_t expected_column_size = [](const H& head, const Tail&...) { + return std::ranges::size(head); + }(columns...); + constexpr static AllocationType allocation_type = AllocationType::PRESIZED; + constexpr static Sparsity sparsity = Sparsity::NOT_PERMITTED; + auto result = SegmentInMemory{ + std::make_shared(descriptor, expected_column_size, allocation_type, sparsity) + }; + util::enumerate( + [&result, expected_column_size](size_t column_index, const auto& column_data) { + const size_t row_count = std::ranges::size(column_data); + internal::check( + row_count == expected_column_size, + "When creating a dense segment all columns must have the same size. Column[0] has {} rows, " + "Column[{}] has {} rows", + expected_column_size, + column_index, + row_count + ); + result.fill_dense_column_data(column_index, column_data); + }, + columns... + ); + return result; + } + [[nodiscard]] iterator begin(); [[nodiscard]] iterator end(); @@ -293,7 +368,49 @@ class SegmentInMemory { private: explicit SegmentInMemory(std::shared_ptr impl) : impl_(std::move(impl)) {} - + template + void fill_dense_column_data(const size_t column_index, const T& input_data) { + using InputValueType = std::decay_t>; + constexpr static bool is_input_string_like = std::is_convertible_v; + const size_t row_count = std::ranges::size(input_data); + details::visit_type(descriptor().field(column_index).type().data_type(), [&, this](auto tdt) { + using col_type_info = ScalarTypeInfo; + using ColRawType = col_type_info::RawType; + constexpr static bool is_sequence = is_sequence_type(col_type_info::data_type); + if constexpr (is_input_string_like && is_sequence) { + // Clang has a bug where it the for_each is just regular range-based for the constexpr if will not + // the body of the if even if the condition is false. This leads to compile time errors because it tries + // to call set_string with non-string values. + // https://stackoverflow.com/questions/79817660/discarded-branch-of-c-constexpr-if-fails-compilation-because-it-calls-non-matc + std::ranges::for_each(input_data, [&, this](const std::string_view& str) { + set_string(column_index, str); + }); + } else if constexpr (!is_sequence && !is_input_string_like) { + internal::check( + std::is_same_v, + "Type mismatch when setting data for Column[{}]. Column data type is {}.", + column_index, + col_type_info::data_type + ); + Column& column_to_fill = column(column_index); + if constexpr (std::ranges::contiguous_range) { + std::memcpy( + column_to_fill.ptr(), std::ranges::data(input_data), row_count * sizeof(InputValueType) + ); + } else { + std::ranges::copy(input_data, column_to_fill.ptr_cast(0, row_count)); + } + } else { + internal::check( + std::is_same_v, + "Type mismatch when setting data for Column[{}]. Column data type is {}.", + column_index, + col_type_info::data_type + ); + } + set_row_data(row_count - 1); + }); + } std::shared_ptr impl_; }; diff --git a/cpp/arcticdb/column_store/memory_segment_impl.hpp b/cpp/arcticdb/column_store/memory_segment_impl.hpp index ed0e9b4bf65..79e77c3dd1d 100644 --- a/cpp/arcticdb/column_store/memory_segment_impl.hpp +++ b/cpp/arcticdb/column_store/memory_segment_impl.hpp @@ -280,78 +280,6 @@ class SegmentInMemoryImpl { OutputFormat output_format, DataTypeMode mode ); - template - requires std::ranges::sized_range> - static SegmentInMemoryImpl create_dense_segment(const StreamDescriptor& descriptor, const T& columns) { - const size_t input_column_count = std::ranges::size(columns); - internal::check( - input_column_count == descriptor.fields().size(), - "When creating a dense segment in memory the number of columns ({}) must match the number of fields in " - "the stream descriptor ({})", - input_column_count, - descriptor.fields().size() - ); - if (input_column_count == 0) { - return SegmentInMemoryImpl(); - } - - const size_t expected_column_size = columns.begin()->first.size(); - constexpr static AllocationType allocation_type = AllocationType::PRESIZED; - constexpr static Sparsity sparsity = Sparsity::NOT_PERMITTED; - SegmentInMemoryImpl result(descriptor, expected_column_size, allocation_type, sparsity); - for (auto const& [column_index, column_data] : folly::enumerate(columns)) { - const size_t row_count = std::ranges::size(column_data); - internal::check( - row_count == expected_column_size, - "When creating a dense segment all columns must have the same size. Column[0] has {} rows, " - "Column[{}] has {} rows", - expected_column_size, - column_index, - row_count - ); - result.fill_dense_column_data(column_index, column_data); - } - return result; - } - - template - requires(... && std::ranges::sized_range) - static SegmentInMemoryImpl create_dense_segment(const StreamDescriptor& descriptor, const T&... columns) { - constexpr size_t input_column_count = sizeof...(T); - internal::check( - input_column_count == descriptor.fields().size(), - "When creating a dense segment in memory the number of columns ({}) must match the number of fields in " - "the stream descriptor ({})", - input_column_count, - descriptor.fields().size() - ); - if (input_column_count == 0) { - return SegmentInMemoryImpl(); - } - const size_t expected_column_size = [](const H& head, const Tail&...) { - return std::ranges::size(head); - }(columns...); - constexpr static AllocationType allocation_type = AllocationType::PRESIZED; - constexpr static Sparsity sparsity = Sparsity::NOT_PERMITTED; - SegmentInMemoryImpl result(descriptor, expected_column_size, allocation_type, sparsity); - util::enumerate( - [&result, expected_column_size](size_t column_index, const auto& column_data) { - const size_t row_count = std::ranges::size(column_data); - internal::check( - row_count == expected_column_size, - "When creating a dense segment all columns must have the same size. Column[0] has {} rows, " - "Column[{}] has {} rows", - expected_column_size, - column_index, - row_count - ); - result.fill_dense_column_data(column_index, column_data); - }, - columns... - ); - return result; - } - ~SegmentInMemoryImpl(); iterator begin(); @@ -636,49 +564,6 @@ class SegmentInMemoryImpl { void drop_empty_columns(); private: - template - void fill_dense_column_data(const size_t column_index, const T& input_data) { - using InputValueType = std::decay_t>; - constexpr static bool is_input_string_like = std::is_convertible_v; - const size_t row_count = std::ranges::size(input_data); - details::visit_type(descriptor().field(column_index).type().data_type(), [&, this](auto tdt) { - using col_type_info = ScalarTypeInfo; - constexpr static bool is_sequence = is_sequence_type(col_type_info::data_type); - if constexpr (is_input_string_like && is_sequence) { - // Clang has a bug where it the for_each is just regular range-based for the constexpr if will not - // the body of the if even if the condition is false. This leads to compile time errors because it tries - // to call set_string with non-string values. - // https://stackoverflow.com/questions/79817660/discarded-branch-of-c-constexpr-if-fails-compilation-because-it-calls-non-matc - std::ranges::for_each(input_data, [&, this](const std::string_view& str) { - set_string(column_index, str); - }); - } else if constexpr (!is_sequence && !is_input_string_like) { - internal::check( - std::is_same_v, - "Type mismatch when setting data for Column[{}]. Column data type is {}.", - column_index, - col_type_info::data_type - ); - Column& column_to_fill = column(column_index); - if constexpr (std::ranges::contiguous_range>) { - std::memcpy( - column_to_fill.ptr(), std::ranges::data(input_data), row_count * sizeof(InputValueType) - ); - } else { - std::ranges::copy(input_data, column_to_fill.ptr_cast(0, row_count)); - } - } else { - internal::check( - std::is_same_v, - "Type mismatch when setting data for Column[{}]. Column data type is {}.", - column_index, - col_type_info::data_type - ); - } - set_row_data(row_count); - }); - } - ssize_t row_id_ = -1; std::shared_ptr descriptor_; std::vector> columns_; diff --git a/cpp/arcticdb/entity/native_tensor.hpp b/cpp/arcticdb/entity/native_tensor.hpp index 6d53f1c6a5d..db8bed42170 100644 --- a/cpp/arcticdb/entity/native_tensor.hpp +++ b/cpp/arcticdb/entity/native_tensor.hpp @@ -67,15 +67,13 @@ struct NativeTensor { } } - template - requires std::ranges::contiguous_range + template static NativeTensor one_dimensional_tensor(const T& data, const DataType data_type) { - using ValueType = std::ranges::range_value_t; + using ValueType = std::decay_t>; constexpr static size_t element_size = sizeof(ValueType); constexpr shape_t shapes = 1; - constexpr stride_t strides = element_size; const int64_t byte_size = data.size() * element_size; - return NativeTensor{byte_size, 1, &shapes, &strides, data_type, element_size, std::ranges::data(data), 1}; + return NativeTensor{byte_size, 1, nullptr, &shapes, data_type, element_size, std::ranges::data(data), 1}; } NativeTensor(const NativeTensor& other) : diff --git a/cpp/arcticdb/pipeline/input_frame.cpp b/cpp/arcticdb/pipeline/input_frame.cpp index da21056dbae..c4dd7a496c9 100644 --- a/cpp/arcticdb/pipeline/input_frame.cpp +++ b/cpp/arcticdb/pipeline/input_frame.cpp @@ -18,14 +18,6 @@ InputFrame::InputFrame() : index(stream::empty_index()) {} InputFrame::InputFrame(SegmentInMemory&& seg) : index(stream::empty_index()) { set_segment(std::move(seg)); } -InputFrame::InputFrame( - StreamDescriptor&& desc, std::vector&& field_tensors, - std::optional&& index_tensor -) : - index(stream::empty_index()) { - set_from_tensors(std::move(desc), std::move(field_tensors), std::move(index_tensor)); -} - void InputFrame::set_segment(SegmentInMemory&& seg) { num_rows = seg.row_count(); util::check(norm_meta.has_experimental_arrow(), "Unexpected non-Arrow norm metadata provided with Arrow data"); @@ -49,13 +41,6 @@ void InputFrame::set_segment(SegmentInMemory&& seg) { input_data.emplace(std::move(seg)); } -void InputFrame::set_from_tensors( - StreamDescriptor&& desc, std::vector&& field_tensors, - std::optional&& index_tensor -) { - input_data.emplace(std::move(index_tensor), std::move(field_tensors), std::move(desc)); -} - StreamDescriptor& InputFrame::desc() { if (has_tensors()) { return std::get(input_data).desc; diff --git a/cpp/arcticdb/pipeline/input_frame.hpp b/cpp/arcticdb/pipeline/input_frame.hpp index f369678ee16..e0e6397f9bc 100644 --- a/cpp/arcticdb/pipeline/input_frame.hpp +++ b/cpp/arcticdb/pipeline/input_frame.hpp @@ -30,16 +30,27 @@ struct InputFrame { public: InputFrame(); InputFrame(SegmentInMemory&& seg); + + template + requires util::forwarding_ref_to InputFrame( - StreamDescriptor&& desc, std::vector&& field_tensors, - std::optional&& index_tensor = std::nullopt - ); - void set_segment(SegmentInMemory&& seg); + DescriptorT&& desc, std::vector&& field_tensors, std::optional&& index_tensor + ) : + index(stream::empty_index()) { + set_from_tensors(std::forward(desc), std::move(field_tensors), std::move(index_tensor)); + } + + template + requires util::forwarding_ref_to void set_from_tensors( - StreamDescriptor&& desc, std::vector&& field_tensors, - std::optional&& index_tensor - ); + DescriptorT&& desc, std::vector&& field_tensors, std::optional&& index_tensor + ) { + input_data.emplace( + std::move(index_tensor), std::move(field_tensors), std::forward(desc) + ); + } + void set_segment(SegmentInMemory&& seg); StreamDescriptor& desc(); const StreamDescriptor& desc() const; // The descriptor of the input frame can differ than that for the timeseries descriptor in the index key for Arrow diff --git a/cpp/arcticdb/processing/clause_utils.cpp b/cpp/arcticdb/processing/clause_utils.cpp index 69a3e2b4219..1b7ff4ccf10 100644 --- a/cpp/arcticdb/processing/clause_utils.cpp +++ b/cpp/arcticdb/processing/clause_utils.cpp @@ -19,6 +19,12 @@ std::vector> structure_by_row_slice( ComponentManager& component_manager, std::vector>&& entity_ids_vec ) { auto entity_ids = flatten_entities(std::move(entity_ids_vec)); + return structure_by_row_slice(component_manager, std::move(entity_ids)); +} + +std::vector> structure_by_row_slice( + ComponentManager& component_manager, std::vector&& entity_ids +) { auto [row_ranges, col_ranges] = component_manager.get_entities, std::shared_ptr>(entity_ids); std::vector ranges_and_entities; diff --git a/cpp/arcticdb/processing/clause_utils.hpp b/cpp/arcticdb/processing/clause_utils.hpp index 50ccbf3c154..1da667b747d 100644 --- a/cpp/arcticdb/processing/clause_utils.hpp +++ b/cpp/arcticdb/processing/clause_utils.hpp @@ -160,6 +160,10 @@ std::vector> structure_by_row_slice( ComponentManager& component_manager, std::vector>&& entity_ids_vec ); +std::vector> structure_by_row_slice( + ComponentManager& component_manager, std::vector&& entity_ids_vec +); + std::vector> offsets_to_entity_ids( const std::vector>& offsets, const std::vector& ranges_and_entities ); diff --git a/cpp/arcticdb/processing/test/test_merge_update.cpp b/cpp/arcticdb/processing/test/test_merge_update.cpp index 41f23673384..d8a13e8b826 100644 --- a/cpp/arcticdb/processing/test/test_merge_update.cpp +++ b/cpp/arcticdb/processing/test/test_merge_update.cpp @@ -6,10 +6,13 @@ * will be governed by the Apache License, version 2.0. */ +#include "util/ranges_from_future.hpp" + #include #include #include #include +#include using namespace arcticdb; @@ -77,8 +80,8 @@ template 0) void slice_data_into_segments( const std::span descriptors, const size_t col_slice, const size_t rows_per_segment, - std::vector& segments, std::vector& col_ranges, - std::vector& row_ranges, IndexCols&& index, Data&&... data + std::vector& segments, std::vector& col_ranges, std::vector& row_ranges, + IndexCols&& index, Data&&... data ) { constexpr size_t index_columns_count = std::tuple_size_v>; if constexpr (sizeof...(Data) > 0) { @@ -92,7 +95,7 @@ void slice_data_into_segments( std::apply( [&](const Cols&... cols) { const size_t rows_to_take = std::min(rows_per_segment, total_rows - start_row); - segments.push_back(SegmentInMemoryImpl::create_dense_segment( + segments.push_back(SegmentInMemory::create_dense_segment( desc, std::ranges::take_view(std::ranges::drop_view(cols, start_row), rows_to_take)... )); row_ranges.emplace_back(start_row, start_row + rows_to_take); @@ -126,12 +129,12 @@ void slice_data_into_segments( } template -std::tuple, std::vector, std::vector> slice_data_into_segments( +std::tuple, std::vector, std::vector> slice_data_into_segments( size_t rows_per_segment, const StreamDescriptor& descriptor, Data&&... data ) { std::vector descriptors = split_descriptor(descriptor, cols_per_segment); auto [index_data, non_index] = take(std::forward(data)...); - std::vector segments; + std::vector segments; std::vector col_ranges; std::vector row_ranges; std::apply( @@ -153,12 +156,55 @@ std::tuple, std::vector, std::vector< } template -std::vector slice_data_into_segments( +std::vector slice_data_into_segments( size_t rows_per_segment, const StreamDescriptor& descriptor, Data&&... data ) { return slice_data_into_segments(rows_per_segment, descriptor, std::forward(data)...); } +template +std::vector> wrap_in_shared_ptr(std::vector&& v) { + std::vector> res; + res.reserve(v.size()); + std::ranges::transform(v, std::back_inserter(res), [](T& x) { return std::make_shared(std::move(x)); }); + return res; +} + +template +void shuffle_multiple(RandomEngine& engine, Ranges&... to_shuffle) { + const size_t num_elements = [](const H& h, const T&...) { + return std::ranges::size(h); + }(to_shuffle...); + std::vector destinations(num_elements); + std::iota(destinations.begin(), destinations.end(), 0); + std::ranges::shuffle(destinations, engine); + for (size_t i = 0; i < num_elements; ++i) { + (std::swap(to_shuffle[i], to_shuffle[destinations[i]]), ...); + std::swap(destinations[i], destinations[destinations[i]]); + } +} + +std::vector generate_ranges_and_keys( + const StreamDescriptor& source_descriptor, const std::span> segments, + const std::span> col_ranges, + const std::span> row_ranges +) { + std::vector ranges_and_keys; + ranges_and_keys.reserve(segments.size()); + for (size_t i = 0; i < segments.size(); ++i) { + const timestamp start_ts = *segments[i]->scalar_at(0, 0); + const timestamp end_ts = *segments[i]->scalar_at(segments[i]->row_count() - 1, 0); + ranges_and_keys.emplace_back( + *row_ranges[i], + *col_ranges[i], + AtomKeyBuilder().start_index(start_ts).end_index(end_ts).build( + source_descriptor.id() + ) + ); + } + return ranges_and_keys; +} + TEST(MergeUpdateUpdateTimeseries, SourceIndexMatchesAllSegments) { using namespace std::ranges; using stream::TimeseriesIndex; @@ -166,9 +212,9 @@ TEST(MergeUpdateUpdateTimeseries, SourceIndexMatchesAllSegments) { MergeStrategy{.matched = MergeAction::UPDATE, .not_matched_by_target = MergeAction::DO_NOTHING}; constexpr static size_t columns_per_segment = 3; constexpr static size_t rows_per_segment = 10; - StreamDescriptor source_descriptor = + const StreamDescriptor source_descriptor = TimeseriesIndex::default_index().create_stream_descriptor("Source", non_string_fields); - auto [target_segments, col_ranges, row_ranges] = slice_data_into_segments( + auto sliced = slice_data_into_segments( rows_per_segment, source_descriptor, iota_view(timestamp{0}, timestamp{30}), @@ -180,10 +226,17 @@ TEST(MergeUpdateUpdateTimeseries, SourceIndexMatchesAllSegments) { iota_view(0, 30) | views::transform([](auto x) { return static_cast(x); }), iota_view(timestamp{0}, timestamp{30}) ); + auto [proc_segments, proc_col_range, proc_row_range] = std::apply( + [](auto&&... args) { return std::tuple{wrap_in_shared_ptr(std::move(args))...}; }, std::move(sliced) + ); + EXPECT_EQ(proc_segments.size(), proc_col_range.size()); + EXPECT_EQ(proc_segments.size(), proc_row_range.size()); + + std::vector ranges_and_keys = + generate_ranges_and_keys(source_descriptor, proc_segments, proc_col_range, proc_row_range); - constexpr std::array source_index{1, 12, 25}; InputFrame source( - std::move(source_descriptor), + source_descriptor, create_one_dimensional_tensors( std::pair{std::array{10, 20, 30}, TypeDescriptor::scalar_type(DataType::INT8)}, std::pair{std::array{100, 200, 300}, TypeDescriptor::scalar_type(DataType::UINT32)}, @@ -194,7 +247,49 @@ TEST(MergeUpdateUpdateTimeseries, SourceIndexMatchesAllSegments) { TypeDescriptor::scalar_type(DataType::NANOSECONDS_UTC64) } ), - NativeTensor::one_dimensional_tensor(source_index, DataType::NANOSECONDS_UTC64) + NativeTensor::one_dimensional_tensor(std::array{1, 12, 25}, DataType::NANOSECONDS_UTC64) ); + source.num_rows = 3; + + constexpr static size_t rand_seed = 0; + std::mt19937 g(rand_seed); + shuffle_multiple(g, ranges_and_keys, proc_segments, proc_col_range, proc_row_range); + std::vector fetch_count(proc_segments.size(), 0); + auto component_manager = std::make_shared(); + std::vector entities = component_manager->add_entities( + std::move(proc_segments), std::move(proc_col_range), std::move(proc_row_range), std::move(fetch_count) + ); + MergeUpdateClause clause({}, strategy, std::make_shared(std::move(source)), true); + clause.set_component_manager(component_manager); + std::vector> structure_indices = clause.structure_for_processing(ranges_and_keys); + constexpr static size_t row_slices_to_read = 3; + EXPECT_EQ(structure_indices.size(), row_slices_to_read); + for (size_t row = 0; row < row_slices_to_read; ++row) { + EXPECT_EQ(structure_indices[row].size(), 2); + for (size_t col = 0; col < structure_indices[row].size(); ++col) { + const size_t entt = structure_indices[row][col]; + const RowRange& row_range = ranges_and_keys[entt].row_range(); + EXPECT_EQ(row_range, ranges_and_keys[structure_indices[row][0]].row_range()); + + const ColRange& col_range = ranges_and_keys[entt].col_range(); + const size_t start_col = TimeseriesIndex::field_count() + col * columns_per_segment; + const size_t end_col = std::min(start_col + columns_per_segment, source_descriptor.field_count()); + const ColRange expected_col_range{start_col, end_col}; + EXPECT_EQ(col_range, expected_col_range); + } + } + + std::vector> row_slice_structure = + structure_by_row_slice(*component_manager, std::move(entities)); + for (size_t row_slice_idx = 0; row_slice_idx < row_slice_structure.size(); ++row_slice_idx) { + std::vector& row_slice = row_slice_structure[row_slice_idx]; + std::vector result_entities = clause.process(std::move(row_slice)); + ProcessingUnit proc = + gather_entities, std::shared_ptr, std::shared_ptr>( + *component_manager, result_entities + ); + EXPECT_EQ(proc.row_ranges_->size(), 2); + EXPECT_EQ(proc.col_ranges_->size(), 2); + } } \ No newline at end of file diff --git a/cpp/arcticdb/util/type_traits.hpp b/cpp/arcticdb/util/type_traits.hpp index ddab32f2019..a4562183c09 100644 --- a/cpp/arcticdb/util/type_traits.hpp +++ b/cpp/arcticdb/util/type_traits.hpp @@ -35,4 +35,7 @@ concept contiguous_type_tagged_data = requires(T t) { std::same_as; }; +template +concept forwarding_ref_to = std::same_as, Base>; + } // namespace arcticdb::util diff --git a/cpp/arcticdb/version/version_core.hpp b/cpp/arcticdb/version/version_core.hpp index bc9e88e3b1e..e83dc358ac5 100644 --- a/cpp/arcticdb/version/version_core.hpp +++ b/cpp/arcticdb/version/version_core.hpp @@ -228,7 +228,6 @@ VersionedItem merge_impl( std::vector&& on, bool match_on_timeseries_index, const MergeStrategy& strategy, const std::shared_ptr& source ); - } // namespace arcticdb::version_store namespace arcticdb { diff --git a/python/test_index.py b/python/test_index.py new file mode 100644 index 00000000000..f1fbac8e492 --- /dev/null +++ b/python/test_index.py @@ -0,0 +1,28 @@ +import arcticdb +import pandas as pd +import numpy as np + +ac = arcticdb.Arctic("lmdb://test") +opts = arcticdb.LibraryOptions(columns_per_segment=2, rows_per_segment=2) +if ac.has_library("test"): + ac.delete_library("test") +lib = ac.get_library("test", library_options=opts, create_if_missing=True) + +arrays = [ + [pd.Timestamp(1), pd.Timestamp(1), pd.Timestamp(2), pd.Timestamp(2), pd.Timestamp(3)], + [10, 20, 10, 20, 10], +] # First index: integers # Second index: integers +index = pd.MultiIndex.from_arrays(arrays, names=("index1", "index2")) + +data = np.random.randn(5, 5) +columns = ["col1", "col2", "col3", "col4", "col5"] + +df = pd.DataFrame(data, index=index, columns=columns) + +ts_df = pd.DataFrame(data, columns=columns, index=pd.date_range("2023-01-01", periods=5, freq="h")) + +qb = arcticdb.QueryBuilder() +qb = qb[qb["col1"] > 0] + +lib.write("test", ts_df) +lib.read("test", query_builder=qb) From 3d02c5022bbca6e4a3591fabbd1fc328707880ec Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Fri, 14 Nov 2025 16:36:04 +0200 Subject: [PATCH 30/46] Fixes to C++ tests --- cpp/arcticdb/entity/native_tensor.hpp | 3 +- cpp/arcticdb/pipeline/input_frame.hpp | 4 +- cpp/arcticdb/processing/clause.hpp | 1 + .../processing/test/test_merge_update.cpp | 165 +++++++++++------- cpp/arcticdb/util/type_traits.hpp | 2 +- 5 files changed, 110 insertions(+), 65 deletions(-) diff --git a/cpp/arcticdb/entity/native_tensor.hpp b/cpp/arcticdb/entity/native_tensor.hpp index db8bed42170..61b63c954fa 100644 --- a/cpp/arcticdb/entity/native_tensor.hpp +++ b/cpp/arcticdb/entity/native_tensor.hpp @@ -175,8 +175,7 @@ struct NativeTensor { /// @tparam Input Parameter pack of pair-like types satisfying the contiguous_range requirement /// @param arrays Variadic pairs of (contiguous_range, DataType) /// @return Vector of one-dimensional NativeTensors -template -requires(... && util::contiguous_type_tagged_data) +template std::vector create_one_dimensional_tensors(const Input&... arrays) { std::vector tensors; tensors.reserve(sizeof...(arrays)); diff --git a/cpp/arcticdb/pipeline/input_frame.hpp b/cpp/arcticdb/pipeline/input_frame.hpp index e0e6397f9bc..4ed68297a75 100644 --- a/cpp/arcticdb/pipeline/input_frame.hpp +++ b/cpp/arcticdb/pipeline/input_frame.hpp @@ -32,7 +32,7 @@ struct InputFrame { InputFrame(SegmentInMemory&& seg); template - requires util::forwarding_ref_to + requires util::decays_to InputFrame( DescriptorT&& desc, std::vector&& field_tensors, std::optional&& index_tensor ) : @@ -41,7 +41,7 @@ struct InputFrame { } template - requires util::forwarding_ref_to + requires util::decays_to void set_from_tensors( DescriptorT&& desc, std::vector&& field_tensors, std::optional&& index_tensor ) { diff --git a/cpp/arcticdb/processing/clause.hpp b/cpp/arcticdb/processing/clause.hpp index 2f754a5aabb..ad04466c8ea 100644 --- a/cpp/arcticdb/processing/clause.hpp +++ b/cpp/arcticdb/processing/clause.hpp @@ -887,6 +887,7 @@ struct MergeUpdateClause { /// row std::vector> filter_index_match(const ProcessingUnit& proc) const; + /// For each row range stores the first and last row in the source that overlaps with the row range ankerl::unordered_dense::map, RowRange::Hasher> source_start_for_row_range_; }; diff --git a/cpp/arcticdb/processing/test/test_merge_update.cpp b/cpp/arcticdb/processing/test/test_merge_update.cpp index d8a13e8b826..02f0a623f88 100644 --- a/cpp/arcticdb/processing/test/test_merge_update.cpp +++ b/cpp/arcticdb/processing/test/test_merge_update.cpp @@ -170,33 +170,18 @@ std::vector> wrap_in_shared_ptr(std::vector&& v) { return res; } -template -void shuffle_multiple(RandomEngine& engine, Ranges&... to_shuffle) { - const size_t num_elements = [](const H& h, const T&...) { - return std::ranges::size(h); - }(to_shuffle...); - std::vector destinations(num_elements); - std::iota(destinations.begin(), destinations.end(), 0); - std::ranges::shuffle(destinations, engine); - for (size_t i = 0; i < num_elements; ++i) { - (std::swap(to_shuffle[i], to_shuffle[destinations[i]]), ...); - std::swap(destinations[i], destinations[destinations[i]]); - } -} - std::vector generate_ranges_and_keys( - const StreamDescriptor& source_descriptor, const std::span> segments, - const std::span> col_ranges, - const std::span> row_ranges + const StreamDescriptor& source_descriptor, const std::span segments, + const std::span col_ranges, const std::span row_ranges ) { std::vector ranges_and_keys; ranges_and_keys.reserve(segments.size()); for (size_t i = 0; i < segments.size(); ++i) { - const timestamp start_ts = *segments[i]->scalar_at(0, 0); - const timestamp end_ts = *segments[i]->scalar_at(segments[i]->row_count() - 1, 0); + const timestamp start_ts = *segments[i].scalar_at(0, 0); + const timestamp end_ts = *segments[i].scalar_at(segments[i].row_count() - 1, 0); ranges_and_keys.emplace_back( - *row_ranges[i], - *col_ranges[i], + row_ranges[i], + col_ranges[i], AtomKeyBuilder().start_index(start_ts).end_index(end_ts).build( source_descriptor.id() ) @@ -205,6 +190,57 @@ std::vector generate_ranges_and_keys( return ranges_and_keys; } +template +void sort_by_rowslice(std::span rows, std::span cols, Other&... other) { + std::vector correct_positions(rows.size()); + std::iota(correct_positions.begin(), correct_positions.end(), 0); + std::ranges::sort(correct_positions, [&](const size_t i, const size_t j) { + return std::tie(rows[i], cols[i]) < std::tie(rows[j], cols[j]); + }); + [](const std::span positions, T&... ts) { + util::BitSet used; + used.resize(positions.size()); + for (size_t i = 0; i < positions.size(); ++i) { + if (used[i] || positions[i] == i) { + continue; + } + auto temp = std::tuple{std::move(ts[i])...}; + size_t current = i; + while (positions[current] != i) { + size_t next = positions[current]; + ((ts[current] = std::move(ts[next])), ...); + used[current] = true; + current = next; + } + [&](std::index_sequence) { + ((ts[current] = std::move(std::get(temp))), ...); + }(std::make_index_sequence{}); + used[current] = true; + } + }(correct_positions, rows, cols, other...); +} + +std::vector> map_entities_to_structure_for_processing_output( + const std::span> structure_for_processing_out, + const std::span entities +) { + std::vector> process_input; + process_input.reserve(structure_for_processing_out.size()); + std::ranges::transform( + structure_for_processing_out, + std::back_inserter(process_input), + [&](const std::vector& indices) { + std::vector result; + result.reserve(indices.size()); + std::ranges::transform(indices, std::back_inserter(result), [&](const size_t idx) { + return entities[idx]; + }); + return result; + } + ); + return process_input; +} + TEST(MergeUpdateUpdateTimeseries, SourceIndexMatchesAllSegments) { using namespace std::ranges; using stream::TimeseriesIndex; @@ -214,7 +250,7 @@ TEST(MergeUpdateUpdateTimeseries, SourceIndexMatchesAllSegments) { constexpr static size_t rows_per_segment = 10; const StreamDescriptor source_descriptor = TimeseriesIndex::default_index().create_stream_descriptor("Source", non_string_fields); - auto sliced = slice_data_into_segments( + auto [segments, col_ranges, row_ranges] = slice_data_into_segments( rows_per_segment, source_descriptor, iota_view(timestamp{0}, timestamp{30}), @@ -226,14 +262,10 @@ TEST(MergeUpdateUpdateTimeseries, SourceIndexMatchesAllSegments) { iota_view(0, 30) | views::transform([](auto x) { return static_cast(x); }), iota_view(timestamp{0}, timestamp{30}) ); - auto [proc_segments, proc_col_range, proc_row_range] = std::apply( - [](auto&&... args) { return std::tuple{wrap_in_shared_ptr(std::move(args))...}; }, std::move(sliced) - ); - EXPECT_EQ(proc_segments.size(), proc_col_range.size()); - EXPECT_EQ(proc_segments.size(), proc_row_range.size()); + sort_by_rowslice(row_ranges, col_ranges, segments); - std::vector ranges_and_keys = - generate_ranges_and_keys(source_descriptor, proc_segments, proc_col_range, proc_row_range); + EXPECT_EQ(segments.size(), col_ranges.size()); + EXPECT_EQ(segments.size(), row_ranges.size()); InputFrame source( source_descriptor, @@ -251,45 +283,58 @@ TEST(MergeUpdateUpdateTimeseries, SourceIndexMatchesAllSegments) { ); source.num_rows = 3; + // Shuffle the input ranges and keys to ensure structure_for_processing sorts correctly constexpr static size_t rand_seed = 0; std::mt19937 g(rand_seed); - shuffle_multiple(g, ranges_and_keys, proc_segments, proc_col_range, proc_row_range); - std::vector fetch_count(proc_segments.size(), 0); + std::vector ranges_and_keys = + generate_ranges_and_keys(source_descriptor, segments, col_ranges, row_ranges); + shuffle(ranges_and_keys, g); + auto component_manager = std::make_shared(); - std::vector entities = component_manager->add_entities( - std::move(proc_segments), std::move(proc_col_range), std::move(proc_row_range), std::move(fetch_count) - ); + std::vector fetch_count(segments.size(), 0); + std::vector> proc_row_range = wrap_in_shared_ptr(std::move(row_ranges)); + std::vector> proc_col_range = wrap_in_shared_ptr(std::move(col_ranges)); + std::vector> proc_seg = wrap_in_shared_ptr(std::move(segments)); + std::vector entities = + component_manager->add_entities(proc_col_range, proc_row_range, proc_seg, std::move(fetch_count)); MergeUpdateClause clause({}, strategy, std::make_shared(std::move(source)), true); clause.set_component_manager(component_manager); - std::vector> structure_indices = clause.structure_for_processing(ranges_and_keys); - constexpr static size_t row_slices_to_read = 3; - EXPECT_EQ(structure_indices.size(), row_slices_to_read); - for (size_t row = 0; row < row_slices_to_read; ++row) { - EXPECT_EQ(structure_indices[row].size(), 2); - for (size_t col = 0; col < structure_indices[row].size(); ++col) { - const size_t entt = structure_indices[row][col]; - const RowRange& row_range = ranges_and_keys[entt].row_range(); - EXPECT_EQ(row_range, ranges_and_keys[structure_indices[row][0]].row_range()); - const ColRange& col_range = ranges_and_keys[entt].col_range(); - const size_t start_col = TimeseriesIndex::field_count() + col * columns_per_segment; - const size_t end_col = std::min(start_col + columns_per_segment, source_descriptor.field_count()); - const ColRange expected_col_range{start_col, end_col}; - EXPECT_EQ(col_range, expected_col_range); - } - } + const std::vector> structure_indices = clause.structure_for_processing(ranges_and_keys); + + constexpr static int row_slices_to_process = 3; + constexpr static int column_slices_per_row_slice = 2; + EXPECT_EQ(structure_indices.size(), row_slices_to_process); + + EXPECT_EQ(structure_indices[0].size(), column_slices_per_row_slice); + EXPECT_EQ(ranges_and_keys[structure_indices[0][0]].row_range(), RowRange(0, 10)); + EXPECT_EQ(ranges_and_keys[structure_indices[0][0]].col_range(), ColRange(1, 4)); + EXPECT_EQ(ranges_and_keys[structure_indices[0][1]].row_range(), RowRange(0, 10)); + EXPECT_EQ(ranges_and_keys[structure_indices[0][1]].col_range(), ColRange(4, 6)); + + EXPECT_EQ(structure_indices[1].size(), column_slices_per_row_slice); + EXPECT_EQ(ranges_and_keys[structure_indices[1][0]].row_range(), RowRange(10, 20)); + EXPECT_EQ(ranges_and_keys[structure_indices[1][0]].col_range(), ColRange(1, 4)); + EXPECT_EQ(ranges_and_keys[structure_indices[1][1]].row_range(), RowRange(10, 20)); + EXPECT_EQ(ranges_and_keys[structure_indices[1][1]].col_range(), ColRange(4, 6)); + + EXPECT_EQ(structure_indices[2].size(), column_slices_per_row_slice); + EXPECT_EQ(ranges_and_keys[structure_indices[2][0]].row_range(), RowRange(20, 30)); + EXPECT_EQ(ranges_and_keys[structure_indices[2][0]].col_range(), ColRange(1, 4)); + EXPECT_EQ(ranges_and_keys[structure_indices[2][1]].row_range(), RowRange(20, 30)); + EXPECT_EQ(ranges_and_keys[structure_indices[2][1]].col_range(), ColRange(4, 6)); - std::vector> row_slice_structure = - structure_by_row_slice(*component_manager, std::move(entities)); - for (size_t row_slice_idx = 0; row_slice_idx < row_slice_structure.size(); ++row_slice_idx) { - std::vector& row_slice = row_slice_structure[row_slice_idx]; - std::vector result_entities = clause.process(std::move(row_slice)); - ProcessingUnit proc = - gather_entities, std::shared_ptr, std::shared_ptr>( - *component_manager, result_entities + // This works because the entities pushed to the component manager are pre-sorted by row range and structure for + // processing of the merge clause also sorts by row slice + std::vector> entities_for_processing = + map_entities_to_structure_for_processing_output(structure_indices, entities); + std::array, 3> processing_result; + processing_result[0] = clause.process(std::move(entities_for_processing[0])); + { + auto proc_0 = + gather_entities, std::shared_ptr, std::shared_ptr>( + *component_manager, processing_result[0] ); - EXPECT_EQ(proc.row_ranges_->size(), 2); - EXPECT_EQ(proc.col_ranges_->size(), 2); } } \ No newline at end of file diff --git a/cpp/arcticdb/util/type_traits.hpp b/cpp/arcticdb/util/type_traits.hpp index a4562183c09..dd040ee1c13 100644 --- a/cpp/arcticdb/util/type_traits.hpp +++ b/cpp/arcticdb/util/type_traits.hpp @@ -36,6 +36,6 @@ concept contiguous_type_tagged_data = requires(T t) { }; template -concept forwarding_ref_to = std::same_as, Base>; +concept decays_to = std::same_as, Base>; } // namespace arcticdb::util From 40610a6c95b3b91cf81d7226e4aae12af01d0323 Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Fri, 14 Nov 2025 23:32:40 +0200 Subject: [PATCH 31/46] Passing unit test --- cpp/arcticdb/entity/native_tensor.hpp | 2 +- cpp/arcticdb/processing/clause.cpp | 78 +++++++---- cpp/arcticdb/processing/clause.hpp | 3 +- .../processing/test/test_merge_update.cpp | 125 +++++++++++------- 4 files changed, 133 insertions(+), 75 deletions(-) diff --git a/cpp/arcticdb/entity/native_tensor.hpp b/cpp/arcticdb/entity/native_tensor.hpp index 61b63c954fa..0c7e0f218d7 100644 --- a/cpp/arcticdb/entity/native_tensor.hpp +++ b/cpp/arcticdb/entity/native_tensor.hpp @@ -126,7 +126,7 @@ struct NativeTensor { [[nodiscard]] auto extent(ssize_t dim) const { return shapes_[dim] * strides_[dim]; } [[nodiscard]] auto expanded_dim() const { return expanded_dim_; } template - const T* ptr_cast(size_t pos) const { + const T* ptr_cast(const size_t pos) const { util::check(ptr != nullptr, "Unexpected null ptr in NativeTensor"); const bool dimension_condition = ndim() == 1; const bool elsize_condition = elsize_ != 0; diff --git a/cpp/arcticdb/processing/clause.cpp b/cpp/arcticdb/processing/clause.cpp index b3b94eb0abf..c74c47c0816 100644 --- a/cpp/arcticdb/processing/clause.cpp +++ b/cpp/arcticdb/processing/clause.cpp @@ -1809,24 +1809,26 @@ std::vector> MergeUpdateClause::structure_for_processing(std auto first_col_slice_in_row = ranges_and_keys.begin(); for (size_t row_slice_idx = 0; row_slice_idx < entities.size() && source_row < source_->num_rows; ++row_slice_idx) { const TimestampRange time_range = first_col_slice_in_row->key_.time_range(); - bool keep_row_slice = source_->num_rows && strategy_.not_matched_by_target == MergeAction::INSERT && - source_->index_value_at(source_row) < time_range.first; + // If we're inserting and the source timestamp is before the start of the row slice, we are going to insert the + // new values at the beginning of each column slice in row slice, thus we need to keep it. + // TODO: Improvement if strategy.matched is DO_NOTHING or there is no other intersection between source and the + // row slice there is no need to read it. We can just create a row slice. + timestamp source_ts = source_->index_value_at(source_row); + bool keep_row_slice = strategy_.not_matched_by_target == MergeAction::INSERT && source_ts < time_range.first; if (keep_row_slice) { source_start_for_row_range_[first_col_slice_in_row->row_range()] = std::pair{source_row, source_row + 1}; } - timestamp source_ts = source_->index_value_at(source_row); - // TODO: If there are values to be inserted before the first segment this will read it and prepend the values - // There is no need to read the segment. We can just create a new segment. - while (source_row < source_->num_rows && strategy_.not_matched_by_target == MergeAction::INSERT && - source_ts < time_range.first) { + // Skip all values in the source that are before the first index value in the row slice + while (source_row < source_->num_rows && source_ts < time_range.first) { source_ts = source_->index_value_at(++source_row); } - const bool source_ts_in_segment_range = source_ts >= time_range.first && source_ts < time_range.second; + const bool source_ts_in_segment_range = source_ts >= time_range.first && source_ts <= time_range.second; if (!keep_row_slice && source_ts_in_segment_range) { source_start_for_row_range_[first_col_slice_in_row->row_range()] = std::pair{source_row, source_row + 1}; } keep_row_slice |= source_ts_in_segment_range; - while (source_row < source_->num_rows && source_ts >= time_range.first && source_ts < time_range.second) { + // Find the first row in source that is after the row slice + while (source_row < source_->num_rows && source_ts >= time_range.first && source_ts <= time_range.second) { source_ts = source_->index_value_at(++source_row); } if (keep_row_slice) { @@ -1836,13 +1838,14 @@ std::vector> MergeUpdateClause::structure_for_processing(std const size_t col_slice_count = entities[row_slice_idx].size(); first_col_slice_in_row += col_slice_count; } - // TODO: If there are values to be inserted after the last segment this will read it and append the values and split + // TODO: If there are values to be inserted after the last segment this will read it, append the values and split // the segment if needed. There is no need to do this as we can just create a new segment. if (source_row < source_->num_rows && strategy_.not_matched_by_target == MergeAction::INSERT) { row_slices_to_keep[entities.size() - 1] = true; } const size_t num_row_slices_to_keep = row_slices_to_keep.count(); - if (num_row_slices_to_keep == entities.size()) { + const bool keep_all_row_slices = num_row_slices_to_keep == entities.size(); + if (keep_all_row_slices) { return entities; } size_t entity_pos = 0; @@ -1871,53 +1874,67 @@ std::vector MergeUpdateClause::process(std::vector&& entity_ ); std::vector> matched = filter_index_match(proc); if (source_->has_segment()) { - update_and_insert(source_->segment(), source_->desc(), proc, matched); + update_and_insert(source_->segment(), source_->desc(), proc, matched); } else { internal::check( source_->has_tensors(), "Input frame does not contain neither a segment nor tensors" ); - update_and_insert>(source_->field_tensors(), source_->desc(), proc, matched); + update_and_insert(source_->field_tensors(), source_->desc(), proc, matched); } return push_entities(*component_manager_, std::move(proc)); } template +requires util::any_of> void MergeUpdateClause::update_and_insert( const T& source, const StreamDescriptor& source_descriptor, const ProcessingUnit& proc, - std::span> rows_to_update + const std::span> rows_to_update ) const { const std::vector> target_segments = *proc.segments_; const std::vector> row_ranges = *proc.row_ranges_; const std::vector> col_ranges = *proc.col_ranges_; const auto [source_row_start, source_row_end] = source_start_for_row_range_.at(*row_ranges[0]); for (size_t segment_idx = 0; segment_idx < target_segments.size(); ++segment_idx) { - const ColRange& col_range = *col_ranges[segment_idx]; - for (size_t column_idx = col_range.first; column_idx < col_range.second; ++column_idx) { - entity::visit_field(source_descriptor.field(column_idx), [&](auto tdt) { + const size_t columns_in_range = target_segments[segment_idx]->num_columns(); + for (size_t column_idx = source_descriptor.index().field_count(); column_idx < columns_in_range; ++column_idx) { + entity::visit_field(target_segments[segment_idx]->descriptor().field(column_idx), [&](auto tdt) { internal::check( !is_sequence_type(tdt.data_type()), "String columns are not supported in merge update yet" ); using TDT = decltype(tdt); size_t source_row = source_row_start; - while (source_row < source_row_end && rows_to_update[source_row].empty()) { + while (source_row < source_row_end && rows_to_update[source_row_start - source_row].empty()) { ++source_row; }; - // For each row in the source rows to update contains a list of rows in the target that match this + // For each row in the source rows_to_update contains a list of rows in the target that match this // row and must be updated. If all rows in the row slice are empty this means that the current row // slice should not be updated // TODO: Do not return in case of MergeAction::INSERT if (source_row >= source_row_end) { return; } - size_t target_data_row = rows_to_update[source_row].front(); + size_t target_data_row = rows_to_update[source_row_start - source_row].front(); auto target_data_it = target_segments[segment_idx]->column(column_idx).data().begin(); // TODO: Implement operator+= because std::advance is linear std::advance(target_data_it, target_data_row); - SourceView source_column_view = get_source_column_iterator(source, column_idx); + const size_t source_column = [&]() { + if constexpr (std::same_as) { + return col_ranges[segment_idx]->first; + } else if constexpr (std::same_as>) { + // If the source frame is a list of tensors, the index frame is kept separately, so the first + // non-index column will be at index 0. If there's an index the first ColRange will start from + // 1 and the first column of each segment will contain the index + return col_ranges[segment_idx]->first + column_idx - + 2 * source_descriptor.index().field_count(); + } else { + static_assert(sizeof(T) == 0, "Invalid type"); + } + }(); + SourceView source_column_view = get_source_column_iterator(source, source_column); while (source_row < source_row_end) { - std::span rows_to_update_for_source_row = rows_to_update[source_row]; + std::span rows_to_update_for_source_row = rows_to_update[source_row_start - source_row]; if (rows_to_update_for_source_row.empty()) { ++source_row; continue; @@ -1930,6 +1947,7 @@ void MergeUpdateClause::update_and_insert( *target_data_it = source_row_value; target_data_row = target_row_to_update; } + ++source_row; } }); } @@ -1946,20 +1964,24 @@ std::vector> MergeUpdateClause::filter_index_match(const Pro using IndexType = ScalarTagType>; const std::vector> target_segments = *proc.segments_; const std::vector> row_ranges = *proc.row_ranges_; - auto [source_row, source_row_end] = source_start_for_row_range_.at(*row_ranges[0]); + const auto [source_row_start, source_row_end] = source_start_for_row_range_.at(*row_ranges[0]); + size_t source_row = source_row_start; const size_t source_rows_in_row_slice = source_row_end - source_row; std::vector> matched_rows(source_rows_in_row_slice); ColumnData target_index = target_segments[0]->column(0).data(); auto target_index_it = target_index.cbegin(); + size_t target_row = 0; const auto target_index_end = target_index.cend(); while (target_index_it != target_index_end && source_row < source_row_end) { const timestamp source_ts = source_->index_value_at(source_row); - target_index_it = std::lower_bound(target_index_it, target_index_end, source_ts); - while (*target_index_it == source_ts) { - const size_t target_row = std::distance(target_index.cbegin(), target_index_it); - matched_rows[source_row].push_back(target_row); - ++target_index_it; + auto lower_bound = std::lower_bound(target_index_it, target_index_end, source_ts); + target_row += std::distance(target_index_it, lower_bound); + while (*lower_bound == source_ts) { + matched_rows[source_row_start - source_row].push_back(target_row); + ++lower_bound; + ++target_row; } + target_index_it = lower_bound; source_row++; } return matched_rows; diff --git a/cpp/arcticdb/processing/clause.hpp b/cpp/arcticdb/processing/clause.hpp index ad04466c8ea..da645d30dba 100644 --- a/cpp/arcticdb/processing/clause.hpp +++ b/cpp/arcticdb/processing/clause.hpp @@ -878,11 +878,12 @@ struct MergeUpdateClause { private: template + requires util::any_of> void update_and_insert(const T&, const StreamDescriptor&, const ProcessingUnit&, std::span>) const; - /// @return Vector of size equal to the number of source data rows that are withing the rows lice being processed. + /// @return Vector of size equal to the number of source data rows that are withing the row slice being processed. /// Each element is a vector of the rows from the target data that has the same index as the corresponding source /// row std::vector> filter_index_match(const ProcessingUnit& proc) const; diff --git a/cpp/arcticdb/processing/test/test_merge_update.cpp b/cpp/arcticdb/processing/test/test_merge_update.cpp index 02f0a623f88..e03d3275ada 100644 --- a/cpp/arcticdb/processing/test/test_merge_update.cpp +++ b/cpp/arcticdb/processing/test/test_merge_update.cpp @@ -241,33 +241,50 @@ std::vector> map_entities_to_structure_for_processing_outp return process_input; } +void print_segment(const SegmentInMemory& segment) { + const StreamDescriptor& desc = segment.descriptor(); + for (unsigned i = 0; i < desc.field_count(); ++i) { + std::cout << fmt::format("Print column[{}]: {}\n", i, desc.field(i)); + visit_field(desc.field(i), [&](auto tdt) { + using TDT = decltype(tdt); + ColumnData cd = segment.column_data(i); + for (auto it = cd.begin(); it != cd.end(); ++it) { + if constexpr (std::same_as) { + std::cout << int(*it) << " "; + } else { + std::cout << *it << " "; + } + } + std::cout << "\n"; + }); + } +} + TEST(MergeUpdateUpdateTimeseries, SourceIndexMatchesAllSegments) { using namespace std::ranges; using stream::TimeseriesIndex; constexpr static auto strategy = MergeStrategy{.matched = MergeAction::UPDATE, .not_matched_by_target = MergeAction::DO_NOTHING}; - constexpr static size_t columns_per_segment = 3; - constexpr static size_t rows_per_segment = 10; + constexpr static int columns_per_segment = 3; + constexpr static int rows_per_segment = 5; const StreamDescriptor source_descriptor = TimeseriesIndex::default_index().create_stream_descriptor("Source", non_string_fields); auto [segments, col_ranges, row_ranges] = slice_data_into_segments( rows_per_segment, source_descriptor, - iota_view(timestamp{0}, timestamp{30}), - iota_view(static_cast(0), static_cast(30)), - iota_view(static_cast(0), static_cast(30)), - std::array{true, false, true, true, false, false, true, false, true, false, - true, true, false, true, false, false, true, true, false, true, - false, true, false, false, true, true, false, true, false, true}, - iota_view(0, 30) | views::transform([](auto x) { return static_cast(x); }), - iota_view(timestamp{0}, timestamp{30}) + iota_view(timestamp{0}, timestamp{3 * rows_per_segment}), + iota_view(static_cast(0), static_cast(3 * rows_per_segment)), + iota_view(static_cast(0), static_cast(3 * rows_per_segment)), + iota_view(0, 3 * rows_per_segment) | views::transform([](auto x) -> bool { return x % 2 == 1; }), + iota_view(0, 3 * rows_per_segment) | views::transform([](auto x) { return static_cast(x); }), + iota_view(timestamp{0}, timestamp{3 * rows_per_segment}) ); sort_by_rowslice(row_ranges, col_ranges, segments); - EXPECT_EQ(segments.size(), col_ranges.size()); - EXPECT_EQ(segments.size(), row_ranges.size()); + ASSERT_EQ(segments.size(), col_ranges.size()); + ASSERT_EQ(segments.size(), row_ranges.size()); - InputFrame source( + auto source = std::make_shared( source_descriptor, create_one_dimensional_tensors( std::pair{std::array{10, 20, 30}, TypeDescriptor::scalar_type(DataType::INT8)}, @@ -279,9 +296,9 @@ TEST(MergeUpdateUpdateTimeseries, SourceIndexMatchesAllSegments) { TypeDescriptor::scalar_type(DataType::NANOSECONDS_UTC64) } ), - NativeTensor::one_dimensional_tensor(std::array{1, 12, 25}, DataType::NANOSECONDS_UTC64) + NativeTensor::one_dimensional_tensor(std::array{0, 7, 14}, DataType::NANOSECONDS_UTC64) ); - source.num_rows = 3; + source->num_rows = 3; // Shuffle the input ranges and keys to ensure structure_for_processing sorts correctly constexpr static size_t rand_seed = 0; @@ -291,50 +308,68 @@ TEST(MergeUpdateUpdateTimeseries, SourceIndexMatchesAllSegments) { shuffle(ranges_and_keys, g); auto component_manager = std::make_shared(); - std::vector fetch_count(segments.size(), 0); - std::vector> proc_row_range = wrap_in_shared_ptr(std::move(row_ranges)); - std::vector> proc_col_range = wrap_in_shared_ptr(std::move(col_ranges)); - std::vector> proc_seg = wrap_in_shared_ptr(std::move(segments)); - std::vector entities = - component_manager->add_entities(proc_col_range, proc_row_range, proc_seg, std::move(fetch_count)); + std::vector entities = component_manager->add_entities( + wrap_in_shared_ptr(std::move(col_ranges)), + wrap_in_shared_ptr(std::move(row_ranges)), + wrap_in_shared_ptr(std::move(segments)), + std::vector(segments.size(), 0) + ); - MergeUpdateClause clause({}, strategy, std::make_shared(std::move(source)), true); + MergeUpdateClause clause({}, strategy, source, true); clause.set_component_manager(component_manager); const std::vector> structure_indices = clause.structure_for_processing(ranges_and_keys); constexpr static int row_slices_to_process = 3; constexpr static int column_slices_per_row_slice = 2; - EXPECT_EQ(structure_indices.size(), row_slices_to_process); - - EXPECT_EQ(structure_indices[0].size(), column_slices_per_row_slice); - EXPECT_EQ(ranges_and_keys[structure_indices[0][0]].row_range(), RowRange(0, 10)); - EXPECT_EQ(ranges_and_keys[structure_indices[0][0]].col_range(), ColRange(1, 4)); - EXPECT_EQ(ranges_and_keys[structure_indices[0][1]].row_range(), RowRange(0, 10)); - EXPECT_EQ(ranges_and_keys[structure_indices[0][1]].col_range(), ColRange(4, 6)); + ASSERT_EQ(structure_indices.size(), row_slices_to_process); - EXPECT_EQ(structure_indices[1].size(), column_slices_per_row_slice); - EXPECT_EQ(ranges_and_keys[structure_indices[1][0]].row_range(), RowRange(10, 20)); - EXPECT_EQ(ranges_and_keys[structure_indices[1][0]].col_range(), ColRange(1, 4)); - EXPECT_EQ(ranges_and_keys[structure_indices[1][1]].row_range(), RowRange(10, 20)); - EXPECT_EQ(ranges_and_keys[structure_indices[1][1]].col_range(), ColRange(4, 6)); - - EXPECT_EQ(structure_indices[2].size(), column_slices_per_row_slice); - EXPECT_EQ(ranges_and_keys[structure_indices[2][0]].row_range(), RowRange(20, 30)); - EXPECT_EQ(ranges_and_keys[structure_indices[2][0]].col_range(), ColRange(1, 4)); - EXPECT_EQ(ranges_and_keys[structure_indices[2][1]].row_range(), RowRange(20, 30)); - EXPECT_EQ(ranges_and_keys[structure_indices[2][1]].col_range(), ColRange(4, 6)); + for (size_t i = 0; i < structure_indices.size(); ++i) { + SCOPED_TRACE(testing::Message() << fmt::format("structure index: {}", i)); + EXPECT_EQ(structure_indices[i].size(), column_slices_per_row_slice); + EXPECT_EQ( + ranges_and_keys[structure_indices[i][0]].row_range(), + RowRange(i * rows_per_segment, (i + 1) * rows_per_segment) + ); + EXPECT_EQ( + ranges_and_keys[structure_indices[i][1]].row_range(), + RowRange(i * rows_per_segment, (i + 1) * rows_per_segment) + ); + EXPECT_EQ(ranges_and_keys[structure_indices[i][0]].col_range(), ColRange(1, 4)); + EXPECT_EQ(ranges_and_keys[structure_indices[i][1]].col_range(), ColRange(4, 6)); + } // This works because the entities pushed to the component manager are pre-sorted by row range and structure for // processing of the merge clause also sorts by row slice std::vector> entities_for_processing = map_entities_to_structure_for_processing_output(structure_indices, entities); - std::array, 3> processing_result; - processing_result[0] = clause.process(std::move(entities_for_processing[0])); - { - auto proc_0 = + + auto [expected_segments, expected_col_ranges, expected_row_ranges] = + slice_data_into_segments( + rows_per_segment, + source_descriptor, + iota_view(timestamp{0}, timestamp{3 * rows_per_segment}), + std::array{10, 1, 2, 3, 4, 5, 6, 20, 8, 9, 10, 11, 12, 13, 30}, + std::array{100, 1, 2, 3, 4, 5, 6, 200, 8, 9, 10, 11, 12, 13, 300}, + std::array{1, 1, 0, 1, 0, 1, 0, 0, 0, 1, 0, 1, 0, 1, 1}, + std::array{11.1f, 1, 2, 3, 4, 5, 6, 22.2f, 8, 9, 10, 11, 12, 13, 33.3f}, + std::array{1000, 1, 2, 3, 4, 5, 6, 2000, 8, 9, 10, 11, 12, 13, 3000} + ); + sort_by_rowslice(expected_row_ranges, expected_col_ranges, expected_segments); + for (size_t i = 0; i < entities_for_processing.size(); ++i) { + auto proc = gather_entities, std::shared_ptr, std::shared_ptr>( - *component_manager, processing_result[0] + *component_manager, clause.process(std::move(entities_for_processing[i])) ); + SCOPED_TRACE(testing::Message() << "processing result = " << i); + EXPECT_EQ(proc.segments_->size(), 2); + EXPECT_EQ(proc.row_ranges_->size(), 2); + EXPECT_EQ(proc.col_ranges_->size(), 2); + EXPECT_EQ(*(proc.row_ranges_.value())[0], RowRange(i * rows_per_segment, (i + 1) * rows_per_segment)); + EXPECT_EQ(*(proc.row_ranges_.value())[1], RowRange(i * rows_per_segment, (i + 1) * rows_per_segment)); + EXPECT_EQ(*(proc.col_ranges_.value())[0], RowRange(1, 4)); + EXPECT_EQ(*(proc.col_ranges_.value())[1], RowRange(4, 6)); + EXPECT_EQ(*(proc.segments_.value())[0], expected_segments[i * 2]); + EXPECT_EQ(*(proc.segments_.value())[1], expected_segments[i * 2 + 1]); } } \ No newline at end of file From b1052de6fba5650e7821bac225819388fabad33c Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Wed, 19 Nov 2025 13:23:11 +0200 Subject: [PATCH 32/46] Fix input_frame_from_tensors Allow all sized ranges for creation of native tensors Parametrize tests Unify code in testing Add more unit tests Change some signatures Add read modify write index strategy --- cpp/arcticdb/entity/native_tensor.hpp | 26 - cpp/arcticdb/processing/clause.cpp | 73 +- .../processing/test/test_merge_update.cpp | 971 ++++++++++++++---- .../version/local_versioned_engine.cpp | 5 +- cpp/arcticdb/version/version_core.cpp | 86 +- cpp/arcticdb/version/version_core.hpp | 19 +- 6 files changed, 901 insertions(+), 279 deletions(-) diff --git a/cpp/arcticdb/entity/native_tensor.hpp b/cpp/arcticdb/entity/native_tensor.hpp index 0c7e0f218d7..a860ddabc10 100644 --- a/cpp/arcticdb/entity/native_tensor.hpp +++ b/cpp/arcticdb/entity/native_tensor.hpp @@ -169,32 +169,6 @@ struct NativeTensor { int expanded_dim_; }; -/// Creates a vector of NativeTensors from pairs of 1D contiguous data and DataType. -/// Each input must be a pair-like object where `first` is a contiguous range and `second` is a DataType. -/// The resulting tensors can be passed to InputFrame. -/// @tparam Input Parameter pack of pair-like types satisfying the contiguous_range requirement -/// @param arrays Variadic pairs of (contiguous_range, DataType) -/// @return Vector of one-dimensional NativeTensors -template -std::vector create_one_dimensional_tensors(const Input&... arrays) { - std::vector tensors; - tensors.reserve(sizeof...(arrays)); - (tensors.emplace_back(NativeTensor::one_dimensional_tensor(arrays.first, arrays.second.data_type())), ...); - return tensors; -} - -template -requires util::contiguous_type_tagged_data -std::vector create_one_dimensional_tensors(std::span contiguous_columns) { - std::vector tensors; - for (const T& contiguous_column_data : contiguous_columns) { - tensors.emplace_back(NativeTensor::one_dimensional_tensor( - contiguous_column_data.first, contiguous_column_data.second.data_type() - )); - } - return tensors; -} - template ssize_t byte_offset_impl(const stride_t*) { return 0; diff --git a/cpp/arcticdb/processing/clause.cpp b/cpp/arcticdb/processing/clause.cpp index c74c47c0816..9f215896783 100644 --- a/cpp/arcticdb/processing/clause.cpp +++ b/cpp/arcticdb/processing/clause.cpp @@ -29,6 +29,32 @@ #include +namespace { +template +void stable_select(std::vector& vec, std::span indexes_to_keep) { + arcticdb::debug::check( + std::ranges::is_sorted(indexes_to_keep), + "Bulk removal of elements from vector requires the indexes of the elements to be sorted" + ); + if (indexes_to_keep.size() == vec.size()) { + return; + } + if (indexes_to_keep.empty()) { + vec.resize(0); + } + size_t free_slot = 0, to_keep = 0; + for (size_t i = 0; i < vec.size(); ++i) { + if (free_slot == indexes_to_keep[to_keep]) { + to_keep++; + free_slot++; + } else if (i == indexes_to_keep[to_keep]) { + vec[free_slot++] = std::move(vec[indexes_to_keep[to_keep++]]); + } + } + vec.resize(indexes_to_keep.size()); +} +} // namespace + namespace arcticdb { namespace ranges = std::ranges; @@ -1804,7 +1830,7 @@ std::vector> MergeUpdateClause::structure_for_processing(std return structure_by_row_slice(ranges_and_keys); } std::vector> entities = structure_by_row_slice(ranges_and_keys); - util::BitSet row_slices_to_keep(entities.size()); + std::vector row_slices_to_keep; size_t source_row = 0; auto first_col_slice_in_row = ranges_and_keys.begin(); for (size_t row_slice_idx = 0; row_slice_idx < entities.size() && source_row < source_->num_rows; ++row_slice_idx) { @@ -1817,6 +1843,7 @@ std::vector> MergeUpdateClause::structure_for_processing(std bool keep_row_slice = strategy_.not_matched_by_target == MergeAction::INSERT && source_ts < time_range.first; if (keep_row_slice) { source_start_for_row_range_[first_col_slice_in_row->row_range()] = std::pair{source_row, source_row + 1}; + row_slices_to_keep.push_back(row_slice_idx); } // Skip all values in the source that are before the first index value in the row slice while (source_row < source_->num_rows && source_ts < time_range.first) { @@ -1825,6 +1852,7 @@ std::vector> MergeUpdateClause::structure_for_processing(std const bool source_ts_in_segment_range = source_ts >= time_range.first && source_ts <= time_range.second; if (!keep_row_slice && source_ts_in_segment_range) { source_start_for_row_range_[first_col_slice_in_row->row_range()] = std::pair{source_row, source_row + 1}; + row_slices_to_keep.push_back(row_slice_idx); } keep_row_slice |= source_ts_in_segment_range; // Find the first row in source that is after the row slice @@ -1834,30 +1862,10 @@ std::vector> MergeUpdateClause::structure_for_processing(std if (keep_row_slice) { source_start_for_row_range_.at(first_col_slice_in_row->row_range()).second = source_row; } - row_slices_to_keep[row_slice_idx] = keep_row_slice; const size_t col_slice_count = entities[row_slice_idx].size(); first_col_slice_in_row += col_slice_count; } - // TODO: If there are values to be inserted after the last segment this will read it, append the values and split - // the segment if needed. There is no need to do this as we can just create a new segment. - if (source_row < source_->num_rows && strategy_.not_matched_by_target == MergeAction::INSERT) { - row_slices_to_keep[entities.size() - 1] = true; - } - const size_t num_row_slices_to_keep = row_slices_to_keep.count(); - const bool keep_all_row_slices = num_row_slices_to_keep == entities.size(); - if (keep_all_row_slices) { - return entities; - } - size_t entity_pos = 0; - for (size_t i = 0; i < entities.size(); ++i) { - if (row_slices_to_keep[i]) { - if (entity_pos != i) { - entities[entity_pos] = std::move(entities[i]); - } - ++entity_pos; - } - } - entities.resize(num_row_slices_to_keep); + stable_select(entities, row_slices_to_keep); return entities; } @@ -1890,9 +1898,9 @@ void MergeUpdateClause::update_and_insert( const T& source, const StreamDescriptor& source_descriptor, const ProcessingUnit& proc, const std::span> rows_to_update ) const { - const std::vector> target_segments = *proc.segments_; - const std::vector> row_ranges = *proc.row_ranges_; - const std::vector> col_ranges = *proc.col_ranges_; + const std::span> target_segments = *proc.segments_; + const std::span> row_ranges = *proc.row_ranges_; + const std::span> col_ranges = *proc.col_ranges_; const auto [source_row_start, source_row_end] = source_start_for_row_range_.at(*row_ranges[0]); for (size_t segment_idx = 0; segment_idx < target_segments.size(); ++segment_idx) { const size_t columns_in_range = target_segments[segment_idx]->num_columns(); @@ -1904,7 +1912,7 @@ void MergeUpdateClause::update_and_insert( using TDT = decltype(tdt); size_t source_row = source_row_start; - while (source_row < source_row_end && rows_to_update[source_row_start - source_row].empty()) { + while (source_row < source_row_end && rows_to_update[source_row - source_row_start].empty()) { ++source_row; }; // For each row in the source rows_to_update contains a list of rows in the target that match this @@ -1914,7 +1922,7 @@ void MergeUpdateClause::update_and_insert( if (source_row >= source_row_end) { return; } - size_t target_data_row = rows_to_update[source_row_start - source_row].front(); + size_t target_data_row = rows_to_update[source_row - source_row_start].front(); auto target_data_it = target_segments[segment_idx]->column(column_idx).data().begin(); // TODO: Implement operator+= because std::advance is linear @@ -1934,7 +1942,7 @@ void MergeUpdateClause::update_and_insert( }(); SourceView source_column_view = get_source_column_iterator(source, source_column); while (source_row < source_row_end) { - std::span rows_to_update_for_source_row = rows_to_update[source_row_start - source_row]; + std::span rows_to_update_for_source_row = rows_to_update[source_row - source_row_start]; if (rows_to_update_for_source_row.empty()) { ++source_row; continue; @@ -1962,8 +1970,8 @@ template void MergeUpdateClause:: std::vector> MergeUpdateClause::filter_index_match(const ProcessingUnit& proc) const { using IndexType = ScalarTagType>; - const std::vector> target_segments = *proc.segments_; - const std::vector> row_ranges = *proc.row_ranges_; + const std::span> target_segments{*proc.segments_}; + const std::span> row_ranges{*proc.row_ranges_}; const auto [source_row_start, source_row_end] = source_start_for_row_range_.at(*row_ranges[0]); size_t source_row = source_row_start; const size_t source_rows_in_row_slice = source_row_end - source_row; @@ -1975,9 +1983,12 @@ std::vector> MergeUpdateClause::filter_index_match(const Pro while (target_index_it != target_index_end && source_row < source_row_end) { const timestamp source_ts = source_->index_value_at(source_row); auto lower_bound = std::lower_bound(target_index_it, target_index_end, source_ts); + if (lower_bound == target_index_end) { + break; + } target_row += std::distance(target_index_it, lower_bound); while (*lower_bound == source_ts) { - matched_rows[source_row_start - source_row].push_back(target_row); + matched_rows[source_row - source_row_start].push_back(target_row); ++lower_bound; ++target_row; } diff --git a/cpp/arcticdb/processing/test/test_merge_update.cpp b/cpp/arcticdb/processing/test/test_merge_update.cpp index e03d3275ada..ef8f96b3079 100644 --- a/cpp/arcticdb/processing/test/test_merge_update.cpp +++ b/cpp/arcticdb/processing/test/test_merge_update.cpp @@ -12,9 +12,11 @@ #include #include #include +#include #include using namespace arcticdb; +using namespace std::ranges; constexpr static std::array non_string_fields = { FieldRef(TypeDescriptor(DataType::INT8, Dimension::Dim0), "int8"), @@ -24,6 +26,10 @@ constexpr static std::array non_string_fields = { FieldRef(TypeDescriptor(DataType::NANOSECONDS_UTC64, Dimension::Dim0), "timestamp") }; +StreamDescriptor non_string_fields_ts_index_descriptor() { + return TimeseriesIndex::default_index().create_stream_descriptor("Source", non_string_fields); +} + std::vector split_descriptor(const StreamDescriptor& descriptor, const size_t cols_per_segment) { if (descriptor.fields().size() <= cols_per_segment) { return std::vector{descriptor}; @@ -49,117 +55,165 @@ std::vector split_descriptor(const StreamDescriptor& descripto return res; } -template -auto take(Data&&... data) { - constexpr size_t to_take = std::min(N, sizeof...(Data)); - return std::pair{ - [&](std::index_sequence) { - return std::tuple{std::get(std::forward_as_tuple(std::forward(data)...))...}; - }(std::make_index_sequence{}), - [&](std::index_sequence) { - return std::tuple{std::get(std::forward_as_tuple(std::forward(data)...))...}; - }(std::make_index_sequence{}) - }; +template +requires util::instantiation_of && util::instantiation_of +auto tuple_cat_ref(const T1& tuple1, const T2& tuple2) { + return std::apply( + [&](auto&&... args1) { + return std::apply( + [&](auto&&... args2) { + return std::tie( + std::forward(args1)..., std::forward(args2)... + ); + }, + tuple2 + ); + }, + tuple1 + ); } -template -requires(util::instantiation_of) -auto take(TupleT&& t) { - constexpr size_t to_take = std::min(N, std::tuple_size_v); - return std::pair{ - [&](std::index_sequence) { - return std::tuple{std::get(t)...}; - }(std::make_index_sequence{}), - [&](std::index_sequence) { - return std::tuple{std::get(t)...}; - }(std::make_index_sequence - to_take>{}), - }; +template +requires(sizeof...(Data) > 0) +void slice_column_slice_into_row_slices( + const std::tuple& index, const std::tuple& column_slice, const StreamDescriptor& desc, + const size_t rows_per_segment, const ColRange& col_range, std::vector& segments, + std::vector& col_ranges, std::vector& row_ranges +) { + const size_t total_rows = std::ranges::size(std::get<0>(column_slice)); + for (size_t start_row = 0; start_row < total_rows; start_row += rows_per_segment) { + std::apply( + [&](const Cols&... cols) { + const size_t rows_to_take = std::min(rows_per_segment, total_rows - start_row); + segments.push_back(SegmentInMemory::create_dense_segment( + desc, std::ranges::take_view(std::ranges::drop_view(cols, start_row), rows_to_take)... + )); + row_ranges.emplace_back(start_row, start_row + rows_to_take); + col_ranges.emplace_back(col_range); + }, + tuple_cat_ref(index, column_slice) + ); + } +} + +template +auto split(Data&&... data) { + constexpr static auto non_index_columns = sizeof...(Data) - N; + return [&]( + std::index_sequence, std::index_sequence, T&& fwd_tuple + ) { + return std::pair{ + std::forward_as_tuple(std::get(std::forward(fwd_tuple))...), + std::forward_as_tuple(std::get(std::forward(fwd_tuple))...) + }; + }(std::make_index_sequence{}, + std::make_index_sequence{}, + std::forward_as_tuple(std::forward(data)...)); } -template -requires(cols_per_segment > 0) +template< + std::ranges::sized_range IndexCols, typename ColumnSlice, std::ranges::sized_range CurrentCol, + std::ranges::sized_range... RestCols> +requires( + util::instantiation_of && + [](std::index_sequence) { + return (std::ranges::sized_range> && ...); + }(std::make_index_sequence>{}) +) void slice_data_into_segments( - const std::span descriptors, const size_t col_slice, const size_t rows_per_segment, - std::vector& segments, std::vector& col_ranges, std::vector& row_ranges, - IndexCols&& index, Data&&... data + const std::span descriptors, const size_t rows_per_segment, + const size_t cols_per_segment, ColRange col_range, std::vector& segments, + std::vector& col_ranges, std::vector& row_ranges, const std::tuple& index, + ColumnSlice&& column_slice, CurrentCol&& current_col, RestCols&&... data ) { - constexpr size_t index_columns_count = std::tuple_size_v>; - if constexpr (sizeof...(Data) > 0) { - const StreamDescriptor& desc = descriptors[col_slice]; - constexpr static size_t current_columns_count = std::min(cols_per_segment, sizeof...(Data)); - auto [current, rest] = take(std::forward(data)...); - const size_t total_rows = std::ranges::size(std::get<0>(current)); - auto current_with_index = - std::tuple_cat(std::forward(index), std::forward(current)); - for (size_t start_row = 0; start_row < total_rows; start_row += rows_per_segment) { - std::apply( - [&](const Cols&... cols) { - const size_t rows_to_take = std::min(rows_per_segment, total_rows - start_row); - segments.push_back(SegmentInMemory::create_dense_segment( - desc, std::ranges::take_view(std::ranges::drop_view(cols, start_row), rows_to_take)... - )); - row_ranges.emplace_back(start_row, start_row + rows_to_take); - col_ranges.emplace_back( - index_columns_count + col_slice * cols_per_segment, - index_columns_count + col_slice * cols_per_segment + - std::min(cols_per_segment, current_columns_count) - ); - }, - current_with_index - ); - } - auto [next_index, _] = - take(std::forward(current_with_index)); - std::apply( - [&](auto&&... cols) { - slice_data_into_segments( - descriptors, - col_slice + 1, - rows_per_segment, - segments, - col_ranges, - row_ranges, - std::forward(next_index), - cols... - ); - }, - std::forward(rest) + auto new_column_slice = std::tuple_cat( + std::forward(column_slice), std::forward_as_tuple(std::forward(current_col)) + ); + + constexpr size_t new_slice_size = std::tuple_size_v + 1; + + if constexpr (sizeof...(RestCols) == 0) { + slice_column_slice_into_row_slices( + index, + new_column_slice, + descriptors.front(), + rows_per_segment, + col_range, + segments, + col_ranges, + row_ranges + ); + } else if (new_slice_size == cols_per_segment) { + slice_column_slice_into_row_slices( + index, + new_column_slice, + descriptors.front(), + rows_per_segment, + col_range, + segments, + col_ranges, + row_ranges + ); + col_range.first = col_range.second; + col_range.second = std::min(col_range.first + cols_per_segment, col_range.first + sizeof...(RestCols)); + slice_data_into_segments( + descriptors.last(descriptors.size() - 1), + rows_per_segment, + cols_per_segment, + col_range, + segments, + col_ranges, + row_ranges, + index, + std::tuple{}, + std::forward(data)... + ); + } else { + slice_data_into_segments( + descriptors, + rows_per_segment, + cols_per_segment, + col_range, + segments, + col_ranges, + row_ranges, + index, + std::move(new_column_slice), + std::forward(data)... ); } } -template +template std::tuple, std::vector, std::vector> slice_data_into_segments( - size_t rows_per_segment, const StreamDescriptor& descriptor, Data&&... data + const StreamDescriptor& descriptor, const size_t rows_per_segment, const size_t cols_per_segment, Data&&... data ) { std::vector descriptors = split_descriptor(descriptor, cols_per_segment); - auto [index_data, non_index] = take(std::forward(data)...); std::vector segments; std::vector col_ranges; std::vector row_ranges; + + auto [index_columns, data_columns] = split(std::forward(data)...); std::apply( - [&](NonIndexCols&&... cols) { - slice_data_into_segments( + [&](Cols&&... cols) { + slice_data_into_segments( descriptors, - 0, rows_per_segment, + cols_per_segment, + ColRange{ + index::field_count(), index::field_count() + std::min(cols_per_segment, sizeof...(Cols)) + }, segments, col_ranges, row_ranges, - std::forward(index_data), - std::forward(cols)... + std::move(index_columns), + std::tuple{}, + std::forward(cols)... ); }, - std::forward(non_index) + std::forward(data_columns) ); - return {std::move(segments), std::move(col_ranges), std::move(row_ranges)}; -} - -template -std::vector slice_data_into_segments( - size_t rows_per_segment, const StreamDescriptor& descriptor, Data&&... data -) { - return slice_data_into_segments(rows_per_segment, descriptor, std::forward(data)...); + return std::make_tuple(std::move(segments), std::move(col_ranges), std::move(row_ranges)); } template @@ -260,116 +314,659 @@ void print_segment(const SegmentInMemory& segment) { } } -TEST(MergeUpdateUpdateTimeseries, SourceIndexMatchesAllSegments) { - using namespace std::ranges; +template +requires std::ranges::sized_range && (std::ranges::sized_range && ...) +auto materialize_ranges(H&& head, T&&... tail) { + if constexpr (sizeof...(T) == 0) { + if constexpr (!std::ranges::contiguous_range) { + return std::forward_as_tuple(std::vector>(std::forward(head))); + } else { + return std::forward_as_tuple(std::forward(head)); + } + } else { + if constexpr (!std::ranges::contiguous_range) { + return std::tuple_cat( + std::forward_as_tuple(std::vector>(std::forward(head))), + materialize_ranges(std::forward(tail)...) + ); + } else { + return std::tuple_cat( + std::forward_as_tuple(std::forward(head)), materialize_ranges(std::forward(tail)...) + ); + } + } +} + +template +requires((Index::field_count() == 0 || Index::field_count() == 1) && (std::ranges::sized_range && ...)) +auto input_frame_from_tensors(const StreamDescriptor& desc, T&&... input) { + constexpr static size_t data_columns = sizeof...(T) - Index::field_count(); + auto materialized_input = materialize_ranges(std::forward(input)...); + [&](std::index_sequence) { + const size_t first_row_count = std::get<0>(materialized_input).size(); + util::check( + ((std::ranges::size(std::get(materialized_input)) == first_row_count) && ...), + "All input data must have the same number of rows" + ); + }(std::make_index_sequence{}); + std::vector tensors = [&](std::index_sequence) { + std::vector result_tensors; + result_tensors.reserve(data_columns); + (result_tensors.push_back(NativeTensor::one_dimensional_tensor( + std::get(materialized_input), + desc.field(Is + Index::field_count()).type().data_type() + )), + ...); + return result_tensors; + }(std::make_index_sequence{}); + const size_t num_rows = std::ranges::size(std::get<0>(materialized_input)); + if constexpr (Index::field_count() == 1) { + InputFrame result_frame( + desc, + std::move(tensors), + NativeTensor::one_dimensional_tensor(std::get<0>(materialized_input), desc.field(0).type().data_type()) + ); + result_frame.num_rows = num_rows; + return std::pair{std::move(result_frame), std::move(materialized_input)}; + } else { + InputFrame result_frame(desc, std::move(tensors), std::nullopt); + result_frame.num_rows = num_rows; + return std::pair{std::move(result_frame), std::move(materialized_input)}; + } +} + +struct MergeUpdateClauseUpdateStrategyTestBase { + MergeUpdateClauseUpdateStrategyTestBase( + const StreamDescriptor& desc, MergeStrategy strategy, + std::tuple, std::vector, std::vector>&& data + ) : + descriptor_(desc), + strategy_(strategy) { + auto [segments, col_ranges, row_ranges] = std::move(data); + sort_by_rowslice(row_ranges, col_ranges, segments); + ranges_and_keys_ = generate_ranges_and_keys(descriptor_, segments, col_ranges, row_ranges); + const size_t entity_count = segments.size(); + initial_entities_ = component_manager_->add_entities( + wrap_in_shared_ptr(std::move(col_ranges)), + wrap_in_shared_ptr(std::move(row_ranges)), + wrap_in_shared_ptr(std::move(segments)), + std::vector(entity_count, 0) + ); + } + + MergeUpdateClause create_clause(InputFrame&& input_frame) const { + MergeUpdateClause clause({}, strategy_, std::make_shared(std::move(input_frame)), true); + clause.set_component_manager(component_manager_); + return clause; + } + + StreamDescriptor descriptor_; + std::vector ranges_and_keys_; + std::shared_ptr component_manager_ = std::make_shared(); + MergeStrategy strategy_; + std::vector initial_entities_; +}; + +/// Param is a tuple of rows_per_segment and cols_per_segment +struct MergeUpdateClauseUpdateStrategyMatchAllSegTest : MergeUpdateClauseUpdateStrategyTestBase, + testing::TestWithParam> { + MergeUpdateClauseUpdateStrategyMatchAllSegTest() : + MergeUpdateClauseUpdateStrategyTestBase( + non_string_fields_ts_index_descriptor(), + MergeStrategy{.matched = MergeAction::UPDATE, .not_matched_by_target = MergeAction::DO_NOTHING}, + slice_data_into_segments( + non_string_fields_ts_index_descriptor(), rows_per_segment(), cols_per_segment(), + iota_view(timestamp{0}, timestamp{num_rows_}), + iota_view(static_cast(0), static_cast(num_rows_)), + iota_view(static_cast(0), static_cast(num_rows_)), + iota_view(0, num_rows_) | views::transform([](auto x) -> bool { return x % 2 == 1; }), + iota_view(0, num_rows_) | views::transform([](auto x) { return static_cast(x); }), + iota_view(timestamp{0}, timestamp{num_rows_}) + ) + ) { + // Shuffle the input ranges and keys to ensure structure_for_processing sorts correctly + constexpr static size_t rand_seed = 0; + std::mt19937 g(rand_seed); + shuffle(ranges_and_keys_, g); + } + + [[nodiscard]] static int rows_per_segment() { return std::get<0>(GetParam()); } + [[nodiscard]] static int cols_per_segment() { return std::get<1>(GetParam()); } + constexpr static int num_rows_ = 15; + constexpr static int num_cols_ = non_string_fields.size(); + [[nodiscard]] static RowRange row_range_for_row_slice(const size_t row_slice) { + const size_t start_row = row_slice * rows_per_segment(); + const size_t end_row = std::min(start_row + rows_per_segment(), size_t(num_rows_)); + return RowRange{start_row, end_row}; + } + [[nodiscard]] static ColRange col_range_for_col_slice(const size_t col_slice_inside_row_slice) { + const size_t start_col = TimeseriesIndex::field_count() + col_slice_inside_row_slice * cols_per_segment(); + const size_t end_col = + std::min(start_col + cols_per_segment(), TimeseriesIndex::field_count() + size_t(num_cols_)); + return ColRange{start_col, end_col}; + } + + void assert_structure_for_processing_creates_ordered_row_sliced_data_containing_all_segments( + const std::span> structure_indices + ) const { + const size_t row_slices_to_process = (num_rows_ + rows_per_segment() - 1) / rows_per_segment(); + const size_t column_slices_per_row_slice = (num_cols_ + cols_per_segment() - 1) / cols_per_segment(); + ASSERT_EQ(structure_indices.size(), row_slices_to_process); + for (size_t i = 0; i < structure_indices.size(); ++i) { + SCOPED_TRACE(testing::Message() << fmt::format("structure index: {}", i)); + EXPECT_EQ(structure_indices[i].size(), column_slices_per_row_slice); + const RowRange current_row_range = row_range_for_row_slice(i); + for (size_t j = 0; j < structure_indices[i].size(); ++j) { + SCOPED_TRACE(testing::Message() << fmt::format("column slice index: {}", j)); + EXPECT_EQ(ranges_and_keys_[structure_indices[i][j]].row_range(), current_row_range); + EXPECT_EQ(ranges_and_keys_[structure_indices[i][j]].col_range(), col_range_for_col_slice(j)); + } + } + } + + void assert_process_results_match_expected( + const MergeUpdateClause& clause, const std::span expected_segments, + const std::span> structure_indices, + std::vector>&& entities_to_process + ) const { + const size_t column_slices_per_row_slice = (num_cols_ + cols_per_segment() - 1) / cols_per_segment(); + for (size_t i = 0; i < entities_to_process.size(); ++i) { + auto proc = gather_entities< + std::shared_ptr, + std::shared_ptr, + std::shared_ptr>(*component_manager_, clause.process(std::move(entities_to_process[i]))); + SCOPED_TRACE(testing::Message() << "processing result (row slice) = " << i); + EXPECT_EQ(proc.segments_->size(), column_slices_per_row_slice); + EXPECT_EQ(proc.row_ranges_->size(), column_slices_per_row_slice); + EXPECT_EQ(proc.col_ranges_->size(), column_slices_per_row_slice); + for (size_t j = 0; j < column_slices_per_row_slice; ++j) { + SCOPED_TRACE(testing::Message() << "processing result (col slice) = " << j); + EXPECT_EQ(*(proc.row_ranges_.value())[j], ranges_and_keys_[structure_indices[i][j]].row_range()); + EXPECT_EQ(*(proc.col_ranges_.value())[j], ranges_and_keys_[structure_indices[i][j]].col_range()); + EXPECT_EQ(*(proc.segments_.value())[j], expected_segments[i * column_slices_per_row_slice + j]); + } + } + } +}; + +TEST_P(MergeUpdateClauseUpdateStrategyMatchAllSegTest, SourceIndexMatchesAllSegments) { using stream::TimeseriesIndex; - constexpr static auto strategy = - MergeStrategy{.matched = MergeAction::UPDATE, .not_matched_by_target = MergeAction::DO_NOTHING}; - constexpr static int columns_per_segment = 3; - constexpr static int rows_per_segment = 5; - const StreamDescriptor source_descriptor = - TimeseriesIndex::default_index().create_stream_descriptor("Source", non_string_fields); - auto [segments, col_ranges, row_ranges] = slice_data_into_segments( - rows_per_segment, - source_descriptor, - iota_view(timestamp{0}, timestamp{3 * rows_per_segment}), - iota_view(static_cast(0), static_cast(3 * rows_per_segment)), - iota_view(static_cast(0), static_cast(3 * rows_per_segment)), - iota_view(0, 3 * rows_per_segment) | views::transform([](auto x) -> bool { return x % 2 == 1; }), - iota_view(0, 3 * rows_per_segment) | views::transform([](auto x) { return static_cast(x); }), - iota_view(timestamp{0}, timestamp{3 * rows_per_segment}) + auto [input_frame, input_frame_data] = input_frame_from_tensors( + descriptor_, + std::array{0, 7, 14}, + std::array{10, 20, 30}, + std::array{100, 200, 300}, + std::array{true, false, true}, + std::array{11.1f, 22.2f, 33.3f}, + std::array{1000, 2000, 3000} ); - sort_by_rowslice(row_ranges, col_ranges, segments); - - ASSERT_EQ(segments.size(), col_ranges.size()); - ASSERT_EQ(segments.size(), row_ranges.size()); - - auto source = std::make_shared( - source_descriptor, - create_one_dimensional_tensors( - std::pair{std::array{10, 20, 30}, TypeDescriptor::scalar_type(DataType::INT8)}, - std::pair{std::array{100, 200, 300}, TypeDescriptor::scalar_type(DataType::UINT32)}, - std::pair{std::array{true, false, true}, TypeDescriptor::scalar_type(DataType::BOOL8)}, - std::pair{std::array{11.1f, 22.2f, 33.3f}, TypeDescriptor::scalar_type(DataType::FLOAT32)}, - std::pair{ - std::array{1000, 2000, 3000}, - TypeDescriptor::scalar_type(DataType::NANOSECONDS_UTC64) - } - ), - NativeTensor::one_dimensional_tensor(std::array{0, 7, 14}, DataType::NANOSECONDS_UTC64) + MergeUpdateClause clause = create_clause(std::move(input_frame)); + const std::vector> structure_indices = clause.structure_for_processing(ranges_and_keys_); + assert_structure_for_processing_creates_ordered_row_sliced_data_containing_all_segments(structure_indices); + auto [expected_segments, expected_col_ranges, expected_row_ranges] = slice_data_into_segments( + descriptor_, + rows_per_segment(), + cols_per_segment(), + iota_view(timestamp{0}, timestamp{3 * rows_per_segment()}), + std::array{10, 1, 2, 3, 4, 5, 6, 20, 8, 9, 10, 11, 12, 13, 30}, + std::array{100, 1, 2, 3, 4, 5, 6, 200, 8, 9, 10, 11, 12, 13, 300}, + std::array{1, 1, 0, 1, 0, 1, 0, 0, 0, 1, 0, 1, 0, 1, 1}, + std::array{11.1f, 1, 2, 3, 4, 5, 6, 22.2f, 8, 9, 10, 11, 12, 13, 33.3f}, + std::array{1000, 1, 2, 3, 4, 5, 6, 2000, 8, 9, 10, 11, 12, 13, 3000} ); - source->num_rows = 3; - - // Shuffle the input ranges and keys to ensure structure_for_processing sorts correctly - constexpr static size_t rand_seed = 0; - std::mt19937 g(rand_seed); - std::vector ranges_and_keys = - generate_ranges_and_keys(source_descriptor, segments, col_ranges, row_ranges); - shuffle(ranges_and_keys, g); - - auto component_manager = std::make_shared(); - std::vector entities = component_manager->add_entities( - wrap_in_shared_ptr(std::move(col_ranges)), - wrap_in_shared_ptr(std::move(row_ranges)), - wrap_in_shared_ptr(std::move(segments)), - std::vector(segments.size(), 0) + sort_by_rowslice(expected_row_ranges, expected_col_ranges, expected_segments); + assert_process_results_match_expected( + clause, + expected_segments, + structure_indices, + map_entities_to_structure_for_processing_output(structure_indices, initial_entities_) ); +} + +TEST_P(MergeUpdateClauseUpdateStrategyMatchAllSegTest, SourceHasValuesOutsideOfTheDataFrame) { + using stream::TimeseriesIndex; - MergeUpdateClause clause({}, strategy, source, true); - clause.set_component_manager(component_manager); + auto [input_frame, input_frame_data] = input_frame_from_tensors( + descriptor_, + std::array{-10, -9, 0, 7, 14, 20, 25}, + std::array{-1, -2, 10, 20, 30, -10, -20}, + std::array{10, 20, 100, 200, 300, 30, 40}, + std::array{false, true, true, false, true, true, false}, + std::array{-10.f, -11.f, 11.1f, 22.2f, 33.3f, -12.f, -14.f}, + std::array{-11, -12, 1000, 2000, 3000, -13, -14} + ); + MergeUpdateClause clause = create_clause(std::move(input_frame)); + const std::vector> structure_indices = clause.structure_for_processing(ranges_and_keys_); + assert_structure_for_processing_creates_ordered_row_sliced_data_containing_all_segments(structure_indices); + auto [expected_segments, expected_col_ranges, expected_row_ranges] = slice_data_into_segments( + descriptor_, + rows_per_segment(), + cols_per_segment(), + iota_view(timestamp{0}, timestamp{3 * rows_per_segment()}), + std::array{10, 1, 2, 3, 4, 5, 6, 20, 8, 9, 10, 11, 12, 13, 30}, + std::array{100, 1, 2, 3, 4, 5, 6, 200, 8, 9, 10, 11, 12, 13, 300}, + std::array{1, 1, 0, 1, 0, 1, 0, 0, 0, 1, 0, 1, 0, 1, 1}, + std::array{11.1f, 1, 2, 3, 4, 5, 6, 22.2f, 8, 9, 10, 11, 12, 13, 33.3f}, + std::array{1000, 1, 2, 3, 4, 5, 6, 2000, 8, 9, 10, 11, 12, 13, 3000} + ); + sort_by_rowslice(expected_row_ranges, expected_col_ranges, expected_segments); + assert_process_results_match_expected( + clause, + expected_segments, + structure_indices, + map_entities_to_structure_for_processing_output(structure_indices, initial_entities_) + ); +} - const std::vector> structure_indices = clause.structure_for_processing(ranges_and_keys); +INSTANTIATE_TEST_SUITE_P( + MergeUpdateUpdateStrategy, MergeUpdateClauseUpdateStrategyMatchAllSegTest, + testing::Values( + /* Default */ std::tuple{100000, 127}, + /* Row slices are multiple of the number of rows*/ std::tuple{5, 3}, + /*Single column slice*/ std::tuple{6, 5}, + /*Single row slice*/ std::tuple{20, 2} + ) +); - constexpr static int row_slices_to_process = 3; - constexpr static int column_slices_per_row_slice = 2; - ASSERT_EQ(structure_indices.size(), row_slices_to_process); +INSTANTIATE_TEST_SUITE_P( + SourceHasValuesOutsideOfTheDataFrame, MergeUpdateClauseUpdateStrategyMatchAllSegTest, + testing::Values( + /* Default */ std::tuple{100000, 127}, + /* Row slices are multiple of the number of rows*/ std::tuple{5, 3}, + /*Single column slice*/ std::tuple{6, 5}, + /*Single row slice*/ std::tuple{20, 2} + ) +); - for (size_t i = 0; i < structure_indices.size(); ++i) { - SCOPED_TRACE(testing::Message() << fmt::format("structure index: {}", i)); - EXPECT_EQ(structure_indices[i].size(), column_slices_per_row_slice); - EXPECT_EQ( - ranges_and_keys[structure_indices[i][0]].row_range(), - RowRange(i * rows_per_segment, (i + 1) * rows_per_segment) - ); - EXPECT_EQ( - ranges_and_keys[structure_indices[i][1]].row_range(), - RowRange(i * rows_per_segment, (i + 1) * rows_per_segment) - ); - EXPECT_EQ(ranges_and_keys[structure_indices[i][0]].col_range(), ColRange(1, 4)); - EXPECT_EQ(ranges_and_keys[structure_indices[i][1]].col_range(), ColRange(4, 6)); +struct MergeUpdateClauseUpdateStrategyMatchSubsetTest : MergeUpdateClauseUpdateStrategyTestBase, testing::Test { + MergeUpdateClauseUpdateStrategyMatchSubsetTest() : + MergeUpdateClauseUpdateStrategyTestBase( + non_string_fields_ts_index_descriptor(), + MergeStrategy{.matched = MergeAction::UPDATE, .not_matched_by_target = MergeAction::DO_NOTHING}, + slice_data_into_segments( + non_string_fields_ts_index_descriptor(), rows_per_segment_, cols_per_segment_, + iota_view(timestamp{0}, timestamp{num_rows_}), + iota_view(static_cast(0), static_cast(num_rows_)), + iota_view(static_cast(0), static_cast(num_rows_)), + iota_view(0, num_rows_) | views::transform([](auto x) -> bool { return x % 2 == 1; }), + iota_view(0, num_rows_) | views::transform([](auto x) { return static_cast(x); }), + iota_view(timestamp{0}, timestamp{num_rows_}) + ) + ) { + // Shuffle the input ranges and keys to ensure structure_for_processing sorts correctly + constexpr static size_t rand_seed = 0; + std::mt19937 g(rand_seed); + shuffle(ranges_and_keys_, g); } + constexpr static int num_rows_ = 14; + constexpr static int num_cols_ = non_string_fields.size(); + constexpr static int rows_per_segment_ = 5; + constexpr static int cols_per_segment_ = 3; +}; + +TEST_F(MergeUpdateClauseUpdateStrategyMatchSubsetTest, NoMatch) { + auto [input_frame, input_frame_data] = input_frame_from_tensors( + descriptor_, + std::array{-10, -9, 20, 25}, + std::array{-1, -2, 10, 20}, + std::array{10, 20, 100, 200}, + std::array{false, true, true, false}, + std::array{-10.f, -11.f, 11.1f, 22.2f}, + std::array{-11, -12, 1000, 2000} + ); + MergeUpdateClause clause = create_clause(std::move(input_frame)); + const std::vector> structure_indices = clause.structure_for_processing(ranges_and_keys_); + ASSERT_EQ(structure_indices.size(), 0); +} + +TEST_F(MergeUpdateClauseUpdateStrategyMatchSubsetTest, MatchFirst) { + auto [input_frame, input_frame_data] = input_frame_from_tensors( + descriptor_, + std::array{3}, + std::array{-2}, + std::array{20}, + std::array{false}, + std::array{-11.f}, + std::array{-12} + ); + MergeUpdateClause clause = create_clause(std::move(input_frame)); + const std::vector> structure_indices = clause.structure_for_processing(ranges_and_keys_); + ASSERT_EQ(structure_indices.size(), 1); + ASSERT_EQ(structure_indices[0].size(), 2); + ASSERT_EQ(ranges_and_keys_[structure_indices[0][0]].row_range(), RowRange(0, 5)); + ASSERT_EQ(ranges_and_keys_[structure_indices[0][1]].row_range(), RowRange(0, 5)); + ASSERT_EQ(ranges_and_keys_[structure_indices[0][0]].col_range(), ColRange(1, 4)); + ASSERT_EQ(ranges_and_keys_[structure_indices[0][1]].col_range(), ColRange(4, 6)); + const std::vector result_entities = clause.process( + std::move(map_entities_to_structure_for_processing_output(structure_indices, initial_entities_)[0]) + ); + auto proc = gather_entities, std::shared_ptr, std::shared_ptr>( + *component_manager_, result_entities + ); + auto [expected_segments, expected_col_ranges, expected_row_ranges] = slice_data_into_segments( + descriptor_, + rows_per_segment_, + cols_per_segment_, + iota_view(0, 10), + std::array{0, 1, 2, -2, 4}, + std::array{0, 1, 2, 20, 4}, + std::array{0, 1, 0, 0, 0}, + std::array{0.f, 1, 2, -11.f, 4}, + std::array{0, 1, 2, -12, 4} + ); + ASSERT_EQ(*(*proc.segments_)[0], expected_segments[0]); + ASSERT_EQ(*(*proc.segments_)[1], expected_segments[1]); + ASSERT_EQ(*(*proc.row_ranges_)[0], RowRange(0, 5)); + ASSERT_EQ(*(*proc.row_ranges_)[1], RowRange(0, 5)); + ASSERT_EQ(*(*proc.col_ranges_)[0], RowRange(1, 4)); + ASSERT_EQ(*(*proc.col_ranges_)[1], RowRange(4, 6)); +} + +TEST_F(MergeUpdateClauseUpdateStrategyMatchSubsetTest, MatchSecond) { + auto [input_frame, input_frame_data] = input_frame_from_tensors( + descriptor_, + std::array{6}, + std::array{-2}, + std::array{20}, + std::array{true}, + std::array{-11.f}, + std::array{-12} + ); + MergeUpdateClause clause = create_clause(std::move(input_frame)); + const std::vector> structure_indices = clause.structure_for_processing(ranges_and_keys_); + ASSERT_EQ(structure_indices.size(), 1); + ASSERT_EQ(structure_indices[0].size(), 2); + ASSERT_EQ(ranges_and_keys_[structure_indices[0][0]].row_range(), RowRange(5, 10)); + ASSERT_EQ(ranges_and_keys_[structure_indices[0][1]].row_range(), RowRange(5, 10)); + ASSERT_EQ(ranges_and_keys_[structure_indices[0][0]].col_range(), ColRange(1, 4)); + ASSERT_EQ(ranges_and_keys_[structure_indices[0][1]].col_range(), ColRange(4, 6)); + const std::vector result_entities = clause.process( + std::move(map_entities_to_structure_for_processing_output(structure_indices, initial_entities_)[0]) + ); + auto proc = gather_entities, std::shared_ptr, std::shared_ptr>( + *component_manager_, result_entities + ); + auto [expected_segments, expected_col_ranges, expected_row_ranges] = slice_data_into_segments( + descriptor_, + rows_per_segment_, + cols_per_segment_, + iota_view(5, 10), + std::array{5, -2, 7, 8, 9}, + std::array{5, 20, 7, 8, 9}, + std::array{1, 1, 1, 0, 1}, + std::array{5, -11.f, 7, 8, 9}, + std::array{5, -12, 7, 8, 9} + ); + ASSERT_EQ(*(*proc.segments_)[0], expected_segments[0]); + ASSERT_EQ(*(*proc.segments_)[1], expected_segments[1]); + ASSERT_EQ(*(*proc.row_ranges_)[0], RowRange(5, 10)); + ASSERT_EQ(*(*proc.row_ranges_)[1], RowRange(5, 10)); + ASSERT_EQ(*(*proc.col_ranges_)[0], RowRange(1, 4)); + ASSERT_EQ(*(*proc.col_ranges_)[1], RowRange(4, 6)); +} - // This works because the entities pushed to the component manager are pre-sorted by row range and structure for - // processing of the merge clause also sorts by row slice - std::vector> entities_for_processing = - map_entities_to_structure_for_processing_output(structure_indices, entities); - - auto [expected_segments, expected_col_ranges, expected_row_ranges] = - slice_data_into_segments( - rows_per_segment, - source_descriptor, - iota_view(timestamp{0}, timestamp{3 * rows_per_segment}), - std::array{10, 1, 2, 3, 4, 5, 6, 20, 8, 9, 10, 11, 12, 13, 30}, - std::array{100, 1, 2, 3, 4, 5, 6, 200, 8, 9, 10, 11, 12, 13, 300}, - std::array{1, 1, 0, 1, 0, 1, 0, 0, 0, 1, 0, 1, 0, 1, 1}, - std::array{11.1f, 1, 2, 3, 4, 5, 6, 22.2f, 8, 9, 10, 11, 12, 13, 33.3f}, - std::array{1000, 1, 2, 3, 4, 5, 6, 2000, 8, 9, 10, 11, 12, 13, 3000} +TEST_F(MergeUpdateClauseUpdateStrategyMatchSubsetTest, MatchThird) { + auto [input_frame, input_frame_data] = input_frame_from_tensors( + descriptor_, + std::array{10}, + std::array{-2}, + std::array{20}, + std::array{true}, + std::array{-11.f}, + std::array{-12} + ); + MergeUpdateClause clause = create_clause(std::move(input_frame)); + const std::vector> structure_indices = clause.structure_for_processing(ranges_and_keys_); + ASSERT_EQ(structure_indices.size(), 1); + ASSERT_EQ(structure_indices[0].size(), 2); + ASSERT_EQ(ranges_and_keys_[structure_indices[0][0]].row_range(), RowRange(10, 14)); + ASSERT_EQ(ranges_and_keys_[structure_indices[0][1]].row_range(), RowRange(10, 14)); + ASSERT_EQ(ranges_and_keys_[structure_indices[0][0]].col_range(), ColRange(1, 4)); + ASSERT_EQ(ranges_and_keys_[structure_indices[0][1]].col_range(), ColRange(4, 6)); + + const std::vector result_entities = clause.process( + std::move(map_entities_to_structure_for_processing_output(structure_indices, initial_entities_)[0]) + ); + auto proc = gather_entities, std::shared_ptr, std::shared_ptr>( + *component_manager_, result_entities + ); + auto [expected_segments, expected_col_ranges, expected_row_ranges] = slice_data_into_segments( + descriptor_, + rows_per_segment_, + cols_per_segment_, + iota_view(10, 14), + std::array{-2, 11, 12, 13}, + std::array{20, 11, 12, 13}, + std::array{1, 1, 0, 1}, + std::array{-11.f, 11, 12, 13}, + std::array{-12, 11, 12, 13} + ); + ASSERT_EQ(*(*proc.segments_)[0], expected_segments[0]); + ASSERT_EQ(*(*proc.segments_)[1], expected_segments[1]); + ASSERT_EQ(*(*proc.row_ranges_)[0], RowRange(10, 14)); + ASSERT_EQ(*(*proc.row_ranges_)[1], RowRange(10, 14)); + ASSERT_EQ(*(*proc.col_ranges_)[0], RowRange(1, 4)); + ASSERT_EQ(*(*proc.col_ranges_)[1], RowRange(4, 6)); +} + +TEST_F(MergeUpdateClauseUpdateStrategyMatchSubsetTest, MatchFirstAndThird) { + auto [input_frame, input_frame_data] = input_frame_from_tensors( + descriptor_, + std::array{4, 10}, + std::array{-2, -3}, + std::array{20, 30}, + std::array{true, false}, + std::array{-11.f, -12.f}, + std::array{-12, -13} + ); + MergeUpdateClause clause = create_clause(std::move(input_frame)); + const std::vector> structure_indices = clause.structure_for_processing(ranges_and_keys_); + ASSERT_EQ(structure_indices.size(), 2); + ASSERT_EQ(structure_indices[0].size(), 2); + ASSERT_EQ(structure_indices[1].size(), 2); + ASSERT_EQ(ranges_and_keys_[structure_indices[0][0]].row_range(), RowRange(0, 5)); + ASSERT_EQ(ranges_and_keys_[structure_indices[0][1]].row_range(), RowRange(0, 5)); + ASSERT_EQ(ranges_and_keys_[structure_indices[0][0]].col_range(), ColRange(1, 4)); + ASSERT_EQ(ranges_and_keys_[structure_indices[0][1]].col_range(), ColRange(4, 6)); + ASSERT_EQ(ranges_and_keys_[structure_indices[1][0]].row_range(), RowRange(10, 14)); + ASSERT_EQ(ranges_and_keys_[structure_indices[1][1]].row_range(), RowRange(10, 14)); + ASSERT_EQ(ranges_and_keys_[structure_indices[1][0]].col_range(), ColRange(1, 4)); + ASSERT_EQ(ranges_and_keys_[structure_indices[1][1]].col_range(), ColRange(4, 6)); + std::vector> entities_to_process = + map_entities_to_structure_for_processing_output(structure_indices, initial_entities_); + const std::vector result_entities_0 = clause.process(std::move(entities_to_process[0])); + auto proc_0 = + gather_entities, std::shared_ptr, std::shared_ptr>( + *component_manager_, result_entities_0 ); - sort_by_rowslice(expected_row_ranges, expected_col_ranges, expected_segments); - for (size_t i = 0; i < entities_for_processing.size(); ++i) { - auto proc = - gather_entities, std::shared_ptr, std::shared_ptr>( - *component_manager, clause.process(std::move(entities_for_processing[i])) - ); - SCOPED_TRACE(testing::Message() << "processing result = " << i); - EXPECT_EQ(proc.segments_->size(), 2); - EXPECT_EQ(proc.row_ranges_->size(), 2); - EXPECT_EQ(proc.col_ranges_->size(), 2); - EXPECT_EQ(*(proc.row_ranges_.value())[0], RowRange(i * rows_per_segment, (i + 1) * rows_per_segment)); - EXPECT_EQ(*(proc.row_ranges_.value())[1], RowRange(i * rows_per_segment, (i + 1) * rows_per_segment)); - EXPECT_EQ(*(proc.col_ranges_.value())[0], RowRange(1, 4)); - EXPECT_EQ(*(proc.col_ranges_.value())[1], RowRange(4, 6)); - EXPECT_EQ(*(proc.segments_.value())[0], expected_segments[i * 2]); - EXPECT_EQ(*(proc.segments_.value())[1], expected_segments[i * 2 + 1]); - } + auto [expected_segments_0, expected_col_ranges_0, expected_row_ranges_0] = + slice_data_into_segments( + descriptor_, + rows_per_segment_, + cols_per_segment_, + iota_view(0, 5), + std::array{0, 1, 2, 3, -2}, + std::array{0, 1, 2, 3, 20}, + std::array{0, 1, 0, 1, 1}, + std::array{0, 1, 2, 3, -11}, + std::array{0, 1, 2, 3, -12} + ); + ASSERT_EQ(*(*proc_0.segments_)[0], expected_segments_0[0]); + ASSERT_EQ(*(*proc_0.segments_)[1], expected_segments_0[1]); + ASSERT_EQ(*(*proc_0.row_ranges_)[0], RowRange(0, 5)); + ASSERT_EQ(*(*proc_0.row_ranges_)[1], RowRange(0, 5)); + ASSERT_EQ(*(*proc_0.col_ranges_)[0], RowRange(1, 4)); + ASSERT_EQ(*(*proc_0.col_ranges_)[1], RowRange(4, 6)); + + const std::vector result_entities_1 = clause.process(std::move(entities_to_process[1])); + auto proc_1 = + gather_entities, std::shared_ptr, std::shared_ptr>( + *component_manager_, result_entities_1 + ); + auto [expected_segments_1, expected_col_ranges_1, expected_row_ranges_1] = + slice_data_into_segments( + descriptor_, + rows_per_segment_, + cols_per_segment_, + iota_view(10, 14), + std::array{-3, 11, 12, 13}, + std::array{30, 11, 12, 13}, + std::array{0, 1, 0, 1}, + std::array{-12, 11, 12, 13}, + std::array{-13, 11, 12, 13} + ); + ASSERT_EQ(*(*proc_1.segments_)[0], expected_segments_1[0]); + ASSERT_EQ(*(*proc_1.segments_)[1], expected_segments_1[1]); + ASSERT_EQ(*(*proc_1.row_ranges_)[0], RowRange(10, 14)); + ASSERT_EQ(*(*proc_1.row_ranges_)[1], RowRange(10, 14)); + ASSERT_EQ(*(*proc_1.col_ranges_)[0], RowRange(1, 4)); + ASSERT_EQ(*(*proc_1.col_ranges_)[1], RowRange(4, 6)); +} + +TEST_F(MergeUpdateClauseUpdateStrategyMatchSubsetTest, MatchFirstAndSecond) { + auto [input_frame, input_frame_data] = input_frame_from_tensors( + descriptor_, + std::array{0, 5}, + std::array{-2, -3}, + std::array{20, 30}, + std::array{true, false}, + std::array{-11.f, -12.f}, + std::array{-12, -13} + ); + MergeUpdateClause clause = create_clause(std::move(input_frame)); + const std::vector> structure_indices = clause.structure_for_processing(ranges_and_keys_); + ASSERT_EQ(structure_indices.size(), 2); + ASSERT_EQ(structure_indices[0].size(), 2); + ASSERT_EQ(structure_indices[1].size(), 2); + ASSERT_EQ(ranges_and_keys_[structure_indices[0][0]].row_range(), RowRange(0, 5)); + ASSERT_EQ(ranges_and_keys_[structure_indices[0][1]].row_range(), RowRange(0, 5)); + ASSERT_EQ(ranges_and_keys_[structure_indices[0][0]].col_range(), ColRange(1, 4)); + ASSERT_EQ(ranges_and_keys_[structure_indices[0][1]].col_range(), ColRange(4, 6)); + ASSERT_EQ(ranges_and_keys_[structure_indices[1][0]].row_range(), RowRange(5, 10)); + ASSERT_EQ(ranges_and_keys_[structure_indices[1][1]].row_range(), RowRange(5, 10)); + ASSERT_EQ(ranges_and_keys_[structure_indices[1][0]].col_range(), ColRange(1, 4)); + ASSERT_EQ(ranges_and_keys_[structure_indices[1][1]].col_range(), ColRange(4, 6)); + std::vector> entities_to_process = + map_entities_to_structure_for_processing_output(structure_indices, initial_entities_); + const std::vector result_entities_0 = clause.process(std::move(entities_to_process[0])); + auto proc_0 = + gather_entities, std::shared_ptr, std::shared_ptr>( + *component_manager_, result_entities_0 + ); + auto [expected_segments_0, expected_col_ranges_0, expected_row_ranges_0] = + slice_data_into_segments( + descriptor_, + rows_per_segment_, + cols_per_segment_, + iota_view(0, 5), + std::array{-2, 1, 2, 3, 4}, + std::array{20, 1, 2, 3, 4}, + std::array{1, 1, 0, 1, 0}, + std::array{-11, 1, 2, 3, 4}, + std::array{-12, 1, 2, 3, 4} + ); + ASSERT_EQ(*(*proc_0.segments_)[0], expected_segments_0[0]); + ASSERT_EQ(*(*proc_0.segments_)[1], expected_segments_0[1]); + ASSERT_EQ(*(*proc_0.row_ranges_)[0], RowRange(0, 5)); + ASSERT_EQ(*(*proc_0.row_ranges_)[1], RowRange(0, 5)); + ASSERT_EQ(*(*proc_0.col_ranges_)[0], RowRange(1, 4)); + ASSERT_EQ(*(*proc_0.col_ranges_)[1], RowRange(4, 6)); + + const std::vector result_entities_1 = clause.process(std::move(entities_to_process[1])); + auto proc_1 = + gather_entities, std::shared_ptr, std::shared_ptr>( + *component_manager_, result_entities_1 + ); + auto [expected_segments_1, expected_col_ranges_1, expected_row_ranges_1] = + slice_data_into_segments( + descriptor_, + rows_per_segment_, + cols_per_segment_, + iota_view(5, 10), + std::array{-3, 6, 7, 8, 9}, + std::array{30, 6, 7, 8, 9}, + std::array{0, 0, 1, 0, 1}, + std::array{-12, 6, 7, 8, 9}, + std::array{-13, 6, 7, 8, 9} + ); + ASSERT_EQ(*(*proc_1.segments_)[0], expected_segments_1[0]); + ASSERT_EQ(*(*proc_1.segments_)[1], expected_segments_1[1]); + ASSERT_EQ(*(*proc_1.row_ranges_)[0], RowRange(5, 10)); + ASSERT_EQ(*(*proc_1.row_ranges_)[1], RowRange(5, 10)); + ASSERT_EQ(*(*proc_1.col_ranges_)[0], RowRange(1, 4)); + ASSERT_EQ(*(*proc_1.col_ranges_)[1], RowRange(4, 6)); +} + +TEST_F(MergeUpdateClauseUpdateStrategyMatchSubsetTest, MatchSecondAndThird) { + auto [input_frame, input_frame_data] = input_frame_from_tensors( + descriptor_, + std::array{7, 12}, + std::array{-2, -3}, + std::array{20, 30}, + std::array{true, true}, + std::array{-11.f, -12.f}, + std::array{-12, -13} + ); + MergeUpdateClause clause = create_clause(std::move(input_frame)); + const std::vector> structure_indices = clause.structure_for_processing(ranges_and_keys_); + ASSERT_EQ(structure_indices.size(), 2); + ASSERT_EQ(structure_indices[0].size(), 2); + ASSERT_EQ(structure_indices[1].size(), 2); + ASSERT_EQ(ranges_and_keys_[structure_indices[0][0]].row_range(), RowRange(5, 10)); + ASSERT_EQ(ranges_and_keys_[structure_indices[0][1]].row_range(), RowRange(5, 10)); + ASSERT_EQ(ranges_and_keys_[structure_indices[0][0]].col_range(), ColRange(1, 4)); + ASSERT_EQ(ranges_and_keys_[structure_indices[0][1]].col_range(), ColRange(4, 6)); + ASSERT_EQ(ranges_and_keys_[structure_indices[1][0]].row_range(), RowRange(10, 14)); + ASSERT_EQ(ranges_and_keys_[structure_indices[1][1]].row_range(), RowRange(10, 14)); + ASSERT_EQ(ranges_and_keys_[structure_indices[1][0]].col_range(), ColRange(1, 4)); + ASSERT_EQ(ranges_and_keys_[structure_indices[1][1]].col_range(), ColRange(4, 6)); + std::vector> entities_to_process = + map_entities_to_structure_for_processing_output(structure_indices, initial_entities_); + const std::vector result_entities_0 = clause.process(std::move(entities_to_process[0])); + auto proc_0 = + gather_entities, std::shared_ptr, std::shared_ptr>( + *component_manager_, result_entities_0 + ); + auto [expected_segments_0, expected_col_ranges_0, expected_row_ranges_0] = + slice_data_into_segments( + descriptor_, + rows_per_segment_, + cols_per_segment_, + iota_view(5, 10), + std::array{5, 6, -2, 8, 9}, + std::array{5, 6, 20, 8, 9}, + std::array{1, 0, 1, 0, 1}, + std::array{5, 6, -11, 8, 9}, + std::array{5, 6, -12, 8, 9} + ); + ASSERT_EQ(*(*proc_0.segments_)[0], expected_segments_0[0]); + ASSERT_EQ(*(*proc_0.segments_)[1], expected_segments_0[1]); + ASSERT_EQ(*(*proc_0.row_ranges_)[0], RowRange(5, 10)); + ASSERT_EQ(*(*proc_0.row_ranges_)[1], RowRange(5, 10)); + ASSERT_EQ(*(*proc_0.col_ranges_)[0], RowRange(1, 4)); + ASSERT_EQ(*(*proc_0.col_ranges_)[1], RowRange(4, 6)); + + const std::vector result_entities_1 = clause.process(std::move(entities_to_process[1])); + auto proc_1 = + gather_entities, std::shared_ptr, std::shared_ptr>( + *component_manager_, result_entities_1 + ); + auto [expected_segments_1, expected_col_ranges_1, expected_row_ranges_1] = + slice_data_into_segments( + descriptor_, + rows_per_segment_, + cols_per_segment_, + iota_view(10, 14), + std::array{10, 11, -3, 13}, + std::array{10, 11, 30, 13}, + std::array{0, 1, 1, 1}, + std::array{10, 11, -12, 13}, + std::array{10, 11, -13, 13} + ); + ASSERT_EQ(*(*proc_1.segments_)[0], expected_segments_1[0]); + ASSERT_EQ(*(*proc_1.segments_)[1], expected_segments_1[1]); + ASSERT_EQ(*(*proc_1.row_ranges_)[0], RowRange(10, 14)); + ASSERT_EQ(*(*proc_1.row_ranges_)[1], RowRange(10, 14)); + ASSERT_EQ(*(*proc_1.col_ranges_)[0], RowRange(1, 4)); + ASSERT_EQ(*(*proc_1.col_ranges_)[1], RowRange(4, 6)); } \ No newline at end of file diff --git a/cpp/arcticdb/version/local_versioned_engine.cpp b/cpp/arcticdb/version/local_versioned_engine.cpp index ef9a02b2cb1..b2104568b53 100644 --- a/cpp/arcticdb/version/local_versioned_engine.cpp +++ b/cpp/arcticdb/version/local_versioned_engine.cpp @@ -455,7 +455,8 @@ VersionedItem LocalVersionedEngine::read_modify_write_internal( read_query, read_options, write_options, - IndexPartialKey{target_stream, target_version} + IndexPartialKey{target_stream, target_version}, + ReadModifyWriteIndexStrategy::REWRITE_INDEX ); if (cfg().symbol_list()) symbol_list().add_symbol(store(), target_stream, versioned_item.key_.version_id()); @@ -2319,7 +2320,7 @@ VersionedItem LocalVersionedEngine::merge_internal( } } const WriteOptions write_options = get_write_options(); - auto versioned_item = merge_impl( + auto versioned_item = merge_update_impl( store(), identifier, std::move(user_meta_proto), diff --git a/cpp/arcticdb/version/version_core.cpp b/cpp/arcticdb/version/version_core.cpp index 6769a9714bf..a0253bf7c28 100644 --- a/cpp/arcticdb/version/version_core.cpp +++ b/cpp/arcticdb/version/version_core.cpp @@ -144,7 +144,7 @@ IndexDescriptorImpl check_index_match(const arcticdb::stream::Index& index, cons /// difference between this and adjust_slice_ranges is that this will not change the column slices. While /// adjust_slice_ranges will change the column slices, making the first slice start from 0 even for timestamp-index /// dataframes (which should have column range starting from 1 when being written to disk). -[[maybe_unused]] void compact_row_slices(std::span slices) { +void compact_row_slices(std::span slices) { size_t current_compacted_row = 0; size_t previous_uncompacted_end = slices.empty() ? 0 : slices.front().slice().row_range.end(); size_t previous_col_slice_end = slices.empty() ? 0 : slices.front().slice().col_range.end(); @@ -164,6 +164,33 @@ IndexDescriptorImpl check_index_match(const arcticdb::stream::Index& index, cons previous_col_slice_end = slice.slice().col_range.end(); } } + +folly::Future rewrite_index_for_read_modify_write( + const PipelineContext& pipeline_context, const WriteOptions& write_options, + const IndexPartialKey& target_partial_index_key, const std::shared_ptr& store, + std::vector&& slices, std::unique_ptr&& user_meta +) { + ranges::sort(slices); + compact_row_slices(slices); + const size_t row_count = + slices.empty() ? 0 : slices.back().slice().row_range.second - slices[0].slice().row_range.first; + const TimeseriesDescriptor tsd = make_timeseries_descriptor( + row_count, + pipeline_context.descriptor(), + std::move(*pipeline_context.norm_meta_), + user_meta ? std::make_optional(*std::move(user_meta)) : std::nullopt, + std::nullopt, + std::nullopt, + write_options.bucketize_dynamic + ); + return index::write_index( + index_type_from_descriptor(pipeline_context.descriptor()), + tsd, + std::move(slices), + target_partial_index_key, + store + ); +} } // namespace void sorted_data_check_append(const InputFrame& frame, index::IndexSegmentReader& index_segment_reader) { @@ -2744,21 +2771,23 @@ folly::Future read_frame_for_version( } VersionedItem read_modify_write_impl( - const std::shared_ptr& store, const std::variant& source_version_info, - std::unique_ptr&& user_meta, - const std::shared_ptr& read_query, const ReadOptions& read_options, - const WriteOptions& write_options, const IndexPartialKey& target_partial_index_key + const std::shared_ptr& store, const std::variant& version_info, + std::unique_ptr&& user_meta, std::shared_ptr read_query, + const ReadOptions& read_options, const WriteOptions& write_options, + const IndexPartialKey& target_partial_index_key, ReadModifyWriteIndexStrategy index_strategy ) { read_query->clauses_.push_back(std::make_shared( WriteClause(write_options, target_partial_index_key, std::make_shared(), store) )); - const auto pipeline_context = setup_pipeline_context(store, source_version_info, *read_query, read_options); + const auto pipeline_context = setup_pipeline_context(store, version_info, *read_query, read_options); check_can_perform_processing(pipeline_context, *read_query); auto component_manager = std::make_shared(); return read_and_schedule_processing(store, pipeline_context, read_query, read_options, component_manager) - .thenValue([component_manager, pipeline_context, read_query](std::vector&& processed_entity_ids) { + .thenValue([component_manager, + pipeline_context, + read_query = std::move(read_query)](std::vector&& processed_entity_ids) { generate_output_schema_and_save_to_pipeline(*pipeline_context, *read_query); std::vector> write_segments_futures; ranges::transform( @@ -2771,31 +2800,23 @@ VersionedItem read_modify_write_impl( return folly::collect(std::move(write_segments_futures)); }) .thenValue([&](std::vector&& slices) { - ranges::sort(slices); - compact_row_slices(slices); - const size_t row_count = - slices.empty() ? 0 : slices.back().slice().row_range.second - slices[0].slice().row_range.first; - const TimeseriesDescriptor tsd = make_timeseries_descriptor( - row_count, - pipeline_context->descriptor(), - std::move(*pipeline_context->norm_meta_), - user_meta ? std::make_optional(*std::move(user_meta)) : std::nullopt, - std::nullopt, - std::nullopt, - write_options.bucketize_dynamic - ); - return index::write_index( - index_type_from_descriptor(pipeline_context->descriptor()), - tsd, - std::move(slices), - target_partial_index_key, - store - ); + if (index_strategy == ReadModifyWriteIndexStrategy::REWRITE_INDEX) { + return rewrite_index_for_read_modify_write( + *pipeline_context, + write_options, + target_partial_index_key, + store, + std::move(slices), + std::move(user_meta) + ); + } else { + return folly::Future(AtomKey{}); + } }) .get(); } -VersionedItem merge_impl( +VersionedItem merge_update_impl( const std::shared_ptr& store, const std::variant& version_info, std::unique_ptr&& user_meta, const ReadOptions& read_options, const WriteOptions& write_options, const IndexPartialKey& target_partial_index_key, @@ -2810,7 +2831,14 @@ VersionedItem merge_impl( read_query->row_filter = source->index_range; } return read_modify_write_impl( - store, version_info, std::move(user_meta), read_query, read_options, write_options, target_partial_index_key + store, + version_info, + std::move(user_meta), + std::move(read_query), + read_options, + write_options, + target_partial_index_key, + ReadModifyWriteIndexStrategy::MERGE_INDEX ); } diff --git a/cpp/arcticdb/version/version_core.hpp b/cpp/arcticdb/version/version_core.hpp index e83dc358ac5..9ca5866688c 100644 --- a/cpp/arcticdb/version/version_core.hpp +++ b/cpp/arcticdb/version/version_core.hpp @@ -36,6 +36,17 @@ struct SymbolProcessingResult { std::vector entity_ids_; }; +enum class ReadModifyWriteIndexStrategy { + /// The index key for the new version will contain only the data produced by the processing query. + /// E.g. Running a filter query will produce a new version containing only the filtered data. + REWRITE_INDEX, + /// The index key will be a merge of the current index key and the new data. + /// E.g. Running merge_update with 3 segments where only the middle segment is matched will keep the first and the + /// third segment in the index and add a new segment in the middle. If REWRITE_INDEX was used it would only keep the + /// new (middle segment) + MERGE_INDEX +}; + struct ReadVersionOutput { ReadVersionOutput() = delete; ReadVersionOutput(VersionedItem&& versioned_item, FrameAndDescriptor&& frame_and_descriptor) : @@ -216,12 +227,12 @@ folly::Future prepare_output_frame( VersionedItem read_modify_write_impl( const std::shared_ptr& store, const std::variant& version_info, - std::unique_ptr&& user_meta, - const std::shared_ptr& read_query, const ReadOptions& read_options, - const WriteOptions& write_options, const IndexPartialKey& target_partial_index_key + std::unique_ptr&& user_meta, std::shared_ptr read_query, + const ReadOptions& read_options, const WriteOptions& write_options, + const IndexPartialKey& target_partial_index_key, ReadModifyWriteIndexStrategy index_strategy ); -VersionedItem merge_impl( +VersionedItem merge_update_impl( const std::shared_ptr& store, const std::variant& version_info, std::unique_ptr&& user_meta, const ReadOptions& read_options, const WriteOptions& write_options, const IndexPartialKey& target_partial_index_key, From dfc8496e91e5c456f086c6b29f0bd37eca9c899e Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Tue, 25 Nov 2025 17:12:13 +0200 Subject: [PATCH 33/46] Make python API for merge propagate to C++ Fix bugs Fix edge case where a index values spans multiple rows Add unit test for the same value spanning several segments Move utility functions in dedicated headers Add comments Bring back old create_dense_column Fix spelling errors --- cpp/arcticdb/CMakeLists.txt | 3 + cpp/arcticdb/column_store/column.hpp | 10 - cpp/arcticdb/column_store/memory_segment.hpp | 119 +---- cpp/arcticdb/column_store/segment_utils.cpp | 61 +++ cpp/arcticdb/column_store/segment_utils.hpp | 296 ++++++++++- cpp/arcticdb/column_store/string_pool.cpp | 1 + cpp/arcticdb/pipeline/input_frame_utils.hpp | 73 +++ cpp/arcticdb/pipeline/write_frame.cpp | 1 + cpp/arcticdb/processing/clause.cpp | 170 ++++-- cpp/arcticdb/processing/clause.hpp | 9 +- cpp/arcticdb/processing/clause_utils.hpp | 4 +- .../processing/test/test_merge_update.cpp | 494 ++++++------------ .../processing/test/test_resample.cpp | 118 ++--- .../test/test_unsorted_aggregation.cpp | 4 +- cpp/arcticdb/util/configs_map.hpp | 1 - cpp/arcticdb/util/test/test_utils.hpp | 17 + cpp/arcticdb/util/type_traits.hpp | 7 - cpp/arcticdb/util/variant.hpp | 7 - .../version/local_versioned_engine.cpp | 36 +- .../version/local_versioned_engine.hpp | 2 +- cpp/arcticdb/version/python_bindings.cpp | 6 + cpp/arcticdb/version/version_core.cpp | 88 +++- cpp/arcticdb/version/version_core.hpp | 10 +- cpp/arcticdb/version/versioned_engine.hpp | 2 +- env.sh | 18 - python/arcticdb/util/test.py | 13 + python/arcticdb/util/utils.py | 1 - python/arcticdb/version_store/library.py | 50 +- python/test_index.py | 28 - .../unit/arcticdb/version_store/test_merge.py | 13 +- .../test_recursive_normalizers.py | 16 +- 31 files changed, 944 insertions(+), 734 deletions(-) create mode 100644 cpp/arcticdb/column_store/segment_utils.cpp create mode 100644 cpp/arcticdb/pipeline/input_frame_utils.hpp delete mode 100644 env.sh delete mode 100644 python/test_index.py diff --git a/cpp/arcticdb/CMakeLists.txt b/cpp/arcticdb/CMakeLists.txt index 538c16e6aa1..28dce79e280 100644 --- a/cpp/arcticdb/CMakeLists.txt +++ b/cpp/arcticdb/CMakeLists.txt @@ -213,6 +213,7 @@ set(arcticdb_srcs column_store/memory_segment_impl.hpp column_store/row_ref.hpp column_store/string_pool.hpp + column_store/segment_utils.hpp entity/atom_key.hpp entity/frame_and_descriptor.hpp entity/index_range.hpp @@ -250,6 +251,7 @@ set(arcticdb_srcs pipeline/index_utils.hpp pipeline/index_writer.hpp pipeline/input_frame.hpp + pipeline/input_frame_utils.hpp pipeline/pandas_output_frame.hpp pipeline/pipeline_common.hpp pipeline/pipeline_utils.hpp @@ -438,6 +440,7 @@ set(arcticdb_srcs column_store/memory_segment.cpp column_store/memory_segment_impl.cpp column_store/memory_segment_impl.cpp + column_store/segment_utils.cpp column_store/statistics.hpp column_store/string_pool.cpp entity/data_error.cpp diff --git a/cpp/arcticdb/column_store/column.hpp b/cpp/arcticdb/column_store/column.hpp index a0cf030384a..d9bd7447cac 100644 --- a/cpp/arcticdb/column_store/column.hpp +++ b/cpp/arcticdb/column_store/column.hpp @@ -228,16 +228,6 @@ class Column { Column(TypeDescriptor type, size_t expected_rows, AllocationType allocation_type, Sparsity allow_sparse, size_t extra_bytes_per_block = 0); - template - requires std::ranges::contiguous_range - static Column create_dense_column(const Input& data, const TypeDescriptor& type) { - constexpr size_t element_size = sizeof(std::ranges::range_value_t); - Column result(type, data.size(), AllocationType::PRESIZED, Sparsity::NOT_PERMITTED); - std::memcpy(result.ptr(), data.data(), data.size() * element_size); - result.set_row_data(data.size()); - return result; - } - ARCTICDB_MOVE_ONLY_DEFAULT(Column) friend bool operator==(const Column& left, const Column& right); diff --git a/cpp/arcticdb/column_store/memory_segment.hpp b/cpp/arcticdb/column_store/memory_segment.hpp index 62ea4e7192e..05dfa37ddc4 100644 --- a/cpp/arcticdb/column_store/memory_segment.hpp +++ b/cpp/arcticdb/column_store/memory_segment.hpp @@ -45,81 +45,6 @@ class SegmentInMemory { ARCTICDB_MOVE_COPY_DEFAULT(SegmentInMemory) - template - requires std::ranges::sized_range> - static SegmentInMemory create_dense_segment(const StreamDescriptor& descriptor, const T& columns) { - const size_t input_column_count = std::ranges::size(columns); - internal::check( - input_column_count == descriptor.fields().size(), - "When creating a dense segment in memory the number of columns ({}) must match the number of fields in " - "the stream descriptor ({})", - input_column_count, - descriptor.fields().size() - ); - if (input_column_count == 0) { - return SegmentInMemory{}; - } - - const size_t expected_column_size = columns.begin()->first.size(); - constexpr static AllocationType allocation_type = AllocationType::PRESIZED; - constexpr static Sparsity sparsity = Sparsity::NOT_PERMITTED; - auto result = SegmentInMemory{std::make_shared( - descriptor, expected_column_size, allocation_type, sparsity, std::nullopt - )}; - for (auto const& [column_index, column_data] : folly::enumerate(columns)) { - const size_t row_count = std::ranges::size(column_data); - internal::check( - row_count == expected_column_size, - "When creating a dense segment all columns must have the same size. Column[0] has {} rows, " - "Column[{}] has {} rows", - expected_column_size, - column_index, - row_count - ); - result.fill_dense_column_data(column_index, column_data); - } - return result; - } - - template - static SegmentInMemory create_dense_segment(const StreamDescriptor& descriptor, const T&... columns) { - constexpr size_t input_column_count = sizeof...(T); - internal::check( - input_column_count == descriptor.fields().size(), - "When creating a dense segment in memory the number of columns ({}) must match the number of fields in " - "the stream descriptor ({})", - input_column_count, - descriptor.fields().size() - ); - if (input_column_count == 0) { - return SegmentInMemory{}; - } - const size_t expected_column_size = [](const H& head, const Tail&...) { - return std::ranges::size(head); - }(columns...); - constexpr static AllocationType allocation_type = AllocationType::PRESIZED; - constexpr static Sparsity sparsity = Sparsity::NOT_PERMITTED; - auto result = SegmentInMemory{std::make_shared( - descriptor, expected_column_size, allocation_type, sparsity, std::nullopt - )}; - util::enumerate( - [&result, expected_column_size](size_t column_index, const auto& column_data) { - const size_t row_count = std::ranges::size(column_data); - internal::check( - row_count == expected_column_size, - "When creating a dense segment all columns must have the same size. Column[0] has {} rows, " - "Column[{}] has {} rows", - expected_column_size, - column_index, - row_count - ); - result.fill_dense_column_data(column_index, column_data); - }, - columns... - ); - return result; - } - [[nodiscard]] iterator begin(); [[nodiscard]] iterator end(); @@ -369,49 +294,7 @@ class SegmentInMemory { private: explicit SegmentInMemory(std::shared_ptr impl) : impl_(std::move(impl)) {} - template - void fill_dense_column_data(const size_t column_index, const T& input_data) { - using InputValueType = std::decay_t>; - constexpr static bool is_input_string_like = std::is_convertible_v; - const size_t row_count = std::ranges::size(input_data); - details::visit_type(descriptor().field(column_index).type().data_type(), [&, this](auto tdt) { - using col_type_info = ScalarTypeInfo; - using ColRawType = col_type_info::RawType; - constexpr static bool is_sequence = is_sequence_type(col_type_info::data_type); - if constexpr (is_input_string_like && is_sequence) { - // Clang has a bug where it the for_each is just regular range-based for the constexpr if will not - // the body of the if even if the condition is false. This leads to compile time errors because it tries - // to call set_string with non-string values. - // https://stackoverflow.com/questions/79817660/discarded-branch-of-c-constexpr-if-fails-compilation-because-it-calls-non-matc - std::ranges::for_each(input_data, [&, this](const std::string_view& str) { - set_string(column_index, str); - }); - } else if constexpr (!is_sequence && !is_input_string_like) { - internal::check( - std::is_same_v, - "Type mismatch when setting data for Column[{}]. Column data type is {}.", - column_index, - col_type_info::data_type - ); - Column& column_to_fill = column(column_index); - if constexpr (std::ranges::contiguous_range) { - std::memcpy( - column_to_fill.ptr(), std::ranges::data(input_data), row_count * sizeof(InputValueType) - ); - } else { - std::ranges::copy(input_data, column_to_fill.ptr_cast(0, row_count)); - } - } else { - internal::check( - std::is_same_v, - "Type mismatch when setting data for Column[{}]. Column data type is {}.", - column_index, - col_type_info::data_type - ); - } - set_row_data(row_count - 1); - }); - } + std::shared_ptr impl_; }; diff --git a/cpp/arcticdb/column_store/segment_utils.cpp b/cpp/arcticdb/column_store/segment_utils.cpp new file mode 100644 index 00000000000..76b07eba8e4 --- /dev/null +++ b/cpp/arcticdb/column_store/segment_utils.cpp @@ -0,0 +1,61 @@ +/* Copyright 2025 Man Group Operations Limited + * + * Use of this software is governed by the Business Source License 1.1 included in the file licenses/BSL.txt. + * + * As of the Change Date specified in that file, in accordance with the Business Source License, use of this software + * will be governed by the Apache License, version 2.0. + */ + +#include +#include +#include +#include + +namespace arcticdb { + +ankerl::unordered_dense::set unique_values_for_string_column(const Column& column) { + ankerl::unordered_dense::set output_set; + // Guessing that unique values is a third of the column length + // TODO would be useful to have actual unique count here from stats + static auto map_reserve_ratio = ConfigsMap::instance()->get_int("UniqueColumns.AllocationRatio", 3); + output_set.reserve(column.row_count() / map_reserve_ratio); + + details::visit_type(column.type().data_type(), [&](auto col_desc_tag) { + using type_info = ScalarTypeInfo; + if constexpr (is_sequence_type(type_info::data_type)) { + arcticdb::for_each(column, [&output_set](auto value) { + output_set.emplace(value); + }); + } else { + util::raise_rte("Column {} is not a string type column"); + } + }); + return output_set; +} + +std::vector split_descriptor(const StreamDescriptor& descriptor, const size_t cols_per_segment) { + if (descriptor.fields().size() <= cols_per_segment) { + return std::vector{descriptor}; + } + const size_t num_segments = descriptor.fields().size() / cols_per_segment; + std::vector res; + res.reserve(num_segments); + + const unsigned field_count = descriptor.field_count(); + for (size_t i = 0, source_field = descriptor.index().field_count(); i < num_segments; ++i) { + StreamDescriptor partial(descriptor.id()); + if (descriptor.index().field_count() > 0) { + partial.set_index(descriptor.index()); + for (unsigned index_field = 0; index_field < descriptor.index().field_count(); ++index_field) { + partial.add_field(descriptor.field(index_field)); + } + } + for (size_t field = 0; field < cols_per_segment && source_field < field_count; ++field) { + partial.add_field(descriptor.field(source_field++)); + } + res.push_back(std::move(partial)); + } + return res; +} + +} // namespace arcticdb diff --git a/cpp/arcticdb/column_store/segment_utils.hpp b/cpp/arcticdb/column_store/segment_utils.hpp index 718917da032..04f05e25e44 100644 --- a/cpp/arcticdb/column_store/segment_utils.hpp +++ b/cpp/arcticdb/column_store/segment_utils.hpp @@ -8,32 +8,292 @@ #pragma once -#include -#include #include -#include +#include +#include +#include #include +#include +#include +#include +#include namespace arcticdb { -inline ankerl::unordered_dense::set unique_values_for_string_column(const Column& column) { - ankerl::unordered_dense::set output_set; - // Guessing that unique values is a third of the column length - // TODO would be useful to have actual unique count here from stats - static auto map_reserve_ratio = ConfigsMap::instance()->get_int("UniqueColumns.AllocationRatio", 3); - output_set.reserve(column.row_count() / map_reserve_ratio); - - details::visit_type(column.type().data_type(), [&](auto col_desc_tag) { - using type_info = ScalarTypeInfo; - if constexpr (is_sequence_type(type_info::data_type)) { - arcticdb::for_each(column, [&output_set](auto value) { - output_set.emplace(value); - }); +class Column; + +ankerl::unordered_dense::set unique_values_for_string_column(const Column& column); + +std::vector split_descriptor(const StreamDescriptor& descriptor, size_t cols_per_segment); + +template +void fill_dense_column_data(SegmentInMemory& seg, const size_t column_index, const T& input_data) { + using InputValueType = std::decay_t>; + constexpr static bool is_input_string_like = std::is_convertible_v; + const size_t row_count = std::ranges::size(input_data); + details::visit_type(seg.descriptor().field(column_index).type().data_type(), [&](auto tdt) { + using col_type_info = ScalarTypeInfo; + using ColRawType = col_type_info::RawType; + constexpr static bool is_sequence = is_sequence_type(col_type_info::data_type); + if constexpr (is_input_string_like && is_sequence) { + // Clang has a bug where it the for_each is just regular range-based for the constexpr if will not + // the body of the if even if the condition is false. This leads to compile time errors because it tries + // to call set_string with non-string values. + // https://stackoverflow.com/questions/79817660/discarded-branch-of-c-constexpr-if-fails-compilation-because-it-calls-non-matc + std::ranges::for_each(input_data, [&](const std::string_view& str) { seg.set_string(column_index, str); }); + } else if constexpr (!is_sequence && !is_input_string_like) { + internal::check( + std::is_same_v, + "Type mismatch when setting data for Column[{}]. Column data type is {}.", + column_index, + col_type_info::data_type + ); + Column& column_to_fill = seg.column(column_index); + if constexpr (std::ranges::contiguous_range) { + std::memcpy(column_to_fill.ptr(), std::ranges::data(input_data), row_count * sizeof(InputValueType)); + } else { + std::ranges::copy(input_data, column_to_fill.ptr_cast(0, row_count)); + } } else { - util::raise_rte("Column {} is not a string type column"); + internal::check( + std::is_same_v, + "Type mismatch when setting data for Column[{}]. Column data type is {}.", + column_index, + col_type_info::data_type + ); } + seg.set_row_data(row_count - 1); }); - return output_set; } +template +static SegmentInMemory create_dense_segment(const StreamDescriptor& descriptor, const T&... columns) { + constexpr static size_t input_column_count = sizeof...(T); + internal::check( + input_column_count == descriptor.fields().size(), + "When creating a dense segment in memory the number of columns ({}) must match the number of fields in " + "the stream descriptor ({})", + input_column_count, + descriptor.fields().size() + ); + if (input_column_count == 0) { + return SegmentInMemory{}; + } + const size_t expected_column_size = [](const H& head, const Tail&...) { + return std::ranges::size(head); + }(columns...); + constexpr static AllocationType allocation_type = AllocationType::PRESIZED; + constexpr static Sparsity sparsity = Sparsity::NOT_PERMITTED; + auto result = SegmentInMemory(descriptor, expected_column_size, allocation_type, sparsity, std::nullopt); + + [&](std::index_sequence) { + ( + [&result, expected_column_size](size_t column_index, const auto& column_data) { + const size_t row_count = std::ranges::size(column_data); + internal::check( + row_count == expected_column_size, + "When creating a dense segment all columns must have the same size. Column[0] has {} rows, " + "Column[{}] has {} rows", + expected_column_size, + column_index, + row_count + ); + fill_dense_column_data(result, column_index, column_data); + }(Is, std::get(std::tie(columns...))), + ... + ); + }(std::make_index_sequence{}); + + return result; +} + +template +requires util::instantiation_of && util::instantiation_of +auto tuple_cat_ref(const Tup1& tuple1, const Tup2& tuple2) { + return std::apply( + [&](T0&&... args1) { + return std::apply( + [&](T1&&... args2) { + return std::tie(std::forward(args1)..., std::forward(args2)...); + }, + tuple2 + ); + }, + tuple1 + ); +} + +template +requires(sizeof...(Data) > 0) +void slice_column_slice_into_row_slices( + const std::tuple& index, const std::tuple& column_slice, const StreamDescriptor& desc, + const size_t rows_per_segment, const pipelines::ColRange& col_range, std::vector& segments, + std::vector& col_ranges, std::vector& row_ranges +) { + const size_t total_rows = std::ranges::size(std::get<0>(column_slice)); + for (size_t start_row = 0; start_row < total_rows; start_row += rows_per_segment) { + std::apply( + [&](const Cols&... cols) { + const size_t rows_to_take = std::min(rows_per_segment, total_rows - start_row); + segments.push_back(create_dense_segment( + desc, std::ranges::take_view(std::ranges::drop_view(cols, start_row), rows_to_take)... + )); + row_ranges.emplace_back(start_row, start_row + rows_to_take); + col_ranges.emplace_back(col_range); + }, + tuple_cat_ref(index, column_slice) + ); + } +} + +template +auto split_pack(Data&&... data) { + constexpr static auto non_index_columns = sizeof...(Data) - N; + return [&]( + std::index_sequence, std::index_sequence, T&& fwd_tuple + ) { + return std::pair{ + std::forward_as_tuple(std::get(std::forward(fwd_tuple))...), + std::forward_as_tuple(std::get(std::forward(fwd_tuple))...) + }; + }(std::make_index_sequence{}, + std::make_index_sequence{}, + std::forward_as_tuple(std::forward(data)...)); +} + +template< + std::ranges::sized_range IndexCols, typename ColumnSlice, std::ranges::sized_range CurrentCol, + std::ranges::sized_range... RestCols> +requires( + util::instantiation_of && + [](std::index_sequence) { + return (std::ranges::sized_range> && ...); + }(std::make_index_sequence>{}) +) +void slice_data_into_segments( + const std::span descriptors, const size_t rows_per_segment, + const size_t cols_per_segment, pipelines::ColRange col_range, std::vector& segments, + std::vector& col_ranges, std::vector& row_ranges, + const std::tuple& index, ColumnSlice&& column_slice, CurrentCol&& current_col, RestCols&&... data +) { + auto new_column_slice = std::tuple_cat( + std::forward(column_slice), std::forward_as_tuple(std::forward(current_col)) + ); + + constexpr size_t new_slice_size = std::tuple_size_v + 1; + + if constexpr (sizeof...(RestCols) == 0) { + slice_column_slice_into_row_slices( + index, + new_column_slice, + descriptors.front(), + rows_per_segment, + col_range, + segments, + col_ranges, + row_ranges + ); + } else if (new_slice_size == cols_per_segment) { + slice_column_slice_into_row_slices( + index, + new_column_slice, + descriptors.front(), + rows_per_segment, + col_range, + segments, + col_ranges, + row_ranges + ); + col_range.first = col_range.second; + col_range.second = std::min(col_range.first + cols_per_segment, col_range.first + sizeof...(RestCols)); + slice_data_into_segments( + descriptors.last(descriptors.size() - 1), + rows_per_segment, + cols_per_segment, + col_range, + segments, + col_ranges, + row_ranges, + index, + std::tuple{}, + std::forward(data)... + ); + } else { + slice_data_into_segments( + descriptors, + rows_per_segment, + cols_per_segment, + col_range, + segments, + col_ranges, + row_ranges, + index, + std::move(new_column_slice), + std::forward(data)... + ); + } +} + +template +std::tuple, std::vector, std::vector> +slice_data_into_segments( + const StreamDescriptor& descriptor, const size_t rows_per_segment, const size_t cols_per_segment, Data&&... data +) { + std::vector descriptors = split_descriptor(descriptor, cols_per_segment); + std::vector segments; + std::vector col_ranges; + std::vector row_ranges; + + auto [index_columns, data_columns] = split_pack(std::forward(data)...); + std::apply( + [&](Cols&&... cols) { + slice_data_into_segments( + descriptors, + rows_per_segment, + cols_per_segment, + pipelines::ColRange{ + index::field_count(), index::field_count() + std::min(cols_per_segment, sizeof...(Cols)) + }, + segments, + col_ranges, + row_ranges, + std::move(index_columns), + std::tuple{}, + std::forward(cols)... + ); + }, + std::forward(data_columns) + ); + return std::make_tuple(std::move(segments), std::move(col_ranges), std::move(row_ranges)); +} } // namespace arcticdb + +namespace fmt { +template<> +struct formatter { + template + constexpr auto parse(ParseContext& ctx) { + return ctx.begin(); + } + + template + constexpr auto format(const arcticdb::SegmentInMemory& segment, FormatContext& ctx) const { + const StreamDescriptor& desc = segment.descriptor(); + fmt::format_to(ctx.out(), "Segment\n"); + for (unsigned i = 0; i < desc.field_count(); ++i) { + fmt::format_to(ctx.out(), "\nColumn[{}]: {}\n", i, desc.field(i)); + visit_field(desc.field(i), [&](auto tdt) { + using TDT = decltype(tdt); + arcticdb::ColumnData cd = segment.column_data(i); + for (auto it = cd.begin(); it != cd.end(); ++it) { + if constexpr (std::same_as) { + fmt::format_to(ctx.out(), "{} ", i, int(*it)); + } else { + fmt::format_to(ctx.out(), "{} ", i, *it); + } + } + }); + } + } +}; +} // namespace fmt \ No newline at end of file diff --git a/cpp/arcticdb/column_store/string_pool.cpp b/cpp/arcticdb/column_store/string_pool.cpp index 82e9e50d395..738f9d2b14d 100644 --- a/cpp/arcticdb/column_store/string_pool.cpp +++ b/cpp/arcticdb/column_store/string_pool.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include namespace arcticdb { diff --git a/cpp/arcticdb/pipeline/input_frame_utils.hpp b/cpp/arcticdb/pipeline/input_frame_utils.hpp new file mode 100644 index 00000000000..75666cf643b --- /dev/null +++ b/cpp/arcticdb/pipeline/input_frame_utils.hpp @@ -0,0 +1,73 @@ +/* Copyright 2023 Man Group Operations Limited + * + * Use of this software is governed by the Business Source License 1.1 included in the file licenses/BSL.txt. + * + * As of the Change Date specified in that file, in accordance with the Business Source License, use of this software + * will be governed by the Apache License, version 2.0. + */ + +#pragma once +#include +namespace arcticdb::pipelines { + +template +requires std::ranges::sized_range && (std::ranges::sized_range && ...) +auto materialize_ranges(H&& head, T&&... tail) { + if constexpr (sizeof...(T) == 0) { + if constexpr (!std::ranges::contiguous_range) { + return std::forward_as_tuple(std::vector>(std::forward(head))); + } else { + return std::forward_as_tuple(std::forward(head)); + } + } else { + if constexpr (!std::ranges::contiguous_range) { + return std::tuple_cat( + std::forward_as_tuple(std::vector>(std::forward(head))), + materialize_ranges(std::forward(tail)...) + ); + } else { + return std::tuple_cat( + std::forward_as_tuple(std::forward(head)), materialize_ranges(std::forward(tail)...) + ); + } + } +} + +template +requires((Index::field_count() == 0 || Index::field_count() == 1) && (std::ranges::sized_range && ...)) +auto input_frame_from_tensors(const StreamDescriptor& desc, T&&... input) { + constexpr static size_t data_columns = sizeof...(T) - Index::field_count(); + auto materialized_input = materialize_ranges(std::forward(input)...); + [&](std::index_sequence) { + const size_t first_row_count = std::get<0>(materialized_input).size(); + util::check( + ((std::ranges::size(std::get(materialized_input)) == first_row_count) && ...), + "All input data must have the same number of rows" + ); + }(std::make_index_sequence{}); + std::vector tensors = [&](std::index_sequence) { + std::vector result_tensors; + result_tensors.reserve(data_columns); + (result_tensors.push_back(NativeTensor::one_dimensional_tensor( + std::get(materialized_input), + desc.field(Is + Index::field_count()).type().data_type() + )), + ...); + return result_tensors; + }(std::make_index_sequence{}); + const size_t num_rows = std::ranges::size(std::get<0>(materialized_input)); + if constexpr (Index::field_count() == 1) { + InputFrame result_frame( + desc, + std::move(tensors), + NativeTensor::one_dimensional_tensor(std::get<0>(materialized_input), desc.field(0).type().data_type()) + ); + result_frame.num_rows = num_rows; + return std::pair{std::move(result_frame), std::move(materialized_input)}; + } else { + InputFrame result_frame(desc, std::move(tensors), std::nullopt); + result_frame.num_rows = num_rows; + return std::pair{std::move(result_frame), std::move(materialized_input)}; + } +} +} // namespace arcticdb::pipelines diff --git a/cpp/arcticdb/pipeline/write_frame.cpp b/cpp/arcticdb/pipeline/write_frame.cpp index 2ec562f0632..709f77c414b 100644 --- a/cpp/arcticdb/pipeline/write_frame.cpp +++ b/cpp/arcticdb/pipeline/write_frame.cpp @@ -281,6 +281,7 @@ folly::Future> slice_and_write( auto slices = slice(*frame, slicing); if (slices.empty()) return folly::makeFuture(std::vector{}); + ARCTICDB_SUBSAMPLE_DEFAULT(SliceAndWrite) TypedStreamVersion tsv{std::move(key.id), key.version_id, KeyType::TABLE_DATA}; return write_slices(frame, std::move(slices), slicing, std::move(tsv), sink, de_dup_map, sparsify_floats) diff --git a/cpp/arcticdb/processing/clause.cpp b/cpp/arcticdb/processing/clause.cpp index 9f215896783..48376b3a325 100644 --- a/cpp/arcticdb/processing/clause.cpp +++ b/cpp/arcticdb/processing/clause.cpp @@ -26,32 +26,55 @@ #include #include #include +#include #include namespace { +/// Removes inplace all elements from vec whose indexes are not in indexes_to_keep. +/// @param indexes_to_keep Must be sorted template -void stable_select(std::vector& vec, std::span indexes_to_keep) { +void select(std::span indexes_to_keep, std::vector& vec) { arcticdb::debug::check( std::ranges::is_sorted(indexes_to_keep), - "Bulk removal of elements from vector requires the indexes of the elements to be sorted" + "Bulk selection of elements from vector requires the indexes of the elements to be sorted" ); if (indexes_to_keep.size() == vec.size()) { return; } if (indexes_to_keep.empty()) { - vec.resize(0); - } - size_t free_slot = 0, to_keep = 0; - for (size_t i = 0; i < vec.size(); ++i) { - if (free_slot == indexes_to_keep[to_keep]) { - to_keep++; - free_slot++; - } else if (i == indexes_to_keep[to_keep]) { - vec[free_slot++] = std::move(vec[indexes_to_keep[to_keep++]]); + vec.clear(); + } + size_t free_slot = 0; + for (const size_t to_keep : indexes_to_keep) { + if (free_slot != to_keep) { + vec[free_slot] = std::move(vec[to_keep]); + } + free_slot++; + } + vec.erase(vec.begin() + free_slot, vec.end()); +} + +/// Remove all row slice entities and ranges and keys whose indexes do not appear in row_slices_to_keep +/// @param row_slices_to_keep Must be sorted +/// @param offsets Must be structured by row slice with ranges and keys +/// @param ranges_and_keys Must be structured by row slice with offsets +void filter_selected_ranges_and_keys_and_reindex_entities( + const std::span row_slices_to_keep, std::vector>& offsets, + std::vector& ranges_and_keys +) { + select(row_slices_to_keep, offsets); + size_t new_entity_id = 0; + for (std::span row_slice : offsets) { + for (size_t& entity_id : row_slice) { + if (entity_id != new_entity_id) { + ranges_and_keys[new_entity_id] = std::move(ranges_and_keys[entity_id]); + entity_id = new_entity_id; + } + new_entity_id++; } } - vec.resize(indexes_to_keep.size()); + ranges_and_keys.erase(ranges_and_keys.begin() + new_entity_id, ranges_and_keys.end()); } } // namespace @@ -1744,6 +1767,10 @@ OutputSchema WriteClause::join_schemas(std::vector&&) const { std::string WriteClause::to_string() const { return "Write"; } +/// Helper class used to iterate over elements inside InputFrame uniformly. +/// +/// It has two template specialisations for each way an InputFrame can store its data. It's supposed to be used only to +/// perform forward iteration. template class SourceView {}; @@ -1782,9 +1809,7 @@ class SourceView { public: SourceView(const T& source, const size_t source_data_row) : source_(source), source_data_row_(source_data_row) {} - void set_row(const size_t new_row) { source_data_row_ = new_row; } - const TDT::DataTypeTag::raw_type& operator*() const { return source_.at(source_data_row_); } private: @@ -1824,55 +1849,92 @@ MergeUpdateClause::MergeUpdateClause( ); } +/// Row range indexes require full table scan +/// In case of timestamp index this will filter out only the ranges and keys whose index span contains at least one +/// value from the source index. This does not mean that there's a match only that a match is possible. A crucial +/// assumption is that the source is ordered. This means that after ranges_and_keys are ordered by row slice we can +/// perform only forward iteration over the source index to find matches (except the edge of one segment starting with +/// the same value as the previous ends, see below) std::vector> MergeUpdateClause::structure_for_processing(std::vector& ranges_and_keys ) { if (!source_->has_index()) { return structure_by_row_slice(ranges_and_keys); } - std::vector> entities = structure_by_row_slice(ranges_and_keys); + std::vector> offsets = structure_by_row_slice(ranges_and_keys); std::vector row_slices_to_keep; size_t source_row = 0; auto first_col_slice_in_row = ranges_and_keys.begin(); - for (size_t row_slice_idx = 0; row_slice_idx < entities.size() && source_row < source_->num_rows; ++row_slice_idx) { + for (size_t row_slice_idx = 0; row_slice_idx < offsets.size() && source_row < source_->num_rows; ++row_slice_idx) { const TimestampRange time_range = first_col_slice_in_row->key_.time_range(); - // If we're inserting and the source timestamp is before the start of the row slice, we are going to insert the - // new values at the beginning of each column slice in row slice, thus we need to keep it. - // TODO: Improvement if strategy.matched is DO_NOTHING or there is no other intersection between source and the - // row slice there is no need to read it. We can just create a row slice. timestamp source_ts = source_->index_value_at(source_row); - bool keep_row_slice = strategy_.not_matched_by_target == MergeAction::INSERT && source_ts < time_range.first; - if (keep_row_slice) { - source_start_for_row_range_[first_col_slice_in_row->row_range()] = std::pair{source_row, source_row + 1}; - row_slices_to_keep.push_back(row_slice_idx); - } + // TODO: Add logic for insertion. If we're skipping source rows and strategy.not_matched_by_target is INSERT + // all the skipped rows must be inserted // Skip all values in the source that are before the first index value in the row slice - while (source_row < source_->num_rows && source_ts < time_range.first) { - source_ts = source_->index_value_at(++source_row); - } - const bool source_ts_in_segment_range = source_ts >= time_range.first && source_ts <= time_range.second; - if (!keep_row_slice && source_ts_in_segment_range) { - source_start_for_row_range_[first_col_slice_in_row->row_range()] = std::pair{source_row, source_row + 1}; - row_slices_to_keep.push_back(row_slice_idx); + while (source_row < source_->num_rows) { + source_ts = source_->index_value_at(source_row); + if (source_ts >= time_range.first) { + break; + } + ++source_row; } - keep_row_slice |= source_ts_in_segment_range; + const bool source_ts_in_segment_range = source_ts >= time_range.first && source_ts < time_range.second; + const size_t first_source_row_in_row_slice = source_row; + // The time range for a segment is [index start value in ns, index end value + 1 in ns). + // Thus, the sequence of end[i], start[i+1] is not always increasing when a segment starts with the same + // value as the previous ends. E.g. + // Segment[0] index = [0, 1, 5] -> TimeRange = [0, 6) + // Segment[1] index = [5, 5] -> TimeRange = [5, 6) + // In this case we need to start matching the next segment from the first occurrence of the last value + size_t last_value_first_occurrence = source_row; // Find the first row in source that is after the row slice - while (source_row < source_->num_rows && source_ts >= time_range.first && source_ts <= time_range.second) { - source_ts = source_->index_value_at(++source_row); + while (source_row < source_->num_rows) { + const timestamp new_source_ts = source_->index_value_at(source_row); + if (source_ts != new_source_ts) { + last_value_first_occurrence = source_row; + } + source_ts = new_source_ts; + if (new_source_ts >= time_range.second) { + break; + } + ++source_row; } - if (keep_row_slice) { - source_start_for_row_range_.at(first_col_slice_in_row->row_range()).second = source_row; + if (source_ts_in_segment_range) { + row_slices_to_keep.push_back(row_slice_idx); + source_start_end_for_row_range_.emplace( + first_col_slice_in_row->row_range(), std::pair{first_source_row_in_row_slice, source_row} + ); + // The last value of the current row range can be the same as the first value of the next segment. Start + // iterating the source from the first occurrence of that index value + if (row_slice_idx + 1 < ranges_and_keys.size()) { + const TimestampRange next_segment_range = ranges_and_keys[row_slice_idx + 1].key_.time_range(); + const bool index_value_spans_two_segments = next_segment_range.first + 1 == time_range.second; + const bool next_segment_starts_with_last_used_source_index = + next_segment_range.first == source_->index_value_at(last_value_first_occurrence); + if (index_value_spans_two_segments && next_segment_starts_with_last_used_source_index) { + source_row = last_value_first_occurrence; + } + } } - const size_t col_slice_count = entities[row_slice_idx].size(); + const size_t col_slice_count = offsets[row_slice_idx].size(); first_col_slice_in_row += col_slice_count; } - stable_select(entities, row_slices_to_keep); - return entities; + filter_selected_ranges_and_keys_and_reindex_entities(row_slices_to_keep, offsets, ranges_and_keys); + return offsets; } std::vector> MergeUpdateClause::structure_for_processing(std::vector>&&) { internal::raise("MergeUpdate clause should be the first clause in the pipeline"); } +/// Decide which rows of should be updated and which rows from source should be inserted. +/// 1. If there's a timestamp index use MergeUpdateClause::filter_index_match this will produce a vector of size equal +/// to the number of rows from the source that fall into the processed slice. Each vector will contain a vector of +/// row-indexes in target that match the corresponding soruce index value. +/// 2. For each column in MergeUpdateClause::on_ iterate over the vector of vectors produced in the previous step. +/// Checking for match only the target rows that are in the inner vector. If there is no match for this particular +/// column remove the target row index. +/// This means that the ordering of the columns in MergeUpdateClause::on_ matters and it would be more efficient to +/// start with the columns that have a lesser chance of matching. std::vector MergeUpdateClause::process(std::vector&& entity_ids) const { if (entity_ids.empty()) { return {}; @@ -1880,6 +1942,8 @@ std::vector MergeUpdateClause::process(std::vector&& entity_ auto proc = gather_entities, std::shared_ptr, std::shared_ptr>( *component_manager_, std::move(entity_ids) ); + // TODO: Add exception handling two source rows matching the same target row. This should be done in the function + // handling the "on" parameter matching multiple columns. std::vector> matched = filter_index_match(proc); if (source_->has_segment()) { update_and_insert(source_->segment(), source_->desc(), proc, matched); @@ -1901,7 +1965,9 @@ void MergeUpdateClause::update_and_insert( const std::span> target_segments = *proc.segments_; const std::span> row_ranges = *proc.row_ranges_; const std::span> col_ranges = *proc.col_ranges_; - const auto [source_row_start, source_row_end] = source_start_for_row_range_.at(*row_ranges[0]); + const auto [source_row_start, source_row_end] = source_start_end_for_row_range_.at(*row_ranges[0]); + // Update one column at a time to increase cache coherency and to avoid calling visit_field for each row being + // updated for (size_t segment_idx = 0; segment_idx < target_segments.size(); ++segment_idx) { const size_t columns_in_range = target_segments[segment_idx]->num_columns(); for (size_t column_idx = source_descriptor.index().field_count(); column_idx < columns_in_range; ++column_idx) { @@ -1942,7 +2008,8 @@ void MergeUpdateClause::update_and_insert( }(); SourceView source_column_view = get_source_column_iterator(source, source_column); while (source_row < source_row_end) { - std::span rows_to_update_for_source_row = rows_to_update[source_row - source_row_start]; + std::span rows_to_update_for_source_row = + rows_to_update[source_row - source_row_start]; if (rows_to_update_for_source_row.empty()) { ++source_row; continue; @@ -1968,11 +2035,19 @@ template void MergeUpdateClause:: update_and_insert(const std::vector&, const StreamDescriptor&, const ProcessingUnit&, std::span>) const; +/// For each row of source that falls in the row slice in proc find all rows whose index matches the source index value. +/// The matching rows will be sorted in increasing order. Since both source and target are timestamp indexed and +/// ordered, only forward iteration on both source and target is needed and binary search can be used to check if a +/// source index value exists in the target index. At the end some vectors in the output can be empty which means that +/// that particular row in source did not match anything in the target. It is allowed for one row in target to be +/// matched by multiple rows in source only if MergeUpdateClause::on_ is not empty. If MergeUpdateClause::on_ is not +/// empty there will be further filtering which might remove some matches. Otherwise, one row will be updated multiple +/// times which is not allowed. std::vector> MergeUpdateClause::filter_index_match(const ProcessingUnit& proc) const { using IndexType = ScalarTagType>; const std::span> target_segments{*proc.segments_}; const std::span> row_ranges{*proc.row_ranges_}; - const auto [source_row_start, source_row_end] = source_start_for_row_range_.at(*row_ranges[0]); + const auto [source_row_start, source_row_end] = source_start_end_for_row_range_.at(*row_ranges[0]); size_t source_row = source_row_start; const size_t source_rows_in_row_slice = source_row_end - source_row; std::vector> matched_rows(source_rows_in_row_slice); @@ -1982,6 +2057,7 @@ std::vector> MergeUpdateClause::filter_index_match(const Pro const auto target_index_end = target_index.cend(); while (target_index_it != target_index_end && source_row < source_row_end) { const timestamp source_ts = source_->index_value_at(source_row); + // TODO: Profile and compare to linear or adaptive (linear below some threshold) search auto lower_bound = std::lower_bound(target_index_it, target_index_end, source_ts); if (lower_bound == target_index_end) { break; @@ -2007,10 +2083,10 @@ void MergeUpdateClause::set_component_manager(std::shared_ptr } OutputSchema MergeUpdateClause::modify_schema(OutputSchema&& output_schema) const { - internal::check( - output_schema.stream_descriptor() == source_->desc(), - "Cannot perform merge update when the source and target schema are not the same. Source schema: {}, target " - "schema: {}", + schema::check( + columns_match(output_schema.stream_descriptor(), source_->desc()), + "Cannot perform merge update when the source and target schema are not the same.\nSource schema: " + "{}\nTarget schema: {}", source_->desc(), output_schema.stream_descriptor() ); diff --git a/cpp/arcticdb/processing/clause.hpp b/cpp/arcticdb/processing/clause.hpp index 62fde101940..95a72a905c1 100644 --- a/cpp/arcticdb/processing/clause.hpp +++ b/cpp/arcticdb/processing/clause.hpp @@ -843,6 +843,11 @@ struct WriteClause { stream::PartialKey create_partial_key(const SegmentInMemory& segment) const; }; +/// This clause will perform update values or insert values based on strategy_ in a segment. The source of new values is +/// the source_ member. Source and target must have the same index type. There are two actions +/// UPDATE: For a particular row in the segment if there's a row source_ for which all values in the columns listed in +/// on and the index (only in case if timeseries) match update will be performed. +/// INSERT: Each row in source_ not matched by the target will be inserted struct MergeUpdateClause { ClauseInfo clause_info_; std::shared_ptr component_manager_; @@ -883,12 +888,12 @@ struct MergeUpdateClause { update_and_insert(const T&, const StreamDescriptor&, const ProcessingUnit&, std::span>) const; - /// @return Vector of size equal to the number of source data rows that are withing the row slice being processed. + /// @return Vector of size equal to the number of source data rows that are within the row slice being processed. /// Each element is a vector of the rows from the target data that has the same index as the corresponding source /// row std::vector> filter_index_match(const ProcessingUnit& proc) const; /// For each row range stores the first and last row in the source that overlaps with the row range - ankerl::unordered_dense::map, RowRange::Hasher> source_start_for_row_range_; + ankerl::unordered_dense::map, RowRange::Hasher> source_start_end_for_row_range_; }; } // namespace arcticdb diff --git a/cpp/arcticdb/processing/clause_utils.hpp b/cpp/arcticdb/processing/clause_utils.hpp index 1da667b747d..98232596567 100644 --- a/cpp/arcticdb/processing/clause_utils.hpp +++ b/cpp/arcticdb/processing/clause_utils.hpp @@ -92,9 +92,9 @@ struct RangesAndEntity { }; template -requires util::any_of +requires std::is_same_v || std::is_same_v std::vector> structure_by_row_slice(std::vector& ranges) { - std::ranges::sort(ranges, [](const T& left, const T& right) { + std::sort(std::begin(ranges), std::end(ranges), [](const T& left, const T& right) { return std::tie(left.row_range().first, left.col_range().first) < std::tie(right.row_range().first, right.col_range().first); }); diff --git a/cpp/arcticdb/processing/test/test_merge_update.cpp b/cpp/arcticdb/processing/test/test_merge_update.cpp index ef8f96b3079..9f0d4959697 100644 --- a/cpp/arcticdb/processing/test/test_merge_update.cpp +++ b/cpp/arcticdb/processing/test/test_merge_update.cpp @@ -14,6 +14,8 @@ #include #include #include +#include +#include using namespace arcticdb; using namespace std::ranges; @@ -30,192 +32,6 @@ StreamDescriptor non_string_fields_ts_index_descriptor() { return TimeseriesIndex::default_index().create_stream_descriptor("Source", non_string_fields); } -std::vector split_descriptor(const StreamDescriptor& descriptor, const size_t cols_per_segment) { - if (descriptor.fields().size() <= cols_per_segment) { - return std::vector{descriptor}; - } - const size_t num_segments = descriptor.fields().size() / cols_per_segment; - std::vector res; - res.reserve(num_segments); - - const unsigned field_count = descriptor.field_count(); - for (size_t i = 0, source_field = descriptor.index().field_count(); i < num_segments; ++i) { - StreamDescriptor partial(descriptor.id()); - if (descriptor.index().field_count() > 0) { - partial.set_index(descriptor.index()); - for (unsigned index_field = 0; index_field < descriptor.index().field_count(); ++index_field) { - partial.add_field(descriptor.field(index_field)); - } - } - for (size_t field = 0; field < cols_per_segment && source_field < field_count; ++field) { - partial.add_field(descriptor.field(source_field++)); - } - res.push_back(std::move(partial)); - } - return res; -} - -template -requires util::instantiation_of && util::instantiation_of -auto tuple_cat_ref(const T1& tuple1, const T2& tuple2) { - return std::apply( - [&](auto&&... args1) { - return std::apply( - [&](auto&&... args2) { - return std::tie( - std::forward(args1)..., std::forward(args2)... - ); - }, - tuple2 - ); - }, - tuple1 - ); -} - -template -requires(sizeof...(Data) > 0) -void slice_column_slice_into_row_slices( - const std::tuple& index, const std::tuple& column_slice, const StreamDescriptor& desc, - const size_t rows_per_segment, const ColRange& col_range, std::vector& segments, - std::vector& col_ranges, std::vector& row_ranges -) { - const size_t total_rows = std::ranges::size(std::get<0>(column_slice)); - for (size_t start_row = 0; start_row < total_rows; start_row += rows_per_segment) { - std::apply( - [&](const Cols&... cols) { - const size_t rows_to_take = std::min(rows_per_segment, total_rows - start_row); - segments.push_back(SegmentInMemory::create_dense_segment( - desc, std::ranges::take_view(std::ranges::drop_view(cols, start_row), rows_to_take)... - )); - row_ranges.emplace_back(start_row, start_row + rows_to_take); - col_ranges.emplace_back(col_range); - }, - tuple_cat_ref(index, column_slice) - ); - } -} - -template -auto split(Data&&... data) { - constexpr static auto non_index_columns = sizeof...(Data) - N; - return [&]( - std::index_sequence, std::index_sequence, T&& fwd_tuple - ) { - return std::pair{ - std::forward_as_tuple(std::get(std::forward(fwd_tuple))...), - std::forward_as_tuple(std::get(std::forward(fwd_tuple))...) - }; - }(std::make_index_sequence{}, - std::make_index_sequence{}, - std::forward_as_tuple(std::forward(data)...)); -} - -template< - std::ranges::sized_range IndexCols, typename ColumnSlice, std::ranges::sized_range CurrentCol, - std::ranges::sized_range... RestCols> -requires( - util::instantiation_of && - [](std::index_sequence) { - return (std::ranges::sized_range> && ...); - }(std::make_index_sequence>{}) -) -void slice_data_into_segments( - const std::span descriptors, const size_t rows_per_segment, - const size_t cols_per_segment, ColRange col_range, std::vector& segments, - std::vector& col_ranges, std::vector& row_ranges, const std::tuple& index, - ColumnSlice&& column_slice, CurrentCol&& current_col, RestCols&&... data -) { - auto new_column_slice = std::tuple_cat( - std::forward(column_slice), std::forward_as_tuple(std::forward(current_col)) - ); - - constexpr size_t new_slice_size = std::tuple_size_v + 1; - - if constexpr (sizeof...(RestCols) == 0) { - slice_column_slice_into_row_slices( - index, - new_column_slice, - descriptors.front(), - rows_per_segment, - col_range, - segments, - col_ranges, - row_ranges - ); - } else if (new_slice_size == cols_per_segment) { - slice_column_slice_into_row_slices( - index, - new_column_slice, - descriptors.front(), - rows_per_segment, - col_range, - segments, - col_ranges, - row_ranges - ); - col_range.first = col_range.second; - col_range.second = std::min(col_range.first + cols_per_segment, col_range.first + sizeof...(RestCols)); - slice_data_into_segments( - descriptors.last(descriptors.size() - 1), - rows_per_segment, - cols_per_segment, - col_range, - segments, - col_ranges, - row_ranges, - index, - std::tuple{}, - std::forward(data)... - ); - } else { - slice_data_into_segments( - descriptors, - rows_per_segment, - cols_per_segment, - col_range, - segments, - col_ranges, - row_ranges, - index, - std::move(new_column_slice), - std::forward(data)... - ); - } -} - -template -std::tuple, std::vector, std::vector> slice_data_into_segments( - const StreamDescriptor& descriptor, const size_t rows_per_segment, const size_t cols_per_segment, Data&&... data -) { - std::vector descriptors = split_descriptor(descriptor, cols_per_segment); - std::vector segments; - std::vector col_ranges; - std::vector row_ranges; - - auto [index_columns, data_columns] = split(std::forward(data)...); - std::apply( - [&](Cols&&... cols) { - slice_data_into_segments( - descriptors, - rows_per_segment, - cols_per_segment, - ColRange{ - index::field_count(), index::field_count() + std::min(cols_per_segment, sizeof...(Cols)) - }, - segments, - col_ranges, - row_ranges, - std::move(index_columns), - std::tuple{}, - std::forward(cols)... - ); - }, - std::forward(data_columns) - ); - return std::make_tuple(std::move(segments), std::move(col_ranges), std::move(row_ranges)); -} - template std::vector> wrap_in_shared_ptr(std::vector&& v) { std::vector> res; @@ -236,9 +52,10 @@ std::vector generate_ranges_and_keys( ranges_and_keys.emplace_back( row_ranges[i], col_ranges[i], - AtomKeyBuilder().start_index(start_ts).end_index(end_ts).build( - source_descriptor.id() - ) + AtomKeyBuilder() + .start_index(start_ts) + .end_index(end_ts + 1) + .build(source_descriptor.id()) ); } return ranges_and_keys; @@ -274,105 +91,38 @@ void sort_by_rowslice(std::span rows, std::span cols, Other& }(correct_positions, rows, cols, other...); } -std::vector> map_entities_to_structure_for_processing_output( - const std::span> structure_for_processing_out, - const std::span entities +MergeUpdateClause create_clause( + const MergeStrategy strategy, std::shared_ptr component_manager, InputFrame&& input_frame ) { - std::vector> process_input; - process_input.reserve(structure_for_processing_out.size()); - std::ranges::transform( - structure_for_processing_out, - std::back_inserter(process_input), - [&](const std::vector& indices) { - std::vector result; - result.reserve(indices.size()); - std::ranges::transform(indices, std::back_inserter(result), [&](const size_t idx) { - return entities[idx]; - }); - return result; - } - ); - return process_input; + MergeUpdateClause clause({}, strategy, std::make_shared(std::move(input_frame)), true); + clause.set_component_manager(std::move(component_manager)); + return clause; } -void print_segment(const SegmentInMemory& segment) { - const StreamDescriptor& desc = segment.descriptor(); - for (unsigned i = 0; i < desc.field_count(); ++i) { - std::cout << fmt::format("Print column[{}]: {}\n", i, desc.field(i)); - visit_field(desc.field(i), [&](auto tdt) { - using TDT = decltype(tdt); - ColumnData cd = segment.column_data(i); - for (auto it = cd.begin(); it != cd.end(); ++it) { - if constexpr (std::same_as) { - std::cout << int(*it) << " "; - } else { - std::cout << *it << " "; - } - } - std::cout << "\n"; - }); - } -} - -template -requires std::ranges::sized_range && (std::ranges::sized_range && ...) -auto materialize_ranges(H&& head, T&&... tail) { - if constexpr (sizeof...(T) == 0) { - if constexpr (!std::ranges::contiguous_range) { - return std::forward_as_tuple(std::vector>(std::forward(head))); - } else { - return std::forward_as_tuple(std::forward(head)); - } - } else { - if constexpr (!std::ranges::contiguous_range) { - return std::tuple_cat( - std::forward_as_tuple(std::vector>(std::forward(head))), - materialize_ranges(std::forward(tail)...) - ); - } else { - return std::tuple_cat( - std::forward_as_tuple(std::forward(head)), materialize_ranges(std::forward(tail)...) - ); +std::vector push_selected_entities( + ComponentManager& component_manager, const std::span ranges_and_keys, + std::vector&& segments_in, std::vector&& col_ranges_in, + std::vector&& row_ranges_in +) { + size_t idx = 0; + std::vector> segments; + std::vector> col_ranges; + std::vector> row_ranges; + for (const RangesAndKey& range : ranges_and_keys) { + while ((idx < segments_in.size()) && + (range.row_range() != row_ranges_in[idx] || range.col_range() != col_ranges_in[idx])) { + ++idx; } + segments.emplace_back(std::make_shared(std::move(segments_in[idx]))); + col_ranges.emplace_back(std::make_shared(std::move(col_ranges_in[idx]))); + row_ranges.emplace_back(std::make_shared(std::move(row_ranges_in[idx]))); } -} - -template -requires((Index::field_count() == 0 || Index::field_count() == 1) && (std::ranges::sized_range && ...)) -auto input_frame_from_tensors(const StreamDescriptor& desc, T&&... input) { - constexpr static size_t data_columns = sizeof...(T) - Index::field_count(); - auto materialized_input = materialize_ranges(std::forward(input)...); - [&](std::index_sequence) { - const size_t first_row_count = std::get<0>(materialized_input).size(); - util::check( - ((std::ranges::size(std::get(materialized_input)) == first_row_count) && ...), - "All input data must have the same number of rows" - ); - }(std::make_index_sequence{}); - std::vector tensors = [&](std::index_sequence) { - std::vector result_tensors; - result_tensors.reserve(data_columns); - (result_tensors.push_back(NativeTensor::one_dimensional_tensor( - std::get(materialized_input), - desc.field(Is + Index::field_count()).type().data_type() - )), - ...); - return result_tensors; - }(std::make_index_sequence{}); - const size_t num_rows = std::ranges::size(std::get<0>(materialized_input)); - if constexpr (Index::field_count() == 1) { - InputFrame result_frame( - desc, - std::move(tensors), - NativeTensor::one_dimensional_tensor(std::get<0>(materialized_input), desc.field(0).type().data_type()) - ); - result_frame.num_rows = num_rows; - return std::pair{std::move(result_frame), std::move(materialized_input)}; - } else { - InputFrame result_frame(desc, std::move(tensors), std::nullopt); - result_frame.num_rows = num_rows; - return std::pair{std::move(result_frame), std::move(materialized_input)}; - } + return component_manager.add_entities( + std::move(segments), + std::move(col_ranges), + std::move(row_ranges), + std::vector(ranges_and_keys.size(), 0) + ); } struct MergeUpdateClauseUpdateStrategyTestBase { @@ -385,28 +135,54 @@ struct MergeUpdateClauseUpdateStrategyTestBase { auto [segments, col_ranges, row_ranges] = std::move(data); sort_by_rowslice(row_ranges, col_ranges, segments); ranges_and_keys_ = generate_ranges_and_keys(descriptor_, segments, col_ranges, row_ranges); - const size_t entity_count = segments.size(); - initial_entities_ = component_manager_->add_entities( - wrap_in_shared_ptr(std::move(col_ranges)), - wrap_in_shared_ptr(std::move(row_ranges)), - wrap_in_shared_ptr(std::move(segments)), - std::vector(entity_count, 0) - ); + row_ranges_ = std::move(row_ranges); + col_ranges_ = std::move(col_ranges); + segments_ = std::move(segments); } MergeUpdateClause create_clause(InputFrame&& input_frame) const { - MergeUpdateClause clause({}, strategy_, std::make_shared(std::move(input_frame)), true); - clause.set_component_manager(component_manager_); - return clause; + return ::create_clause(strategy_, component_manager_, std::move(input_frame)); + } + + std::vector push_entities() { + return push_selected_entities( + *component_manager_, + ranges_and_keys_, + std::move(segments_), + std::move(col_ranges_), + std::move(row_ranges_) + ); } StreamDescriptor descriptor_; std::vector ranges_and_keys_; std::shared_ptr component_manager_ = std::make_shared(); MergeStrategy strategy_; - std::vector initial_entities_; + std::vector segments_; + std::vector row_ranges_; + std::vector col_ranges_; }; +std::vector> structure_entities( + const std::span> structure_indices, const std::vector& entities +) { + std::vector> structured_entities; + structured_entities.reserve(structure_indices.size()); + std::ranges::transform( + structure_indices, + std::back_inserter(structured_entities), + [&](std::span indices) { + std::vector result; + result.reserve(indices.size()); + std::ranges::transform(indices, std::back_inserter(result), [&](const size_t idx) { + return entities[idx]; + }); + return result; + } + ); + return structured_entities; +} + /// Param is a tuple of rows_per_segment and cols_per_segment struct MergeUpdateClauseUpdateStrategyMatchAllSegTest : MergeUpdateClauseUpdateStrategyTestBase, testing::TestWithParam> { @@ -452,6 +228,7 @@ struct MergeUpdateClauseUpdateStrategyMatchAllSegTest : MergeUpdateClauseUpdateS const size_t row_slices_to_process = (num_rows_ + rows_per_segment() - 1) / rows_per_segment(); const size_t column_slices_per_row_slice = (num_cols_ + cols_per_segment() - 1) / cols_per_segment(); ASSERT_EQ(structure_indices.size(), row_slices_to_process); + ASSERT_EQ(ranges_and_keys_.size(), row_slices_to_process * column_slices_per_row_slice); for (size_t i = 0; i < structure_indices.size(); ++i) { SCOPED_TRACE(testing::Message() << fmt::format("structure index: {}", i)); EXPECT_EQ(structure_indices[i].size(), column_slices_per_row_slice); @@ -466,15 +243,15 @@ struct MergeUpdateClauseUpdateStrategyMatchAllSegTest : MergeUpdateClauseUpdateS void assert_process_results_match_expected( const MergeUpdateClause& clause, const std::span expected_segments, - const std::span> structure_indices, - std::vector>&& entities_to_process + const std::span> structure_indices, const std::vector& entities ) const { + std::vector> structured_entities = structure_entities(structure_indices, entities); const size_t column_slices_per_row_slice = (num_cols_ + cols_per_segment() - 1) / cols_per_segment(); - for (size_t i = 0; i < entities_to_process.size(); ++i) { + for (size_t i = 0; i < structured_entities.size(); ++i) { auto proc = gather_entities< std::shared_ptr, std::shared_ptr, - std::shared_ptr>(*component_manager_, clause.process(std::move(entities_to_process[i]))); + std::shared_ptr>(*component_manager_, clause.process(std::move(structured_entities[i]))); SCOPED_TRACE(testing::Message() << "processing result (row slice) = " << i); EXPECT_EQ(proc.segments_->size(), column_slices_per_row_slice); EXPECT_EQ(proc.row_ranges_->size(), column_slices_per_row_slice); @@ -515,12 +292,7 @@ TEST_P(MergeUpdateClauseUpdateStrategyMatchAllSegTest, SourceIndexMatchesAllSegm std::array{1000, 1, 2, 3, 4, 5, 6, 2000, 8, 9, 10, 11, 12, 13, 3000} ); sort_by_rowslice(expected_row_ranges, expected_col_ranges, expected_segments); - assert_process_results_match_expected( - clause, - expected_segments, - structure_indices, - map_entities_to_structure_for_processing_output(structure_indices, initial_entities_) - ); + assert_process_results_match_expected(clause, expected_segments, structure_indices, push_entities()); } TEST_P(MergeUpdateClauseUpdateStrategyMatchAllSegTest, SourceHasValuesOutsideOfTheDataFrame) { @@ -550,12 +322,7 @@ TEST_P(MergeUpdateClauseUpdateStrategyMatchAllSegTest, SourceHasValuesOutsideOfT std::array{1000, 1, 2, 3, 4, 5, 6, 2000, 8, 9, 10, 11, 12, 13, 3000} ); sort_by_rowslice(expected_row_ranges, expected_col_ranges, expected_segments); - assert_process_results_match_expected( - clause, - expected_segments, - structure_indices, - map_entities_to_structure_for_processing_output(structure_indices, initial_entities_) - ); + assert_process_results_match_expected(clause, expected_segments, structure_indices, push_entities()); } INSTANTIATE_TEST_SUITE_P( @@ -617,6 +384,7 @@ TEST_F(MergeUpdateClauseUpdateStrategyMatchSubsetTest, NoMatch) { MergeUpdateClause clause = create_clause(std::move(input_frame)); const std::vector> structure_indices = clause.structure_for_processing(ranges_and_keys_); ASSERT_EQ(structure_indices.size(), 0); + ASSERT_EQ(ranges_and_keys_.size(), 0); } TEST_F(MergeUpdateClauseUpdateStrategyMatchSubsetTest, MatchFirst) { @@ -631,15 +399,16 @@ TEST_F(MergeUpdateClauseUpdateStrategyMatchSubsetTest, MatchFirst) { ); MergeUpdateClause clause = create_clause(std::move(input_frame)); const std::vector> structure_indices = clause.structure_for_processing(ranges_and_keys_); + ASSERT_EQ(ranges_and_keys_.size(), 2); ASSERT_EQ(structure_indices.size(), 1); ASSERT_EQ(structure_indices[0].size(), 2); ASSERT_EQ(ranges_and_keys_[structure_indices[0][0]].row_range(), RowRange(0, 5)); ASSERT_EQ(ranges_and_keys_[structure_indices[0][1]].row_range(), RowRange(0, 5)); ASSERT_EQ(ranges_and_keys_[structure_indices[0][0]].col_range(), ColRange(1, 4)); ASSERT_EQ(ranges_and_keys_[structure_indices[0][1]].col_range(), ColRange(4, 6)); - const std::vector result_entities = clause.process( - std::move(map_entities_to_structure_for_processing_output(structure_indices, initial_entities_)[0]) - ); + std::vector entities = push_entities(); + std::vector> structured_entities = structure_entities(structure_indices, entities); + const std::vector result_entities = clause.process(std::move(structured_entities[0])); auto proc = gather_entities, std::shared_ptr, std::shared_ptr>( *component_manager_, result_entities ); @@ -674,15 +443,16 @@ TEST_F(MergeUpdateClauseUpdateStrategyMatchSubsetTest, MatchSecond) { ); MergeUpdateClause clause = create_clause(std::move(input_frame)); const std::vector> structure_indices = clause.structure_for_processing(ranges_and_keys_); + ASSERT_EQ(ranges_and_keys_.size(), 2); ASSERT_EQ(structure_indices.size(), 1); ASSERT_EQ(structure_indices[0].size(), 2); ASSERT_EQ(ranges_and_keys_[structure_indices[0][0]].row_range(), RowRange(5, 10)); ASSERT_EQ(ranges_and_keys_[structure_indices[0][1]].row_range(), RowRange(5, 10)); ASSERT_EQ(ranges_and_keys_[structure_indices[0][0]].col_range(), ColRange(1, 4)); ASSERT_EQ(ranges_and_keys_[structure_indices[0][1]].col_range(), ColRange(4, 6)); - const std::vector result_entities = clause.process( - std::move(map_entities_to_structure_for_processing_output(structure_indices, initial_entities_)[0]) - ); + std::vector entities = push_entities(); + std::vector> structured_entities = structure_entities(structure_indices, entities); + const std::vector result_entities = clause.process(std::move(structured_entities[0])); auto proc = gather_entities, std::shared_ptr, std::shared_ptr>( *component_manager_, result_entities ); @@ -717,6 +487,7 @@ TEST_F(MergeUpdateClauseUpdateStrategyMatchSubsetTest, MatchThird) { ); MergeUpdateClause clause = create_clause(std::move(input_frame)); const std::vector> structure_indices = clause.structure_for_processing(ranges_and_keys_); + ASSERT_EQ(ranges_and_keys_.size(), 2); ASSERT_EQ(structure_indices.size(), 1); ASSERT_EQ(structure_indices[0].size(), 2); ASSERT_EQ(ranges_and_keys_[structure_indices[0][0]].row_range(), RowRange(10, 14)); @@ -724,9 +495,9 @@ TEST_F(MergeUpdateClauseUpdateStrategyMatchSubsetTest, MatchThird) { ASSERT_EQ(ranges_and_keys_[structure_indices[0][0]].col_range(), ColRange(1, 4)); ASSERT_EQ(ranges_and_keys_[structure_indices[0][1]].col_range(), ColRange(4, 6)); - const std::vector result_entities = clause.process( - std::move(map_entities_to_structure_for_processing_output(structure_indices, initial_entities_)[0]) - ); + std::vector entities = push_entities(); + std::vector> structured_entities = structure_entities(structure_indices, entities); + const std::vector result_entities = clause.process(std::move(structured_entities[0])); auto proc = gather_entities, std::shared_ptr, std::shared_ptr>( *component_manager_, result_entities ); @@ -761,6 +532,7 @@ TEST_F(MergeUpdateClauseUpdateStrategyMatchSubsetTest, MatchFirstAndThird) { ); MergeUpdateClause clause = create_clause(std::move(input_frame)); const std::vector> structure_indices = clause.structure_for_processing(ranges_and_keys_); + ASSERT_EQ(ranges_and_keys_.size(), 4); ASSERT_EQ(structure_indices.size(), 2); ASSERT_EQ(structure_indices[0].size(), 2); ASSERT_EQ(structure_indices[1].size(), 2); @@ -772,9 +544,9 @@ TEST_F(MergeUpdateClauseUpdateStrategyMatchSubsetTest, MatchFirstAndThird) { ASSERT_EQ(ranges_and_keys_[structure_indices[1][1]].row_range(), RowRange(10, 14)); ASSERT_EQ(ranges_and_keys_[structure_indices[1][0]].col_range(), ColRange(1, 4)); ASSERT_EQ(ranges_and_keys_[structure_indices[1][1]].col_range(), ColRange(4, 6)); - std::vector> entities_to_process = - map_entities_to_structure_for_processing_output(structure_indices, initial_entities_); - const std::vector result_entities_0 = clause.process(std::move(entities_to_process[0])); + std::vector entities = push_entities(); + std::vector> structured_entities = structure_entities(structure_indices, entities); + const std::vector result_entities_0 = clause.process(std::move(structured_entities[0])); auto proc_0 = gather_entities, std::shared_ptr, std::shared_ptr>( *component_manager_, result_entities_0 @@ -798,7 +570,7 @@ TEST_F(MergeUpdateClauseUpdateStrategyMatchSubsetTest, MatchFirstAndThird) { ASSERT_EQ(*(*proc_0.col_ranges_)[0], RowRange(1, 4)); ASSERT_EQ(*(*proc_0.col_ranges_)[1], RowRange(4, 6)); - const std::vector result_entities_1 = clause.process(std::move(entities_to_process[1])); + const std::vector result_entities_1 = clause.process(std::move(structured_entities[1])); auto proc_1 = gather_entities, std::shared_ptr, std::shared_ptr>( *component_manager_, result_entities_1 @@ -835,6 +607,7 @@ TEST_F(MergeUpdateClauseUpdateStrategyMatchSubsetTest, MatchFirstAndSecond) { ); MergeUpdateClause clause = create_clause(std::move(input_frame)); const std::vector> structure_indices = clause.structure_for_processing(ranges_and_keys_); + ASSERT_EQ(ranges_and_keys_.size(), 4); ASSERT_EQ(structure_indices.size(), 2); ASSERT_EQ(structure_indices[0].size(), 2); ASSERT_EQ(structure_indices[1].size(), 2); @@ -846,9 +619,9 @@ TEST_F(MergeUpdateClauseUpdateStrategyMatchSubsetTest, MatchFirstAndSecond) { ASSERT_EQ(ranges_and_keys_[structure_indices[1][1]].row_range(), RowRange(5, 10)); ASSERT_EQ(ranges_and_keys_[structure_indices[1][0]].col_range(), ColRange(1, 4)); ASSERT_EQ(ranges_and_keys_[structure_indices[1][1]].col_range(), ColRange(4, 6)); - std::vector> entities_to_process = - map_entities_to_structure_for_processing_output(structure_indices, initial_entities_); - const std::vector result_entities_0 = clause.process(std::move(entities_to_process[0])); + std::vector entities = push_entities(); + std::vector> structured_entities = structure_entities(structure_indices, entities); + const std::vector result_entities_0 = clause.process(std::move(structured_entities[0])); auto proc_0 = gather_entities, std::shared_ptr, std::shared_ptr>( *component_manager_, result_entities_0 @@ -872,7 +645,7 @@ TEST_F(MergeUpdateClauseUpdateStrategyMatchSubsetTest, MatchFirstAndSecond) { ASSERT_EQ(*(*proc_0.col_ranges_)[0], RowRange(1, 4)); ASSERT_EQ(*(*proc_0.col_ranges_)[1], RowRange(4, 6)); - const std::vector result_entities_1 = clause.process(std::move(entities_to_process[1])); + const std::vector result_entities_1 = clause.process(std::move(structured_entities[1])); auto proc_1 = gather_entities, std::shared_ptr, std::shared_ptr>( *component_manager_, result_entities_1 @@ -909,6 +682,7 @@ TEST_F(MergeUpdateClauseUpdateStrategyMatchSubsetTest, MatchSecondAndThird) { ); MergeUpdateClause clause = create_clause(std::move(input_frame)); const std::vector> structure_indices = clause.structure_for_processing(ranges_and_keys_); + ASSERT_EQ(ranges_and_keys_.size(), 4); ASSERT_EQ(structure_indices.size(), 2); ASSERT_EQ(structure_indices[0].size(), 2); ASSERT_EQ(structure_indices[1].size(), 2); @@ -920,9 +694,9 @@ TEST_F(MergeUpdateClauseUpdateStrategyMatchSubsetTest, MatchSecondAndThird) { ASSERT_EQ(ranges_and_keys_[structure_indices[1][1]].row_range(), RowRange(10, 14)); ASSERT_EQ(ranges_and_keys_[structure_indices[1][0]].col_range(), ColRange(1, 4)); ASSERT_EQ(ranges_and_keys_[structure_indices[1][1]].col_range(), ColRange(4, 6)); - std::vector> entities_to_process = - map_entities_to_structure_for_processing_output(structure_indices, initial_entities_); - const std::vector result_entities_0 = clause.process(std::move(entities_to_process[0])); + std::vector entities = push_entities(); + std::vector> structured_entities = structure_entities(structure_indices, entities); + const std::vector result_entities_0 = clause.process(std::move(structured_entities[0])); auto proc_0 = gather_entities, std::shared_ptr, std::shared_ptr>( *component_manager_, result_entities_0 @@ -946,7 +720,7 @@ TEST_F(MergeUpdateClauseUpdateStrategyMatchSubsetTest, MatchSecondAndThird) { ASSERT_EQ(*(*proc_0.col_ranges_)[0], RowRange(1, 4)); ASSERT_EQ(*(*proc_0.col_ranges_)[1], RowRange(4, 6)); - const std::vector result_entities_1 = clause.process(std::move(entities_to_process[1])); + const std::vector result_entities_1 = clause.process(std::move(structured_entities[1])); auto proc_1 = gather_entities, std::shared_ptr, std::shared_ptr>( *component_manager_, result_entities_1 @@ -969,4 +743,62 @@ TEST_F(MergeUpdateClauseUpdateStrategyMatchSubsetTest, MatchSecondAndThird) { ASSERT_EQ(*(*proc_1.row_ranges_)[1], RowRange(10, 14)); ASSERT_EQ(*(*proc_1.col_ranges_)[0], RowRange(1, 4)); ASSERT_EQ(*(*proc_1.col_ranges_)[1], RowRange(4, 6)); +} + +TEST(RepeatedValues, RepeatedValueSpansSeveralSegments) { + for (timestamp source_index = 1; source_index < 4; ++source_index) { + constexpr static std::array fields{ + FieldRef(TypeDescriptor(DataType::INT32, Dimension::Dim0), "a"), + FieldRef(TypeDescriptor(DataType::INT32, Dimension::Dim0), "b") + }; + constexpr static MergeStrategy strategy = { + .matched = MergeAction::UPDATE, .not_matched_by_target = MergeAction::DO_NOTHING + }; + const StreamDescriptor desc = TimeseriesIndex::default_index().create_stream_descriptor("TestStream", fields); + auto [target_segments, target_column_ranges, target_row_ranges] = slice_data_into_segments( + desc, 2, 1, std::array{2, 2, 2, 2, 2, 2}, iota_view{0, 6}, iota_view{0, 6} + ); + sort_by_rowslice(target_row_ranges, target_column_ranges, target_segments); + std::vector ranges_and_keys = + generate_ranges_and_keys(desc, target_segments, target_column_ranges, target_row_ranges); + auto component_manager = std::make_shared(); + auto [input_frame, _] = input_frame_from_tensors( + desc, std::array{source_index}, std::array{100}, std::array{200} + ); + MergeUpdateClause clause = create_clause(strategy, component_manager, std::move(input_frame)); + const std::vector> structure_indices = clause.structure_for_processing(ranges_and_keys); + if (source_index != 2) { + ASSERT_EQ(structure_indices.size(), 0); + } else { + ASSERT_EQ(structure_indices.size(), 3); + std::vector entities = push_selected_entities( + *component_manager, + ranges_and_keys, + std::move(target_segments), + std::move(target_column_ranges), + std::move(target_row_ranges) + ); + std::vector> structured_entities = structure_entities(structure_indices, entities); + const std::vector result_entities = clause.process(std::move(structured_entities[0])); + auto proc = gather_entities< + std::shared_ptr, + std::shared_ptr, + std::shared_ptr>(*component_manager, result_entities); + auto [expected_segments, expected_col_slices, expected_row_slices] = + slice_data_into_segments( + desc, + 2, + 1, + std::array{2, 2, 2, 2, 2, 2}, + iota_view{0, 6} | views::transform([](auto) { return 100; }), + iota_view{0, 6} | views::transform([](auto) { return 200; }) + ); + sort_by_rowslice(expected_row_slices, expected_col_slices, expected_segments); + for (size_t i = 0; i < proc.segments_->size(); ++i) { + ASSERT_EQ(*proc.segments_->at(i), expected_segments[i]); + ASSERT_EQ(*proc.col_ranges_->at(i), expected_col_slices[i]); + ASSERT_EQ(*proc.row_ranges_->at(i), expected_row_slices[i]); + } + } + } } \ No newline at end of file diff --git a/cpp/arcticdb/processing/test/test_resample.cpp b/cpp/arcticdb/processing/test/test_resample.cpp index 9cf66c58c2c..1d335cd4a92 100644 --- a/cpp/arcticdb/processing/test/test_resample.cpp +++ b/cpp/arcticdb/processing/test/test_resample.cpp @@ -476,25 +476,19 @@ TYPED_TEST(SortedAggregatorSparseStructure, NoMissingInputColumnsProducesDenseCo constexpr ResampleBoundary label = TypeParam::label; constexpr static std::array bucket_boundaries{0, 10, 20, 30, 40, 50}; constexpr static std::array output_index = generate_labels(bucket_boundaries, label); - Column output_index_column = Column::create_dense_column(output_index, IndexTDT::type_descriptor()); + Column output_index_column = create_dense_column(output_index); const std::array input_index_columns{ - std::make_shared( - Column::create_dense_column(std::array{1, 2, 3}, IndexTDT::type_descriptor()) - ), - std::make_shared( - Column::create_dense_column(std::array{11, 21, 31, 41}, IndexTDT::type_descriptor()) - ) + std::make_shared(create_dense_column(std::array{1, 2, 3})), + std::make_shared(create_dense_column(std::array{11, 21, 31, 41})) }; const std::array input_agg_columns{ std::make_optional(ColumnWithStrings{ - Column::create_dense_column(std::array{0, 5, 6}, TypeDescriptor::scalar_type(DataType::INT32)), + create_dense_column>>(std::array{0, 5, 6}), nullptr, "col1" }), std::make_optional(ColumnWithStrings{ - Column::create_dense_column( - std::array{10, 35, 56, 1, 2}, TypeDescriptor::scalar_type(DataType::INT32) - ), + create_dense_column>>(std::array{10, 35, 56, 1, 2}), nullptr, "col1" }) @@ -532,18 +526,14 @@ TYPED_TEST(SortedAggregatorSparseStructure, FirstColumnExistSecondIsMissing) { constexpr ResampleBoundary label = TypeParam::label; constexpr static std::array bucket_boundaries{0, 10, 20}; constexpr static std::array output_index = generate_labels(bucket_boundaries, label); - const Column output_index_column = Column::create_dense_column(output_index, IndexTDT::type_descriptor()); + const Column output_index_column = create_dense_column(output_index); const std::array input_index_columns{ - std::make_shared( - Column::create_dense_column(std::array{0, 2, 3}, IndexTDT::type_descriptor()) - ), - std::make_shared( - Column::create_dense_column(std::array{11, 21, 22, 24}, IndexTDT::type_descriptor()) - ) + std::make_shared(create_dense_column(std::array{0, 2, 3})), + std::make_shared(create_dense_column(std::array{11, 21, 22, 24})) }; const std::array input_agg_columns{ std::make_optional(ColumnWithStrings{ - Column::create_dense_column(std::array{0, 5, 6}, TypeDescriptor::scalar_type(DataType::INT32)), + create_dense_column>>(std::array{0, 5, 6}), nullptr, "col1" }), @@ -572,22 +562,18 @@ TYPED_TEST(SortedAggregatorSparseStructure, FirstColumnExistWithValueOnRightBoun constexpr ResampleBoundary closed = TypeParam::SortedAggregator::closed; const Column output_index_column = []() { if constexpr (label == ResampleBoundary::LEFT) { - return Column::create_dense_column(std::array{0, 10, 30}, IndexTDT::type_descriptor()); + return create_dense_column(std::array{0, 10, 30}); } else { - return Column::create_dense_column(std::array{10, 20, 40}, IndexTDT::type_descriptor()); + return create_dense_column(std::array{10, 20, 40}); } }(); const std::array input_index_columns{ - std::make_shared( - Column::create_dense_column(std::array{0, 2, 10}, IndexTDT::type_descriptor()) - ), - std::make_shared( - Column::create_dense_column(std::array{35, 36}, IndexTDT::type_descriptor()) - ) + std::make_shared(create_dense_column(std::array{0, 2, 10})), + std::make_shared(create_dense_column(std::array{35, 36})) }; const std::array input_agg_columns{ std::make_optional(ColumnWithStrings{ - Column::create_dense_column(std::array{0, 5, 6}, TypeDescriptor::scalar_type(DataType::INT32)), + create_dense_column>>(std::array{0, 5, 6}), nullptr, "col1" }), @@ -623,18 +609,14 @@ TYPED_TEST(SortedAggregatorSparseStructure, ReturnDenseInCaseOutputIndexIsFilled constexpr ResampleBoundary label = TypeParam::label; constexpr static std::array bucket_boundaries{0, 10, 20}; constexpr static std::array output_index = generate_labels(bucket_boundaries, label); - const Column output_index_column = Column::create_dense_column(output_index, IndexTDT::type_descriptor()); + const Column output_index_column = create_dense_column(output_index); const std::array input_index_columns{ - std::make_shared( - Column::create_dense_column(std::array{0, 2, 12}, IndexTDT::type_descriptor()) - ), - std::make_shared( - Column::create_dense_column(std::array{15, 16, 18, 20}, IndexTDT::type_descriptor()) - ) + std::make_shared(create_dense_column(std::array{0, 2, 12})), + std::make_shared(create_dense_column(std::array{15, 16, 18, 20})) }; const std::array input_agg_columns{ std::make_optional(ColumnWithStrings{ - Column::create_dense_column(std::array{0, 5, 6}, TypeDescriptor::scalar_type(DataType::INT32)), + create_dense_column>>(std::array{0, 5, 6}), nullptr, "col1" }), @@ -658,19 +640,15 @@ TYPED_TEST(SortedAggregatorSparseStructure, ReturnDenseInCaseOutputIndexIsFilled constexpr ResampleBoundary label = TypeParam::label; constexpr static std::array bucket_boundaries{0, 10, 20}; constexpr static std::array output_index = generate_labels(bucket_boundaries, label); - const Column output_index_column = Column::create_dense_column(output_index, IndexTDT::type_descriptor()); + const Column output_index_column = create_dense_column(output_index); const std::array input_index_columns{ - std::make_shared( - Column::create_dense_column(std::array{0, 2, 5}, IndexTDT::type_descriptor()) - ), - std::make_shared( - Column::create_dense_column(std::array{7, 8, 9, 15}, IndexTDT::type_descriptor()) - ) + std::make_shared(create_dense_column(std::array{0, 2, 5})), + std::make_shared(create_dense_column(std::array{7, 8, 9, 15})) }; const std::array input_agg_columns{ std::optional{}, std::make_optional(ColumnWithStrings{ - Column::create_dense_column(std::array{0, 5, 6, 5}, TypeDescriptor::scalar_type(DataType::INT32)), + create_dense_column>>(std::array{0, 5, 6, 5}), nullptr, "col1" }), @@ -691,19 +669,15 @@ TYPED_TEST(SortedAggregatorSparseStructure, FirstColumnIsMissing) { constexpr ResampleBoundary label = TypeParam::label; constexpr static std::array bucket_boundaries{0, 10, 20}; constexpr static std::array output_index = generate_labels(bucket_boundaries, label); - const Column output_index_column = Column::create_dense_column(output_index, IndexTDT::type_descriptor()); + const Column output_index_column = create_dense_column(output_index); const std::array input_index_columns{ - std::make_shared( - Column::create_dense_column(std::array{0, 2, 3}, IndexTDT::type_descriptor()) - ), - std::make_shared( - Column::create_dense_column(std::array{11, 15, 16, 17}, IndexTDT::type_descriptor()) - ) + std::make_shared(create_dense_column(std::array{0, 2, 3})), + std::make_shared(create_dense_column(std::array{11, 15, 16, 17})) }; const std::array input_agg_columns{ std::optional{}, std::make_optional(ColumnWithStrings{ - Column::create_dense_column(std::array{0, 5, 6, 5}, TypeDescriptor::scalar_type(DataType::INT32)), + create_dense_column>>(std::array{0, 5, 6, 5}), nullptr, "col1" }), @@ -731,30 +705,20 @@ TYPED_TEST(SortedAggregatorSparseStructure, ThreeSegmentsInABucketMiddleIsMissin constexpr ResampleBoundary label = TypeParam::label; constexpr static std::array bucket_boundaries{0, 10}; constexpr static std::array output_index = generate_labels(bucket_boundaries, label); - const Column output_index_column = Column::create_dense_column(output_index, IndexTDT::type_descriptor()); + const Column output_index_column = create_dense_column(output_index); const std::array input_index_columns{ - std::make_shared( - Column::create_dense_column(std::array{0, 1}, IndexTDT::type_descriptor()) - ), - std::make_shared( - Column::create_dense_column(std::array{2}, IndexTDT::type_descriptor()) - ), - std::make_shared( - Column::create_dense_column(std::array{3}, IndexTDT::type_descriptor()) - ) + std::make_shared(create_dense_column(std::array{0, 1})), + std::make_shared(create_dense_column(std::array{2})), + std::make_shared(create_dense_column(std::array{3})) }; const std::array input_agg_columns{ std::make_optional(ColumnWithStrings{ - Column::create_dense_column(std::array{1, 2}, TypeDescriptor::scalar_type(DataType::INT32)), - nullptr, - "col1" + create_dense_column>>(std::array{1, 2}), nullptr, "col1" }), std::optional{}, std::make_optional(ColumnWithStrings{ - Column::create_dense_column(std::array{3, 4}, TypeDescriptor::scalar_type(DataType::INT32)), - nullptr, - "col1" + create_dense_column>>(std::array{3, 4}), nullptr, "col1" }), }; const std::optional output = aggregator.generate_resampling_output_column( @@ -773,25 +737,17 @@ TYPED_TEST(SortedAggregatorSparseStructure, ThreeSegmentsInABuckeOnlyMiddleIsPre constexpr ResampleBoundary label = TypeParam::label; constexpr static std::array bucket_boundaries{0, 10}; constexpr static std::array output_index = generate_labels(bucket_boundaries, label); - const Column output_index_column = Column::create_dense_column(output_index, IndexTDT::type_descriptor()); + const Column output_index_column = create_dense_column(output_index); const std::array input_index_columns{ - std::make_shared( - Column::create_dense_column(std::array{0, 1}, IndexTDT::type_descriptor()) - ), - std::make_shared( - Column::create_dense_column(std::array{2}, IndexTDT::type_descriptor()) - ), - std::make_shared( - Column::create_dense_column(std::array{3}, IndexTDT::type_descriptor()) - ) + std::make_shared(create_dense_column(std::array{0, 1})), + std::make_shared(create_dense_column(std::array{2})), + std::make_shared(create_dense_column(std::array{3})) }; const std::array input_agg_columns{ std::optional{}, std::make_optional(ColumnWithStrings{ - Column::create_dense_column(std::array{1}, TypeDescriptor::scalar_type(DataType::INT32)), - nullptr, - "col1" + create_dense_column>>(std::array{1}), nullptr, "col1" }), std::optional{} }; diff --git a/cpp/arcticdb/processing/test/test_unsorted_aggregation.cpp b/cpp/arcticdb/processing/test/test_unsorted_aggregation.cpp index 6f40effa17e..704aa93d141 100644 --- a/cpp/arcticdb/processing/test/test_unsorted_aggregation.cpp +++ b/cpp/arcticdb/processing/test/test_unsorted_aggregation.cpp @@ -142,9 +142,7 @@ TEST_P(AggregationResult, Mean) { } else { ASSERT_EQ(data.size(), 0); } - const ColumnWithStrings input( - Column::create_dense_column(data, InputDataTypeTag::type_descriptor()), nullptr, "input" - ); + const ColumnWithStrings input(create_dense_column(data), nullptr, "input"); aggregator_data.aggregate(input, groups, group_count); const SegmentInMemory result = aggregator_data.finalize(ColumnName{"output"}, false, group_count); ASSERT_EQ(result.field(0).type(), make_scalar_type(OutputDataTypeTag::data_type())); diff --git a/cpp/arcticdb/util/configs_map.hpp b/cpp/arcticdb/util/configs_map.hpp index 98a3a53654d..981e3427011 100644 --- a/cpp/arcticdb/util/configs_map.hpp +++ b/cpp/arcticdb/util/configs_map.hpp @@ -14,7 +14,6 @@ #include #include #include -#include namespace arcticdb { diff --git a/cpp/arcticdb/util/test/test_utils.hpp b/cpp/arcticdb/util/test/test_utils.hpp index 9f89630fe35..2ffe00cce64 100644 --- a/cpp/arcticdb/util/test/test_utils.hpp +++ b/cpp/arcticdb/util/test/test_utils.hpp @@ -259,3 +259,20 @@ class StorageGenerator { const std::string storage_; inline static const fs::path TEST_DATABASES_PATH = "./test_databases"; }; + +template +requires requires(Input in) { + requires util::instantiation_of; + requires std::same_as>; +} +Column create_dense_column(const Input& data) { + constexpr static size_t element_size = sizeof(std::ranges::range_value_t); + Column result(TagType::type_descriptor(), data.size(), AllocationType::PRESIZED, Sparsity::NOT_PERMITTED); + if constexpr (std::ranges::contiguous_range) { + std::memcpy(result.ptr(), data.data(), data.size() * element_size); + } else { + std::ranges::copy(data, data.data()); + } + result.set_row_data(data.size()); + return result; +} \ No newline at end of file diff --git a/cpp/arcticdb/util/type_traits.hpp b/cpp/arcticdb/util/type_traits.hpp index dd040ee1c13..8e63f31e8b3 100644 --- a/cpp/arcticdb/util/type_traits.hpp +++ b/cpp/arcticdb/util/type_traits.hpp @@ -28,13 +28,6 @@ concept instantiation_of = is_instantiation_of_v; template concept any_of = std::disjunction_v...>; -template -concept contiguous_type_tagged_data = requires(T t) { - instantiation_of; - std::ranges::contiguous_range; - std::same_as; -}; - template concept decays_to = std::same_as, Base>; diff --git a/cpp/arcticdb/util/variant.hpp b/cpp/arcticdb/util/variant.hpp index 7970cc3ca95..8563c8e7806 100644 --- a/cpp/arcticdb/util/variant.hpp +++ b/cpp/arcticdb/util/variant.hpp @@ -47,11 +47,4 @@ auto variant_match(Variant&& v, Ts&&... ts) { return std::visit(overload{std::forward(ts)...}, std::forward(v)); } -template -void enumerate(InputFun&& callback, Elements&&... elements) { - [&callback, &elements...](std::index_sequence) { - (callback(Is_, std::forward(elements)), ...); - }(std::index_sequence_for{}); -} - } // namespace arcticdb::util diff --git a/cpp/arcticdb/version/local_versioned_engine.cpp b/cpp/arcticdb/version/local_versioned_engine.cpp index b567c4d9683..7e86f3ad55a 100644 --- a/cpp/arcticdb/version/local_versioned_engine.cpp +++ b/cpp/arcticdb/version/local_versioned_engine.cpp @@ -471,13 +471,13 @@ VersionedItem LocalVersionedEngine::read_modify_write_internal( } VersionedItem versioned_item = read_modify_write_impl( store(), - identifier, std::move(user_meta_proto), read_query, read_options, write_options, IndexPartialKey{target_stream, target_version}, - ReadModifyWriteIndexStrategy::REWRITE_INDEX + ReadModifyWriteIndexStrategy::REWRITE_INDEX, + setup_pipeline_context(store(), identifier, *read_query, read_options) ); if (cfg().symbol_list()) symbol_list().add_symbol(store(), target_stream, versioned_item.key_.version_id()); @@ -2321,11 +2321,26 @@ std::shared_ptr LocalVersionedEngine::_test_get_version_map() { retu void LocalVersionedEngine::_test_set_store(std::shared_ptr store) { set_store(std::move(store)); } VersionedItem LocalVersionedEngine::merge_internal( - const StreamId& stream_id, const std::shared_ptr& source, const py::object& user_meta, + const StreamId& stream_id, std::shared_ptr source, const py::object& user_meta, const bool prune_previous_versions, const MergeStrategy& strategy, std::vector&& on, const bool match_on_timeseries_index ) { - ARCTICDB_RUNTIME_DEBUG(log::version(), "Command: merge"); + ARCTICDB_RUNTIME_DEBUG(log::version(), "Command: merge_update"); + sorting::check( + index_is_not_timeseries_or_is_sorted_ascending(*source), + "If the source data is timeseries indexed it must be sorted in ascending order." + ); + // TODO: read_modify_write uses the same piece of code. Move it to a function. + std::unique_ptr user_meta_proto{ + [](const py::object& user_meta) -> proto::descriptors::UserDefinedMetadata* { + if (user_meta.is_none()) { + return nullptr; + } + const auto user_meta_proto = new proto::descriptors::UserDefinedMetadata(); + python_util::pb_from_python(user_meta, *user_meta_proto); + return user_meta_proto; + }(user_meta) + }; py::gil_scoped_release release_gil; const UpdateInfo update_info = get_latest_undeleted_version_and_next_version_id(store(), version_map(), stream_id); if (update_info.previous_index_key_.has_value()) { @@ -2342,17 +2357,6 @@ VersionedItem LocalVersionedEngine::merge_internal( const ReadOptions read_options; const auto source_version = get_version_to_read(stream_id, version_query); const auto identifier = get_version_identifier(stream_id, version_query, read_options, source_version); - // TODO: read_modify_write uses the same piece of code. Move it to a function. - std::unique_ptr user_meta_proto{ - [](const py::object& user_meta) -> proto::descriptors::UserDefinedMetadata* { - if (user_meta.is_none()) { - return nullptr; - } - const auto user_meta_proto = new proto::descriptors::UserDefinedMetadata(); - python_util::pb_from_python(user_meta, *user_meta_proto); - return user_meta_proto; - }(user_meta) - }; auto [maybe_prev, deleted] = ::arcticdb::get_latest_version(store(), version_map(), stream_id); const auto target_version = get_next_version_from_key(maybe_prev); if (target_version == 0) { @@ -2371,7 +2375,7 @@ VersionedItem LocalVersionedEngine::merge_internal( std::move(on), match_on_timeseries_index, strategy, - source + std::move(source) ); write_version_and_prune_previous(prune_previous_versions, versioned_item.key_, update_info.previous_index_key_); return versioned_item; diff --git a/cpp/arcticdb/version/local_versioned_engine.hpp b/cpp/arcticdb/version/local_versioned_engine.hpp index 7299c54569f..cb9f5ec1c15 100644 --- a/cpp/arcticdb/version/local_versioned_engine.hpp +++ b/cpp/arcticdb/version/local_versioned_engine.hpp @@ -349,7 +349,7 @@ class LocalVersionedEngine : public VersionedEngine { const arcticdb::proto::storage::VersionStoreConfig& cfg() const override { return cfg_; } VersionedItem merge_internal( - const StreamId& stream_id, const std::shared_ptr& source, const py::object& user_meta, + const StreamId& stream_id, std::shared_ptr source, const py::object& user_meta, bool prune_previous_versions, const MergeStrategy& strategy, std::vector&& on, bool match_on_timeseries_index ) override; diff --git a/cpp/arcticdb/version/python_bindings.cpp b/cpp/arcticdb/version/python_bindings.cpp index 9641511bcad..8dd32978399 100644 --- a/cpp/arcticdb/version/python_bindings.cpp +++ b/cpp/arcticdb/version/python_bindings.cpp @@ -225,6 +225,12 @@ void register_bindings(py::module& version, py::exception); version.def("Value", &construct_string_value); + py::enum_(version, "MergeAction") + .value("DO_NOTHING", MergeAction::DO_NOTHING) + .value("UPDATE", MergeAction::UPDATE) + .value("INSERT", MergeAction::INSERT) + .export_values(); + py::class_>(version, "ValueSet") .def(py::init([](std::vector&& value_list) { return std::make_shared(std::move(value_list)); diff --git a/cpp/arcticdb/version/version_core.cpp b/cpp/arcticdb/version/version_core.cpp index 42b6bd71e19..4602f2af7d3 100644 --- a/cpp/arcticdb/version/version_core.cpp +++ b/cpp/arcticdb/version/version_core.cpp @@ -38,6 +38,7 @@ #include #include #include +#include namespace arcticdb::version_store { @@ -201,6 +202,46 @@ folly::Future rewrite_index_for_read_modify_write( store ); } + +folly::Future merge_index_key_for_read_modify_write( + PipelineContext& pipeline_context, const WriteOptions& write_options, + const IndexPartialKey& target_partial_index_key, const std::shared_ptr& store, + std::vector&& new_slices, std::unique_ptr&& user_meta +) { + ranges::sort(new_slices, [](const SliceAndKey& a, const SliceAndKey& b) { + return std::tie(a.slice_.col_range, a.slice_.row_range) < std::tie(b.slice_.col_range, b.slice_.row_range); + }); + std::vector merged_ranges_and_keys; + auto new_slice = new_slices.begin(); + for (SliceAndKey& slice : pipeline_context.slice_and_keys_) { + if (new_slice != new_slices.end() && new_slice->slice_ == slice.slice_) { + merged_ranges_and_keys.push_back(std::move(*new_slice)); + ++new_slice; + } else { + merged_ranges_and_keys.push_back(std::move(slice)); + } + } + const size_t row_count = merged_ranges_and_keys.empty() ? 0 + : merged_ranges_and_keys.back().slice().row_range.second - + merged_ranges_and_keys[0].slice().row_range.first; + const TimeseriesDescriptor tsd = make_timeseries_descriptor( + row_count, + pipeline_context.descriptor(), + std::move(*pipeline_context.norm_meta_), + user_meta ? std::make_optional(*std::move(user_meta)) : std::nullopt, + std::nullopt, + std::nullopt, + write_options.bucketize_dynamic + ); + return index::write_index( + index_type_from_descriptor(pipeline_context.descriptor()), + tsd, + std::move(merged_ranges_and_keys), + target_partial_index_key, + store + ); + pipeline_context.slice_and_keys_.clear(); +} } // namespace void sorted_data_check_append(const InputFrame& frame, index::IndexSegmentReader& index_segment_reader) { @@ -2786,17 +2827,14 @@ folly::Future read_frame_for_version( } VersionedItem read_modify_write_impl( - const std::shared_ptr& store, const std::variant& version_info, - std::unique_ptr&& user_meta, std::shared_ptr read_query, - const ReadOptions& read_options, const WriteOptions& write_options, - const IndexPartialKey& target_partial_index_key, ReadModifyWriteIndexStrategy index_strategy + const std::shared_ptr& store, std::unique_ptr&& user_meta, + std::shared_ptr read_query, const ReadOptions& read_options, const WriteOptions& write_options, + const IndexPartialKey& target_partial_index_key, ReadModifyWriteIndexStrategy index_strategy, + std::shared_ptr pipeline_context ) { - read_query->clauses_.push_back(std::make_shared( WriteClause(write_options, target_partial_index_key, std::make_shared(), store) )); - const auto pipeline_context = setup_pipeline_context(store, version_info, *read_query, read_options); - check_can_perform_processing(pipeline_context, *read_query); auto component_manager = std::make_shared(); return read_and_schedule_processing(store, pipeline_context, read_query, read_options, component_manager) @@ -2824,8 +2862,20 @@ VersionedItem read_modify_write_impl( std::move(slices), std::move(user_meta) ); + } else if (index_strategy == ReadModifyWriteIndexStrategy::MERGE_INDEX) { + return merge_index_key_for_read_modify_write( + *pipeline_context, + write_options, + target_partial_index_key, + store, + std::move(slices), + std::move(user_meta) + ); } else { - return folly::Future(AtomKey{}); + internal::raise( + "Unknown read-modify-write index write strategy: {}", + static_cast>(index_strategy) + ); } }) .get(); @@ -2836,24 +2886,34 @@ VersionedItem merge_update_impl( std::unique_ptr&& user_meta, const ReadOptions& read_options, const WriteOptions& write_options, const IndexPartialKey& target_partial_index_key, std::vector&& on, const bool match_on_timeseries_index, const MergeStrategy& strategy, - const std::shared_ptr& source + std::shared_ptr source ) { auto read_query = std::make_shared(); - read_query->clauses_.push_back( - std::make_shared(MergeUpdateClause(std::move(on), strategy, source, match_on_timeseries_index)) - ); if (source->has_index()) { read_query->row_filter = source->index_range; } + const StreamDescriptor& source_descriptor = source->desc(); + read_query->clauses_.push_back(std::make_shared( + MergeUpdateClause(std::move(on), strategy, std::move(source), match_on_timeseries_index) + )); + std::shared_ptr pipeline_context = + setup_pipeline_context(store, version_info, *read_query, read_options); + schema::check( + columns_match(pipeline_context->descriptor(), source_descriptor), + "Cannot perform merge update when the source and target schema are not the same.\nSource schema: " + "{}\nTarget schema: {}", + source_descriptor, + pipeline_context->descriptor() + ); return read_modify_write_impl( store, - version_info, std::move(user_meta), std::move(read_query), read_options, write_options, target_partial_index_key, - ReadModifyWriteIndexStrategy::MERGE_INDEX + ReadModifyWriteIndexStrategy::MERGE_INDEX, + std::move(pipeline_context) ); } diff --git a/cpp/arcticdb/version/version_core.hpp b/cpp/arcticdb/version/version_core.hpp index fca9f180b7a..89e87657c6e 100644 --- a/cpp/arcticdb/version/version_core.hpp +++ b/cpp/arcticdb/version/version_core.hpp @@ -195,10 +195,10 @@ folly::Future prepare_output_frame( ); VersionedItem read_modify_write_impl( - const std::shared_ptr& store, const std::variant& version_info, - std::unique_ptr&& user_meta, std::shared_ptr read_query, - const ReadOptions& read_options, const WriteOptions& write_options, - const IndexPartialKey& target_partial_index_key, ReadModifyWriteIndexStrategy index_strategy + const std::shared_ptr& store, std::unique_ptr&& user_meta, + std::shared_ptr read_query, const ReadOptions& read_options, const WriteOptions& write_options, + const IndexPartialKey& target_partial_index_key, ReadModifyWriteIndexStrategy index_strategy, + std::shared_ptr pipeline_context ); VersionedItem merge_update_impl( @@ -206,7 +206,7 @@ VersionedItem merge_update_impl( std::unique_ptr&& user_meta, const ReadOptions& read_options, const WriteOptions& write_options, const IndexPartialKey& target_partial_index_key, std::vector&& on, bool match_on_timeseries_index, const MergeStrategy& strategy, - const std::shared_ptr& source + std::shared_ptr source ); std::shared_ptr setup_pipeline_context( diff --git a/cpp/arcticdb/version/versioned_engine.hpp b/cpp/arcticdb/version/versioned_engine.hpp index 344e7eff67d..cb5b4d08d14 100644 --- a/cpp/arcticdb/version/versioned_engine.hpp +++ b/cpp/arcticdb/version/versioned_engine.hpp @@ -147,7 +147,7 @@ class VersionedEngine { virtual void flush_version_map() = 0; virtual VersionedItem merge_internal( - const StreamId& stream_id, const std::shared_ptr& source, const py::object& user_meta, + const StreamId& stream_id, std::shared_ptr source, const py::object& user_meta, bool prune_previous_versions, const MergeStrategy& strategy, std::vector&& on, bool match_on_timeseries_index ) = 0; diff --git a/env.sh b/env.sh deleted file mode 100644 index 35bae234239..00000000000 --- a/env.sh +++ /dev/null @@ -1,18 +0,0 @@ -export ARCTICDB_PERSISTENT_STORAGE_TESTS=1 -export ARCTICDB_PERSISTENT_STORAGE_STRATEGY_BRANCH=blah -export ARCTICDB_PERSISTENT_STORAGE_SHARED_PATH_PREFIX=blah -export ARCTICDB_PERSISTENT_STORAGE_UNIQUE_PATH_PREFIX=blah -export ARCTICDB_REAL_S3_BUCKET=blah -export ARCTICDB_REAL_S3_ENDPOINT=https://s3.eu-west-1.amazonaws.com/ -export ARCTICDB_REAL_S3_REGION=eu-west-1 -export ARCTICDB_REAL_S3_CLEAR=0 -export ARCTICDB_REAL_S3_ACCESS_KEY=blah -export ARCTICDB_REAL_S3_SECRET_KEY=blah - -export ARCTICDB_REAL_AZURE_CONNECTION_STRING=blah -export ARCTICDB_REAL_AZURE_CONTAINER=blah - - -python -m asv run -v --show-stderr master^! --bench .*Resample.peakmem.*mean.* -python -m asv run -v --show-stderr HEAD^! --bench .*Resample.peakmem.*mean.* -python -m asv compare master HEAD diff --git a/python/arcticdb/util/test.py b/python/arcticdb/util/test.py index db80d629a52..cd88d642fb7 100644 --- a/python/arcticdb/util/test.py +++ b/python/arcticdb/util/test.py @@ -1248,3 +1248,16 @@ def compute_common_type_for_columns(segment_columns: List[dict]): else: common_types[name] = valid_common_type(common_types[name], np.dtype(dtype)) return common_types + + +def assert_vit_equals_except_data(left, right): + """ + Checks if two VersionedItem objects are equal disregarding differences in the data field. This is because when a + write is performed the returned VersionedItem does not contain a data field. + """ + assert left.symbol == right.symbol + assert left.library == right.library + assert left.version == right.version + assert left.metadata == right.metadata + assert left.host == right.host + assert left.timestamp == right.timestamp diff --git a/python/arcticdb/util/utils.py b/python/arcticdb/util/utils.py index d07a411e745..a976e8744ec 100644 --- a/python/arcticdb/util/utils.py +++ b/python/arcticdb/util/utils.py @@ -17,7 +17,6 @@ from arcticdb.util.test import create_datetime_index, get_sample_dataframe, random_integers, random_string from arcticdb.version_store.library import Library - # Types supported by arctic ArcticIntType = Union[np.uint8, np.uint16, np.uint32, np.uint64, np.int8, np.int16, np.int32, np.int64] ArcticFloatType = Union[np.float64, np.float32] diff --git a/python/arcticdb/version_store/library.py b/python/arcticdb/version_store/library.py index 554aa258916..492ab5ca828 100644 --- a/python/arcticdb/version_store/library.py +++ b/python/arcticdb/version_store/library.py @@ -26,7 +26,7 @@ from arcticdb.version_store.processing import ExpressionNode, QueryBuilder from arcticdb.version_store._store import NativeVersionStore, VersionedItem, VersionedItemWithJoin, VersionQueryInput from arcticdb_ext.exceptions import ArcticException -from arcticdb_ext.version_store import DataError, StageResult, KeyNotFoundInStageResultInfo +from arcticdb_ext.version_store import DataError, StageResult, KeyNotFoundInStageResultInfo, MergeAction import pandas as pd import numpy as np @@ -425,6 +425,11 @@ def __repr__(self): return res +class MergeStrategy(NamedTuple): + matched: Union[MergeAction, str] = MergeAction.UPDATE + not_matched_by_target: Union[MergeAction, str] = MergeAction.INSERT + + class LazyDataFrame(QueryBuilder): """ Lazy dataframe implementation, allowing chains of queries to be added before the read is actually executed. @@ -1995,7 +2000,7 @@ def read( DateRange to restrict read data to. Applicable only for time-indexed Pandas dataframes or series. Returns only the - part of the data that falls withing the given range (inclusive). None on either end leaves that part of the + part of the data that falls within the given range (inclusive). None on either end leaves that part of the range open-ended. Hence specifying ``(None, datetime(2025, 1, 1)`` declares that you wish to read all data up to and including 20250101. The same effect can be achieved by using the date_range clause of the QueryBuilder class, which will be @@ -3232,11 +3237,50 @@ def merge( self, symbol: str, data: NormalizableType, + strategy: MergeStrategy, on: Optional[List[str]] = None, metadata: Any = None, prune_previous_versions: bool = False, + upsert: bool = False, ): - pass + strategy = MergeStrategy( + matched=Library._normalize_merge_action(strategy.matched), + not_matched_by_target=Library._normalize_merge_action(strategy.not_matched_by_target), + ) + udm, item, norm_meta = self._nvs._try_normalize( + symbol, + data, + metadata, + pickle_on_failure=False, + dynamic_strings=True, + coerce_columns=None, + norm_failure_options_msg="Source data must be normalizable in order to merge it into existing dataframe", + ) + on_timeseries_index = True + on = [] if on is None else on + vit = self._nvs.version_store.merge( + symbol, item, norm_meta, udm, prune_previous_versions, strategy, on, on_timeseries_index + ) + return self._nvs._convert_thin_cxx_item_to_python(vit, metadata) + + @staticmethod + def _normalize_merge_action(action: Union[MergeAction, str]) -> MergeAction: + if isinstance(action, MergeAction): + return action + + if isinstance(action, str): + action = action.lower() + + if action == "update": + return MergeAction.UPDATE + elif action == "insert": + return MergeAction.INSERT + elif action == "do_nothing": + return MergeAction.DO_NOTHING + else: + raise ArcticInvalidApiUsageException( + f"Invalid MergeAction: {action}. Must be one of: update, insert, do_nothing." + ) @property def name(self) -> str: diff --git a/python/test_index.py b/python/test_index.py deleted file mode 100644 index f1fbac8e492..00000000000 --- a/python/test_index.py +++ /dev/null @@ -1,28 +0,0 @@ -import arcticdb -import pandas as pd -import numpy as np - -ac = arcticdb.Arctic("lmdb://test") -opts = arcticdb.LibraryOptions(columns_per_segment=2, rows_per_segment=2) -if ac.has_library("test"): - ac.delete_library("test") -lib = ac.get_library("test", library_options=opts, create_if_missing=True) - -arrays = [ - [pd.Timestamp(1), pd.Timestamp(1), pd.Timestamp(2), pd.Timestamp(2), pd.Timestamp(3)], - [10, 20, 10, 20, 10], -] # First index: integers # Second index: integers -index = pd.MultiIndex.from_arrays(arrays, names=("index1", "index2")) - -data = np.random.randn(5, 5) -columns = ["col1", "col2", "col3", "col4", "col5"] - -df = pd.DataFrame(data, index=index, columns=columns) - -ts_df = pd.DataFrame(data, columns=columns, index=pd.date_range("2023-01-01", periods=5, freq="h")) - -qb = arcticdb.QueryBuilder() -qb = qb[qb["col1"] > 0] - -lib.write("test", ts_df) -lib.read("test", query_builder=qb) diff --git a/python/tests/unit/arcticdb/version_store/test_merge.py b/python/tests/unit/arcticdb/version_store/test_merge.py index 37e3895a81d..0aa4b686425 100644 --- a/python/tests/unit/arcticdb/version_store/test_merge.py +++ b/python/tests/unit/arcticdb/version_store/test_merge.py @@ -11,6 +11,7 @@ from arcticdb.util.test import assert_frame_equal, assert_vit_equals_except_data import arcticdb from arcticdb.version_store import VersionedItem +from arcticdb_ext.exceptions import SchemaException from arcticdb_ext.storage import KeyType import numpy as np from arcticdb.exceptions import StreamDescriptorMismatch, UserInputException, SortingException, StorageException @@ -151,7 +152,7 @@ def test_static_schema_merge_throws_when_schemas_differ(self, lmdb_library, stra lib = lmdb_library target = pd.DataFrame({"a": [1, 2, 3], "b": [1.0, 2.0, 3.0]}, index=pd.date_range("2024-01-01", periods=3)) lib.write("sym", target) - with pytest.raises(StreamDescriptorMismatch): + with pytest.raises(SchemaException): lib.merge("sym", source, strategy=strategy) @pytest.mark.parametrize( @@ -182,7 +183,7 @@ def test_throws_if_source_is_not_sorted(self, lmdb_library, strategy): ), ) - with pytest.raises(UserInputException): + with pytest.raises(SortingException): lib.merge("sym", source, strategy=strategy) # ================================================================================================ @@ -264,19 +265,19 @@ def test_merge_update_writes_new_version_even_if_nothing_is_changed(self, lmdb_l def test_merge_update_row_slicing(self, lmdb_library_factory, slicing_policy): lib = lmdb_library_factory(arcticdb.LibraryOptions(**slicing_policy)) target = pd.DataFrame( - {"a": [1, 2, 3, 4, 5], "b": [1.0, 2.0, 3.0, 4.0, 5.0], "c": ["a", "b", "c", "d", "e"]}, + {"a": [1, 2, 3, 4, 5], "b": [1.0, 2.0, 3.0, 4.0, 5.0], "c": [True, False, True, False, True]}, index=pd.date_range("2024-01-01", periods=5), ) lib.write("sym", target) source = pd.DataFrame( - {"a": [30, 50], "b": [30.1, 50.1], "c": ["C", "E"]}, + {"a": [30, 50], "b": [30.1, 50.1], "c": [False, False]}, index=pd.DatetimeIndex([pd.Timestamp("2024-01-03"), pd.Timestamp("2024-01-05")]), ) lib.merge("sym", source, strategy=MergeStrategy(not_matched_by_target=MergeAction.DO_NOTHING)) expected = pd.DataFrame( - {"a": [1, 2, 30, 4, 50], "b": [1.0, 2.0, 30.1, 4.0, 50.1], "c": ["a", "b", "C", "d", "E"]}, + {"a": [1, 2, 30, 4, 50], "b": [1.0, 2.0, 30.1, 4.0, 50.1], "c": [True, False, False, False, False]}, index=pd.date_range("2024-01-01", periods=5), ) received = lib.read("sym").data @@ -301,6 +302,7 @@ def test_merge_update_row_slicing(self, lmdb_library_factory, slicing_policy): assert len(lt.find_keys_for_symbol(KeyType.TABLE_INDEX, "sym")) == 2 assert len(lt.find_keys_for_symbol(KeyType.VERSION, "sym")) == 2 + @pytest.mark.skip(reason="Not implemented") def test_merge_update_on_index_and_column(self, lmdb_library, monkeypatch): lib = lmdb_library target = pd.DataFrame({"a": [1, 2, 3], "b": [1.0, 2.0, 3.0]}, index=pd.date_range("2024-01-01", periods=3)) @@ -326,6 +328,7 @@ def test_merge_update_on_index_and_column(self, lmdb_library, monkeypatch): assert_frame_equal(received, expected) + @pytest.mark.skip(reason="String columns not implemented") def test_merge_update_on_multiple_columns(self, lmdb_library, monkeypatch): lib = lmdb_library target = pd.DataFrame( diff --git a/python/tests/unit/arcticdb/version_store/test_recursive_normalizers.py b/python/tests/unit/arcticdb/version_store/test_recursive_normalizers.py index 856bb6dd2a5..b1532a5d4cf 100644 --- a/python/tests/unit/arcticdb/version_store/test_recursive_normalizers.py +++ b/python/tests/unit/arcticdb/version_store/test_recursive_normalizers.py @@ -14,7 +14,7 @@ from arcticc.pb2.descriptors_pb2 import NormalizationMetadata # Importing from arcticdb dynamically loads arcticc.pb2 from arcticdb.exceptions import ArcticDbNotYetImplemented from arcticdb.util.venv import CompatLibrary -from arcticdb.util.test import assert_frame_equal +from arcticdb.util.test import assert_frame_equal, assert_vit_equals_except_data from arcticdb.exceptions import ( DataTooNestedException, UnsupportedKeyInDictionary, @@ -23,7 +23,6 @@ from arcticdb.version_store.library import ArcticUnsupportedDataTypeException from arcticdb_ext.storage import KeyType, ModifiableLibraryOption from arcticdb_ext.version_store import NoSuchVersionException -import arcticdb_ext.stream as adb_stream import arcticdb_ext from tests.util.mark import MACOS_WHEEL_BUILD @@ -45,19 +44,6 @@ def denormalize(self, item, norm_meta): return AlmostAList(item) -def assert_vit_equals_except_data(left, right): - """ - Checks if two VersionedItem objects are equal disregarding differences in the data field. This is because when a - write is performed the returned VersionedItem does not contain a data field. - """ - assert left.symbol == right.symbol - assert left.library == right.library - assert left.version == right.version - assert left.metadata == right.metadata - assert left.host == right.host - assert left.timestamp == right.timestamp - - @pytest.mark.parametrize("staged", (True, False, None)) @pytest.mark.parametrize("lib_option", (True, False, None)) @pytest.mark.parametrize("recursive_normalizers", (True, False, None)) From 0687e03406e046e68d8f25e189a483fbce49a646 Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Thu, 27 Nov 2025 01:57:01 +0200 Subject: [PATCH 34/46] vcpkg --- cpp/vcpkg | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/vcpkg b/cpp/vcpkg index 2dc54dfc5af..7220a4eebf2 160000 --- a/cpp/vcpkg +++ b/cpp/vcpkg @@ -1 +1 @@ -Subproject commit 2dc54dfc5af9fb973860a38e4245ae0063740988 +Subproject commit 7220a4eebf20515cdd9c34721e15ca082bae9038 From 29a28e5b5d4b3841c4126c8186425704309488e8 Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Fri, 28 Nov 2025 00:05:17 +0200 Subject: [PATCH 35/46] Fix custom formatters for gtest --- cpp/arcticdb/CMakeLists.txt | 6 ++++ cpp/arcticdb/column_store/memory_segment.hpp | 31 ++++++++++++++++++ cpp/arcticdb/column_store/segment_utils.hpp | 32 +------------------ .../processing/test/test_merge_update.cpp | 3 -- cpp/arcticdb/util/test/gtest_utils.hpp | 10 +++--- 5 files changed, 44 insertions(+), 38 deletions(-) diff --git a/cpp/arcticdb/CMakeLists.txt b/cpp/arcticdb/CMakeLists.txt index 28dce79e280..88821b22a39 100644 --- a/cpp/arcticdb/CMakeLists.txt +++ b/cpp/arcticdb/CMakeLists.txt @@ -1110,6 +1110,12 @@ if(${TEST}) ) endif() + # In order for the custom formatting to work the ODR must be satisfied and every TU that uses gtest must include + # the gtest_utils.hpp https://github.com/google/googletest/issues/1149 + target_compile_options(test_unit_arcticdb PRIVATE + $<$:/FIutil/test/gtest_utils.hpp> + $<$>:-include util/test/gtest_utils.hpp> + ) gtest_discover_tests(test_unit_arcticdb PROPERTIES DISCOVERY_TIMEOUT 60) set(benchmark_srcs diff --git a/cpp/arcticdb/column_store/memory_segment.hpp b/cpp/arcticdb/column_store/memory_segment.hpp index 05dfa37ddc4..d8a08c7384b 100644 --- a/cpp/arcticdb/column_store/memory_segment.hpp +++ b/cpp/arcticdb/column_store/memory_segment.hpp @@ -299,3 +299,34 @@ class SegmentInMemory { }; } // namespace arcticdb + +namespace fmt { +template<> +struct formatter { + template + constexpr auto parse(ParseContext& ctx) { + return ctx.begin(); + } + + template + constexpr auto format(const arcticdb::SegmentInMemory& segment, FormatContext& ctx) const { + const StreamDescriptor& desc = segment.descriptor(); + auto out = fmt::format_to(ctx.out(), "Segment\n"); + for (unsigned i = 0; i < desc.field_count(); ++i) { + out = fmt::format_to(out, "\nColumn[{}]: {}\n", i, desc.field(i)); + visit_field(desc.field(i), [&](auto tdt) { + using TDT = decltype(tdt); + arcticdb::ColumnData cd = segment.column_data(i); + for (auto it = cd.begin(); it != cd.end(); ++it) { + if constexpr (std::same_as) { + out = fmt::format_to(out, "{} ", i, int(*it)); + } else { + out = fmt::format_to(out, "{} ", i, *it); + } + } + }); + } + return out; + } +}; +} // namespace fmt \ No newline at end of file diff --git a/cpp/arcticdb/column_store/segment_utils.hpp b/cpp/arcticdb/column_store/segment_utils.hpp index 04f05e25e44..0d5dc16259c 100644 --- a/cpp/arcticdb/column_store/segment_utils.hpp +++ b/cpp/arcticdb/column_store/segment_utils.hpp @@ -266,34 +266,4 @@ slice_data_into_segments( ); return std::make_tuple(std::move(segments), std::move(col_ranges), std::move(row_ranges)); } -} // namespace arcticdb - -namespace fmt { -template<> -struct formatter { - template - constexpr auto parse(ParseContext& ctx) { - return ctx.begin(); - } - - template - constexpr auto format(const arcticdb::SegmentInMemory& segment, FormatContext& ctx) const { - const StreamDescriptor& desc = segment.descriptor(); - fmt::format_to(ctx.out(), "Segment\n"); - for (unsigned i = 0; i < desc.field_count(); ++i) { - fmt::format_to(ctx.out(), "\nColumn[{}]: {}\n", i, desc.field(i)); - visit_field(desc.field(i), [&](auto tdt) { - using TDT = decltype(tdt); - arcticdb::ColumnData cd = segment.column_data(i); - for (auto it = cd.begin(); it != cd.end(); ++it) { - if constexpr (std::same_as) { - fmt::format_to(ctx.out(), "{} ", i, int(*it)); - } else { - fmt::format_to(ctx.out(), "{} ", i, *it); - } - } - }); - } - } -}; -} // namespace fmt \ No newline at end of file +} // namespace arcticdb \ No newline at end of file diff --git a/cpp/arcticdb/processing/test/test_merge_update.cpp b/cpp/arcticdb/processing/test/test_merge_update.cpp index 9f0d4959697..f58ab711e6e 100644 --- a/cpp/arcticdb/processing/test/test_merge_update.cpp +++ b/cpp/arcticdb/processing/test/test_merge_update.cpp @@ -5,9 +5,6 @@ * As of the Change Date specified in that file, in accordance with the Business Source License, use of this software * will be governed by the Apache License, version 2.0. */ - -#include "util/ranges_from_future.hpp" - #include #include #include diff --git a/cpp/arcticdb/util/test/gtest_utils.hpp b/cpp/arcticdb/util/test/gtest_utils.hpp index 854c449fc9c..8e8a48ac0ab 100644 --- a/cpp/arcticdb/util/test/gtest_utils.hpp +++ b/cpp/arcticdb/util/test/gtest_utils.hpp @@ -10,13 +10,12 @@ #include #include +#include +#include #define MAKE_GTEST_FMT(our_type, fstr) \ namespace testing::internal { \ - template<> \ - inline void PrintTo(const our_type& val, ::std::ostream* os) { \ - fmt::print(*os, fstr, val); \ - } \ + inline void PrintTo(const our_type& val, ::std::ostream* os) { fmt::print(*os, fstr, val); } \ } // For the most common types, format them by default: @@ -24,6 +23,9 @@ MAKE_GTEST_FMT(arcticdb::entity::RefKey, "{}") MAKE_GTEST_FMT(arcticdb::entity::AtomKeyImpl, "{}") MAKE_GTEST_FMT(arcticdb::entity::VariantKey, "VariantKey({})") MAKE_GTEST_FMT(arcticdb::VariantId, "VariantId({})") +MAKE_GTEST_FMT(arcticdb::SegmentInMemory, "{}") +MAKE_GTEST_FMT(arcticdb::pipelines::RowRange, "{}") +MAKE_GTEST_FMT(arcticdb::pipelines::ColRange, "{}") // FUTURE (C++20): with capabilities, we can write a generic PrintTo that covers all fmt::format-able types that is // not ambiguous with the built-in From c88363d74ace7cebec20b32b515d3af429b97544 Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Fri, 28 Nov 2025 00:15:43 +0200 Subject: [PATCH 36/46] Split utils from formatters --- cpp/arcticdb/CMakeLists.txt | 8 +++-- .../storage/test/test_azure_storage.cpp | 2 -- cpp/arcticdb/storage/test/test_s3_storage.cpp | 2 +- .../util/test/gtest_custom_formatters.hpp | 30 +++++++++++++++++++ cpp/arcticdb/util/test/gtest_utils.hpp | 25 +--------------- .../util/test/test_reliable_storage_lock.cpp | 2 -- cpp/arcticdb/version/test/test_sparse.cpp | 2 +- .../version/test/test_symbol_list.cpp | 1 - .../version/test/test_version_map.cpp | 1 - 9 files changed, 38 insertions(+), 35 deletions(-) create mode 100644 cpp/arcticdb/util/test/gtest_custom_formatters.hpp diff --git a/cpp/arcticdb/CMakeLists.txt b/cpp/arcticdb/CMakeLists.txt index 88821b22a39..7fac16d7827 100644 --- a/cpp/arcticdb/CMakeLists.txt +++ b/cpp/arcticdb/CMakeLists.txt @@ -1027,6 +1027,8 @@ if(${TEST}) util/test/test_string_pool.cpp util/test/test_string_utils.cpp util/test/test_tracing_allocator.cpp + util/test/gtest_custom_formatters.hpp + util/test/gtest_utils.hpp version/test/test_append.cpp version/test/test_key_block.cpp version/test/test_sort_index.cpp @@ -1111,10 +1113,10 @@ if(${TEST}) endif() # In order for the custom formatting to work the ODR must be satisfied and every TU that uses gtest must include - # the gtest_utils.hpp https://github.com/google/googletest/issues/1149 + # the gtest_custom_formatters.hpp https://github.com/google/googletest/issues/1149 target_compile_options(test_unit_arcticdb PRIVATE - $<$:/FIutil/test/gtest_utils.hpp> - $<$>:-include util/test/gtest_utils.hpp> + $<$:/FIutil/test/gtest_custom_formatters.hpp> + $<$>:-include util/test/gtest_custom_formatters.hpp> ) gtest_discover_tests(test_unit_arcticdb PROPERTIES DISCOVERY_TIMEOUT 60) diff --git a/cpp/arcticdb/storage/test/test_azure_storage.cpp b/cpp/arcticdb/storage/test/test_azure_storage.cpp index e5382694e60..44c11936786 100644 --- a/cpp/arcticdb/storage/test/test_azure_storage.cpp +++ b/cpp/arcticdb/storage/test/test_azure_storage.cpp @@ -7,12 +7,10 @@ */ #include -#include #include #include #include -#include #include using namespace arcticdb; diff --git a/cpp/arcticdb/storage/test/test_s3_storage.cpp b/cpp/arcticdb/storage/test/test_s3_storage.cpp index 0079e511a77..1f43bac0bc2 100644 --- a/cpp/arcticdb/storage/test/test_s3_storage.cpp +++ b/cpp/arcticdb/storage/test/test_s3_storage.cpp @@ -7,7 +7,6 @@ */ #include -#include #include #include #include @@ -17,6 +16,7 @@ #include #include #include +#include #include diff --git a/cpp/arcticdb/util/test/gtest_custom_formatters.hpp b/cpp/arcticdb/util/test/gtest_custom_formatters.hpp new file mode 100644 index 00000000000..5bf06967a79 --- /dev/null +++ b/cpp/arcticdb/util/test/gtest_custom_formatters.hpp @@ -0,0 +1,30 @@ +/* Copyright 2023 Man Group Operations Limited + * + * Use of this software is governed by the Business Source License 1.1 included in the file licenses/BSL.txt. + * + * As of the Change Date specified in that file, in accordance with the Business Source License, use of this software + * will be governed by the Apache License, version 2.0. + */ + +#pragma once + +#include +#include +#include +#include + +#define MAKE_GTEST_FMT(our_type, fstr) \ + namespace testing::internal { \ + inline void PrintTo(const our_type& val, ::std::ostream* os) { fmt::print(*os, fstr, val); } \ + } + +MAKE_GTEST_FMT(arcticdb::entity::RefKey, "{}") +MAKE_GTEST_FMT(arcticdb::entity::AtomKeyImpl, "{}") +MAKE_GTEST_FMT(arcticdb::entity::VariantKey, "VariantKey({})") +MAKE_GTEST_FMT(arcticdb::VariantId, "VariantId({})") +MAKE_GTEST_FMT(arcticdb::SegmentInMemory, "{}") +MAKE_GTEST_FMT(arcticdb::pipelines::RowRange, "{}") +MAKE_GTEST_FMT(arcticdb::pipelines::ColRange, "{}") + +// FUTURE (C++20): with capabilities, we can write a generic PrintTo that covers all fmt::format-able types that is +// not ambiguous with the built-in \ No newline at end of file diff --git a/cpp/arcticdb/util/test/gtest_utils.hpp b/cpp/arcticdb/util/test/gtest_utils.hpp index 8e8a48ac0ab..6a19d4654c0 100644 --- a/cpp/arcticdb/util/test/gtest_utils.hpp +++ b/cpp/arcticdb/util/test/gtest_utils.hpp @@ -1,35 +1,12 @@ -/* Copyright 2023 Man Group Operations Limited +/* Copyright 2025 Man Group Operations Limited * * Use of this software is governed by the Business Source License 1.1 included in the file licenses/BSL.txt. * * As of the Change Date specified in that file, in accordance with the Business Source License, use of this software * will be governed by the Apache License, version 2.0. */ - #pragma once -#include -#include -#include -#include - -#define MAKE_GTEST_FMT(our_type, fstr) \ - namespace testing::internal { \ - inline void PrintTo(const our_type& val, ::std::ostream* os) { fmt::print(*os, fstr, val); } \ - } - -// For the most common types, format them by default: -MAKE_GTEST_FMT(arcticdb::entity::RefKey, "{}") -MAKE_GTEST_FMT(arcticdb::entity::AtomKeyImpl, "{}") -MAKE_GTEST_FMT(arcticdb::entity::VariantKey, "VariantKey({})") -MAKE_GTEST_FMT(arcticdb::VariantId, "VariantId({})") -MAKE_GTEST_FMT(arcticdb::SegmentInMemory, "{}") -MAKE_GTEST_FMT(arcticdb::pipelines::RowRange, "{}") -MAKE_GTEST_FMT(arcticdb::pipelines::ColRange, "{}") - -// FUTURE (C++20): with capabilities, we can write a generic PrintTo that covers all fmt::format-able types that is -// not ambiguous with the built-in - // Macro to skip tests when running on Windows #ifdef _WIN32 #define SKIP_WIN(REASON) GTEST_SKIP() << "Skipping test on Windows, reason: " << '[' << #REASON << ']' diff --git a/cpp/arcticdb/util/test/test_reliable_storage_lock.cpp b/cpp/arcticdb/util/test/test_reliable_storage_lock.cpp index 7cacbb23162..66127a774c4 100644 --- a/cpp/arcticdb/util/test/test_reliable_storage_lock.cpp +++ b/cpp/arcticdb/util/test/test_reliable_storage_lock.cpp @@ -11,13 +11,11 @@ #include #include #include -#include #include #include #include #include -#include #include #include diff --git a/cpp/arcticdb/version/test/test_sparse.cpp b/cpp/arcticdb/version/test/test_sparse.cpp index 0ba27e456db..6c1f339ff21 100644 --- a/cpp/arcticdb/version/test/test_sparse.cpp +++ b/cpp/arcticdb/version/test/test_sparse.cpp @@ -7,7 +7,6 @@ */ #include -#include #include #include @@ -18,6 +17,7 @@ #include #include #include +#include struct SparseTestStore : arcticdb::TestStore { protected: diff --git a/cpp/arcticdb/version/test/test_symbol_list.cpp b/cpp/arcticdb/version/test/test_symbol_list.cpp index 24b0d432de9..80361121b3e 100644 --- a/cpp/arcticdb/version/test/test_symbol_list.cpp +++ b/cpp/arcticdb/version/test/test_symbol_list.cpp @@ -13,7 +13,6 @@ #include #include #include -#include #include #include diff --git a/cpp/arcticdb/version/test/test_version_map.cpp b/cpp/arcticdb/version/test/test_version_map.cpp index cded5a45543..3b203f4af08 100644 --- a/cpp/arcticdb/version/test/test_version_map.cpp +++ b/cpp/arcticdb/version/test/test_version_map.cpp @@ -15,7 +15,6 @@ #include #include #include -#include #include namespace arcticdb { From 967b70c3bde556009b7822ddbc246e43595e93ae Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Fri, 28 Nov 2025 00:45:16 +0200 Subject: [PATCH 37/46] Fix compilation issues on the CI --- cpp/arcticdb/column_store/memory_segment_impl.hpp | 2 -- cpp/arcticdb/version/local_versioned_engine.cpp | 2 +- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/cpp/arcticdb/column_store/memory_segment_impl.hpp b/cpp/arcticdb/column_store/memory_segment_impl.hpp index 77e0caa861a..916c5288102 100644 --- a/cpp/arcticdb/column_store/memory_segment_impl.hpp +++ b/cpp/arcticdb/column_store/memory_segment_impl.hpp @@ -16,8 +16,6 @@ #include #include #include -#include -#include namespace arcticdb { diff --git a/cpp/arcticdb/version/local_versioned_engine.cpp b/cpp/arcticdb/version/local_versioned_engine.cpp index 7e86f3ad55a..dcccb779289 100644 --- a/cpp/arcticdb/version/local_versioned_engine.cpp +++ b/cpp/arcticdb/version/local_versioned_engine.cpp @@ -2353,7 +2353,7 @@ VersionedItem LocalVersionedEngine::merge_internal( ); return VersionedItem{*std::move(update_info.previous_index_key_)}; } - constexpr static VersionQuery version_query; + const VersionQuery version_query; const ReadOptions read_options; const auto source_version = get_version_to_read(stream_id, version_query); const auto identifier = get_version_identifier(stream_id, version_query, read_options, source_version); From 51f8854de7d7fec771fcbe40648ddf18b2cba89b Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Fri, 28 Nov 2025 13:13:13 +0200 Subject: [PATCH 38/46] Fix unreachable code --- cpp/arcticdb/version/version_core.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/arcticdb/version/version_core.cpp b/cpp/arcticdb/version/version_core.cpp index 4602f2af7d3..100b76a2dcd 100644 --- a/cpp/arcticdb/version/version_core.cpp +++ b/cpp/arcticdb/version/version_core.cpp @@ -221,6 +221,7 @@ folly::Future merge_index_key_for_read_modify_write( merged_ranges_and_keys.push_back(std::move(slice)); } } + pipeline_context.slice_and_keys_.clear(); const size_t row_count = merged_ranges_and_keys.empty() ? 0 : merged_ranges_and_keys.back().slice().row_range.second - merged_ranges_and_keys[0].slice().row_range.first; @@ -240,7 +241,6 @@ folly::Future merge_index_key_for_read_modify_write( target_partial_index_key, store ); - pipeline_context.slice_and_keys_.clear(); } } // namespace From 07580e9156b6c43394c33a44d89f807b84b500e9 Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Fri, 28 Nov 2025 17:38:26 +0200 Subject: [PATCH 39/46] Fix windows build issues --- cpp/arcticdb/column_store/segment_utils.hpp | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/cpp/arcticdb/column_store/segment_utils.hpp b/cpp/arcticdb/column_store/segment_utils.hpp index 0d5dc16259c..14ba1e6946f 100644 --- a/cpp/arcticdb/column_store/segment_utils.hpp +++ b/cpp/arcticdb/column_store/segment_utils.hpp @@ -166,11 +166,11 @@ template< std::ranges::sized_range... RestCols> requires( util::instantiation_of && - [](std::index_sequence) { - return (std::ranges::sized_range> && ...); - }(std::make_index_sequence>{}) + [](std::tuple*) { + return (std::ranges::sized_range && ...); + }(static_cast(nullptr)) ) -void slice_data_into_segments( +void slice_data_into_segments_helper( const std::span descriptors, const size_t rows_per_segment, const size_t cols_per_segment, pipelines::ColRange col_range, std::vector& segments, std::vector& col_ranges, std::vector& row_ranges, @@ -206,7 +206,7 @@ void slice_data_into_segments( ); col_range.first = col_range.second; col_range.second = std::min(col_range.first + cols_per_segment, col_range.first + sizeof...(RestCols)); - slice_data_into_segments( + slice_data_into_segments_helper( descriptors.last(descriptors.size() - 1), rows_per_segment, cols_per_segment, @@ -219,7 +219,7 @@ void slice_data_into_segments( std::forward(data)... ); } else { - slice_data_into_segments( + slice_data_into_segments_helper( descriptors, rows_per_segment, cols_per_segment, @@ -247,7 +247,7 @@ slice_data_into_segments( auto [index_columns, data_columns] = split_pack(std::forward(data)...); std::apply( [&](Cols&&... cols) { - slice_data_into_segments( + slice_data_into_segments_helper( descriptors, rows_per_segment, cols_per_segment, From 546ccb7c56978db941b89573f0bcb1219eb970a0 Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Sat, 29 Nov 2025 02:25:37 +0200 Subject: [PATCH 40/46] Fix use after stack free --- cpp/arcticdb/pipeline/input_frame_utils.hpp | 30 +++++---------------- 1 file changed, 6 insertions(+), 24 deletions(-) diff --git a/cpp/arcticdb/pipeline/input_frame_utils.hpp b/cpp/arcticdb/pipeline/input_frame_utils.hpp index 75666cf643b..b1590326ed9 100644 --- a/cpp/arcticdb/pipeline/input_frame_utils.hpp +++ b/cpp/arcticdb/pipeline/input_frame_utils.hpp @@ -10,34 +10,16 @@ #include namespace arcticdb::pipelines { -template -requires std::ranges::sized_range && (std::ranges::sized_range && ...) -auto materialize_ranges(H&& head, T&&... tail) { - if constexpr (sizeof...(T) == 0) { - if constexpr (!std::ranges::contiguous_range) { - return std::forward_as_tuple(std::vector>(std::forward(head))); - } else { - return std::forward_as_tuple(std::forward(head)); - } - } else { - if constexpr (!std::ranges::contiguous_range) { - return std::tuple_cat( - std::forward_as_tuple(std::vector>(std::forward(head))), - materialize_ranges(std::forward(tail)...) - ); - } else { - return std::tuple_cat( - std::forward_as_tuple(std::forward(head)), materialize_ranges(std::forward(tail)...) - ); - } - } -} - template requires((Index::field_count() == 0 || Index::field_count() == 1) && (std::ranges::sized_range && ...)) auto input_frame_from_tensors(const StreamDescriptor& desc, T&&... input) { constexpr static size_t data_columns = sizeof...(T) - Index::field_count(); - auto materialized_input = materialize_ranges(std::forward(input)...); + std::tuple materialized_input{std::vector, bool>, + uint8_t, + std::ranges::range_value_t>>( + std::make_move_iterator(std::begin(input)), std::make_move_iterator(std::end(input)) + )...}; [&](std::index_sequence) { const size_t first_row_count = std::get<0>(materialized_input).size(); util::check( From 336bae964386f5d5f80a8019a972891a7b1f733e Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Mon, 1 Dec 2025 13:34:52 +0200 Subject: [PATCH 41/46] Fix out ouf bounds access --- cpp/arcticdb/processing/clause.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/cpp/arcticdb/processing/clause.cpp b/cpp/arcticdb/processing/clause.cpp index 48376b3a325..9654dd3c48d 100644 --- a/cpp/arcticdb/processing/clause.cpp +++ b/cpp/arcticdb/processing/clause.cpp @@ -2058,17 +2058,17 @@ std::vector> MergeUpdateClause::filter_index_match(const Pro while (target_index_it != target_index_end && source_row < source_row_end) { const timestamp source_ts = source_->index_value_at(source_row); // TODO: Profile and compare to linear or adaptive (linear below some threshold) search - auto lower_bound = std::lower_bound(target_index_it, target_index_end, source_ts); - if (lower_bound == target_index_end) { + auto target_match_it = std::lower_bound(target_index_it, target_index_end, source_ts); + if (target_match_it == target_index_end) { break; } - target_row += std::distance(target_index_it, lower_bound); - while (*lower_bound == source_ts) { + target_row += std::distance(target_index_it, target_match_it); + while (target_match_it != target_index_end && *target_match_it == source_ts) { matched_rows[source_row - source_row_start].push_back(target_row); - ++lower_bound; + ++target_match_it; ++target_row; } - target_index_it = lower_bound; + target_index_it = target_match_it; source_row++; } return matched_rows; From c5f0516ae173b60f114085ea9b338cde31e62277 Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Mon, 1 Dec 2025 14:13:45 +0200 Subject: [PATCH 42/46] Fix column selection when the source is SegmetInMemory --- cpp/arcticdb/processing/clause.cpp | 4 ++-- python/test.sh | 21 +++++++++++++++++++++ 2 files changed, 23 insertions(+), 2 deletions(-) create mode 100644 python/test.sh diff --git a/cpp/arcticdb/processing/clause.cpp b/cpp/arcticdb/processing/clause.cpp index 9654dd3c48d..17ca9b5bcde 100644 --- a/cpp/arcticdb/processing/clause.cpp +++ b/cpp/arcticdb/processing/clause.cpp @@ -1993,9 +1993,9 @@ void MergeUpdateClause::update_and_insert( auto target_data_it = target_segments[segment_idx]->column(column_idx).data().begin(); // TODO: Implement operator+= because std::advance is linear std::advance(target_data_it, target_data_row); - const size_t source_column = [&]() { + const size_t source_column = [&] { if constexpr (std::same_as) { - return col_ranges[segment_idx]->first; + return col_ranges[segment_idx]->first + column_idx; } else if constexpr (std::same_as>) { // If the source frame is a list of tensors, the index frame is kept separately, so the first // non-index column will be at index 0. If there's an index the first ColRange will start from diff --git a/python/test.sh b/python/test.sh new file mode 100644 index 00000000000..fdbd35cfd8c --- /dev/null +++ b/python/test.sh @@ -0,0 +1,21 @@ +MAX_RUNS=1000 +export LD_PRELOAD=/usr/local/lib/clang/19/lib/x86_64-unknown-linux-gnu/libclang_rt.asan.so +export ASAN_OPTIONS=detect_leaks=0 +TEST_CMD="python -m pytest tests/unit/arcticdb/version_store/test_merge.py::TestMergeTimeseries::test_merge_update_row_from_source_matches_multiple_rows_from_target_in_separate_slices" + +echo "Running test up to $MAX_RUNS times or until ASan failure..." + +for i in $(seq 1 $MAX_RUNS); do + echo "=== Run $i/$MAX_RUNS ===" + $TEST_CMD + EXIT_CODE=$? + + if [ $EXIT_CODE -ne 0 ]; then + echo "Test failed on run $i with exit code $EXIT_CODE" + exit $EXIT_CODE + fi + + echo "Run $i completed successfully" +done + +echo "All $MAX_RUNS runs completed without ASan failures" From ba437a8928405c4d3ba017669e065ff6813d871d Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Wed, 3 Dec 2025 14:02:30 +0200 Subject: [PATCH 43/46] Python tests for string column pass --- cpp/arcticdb/column_store/string_pool.cpp | 9 - cpp/arcticdb/column_store/string_pool.hpp | 4 +- cpp/arcticdb/pipeline/frame_utils.hpp | 67 ++++--- cpp/arcticdb/processing/clause.cpp | 165 ++++++++++++++---- cpp/arcticdb/util/type_traits.hpp | 2 + .../unit/arcticdb/version_store/test_merge.py | 32 +++- 6 files changed, 194 insertions(+), 85 deletions(-) diff --git a/cpp/arcticdb/column_store/string_pool.cpp b/cpp/arcticdb/column_store/string_pool.cpp index 738f9d2b14d..21a5ab76bb6 100644 --- a/cpp/arcticdb/column_store/string_pool.cpp +++ b/cpp/arcticdb/column_store/string_pool.cpp @@ -78,15 +78,6 @@ std::shared_ptr StringPool::clone() const { return output; } -StringPool& StringPool::operator=(StringPool&& that) noexcept { - if (this != &that) { - block_ = std::move(that.block_); - map_ = std::move(that.map_); - shapes_ = std::move(that.shapes_); - } - return *this; -} - ColumnData StringPool::column_data() const { return {&block_.buffer(), &shapes_.buffer(), string_pool_descriptor().type(), nullptr}; } diff --git a/cpp/arcticdb/column_store/string_pool.hpp b/cpp/arcticdb/column_store/string_pool.hpp index c346637d2a8..9fd42d8ad95 100644 --- a/cpp/arcticdb/column_store/string_pool.hpp +++ b/cpp/arcticdb/column_store/string_pool.hpp @@ -133,11 +133,11 @@ class StringPool { ~StringPool() = default; StringPool& operator=(const StringPool&) = delete; StringPool(const StringPool&) = delete; - StringPool(StringPool&& that) = delete; + StringPool(StringPool&& that) noexcept = default; std::shared_ptr clone() const; - StringPool& operator=(StringPool&& that) noexcept; + StringPool& operator=(StringPool&& that) noexcept = default; ColumnData column_data() const; diff --git a/cpp/arcticdb/pipeline/frame_utils.hpp b/cpp/arcticdb/pipeline/frame_utils.hpp index 2f4f110b96c..72790de4bd4 100644 --- a/cpp/arcticdb/pipeline/frame_utils.hpp +++ b/cpp/arcticdb/pipeline/frame_utils.hpp @@ -89,6 +89,37 @@ RawType* flatten_tensor( return reinterpret_cast(flattened_buffer->data()); } +template +std::optional read_py_string_to_buffer( + PyObject* py_string_object, position_t& out, std::optional& scoped_gil_lock, StringPool& pool +) { + if (is_py_none(py_string_object)) { + out = not_a_string(); + return std::nullopt; + } else if (is_py_nan(py_string_object)) { + out = nan_placeholder(); + return std::nullopt; + } else { + std::variant wrapper_or_error; + if constexpr (is_utf_type(slice_value_type(dt))) { + wrapper_or_error = convert::py_unicode_to_buffer(py_string_object, scoped_gil_lock); + } else { + wrapper_or_error = convert::pystring_to_buffer(py_string_object, false); + } + // Cannot use util::variant_match as only one of the branches would have a return type + if (std::holds_alternative(wrapper_or_error)) { + const convert::PyStringWrapper wrapper(std::move(std::get(wrapper_or_error))); + const auto offset = pool.get(wrapper.buffer_, wrapper.length_); + out = offset.offset(); + return std::nullopt; + } else if (std::holds_alternative(wrapper_or_error)) { + return std::get(std::move(wrapper_or_error)); + } else { + internal::raise("Unexpected variant alternative"); + } + } +} + template std::optional set_sequence_type( AggregatorType& agg, const entity::NativeTensor& tensor, size_t col, size_t rows_to_write, size_t row, @@ -99,25 +130,23 @@ std::optional set_sequence_type( std::optional flattened_buffer; ARCTICDB_SAMPLE_DEFAULT(SetDataString) + const auto data = const_cast(tensor.data()); if (is_fixed_string_type(dt)) { // deduplicate the strings auto str_stride = tensor.strides(0); - auto data = const_cast(tensor.data()); - auto char_data = reinterpret_cast(data) + row * str_stride; + auto char_data = static_cast(data) + row * str_stride; auto str_len = tensor.elsize(); for (size_t s = 0; s < rows_to_write; ++s, char_data += str_stride) { agg.set_string_at(col, s, char_data, str_len); } } else { - auto data = const_cast(tensor.data()); - auto ptr_data = reinterpret_cast(data); + auto ptr_data = static_cast(data); ptr_data += row; if (!c_style) ptr_data = flatten_tensor(flattened_buffer, rows_to_write, tensor, slice_num, regular_slice_size); - std::variant wrapper_or_error; // GIL will be acquired if there is a string that is not pure ASCII/UTF-8 // In this case a PyObject will be allocated by convert::py_unicode_to_buffer // If such a string is encountered in a column, then the GIL will be held until that whole column has @@ -128,29 +157,13 @@ std::optional set_sequence_type( auto out_ptr = reinterpret_cast(column.buffer().data()); auto& string_pool = agg.segment().string_pool(); for (size_t s = 0; s < rows_to_write; ++s, ++ptr_data) { - if (is_py_none(*ptr_data)) { - *out_ptr++ = not_a_string(); - } else if (is_py_nan(*ptr_data)) { - *out_ptr++ = nan_placeholder(); - } else { - if constexpr (is_utf_type(slice_value_type(dt))) { - wrapper_or_error = convert::py_unicode_to_buffer(*ptr_data, scoped_gil_lock); - } else { - wrapper_or_error = convert::pystring_to_buffer(*ptr_data, false); - } - // Cannot use util::variant_match as only one of the branches would have a return type - if (std::holds_alternative(wrapper_or_error)) { - convert::PyStringWrapper wrapper(std::move(std::get(wrapper_or_error))); - const auto offset = string_pool.get(wrapper.buffer_, wrapper.length_); - *out_ptr++ = offset.offset(); - } else if (std::holds_alternative(wrapper_or_error)) { - auto error = std::get(wrapper_or_error); - error.row_index_in_slice_ = s; - return std::optional(error); - } else { - internal::raise("Unexpected variant alternative"); - } + std::optional maybe_error = + read_py_string_to_buffer
(*ptr_data, *out_ptr, scoped_gil_lock, string_pool); + if (maybe_error.has_value()) { + maybe_error->row_index_in_slice_ = s; + return maybe_error; } + ++out_ptr; } } return std::optional{}; diff --git a/cpp/arcticdb/processing/clause.cpp b/cpp/arcticdb/processing/clause.cpp index 17ca9b5bcde..9100ccc3bab 100644 --- a/cpp/arcticdb/processing/clause.cpp +++ b/cpp/arcticdb/processing/clause.cpp @@ -27,6 +27,7 @@ #include #include #include +#include #include @@ -76,6 +77,24 @@ void filter_selected_ranges_and_keys_and_reindex_entities( } ranges_and_keys.erase(ranges_and_keys.begin() + new_entity_id, ranges_and_keys.end()); } + +template +requires arcticdb::util::any_of> +size_t source_column_index( + const arcticdb::ColRange& col_range, const size_t index_in_range, + [[maybe_unused]] const uint32_t index_field_count +) { + if constexpr (std::same_as) { + return col_range.first + index_in_range; + } else if constexpr (std::same_as>) { + // If the source frame is a list of tensors, the index frame is kept separately, so the first + // non-index column will be at index 0. If there's an index the first ColRange will start from + // 1 and the first column of each segment will contain the index + return col_range.first + index_in_range - 2 * index_field_count; + } else { + static_assert(sizeof(Source) == 0, "Invalid source type"); + } +} } // namespace namespace arcticdb { @@ -1962,6 +1981,9 @@ void MergeUpdateClause::update_and_insert( const T& source, const StreamDescriptor& source_descriptor, const ProcessingUnit& proc, const std::span> rows_to_update ) const { + user_input::check( + std::same_as>, "Arrow format is not supported for merge updates yet" + ); const std::span> target_segments = *proc.segments_; const std::span> row_ranges = *proc.row_ranges_; const std::span> col_ranges = *proc.col_ranges_; @@ -1969,15 +1991,19 @@ void MergeUpdateClause::update_and_insert( // Update one column at a time to increase cache coherency and to avoid calling visit_field for each row being // updated for (size_t segment_idx = 0; segment_idx < target_segments.size(); ++segment_idx) { - const size_t columns_in_range = target_segments[segment_idx]->num_columns(); + // TODO: Implement equivalent QueryBuilder.optimise_for_speed. Initial implementation will always drop the + // current string pool and recreate it from scratch taking account the new string data from source. This is + // equivalent to QueryBuilder.optimise_for_memory. + StringPool new_string_pool; + bool segment_contains_string_column = false; + SegmentInMemory& target_segment = *target_segments[segment_idx]; + const size_t columns_in_range = target_segment.num_columns(); for (size_t column_idx = source_descriptor.index().field_count(); column_idx < columns_in_range; ++column_idx) { - entity::visit_field(target_segments[segment_idx]->descriptor().field(column_idx), [&](auto tdt) { - internal::check( - !is_sequence_type(tdt.data_type()), "String columns are not supported in merge update yet" - ); + const Field& target_field = target_segment.descriptor().field(column_idx); + entity::visit_field(target_field, [&](auto tdt) { using TDT = decltype(tdt); - size_t source_row = source_row_start; + // TODO: Handle insert while (source_row < source_row_end && rows_to_update[source_row - source_row_start].empty()) { ++source_row; }; @@ -1988,44 +2014,107 @@ void MergeUpdateClause::update_and_insert( if (source_row >= source_row_end) { return; } - size_t target_data_row = rows_to_update[source_row - source_row_start].front(); - - auto target_data_it = target_segments[segment_idx]->column(column_idx).data().begin(); - // TODO: Implement operator+= because std::advance is linear - std::advance(target_data_it, target_data_row); - const size_t source_column = [&] { - if constexpr (std::same_as) { - return col_ranges[segment_idx]->first + column_idx; - } else if constexpr (std::same_as>) { - // If the source frame is a list of tensors, the index frame is kept separately, so the first - // non-index column will be at index 0. If there's an index the first ColRange will start from - // 1 and the first column of each segment will contain the index - return col_ranges[segment_idx]->first + column_idx - - 2 * source_descriptor.index().field_count(); - } else { - static_assert(sizeof(T) == 0, "Invalid type"); + Column& target_column = target_segment.column(column_idx); + auto target_data_it = target_column.data().begin(); + const size_t source_column_position = source_column_index( + *col_ranges[segment_idx], column_idx, source_descriptor.index().field_count() + ); + SourceView source_column_view = get_source_column_iterator(source, source_column_position); + source_column_view.set_row(source_row); + if constexpr (is_sequence_type(tdt.data_type())) { + if constexpr (is_fixed_string_type(tdt.data_type())) { + user_input::raise( + "Fixed string sequences are not supported for merge update" + ); + } else if constexpr (is_dynamic_string_type(tdt.data_type())) { + segment_contains_string_column = true; + std::optional gil_lock; + // TODO: size must take into account rows that are going to be inserted + Column new_string_column( + target_column.type(), + target_column.row_count(), + AllocationType::PRESIZED, + target_column.is_sparse() ? Sparsity::PERMITTED : Sparsity::NOT_PERMITTED + ); + ColumnData new_column_data = new_string_column.data(); + auto new_column_data_it = new_column_data.begin(); + auto target_row_to_update_it = + std::make_optional(rows_to_update[source_row - source_row_start].cbegin()); + arcticdb::for_each_enumerated(target_column, [&](auto row) { + const bool current_target_row_is_matched = + target_row_to_update_it.has_value() && + static_cast(row.idx()) == *(*target_row_to_update_it); + if (current_target_row_is_matched) { + position_t new_value; + const auto py_string_object = reinterpret_cast(*source_column_view); + std::optional maybe_error = + read_py_string_to_buffer( + py_string_object, new_value, gil_lock, new_string_pool + ); + if (maybe_error) { + maybe_error->row_index_in_slice_ = row.idx(); + maybe_error->raise(target_field.name(), row_ranges[segment_idx]->first); + } + *new_column_data_it = new_value; + // All target rows that match the current source row are updated. Find the next source + // row which matches something in the target. Since both the source and the target are + // ordered, it's guaranteed that the next matched target row will be larger than the + // current target row. + if (++(*target_row_to_update_it) == + rows_to_update[source_row - source_row_start].cend()) { + do { + ++source_row; + } while (source_row < source_row_end && + rows_to_update[source_row - source_row_start].empty()); + if (source_row < source_row_end) { + target_row_to_update_it = + rows_to_update[source_row - source_row_start].cbegin(); + source_column_view.set_row(source_row); + } else { + target_row_to_update_it = std::nullopt; + } + } + } else { + if (is_a_string(row.value())) { + const std::string_view string_in_target = + target_segment.const_string_pool().get_const_view(row.value()); + const OffsetString offset_in_new_pool = new_string_pool.get(string_in_target); + *new_column_data_it = offset_in_new_pool.offset(); + } else { + *new_column_data_it = row.value(); + } + } + ++new_column_data_it; + }); + target_segment.column(column_idx) = std::move(new_string_column); } - }(); - SourceView source_column_view = get_source_column_iterator(source, source_column); - while (source_row < source_row_end) { - std::span rows_to_update_for_source_row = - rows_to_update[source_row - source_row_start]; - if (rows_to_update_for_source_row.empty()) { + } else { + auto target_data_row_to_update = rows_to_update[source_row - source_row_start].front(); + // TODO: Implement operator+= because std::advance is linear + std::advance(target_data_it, target_data_row_to_update); + while (source_row < source_row_end) { + std::span rows_to_update_for_source_row = + rows_to_update[source_row - source_row_start]; + if (rows_to_update_for_source_row.empty()) { + ++source_row; + continue; + } + source_column_view.set_row(source_row); + const auto& source_row_value = *source_column_view; + for (const size_t target_row_to_update : rows_to_update_for_source_row) { + // TODO: Implement operator+= because std::advance is linear + std::advance(target_data_it, target_row_to_update - target_data_row_to_update); + *target_data_it = source_row_value; + target_data_row_to_update = target_row_to_update; + } ++source_row; - continue; } - source_column_view.set_row(source_row); - const auto& source_row_value = *source_column_view; - for (const size_t target_row_to_update : rows_to_update_for_source_row) { - // TODO: Implement operator+= because std::advance is linear - std::advance(target_data_it, target_row_to_update - target_data_row); - *target_data_it = source_row_value; - target_data_row = target_row_to_update; - } - ++source_row; } }); } + if (segment_contains_string_column) { + target_segment.set_string_pool(std::make_shared(std::move(new_string_pool))); + } } } template void MergeUpdateClause:: diff --git a/cpp/arcticdb/util/type_traits.hpp b/cpp/arcticdb/util/type_traits.hpp index 8e63f31e8b3..58edb99f436 100644 --- a/cpp/arcticdb/util/type_traits.hpp +++ b/cpp/arcticdb/util/type_traits.hpp @@ -31,4 +31,6 @@ concept any_of = std::disjunction_v...>; template concept decays_to = std::same_as, Base>; +template +concept invocable_r = std::is_invocable_r_v; } // namespace arcticdb::util diff --git a/python/tests/unit/arcticdb/version_store/test_merge.py b/python/tests/unit/arcticdb/version_store/test_merge.py index 887138a77ac..9122a2fe3c0 100644 --- a/python/tests/unit/arcticdb/version_store/test_merge.py +++ b/python/tests/unit/arcticdb/version_store/test_merge.py @@ -204,11 +204,13 @@ def test_throws_if_source_is_not_sorted(self, lmdb_library, strategy): def test_merge_update(self, lmdb_library, strategy): lib = lmdb_library - target = pd.DataFrame({"a": [1, 2, 3], "b": [1.0, 2.0, 3.0]}, index=pd.date_range("2024-01-01", periods=3)) + target = pd.DataFrame( + {"a": [1, 2, 3], "b": [1.0, 2.0, 3.0], "c": ["a", "b", "c"]}, index=pd.date_range("2024-01-01", periods=3) + ) write_vit = lib.write("sym", target) source = pd.DataFrame( - {"a": [4, 5, 6], "b": [7.0, 8.0, 9.0]}, + {"a": [4, 5, 6], "b": [7.0, 8.0, 9.0], "c": ["A", "B", "C"]}, # Only the second row: "2024-01-02" matches index=pd.DatetimeIndex(["2024-01-01 10:00:00", "2024-01-02", "2024-01-04"]), ) @@ -223,7 +225,9 @@ def test_merge_update(self, lmdb_library, strategy): assert merge_vit.data is None # Only the second row: "2024-01-02" is updated - expected = pd.DataFrame({"a": [1, 5, 3], "b": [1.0, 8.0, 3.0]}, index=pd.date_range("2024-01-01", periods=3)) + expected = pd.DataFrame( + {"a": [1, 5, 3], "b": [1.0, 8.0, 3.0], "c": ["a", "B", "c"]}, index=pd.date_range("2024-01-01", periods=3) + ) read_vit = lib.read("sym") assert_vit_equals_except_data(merge_vit, read_vit) @@ -265,19 +269,29 @@ def test_merge_update_writes_new_version_even_if_nothing_is_changed(self, lmdb_l def test_merge_update_row_slicing(self, lmdb_library_factory, slicing_policy): lib = lmdb_library_factory(arcticdb.LibraryOptions(**slicing_policy)) target = pd.DataFrame( - {"a": [1, 2, 3, 4, 5], "b": [1.0, 2.0, 3.0, 4.0, 5.0], "c": [True, False, True, False, True]}, + { + "a": [1, 2, 3, 4, 5], + "b": [1.0, 2.0, 3.0, 4.0, 5.0], + "c": [True, False, True, False, True], + "d": ["a", "b", "c", "d", "e"], + }, index=pd.date_range("2024-01-01", periods=5), ) lib.write("sym", target) source = pd.DataFrame( - {"a": [30, 50], "b": [30.1, 50.1], "c": [False, False]}, + {"a": [30, 50], "b": [30.1, 50.1], "c": [False, False], "d": ["C", "E"]}, index=pd.DatetimeIndex([pd.Timestamp("2024-01-03"), pd.Timestamp("2024-01-05")]), ) lib.merge("sym", source, strategy=MergeStrategy(not_matched_by_target=MergeAction.DO_NOTHING)) expected = pd.DataFrame( - {"a": [1, 2, 30, 4, 50], "b": [1.0, 2.0, 30.1, 4.0, 50.1], "c": [True, False, False, False, False]}, + { + "a": [1, 2, 30, 4, 50], + "b": [1.0, 2.0, 30.1, 4.0, 50.1], + "c": [True, False, False, False, False], + "d": ["a", "b", "C", "d", "E"], + }, index=pd.date_range("2024-01-01", periods=5), ) received = lib.read("sym").data @@ -406,16 +420,16 @@ def test_merge_update_row_from_source_matches_multiple_rows_from_target_in_separ ): lib = lmdb_library_factory(arcticdb.LibraryOptions(rows_per_segment=2)) target = pd.DataFrame( - {"a": [1, 2, 3], "b": [1.0, 2.0, 3.0]}, + {"a": [1, 2, 3], "b": [1.0, 2.0, 3.0], "c": ["a", "b", "c"]}, index=pd.DatetimeIndex( [pd.Timestamp("2024-01-01"), pd.Timestamp("2024-01-02"), pd.Timestamp("2024-01-02")] ), ) lib.write("sym", target) - source = pd.DataFrame({"a": [5], "b": [20.0]}, index=pd.DatetimeIndex([pd.Timestamp("2024-01-02")])) + source = pd.DataFrame({"a": [5], "b": [20.0], "c": ["B"]}, index=pd.DatetimeIndex([pd.Timestamp("2024-01-02")])) lib.merge("sym", source, strategy=MergeStrategy(not_matched_by_target=MergeAction.DO_NOTHING)) expected = pd.DataFrame( - {"a": [1, 5, 5], "b": [1.0, 20.0, 20.0]}, + {"a": [1, 5, 5], "b": [1.0, 20.0, 20.0], "c": ["a", "B", "B"]}, index=pd.DatetimeIndex( [pd.Timestamp("2024-01-01"), pd.Timestamp("2024-01-02"), pd.Timestamp("2024-01-02")] ), From 84310699ff63b7303a009cc9292221a5d002817a Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Wed, 3 Dec 2025 18:31:59 +0200 Subject: [PATCH 44/46] Refactor --- cpp/arcticdb/processing/clause.cpp | 319 ++++++++++++++++------------- 1 file changed, 179 insertions(+), 140 deletions(-) diff --git a/cpp/arcticdb/processing/clause.cpp b/cpp/arcticdb/processing/clause.cpp index 9100ccc3bab..42697a8c538 100644 --- a/cpp/arcticdb/processing/clause.cpp +++ b/cpp/arcticdb/processing/clause.cpp @@ -32,6 +32,7 @@ #include namespace { +using namespace arcticdb; /// Removes inplace all elements from vec whose indexes are not in indexes_to_keep. /// @param indexes_to_keep Must be sorted template @@ -78,15 +79,79 @@ void filter_selected_ranges_and_keys_and_reindex_entities( ranges_and_keys.erase(ranges_and_keys.begin() + new_entity_id, ranges_and_keys.end()); } +/// Helper class used to iterate over elements inside InputFrame uniformly. +/// +/// It has two template specialisations for each way an InputFrame can store its data. It's supposed to be used only to +/// perform forward iteration. +template +class SourceView {}; + +template +requires(std::same_as && util::instantiation_of) +class SourceView { + using Iterator = decltype(std::declval().data().template begin()); + + public: + SourceView(const T& source, size_t source_data_row) : + it_(source.data().template begin()), + source_data_row_(source_data_row) { + std::advance(it_, source_data_row); + } + + void set_row(const size_t new_row) { + debug::check( + new_row >= source_data_row_, "Cannot move SourceColumnIterator backwards" + ); + // TODO: Implement operator+= because std::advance is linear + std::advance(it_, new_row - source_data_row_); + source_data_row_ = new_row; + } + + size_t row() const { return source_data_row_; } + + const TDT::DataTypeTag::raw_type& operator*() const { return *it_; } + + private: + Iterator it_; + size_t source_data_row_; +}; + +template +requires std::same_as && util::instantiation_of +class SourceView { + using Iterator = const TDT::DataTypeTag::raw_type*; + + public: + SourceView(const T& source, const size_t source_data_row) : source_(source), source_data_row_(source_data_row) {} + void set_row(const size_t new_row) { source_data_row_ = new_row; } + const TDT::DataTypeTag::raw_type& operator*() const { return source_.at(source_data_row_); } + size_t row() const { return source_data_row_; } + + private: + TypedTensor source_; + size_t source_data_row_; +}; + +template +requires util::any_of> && util::instantiation_of +auto get_source_column_iterator(const T& source, size_t column_index) { + if constexpr (std::is_same_v) { + return SourceView(source.column(column_index), 0); + } else if constexpr (std::is_same_v>) { + return SourceView(source[column_index], 0); + } else { + static_assert(sizeof(T) == 0, "Invalid type"); + } +} + template -requires arcticdb::util::any_of> +requires arcticdb::util::any_of> size_t source_column_index( - const arcticdb::ColRange& col_range, const size_t index_in_range, - [[maybe_unused]] const uint32_t index_field_count + const ColRange& col_range, const size_t index_in_range, [[maybe_unused]] const uint32_t index_field_count ) { if constexpr (std::same_as) { return col_range.first + index_in_range; - } else if constexpr (std::same_as>) { + } else if constexpr (std::same_as>) { // If the source frame is a list of tensors, the index frame is kept separately, so the first // non-index column will be at index 0. If there's an index the first ColRange will start from // 1 and the first column of each segment will contain the index @@ -95,6 +160,83 @@ size_t source_column_index( static_assert(sizeof(Source) == 0, "Invalid source type"); } } + +template +requires util::instantiation_of +position_t write_py_string_to_pool_or_throw( + PyObject* const py_string_object, const size_t row_in_segment, const RowRange& row_range, + std::optional& gil_lock, StringPool& new_string_pool, const std::string_view column_name +) { + position_t new_value; + std::optional maybe_error = + read_py_string_to_buffer(py_string_object, new_value, gil_lock, new_string_pool); + if (maybe_error) { + maybe_error->row_index_in_slice_ = row_in_segment; + maybe_error->raise(column_name, row_range.first); + } + return new_value; +} + +template +requires(util::instantiation_of && util::any_of) +Column merge_update_string_column( + const Column& target_column, const std::span> rows_to_update, + const Field& target_field, const RowRange& row_range, const size_t source_row_start, + const size_t source_row_end, const StringPool& target_string_pool, StringPool& new_string_pool, + SourceView& source_column_view +) { + if constexpr (is_fixed_string_type(TDT::data_type())) { + user_input::raise( + "Fixed string sequences are not supported for merge update" + ); + } else if constexpr (is_dynamic_string_type(TDT::data_type())) { + std::optional gil_lock; + // TODO: size must take into account rows that are going to be inserted + Column new_string_column( + target_column.type(), + target_column.row_count(), + AllocationType::PRESIZED, + target_column.is_sparse() ? Sparsity::PERMITTED : Sparsity::NOT_PERMITTED + ); + size_t source_row = source_column_view.row(); + ColumnData new_column_data = new_string_column.data(); + auto new_column_data_it = new_column_data.begin(); + auto target_row_to_update_it = rows_to_update[source_row - source_row_start].cbegin(); + arcticdb::for_each_enumerated(target_column, [&](auto row) { + const bool current_target_row_is_matched = + source_row < source_row_end && static_cast(row.idx()) == *target_row_to_update_it; + if (current_target_row_is_matched) { + const auto py_string_object = reinterpret_cast(*source_column_view); + *new_column_data_it = write_py_string_to_pool_or_throw( + py_string_object, row.idx(), row_range, gil_lock, new_string_pool, target_field.name() + ); + // All target rows that match the current source row are updated. Find the next source + // row which matches something in the target. Since both the source and the target are + // ordered, it's guaranteed that the next matched target row will be larger than the + // current target row. + if (++target_row_to_update_it == rows_to_update[source_row - source_row_start].cend()) { + do { + ++source_row; + } while (source_row < source_row_end && rows_to_update[source_row - source_row_start].empty()); + if (source_row < source_row_end) { + target_row_to_update_it = rows_to_update[source_row - source_row_start].cbegin(); + source_column_view.set_row(source_row); + } + } + } else { + if (is_a_string(row.value())) { + const std::string_view string_in_target = target_string_pool.get_const_view(row.value()); + const OffsetString offset_in_new_pool = new_string_pool.get(string_in_target); + *new_column_data_it = offset_in_new_pool.offset(); + } else { + *new_column_data_it = row.value(); + } + } + ++new_column_data_it; + }); + return new_string_column; + } +} } // namespace namespace arcticdb { @@ -1786,68 +1928,6 @@ OutputSchema WriteClause::join_schemas(std::vector&&) const { std::string WriteClause::to_string() const { return "Write"; } -/// Helper class used to iterate over elements inside InputFrame uniformly. -/// -/// It has two template specialisations for each way an InputFrame can store its data. It's supposed to be used only to -/// perform forward iteration. -template -class SourceView {}; - -template -requires(std::same_as && util::instantiation_of) -class SourceView { - using Iterator = decltype(std::declval().data().template begin()); - - public: - SourceView(const T& source, size_t source_data_row) : - it_(source.data().template begin()), - source_data_row_(source_data_row) { - std::advance(it_, source_data_row); - } - - void set_row(const size_t new_row) { - debug::check( - new_row >= source_data_row_, "Cannot move SourceColumnIterator backwards" - ); - // TODO: Implement operator+= because std::advance is linear - std::advance(it_, new_row - source_data_row_); - source_data_row_ = new_row; - } - - const TDT::DataTypeTag::raw_type& operator*() const { return *it_; } - - private: - Iterator it_; - size_t source_data_row_; -}; - -template -requires std::same_as && util::instantiation_of -class SourceView { - using Iterator = const TDT::DataTypeTag::raw_type*; - - public: - SourceView(const T& source, const size_t source_data_row) : source_(source), source_data_row_(source_data_row) {} - void set_row(const size_t new_row) { source_data_row_ = new_row; } - const TDT::DataTypeTag::raw_type& operator*() const { return source_.at(source_data_row_); } - - private: - TypedTensor source_; - size_t source_data_row_; -}; - -template -requires util::any_of> && util::instantiation_of -auto get_source_column_iterator(const T& source, size_t column_index) { - if constexpr (std::is_same_v) { - return SourceView(source.column(column_index), 0); - } else if constexpr (std::is_same_v>) { - return SourceView(source[column_index], 0); - } else { - static_assert(sizeof(T) == 0, "Invalid type"); - } -} - MergeUpdateClause::MergeUpdateClause( std::vector&& on, MergeStrategy strategy, std::shared_ptr source, bool match_on_timeseries_index @@ -2019,75 +2099,34 @@ void MergeUpdateClause::update_and_insert( const size_t source_column_position = source_column_index( *col_ranges[segment_idx], column_idx, source_descriptor.index().field_count() ); + if constexpr (std::same_as) { + user_input::check( + util::is_cstyle_array(source), + "Fortran-style arrays are not supported by merge update yet. Column \"{}\" has data " + "type " + "{} of size {} bytes but the stride is {} bytes", + target_field.name(), + target_field.type(), + sizeof(TDT::DataTypeTag::raw_type), + source.strides()[0] + ); + } SourceView source_column_view = get_source_column_iterator(source, source_column_position); source_column_view.set_row(source_row); if constexpr (is_sequence_type(tdt.data_type())) { - if constexpr (is_fixed_string_type(tdt.data_type())) { - user_input::raise( - "Fixed string sequences are not supported for merge update" - ); - } else if constexpr (is_dynamic_string_type(tdt.data_type())) { - segment_contains_string_column = true; - std::optional gil_lock; - // TODO: size must take into account rows that are going to be inserted - Column new_string_column( - target_column.type(), - target_column.row_count(), - AllocationType::PRESIZED, - target_column.is_sparse() ? Sparsity::PERMITTED : Sparsity::NOT_PERMITTED - ); - ColumnData new_column_data = new_string_column.data(); - auto new_column_data_it = new_column_data.begin(); - auto target_row_to_update_it = - std::make_optional(rows_to_update[source_row - source_row_start].cbegin()); - arcticdb::for_each_enumerated(target_column, [&](auto row) { - const bool current_target_row_is_matched = - target_row_to_update_it.has_value() && - static_cast(row.idx()) == *(*target_row_to_update_it); - if (current_target_row_is_matched) { - position_t new_value; - const auto py_string_object = reinterpret_cast(*source_column_view); - std::optional maybe_error = - read_py_string_to_buffer( - py_string_object, new_value, gil_lock, new_string_pool - ); - if (maybe_error) { - maybe_error->row_index_in_slice_ = row.idx(); - maybe_error->raise(target_field.name(), row_ranges[segment_idx]->first); - } - *new_column_data_it = new_value; - // All target rows that match the current source row are updated. Find the next source - // row which matches something in the target. Since both the source and the target are - // ordered, it's guaranteed that the next matched target row will be larger than the - // current target row. - if (++(*target_row_to_update_it) == - rows_to_update[source_row - source_row_start].cend()) { - do { - ++source_row; - } while (source_row < source_row_end && - rows_to_update[source_row - source_row_start].empty()); - if (source_row < source_row_end) { - target_row_to_update_it = - rows_to_update[source_row - source_row_start].cbegin(); - source_column_view.set_row(source_row); - } else { - target_row_to_update_it = std::nullopt; - } - } - } else { - if (is_a_string(row.value())) { - const std::string_view string_in_target = - target_segment.const_string_pool().get_const_view(row.value()); - const OffsetString offset_in_new_pool = new_string_pool.get(string_in_target); - *new_column_data_it = offset_in_new_pool.offset(); - } else { - *new_column_data_it = row.value(); - } - } - ++new_column_data_it; - }); - target_segment.column(column_idx) = std::move(new_string_column); - } + segment_contains_string_column = true; + Column new_string_column = merge_update_string_column( + target_column, + rows_to_update, + target_field, + *row_ranges[segment_idx], + source_row_start, + source_row_end, + target_segment.string_pool(), + new_string_pool, + source_column_view + ); + target_segment.column(column_idx) = std::move(new_string_column); } else { auto target_data_row_to_update = rows_to_update[source_row - source_row_start].front(); // TODO: Implement operator+= because std::advance is linear @@ -2124,14 +2163,14 @@ template void MergeUpdateClause:: update_and_insert(const std::vector&, const StreamDescriptor&, const ProcessingUnit&, std::span>) const; -/// For each row of source that falls in the row slice in proc find all rows whose index matches the source index value. -/// The matching rows will be sorted in increasing order. Since both source and target are timestamp indexed and -/// ordered, only forward iteration on both source and target is needed and binary search can be used to check if a -/// source index value exists in the target index. At the end some vectors in the output can be empty which means that -/// that particular row in source did not match anything in the target. It is allowed for one row in target to be -/// matched by multiple rows in source only if MergeUpdateClause::on_ is not empty. If MergeUpdateClause::on_ is not -/// empty there will be further filtering which might remove some matches. Otherwise, one row will be updated multiple -/// times which is not allowed. +/// For each row of source that falls in the row slice in proc find all rows whose index matches the source index +/// value. The matching rows will be sorted in increasing order. Since both source and target are timestamp indexed +/// and ordered, only forward iteration on both source and target is needed and binary search can be used to check +/// if a source index value exists in the target index. At the end some vectors in the output can be empty which +/// means that that particular row in source did not match anything in the target. It is allowed for one row in +/// target to be matched by multiple rows in source only if MergeUpdateClause::on_ is not empty. If +/// MergeUpdateClause::on_ is not empty there will be further filtering which might remove some matches. Otherwise, +/// one row will be updated multiple times which is not allowed. std::vector> MergeUpdateClause::filter_index_match(const ProcessingUnit& proc) const { using IndexType = ScalarTagType>; const std::span> target_segments{*proc.segments_}; From 051af2c4a75762ce379f0aeaa9f093f3e26367b0 Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Wed, 3 Dec 2025 18:59:10 +0200 Subject: [PATCH 45/46] Add comments --- cpp/arcticdb/pipeline/input_frame_utils.hpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/cpp/arcticdb/pipeline/input_frame_utils.hpp b/cpp/arcticdb/pipeline/input_frame_utils.hpp index b1590326ed9..7fcaf7d1a06 100644 --- a/cpp/arcticdb/pipeline/input_frame_utils.hpp +++ b/cpp/arcticdb/pipeline/input_frame_utils.hpp @@ -11,9 +11,14 @@ namespace arcticdb::pipelines { template -requires((Index::field_count() == 0 || Index::field_count() == 1) && (std::ranges::sized_range && ...)) +requires( + (Index::field_count() == 0 || Index::field_count() == 1) && (std::ranges::sized_range && ...) && + // strings are not supported yet, in order tu support them we need to initialise python strings + (!std::convertible_to, std::string_view>) +) auto input_frame_from_tensors(const StreamDescriptor& desc, T&&... input) { constexpr static size_t data_columns = sizeof...(T) - Index::field_count(); + // TODO: If the range is a vector move the vector in the materialized output std::tuple materialized_input{std::vector, bool>, uint8_t, From 686d8d7c7189344c3231f5a40d45ed312c1dedb3 Mon Sep 17 00:00:00 2001 From: Vasil Pashov Date: Wed, 3 Dec 2025 19:04:04 +0200 Subject: [PATCH 46/46] Fix concept --- cpp/arcticdb/pipeline/input_frame_utils.hpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cpp/arcticdb/pipeline/input_frame_utils.hpp b/cpp/arcticdb/pipeline/input_frame_utils.hpp index 7fcaf7d1a06..3ae0c8e6094 100644 --- a/cpp/arcticdb/pipeline/input_frame_utils.hpp +++ b/cpp/arcticdb/pipeline/input_frame_utils.hpp @@ -13,8 +13,8 @@ namespace arcticdb::pipelines { template requires( (Index::field_count() == 0 || Index::field_count() == 1) && (std::ranges::sized_range && ...) && - // strings are not supported yet, in order tu support them we need to initialise python strings - (!std::convertible_to, std::string_view>) + // strings are not supported yet; in order tu support them we need to initialise python strings + (!std::convertible_to, std::string_view> && ...) ) auto input_frame_from_tensors(const StreamDescriptor& desc, T&&... input) { constexpr static size_t data_columns = sizeof...(T) - Index::field_count();