diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index d1d9920319..72518bd06c 100755 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -26,6 +26,7 @@ option(ARCTICDB_USING_ADDRESS_SANITIZER "Enable address sanitizer." OFF) option(ARCTICDB_USING_THREAD_SANITIZER "Enable thread sanitizer." OFF) option(ARCTICDB_USING_UB_SANITIZER "Enable undefined behavior sanitizer." OFF) option(ARCTICDB_LOG_PERFORMANCE "Whether to log performance timings." OFF) +option(ARCTICDB_COUNT_ALLOCATION "Override new and delete to count allocations." OFF) set(ARCTICDB_SANITIZER_FLAGS) if(${ARCTICDB_USING_ADDRESS_SANITIZER}) @@ -47,6 +48,17 @@ if(${ARCTICDB_LOG_PERFORMANCE}) add_compile_definitions(ARCTICDB_LOG_PERFORMANCE) endif() +if(${ARCTICDB_COUNT_ALLOCATIONS}) + add_compile_definitions(ARCTICDB_COUNT_ALLOCATIONS) + include(FetchContent) + FetchContent_Declare( + cpptrace + GIT_REPOSITORY https://github.com/jeremy-rifkin/cpptrace.git + GIT_TAG v0.7.3 # + ) + FetchContent_MakeAvailable(cpptrace) +endif() + if(ARCTICDB_SANITIZER_FLAGS) list(APPEND ARCTICDB_SANITIZER_FLAGS "-g;-fno-omit-frame-pointer;-fno-optimize-sibling-calls;") if (CMAKE_CXX_COMPILER_ID STREQUAL "Clang") diff --git a/cpp/arcticdb/CMakeLists.txt b/cpp/arcticdb/CMakeLists.txt index e41495c5d4..9b88cf8acf 100644 --- a/cpp/arcticdb/CMakeLists.txt +++ b/cpp/arcticdb/CMakeLists.txt @@ -268,6 +268,7 @@ set(arcticdb_srcs processing/query_planner.hpp processing/sorted_aggregation.hpp processing/unsorted_aggregation.hpp + storage/async_storage.hpp storage/constants.hpp storage/common.hpp storage/config_resolvers.hpp @@ -277,22 +278,22 @@ set(arcticdb_srcs storage/library.hpp storage/library_index.hpp storage/library_manager.hpp - storage/storage_mock_client.hpp - storage/azure/azure_client_wrapper.hpp - storage/azure/azure_mock_client.hpp - storage/azure/azure_real_client.hpp + storage/mock/storage_mock_client.hpp + storage/azure/azure_client_interface.hpp + storage/mock/azure_mock_client.hpp + storage/azure/azure_client_impl.hpp storage/azure/azure_storage.hpp storage/lmdb/lmdb.hpp - storage/lmdb/lmdb_client_wrapper.hpp - storage/lmdb/lmdb_mock_client.hpp - storage/lmdb/lmdb_real_client.hpp + storage/lmdb/lmdb_client_interface.hpp + storage/mock/lmdb_mock_client.hpp + storage/lmdb/lmdb_client_impl.hpp storage/lmdb/lmdb_storage.hpp storage/memory/memory_storage.hpp storage/memory/memory_storage.cpp storage/mongo/mongo_client.hpp storage/mongo/mongo_instance.hpp - storage/mongo/mongo_client_wrapper.hpp - storage/mongo/mongo_mock_client.hpp + storage/mongo/mongo_client_interface.hpp + storage/mock/mongo_mock_client.hpp storage/mongo/mongo_storage.hpp storage/object_store_utils.hpp storage/file/file_store.hpp @@ -300,7 +301,7 @@ set(arcticdb_srcs storage/file/file_store.hpp storage/single_file_storage.hpp storage/s3/nfs_backed_storage.hpp - storage/s3/s3_client_wrapper.hpp + storage/s3/s3_client_interface.hpp storage/s3/s3_storage_tool.hpp storage/s3/s3_settings.hpp storage/storage_factory.hpp @@ -331,6 +332,7 @@ set(arcticdb_srcs toolbox/library_tool.hpp toolbox/storage_mover.hpp util/allocator.hpp + util/allocation_tracing.hpp util/bitset.hpp util/buffer.hpp util/buffer_pool.hpp @@ -470,21 +472,21 @@ set(arcticdb_srcs storage/failure_simulation.cpp storage/library_manager.cpp storage/azure/azure_storage.cpp - storage/azure/azure_real_client.cpp - storage/azure/azure_mock_client.cpp - storage/lmdb/lmdb_mock_client.cpp - storage/lmdb/lmdb_real_client.cpp + storage/azure/azure_client_impl.cpp + storage/mock/azure_mock_client.cpp + storage/mock/lmdb_mock_client.cpp + storage/lmdb/lmdb_client_impl.cpp storage/lmdb/lmdb_storage.cpp storage/file/mapped_file_storage.cpp storage/mongo/mongo_client.cpp storage/mongo/mongo_instance.cpp - storage/mongo/mongo_mock_client.cpp + storage/mock/mongo_mock_client.cpp storage/mongo/mongo_storage.cpp storage/s3/nfs_backed_storage.cpp storage/s3/ec2_utils.cpp storage/s3/s3_api.cpp - storage/s3/s3_real_client.cpp - storage/s3/s3_mock_client.cpp + storage/s3/s3_client_impl.cpp + storage/mock/s3_mock_client.cpp storage/s3/s3_storage.cpp storage/s3/s3_storage_tool.cpp storage/storage_factory.cpp @@ -497,6 +499,7 @@ set(arcticdb_srcs toolbox/library_tool.cpp util/allocator.cpp util/buffer_holder.cpp + util/allocation_tracing.cpp util/buffer_pool.cpp util/configs_map.cpp util/decimal.cpp @@ -675,6 +678,12 @@ set (arcticdb_core_libraries Azure::azure-storage-blobs ) +if(${ARCTICDB_COUNT_ALLOCATIONS}) + list(APPEND arcticdb_core_libraries + cpptrace::cpptrace + ) +endif() + if(${ARCTICDB_PYTHON_EXPLICIT_LINK}) # Even though python is transitive dependency MSVS builds fail if we don't link against python explicitly # TODO: Figure out why diff --git a/cpp/arcticdb/async/async_store.hpp b/cpp/arcticdb/async/async_store.hpp index 3bfe961703..44f4013bff 100644 --- a/cpp/arcticdb/async/async_store.hpp +++ b/cpp/arcticdb/async/async_store.hpp @@ -31,10 +31,9 @@ std::pair> lookup_match_in_dedup_map( template auto read_and_continue(const VariantKey& key, std::shared_ptr library, const storage::ReadKeyOpts& opts, Callable&& c) { return async::submit_io_task(ReadCompressedTask{key, library, opts, std::forward(c)}) - .via(&async::cpu_executor()) - .thenValue([](auto &&result) mutable { - auto&& [key_seg, continuation] = std::forward(result); - return continuation(std::move(key_seg)); + .thenValueInline([](auto &&result) mutable { + auto&& [key_seg_fut, continuation] = std::forward(result); + return std::move(key_seg_fut).thenValueInline([continuation=std::move(continuation)] (storage::KeySegmentPair&& key_seg) mutable { return continuation(std::move(key_seg)); }); } ); } @@ -67,17 +66,13 @@ class AsyncStore : public Store { IndexValue end_index, SegmentInMemory &&segment) override { - util::check(segment.descriptor().id() == stream_id, - "Descriptor id mismatch in atom key {} != {}", - stream_id, - segment.descriptor().id()); + util::check(segment.descriptor().id() == stream_id, "Descriptor id mismatch in atom key {} != {}", stream_id, segment.descriptor().id()); return async::submit_cpu_task(EncodeAtomTask{ key_type, version_id, stream_id, start_index, end_index, current_timestamp(), std::move(segment), codec_, encoding_version_ - }) - .via(&async::io_executor()) - .thenValue(WriteSegmentTask{library_}); + }).via(&async::io_executor()) + .thenValue(WriteSegmentTask{library_}); } folly::Future write( @@ -89,10 +84,7 @@ folly::Future write( IndexValue end_index, SegmentInMemory &&segment) override { - util::check(segment.descriptor().id() == stream_id, - "Descriptor id mismatch in atom key {} != {}", - stream_id, - segment.descriptor().id()); + util::check(segment.descriptor().id() == stream_id, "Descriptor id mismatch in atom key {} != {}", stream_id, segment.descriptor().id()); return async::submit_cpu_task(EncodeAtomTask{ key_type, version_id, stream_id, start_index, end_index, creation_ts, @@ -169,7 +161,7 @@ folly::Future write_compressed(storage::KeySegmentPair ks) override } void write_compressed_sync(storage::KeySegmentPair ks) override { - library_->write(Composite(std::move(ks))); + library_->write(std::move(ks)); } folly::Future update(const entity::VariantKey &key, @@ -226,9 +218,8 @@ folly::Future> read( return read_and_continue(key, library_, opts, DecodeSegmentTask{}); } -std::pair read_sync(const entity::VariantKey &key, - storage::ReadKeyOpts opts) override { - return DecodeSegmentTask{}(read_dispatch(key, library_, opts)); +std::pair read_sync(const entity::VariantKey& key, storage::ReadKeyOpts opts) override { + return DecodeSegmentTask{}(read_sync_dispatch(key, library_, opts)); } folly::Future read_compressed( @@ -237,11 +228,8 @@ folly::Future read_compressed( return read_and_continue(key, library_, opts, PassThroughTask{}); } -storage::KeySegmentPair read_compressed_sync( - const entity::VariantKey& key, - storage::ReadKeyOpts opts - ) override { - return read_dispatch( key, library_, opts ); +storage::KeySegmentPair read_compressed_sync(const entity::VariantKey& key, storage::ReadKeyOpts opts) override { + return read_sync_dispatch(key, library_, opts); } folly::Future, std::optional>> read_metadata(const entity::VariantKey &key, storage::ReadKeyOpts opts) override { @@ -256,7 +244,7 @@ folly::Future, Strea folly::Future> read_timeseries_descriptor( const entity::VariantKey &key, - storage::ReadKeyOpts opts = storage::ReadKeyOpts{}) override { + storage::ReadKeyOpts opts) override { return read_and_continue(key, library_, opts, DecodeTimeseriesDescriptorTask{}); } @@ -349,12 +337,17 @@ folly::Future> batch_read_compressed( std::vector> batch_read_uncompressed( std::vector&& ranges_and_keys, std::shared_ptr> columns_to_decode) override { - return folly::window( - std::move(ranges_and_keys), - [this, columns_to_decode](auto&& ranges_and_key) { - const auto key = ranges_and_key.key_; - return read_and_continue(key, library_, storage::ReadKeyOpts{}, DecodeSliceTask{std::move(ranges_and_key), columns_to_decode}); - }, async::TaskScheduler::instance()->io_thread_count() * 2); + ARCTICDB_RUNTIME_DEBUG(log::version(), "Reading {} keys", ranges_and_keys.size()); + std::vector> output; + for(auto&& ranges_and_key : ranges_and_keys) { + const auto key = ranges_and_key.key_; + output.emplace_back(read_and_continue( + key, + library_, + storage::ReadKeyOpts{}, + DecodeSliceTask{std::move(ranges_and_key), columns_to_decode})); + } + return output; } std::vector> batch_key_exists( @@ -367,7 +360,6 @@ std::vector> batch_key_exists( return res; } - folly::Future async_write( folly::Future> &&input_fut, const std::shared_ptr &de_dup_map) override { @@ -390,8 +382,7 @@ folly::Future async_write( .thenValue([lib=library_](auto &&item) { auto [key_opt_segment, slice] = std::forward(item); if (key_opt_segment.second) - lib->write(Composite({VariantKey{key_opt_segment.first}, - std::move(*key_opt_segment.second)})); + lib->write({VariantKey{key_opt_segment.first}, std::move(*key_opt_segment.second)}); return SliceAndKey{slice, to_atom(key_opt_segment.first)}; }); diff --git a/cpp/arcticdb/async/task_scheduler.hpp b/cpp/arcticdb/async/task_scheduler.hpp index 4efd408cdd..6ec9e16e3d 100644 --- a/cpp/arcticdb/async/task_scheduler.hpp +++ b/cpp/arcticdb/async/task_scheduler.hpp @@ -174,11 +174,6 @@ inline auto get_default_num_cpus([[maybe_unused]] const std::string& cgroup_fold * 3/ Priority: How to assign priorities to task in order to treat the most pressing first. * 4/ Throttling: (similar to priority) how to absorb work spikes and apply memory backpressure */ - -inline int64_t default_io_thread_count(uint64_t cpu_count) { - return std::min(int64_t(100L), static_cast(static_cast(cpu_count) * 1.5)); -} - class TaskScheduler { public: using CPUSchedulerType = folly::FutureExecutor; @@ -199,7 +194,7 @@ class TaskScheduler { auto task = std::forward(t); static_assert(std::is_base_of_v>, "Only supports Task derived from BaseTask"); ARCTICDB_DEBUG(log::schedule(), "{} Submitting CPU task {}: {} of {}", uintptr_t(this), typeid(task).name(), cpu_exec_.getTaskQueueSize(), cpu_exec_.kDefaultMaxQueueSize); - std::lock_guard lock{cpu_mutex_}; + //std::lock_guard lock{cpu_mutex_}; return cpu_exec_.addFuture(std::move(task)); } @@ -208,7 +203,7 @@ class TaskScheduler { auto task = std::forward(t); static_assert(std::is_base_of_v>, "Only support Tasks derived from BaseTask"); ARCTICDB_DEBUG(log::schedule(), "{} Submitting IO task {}: {}", uintptr_t(this), typeid(task).name(), io_exec_.getPendingTaskCount()); - std::lock_guard lock{io_mutex_}; + //std::lock_guard lock{io_mutex_}; return io_exec_.addFuture(std::move(task)); } diff --git a/cpp/arcticdb/async/tasks.cpp b/cpp/arcticdb/async/tasks.cpp index b209f8528e..e7f0696e9e 100644 --- a/cpp/arcticdb/async/tasks.cpp +++ b/cpp/arcticdb/async/tasks.cpp @@ -50,23 +50,4 @@ namespace arcticdb::async { decode_into_memory_segment(seg, hdr, segment_in_memory, desc); return pipelines::SegmentAndSlice(std::move(ranges_and_key_), std::move(segment_in_memory)); } - - pipelines::SliceAndKey DecodeSlicesTask::decode_into_slice(std::pair&& sk_pair) const { - auto [seg, sk] = std::move(sk_pair); - ARCTICDB_DEBUG(log::storage(), "ReadAndDecodeAtomTask decoding segment with key {}", - variant_key_view(sk.key())); - - auto &hdr = seg.header(); - const auto& desc = seg.descriptor(); - auto descriptor = async::get_filtered_descriptor(desc, filter_columns_); - sk.slice_.adjust_columns(descriptor.field_count() - descriptor.index().field_count()); - - ARCTICDB_TRACE(log::codec(), "Creating segment"); - SegmentInMemory res(std::move(descriptor)); - - decode_into_memory_segment(seg, hdr, res, desc); - sk.set_segment(std::move(res)); - return sk; - } - } //namespace arcticdb::async \ No newline at end of file diff --git a/cpp/arcticdb/async/tasks.hpp b/cpp/arcticdb/async/tasks.hpp index f28c5ce2b3..71d8ff32e1 100644 --- a/cpp/arcticdb/async/tasks.hpp +++ b/cpp/arcticdb/async/tasks.hpp @@ -182,7 +182,7 @@ struct WriteSegmentTask : BaseTask { VariantKey operator()(storage::KeySegmentPair &&key_seg) const { ARCTICDB_SAMPLE(WriteSegmentTask, 0) auto k = key_seg.variant_key(); - lib_->write(Composite(std::move(key_seg))); + lib_->write(std::move(key_seg)); return k; } }; @@ -218,23 +218,29 @@ struct UpdateSegmentTask : BaseTask { VariantKey operator()(storage::KeySegmentPair &&key_seg) const { ARCTICDB_SAMPLE(UpdateSegmentTask, 0) auto k = key_seg.variant_key(); - lib_->update(Composite(std::move(key_seg)), opts_); + lib_->update(std::move(key_seg), opts_); return k; } }; template struct KeySegmentContinuation { - storage::KeySegmentPair key_seg_; + folly::Future key_seg_; Callable continuation_; }; -inline storage::KeySegmentPair read_dispatch(const entity::VariantKey& variant_key, const std::shared_ptr& lib, const storage::ReadKeyOpts& opts) { - return util::variant_match(variant_key, [&lib, &opts](const auto &key) { +inline folly::Future read_dispatch(entity::VariantKey&& variant_key, const std::shared_ptr& lib, const storage::ReadKeyOpts& opts) { + return util::variant_match(variant_key, [&lib, &opts](auto&& key) { return lib->read(key, opts); }); } +inline storage::KeySegmentPair read_sync_dispatch(const entity::VariantKey& variant_key, const std::shared_ptr& lib, storage::ReadKeyOpts opts) { + return util::variant_match(variant_key, [&lib, opts](const auto &key) { + return lib->read_sync(key, opts); + }); +} + template struct ReadCompressedTask : BaseTask { entity::VariantKey key_; @@ -258,7 +264,7 @@ struct ReadCompressedTask : BaseTask { KeySegmentContinuation operator()() { ARCTICDB_SAMPLE(ReadCompressed, 0) - return KeySegmentContinuation{read_dispatch(key_, lib_, opts_), std::move(continuation_)}; + return KeySegmentContinuation{read_dispatch(std::move(key_), lib_, opts_), std::move(continuation_)}; } }; @@ -296,11 +302,11 @@ struct CopyCompressedTask : BaseTask { ARCTICDB_MOVE_ONLY_DEFAULT(CopyCompressedTask) VariantKey copy() { - return std::visit([that = this](const auto &source_key) { - auto key_seg = that->lib_->read(source_key); - auto target_key_seg = stream::make_target_key(that->key_type_, that->stream_id_, that->version_id_, source_key, std::move(key_seg.segment())); + return std::visit([this](const auto &source_key) { + auto key_seg = lib_->read_sync(source_key); + auto target_key_seg = stream::make_target_key(key_type_, stream_id_, version_id_, source_key, std::move(key_seg.segment())); auto return_key = target_key_seg.variant_key(); - that->lib_->write(Composite{std::move(target_key_seg) }); + lib_->write(std::move(target_key_seg)); return return_key; }, source_key_); } @@ -380,7 +386,7 @@ struct CopyCompressedInterStoreTask : async::BaseTask { if (!target_stores_.empty()) { storage::KeySegmentPair key_segment_pair; try { - key_segment_pair = source_store_->read_compressed_sync(key_to_read_, storage::ReadKeyOpts{}); + key_segment_pair = source_store_->read_compressed_sync(key_to_read_); } catch (const storage::KeyNotFoundException& e) { log::storage().debug("Key {} not found on the source: {}", variant_key_view(key_to_read_), e.what()); return failed_targets; @@ -438,11 +444,12 @@ struct DecodeSliceTask : BaseTask { pipelines::RangesAndKey&& ranges_and_key, std::shared_ptr> columns_to_decode): ranges_and_key_(std::move(ranges_and_key)), - columns_to_decode_(columns_to_decode) { + columns_to_decode_(std::move(columns_to_decode)) { } pipelines::SegmentAndSlice operator()(storage::KeySegmentPair&& key_segment_pair) { ARCTICDB_SAMPLE(DecodeSliceTask, 0) + ARCTICDB_DEBUG(log::memory(), "Decode into slice {}", key_segment_pair.variant_key()); return decode_into_slice(std::move(key_segment_pair)); } @@ -450,30 +457,6 @@ struct DecodeSliceTask : BaseTask { pipelines::SegmentAndSlice decode_into_slice(storage::KeySegmentPair&& key_segment_pair); }; -struct DecodeSlicesTask : BaseTask { - ARCTICDB_MOVE_ONLY_DEFAULT(DecodeSlicesTask) - - std::shared_ptr> filter_columns_; - - explicit DecodeSlicesTask( - const std::shared_ptr>& filter_columns) : - filter_columns_(filter_columns) { - } - - Composite operator()(Composite> && skp) const { - ARCTICDB_SAMPLE(DecodeSlicesTask, 0) - auto sk_pairs = std::move(skp); - return sk_pairs.transform([this] (auto&& ssp){ - auto seg_slice_pair = std::move(ssp); - ARCTICDB_DEBUG(log::version(), "Decoding slice {}", seg_slice_pair.second.key()); - return decode_into_slice(std::move(seg_slice_pair)); - }); - } - -private: - pipelines::SliceAndKey decode_into_slice(std::pair&& sk_pair) const; -}; - struct SegmentFunctionTask : BaseTask { stream::StreamSource::ReadContinuation func_; @@ -493,18 +476,23 @@ struct SegmentFunctionTask : BaseTask { struct MemSegmentProcessingTask : BaseTask { std::vector> clauses_; std::vector entity_ids_; + timestamp creation_time_; explicit MemSegmentProcessingTask( std::vector> clauses, std::vector&& entity_ids) : clauses_(std::move(clauses)), - entity_ids_(std::move(entity_ids)) { + entity_ids_(std::move(entity_ids)), + creation_time_(util::SysClock::coarse_nanos_since_epoch()){ } ARCTICDB_MOVE_ONLY_DEFAULT(MemSegmentProcessingTask) std::vector operator()() { ARCTICDB_DEBUG_THROW(5) + const auto nanos_start = util::SysClock::coarse_nanos_since_epoch(); + const auto time_in_queue = double(nanos_start - creation_time_) / BILLION; + ARCTICDB_RUNTIME_DEBUG(log::inmem(), "Segment processing task running after {}s queue time", time_in_queue); for (auto it = clauses_.cbegin(); it != clauses_.cend(); ++it) { entity_ids_ = (*it)->process(std::move(entity_ids_)); @@ -512,6 +500,9 @@ struct MemSegmentProcessingTask : BaseTask { if(next_it != clauses_.cend() && (*it)->clause_info().output_structure_ != (*next_it)->clause_info().input_structure_) break; } + const auto nanos_end = util::SysClock::coarse_nanos_since_epoch(); + const auto time_taken = double(nanos_end - nanos_start) / BILLION; + ARCTICDB_RUNTIME_DEBUG(log::inmem(), "Segment processing task completed after {}s run time", time_taken); return std::move(entity_ids_); } @@ -593,15 +584,16 @@ struct WriteCompressedTask : BaseTask { storage::KeySegmentPair kv_; std::shared_ptr lib_; - WriteCompressedTask(storage::KeySegmentPair &&kv, std::shared_ptr lib) : - kv_(std::move(kv)), lib_(std::move(lib)) { + WriteCompressedTask(storage::KeySegmentPair&& key_seg, std::shared_ptr lib) : + kv_(std::move(key_seg)), + lib_(std::move(lib)) { ARCTICDB_DEBUG(log::storage(), "Creating write compressed task"); } ARCTICDB_MOVE_ONLY_DEFAULT(WriteCompressedTask) folly::Future write() { - lib_->write(Composite(std::move(kv_))); + lib_->write(std::move(kv_)); return folly::makeFuture(); } @@ -625,7 +617,9 @@ struct WriteCompressedBatchTask : BaseTask { ARCTICDB_MOVE_ONLY_DEFAULT(WriteCompressedBatchTask) folly::Future write() { - lib_->write(Composite(std::move(kvs_))); + for(auto&& kv : kvs_) + lib_->write(std::move(kv)); + return folly::makeFuture(); } @@ -650,7 +644,7 @@ struct RemoveTask : BaseTask { ARCTICDB_MOVE_ONLY_DEFAULT(RemoveTask) stream::StreamSink::RemoveKeyResultType operator()() { - lib_->remove(Composite(std::move(key_)), opts_); + lib_->remove(std::move(key_), opts_); return {}; } }; @@ -674,7 +668,7 @@ struct RemoveBatchTask : BaseTask { std::vector operator()() { - lib_->remove(Composite(std::move(keys_)), opts_); + lib_->remove(std::span(keys_), opts_); return {}; } }; diff --git a/cpp/arcticdb/async/test/test_async.cpp b/cpp/arcticdb/async/test/test_async.cpp index d981a70706..0719f3d4a2 100644 --- a/cpp/arcticdb/async/test/test_async.cpp +++ b/cpp/arcticdb/async/test/test_async.cpp @@ -23,9 +23,9 @@ #include #include -#include +#include #include -#include +#include #include using namespace arcticdb; diff --git a/cpp/arcticdb/column_store/segment_utils.hpp b/cpp/arcticdb/column_store/segment_utils.hpp index 26288b59ac..8f6d05f686 100644 --- a/cpp/arcticdb/column_store/segment_utils.hpp +++ b/cpp/arcticdb/column_store/segment_utils.hpp @@ -17,6 +17,7 @@ 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); @@ -24,7 +25,7 @@ inline ankerl::unordered_dense::set unique_values_for_string using type_info = ScalarTypeInfo; if constexpr(is_sequence_type(type_info::data_type)) { Column::for_each(column, [&output_set](auto value) { - output_set.insert(value); + output_set.emplace(value); }); } else { util::raise_rte("Column {} is not a string type column"); diff --git a/cpp/arcticdb/entity/atom_key.hpp b/cpp/arcticdb/entity/atom_key.hpp index 7240979cd0..e284d4f7c4 100644 --- a/cpp/arcticdb/entity/atom_key.hpp +++ b/cpp/arcticdb/entity/atom_key.hpp @@ -22,14 +22,13 @@ class AtomKeyImpl { template AtomKeyImpl( - StreamId id, - VersionId version_id, - timestamp creation_ts, - ContentHash content_hash, - IndexValueType start_index, - IndexValueType end_index, - KeyType key_type) - : + StreamId id, + VersionId version_id, + timestamp creation_ts, + ContentHash content_hash, + IndexValueType start_index, + IndexValueType end_index, + KeyType key_type) : id_(std::move(id)), version_id_(version_id), creation_ts_(creation_ts), @@ -59,11 +58,6 @@ class AtomKeyImpl { const IndexValue &end_index() const { return index_end_; } IndexRange index_range() const { IndexRange ir = {index_start_, index_end_}; ir.end_closed_ = false; return ir;} - auto change_type(KeyType new_type) { - key_type_ = new_type; - reset_cached(); - } - /** * Useful for caching/replacing the ID with an existing shared instance. * @param id Will be moved. diff --git a/cpp/arcticdb/log/log.hpp b/cpp/arcticdb/log/log.hpp index 1714ee1e91..4dfb0ca1c7 100644 --- a/cpp/arcticdb/log/log.hpp +++ b/cpp/arcticdb/log/log.hpp @@ -11,7 +11,6 @@ #include - #ifdef DEBUG_BUILD #define ARCTICDB_DEBUG(logger, ...) logger.debug(__VA_ARGS__) #define ARCTICDB_TRACE(logger, ...) logger.trace(__VA_ARGS__) diff --git a/cpp/arcticdb/pipeline/frame_slice.hpp b/cpp/arcticdb/pipeline/frame_slice.hpp index 9a76e84fad..c4aad4498c 100644 --- a/cpp/arcticdb/pipeline/frame_slice.hpp +++ b/cpp/arcticdb/pipeline/frame_slice.hpp @@ -72,24 +72,24 @@ struct FrameSlice { FrameSlice( std::shared_ptr desc, - const ColRange& col_range, - const RowRange& row_range, + ColRange col_range, + RowRange row_range, std::optional hash = std::nullopt, std::optional num_buckets = std::nullopt, std::optional> indices = std::nullopt) : - col_range(col_range), - row_range(row_range), + col_range(std::move(col_range)), + row_range(std::move(row_range)), desc_(std::move(desc)), hash_bucket_(hash), num_buckets_(num_buckets), indices_(std::move(indices)) { } - FrameSlice(const ColRange& col_range, const RowRange& row_range, + FrameSlice(ColRange col_range, RowRange row_range, std::optional hash_bucket = std::nullopt, std::optional num_buckets = std::nullopt) : - col_range(col_range), - row_range(row_range), + col_range(std::move(col_range)), + row_range(std::move(row_range)), hash_bucket_(hash_bucket), num_buckets_(num_buckets) { } @@ -174,16 +174,16 @@ struct FrameSlice { // Collection of these objects is the input to batch_read_uncompressed struct RangesAndKey { explicit RangesAndKey(const FrameSlice& frame_slice, entity::AtomKey&& key, bool is_incomplete): - row_range_(frame_slice.rows()), - col_range_(frame_slice.columns()), - key_(std::move(key)), - is_incomplete_(is_incomplete) { + row_range_(frame_slice.rows()), + col_range_(frame_slice.columns()), + key_(std::move(key)), + is_incomplete_(is_incomplete) { } - explicit RangesAndKey(const RowRange& row_range, const ColRange& col_range, const entity::AtomKey& key): - row_range_(row_range), - col_range_(col_range), - key_(key) { + explicit RangesAndKey(RowRange row_range, ColRange col_range, entity::AtomKey key): + row_range_(std::move(row_range)), + col_range_(std::move(col_range)), + key_(std::move(key)) { } RangesAndKey() = delete; ARCTICDB_MOVE_COPY_DEFAULT(RangesAndKey) diff --git a/cpp/arcticdb/processing/clause.cpp b/cpp/arcticdb/processing/clause.cpp index 308bb0d2f0..a69d9a671a 100644 --- a/cpp/arcticdb/processing/clause.cpp +++ b/cpp/arcticdb/processing/clause.cpp @@ -103,11 +103,13 @@ std::vector PassthroughClause::process(std::vector&& entity_ } std::vector FilterClause::process(std::vector&& entity_ids) const { + ARCTICDB_SAMPLE(FilterClause, 0) if (entity_ids.empty()) { return {}; } auto proc = gather_entities, std::shared_ptr, std::shared_ptr>(*component_manager_, std::move(entity_ids)); proc.set_expression_context(expression_context_); + ARCTICDB_RUNTIME_DEBUG(log::memory(), "Doing filter {} for entity ids {}", expression_context_->root_node_name_, entity_ids); auto variant_data = proc.get(expression_context_->root_node_name_); std::vector output; util::variant_match(variant_data, @@ -116,11 +118,11 @@ std::vector FilterClause::process(std::vector&& entity_ids) proc.apply_filter(std::move(bitset), optimisation_); output = push_entities(*component_manager_, std::move(proc)); } else { - log::version().debug("Filter returned empty result"); + log::memory().debug("Filter returned empty result"); } }, [](EmptyResult) { - log::version().debug("Filter returned empty result"); + log::memory().debug("Filter returned empty result"); }, [&output, &proc, this](FullResult) { output = push_entities(*component_manager_, std::move(proc)); @@ -240,6 +242,7 @@ std::vector> AggregationClause::structure_for_processing(s std::vector AggregationClause::process(std::vector&& entity_ids) const { ARCTICDB_DEBUG_THROW(5) + ARCTICDB_SAMPLE(AggregationClause, 0) if (entity_ids.empty()) { return {}; } @@ -521,6 +524,7 @@ void ResampleClause::set_processing_config(const ProcessingConf template std::vector> ResampleClause::structure_for_processing( std::vector& ranges_and_keys) { + ARCTICDB_RUNTIME_DEBUG(log::memory(), "ResampleClause: structure for processing 1"); if (ranges_and_keys.empty()) { return {}; } @@ -555,6 +559,7 @@ std::vector> ResampleClause::structure_fo if (entity_ids.empty()) { return {}; } + ARCTICDB_RUNTIME_DEBUG(log::memory(), "ResampleClause: structure for processing 2"); auto [segments, row_ranges, col_ranges] = component_manager_->get_entities, std::shared_ptr, std::shared_ptr>(entity_ids, false); std::vector ranges_and_entities; ranges_and_entities.reserve(entity_ids.size()); @@ -605,10 +610,12 @@ std::vector> ResampleClause::structure_fo template std::vector ResampleClause::process(std::vector&& entity_ids) const { + ARCTICDB_SAMPLE(ResampleClause, 0) if (entity_ids.empty()) { return {}; } auto proc = gather_entities, std::shared_ptr, std::shared_ptr, EntityFetchCount>(*component_manager_, std::move(entity_ids)); + ARCTICDB_RUNTIME_DEBUG(log::memory(), "ResampleClause: processing entities {}", entity_ids); auto row_slices = split_by_row_slice(std::move(proc)); // If the entity fetch counts for the entities in the first row slice are 2, the first bucket overlapping this row // slice is being computed by the call to process dealing with the row slices above these. Otherwise, this call diff --git a/cpp/arcticdb/processing/clause_utils.cpp b/cpp/arcticdb/processing/clause_utils.cpp index 910dfa4c84..fa82dd32f6 100644 --- a/cpp/arcticdb/processing/clause_utils.cpp +++ b/cpp/arcticdb/processing/clause_utils.cpp @@ -84,12 +84,18 @@ std::vector flatten_entities(std::vector>&& enti return res; } -std::vector> split_futures( - std::vector>&& segment_and_slice_futures) { - std::vector> res; +using SegmentAndSlice = pipelines::SegmentAndSlice; + +std::vector split_futures( + std::vector>&& segment_and_slice_futures, + std::vector& segment_fetch_counts) { + std::vector res; res.reserve(segment_and_slice_futures.size()); - for (auto&& future: segment_and_slice_futures) { - res.emplace_back(folly::splitFuture(std::move(future))); + for (auto&& [index, future]: folly::enumerate(segment_and_slice_futures)) { + if(segment_fetch_counts[index] > 1) + res.emplace_back(folly::splitFuture(std::move(future))); + else + res.emplace_back(std::move(future)); } return res; } diff --git a/cpp/arcticdb/processing/clause_utils.hpp b/cpp/arcticdb/processing/clause_utils.hpp index 7d0942c23d..5f54e3c235 100644 --- a/cpp/arcticdb/processing/clause_utils.hpp +++ b/cpp/arcticdb/processing/clause_utils.hpp @@ -70,19 +70,19 @@ struct RangesAndEntity { } ARCTICDB_MOVE_COPY_DEFAULT(RangesAndEntity) - const RowRange& row_range() const { + [[nodiscard]] const RowRange& row_range() const { return *row_range_; } - const ColRange& col_range() const { + [[nodiscard]] const ColRange& col_range() const { return *col_range_; } - timestamp start_time() const { + [[nodiscard]] timestamp start_time() const { return timestamp_range_->first; } - timestamp end_time() const { + [[nodiscard]] timestamp end_time() const { return timestamp_range_->second; } @@ -193,16 +193,21 @@ ProcessingUnit gather_entities(ComponentManager& component_manager, std::vector< }(), ...); return res; } +std::vector flatten_entities(std::vector>&& entity_ids_vec); -std::vector push_entities(ComponentManager& component_manager, ProcessingUnit&& proc, EntityFetchCount entity_fetch_count=1); +using FutureOrSplitter = std::variant, folly::FutureSplitter>; -std::vector flatten_entities(std::vector>&& entity_ids_vec); +std::vector split_futures( + std::vector>&& segment_and_slice_futures, + std::vector& segment_fetch_counts); -std::vector> split_futures( - std::vector>&& segment_and_slice_futures); +std::vector push_entities( + ComponentManager& component_manager, + ProcessingUnit&& proc, + EntityFetchCount entity_fetch_count=1); std::shared_ptr> generate_segment_fetch_counts( - const std::vector>& processing_unit_indexes, - size_t num_segments); + const std::vector>& processing_unit_indexes, + size_t num_segments); -}//namespace arcticdb +} //namespace arcticdb diff --git a/cpp/arcticdb/processing/component_manager.cpp b/cpp/arcticdb/processing/component_manager.cpp index 187ef90d1b..870275a3bd 100644 --- a/cpp/arcticdb/processing/component_manager.cpp +++ b/cpp/arcticdb/processing/component_manager.cpp @@ -27,6 +27,7 @@ void ComponentManager::decrement_entity_fetch_count(EntityId id) { // shared_mutex, so just decrement the ref count of the only sizeable component, so that when the shared pointer // goes out of scope in the calling function, the memory is freed registry_.get>(id).reset(); + ARCTICDB_DEBUG(log::memory(), "Releasing entity {}", id); debug::check(!registry_.get>(id), "SegmentInMemory memory retained in ComponentManager"); } diff --git a/cpp/arcticdb/processing/component_manager.hpp b/cpp/arcticdb/processing/component_manager.hpp index df204f443a..5736548250 100644 --- a/cpp/arcticdb/processing/component_manager.hpp +++ b/cpp/arcticdb/processing/component_manager.hpp @@ -54,6 +54,7 @@ class ComponentManager { std::vector add_entities(Args... args) { std::vector ids; size_t entity_count{0}; + ARCTICDB_SAMPLE_DEFAULT(AddEntities) std::unique_lock lock(mtx_); ([&]{ if (entity_count == 0) { @@ -79,6 +80,7 @@ class ComponentManager { template void replace_entities(const std::vector& ids, T value) { + ARCTICDB_SAMPLE_DEFAULT(ReplaceEntities) std::unique_lock lock(mtx_); for (auto id: ids) { registry_.replace(id, value); @@ -90,6 +92,7 @@ class ComponentManager { template void replace_entities(const std::vector& ids, const std::vector& values) { + ARCTICDB_SAMPLE_DEFAULT(ReplaceEntityValues) internal::check(ids.size() == values.size(), "Received vectors of differing lengths in ComponentManager::replace_entities"); std::unique_lock lock(mtx_); for (auto [idx, id]: folly::enumerate(ids)) { @@ -104,6 +107,7 @@ class ComponentManager { template std::tuple...> get_entities(const std::vector& ids, const bool decrement_fetch_count=true) { std::vector> tuple_res; + ARCTICDB_SAMPLE_DEFAULT(GetEntities) tuple_res.reserve(ids.size()); { std::shared_lock lock(mtx_); @@ -139,4 +143,18 @@ class ComponentManager { std::shared_mutex mtx_; }; -} // namespace arcticdb \ No newline at end of file +} // namespace arcticdb + +namespace fmt { +template<> +struct formatter { + template + constexpr auto parse(ParseContext& ctx) { return ctx.begin(); } + + template + auto format(const arcticdb::EntityId& id, FormatContext& ctx) const { + return fmt::format_to(ctx.out(), "{}", static_cast(id)); + } +}; + +} //namespace fmt \ No newline at end of file diff --git a/cpp/arcticdb/processing/test/test_parallel_processing.cpp b/cpp/arcticdb/processing/test/test_parallel_processing.cpp index 5d2cb54a1b..f914f5587c 100644 --- a/cpp/arcticdb/processing/test/test_parallel_processing.cpp +++ b/cpp/arcticdb/processing/test/test_parallel_processing.cpp @@ -144,6 +144,10 @@ TEST(Clause, ScheduleClauseProcessingStress) { processing_unit_indexes.emplace_back(std::vector{idx}); } + // Map from index in segment_and_slice_future_splitters to the number of calls to process in the first clause that + // will require that segment + 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), diff --git a/cpp/arcticdb/storage/async_storage.hpp b/cpp/arcticdb/storage/async_storage.hpp new file mode 100644 index 0000000000..e49da7f9b9 --- /dev/null +++ b/cpp/arcticdb/storage/async_storage.hpp @@ -0,0 +1,27 @@ +#pragma once + +#include +#include +#include +#include + +#include + + +namespace arcticdb::storage { +class AsyncStorage { +public: + folly::Future async_read(entity::VariantKey&& variant_key, const ReadVisitor& visitor, ReadKeyOpts opts) { + return do_async_read(std::move(variant_key), visitor, opts); + } + + folly::Future async_read(entity::VariantKey&& variant_key, ReadKeyOpts opts) { + return do_async_read(std::move(variant_key), opts); + } + +private: + virtual folly::Future do_async_read(entity::VariantKey&& variant_key, const ReadVisitor& visitor, ReadKeyOpts opts) = 0; + + virtual folly::Future do_async_read(entity::VariantKey&& variant_key, ReadKeyOpts opts) = 0; +}; +} // namespace arcticdb diff --git a/cpp/arcticdb/storage/azure/azure_real_client.cpp b/cpp/arcticdb/storage/azure/azure_client_impl.cpp similarity index 97% rename from cpp/arcticdb/storage/azure/azure_real_client.cpp rename to cpp/arcticdb/storage/azure/azure_client_impl.cpp index 7da760dd38..2d33d03bfa 100644 --- a/cpp/arcticdb/storage/azure/azure_real_client.cpp +++ b/cpp/arcticdb/storage/azure/azure_client_impl.cpp @@ -11,8 +11,8 @@ #include #include -#include -#include +#include +#include #include namespace arcticdb::storage { diff --git a/cpp/arcticdb/storage/azure/azure_real_client.hpp b/cpp/arcticdb/storage/azure/azure_client_impl.hpp similarity index 96% rename from cpp/arcticdb/storage/azure/azure_real_client.hpp rename to cpp/arcticdb/storage/azure/azure_client_impl.hpp index 9fc8116de0..6156b097ff 100644 --- a/cpp/arcticdb/storage/azure/azure_real_client.hpp +++ b/cpp/arcticdb/storage/azure/azure_client_impl.hpp @@ -12,7 +12,7 @@ #include #include -#include +#include namespace arcticdb::storage::azure { class RealAzureClient : public AzureClientWrapper { diff --git a/cpp/arcticdb/storage/azure/azure_client_wrapper.hpp b/cpp/arcticdb/storage/azure/azure_client_interface.hpp similarity index 100% rename from cpp/arcticdb/storage/azure/azure_client_wrapper.hpp rename to cpp/arcticdb/storage/azure/azure_client_interface.hpp diff --git a/cpp/arcticdb/storage/azure/azure_storage.cpp b/cpp/arcticdb/storage/azure/azure_storage.cpp index 1aef3183c5..0324b189d4 100644 --- a/cpp/arcticdb/storage/azure/azure_storage.cpp +++ b/cpp/arcticdb/storage/azure/azure_storage.cpp @@ -10,30 +10,25 @@ #include #include - #include -#include -#include #include #include -#include #include #include #include #include -#include #include -#include +#include +#include +#include +#include -#include #include #include -#include -#include -#include +#include #undef GetMessage @@ -43,8 +38,6 @@ using namespace object_store_utils; namespace azure { -namespace fg = folly::gen; - namespace detail { // TODO: fix this temporary workaround to read error code. azure-sdk-cpp client sometimes doesn't properly set the error code. @@ -52,9 +45,9 @@ namespace detail { std::string get_error_code(const Azure::Core::RequestFailedException& e) { auto error_code = e.ErrorCode; - if(error_code.empty() && e.RawResponse ) { + if (error_code.empty() && e.RawResponse) { auto headers_map = e.RawResponse->GetHeaders(); - if(auto ec = headers_map.find("x-ms-error-code") ; ec != headers_map.end()) { + if (auto ec = headers_map.find("x-ms-error-code"); ec != headers_map.end()) { error_code = ec->second; } } @@ -67,21 +60,23 @@ void raise_azure_exception(const Azure::Core::RequestFailedException& e, const s std::string error_message; auto error_message_suffix = fmt::format("AzureError#{} {}: {} {} for object {}", - static_cast(status_code), - error_code, - e.ReasonPhrase, - e.what(), - object_name); - - if(status_code == Azure::Core::Http::HttpStatusCode::NotFound && error_code == AzureErrorCode_to_string(AzureErrorCode::BlobNotFound)) { + static_cast(status_code), + error_code, + e.ReasonPhrase, + e.what(), + object_name); + + if (status_code == Azure::Core::Http::HttpStatusCode::NotFound + && error_code == AzureErrorCode_to_string(AzureErrorCode::BlobNotFound)) { throw KeyNotFoundException(fmt::format("Key Not Found Error: {}", error_message_suffix)); } - if(status_code == Azure::Core::Http::HttpStatusCode::Unauthorized || status_code == Azure::Core::Http::HttpStatusCode::Forbidden) { + if (status_code == Azure::Core::Http::HttpStatusCode::Unauthorized + || status_code == Azure::Core::Http::HttpStatusCode::Forbidden) { raise(fmt::format("Permission Error: {}", error_message_suffix)); } - if(static_cast(status_code) >= 500) { + if (static_cast(status_code) >= 500) { error_message = fmt::format("Unexpected Server Error: {}", error_message_suffix); } else { error_message = fmt::format("Unexpected Error: {}", error_message_suffix); @@ -91,69 +86,61 @@ void raise_azure_exception(const Azure::Core::RequestFailedException& e, const s } bool is_expected_error_type(const std::string& error_code, Azure::Core::Http::HttpStatusCode status_code) { - return status_code == Azure::Core::Http::HttpStatusCode::NotFound && (error_code == AzureErrorCode_to_string(AzureErrorCode::BlobNotFound) || - error_code == AzureErrorCode_to_string(AzureErrorCode::ContainerNotFound)); + return status_code == Azure::Core::Http::HttpStatusCode::NotFound + && (error_code == AzureErrorCode_to_string(AzureErrorCode::BlobNotFound) || + error_code == AzureErrorCode_to_string(AzureErrorCode::ContainerNotFound)); } void raise_if_unexpected_error(const Azure::Core::RequestFailedException& e, const std::string& object_name) { auto error_code = get_error_code(e); auto status_code = e.StatusCode; - if(!is_expected_error_type(error_code, status_code)) { + if (!is_expected_error_type(error_code, status_code)) { raise_azure_exception(e, object_name); } } template void do_write_impl( - Composite&& kvs, + KeySegmentPair&& key_seg, const std::string& root_folder, AzureClientWrapper& azure_client, KeyBucketizer&& bucketizer, const Azure::Storage::Blobs::UploadBlockBlobFromOptions& upload_option, unsigned int request_timeout) { - ARCTICDB_SAMPLE(AzureStorageWrite, 0) - auto fmt_db = [](auto&& kv) { return kv.key_type(); }; - - (fg::from(kvs.as_range()) | fg::move | fg::groupBy(fmt_db)).foreach( - [&azure_client, &root_folder, b=std::move(bucketizer), &upload_option, &request_timeout] (auto&& group) { - auto key_type_dir = key_type_folder(root_folder, group.key()); - ARCTICDB_TRACE(log::storage(), "Azure key_type_folder is {}", key_type_dir); - - ARCTICDB_SUBSAMPLE(AzureStorageWriteValues, 0) - for (auto& kv : group.values()) { - auto& k = kv.variant_key(); - auto blob_name = object_path(b.bucketize(key_type_dir, k), k); - auto& seg = kv.segment(); - - try { - azure_client.write_blob(blob_name, std::move(seg), upload_option, request_timeout); - } - catch (const Azure::Core::RequestFailedException& e) { - raise_azure_exception(e, blob_name); - } + ARCTICDB_SAMPLE(AzureStorageWrite, 0) - } - } - ); + auto key_type_dir = key_type_folder(root_folder, key_seg.key_type()); + ARCTICDB_TRACE(log::storage(), "Azure key_type_folder is {}", key_type_dir); + + ARCTICDB_SUBSAMPLE(AzureStorageWriteValues, 0) + auto& k = key_seg.variant_key(); + auto blob_name = object_path(bucketizer.bucketize(key_type_dir, k), k); + auto& seg = key_seg.segment(); + + try { + azure_client.write_blob(blob_name, std::move(seg), upload_option, request_timeout); + } + catch (const Azure::Core::RequestFailedException& e) { + raise_azure_exception(e, blob_name); + } } template void do_update_impl( - Composite&& kvs, + KeySegmentPair&& key_seg, const std::string& root_folder, AzureClientWrapper& azure_client, KeyBucketizer&& bucketizer, const Azure::Storage::Blobs::UploadBlockBlobFromOptions& upload_option, unsigned int request_timeout) { - // azure updates the key if it already exists - do_write_impl(std::move(kvs), root_folder, azure_client, std::move(bucketizer), upload_option, request_timeout); + // azure updates the key if it already exists + do_write_impl(std::move(key_seg), root_folder, azure_client, bucketizer, upload_option, request_timeout); } -struct UnexpectedAzureErrorException : public std::exception {}; - template -void do_read_impl(Composite && ks, +void do_read_impl( + VariantKey&& variant_key, const ReadVisitor& visitor, const std::string& root_folder, AzureClientWrapper& azure_client, @@ -161,144 +148,184 @@ void do_read_impl(Composite && ks, ReadKeyOpts opts, const Azure::Storage::Blobs::DownloadBlobToOptions& download_option, unsigned int request_timeout) { - ARCTICDB_SAMPLE(AzureStorageRead, 0) - auto fmt_db = [](auto&& k) { return variant_key_type(k); }; - std::vector failed_reads; - - (fg::from(ks.as_range()) | fg::move | fg::groupBy(fmt_db)).foreach( - [&azure_client, &root_folder, b=std::move(bucketizer), &visitor, &failed_reads, - opts=opts, &download_option, &request_timeout] (auto&& group) { - for (auto& k : group.values()) { - auto key_type_dir = key_type_folder(root_folder, variant_key_type(k)); - auto blob_name = object_path(b.bucketize(key_type_dir, k), k); - try { - visitor(k, azure_client.read_blob(blob_name, download_option, request_timeout)); - - ARCTICDB_DEBUG(log::storage(), "Read key {}: {}", variant_key_type(k), variant_key_view(k)); - } - catch (const Azure::Core::RequestFailedException& e) { - raise_if_unexpected_error(e, blob_name); - if (!opts.dont_warn_about_missing_key) { - log::storage().warn("Failed to read azure segment with key '{}' {} {}: {}", - k, - blob_name, - static_cast(e.StatusCode), - e.ReasonPhrase); - } - failed_reads.push_back(k); - } - } - }); - if(!failed_reads.empty()) - throw KeyNotFoundException(Composite{std::move(failed_reads)}); + ARCTICDB_SAMPLE(AzureStorageRead, 0) + std::optional failed_read; + + auto key_type_dir = key_type_folder(root_folder, variant_key_type(variant_key)); + auto blob_name = object_path(bucketizer.bucketize(key_type_dir, variant_key), variant_key); + try { + visitor(variant_key, azure_client.read_blob(blob_name, download_option, request_timeout)); + ARCTICDB_DEBUG(log::storage(), "Read key {}: {}", variant_key_type(variant_key), variant_key_view(variant_key)); + } + catch (const Azure::Core::RequestFailedException& e) { + raise_if_unexpected_error(e, blob_name); + if (!opts.dont_warn_about_missing_key) { + log::storage().warn("Failed to read azure segment with key '{}' {} {}: {}", + variant_key, + blob_name, + static_cast(e.StatusCode), + e.ReasonPhrase); + } + failed_read.emplace(variant_key); + } + if (failed_read) + throw KeyNotFoundException(*failed_read); } template -void do_remove_impl(Composite&& ks, +KeySegmentPair do_read_impl( + VariantKey&& variant_key, + const std::string& root_folder, + AzureClientWrapper& azure_client, + KeyBucketizer&& bucketizer, + ReadKeyOpts opts, + const Azure::Storage::Blobs::DownloadBlobToOptions& download_option, + unsigned int request_timeout) { + ARCTICDB_SAMPLE(AzureStorageRead, 0) + std::optional failed_read; + + auto key_type_dir = key_type_folder(root_folder, variant_key_type(variant_key)); + auto blob_name = object_path(bucketizer.bucketize(key_type_dir, variant_key), variant_key); + try { + return {VariantKey{variant_key}, azure_client.read_blob(blob_name, download_option, request_timeout)}; + ARCTICDB_DEBUG(log::storage(), "Read key {}: {}", variant_key_type(variant_key), variant_key_view(variant_key)); + } + catch (const Azure::Core::RequestFailedException& e) { + raise_if_unexpected_error(e, blob_name); + if (!opts.dont_warn_about_missing_key) { + log::storage().warn("Failed to read azure segment with key '{}' {} {}: {}", + variant_key, + blob_name, + static_cast(e.StatusCode), + e.ReasonPhrase); + } + throw KeyNotFoundException( + variant_key, + fmt::format("Failed to read azure segment with key '{}' {} {}: {}", + variant_key, + blob_name, + static_cast(e.StatusCode), + e.ReasonPhrase)); + } catch(const std::exception&) { + throw KeyNotFoundException(variant_key); + } + return KeySegmentPair{}; +} + +namespace fg = folly::gen; + +template +void do_remove_impl( + std::span variant_keys, const std::string& root_folder, AzureClientWrapper& azure_client, KeyBucketizer&& bucketizer, - unsigned int request_timeout) { - ARCTICDB_SUBSAMPLE(AzureStorageDeleteBatch, 0) - auto fmt_db = [](auto&& k) { return variant_key_type(k); }; - std::vector to_delete; - static const size_t delete_object_limit = - std::min(BATCH_SUBREQUEST_LIMIT, static_cast(ConfigsMap::instance()->get_int("AzureStorage.DeleteBatchSize", BATCH_SUBREQUEST_LIMIT))); - - auto submit_batch = [&azure_client, &request_timeout](auto &to_delete) { - try { - azure_client.delete_blobs(to_delete, request_timeout); - } catch (const Azure::Core::RequestFailedException& e) { - std::string failed_objects = fmt::format("{}", fmt::join(to_delete, ", ")); - raise_azure_exception(e, failed_objects); - } - to_delete.clear(); - }; - - (fg::from(ks.as_range()) | fg::move | fg::groupBy(fmt_db)).foreach( - [&root_folder, b=std::move(bucketizer), delete_object_limit=delete_object_limit, &to_delete, &submit_batch] (auto&& group) {//bypass incorrect 'set but no used" error for delete_object_limit - auto key_type_dir = key_type_folder(root_folder, group.key()); - for (auto k : folly::enumerate(group.values())) { - auto blob_name = object_path(b.bucketize(key_type_dir, *k), *k); - to_delete.emplace_back(std::move(blob_name)); - if (to_delete.size() == delete_object_limit) { - submit_batch(to_delete); - } + unsigned int request_timeout) { + ARCTICDB_SUBSAMPLE(AzureStorageDeleteBatch, 0) + auto fmt_db = [](auto&& k) { return variant_key_type(k); }; + std::vector to_delete; + static const size_t delete_object_limit = + std::min(BATCH_SUBREQUEST_LIMIT, static_cast(ConfigsMap::instance()->get_int("AzureStorage.DeleteBatchSize", BATCH_SUBREQUEST_LIMIT))); + + auto submit_batch = [&azure_client, &request_timeout](auto &to_delete) { + try { + azure_client.delete_blobs(to_delete, request_timeout); + } catch (const Azure::Core::RequestFailedException& e) { + std::string failed_objects = fmt::format("{}", fmt::join(to_delete, ", ")); + raise_azure_exception(e, failed_objects); + } + to_delete.clear(); + }; + + (fg::from(variant_keys) | fg::move | fg::groupBy(fmt_db)).foreach( + [&root_folder, b=std::move(bucketizer), delete_object_limit=delete_object_limit, &to_delete, &submit_batch] (auto&& group) {//bypass incorrect 'set but no used" error for delete_object_limit + auto key_type_dir = key_type_folder(root_folder, group.key()); + for (auto k : folly::enumerate(group.values())) { + auto blob_name = object_path(b.bucketize(key_type_dir, *k), *k); + to_delete.emplace_back(std::move(blob_name)); + if (to_delete.size() == delete_object_limit) { + submit_batch(to_delete); } } - ); - if (!to_delete.empty()) { - submit_batch(to_delete); } + ); + if (!to_delete.empty()) { + submit_batch(to_delete); + } } -std::string prefix_handler(const std::string& prefix, const std::string& key_type_dir, const KeyDescriptor& key_descriptor, KeyType) { +std::string prefix_handler(const std::string& prefix, + const std::string& key_type_dir, + const KeyDescriptor& key_descriptor, + KeyType) { return !prefix.empty() ? fmt::format("{}/{}*{}", key_type_dir, key_descriptor, prefix) : key_type_dir; } bool do_iterate_type_impl(KeyType key_type, - const IterateTypePredicate& visitor, - const std::string& root_folder, - AzureClientWrapper& azure_client, - const std::string& prefix = std::string{}) { - ARCTICDB_SAMPLE(AzureStorageIterateType, 0) - auto key_type_dir = key_type_folder(root_folder, key_type); - const auto path_to_key_size = key_type_dir.size() + 1; - // if prefix is empty, add / to avoid matching both log and logc when key_type_dir is {root_folder}/log - if (prefix.empty()) { - key_type_dir += "/"; - } - - KeyDescriptor key_descriptor(prefix, - is_ref_key_class(key_type) ? IndexDescriptorImpl::Type::UNKNOWN : IndexDescriptorImpl::Type::TIMESTAMP, FormatType::TOKENIZED); - auto key_prefix = prefix_handler(prefix, key_type_dir, key_descriptor, key_type); + const IterateTypePredicate& visitor, + const std::string& root_folder, + AzureClientWrapper& azure_client, + const std::string& prefix = std::string{}) { + ARCTICDB_SAMPLE(AzureStorageIterateType, 0) + auto key_type_dir = key_type_folder(root_folder, key_type); + const auto path_to_key_size = key_type_dir.size() + 1; + // if prefix is empty, add / to avoid matching both log and logc when key_type_dir is {root_folder}/log + if (prefix.empty()) { + key_type_dir += "/"; + } - try { - for (auto page = azure_client.list_blobs(key_prefix); page.HasPage(); page.MoveToNextPage()) { - for (const auto& blob : page.Blobs) { - auto key = blob.Name.substr(path_to_key_size); - ARCTICDB_TRACE(log::version(), "Got object_list: {}, key: {}", blob.Name, key); - auto k = variant_key_from_bytes( - reinterpret_cast(key.data()), - key.size(), - key_type); - ARCTICDB_DEBUG(log::storage(), "Iterating key {}: {}", variant_key_type(k), variant_key_view(k)); - ARCTICDB_SUBSAMPLE(AzureStorageVisitKey, 0) - if (visitor(std::move(k))) { - return true; - } - ARCTICDB_SUBSAMPLE(AzureStorageCursorNext, 0) + KeyDescriptor key_descriptor(prefix, + is_ref_key_class(key_type) ? IndexDescriptorImpl::Type::UNKNOWN + : IndexDescriptorImpl::Type::TIMESTAMP, + FormatType::TOKENIZED); + auto key_prefix = prefix_handler(prefix, key_type_dir, key_descriptor, key_type); + + try { + for (auto page = azure_client.list_blobs(key_prefix); page.HasPage(); page.MoveToNextPage()) { + for (const auto& blob : page.Blobs) { + auto key = blob.Name.substr(path_to_key_size); + ARCTICDB_TRACE(log::version(), "Got object_list: {}, key: {}", blob.Name, key); + auto k = variant_key_from_bytes( + reinterpret_cast(key.data()), + key.size(), + key_type); + ARCTICDB_DEBUG(log::storage(), "Iterating key {}: {}", variant_key_type(k), variant_key_view(k)); + ARCTICDB_SUBSAMPLE(AzureStorageVisitKey, 0) + if (visitor(std::move(k))) { + return true; } + ARCTICDB_SUBSAMPLE(AzureStorageCursorNext, 0) } } - catch (const Azure::Core::RequestFailedException& e) { - raise_if_unexpected_error(e, key_prefix); - log::storage().warn("Failed to iterate azure blobs '{}' {}: {}", - key_type, - static_cast(e.StatusCode), - e.ReasonPhrase); - } - return false; + } + catch (const Azure::Core::RequestFailedException& e) { + raise_if_unexpected_error(e, key_prefix); + log::storage().warn("Failed to iterate azure blobs '{}' {}: {}", + key_type, + static_cast(e.StatusCode), + e.ReasonPhrase); + } + return false; } bool do_key_exists_impl( const VariantKey& key, const std::string& root_folder, AzureClientWrapper& azure_client) { - auto key_type_dir = key_type_folder(root_folder, variant_key_type(key)); - auto blob_name = object_path(key_type_dir, key); - try { - return azure_client.blob_exists(blob_name); - } - catch (const Azure::Core::RequestFailedException& e) { - raise_if_unexpected_error(e, blob_name); - log::storage().debug("Failed to check azure key '{}' {} {}: {}", - key, - blob_name, - static_cast(e.StatusCode), - e.ReasonPhrase); - } - return false; + auto key_type_dir = key_type_folder(root_folder, variant_key_type(key)); + auto blob_name = object_path(key_type_dir, key); + try { + return azure_client.blob_exists(blob_name); + } + catch (const Azure::Core::RequestFailedException& e) { + raise_if_unexpected_error(e, blob_name); + log::storage().debug("Failed to check azure key '{}' {} {}: {}", + key, + blob_name, + static_cast(e.StatusCode), + e.ReasonPhrase); + } + return false; } } //namespace detail @@ -306,23 +333,57 @@ std::string AzureStorage::name() const { return fmt::format("azure_storage-{}/{}", container_name_, root_folder_); } -void AzureStorage::do_write(Composite&& kvs) { - detail::do_write_impl(std::move(kvs), root_folder_, *azure_client_, FlatBucketizer{}, upload_option_, request_timeout_); +void AzureStorage::do_write(KeySegmentPair&& key_seg) { + detail::do_write_impl(std::move(key_seg), + root_folder_, + *azure_client_, + FlatBucketizer{}, + upload_option_, + request_timeout_); +} + +void AzureStorage::do_update(KeySegmentPair&& key_seg, UpdateOpts) { + detail::do_update_impl(std::move(key_seg), + root_folder_, + *azure_client_, + FlatBucketizer{}, + upload_option_, + request_timeout_); +} + +void AzureStorage::do_read(VariantKey&& variant_key, const ReadVisitor& visitor, ReadKeyOpts opts) { + detail::do_read_impl(std::move(variant_key), + visitor, + root_folder_, + *azure_client_, + FlatBucketizer{}, + opts, + download_option_, + request_timeout_); } -void AzureStorage::do_update(Composite&& kvs, UpdateOpts) { - detail::do_update_impl(std::move(kvs), root_folder_, *azure_client_, FlatBucketizer{}, upload_option_, request_timeout_); +KeySegmentPair AzureStorage::do_read(VariantKey&& variant_key, ReadKeyOpts opts) { + return detail::do_read_impl(std::move(variant_key), + root_folder_, + *azure_client_, + FlatBucketizer{}, + opts, + download_option_, + request_timeout_); } -void AzureStorage::do_read(Composite&& ks, const ReadVisitor& visitor, ReadKeyOpts opts) { - detail::do_read_impl(std::move(ks), visitor, root_folder_, *azure_client_, FlatBucketizer{}, opts, download_option_, request_timeout_); +void AzureStorage::do_remove(VariantKey&& variant_key, RemoveOpts) { + std::array arr{std::move(variant_key)}; + detail::do_remove_impl(std::span(arr), root_folder_, *azure_client_, FlatBucketizer{}, request_timeout_); } -void AzureStorage::do_remove(Composite&& ks, RemoveOpts) { - detail::do_remove_impl(std::move(ks), root_folder_, *azure_client_, FlatBucketizer{}, request_timeout_); +void AzureStorage::do_remove(std::span variant_keys, RemoveOpts) { + detail::do_remove_impl(std::move(variant_keys), root_folder_, *azure_client_, FlatBucketizer{}, request_timeout_); } -bool AzureStorage::do_iterate_type_until_match(KeyType key_type, const IterateTypePredicate& visitor, const std::string &prefix) { +bool AzureStorage::do_iterate_type_until_match(KeyType key_type, + const IterateTypePredicate& visitor, + const std::string& prefix) { return detail::do_iterate_type_impl(key_type, visitor, root_folder_, *azure_client_, prefix); } @@ -335,45 +396,51 @@ bool AzureStorage::do_key_exists(const VariantKey& key) { } // namespace arcticdb::storage -namespace arcticdb::storage::azure{ +namespace arcticdb::storage::azure { using namespace Azure::Storage; using namespace Azure::Storage::Blobs; - -AzureStorage::AzureStorage(const LibraryPath &library_path, OpenMode mode, const Config &conf) : +AzureStorage::AzureStorage(const LibraryPath& library_path, OpenMode mode, const Config& conf) : Storage(library_path, mode), root_folder_(object_store_utils::get_root_folder(library_path)), container_name_(conf.container_name()), request_timeout_(conf.request_timeout() == 0 ? 200000 : conf.request_timeout()) { - if(conf.use_mock_storage_for_testing()) { - ARCTICDB_RUNTIME_DEBUG(log::storage(), "Using Mock Azure storage"); - azure_client_ = std::make_unique(); - } else { - ARCTICDB_RUNTIME_DEBUG(log::storage(), "Using Real Azure storage"); - azure_client_ = std::make_unique(conf); - } - if (conf.ca_cert_path().empty()) { - ARCTICDB_RUNTIME_DEBUG(log::storage(), "Using default CA cert path"); - } else { - ARCTICDB_RUNTIME_DEBUG(log::storage(), "CA cert path: {}", conf.ca_cert_path()); - } - if (conf.ca_cert_dir().empty()) { - ARCTICDB_RUNTIME_DEBUG(log::storage(), "Using default CA cert directory"); - } else { - ARCTICDB_RUNTIME_DEBUG(log::storage(), "CA cert directory: {}", conf.ca_cert_dir()); - } - ARCTICDB_RUNTIME_DEBUG(log::storage(), "Connecting to Azure Blob Storage: {} Container: {}", conf.endpoint(), conf.container_name()); - - if (!conf.prefix().empty()) { - ARCTICDB_RUNTIME_DEBUG(log::storage(), "Azure prefix found, using: {}", conf.prefix()); - auto prefix_path = LibraryPath::from_delim_path(conf.prefix(), '.'); - root_folder_ = object_store_utils::get_root_folder(prefix_path); - } else - ARCTICDB_RUNTIME_DEBUG(log::storage(), "Azure prefix not found, will use {}", root_folder_); - unsigned int max_connections = conf.max_connections() == 0 ? ConfigsMap::instance()->get_int("VersionStore.NumIOThreads", 16) : conf.max_connections(); - upload_option_.TransferOptions.Concurrency = max_connections; - download_option_.TransferOptions.Concurrency = max_connections; + if (conf.use_mock_storage_for_testing()) { + ARCTICDB_RUNTIME_DEBUG(log::storage(), "Using Mock Azure storage"); + azure_client_ = std::make_unique(); + } else { + ARCTICDB_RUNTIME_DEBUG(log::storage(), "Using Real Azure storage"); + azure_client_ = std::make_unique(conf); + } + if (conf.ca_cert_path().empty()) { + ARCTICDB_RUNTIME_DEBUG(log::storage(), "Using default CA cert path"); + } else { + ARCTICDB_RUNTIME_DEBUG(log::storage(), "CA cert path: {}", conf.ca_cert_path()); + } + if (conf.ca_cert_dir().empty()) { + ARCTICDB_RUNTIME_DEBUG(log::storage(), "Using default CA cert directory"); + } else { + ARCTICDB_RUNTIME_DEBUG(log::storage(), "CA cert directory: {}", conf.ca_cert_dir()); + } + ARCTICDB_RUNTIME_DEBUG(log::storage(), + "Connecting to Azure Blob Storage: {} Container: {}", + conf.endpoint(), + conf.container_name()); + + if (!conf.prefix().empty()) { + ARCTICDB_RUNTIME_DEBUG(log::storage(), "Azure prefix found, using: {}", conf.prefix()); + auto prefix_path = LibraryPath::from_delim_path(conf.prefix(), '.'); + root_folder_ = object_store_utils::get_root_folder(prefix_path); + } else { + ARCTICDB_RUNTIME_DEBUG(log::storage(), "Azure prefix not found, will use {}", root_folder_); + } + + unsigned int max_connections = + conf.max_connections() == 0 ? ConfigsMap::instance()->get_int("VersionStore.NumIOThreads", 16) + : conf.max_connections(); + upload_option_.TransferOptions.Concurrency = static_cast(max_connections); + download_option_.TransferOptions.Concurrency = static_cast(max_connections); } } // namespace arcticdb::storage::azure diff --git a/cpp/arcticdb/storage/azure/azure_storage.hpp b/cpp/arcticdb/storage/azure/azure_storage.hpp index 1c679cfab6..803f83d6d3 100644 --- a/cpp/arcticdb/storage/azure/azure_storage.hpp +++ b/cpp/arcticdb/storage/azure/azure_storage.hpp @@ -10,7 +10,7 @@ #include #include #include -#include +#include #include #include #include @@ -35,17 +35,21 @@ class AzureStorage final : public Storage { std::string name() const final; protected: - void do_write(Composite&& kvs) final; + void do_write(KeySegmentPair&& key_seg) final; void do_write_if_none(KeySegmentPair&& kv [[maybe_unused]]) final { storage::raise("Atomic operations are only supported for s3 backend"); }; - void do_update(Composite&& kvs, UpdateOpts opts) final; + void do_update(KeySegmentPair&& key_seg, UpdateOpts opts) final; - void do_read(Composite&& ks, const ReadVisitor& visitor, ReadKeyOpts opts) final; + void do_read(VariantKey&& variant_key, const ReadVisitor& visitor, ReadKeyOpts opts) final; - void do_remove(Composite&& ks, RemoveOpts opts) final; + KeySegmentPair do_read(VariantKey&& variant_key, ReadKeyOpts opts) final; + + void do_remove(VariantKey&& variant_key, RemoveOpts opts) final; + + void do_remove(std::span variant_keys, RemoveOpts opts) final; bool do_iterate_type_until_match(KeyType key_type, const IterateTypePredicate& visitor, const std::string &prefix) final; diff --git a/cpp/arcticdb/storage/common.hpp b/cpp/arcticdb/storage/common.hpp index 0cd1c42aac..63d04d1662 100644 --- a/cpp/arcticdb/storage/common.hpp +++ b/cpp/arcticdb/storage/common.hpp @@ -16,22 +16,26 @@ #include #include +namespace arcticdb { +class Segment; +} + namespace arcticdb::storage { +using ReadVisitor = std::function; -struct EnvironmentNameTag{}; +struct EnvironmentNameTag {}; using EnvironmentName = util::StringWrappingValue; -struct StorageNameTag{}; +struct StorageNameTag {}; using StorageName = util::StringWrappingValue; -struct InstanceUriTag{}; +struct InstanceUriTag {}; using InstanceUri = util::StringWrappingValue; -template -requires std::is_same_v || std::is_same_v -bool operator==(const T &l, const T &r) { - return l.value == r.value; +template requires std::is_same_v || std::is_same_v +bool operator==(const T& l, const T& r) { + return l.value == r.value; } /* @@ -46,17 +50,13 @@ struct LibraryDescriptor { std::vector storage_ids_; using VariantStoreConfig = std::variant< - std::monostate, // make variant default constructible and unconfigured - arcticdb::proto::storage::VersionStoreConfig - >; + std::monostate, // make variant default constructible and unconfigured + arcticdb::proto::storage::VersionStoreConfig + >; - VariantStoreConfig config_ = std::monostate{}; + VariantStoreConfig config_ = std::monostate{}; }; -inline std::vector stream_to_vector(std::vector &src) { - return src; -} - inline size_t get_stream_length(std::iostream& src) { src.seekg(0, std::ios::end); auto len = src.tellg(); @@ -64,33 +64,14 @@ inline size_t get_stream_length(std::iostream& src) { return static_cast(len); } -inline std::vector stream_to_vector(std::iostream &src) -{ +inline std::vector stream_to_vector(std::iostream& src) { ARCTICDB_SAMPLE(StreamToVector, 0) auto len = get_stream_length(src); std::vector v(len); - src.read(v.data(), len); + src.read(v.data(), static_cast(len)); return v; } -inline void vector_to_stream(std::vector& src, std::stringstream& output) { - output.write(src.data(), src.size()); -} - -template -struct is_key_type : std::false_type {}; - -template<> -struct is_key_type : std::true_type {}; - -template<> -struct is_key_type : std::true_type {}; - -template<> -struct is_key_type : std::true_type {}; - -template -inline constexpr bool is_key_type_v = is_key_type::value; class NativeVariantStorage { public: @@ -105,4 +86,5 @@ class NativeVariantStorage { private: VariantStorageConfig config_; }; -} //namespace arcticdb::storage \ No newline at end of file + +} //namespace arcticdb::storage diff --git a/cpp/arcticdb/storage/file/mapped_file_storage.cpp b/cpp/arcticdb/storage/file/mapped_file_storage.cpp index 3f04003bf2..68f1accaae 100644 --- a/cpp/arcticdb/storage/file/mapped_file_storage.cpp +++ b/cpp/arcticdb/storage/file/mapped_file_storage.cpp @@ -85,30 +85,32 @@ uint64_t MappedFileStorage::write_segment(Segment&& seg) { return offset; } -void MappedFileStorage::do_write(Composite&& kvs) { +void MappedFileStorage::do_write(KeySegmentPair&& key_seg) { ARCTICDB_SAMPLE(MappedFileStorageWriteValues, 0) - auto key_values = std::move(kvs); - key_values.broadcast([this] (auto key_seg) { - const auto offset = write_segment(std::move(key_seg.segment())); - const auto size = key_seg.segment().size(); - multi_segment_header_.add_key_and_offset(key_seg.atom_key(), offset, size); - }); + const auto offset = write_segment(std::move(key_seg.segment())); + const auto size = key_seg.segment().size(); + multi_segment_header_.add_key_and_offset(key_seg.atom_key(), offset, size); } -void MappedFileStorage::do_update(Composite&&, UpdateOpts) { +void MappedFileStorage::do_update(KeySegmentPair&&, UpdateOpts) { util::raise_rte("Update not implemented for file storages"); } -void MappedFileStorage::do_read(Composite&& ks, const ReadVisitor& visitor, storage::ReadKeyOpts) { +void MappedFileStorage::do_read(VariantKey&& variant_key, const ReadVisitor& visitor, storage::ReadKeyOpts) { ARCTICDB_SAMPLE(MappedFileStorageRead, 0) - auto keys = std::move(ks); - keys.broadcast([&visitor, this] (const auto& key) { - auto maybe_offset = multi_segment_header_.get_offset_for_key(to_atom(key)); - util::check(maybe_offset.has_value(), "Failed to find key {} in file", key); + auto maybe_offset = multi_segment_header_.get_offset_for_key(to_atom(variant_key)); + util::check(maybe_offset.has_value(), "Failed to find key {} in file", variant_key); auto [offset, bytes] = std::move(maybe_offset.value()); auto segment = Segment::from_bytes(file_.data() + offset, bytes); - visitor(key, std::move(segment)); - }); + visitor(variant_key, std::move(segment)); +} + +KeySegmentPair MappedFileStorage::do_read(VariantKey&& variant_key, storage::ReadKeyOpts) { + ARCTICDB_SAMPLE(MappedFileStorageRead, 0) + auto maybe_offset = multi_segment_header_.get_offset_for_key(to_atom(variant_key)); + util::check(maybe_offset.has_value(), "Failed to find key {} in file", variant_key); + auto [offset, bytes] = std::move(maybe_offset.value()); + return {std::move(variant_key), Segment::from_bytes(file_.data() + offset, bytes)}; } bool MappedFileStorage::do_key_exists(const VariantKey& key) { @@ -116,7 +118,11 @@ bool MappedFileStorage::do_key_exists(const VariantKey& key) { return multi_segment_header_.get_offset_for_key(to_atom(key)) != std::nullopt; } -void MappedFileStorage::do_remove(Composite&&, RemoveOpts) { +void MappedFileStorage::do_remove(VariantKey&&, RemoveOpts) { + util::raise_rte("Remove not implemented for file storages"); +} + +void MappedFileStorage::do_remove(std::span, RemoveOpts) { util::raise_rte("Remove not implemented for file storages"); } diff --git a/cpp/arcticdb/storage/file/mapped_file_storage.hpp b/cpp/arcticdb/storage/file/mapped_file_storage.hpp index ee4b395462..ce9335b28b 100644 --- a/cpp/arcticdb/storage/file/mapped_file_storage.hpp +++ b/cpp/arcticdb/storage/file/mapped_file_storage.hpp @@ -34,17 +34,21 @@ class MappedFileStorage final : public SingleFileStorage { private: void do_write_raw(const uint8_t* data, size_t bytes) override; - void do_write(Composite&& kvs) override; + void do_write(KeySegmentPair&& key_seg) override; void do_write_if_none(KeySegmentPair&& kv [[maybe_unused]]) final { storage::raise("Atomic operations are only supported for s3 backend"); }; - void do_update(Composite&& kvs, UpdateOpts opts) override; + void do_update(KeySegmentPair&& key_seg, UpdateOpts opts) override; - void do_read(Composite&& ks, const ReadVisitor& visitor, storage::ReadKeyOpts opts) override; + void do_read(VariantKey&& variant_key, const ReadVisitor& visitor, storage::ReadKeyOpts opts) override; - void do_remove(Composite&& ks, RemoveOpts opts) override; + KeySegmentPair do_read(VariantKey&& variant_key, ReadKeyOpts) final; + + void do_remove(VariantKey&& variant_key, RemoveOpts opts) override; + + void do_remove(std::span variant_keys, RemoveOpts opts) final; bool do_supports_prefix_matching() const override { return false; diff --git a/cpp/arcticdb/storage/key_segment_pair.hpp b/cpp/arcticdb/storage/key_segment_pair.hpp index 7baa92e6ed..2ef090f0e0 100644 --- a/cpp/arcticdb/storage/key_segment_pair.hpp +++ b/cpp/arcticdb/storage/key_segment_pair.hpp @@ -21,6 +21,7 @@ namespace arcticdb::storage { public: KeySegmentPair() = default; + explicit KeySegmentPair(VariantKey &&key) : key_(std::make_shared(std::move(key))) {} diff --git a/cpp/arcticdb/storage/library.hpp b/cpp/arcticdb/storage/library.hpp index 6b3d58837c..41e9caf12e 100644 --- a/cpp/arcticdb/storage/library.hpp +++ b/cpp/arcticdb/storage/library.hpp @@ -81,13 +81,13 @@ class Library { return storages_->scan_for_matching_key(key_type, predicate); } - void write(Composite&& kvs) { + void write(KeySegmentPair&& key_seg) { ARCTICDB_SAMPLE(LibraryWrite, 0) if (open_mode() < OpenMode::WRITE) { throw LibraryPermissionException(library_path_, open_mode(), "write"); } - storages_->write(std::move(kvs)); + storages_->write(std::move(key_seg)); } void write_if_none(KeySegmentPair&& kv) { @@ -98,29 +98,52 @@ class Library { storages_->write_if_none(std::move(kv)); } - void update(Composite&& kvs, storage::UpdateOpts opts) { + void update(KeySegmentPair&& key_seg, storage::UpdateOpts opts) { ARCTICDB_SAMPLE(LibraryUpdate, 0) if (open_mode() < OpenMode::WRITE) throw LibraryPermissionException(library_path_, open_mode(), "update"); - storages_->update(std::move(kvs), opts); + storages_->update(std::move(key_seg), opts); } - void read(Composite&& ks, const ReadVisitor& visitor, ReadKeyOpts opts) { + folly::Future read(VariantKey&& variant_key, const ReadVisitor& visitor, ReadKeyOpts opts) { ARCTICDB_SAMPLE(LibraryRead, 0) - storages_->read(std::move(ks), visitor, opts, !storage_fallthrough_); + return storages_->read(std::move(variant_key), visitor, opts, !storage_fallthrough_); } - void remove(Composite&& ks, storage::RemoveOpts opts) { + folly::Future read(VariantKey variant_key, ReadKeyOpts opts = ReadKeyOpts{}) { + return storages_->read(std::move(variant_key), opts); + } + + void read_sync(VariantKey&& variant_key, const ReadVisitor& visitor, ReadKeyOpts opts) { + ARCTICDB_SAMPLE(LibraryRead, 0) + storages_->read_sync(variant_key, visitor, opts, !storage_fallthrough_); + } + + KeySegmentPair read_sync(const VariantKey& key, ReadKeyOpts opts = ReadKeyOpts{}) { + util::check(!std::holds_alternative(variant_key_id(key)) || !std::get(variant_key_id(key)).empty(), "Unexpected empty id"); + return storages_->read_sync(key, opts, !storage_fallthrough_); + } + + void remove(std::span variant_keys, storage::RemoveOpts opts) { + if (open_mode() < arcticdb::storage::OpenMode::DELETE) { + throw LibraryPermissionException(library_path_, open_mode(), "delete"); + } + + ARCTICDB_SAMPLE(LibraryRemove, 0) + storages_->remove(variant_keys, opts); + } + + void remove(VariantKey&& variant_key, storage::RemoveOpts opts) { if (open_mode() < arcticdb::storage::OpenMode::DELETE) { throw LibraryPermissionException(library_path_, open_mode(), "delete"); } ARCTICDB_SAMPLE(LibraryRemove, 0) - storages_->remove(std::move(ks), opts); + storages_->remove(std::move(variant_key), opts); } - std::optional> get_single_file_storage() const { + [[nodiscard]] std::optional> get_single_file_storage() const { return storages_->get_single_file_storage(); } @@ -136,24 +159,12 @@ class Library { return storages_->key_exists(key); } - bool is_path_valid(const std::string_view path) const { + [[nodiscard]] bool is_path_valid(const std::string_view path) const { return storages_->is_path_valid(path); } - KeySegmentPair read(VariantKey key, ReadKeyOpts opts = ReadKeyOpts{}) { - KeySegmentPair res{VariantKey{key}}; - util::check(!std::holds_alternative(variant_key_id(key)) || !std::get(variant_key_id(key)).empty(), "Unexpected empty id"); - const ReadVisitor& visitor = [&res](const VariantKey&, Segment&& value) { - res.segment() = std::move(value); - }; - - read(Composite(std::move(key)), visitor, opts); - - return res; - } - /** Calls VariantStorage::do_key_path on the primary storage */ - std::string key_path(const VariantKey& key) const { + [[nodiscard]] std::string key_path(const VariantKey& key) const { return storages_->key_path(key); } @@ -161,17 +172,17 @@ class Library { storages_->move_storage(key_type, horizon, storage_index); } - bool supports_prefix_matching() const { return storages_->supports_prefix_matching(); } + [[nodiscard]] bool supports_prefix_matching() const { return storages_->supports_prefix_matching(); } bool supports_atomic_writes() const { return storages_->supports_atomic_writes(); } - const LibraryPath &library_path() const { return library_path_; } + [[nodiscard]] const LibraryPath &library_path() const { return library_path_; } - OpenMode open_mode() const { return storages_->open_mode(); } + [[nodiscard]] OpenMode open_mode() const { return storages_->open_mode(); } - const auto & config() const { return config_;} + [[nodiscard]] const auto & config() const { return config_;} - void set_failure_sim(const arcticdb::proto::storage::VersionStoreConfig::StorageFailureSimulator& cfg) { + static void set_failure_sim(const arcticdb::proto::storage::VersionStoreConfig::StorageFailureSimulator& cfg) { StorageFailureSimulator::instance()->configure(cfg); } diff --git a/cpp/arcticdb/storage/library_manager.cpp b/cpp/arcticdb/storage/library_manager.cpp index 8da055906e..403283dc14 100644 --- a/cpp/arcticdb/storage/library_manager.cpp +++ b/cpp/arcticdb/storage/library_manager.cpp @@ -245,6 +245,13 @@ std::vector LibraryManager::get_library_paths() const { } bool LibraryManager::has_library(const LibraryPath& path) const { + { + std::lock_guard lock{open_libraries_mutex_}; + if (auto cached = open_libraries_.get(path); cached) { + return true; + } + } + return store_->key_exists_sync(RefKey{StreamId(path.to_delim_path()), entity::KeyType::LIBRARY_CONFIG}); } diff --git a/cpp/arcticdb/storage/library_manager.hpp b/cpp/arcticdb/storage/library_manager.hpp index 2a0597b521..62f2746246 100644 --- a/cpp/arcticdb/storage/library_manager.hpp +++ b/cpp/arcticdb/storage/library_manager.hpp @@ -71,7 +71,7 @@ namespace arcticdb::storage { std::shared_ptr store_; LRUCache> open_libraries_; - std::mutex open_libraries_mutex_; // for open_libraries_ + mutable std::mutex open_libraries_mutex_; // for open_libraries_ }; } diff --git a/cpp/arcticdb/storage/lmdb/lmdb_real_client.cpp b/cpp/arcticdb/storage/lmdb/lmdb_client_impl.cpp similarity index 98% rename from cpp/arcticdb/storage/lmdb/lmdb_real_client.cpp rename to cpp/arcticdb/storage/lmdb/lmdb_client_impl.cpp index 547abcf114..dbbff11a22 100644 --- a/cpp/arcticdb/storage/lmdb/lmdb_real_client.cpp +++ b/cpp/arcticdb/storage/lmdb/lmdb_client_impl.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include diff --git a/cpp/arcticdb/storage/lmdb/lmdb_real_client.hpp b/cpp/arcticdb/storage/lmdb/lmdb_client_impl.hpp similarity index 96% rename from cpp/arcticdb/storage/lmdb/lmdb_real_client.hpp rename to cpp/arcticdb/storage/lmdb/lmdb_client_impl.hpp index 279ed5f006..98fa4b84c1 100644 --- a/cpp/arcticdb/storage/lmdb/lmdb_real_client.hpp +++ b/cpp/arcticdb/storage/lmdb/lmdb_client_impl.hpp @@ -7,7 +7,7 @@ #pragma once -#include +#include #include #include diff --git a/cpp/arcticdb/storage/lmdb/lmdb_client_wrapper.hpp b/cpp/arcticdb/storage/lmdb/lmdb_client_interface.hpp similarity index 100% rename from cpp/arcticdb/storage/lmdb/lmdb_client_wrapper.hpp rename to cpp/arcticdb/storage/lmdb/lmdb_client_interface.hpp diff --git a/cpp/arcticdb/storage/lmdb/lmdb_storage.cpp b/cpp/arcticdb/storage/lmdb/lmdb_storage.cpp index fbd40a7503..5844372912 100644 --- a/cpp/arcticdb/storage/lmdb/lmdb_storage.cpp +++ b/cpp/arcticdb/storage/lmdb/lmdb_storage.cpp @@ -6,8 +6,8 @@ */ #include -#include -#include +#include +#include #include @@ -23,6 +23,7 @@ #include #include +#include namespace arcticdb::storage::lmdb { @@ -33,22 +34,19 @@ struct LmdbKeepalive { LmdbKeepalive( std::shared_ptr instance, std::shared_ptr<::lmdb::txn> transaction - ) : - instance_(std::move(instance)), - transaction_(std::move(transaction)) { + ) : + instance_(std::move(instance)), + transaction_(std::move(transaction)) { } }; - -namespace fg = folly::gen; - static void raise_lmdb_exception(const ::lmdb::error& e, const std::string& object_name) { auto error_code = e.code(); auto error_message_suffix = fmt::format("LMDBError#{}: {} for object {}", - error_code, - e.what(), - object_name); + error_code, + e.what(), + object_name); if (error_code == MDB_NOTFOUND) { throw KeyNotFoundException(fmt::format("Key Not Found Error: {}", error_message_suffix)); @@ -66,7 +64,7 @@ static void raise_lmdb_exception(const ::lmdb::error& e, const std::string& obje } ::lmdb::env& LmdbStorage::env() { - storage::check( + storage::check( lmdb_instance_, "Unexpected LMDB Error: Invalid operation: LMDB environment has been removed. Possibly because the library has been deleted"); return lmdb_instance_->env_; @@ -79,106 +77,125 @@ ::lmdb::dbi& LmdbStorage::get_dbi(const std::string& db_name) { return *(lmdb_instance_->dbi_by_key_type_.at(db_name)); } -void LmdbStorage::do_write_internal(Composite&& kvs, ::lmdb::txn& txn) { - auto fmt_db = [](auto &&kv) { return kv.key_type(); }; - - (fg::from(kvs.as_range()) | fg::move | fg::groupBy(fmt_db)).foreach([&](auto &&group) { - auto db_name = fmt::format("{}", group.key()); +void LmdbStorage::do_write_internal(KeySegmentPair&& key_seg, ::lmdb::txn& txn) { + ARCTICDB_SUBSAMPLE(LmdbStorageOpenDb, 0) - ARCTICDB_SUBSAMPLE(LmdbStorageOpenDb, 0) - ::lmdb::dbi& dbi = get_dbi(db_name); + auto db_name = fmt::format(FMT_COMPILE("{}"), key_seg.key_type()); + ::lmdb::dbi& dbi = get_dbi(db_name); - ARCTICDB_SUBSAMPLE(LmdbStorageWriteValues, 0) - for (auto &kv : group.values()) { - ARCTICDB_DEBUG(log::storage(), "Lmdb storage writing segment with key {}", kv.key_view()); - auto k = to_serialized_key(kv.variant_key()); - auto &seg = kv.segment(); - int64_t overwrite_flag = std::holds_alternative(kv.variant_key()) ? 0 : MDB_NOOVERWRITE; - try { - lmdb_client_->write(db_name, k, std::move(seg), txn, dbi, overwrite_flag); - } catch (const ::lmdb::key_exist_error& e) { - throw DuplicateKeyException(fmt::format("Key already exists: {}: {}", kv.variant_key(), e.what())); - } catch (const ::lmdb::error& ex) { - raise_lmdb_exception(ex, k); - } - } - }); + ARCTICDB_SUBSAMPLE(LmdbStorageWriteValues, 0) + ARCTICDB_DEBUG(log::storage(), "Lmdb storage writing segment with key {}", key_seg.key_view()); + auto k = to_serialized_key(key_seg.variant_key()); + auto& seg = key_seg.segment(); + int64_t overwrite_flag = std::holds_alternative(key_seg.variant_key()) ? 0 : MDB_NOOVERWRITE; + try { + lmdb_client_->write(db_name, k, std::move(seg), txn, dbi, overwrite_flag); + } catch (const ::lmdb::key_exist_error& e) { + throw DuplicateKeyException(fmt::format("Key already exists: {}: {}", key_seg.variant_key(), e.what())); + } catch (const ::lmdb::error& ex) { + raise_lmdb_exception(ex, k); + } } std::string LmdbStorage::name() const { return fmt::format("lmdb_storage-{}", lib_dir_.string()); } -void LmdbStorage::do_write(Composite&& kvs) { +void LmdbStorage::do_write(KeySegmentPair&& key_seg) { ARCTICDB_SAMPLE(LmdbStorageWrite, 0) std::lock_guard lock{*write_mutex_}; auto txn = ::lmdb::txn::begin(env()); // scoped abort on exception, so no partial writes ARCTICDB_SUBSAMPLE(LmdbStorageInTransaction, 0) - do_write_internal(std::move(kvs), txn); + do_write_internal(std::move(key_seg), txn); ARCTICDB_SUBSAMPLE(LmdbStorageCommit, 0) txn.commit(); } -void LmdbStorage::do_update(Composite&& kvs, UpdateOpts opts) { +void LmdbStorage::do_update(KeySegmentPair&& key_seg, UpdateOpts opts) { ARCTICDB_SAMPLE(LmdbStorageUpdate, 0) std::lock_guard lock{*write_mutex_}; auto txn = ::lmdb::txn::begin(env()); ARCTICDB_SUBSAMPLE(LmdbStorageInTransaction, 0) - auto keys = kvs.transform([](const auto& kv){return kv.variant_key();}); + auto key = key_seg.variant_key(); // Deleting keys (no error is thrown if the keys already exist) RemoveOpts remove_opts; remove_opts.ignores_missing_key_ = opts.upsert_; - auto failed_deletes = do_remove_internal(std::move(keys), txn, remove_opts); - if(!failed_deletes.empty()) { + std::array arr{std::move(key)}; + auto failed_deletes = do_remove_internal(std::span(arr), txn, remove_opts); + if (!failed_deletes.empty()) { ARCTICDB_SUBSAMPLE(LmdbStorageCommit, 0) txn.commit(); std::string err_message = fmt::format("do_update called with upsert=false on non-existent key(s): {}", failed_deletes); - throw KeyNotFoundException(Composite(std::move(failed_deletes)), err_message); + throw KeyNotFoundException(failed_deletes, err_message); } - do_write_internal(std::move(kvs), txn); + do_write_internal(std::move(key_seg), txn); ARCTICDB_SUBSAMPLE(LmdbStorageCommit, 0) txn.commit(); } -void LmdbStorage::do_read(Composite&& ks, const ReadVisitor& visitor, storage::ReadKeyOpts) { - ARCTICDB_SAMPLE(LmdbStorageRead, 0) - - auto fmt_db = [](auto &&k) { return variant_key_type(k); }; - std::vector failed_reads; +KeySegmentPair LmdbStorage::do_read(VariantKey&& variant_key, ReadKeyOpts) { + ARCTICDB_SAMPLE(LmdbStorageReadReturn, 0) + std::optional failed_read; + auto db_name = fmt::format(FMT_COMPILE("{}"), variant_key_type(variant_key)); + ::lmdb::dbi& dbi = get_dbi(db_name); + ARCTICDB_SUBSAMPLE(LmdbStorageOpenDb, 0) + auto stored_key = to_serialized_key(variant_key); + try { + auto txn = std::make_shared<::lmdb::txn>(::lmdb::txn::begin(env(), nullptr, MDB_RDONLY)); + ARCTICDB_SUBSAMPLE(LmdbStorageInTransaction, 0) + auto segment = lmdb_client_->read(db_name, stored_key, *txn, dbi); + + if (segment.has_value()) { + ARCTICDB_SUBSAMPLE(LmdbStorageVisitSegment, 0) + segment->set_keepalive(std::any{LmdbKeepalive{lmdb_instance_, std::move(txn)}}); + ARCTICDB_DEBUG(log::storage(), "Read key {}: {}, with {} bytes of data",variant_key_type(variant_key), variant_key_view(variant_key), segment->size()); + return {VariantKey{variant_key}, std::move(*segment)}; + } else { + ARCTICDB_DEBUG(log::storage(), "Failed to find segment for key {}", variant_key_view(variant_key)); + throw KeyNotFoundException(variant_key); + } + } catch (const ::lmdb::not_found_error&) { + ARCTICDB_DEBUG(log::storage(), "Failed to find segment for key {}", variant_key_view(variant_key)); + throw KeyNotFoundException(variant_key); + } catch (const ::lmdb::error& ex) { + raise_lmdb_exception(ex, stored_key); + } + return KeySegmentPair{}; +} - (fg::from(ks.as_range()) | fg::move | fg::groupBy(fmt_db)).foreach([&](auto &&group) { - auto db_name = fmt::format("{}", group.key()); - ARCTICDB_SUBSAMPLE(LmdbStorageOpenDb, 0) - ::lmdb::dbi& dbi = get_dbi(db_name); - for (auto &k : group.values()) { - auto stored_key = to_serialized_key(k); - try { - auto txn = std::make_shared<::lmdb::txn>(::lmdb::txn::begin(env(), nullptr, MDB_RDONLY)); - ARCTICDB_SUBSAMPLE(LmdbStorageInTransaction, 0) - auto segment = lmdb_client_->read(db_name, stored_key, *txn, dbi); - - if (segment.has_value()) { - ARCTICDB_SUBSAMPLE(LmdbStorageVisitSegment, 0) - segment->set_keepalive(std::any{LmdbKeepalive{lmdb_instance_, std::move(txn)}}); - ARCTICDB_DEBUG(log::storage(), "Read key {}: {}, with {} bytes of data", variant_key_type(k), variant_key_view(k), segment->size()); - visitor(k, std::move(*segment)); - } else { - ARCTICDB_DEBUG(log::storage(), "Failed to find segment for key {}", variant_key_view(k)); - failed_reads.push_back(k); - } - } catch (const ::lmdb::not_found_error&) { - ARCTICDB_DEBUG(log::storage(), "Failed to find segment for key {}", variant_key_view(k)); - failed_reads.push_back(k); - } catch (const ::lmdb::error& ex) { - raise_lmdb_exception(ex, stored_key); - } +void LmdbStorage::do_read(VariantKey&& key, const ReadVisitor& visitor, storage::ReadKeyOpts) { + ARCTICDB_SAMPLE(LmdbStorageRead, 0) + std::optional failed_read; + auto db_name = fmt::format(FMT_COMPILE("{}"), variant_key_type(key)); + ::lmdb::dbi& dbi = get_dbi(db_name); + ARCTICDB_SUBSAMPLE(LmdbStorageOpenDb, 0) + auto stored_key = to_serialized_key(key); + try { + auto txn = std::make_shared<::lmdb::txn>(::lmdb::txn::begin(env(), nullptr, MDB_RDONLY)); + ARCTICDB_SUBSAMPLE(LmdbStorageInTransaction, 0) + auto segment = lmdb_client_->read(db_name, stored_key, *txn, dbi); + + if (segment.has_value()) { + ARCTICDB_SUBSAMPLE(LmdbStorageVisitSegment, 0) + segment->set_keepalive(std::any{LmdbKeepalive{lmdb_instance_, std::move(txn)}}); + ARCTICDB_DEBUG(log::storage(), "Read key {}: {}, with {} bytes of data",variant_key_type(key), variant_key_view(key), segment->size()); + visitor(key, std::move(*segment)); + } else { + ARCTICDB_DEBUG(log::storage(), "Failed to find segment for key {}", variant_key_view(key)); + failed_read.emplace(key); } - }); - if(!failed_reads.empty()) - throw KeyNotFoundException(Composite(std::move(failed_reads))); + } catch (const ::lmdb::not_found_error&) { + ARCTICDB_DEBUG(log::storage(), "Failed to find segment for key {}", variant_key_view(key)); + failed_read.emplace(key); + } catch (const ::lmdb::error& ex) { + raise_lmdb_exception(ex, stored_key); + } + + if (failed_read) + throw KeyNotFoundException(*failed_read); } -bool LmdbStorage::do_key_exists(const VariantKey&key) { +bool LmdbStorage::do_key_exists(const VariantKey& key) { ARCTICDB_SAMPLE(LmdbStorageKeyExists, 0) auto txn = ::lmdb::txn::begin(env(), nullptr, MDB_RDONLY); ARCTICDB_SUBSAMPLE(LmdbStorageInTransaction, 0) @@ -189,7 +206,7 @@ bool LmdbStorage::do_key_exists(const VariantKey&key) { try { ::lmdb::dbi& dbi = get_dbi(db_name); return lmdb_client_->exists(db_name, stored_key, txn, dbi); - } catch ([[maybe_unused]] const ::lmdb::not_found_error &ex) { + } catch ([[maybe_unused]] const ::lmdb::not_found_error& ex) { ARCTICDB_DEBUG(log::storage(), "Caught lmdb not found error: {}", ex.what()); } catch (const ::lmdb::error& ex) { raise_lmdb_exception(ex, stored_key); @@ -197,59 +214,66 @@ bool LmdbStorage::do_key_exists(const VariantKey&key) { return false; } -std::vector LmdbStorage::do_remove_internal(Composite&& ks, ::lmdb::txn& txn, RemoveOpts opts) -{ - auto fmt_db = [](auto &&k) { return variant_key_type(k); }; - std::vector failed_deletes; +boost::container::small_vector LmdbStorage::do_remove_internal(std::span variant_keys, ::lmdb::txn& txn, RemoveOpts opts) { + boost::container::small_vector failed_deletes; ARCTICDB_DEBUG_THROW(5) - (fg::from(ks.as_range()) | fg::move | fg::groupBy(fmt_db)).foreach([&](auto &&group) { - auto db_name = fmt::format("{}", group.key()); + for(auto&& key : variant_keys) { + auto db_name = fmt::format("{}", variant_key_type(key)); ARCTICDB_SUBSAMPLE(LmdbStorageOpenDb, 0) try { - // If no key of this type has been written before, this can fail ::lmdb::dbi& dbi = get_dbi(db_name); - - for (auto &k : group.values()) { - auto stored_key = to_serialized_key(k); - - try { - if (lmdb_client_->remove(db_name, stored_key, txn, dbi)) { - ARCTICDB_DEBUG(log::storage(), "Deleted segment for key {}", variant_key_view(k)); - } else { - if (!opts.ignores_missing_key_) { - log::storage().warn("Failed to delete segment for key {}", variant_key_view(k)); - failed_deletes.push_back(k); - } - } - } catch (const ::lmdb::not_found_error&) { + auto stored_key = to_serialized_key(key); + + try { + if (lmdb_client_->remove(db_name, stored_key, txn, dbi)) { + ARCTICDB_DEBUG(log::storage(), "Deleted segment for key {}", variant_key_view(key)); + } else { if (!opts.ignores_missing_key_) { - log::storage().warn("Failed to delete segment for key {}", variant_key_view(k)); - failed_deletes.push_back(k); + log::storage().warn("Failed to delete segment for key {}", variant_key_view(key)); + failed_deletes.emplace_back(key); } - } catch (const ::lmdb::error& ex) { - raise_lmdb_exception(ex, stored_key); } + } catch (const ::lmdb::not_found_error&) { + if (!opts.ignores_missing_key_) { + log::storage().warn("Failed to delete segment for key {}", variant_key_view(key)); + failed_deletes.emplace_back(key); + } + } catch (const ::lmdb::error& ex) { + raise_lmdb_exception(ex, stored_key); } } catch (const ::lmdb::error& ex) { raise_lmdb_exception(ex, db_name); } - }); + } return failed_deletes; } -void LmdbStorage::do_remove(Composite&& ks, RemoveOpts opts) -{ +void LmdbStorage::do_remove(VariantKey&& variant_key, RemoveOpts opts) { ARCTICDB_SAMPLE(LmdbStorageRemove, 0) std::lock_guard lock{*write_mutex_}; auto txn = ::lmdb::txn::begin(env()); ARCTICDB_SUBSAMPLE(LmdbStorageInTransaction, 0) - auto failed_deletes = do_remove_internal(std::move(ks), txn, opts); + std::array arr{std::move(variant_key)}; + auto failed_deletes = do_remove_internal(std::span{arr}, txn, opts); + ARCTICDB_SUBSAMPLE(LmdbStorageCommit, 0) + txn.commit(); + + if (!failed_deletes.empty()) + throw KeyNotFoundException(failed_deletes); +} + +void LmdbStorage::do_remove(std::span variant_keys, RemoveOpts opts) { + ARCTICDB_SAMPLE(LmdbStorageRemoveMultiple, 0) + std::lock_guard lock{*write_mutex_}; + auto txn = ::lmdb::txn::begin(env()); + ARCTICDB_SUBSAMPLE(LmdbStorageInTransaction, 0) + auto failed_deletes = do_remove_internal(variant_keys, txn, opts); ARCTICDB_SUBSAMPLE(LmdbStorageCommit, 0) txn.commit(); - if(!failed_deletes.empty()) - throw KeyNotFoundException(Composite(std::move(failed_deletes))); + if (!failed_deletes.empty()) + throw KeyNotFoundException(failed_deletes); } bool LmdbStorage::do_fast_delete() { @@ -257,7 +281,7 @@ bool LmdbStorage::do_fast_delete() { // bool is probably not the best return type here but it does help prevent the insane boilerplate for // an additional function that checks whether this is supported (like the prefix matching) auto dtxn = ::lmdb::txn::begin(env()); - foreach_key_type([&] (KeyType key_type) { + foreach_key_type([&](KeyType key_type) { if (key_type == KeyType::TOMBSTONE) { // TOMBSTONE and LOCK both format to code 'x' - do not try to drop both return; @@ -277,18 +301,20 @@ bool LmdbStorage::do_fast_delete() { return true; } -bool LmdbStorage::do_iterate_type_until_match(KeyType key_type, const IterateTypePredicate& visitor, const std::string &prefix) { - ARCTICDB_SAMPLE(LmdbStorageItType, 0); +bool LmdbStorage::do_iterate_type_until_match(KeyType key_type, + const IterateTypePredicate& visitor, + const std::string& prefix) { + ARCTICDB_SAMPLE(LmdbStorageItType, 0) auto txn = ::lmdb::txn::begin(env(), nullptr, MDB_RDONLY); // scoped abort on std::string type_db = fmt::format("{}", key_type); ::lmdb::dbi& dbi = get_dbi(type_db); try { auto keys = lmdb_client_->list(type_db, prefix, txn, dbi, key_type); - for (auto &k: keys) { + for (auto& k : keys) { ARCTICDB_SUBSAMPLE(LmdbStorageVisitKey, 0) - if(visitor(std::move(k))) { - return true; + if (visitor(std::move(k))) { + return true; } } } catch (const ::lmdb::error& ex) { @@ -297,18 +323,18 @@ bool LmdbStorage::do_iterate_type_until_match(KeyType key_type, const IterateTyp return false; } -bool LmdbStorage::do_is_path_valid(const std::string_view pathString ARCTICDB_UNUSED) const { +bool LmdbStorage::do_is_path_valid(std::string_view path ARCTICDB_UNUSED) const { #ifdef _WIN32 // Note that \ and / are valid characters as they will create subdirectories which are expected to work. // The filenames such as COM1, LPT1, AUX, CON etc. are reserved but not strictly disallowed by Windows as directory names. // Therefore, paths with these names are allowed. std::string_view invalid_win32_chars = "<>:\"|?*"; - auto found = pathString.find_first_of(invalid_win32_chars); + auto found = path.find_first_of(invalid_win32_chars); if (found != std::string::npos) { return false; } - if (!pathString.empty() && (pathString.back() == '.' || std::isspace(pathString.back()))) { + if (!path.empty() && (path.back() == '.' || std::isspace(path.back()))) { return false; } #endif @@ -326,8 +352,8 @@ void remove_db_files(const fs::path& lib_path) { } } catch (const std::filesystem::filesystem_error& e) { raise( - fmt::format("Unexpected LMDB Error: Failed to remove LMDB file at path: {} error: {}", - file_path.string(), e.what())); + fmt::format("Unexpected LMDB Error: Failed to remove LMDB file at path: {} error: {}", + file_path.string(), e.what())); } } @@ -340,8 +366,8 @@ void remove_db_files(const fs::path& lib_path) { fs::remove_all(lib_path); } catch (const fs::filesystem_error& e) { raise( - fmt::format("Unexpected LMDB Error: Failed to remove directory: {} error: {}", - lib_path.string(), e.what())); + fmt::format("Unexpected LMDB Error: Failed to remove directory: {} error: {}", + lib_path.string(), e.what())); } } } @@ -352,7 +378,6 @@ void LmdbStorage::cleanup() { remove_db_files(lib_dir_); } - namespace { template T or_else(T val, T or_else_val, T def = T()) { @@ -360,12 +385,11 @@ T or_else(T val, T or_else_val, T def = T()) { } } // anonymous -LmdbStorage::LmdbStorage(const LibraryPath &library_path, OpenMode mode, const Config &conf) : - Storage(library_path, mode) { +LmdbStorage::LmdbStorage(const LibraryPath& library_path, OpenMode mode, const Config& conf) : + Storage(library_path, mode) { if (conf.use_mock_storage_for_testing()) { lmdb_client_ = std::make_unique(); - } - else { + } else { lmdb_client_ = std::make_unique(); } const auto lib_path_str = library_path.to_delim_path(fs::path::preferred_separator); @@ -413,7 +437,7 @@ LmdbStorage::LmdbStorage(const LibraryPath &library_path, OpenMode mode, const C auto txn = ::lmdb::txn::begin(env()); try { - arcticdb::entity::foreach_key_type([&txn, this](KeyType &&key_type) { + arcticdb::entity::foreach_key_type([&txn, this](KeyType&& key_type) { std::string db_name = fmt::format("{}", key_type); ::lmdb::dbi dbi = ::lmdb::dbi::open(txn, db_name.data(), MDB_CREATE); lmdb_instance_->dbi_by_key_type_.emplace(std::move(db_name), std::make_unique<::lmdb::dbi>(std::move(dbi))); @@ -424,7 +448,10 @@ LmdbStorage::LmdbStorage(const LibraryPath &library_path, OpenMode mode, const C txn.commit(); - ARCTICDB_DEBUG(log::storage(), "Opened lmdb storage at {} with map size {}", lib_dir_.string(), format_bytes(mapsize)); + ARCTICDB_DEBUG(log::storage(), + "Opened lmdb storage at {} with map size {}", + lib_dir_.string(), + format_bytes(mapsize)); } void LmdbStorage::warn_if_lmdb_already_open() { @@ -435,16 +462,16 @@ void LmdbStorage::warn_if_lmdb_already_open() { // Strip magic name from warning as it will confuse users user_facing_path.remove_filename(); log::storage().warn(fmt::format( - "LMDB path at {} has already been opened in this process which is not supported by LMDB. " - "You should only open a single Arctic instance over a given LMDB path. " - "To continue safely, you should delete this Arctic instance and any others over the LMDB path in this " - "process and then try again. Current process ID=[{}]", - user_facing_path.string(), getpid())); + "LMDB path at {} has already been opened in this process which is not supported by LMDB. " + "You should only open a single Arctic instance over a given LMDB path. " + "To continue safely, you should delete this Arctic instance and any others over the LMDB path in this " + "process and then try again. Current process ID=[{}]", + user_facing_path.string(), getpid())); } } -LmdbStorage::LmdbStorage(LmdbStorage&& other) noexcept - : Storage(std::move(static_cast(other))), +LmdbStorage::LmdbStorage(LmdbStorage&& other) noexcept: + Storage(std::move(static_cast(other))), write_mutex_(std::move(other.write_mutex_)), lmdb_instance_(std::move(other.lmdb_instance_)), lib_dir_(std::move(other.lib_dir_)) { diff --git a/cpp/arcticdb/storage/lmdb/lmdb_storage.hpp b/cpp/arcticdb/storage/lmdb/lmdb_storage.hpp index 9c8bf9cbdc..4ffa01e5fc 100644 --- a/cpp/arcticdb/storage/lmdb/lmdb_storage.hpp +++ b/cpp/arcticdb/storage/lmdb/lmdb_storage.hpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include @@ -36,17 +36,21 @@ class LmdbStorage final : public Storage { std::string name() const final; private: - void do_write(Composite&& kvs) final; + void do_write(KeySegmentPair&& key_seg) final; void do_write_if_none(KeySegmentPair&& kv [[maybe_unused]]) final { storage::raise("Atomic operations are only supported for s3 backend"); }; - void do_update(Composite&& kvs, UpdateOpts opts) final; + void do_update(KeySegmentPair&& key_seg, UpdateOpts opts) final; - void do_read(Composite&& ks, const ReadVisitor& visitor, storage::ReadKeyOpts opts) final; + void do_read(VariantKey&& variant_key, const ReadVisitor& visitor, storage::ReadKeyOpts opts) final; - void do_remove(Composite&& ks, RemoveOpts opts) final; + KeySegmentPair do_read(VariantKey&& variant_key, ReadKeyOpts) final; + + void do_remove(VariantKey&& variant_key, RemoveOpts opts) final; + + void do_remove(std::span variant_keys, RemoveOpts opts) final; bool do_supports_prefix_matching() const final { return false; @@ -64,7 +68,7 @@ class LmdbStorage final : public Storage { bool do_key_exists(const VariantKey & key) final; - bool do_is_path_valid(const std::string_view path) const final; + bool do_is_path_valid(std::string_view path) const final; ::lmdb::env& env(); @@ -75,8 +79,8 @@ class LmdbStorage final : public Storage { void warn_if_lmdb_already_open(); // _internal methods assume the write mutex is already held - void do_write_internal(Composite&& kvs, ::lmdb::txn& txn); - std::vector do_remove_internal(Composite&& ks, ::lmdb::txn& txn, RemoveOpts opts); + void do_write_internal(KeySegmentPair&& key_seg, ::lmdb::txn& txn); + boost::container::small_vector do_remove_internal(std::span variant_key, ::lmdb::txn& txn, RemoveOpts opts); std::unique_ptr write_mutex_; std::shared_ptr lmdb_instance_; diff --git a/cpp/arcticdb/storage/memory/memory_storage.cpp b/cpp/arcticdb/storage/memory/memory_storage.cpp index ce180910dd..63c4638501 100644 --- a/cpp/arcticdb/storage/memory/memory_storage.cpp +++ b/cpp/arcticdb/storage/memory/memory_storage.cpp @@ -6,164 +6,148 @@ */ #include - -#include - #include #include #include #include #include +#include namespace arcticdb::storage::memory { -void add_serialization_fields(KeySegmentPair& kv) { - auto& segment = kv.segment(); - auto& hdr = segment.header(); - (void)segment.calculate_size(); - if(hdr.encoding_version() == EncodingVersion::V2) { - const auto* src = segment.buffer().data(); +void add_serialization_fields(KeySegmentPair &kv) { + auto &segment = kv.segment(); + auto &hdr = segment.header(); + (void) segment.calculate_size(); + if (hdr.encoding_version() == EncodingVersion::V2) { + const auto *src = segment.buffer().data(); set_body_fields(hdr, src); } } - namespace fg = folly::gen; +std::string MemoryStorage::name() const { + return "memory_storage-0"; +} - std::string MemoryStorage::name() const { - return "memory_storage-0"; - } +void MemoryStorage::do_write(KeySegmentPair &&key_seg) { + ARCTICDB_SAMPLE(MemoryStorageWrite, 0) - void MemoryStorage::do_write(Composite&& kvs) { - ARCTICDB_SAMPLE(MemoryStorageWrite, 0) - - auto fmt_db = [](auto &&k) { return variant_key_type(k.variant_key()); }; - - (fg::from(kvs.as_range()) | fg::move | fg::groupBy(fmt_db)).foreach([&](auto &&group) { - auto& key_vec = data_[group.key()]; - - for (auto &kv : group.values()) { - util::variant_match(kv.variant_key(), - [&](const RefKey &key) { - if (auto it = key_vec.find(key); it != key_vec.end()) { - key_vec.erase(it); - } - add_serialization_fields(kv); - key_vec.try_emplace(key, std::move(kv.segment())); - }, - [&](const AtomKey &key) { - if (key_vec.find(key) != key_vec.end()) { - throw DuplicateKeyException(key); - } - add_serialization_fields(kv); - key_vec.try_emplace(key, std::move(kv.segment())); - } - ); - } - }); - } + auto &key_vec = data_[variant_key_type(key_seg.variant_key())]; - void MemoryStorage::do_update(Composite&& kvs, UpdateOpts opts) { - ARCTICDB_SAMPLE(MemoryStorageUpdate, 0) + util::variant_match(key_seg.variant_key(), + [&](const RefKey &key) { + if (auto it = key_vec.find(key); it != key_vec.end()) { + key_vec.erase(it); + } + add_serialization_fields(key_seg); + key_vec.try_emplace(key, std::move(key_seg.segment())); + }, + [&](const AtomKey &key) { + if (key_vec.find(key) != key_vec.end()) { + throw DuplicateKeyException(key); + } + add_serialization_fields(key_seg); + key_vec.try_emplace(key, std::move(key_seg.segment())); + } + ); +} - auto fmt_db = [](auto &&k) { return variant_key_type(k.variant_key()); }; +void MemoryStorage::do_update(KeySegmentPair &&key_seg, UpdateOpts opts) { + ARCTICDB_SAMPLE(MemoryStorageUpdate, 0) - (fg::from(kvs.as_range()) | fg::move | fg::groupBy(fmt_db)).foreach([&](auto &&group) { - auto& key_vec = data_[group.key()]; + auto &key_vec = data_[variant_key_type(key_seg.variant_key())]; + auto it = key_vec.find(key_seg.variant_key()); - for (auto &kv : group.values()) { - auto it = key_vec.find(kv.variant_key()); + if (!opts.upsert_ && it == key_vec.end()) { + std::string err_message = + fmt::format("do_update called with upsert=false on non-existent key(s): {}", key_seg.variant_key()); + throw KeyNotFoundException(key_seg.variant_key(), err_message); + } - if (!opts.upsert_ && it == key_vec.end()) { - std::string err_message = fmt::format("do_update called with upsert=false on non-existent key(s): {}", kv.variant_key()); - throw KeyNotFoundException(std::move(kv.variant_key()), err_message); - } + if (it != key_vec.end()) { + key_vec.erase(it); + } - if(it != key_vec.end()) { - key_vec.erase(it); - } + add_serialization_fields(key_seg); + key_vec.insert(std::make_pair(key_seg.variant_key(), key_seg.segment().clone())); +} - add_serialization_fields(kv); - key_vec.insert(std::make_pair(kv.variant_key(), kv.segment().clone())); - } - }); - } +void MemoryStorage::do_read(VariantKey &&variant_key, const ReadVisitor &visitor, ReadKeyOpts) { + auto key_seg = do_read(std::move(variant_key), ReadKeyOpts{}); + visitor(key_seg.variant_key(), std::move(key_seg.segment())); +} - void MemoryStorage::do_read(Composite&& ks, const ReadVisitor& visitor, ReadKeyOpts) { - ARCTICDB_SAMPLE(MemoryStorageRead, 0) - auto fmt_db = [](auto &&k) { return variant_key_type(k); }; - - (fg::from(ks.as_range()) | fg::move | fg::groupBy(fmt_db)).foreach([&](auto &&group) { - auto& key_vec = data_[group.key()]; - for (auto &k : group.values()) { - auto it = key_vec.find(k); - - if(it != key_vec.end()) { - ARCTICDB_DEBUG(log::storage(), "Read key {}: {}", variant_key_type(k), variant_key_view(k)); - visitor(k, it->second.clone()); - } else { - throw KeyNotFoundException(std::move(ks)); - } - } - }); - } +KeySegmentPair MemoryStorage::do_read(VariantKey &&variant_key, ReadKeyOpts) { + ARCTICDB_SAMPLE(MemoryStorageRead, 0) + auto& key_vec = data_[variant_key_type(variant_key)]; + auto it = key_vec.find(variant_key); - bool MemoryStorage::do_key_exists(const VariantKey& key) { - ARCTICDB_SAMPLE(MemoryStorageKeyExists, 0) - const auto& key_vec = data_[variant_key_type(key)]; - auto it = key_vec.find(key); - return it != key_vec.end(); + if (it != key_vec.end()) { + ARCTICDB_DEBUG(log::storage(), "Read key {}: {}", variant_key_type(variant_key), variant_key_view(variant_key)); + return {std::move(variant_key), it->second.clone()}; + } else { + throw KeyNotFoundException(variant_key); } +} - void MemoryStorage::do_remove(Composite&& ks, RemoveOpts opts) - { - ARCTICDB_SAMPLE(MemoryStorageRemove, 0) - auto fmt_db = [](auto &&k) { return variant_key_type(k); }; +bool MemoryStorage::do_key_exists(const VariantKey &key) { + ARCTICDB_SAMPLE(MemoryStorageKeyExists, 0) + const auto &key_vec = data_[variant_key_type(key)]; + auto it = key_vec.find(key); + return it != key_vec.end(); +} - (fg::from(ks.as_range()) | fg::move | fg::groupBy(fmt_db)).foreach([&](auto &&group) { - auto& key_vec = data_[group.key()]; +void MemoryStorage::do_remove(VariantKey&& variant_key, RemoveOpts opts) { + ARCTICDB_SAMPLE(MemoryStorageRemove, 0) + auto &key_vec = data_[variant_key_type(variant_key)]; - for (auto &k : group.values()) { - auto it = key_vec.find(k); + auto it = key_vec.find(variant_key); - if(it != key_vec.end()) { - ARCTICDB_DEBUG(log::storage(), "Removed key {}: {}", variant_key_type(k), variant_key_view(k)); - key_vec.erase(it); - } else if (!opts.ignores_missing_key_) { - throw KeyNotFoundException(std::move(k)); - } - } - }); + if (it != key_vec.end()) { + ARCTICDB_DEBUG(log::storage(), "Removed key {}: {}", variant_key_type(variant_key), variant_key_view(variant_key)); + key_vec.erase(it); + } else if (!opts.ignores_missing_key_) { + throw KeyNotFoundException(variant_key); } +} - bool MemoryStorage::do_fast_delete() { - foreach_key_type([&] (KeyType key_type) { - data_[key_type].clear(); - }); - return true; - } +void MemoryStorage::do_remove(std::span variant_keys, RemoveOpts opts) { + ARCTICDB_SAMPLE(MemoryStorageRemoveMultiple, 0) + for(auto&& variant_key : variant_keys) + do_remove(std::move(variant_key), opts); +} + +bool MemoryStorage::do_fast_delete() { + foreach_key_type([&](KeyType key_type) { + data_[key_type].clear(); + }); + return true; +} - bool MemoryStorage::do_iterate_type_until_match(KeyType key_type, const IterateTypePredicate& visitor, const std::string& prefix) { - ARCTICDB_SAMPLE(MemoryStorageItType, 0) - auto& key_vec = data_[key_type]; - auto prefix_matcher = stream_id_prefix_matcher(prefix); +bool MemoryStorage::do_iterate_type_until_match(KeyType key_type, + const IterateTypePredicate &visitor, + const std::string &prefix) { + ARCTICDB_SAMPLE(MemoryStorageItType, 0) + auto &key_vec = data_[key_type]; + auto prefix_matcher = stream_id_prefix_matcher(prefix); - for(auto& key_value : key_vec) { - auto key = key_value.first; + for (auto &key_value : key_vec) { + auto key = key_value.first; - if (prefix_matcher(variant_key_id(key))) { - if (visitor(std::move(key))) { - return true; - } + if (prefix_matcher(variant_key_id(key))) { + if (visitor(std::move(key))) { + return true; } } - return false; } + return false; +} - MemoryStorage::MemoryStorage(const LibraryPath &library_path, OpenMode mode, const Config&) : - Storage(library_path, mode) { - arcticdb::entity::foreach_key_type([this](KeyType&& key_type) { - data_[key_type]; - }); - } +MemoryStorage::MemoryStorage(const LibraryPath &library_path, OpenMode mode, const Config &) : + Storage(library_path, mode) { + arcticdb::entity::foreach_key_type([this](KeyType &&key_type) { + data_[key_type]; + }); +} } diff --git a/cpp/arcticdb/storage/memory/memory_storage.hpp b/cpp/arcticdb/storage/memory/memory_storage.hpp index 6cec399494..43edc7623d 100644 --- a/cpp/arcticdb/storage/memory/memory_storage.hpp +++ b/cpp/arcticdb/storage/memory/memory_storage.hpp @@ -27,17 +27,21 @@ namespace arcticdb::storage::memory { std::string name() const final; private: - void do_write(Composite&& kvs) final; + void do_write(KeySegmentPair&& key_seg) final; void do_write_if_none(KeySegmentPair&& kv [[maybe_unused]]) final { storage::raise("Atomic operations are only supported for s3 backend"); }; - void do_update(Composite&& kvs, UpdateOpts opts) final; + void do_update(KeySegmentPair&& key_seg, UpdateOpts opts) final; - void do_read(Composite&& ks, const ReadVisitor& visitor, ReadKeyOpts opts) final; + void do_read(VariantKey&& variant_key, const ReadVisitor& visitor, ReadKeyOpts opts) final; - void do_remove(Composite&& ks, RemoveOpts opts) final; + KeySegmentPair do_read(VariantKey&& variant_key, ReadKeyOpts) final; + + void do_remove(VariantKey&& variant_key, RemoveOpts opts) final; + + void do_remove(std::span variant_key, RemoveOpts opts) final; bool do_key_exists(const VariantKey& key) final; diff --git a/cpp/arcticdb/storage/azure/azure_mock_client.cpp b/cpp/arcticdb/storage/mock/azure_mock_client.cpp similarity index 97% rename from cpp/arcticdb/storage/azure/azure_mock_client.cpp rename to cpp/arcticdb/storage/mock/azure_mock_client.cpp index b22cacf3a3..3e00e3e6ce 100644 --- a/cpp/arcticdb/storage/azure/azure_mock_client.cpp +++ b/cpp/arcticdb/storage/mock/azure_mock_client.cpp @@ -4,16 +4,15 @@ * * 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 #include #include #include -#include -#include -#include -#include - namespace arcticdb::storage::azure { std::string MockAzureClient::get_failure_trigger( diff --git a/cpp/arcticdb/storage/azure/azure_mock_client.hpp b/cpp/arcticdb/storage/mock/azure_mock_client.hpp similarity index 56% rename from cpp/arcticdb/storage/azure/azure_mock_client.hpp rename to cpp/arcticdb/storage/mock/azure_mock_client.hpp index 147324f61e..57de492288 100644 --- a/cpp/arcticdb/storage/azure/azure_mock_client.hpp +++ b/cpp/arcticdb/storage/mock/azure_mock_client.hpp @@ -13,39 +13,38 @@ #include #include -#include -#include +#include +#include namespace arcticdb::storage::azure { class MockAzureClient : public AzureClientWrapper { public: - void write_blob( - const std::string& blob_name, - Segment&& segment, - const Azure::Storage::Blobs::UploadBlockBlobFromOptions& upload_option, - unsigned int request_timeout) override; + const std::string& blob_name, + Segment&& segment, + const Azure::Storage::Blobs::UploadBlockBlobFromOptions& upload_option, + unsigned int request_timeout) override; Segment read_blob( - const std::string& blob_name, - const Azure::Storage::Blobs::DownloadBlobToOptions& download_option, - unsigned int request_timeout) override; + const std::string& blob_name, + const Azure::Storage::Blobs::DownloadBlobToOptions& download_option, + unsigned int request_timeout) override; void delete_blobs( - const std::vector& blob_names, - unsigned int request_timeout) override; + const std::vector& blob_names, + unsigned int request_timeout) override; bool blob_exists(const std::string& blob_name) override; Azure::Storage::Blobs::ListBlobsPagedResponse list_blobs(const std::string& prefix) override; static std::string get_failure_trigger( - const std::string& blob_name, - StorageOperation operation_to_fail, - const std::string& error_code, - Azure::Core::Http::HttpStatusCode error_to_fail_with); + const std::string& blob_name, + StorageOperation operation_to_fail, + const std::string& error_code, + Azure::Core::Http::HttpStatusCode error_to_fail_with); private: // Stores a mapping from blob_name to a Segment. diff --git a/cpp/arcticdb/storage/lmdb/lmdb_mock_client.cpp b/cpp/arcticdb/storage/mock/lmdb_mock_client.cpp similarity index 94% rename from cpp/arcticdb/storage/lmdb/lmdb_mock_client.cpp rename to cpp/arcticdb/storage/mock/lmdb_mock_client.cpp index 6c351d7932..b6455c4a9a 100644 --- a/cpp/arcticdb/storage/lmdb/lmdb_mock_client.cpp +++ b/cpp/arcticdb/storage/mock/lmdb_mock_client.cpp @@ -5,13 +5,13 @@ * 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 -#include -#include -#include +#include "arcticdb/codec/segment.hpp" +#include "arcticdb/entity/atom_key.hpp" +#include "arcticdb/entity/serialized_key.hpp" +#include "arcticdb/util/string_utils.hpp" +#include "arcticdb/storage/lmdb/lmdb.hpp" namespace arcticdb::storage::lmdb { diff --git a/cpp/arcticdb/storage/lmdb/lmdb_mock_client.hpp b/cpp/arcticdb/storage/mock/lmdb_mock_client.hpp similarity index 95% rename from cpp/arcticdb/storage/lmdb/lmdb_mock_client.hpp rename to cpp/arcticdb/storage/mock/lmdb_mock_client.hpp index 6e81bf6730..cf3df8bb5c 100644 --- a/cpp/arcticdb/storage/lmdb/lmdb_mock_client.hpp +++ b/cpp/arcticdb/storage/mock/lmdb_mock_client.hpp @@ -7,15 +7,14 @@ #pragma once -#include +#include #include #include #include -#include +#include #include - namespace arcticdb::storage::lmdb { struct LmdbKey { diff --git a/cpp/arcticdb/storage/mongo/mongo_mock_client.cpp b/cpp/arcticdb/storage/mock/mongo_mock_client.cpp similarity index 93% rename from cpp/arcticdb/storage/mongo/mongo_mock_client.cpp rename to cpp/arcticdb/storage/mock/mongo_mock_client.cpp index 46ae0a39e3..ce573add1b 100644 --- a/cpp/arcticdb/storage/mongo/mongo_mock_client.cpp +++ b/cpp/arcticdb/storage/mock/mongo_mock_client.cpp @@ -5,9 +5,10 @@ * 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 #include + #include #include #include @@ -98,8 +99,8 @@ bool MockMongoClient::has_key(const MongoKey& key) { bool MockMongoClient::write_segment( const std::string& database_name, const std::string& collection_name, - storage::KeySegmentPair&& kv) { - auto key = MongoKey(database_name, collection_name, kv.variant_key()); + storage::KeySegmentPair&& key_seg) { + auto key = MongoKey(database_name, collection_name, key_seg.variant_key()); auto failure = has_failure_trigger(key, StorageOperation::WRITE); if (failure.has_value()) { @@ -107,16 +108,16 @@ bool MockMongoClient::write_segment( return false; } - mongo_contents.insert_or_assign(std::move(key), std::move(kv.segment())); + mongo_contents.insert_or_assign(std::move(key), std::move(key_seg.segment())); return true; } UpdateResult MockMongoClient::update_segment( const std::string& database_name, const std::string& collection_name, - storage::KeySegmentPair&& kv, + storage::KeySegmentPair&& key_seg, bool upsert) { - auto key = MongoKey(database_name, collection_name, kv.variant_key()); + auto key = MongoKey(database_name, collection_name, key_seg.variant_key()); auto failure = has_failure_trigger(key, StorageOperation::WRITE); if (failure.has_value()) { @@ -129,7 +130,7 @@ UpdateResult MockMongoClient::update_segment( return {0}; // upsert is false, don't update and return 0 as modified_count } - mongo_contents.insert_or_assign(std::move(key), std::move(kv.segment())); + mongo_contents.insert_or_assign(std::move(key), std::move(key_seg.segment())); return {key_found ? 1 : 0}; } diff --git a/cpp/arcticdb/storage/mongo/mongo_mock_client.hpp b/cpp/arcticdb/storage/mock/mongo_mock_client.hpp similarity index 94% rename from cpp/arcticdb/storage/mongo/mongo_mock_client.hpp rename to cpp/arcticdb/storage/mock/mongo_mock_client.hpp index 3c2ab02d69..bad69a5ec2 100644 --- a/cpp/arcticdb/storage/mongo/mongo_mock_client.hpp +++ b/cpp/arcticdb/storage/mock/mongo_mock_client.hpp @@ -8,8 +8,8 @@ #pragma once #include -#include -#include +#include +#include #include namespace arcticdb::storage::mongo { @@ -78,12 +78,12 @@ class MockMongoClient : public MongoClientWrapper { bool write_segment( const std::string& database_name, const std::string& collection_name, - storage::KeySegmentPair&& kv) override; + storage::KeySegmentPair&& key_seg) override; UpdateResult update_segment( const std::string& database_name, const std::string& collection_name, - storage::KeySegmentPair&& kv, + storage::KeySegmentPair&& key_seg, bool upsert) override; std::optional read_segment( diff --git a/cpp/arcticdb/storage/s3/s3_mock_client.cpp b/cpp/arcticdb/storage/mock/s3_mock_client.cpp similarity index 93% rename from cpp/arcticdb/storage/s3/s3_mock_client.cpp rename to cpp/arcticdb/storage/mock/s3_mock_client.cpp index 3d614d5a05..bbb999f315 100644 --- a/cpp/arcticdb/storage/s3/s3_mock_client.cpp +++ b/cpp/arcticdb/storage/mock/s3_mock_client.cpp @@ -5,12 +5,10 @@ * 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 #include #include - #include #include @@ -81,6 +79,12 @@ S3Result MockS3Client::get_object( return {pos->second.clone()}; } +folly::Future> MockS3Client::get_object_async( + const std::string &s3_object_name, + const std::string &bucket_name) const { + return folly::makeFuture(get_object(s3_object_name, bucket_name)); +} + S3Result MockS3Client::put_object( const std::string &s3_object_name, Segment &&segment, @@ -128,7 +132,7 @@ constexpr auto page_size = 10; S3Result MockS3Client::list_objects( const std::string& name_prefix, const std::string& bucket_name, - const std::optional continuation_token) const { + const std::optional& continuation_token) const { // Terribly inefficient but fine for tests. auto matching_names = std::vector(); for (auto& key : s3_contents) { diff --git a/cpp/arcticdb/storage/s3/s3_mock_client.hpp b/cpp/arcticdb/storage/mock/s3_mock_client.hpp similarity index 62% rename from cpp/arcticdb/storage/s3/s3_mock_client.hpp rename to cpp/arcticdb/storage/mock/s3_mock_client.hpp index 75561adb91..0dba1ae3a5 100644 --- a/cpp/arcticdb/storage/s3/s3_mock_client.hpp +++ b/cpp/arcticdb/storage/mock/s3_mock_client.hpp @@ -9,8 +9,8 @@ #include -#include -#include +#include +#include #include #include @@ -35,12 +35,12 @@ struct S3Key { } }; -// A mock S3ClientWrapper which can simulate failures. +// A mock S3ClientInterface which can simulate failures. // The MockS3Client stores the segments in memory to simulate regular S3 behavior for unit tests. // The MockS3Client can simulate storage failures by using the get_failure_trigger for s3_object_names. -class MockS3Client : public S3ClientWrapper { +class MockS3Client : public S3ClientInterface { public: - MockS3Client(){} + MockS3Client() {} // Can be used to trigger a simulated failure inside MockS3Client. For example: // auto object_to_trigger_put_failure = get_failure_trigger("test", StorageOperation::WRITE, Aws::S3::S3Errors::NETWORK_FAILURE, false); @@ -49,29 +49,37 @@ class MockS3Client : public S3ClientWrapper { // The returned name looks like "{s3_object_name}#Failure_{operation_to_fail}_{error_to_fail_with}_{retryable}". // For example: "symbol_1#Failure_Delete_99_1" will trigger a delete failure with code 99 which is retryable. static std::string get_failure_trigger( - const std::string& s3_object_name, - StorageOperation operation_to_fail, - Aws::S3::S3Errors error_to_fail_with, - bool retryable=true); + const std::string& s3_object_name, + StorageOperation operation_to_fail, + Aws::S3::S3Errors error_to_fail_with, + bool retryable = true); - S3Result head_object(const std::string& s3_object_name, const std::string& bucket_name) const override; + [[nodiscard]] S3Result head_object( + const std::string& s3_object_name, + const std::string& bucket_name) const override; - S3Result get_object(const std::string& s3_object_name, const std::string& bucket_name) const override; + [[nodiscard]] S3Result get_object( + const std::string& s3_object_name, + const std::string& bucket_name) const override; + + [[nodiscard]] folly::Future> get_object_async( + const std::string& s3_object_name, + const std::string& bucket_name) const override; S3Result put_object( - const std::string& s3_object_name, - Segment&& segment, - const std::string& bucket_name, - PutHeader header = PutHeader::NONE) override; + const std::string& s3_object_name, + Segment&& segment, + const std::string& bucket_name, + PutHeader header = PutHeader::NONE) override; S3Result delete_objects( - const std::vector& s3_object_names, - const std::string& bucket_name) override; + const std::vector& s3_object_names, + const std::string& bucket_name) override; S3Result list_objects( - const std::string& prefix, - const std::string& bucket_name, - const std::optional continuation_token) const override; + const std::string& prefix, + const std::string& bucket_name, + const std::optional& continuation_token) const override; private: std::map s3_contents; diff --git a/cpp/arcticdb/storage/storage_mock_client.hpp b/cpp/arcticdb/storage/mock/storage_mock_client.hpp similarity index 78% rename from cpp/arcticdb/storage/storage_mock_client.hpp rename to cpp/arcticdb/storage/mock/storage_mock_client.hpp index b23fd91e32..f397e40e41 100644 --- a/cpp/arcticdb/storage/storage_mock_client.hpp +++ b/cpp/arcticdb/storage/mock/storage_mock_client.hpp @@ -34,20 +34,4 @@ inline std::string operation_to_string(StorageOperation operation) { util::raise_rte("Invalid Storage operation provided for mock client"); } -template -struct StorageResult { - std::variant result; - - [[nodiscard]] bool is_success() const { - return std::holds_alternative(result); - } - - Error& get_error() { - return std::get(result); - } - Output& get_output() { - return std::get(result); - } -}; - } diff --git a/cpp/arcticdb/storage/mongo/mongo_client.cpp b/cpp/arcticdb/storage/mongo/mongo_client.cpp index 5714c648af..3f80d74de2 100644 --- a/cpp/arcticdb/storage/mongo/mongo_client.cpp +++ b/cpp/arcticdb/storage/mongo/mongo_client.cpp @@ -17,11 +17,8 @@ #include #include #include -#include #include -#include #include -#include #include namespace arcticdb::storage::mongo { @@ -154,12 +151,12 @@ auto build_document(storage::KeySegmentPair &kv) { return basic_builder.extract(); } -} //namespace detail +} // namespace detail class MongoClientImpl { using Config = arcticdb::proto::mongo_storage::Config; - std::string get_connection_string( + static std::string get_connection_string( std::string uri, uint64_t min_pool_size, uint64_t max_pool_size, @@ -188,12 +185,12 @@ class MongoClientImpl { bool write_segment( const std::string &database_name, const std::string &collection_name, - storage::KeySegmentPair&& kv); + storage::KeySegmentPair&& key_seg); UpdateResult update_segment( const std::string &database_name, const std::string &collection_name, - storage::KeySegmentPair&& kv, + storage::KeySegmentPair&& key_seg, bool upsert); std::optional read_segment( @@ -247,25 +244,26 @@ class MongoClientImpl { mongocxx::pool pool_; }; -bool MongoClientImpl::write_segment(const std::string &database_name, - const std::string &collection_name, - storage::KeySegmentPair &&kv) { +bool MongoClientImpl::write_segment( + const std::string &database_name, + const std::string &collection_name, + storage::KeySegmentPair&& key_seg) { using namespace bsoncxx::builder::stream; using bsoncxx::builder::stream::document; ARCTICDB_SUBSAMPLE(MongoStorageWriteGetClient, 0) auto client = get_client(); ARCTICDB_SUBSAMPLE(MongoStorageWriteBuildDoc, 0) - auto doc = detail::build_document(kv); + auto doc = detail::build_document(key_seg); ARCTICDB_SUBSAMPLE(MongoStorageWriteGetCol, 0) mongocxx::database database = client->database(database_name.c_str()); auto collection = database[collection_name]; ARCTICDB_SUBSAMPLE(MongoStorageWriteInsertOne, 0) - ARCTICDB_DEBUG(log::storage(), "Mongo client writing data with key {}", variant_key_view(kv.variant_key())); - if(std::holds_alternative(kv.variant_key())) { - mongocxx::model::replace_one replace{document{} << "key" << fmt::format("{}", kv.ref_key()) << finalize, doc.view()}; + ARCTICDB_DEBUG(log::storage(), "Mongo client writing data with key {}", variant_key_view(key_seg.variant_key())); + if(std::holds_alternative(key_seg.variant_key())) { + mongocxx::model::replace_one replace{document{} << "key" << fmt::format("{}", key_seg.ref_key()) << finalize, doc.view()}; replace.upsert(true); auto bulk_write = collection.create_bulk_write(); bulk_write.append(replace); @@ -275,24 +273,25 @@ bool MongoClientImpl::write_segment(const std::string &database_name, } } -UpdateResult MongoClientImpl::update_segment(const std::string &database_name, - const std::string &collection_name, - storage::KeySegmentPair &&kv, - bool upsert) { +UpdateResult MongoClientImpl::update_segment( + const std::string &database_name, + const std::string &collection_name, + storage::KeySegmentPair&& key_seg, + bool upsert) { using namespace bsoncxx::builder::stream; using bsoncxx::builder::stream::document; ARCTICDB_SUBSAMPLE(MongoStorageUpdateGetClient, 0) auto client = get_client(); ARCTICDB_SUBSAMPLE(MongoStorageUpdateBuildDoc, 0) - auto doc = detail::build_document(kv); + auto doc = detail::build_document(key_seg); ARCTICDB_SUBSAMPLE(MongoStorageUpdateGetCol, 0) mongocxx::database database = client->database(database_name.c_str()); auto collection = database[collection_name]; ARCTICDB_SUBSAMPLE(MongoStorageUpdateInsertOne, 0) - mongocxx::model::replace_one replace{document{} << "key" << fmt::format("{}", kv.variant_key()) << finalize, doc.view()}; + mongocxx::model::replace_one replace{document{} << "key" << fmt::format("{}", key_seg.variant_key()) << finalize, doc.view()}; replace.upsert(upsert); auto bulk_write = collection.create_bulk_write(); bulk_write.append(replace); @@ -300,9 +299,10 @@ UpdateResult MongoClientImpl::update_segment(const std::string &database_name, return {result ? std::optional(result->modified_count()) : std::nullopt}; } -std::optional MongoClientImpl::read_segment(const std::string &database_name, - const std::string &collection_name, - const entity::VariantKey &key) { +std::optional MongoClientImpl::read_segment( + const std::string &database_name, + const std::string &collection_name, + const entity::VariantKey &key) { using namespace bsoncxx::builder::stream; using bsoncxx::builder::stream::document; ARCTICDB_SUBSAMPLE(MongoStorageReadGetClient, 0) @@ -335,9 +335,10 @@ std::optional MongoClientImpl::read_segment(const std::string &d } } -bool MongoClientImpl::key_exists(const std::string &database_name, - const std::string &collection_name, - const entity::VariantKey &key) { +bool MongoClientImpl::key_exists( + const std::string &database_name, + const std::string &collection_name, + const entity::VariantKey &key) { using namespace bsoncxx::builder::stream; using bsoncxx::builder::stream::document; ARCTICDB_SUBSAMPLE(MongoStorageReadGetClient, 0) @@ -354,9 +355,10 @@ bool MongoClientImpl::key_exists(const std::string &database_name, } -DeleteResult MongoClientImpl::remove_keyvalue(const std::string &database_name, - const std::string &collection_name, - const entity::VariantKey &key) { +DeleteResult MongoClientImpl::remove_keyvalue( + const std::string &database_name, + const std::string &collection_name, + const entity::VariantKey &key) { using namespace bsoncxx::builder::stream; using bsoncxx::builder::stream::document; ARCTICDB_SUBSAMPLE(MongoStorageRemoveGetClient, 0) @@ -377,11 +379,11 @@ DeleteResult MongoClientImpl::remove_keyvalue(const std::string &database_name, return {result ? std::optional(result->deleted_count()) : std::nullopt}; } -std::vector MongoClientImpl::list_keys(const std::string &database_name, - const std::string &collection_name, - KeyType key_type, - const std::optional &prefix - ) { +std::vector MongoClientImpl::list_keys( + const std::string &database_name, + const std::string &collection_name, + KeyType key_type, + const std::optional &prefix) { using namespace bsoncxx::builder::stream; using bsoncxx::builder::stream::document; ARCTICDB_SUBSAMPLE(MongoStorageItTypeGetClient, 0) @@ -447,36 +449,40 @@ MongoClient::~MongoClient() { delete client_; } -bool MongoClient::write_segment(const std::string &database_name, - const std::string &collection_name, - storage::KeySegmentPair &&kv) { - return client_->write_segment(database_name, collection_name, std::move(kv)); +bool MongoClient::write_segment( + const std::string &database_name, + const std::string &collection_name, + storage::KeySegmentPair&& key_seg) { + return client_->write_segment(database_name, collection_name, std::move(key_seg)); } -UpdateResult MongoClient::update_segment(const std::string &database_name, - const std::string &collection_name, - storage::KeySegmentPair &&kv, - bool upsert) { - return client_->update_segment(database_name, collection_name, std::move(kv), upsert); +UpdateResult MongoClient::update_segment( + const std::string &database_name, + const std::string &collection_name, + storage::KeySegmentPair&& key_seg, + bool upsert) { + return client_->update_segment(database_name, collection_name, std::move(key_seg), upsert); } -std::optional MongoClient::read_segment(const std::string &database_name, - const std::string &collection_name, - const entity::VariantKey &key) { +std::optional MongoClient::read_segment( + const std::string &database_name, + const std::string &collection_name, + const entity::VariantKey &key) { return client_->read_segment(database_name, collection_name, key); } -DeleteResult MongoClient::remove_keyvalue(const std::string &database_name, - const std::string &collection_name, - const entity::VariantKey &key) { +DeleteResult MongoClient::remove_keyvalue( + const std::string &database_name, + const std::string &collection_name, + const entity::VariantKey &key) { return client_->remove_keyvalue(database_name, collection_name, key); } -std::vector MongoClient::list_keys(const std::string &database_name, - const std::string &collection_name, - KeyType key_type, - const std::optional &prefix - ) { +std::vector MongoClient::list_keys( + const std::string &database_name, + const std::string &collection_name, + KeyType key_type, + const std::optional &prefix) { return client_->list_keys(database_name, collection_name, key_type, prefix); } diff --git a/cpp/arcticdb/storage/mongo/mongo_client.hpp b/cpp/arcticdb/storage/mongo/mongo_client.hpp index 1eb0078bce..8780f868a6 100644 --- a/cpp/arcticdb/storage/mongo/mongo_client.hpp +++ b/cpp/arcticdb/storage/mongo/mongo_client.hpp @@ -9,7 +9,7 @@ #include #include -#include +#include #include namespace arcticdb::storage::mongo { @@ -30,12 +30,12 @@ class MongoClient : public MongoClientWrapper { bool write_segment( const std::string &database_name, const std::string &collection_name, - storage::KeySegmentPair&& kv) override; + storage::KeySegmentPair&& key_seg) override; UpdateResult update_segment( const std::string &database_name, const std::string &collection_name, - storage::KeySegmentPair&& kv, + storage::KeySegmentPair&& key_seg, bool upsert) override; std::optional read_segment( diff --git a/cpp/arcticdb/storage/mongo/mongo_client_wrapper.hpp b/cpp/arcticdb/storage/mongo/mongo_client_interface.hpp similarity index 96% rename from cpp/arcticdb/storage/mongo/mongo_client_wrapper.hpp rename to cpp/arcticdb/storage/mongo/mongo_client_interface.hpp index c87f989d0d..96852451a6 100644 --- a/cpp/arcticdb/storage/mongo/mongo_client_wrapper.hpp +++ b/cpp/arcticdb/storage/mongo/mongo_client_interface.hpp @@ -42,12 +42,12 @@ class MongoClientWrapper { virtual bool write_segment( const std::string &database_name, const std::string &collection_name, - storage::KeySegmentPair&& kv) = 0; + storage::KeySegmentPair&& key_seg) = 0; virtual UpdateResult update_segment( const std::string &database_name, const std::string &collection_name, - storage::KeySegmentPair&& kv, + storage::KeySegmentPair&& key_seg, bool upsert) = 0; virtual std::optional read_segment( diff --git a/cpp/arcticdb/storage/mongo/mongo_storage.cpp b/cpp/arcticdb/storage/mongo/mongo_storage.cpp index b4511b83e9..154f39cd05 100644 --- a/cpp/arcticdb/storage/mongo/mongo_storage.cpp +++ b/cpp/arcticdb/storage/mongo/mongo_storage.cpp @@ -16,12 +16,11 @@ #include #include #include -#include +#include #include #include #include -#include -#include +#include namespace arcticdb::storage::mongo { @@ -38,15 +37,17 @@ std::string MongoStorage::collection_name(KeyType k) { * - mongocxx::operation_exception has an error_code which is returned by the server as documented here: https://www.mongodb.com/docs/manual/reference/error-codes/ * - some relevant error codes returned by the server are defined in MongoError enum. */ -void raise_mongo_exception(const mongocxx::operation_exception& e, const std::string& object_name) { +void raise_mongo_exception(const mongocxx::operation_exception &e, const std::string &object_name) { auto error_code = e.code().value(); auto mongo_error_suffix = fmt::format("MongoError#{}: {} for object {}", error_code, e.what(), object_name); - if (error_code == static_cast(MongoError::NoSuchKey) || error_code == static_cast(MongoError::KeyNotFound)) { + if (error_code == static_cast(MongoError::NoSuchKey) + || error_code == static_cast(MongoError::KeyNotFound)) { throw KeyNotFoundException(fmt::format("Key Not Found Error: {}", mongo_error_suffix)); } - if (error_code == static_cast(MongoError::UnAuthorized) || error_code == static_cast(MongoError::AuthenticationFailed)) { + if (error_code == static_cast(MongoError::UnAuthorized) + || error_code == static_cast(MongoError::AuthenticationFailed)) { raise(fmt::format("Permission error: {}", mongo_error_suffix)); } @@ -57,10 +58,11 @@ void raise_mongo_exception(const mongocxx::operation_exception& e, const std::st } bool is_expected_error_type(int error_code) { - return error_code == static_cast(MongoError::KeyNotFound) || error_code == static_cast(MongoError::NoSuchKey); + return error_code == static_cast(MongoError::KeyNotFound) + || error_code == static_cast(MongoError::NoSuchKey); } -void raise_if_unexpected_error(const mongocxx::operation_exception& e, const std::string& object_name) { +void raise_if_unexpected_error(const mongocxx::operation_exception &e, const std::string &object_name) { auto error_code = e.code().value(); if (!is_expected_error_type(error_code)) { @@ -72,108 +74,89 @@ std::string MongoStorage::name() const { return fmt::format("mongo_storage-{}", db_); } -void MongoStorage::do_write(Composite&& kvs) { - namespace fg = folly::gen; - auto fmt_db = [](auto &&kv) { return kv.key_type(); }; - +void MongoStorage::do_write(KeySegmentPair &&key_seg) { ARCTICDB_SAMPLE(MongoStorageWrite, 0) - (fg::from(kvs.as_range()) | fg::move | fg::groupBy(fmt_db)).foreach([&](auto &&group) { - for (auto &kv : group.values()) { - auto collection = collection_name(kv.key_type()); - auto key_view = kv.key_view(); - try { - auto success = client_->write_segment(db_, collection, std::move(kv)); - storage::check(success, "Mongo did not acknowledge write for key {}", key_view); - } catch (const mongocxx::operation_exception& ex) { - std::string object_name = std::string(key_view); - raise_mongo_exception(ex, object_name); - } - } - }); + auto collection = collection_name(key_seg.key_type()); + auto key_view = key_seg.key_view(); + try { + auto success = client_->write_segment(db_, collection, std::move(key_seg)); + storage::check(success, + "Mongo did not acknowledge write for key {}", + key_view); + } catch (const mongocxx::operation_exception &ex) { + std::string object_name = std::string(key_view); + raise_mongo_exception(ex, object_name); + } } -void MongoStorage::do_update(Composite&& kvs, UpdateOpts opts) { - namespace fg = folly::gen; - auto fmt_db = [](auto &&kv) { return kv.key_type(); }; - +void MongoStorage::do_update(KeySegmentPair &&key_seg, UpdateOpts opts) { ARCTICDB_SAMPLE(MongoStorageWrite, 0) - (fg::from(kvs.as_range()) | fg::move | fg::groupBy(fmt_db)).foreach([&](auto &&group) { - for (auto &kv : group.values()) { - auto collection = collection_name(kv.key_type()); - auto key_view = kv.key_view(); - try { - auto result = client_->update_segment(db_, collection, std::move(kv), opts.upsert_); - storage::check(result.modified_count.has_value(), - "Mongo did not acknowledge write for key {}", - key_view); - if (!opts.upsert_ && result.modified_count.value() == 0) { - throw storage::KeyNotFoundException( - fmt::format("update called with upsert=false but key does not exist: {}", key_view)); - } - } catch (const mongocxx::operation_exception& ex) { - std::string object_name = std::string(key_view); - raise_mongo_exception(ex, object_name); - } + auto collection = collection_name(key_seg.key_type()); + auto key_view = key_seg.key_view(); + try { + auto result = client_->update_segment(db_, collection, std::move(key_seg), opts.upsert_); + storage::check(result.modified_count.has_value(), + "Mongo did not acknowledge write for key {}", + key_view); + if (!opts.upsert_ && result.modified_count.value() == 0) { + throw storage::KeyNotFoundException( + fmt::format("update called with upsert=false but key does not exist: {}", key_view)); } - }); + } catch (const mongocxx::operation_exception &ex) { + std::string object_name = std::string(key_view); + raise_mongo_exception(ex, object_name); + } } -void MongoStorage::do_read(Composite&& ks, const ReadVisitor& visitor, ReadKeyOpts opts) { - namespace fg = folly::gen; - auto fmt_db = [](auto &&k) { return variant_key_type(k); }; - ARCTICDB_SAMPLE(MongoStorageRead, 0) - std::vector keys_not_found; +void MongoStorage::do_read(VariantKey &&variant_key, const ReadVisitor &visitor, ReadKeyOpts opts) { + auto key_seg = do_read(std::move(variant_key), opts); + visitor(key_seg.variant_key(), std::move(key_seg.segment())); +} - (fg::from(ks.as_range()) | fg::move | fg::groupBy(fmt_db)).foreach([&](auto &&group) { - for (auto &k : group.values()) { - auto collection = collection_name(variant_key_type(k)); - try { - auto kv = client_->read_segment(db_, collection, k); - // later we should add the key to failed_reads in this case - if (!kv.has_value()) { - keys_not_found.push_back(k); - } - else { - visitor(k, std::move(kv->segment())); - } +KeySegmentPair MongoStorage::do_read(VariantKey&& variant_key, ReadKeyOpts opts) { + ARCTICDB_SAMPLE(MongoStorageRead, 0) + boost::container::small_vector keys_not_found; - } catch (const mongocxx::operation_exception& ex) { - std::string object_name = std::string(variant_key_view(k)); - raise_if_unexpected_error(ex, object_name); - - log::storage().log( - opts.dont_warn_about_missing_key ? spdlog::level::debug : spdlog::level::warn, - "Failed to find segment for key '{}' {}: {}", - variant_key_view(k), - ex.code().value(), - ex.what()); - keys_not_found.push_back(k); - } + auto collection = collection_name(variant_key_type(variant_key)); + try { + auto kv = client_->read_segment(db_, collection, variant_key); + // later we should add the key to failed_reads in this case + if (!kv.has_value()) { + throw KeyNotFoundException(variant_key); + } else { + return {VariantKey{variant_key}, std::move(kv->segment())}; } - }); - - if (!keys_not_found.empty()) { - throw KeyNotFoundException(Composite{std::move(keys_not_found)}); + } catch (const mongocxx::operation_exception &ex) { + std::string object_name = std::string(variant_key_view(variant_key)); + raise_if_unexpected_error(ex, object_name); + log::storage().log( + opts.dont_warn_about_missing_key ? spdlog::level::debug : spdlog::level::warn, + "Failed to find segment for key '{}' {}: {}", + variant_key_view(variant_key), + ex.code().value(), + ex.what()); + + throw KeyNotFoundException(keys_not_found); } } bool MongoStorage::do_fast_delete() { - foreach_key_type([&] (KeyType key_type) { + foreach_key_type([&](KeyType key_type) { auto collection = collection_name(key_type); client_->drop_collection(db_, collection); }); return true; } -void MongoStorage::do_remove(Composite&& ks, RemoveOpts opts) { +void MongoStorage::do_remove(std::span variant_keys, RemoveOpts opts) { namespace fg = folly::gen; auto fmt_db = [](auto &&k) { return variant_key_type(k); }; ARCTICDB_SAMPLE(MongoStorageRemove, 0) - Composite keys_not_found; + std::vector keys_not_found; - (fg::from(ks.as_range()) | fg::move | fg::groupBy(fmt_db)).foreach([&](auto &&group) { + (fg::from(variant_keys) | fg::move | fg::groupBy(fmt_db)).foreach([&](auto &&group) { for (auto &k : group.values()) { auto collection = collection_name(variant_key_type(k)); try { @@ -186,7 +169,7 @@ void MongoStorage::do_remove(Composite&& ks, RemoveOpts opts) { if (result.delete_count.value() == 0 && !opts.ignores_missing_key_) { keys_not_found.push_back(k); } - } catch (const mongocxx::operation_exception& ex) { + } catch (const mongocxx::operation_exception &ex) { // mongo delete does not throw exception if key not found, it returns 0 as delete count std::string object_name = std::string(variant_key_view(k)); raise_mongo_exception(ex, object_name); @@ -198,13 +181,20 @@ void MongoStorage::do_remove(Composite&& ks, RemoveOpts opts) { } } -bool MongoStorage::do_iterate_type_until_match(KeyType key_type, const IterateTypePredicate& visitor, const std::string &prefix) { +void MongoStorage::do_remove(VariantKey&& variant_key, RemoveOpts opts) { + std::array arr{std::move(variant_key)}; + do_remove(std::span(arr), opts); +} + +bool MongoStorage::do_iterate_type_until_match(KeyType key_type, + const IterateTypePredicate &visitor, + const std::string &prefix) { auto collection = collection_name(key_type); ARCTICDB_SAMPLE(MongoStorageItType, 0) std::vector keys; try { keys = client_->list_keys(db_, collection, key_type, prefix); - } catch (const mongocxx::operation_exception& ex) { + } catch (const mongocxx::operation_exception &ex) { // We don't raise when key is not found because we want to return an empty list instead of raising. raise_if_unexpected_error(ex, collection); log::storage().warn("Failed to iterate key type with key '{}' {}: {}", @@ -214,23 +204,23 @@ bool MongoStorage::do_iterate_type_until_match(KeyType key_type, const IterateTy } for (auto &key : keys) { if (visitor(std::move(key))) { - return true; + return true; } } return false; } -bool MongoStorage::do_is_path_valid(const std::string_view path) const { +bool MongoStorage::do_is_path_valid(std::string_view path) const { return std::none_of(path.cbegin(), path.cend(), [](auto c) { return UNSUPPORTED_MONGO_CHARS.contains(c); }); } -bool MongoStorage::do_key_exists(const VariantKey& key) { +bool MongoStorage::do_key_exists(const VariantKey &key) { auto collection = collection_name(variant_key_type(key)); try { return client_->key_exists(db_, collection, key); - } catch (const mongocxx::operation_exception& ex) { + } catch (const mongocxx::operation_exception &ex) { std::string object_name = std::string(variant_key_view(key)); raise_if_unexpected_error(ex, object_name); } @@ -238,7 +228,6 @@ bool MongoStorage::do_key_exists(const VariantKey& key) { return false; } - using Config = arcticdb::proto::mongo_storage::Config; MongoStorage::MongoStorage( @@ -246,16 +235,16 @@ MongoStorage::MongoStorage( OpenMode mode, const Config &config) : Storage(lib, mode) { - if(config.use_mock_storage_for_testing()) { + if (config.use_mock_storage_for_testing()) { ARCTICDB_RUNTIME_DEBUG(log::storage(), "Using Mock Mongo storage"); client_ = std::make_unique(); } else { ARCTICDB_RUNTIME_DEBUG(log::storage(), "Using Real Mongo storage"); client_ = std::make_unique( - config, - ConfigsMap::instance()->get_int("MongoClient.MinPoolSize", 100), - ConfigsMap::instance()->get_int("MongoClient.MaxPoolSize", 1000), - ConfigsMap::instance()->get_int("MongoClient.SelectionTimeoutMs", 120000)); + config, + ConfigsMap::instance()->get_int("MongoClient.MinPoolSize", 100), + ConfigsMap::instance()->get_int("MongoClient.MaxPoolSize", 1000), + ConfigsMap::instance()->get_int("MongoClient.SelectionTimeoutMs", 120000)); } auto key_rg = lib.as_range(); auto it = key_rg.begin(); diff --git a/cpp/arcticdb/storage/mongo/mongo_storage.hpp b/cpp/arcticdb/storage/mongo/mongo_storage.hpp index fe505b469c..7c2b3a6497 100644 --- a/cpp/arcticdb/storage/mongo/mongo_storage.hpp +++ b/cpp/arcticdb/storage/mongo/mongo_storage.hpp @@ -9,7 +9,7 @@ #include #include -#include +#include #include #include #include @@ -29,17 +29,21 @@ class MongoStorage final : public Storage { std::string name() const final; private: - void do_write(Composite&& kvs) final; + void do_write(KeySegmentPair&& key_seg) final; void do_write_if_none(KeySegmentPair&& kv [[maybe_unused]]) final { storage::raise("Atomic operations are only supported for s3 backend"); }; - void do_update(Composite&& kvs, UpdateOpts opts) final; + void do_update(KeySegmentPair&& key_seg, UpdateOpts opts) final; - void do_read(Composite&& ks, const ReadVisitor& visitor, ReadKeyOpts opts) final; + void do_read(VariantKey&& variant_key, const ReadVisitor& visitor, ReadKeyOpts opts) final; - void do_remove(Composite&& ks, RemoveOpts opts) final; + KeySegmentPair do_read(VariantKey&& variant_key, ReadKeyOpts) final; + + void do_remove(VariantKey&& variant_key, RemoveOpts opts) final; + + void do_remove(std::span variant_keys, RemoveOpts opts) final; bool do_key_exists(const VariantKey& key) final; @@ -57,7 +61,7 @@ class MongoStorage final : public Storage { std::string do_key_path(const VariantKey&) const final { return {}; }; - bool do_is_path_valid(const std::string_view path) const final; + bool do_is_path_valid(std::string_view path) const final; std::string collection_name(KeyType k); diff --git a/cpp/arcticdb/storage/s3/detail-inl.hpp b/cpp/arcticdb/storage/s3/detail-inl.hpp index ceb3fc29cb..68945d4082 100644 --- a/cpp/arcticdb/storage/s3/detail-inl.hpp +++ b/cpp/arcticdb/storage/s3/detail-inl.hpp @@ -1,3 +1,4 @@ + #pragma once #include @@ -10,7 +11,8 @@ #include #include #include -#include +#include +#include #include #include #include @@ -36,109 +38,267 @@ using namespace object_store_utils; namespace s3 { - namespace fg = folly::gen; - namespace detail { - - static const size_t DELETE_OBJECTS_LIMIT = 1000; - - template - using Range = folly::Range; - - inline void raise_s3_exception(const Aws::S3::S3Error& err, const std::string& object_name) { - std::string error_message; - auto type = err.GetErrorType(); - - auto error_message_suffix = fmt::format("S3Error#{} {}: {} for object '{}'", - int(err.GetErrorType()), - err.GetExceptionName().c_str(), - err.GetMessage().c_str(), - object_name); - - // s3_client.HeadObject returns RESOURCE_NOT_FOUND if a key is not found. - if(type == Aws::S3::S3Errors::NO_SUCH_KEY || type == Aws::S3::S3Errors::RESOURCE_NOT_FOUND) { - throw KeyNotFoundException(fmt::format("Key Not Found Error: {}", - error_message_suffix)); - } - - if(type == Aws::S3::S3Errors::ACCESS_DENIED || type == Aws::S3::S3Errors::INVALID_ACCESS_KEY_ID || type == Aws::S3::S3Errors::SIGNATURE_DOES_NOT_MATCH) { - raise(fmt::format("Permission error: {}", - error_message_suffix)); - } - - if(err.ShouldRetry()) { - raise(fmt::format("Retry-able error: {}", - error_message_suffix)); - } - - if (type == Aws::S3::S3Errors::UNKNOWN && err.GetExceptionName().find("Precondition") != std::string::npos) { - raise(fmt::format("Atomic operation failed: {}", error_message_suffix)); - } - - // We create a more detailed error explanation in case of NETWORK_CONNECTION errors to remedy #880. - if (type == Aws::S3::S3Errors::NETWORK_CONNECTION) { - error_message = fmt::format("Unexpected network error: {} " - "This could be due to a connectivity issue or too many open Arctic instances. " - "Having more than one open Arctic instance is not advised, you should reuse them. " - "If you absolutely need many open Arctic instances, consider increasing `ulimit -n`.", - error_message_suffix); - } - else { - error_message = fmt::format("Unexpected error: {}", - error_message_suffix); +namespace fg = folly::gen; +namespace detail { + +static const size_t DELETE_OBJECTS_LIMIT = 1000; + +template +using Range = folly::Range; + +[[noreturn]] inline void raise_s3_exception(const Aws::S3::S3Error& err, const std::string& object_name) { + std::string error_message; + auto type = err.GetErrorType(); + + auto error_message_suffix = fmt::format("S3Error#{} {}: {} for object '{}'", + int(err.GetErrorType()), + err.GetExceptionName().c_str(), + err.GetMessage().c_str(), + object_name); + + // s3_client.HeadObject returns RESOURCE_NOT_FOUND if a key is not found. + if (type == Aws::S3::S3Errors::NO_SUCH_KEY || type == Aws::S3::S3Errors::RESOURCE_NOT_FOUND) { + throw KeyNotFoundException(fmt::format("Key Not Found Error: {}", + error_message_suffix)); + } + + if (type == Aws::S3::S3Errors::ACCESS_DENIED || type == Aws::S3::S3Errors::INVALID_ACCESS_KEY_ID + || type == Aws::S3::S3Errors::SIGNATURE_DOES_NOT_MATCH) { + raise(fmt::format("Permission error: {}", + error_message_suffix)); + } + + if (err.ShouldRetry()) { + raise(fmt::format("Retry-able error: {}", + error_message_suffix)); + } + + // We create a more detailed error explanation in case of NETWORK_CONNECTION errors to remedy #880. + if (type == Aws::S3::S3Errors::NETWORK_CONNECTION) { + error_message = fmt::format("Unexpected network error: {} " + "This could be due to a connectivity issue or too many open Arctic instances. " + "Having more than one open Arctic instance is not advised, you should reuse them. " + "If you absolutely need many open Arctic instances, consider increasing `ulimit -n`.", + error_message_suffix); + } else { + error_message = fmt::format("Unexpected error: {}", + error_message_suffix); + } + + log::storage().error(error_message); + raise(error_message); +} + +inline bool is_expected_error_type(Aws::S3::S3Errors err) { + return err == Aws::S3::S3Errors::NO_SUCH_KEY || err == Aws::S3::S3Errors::RESOURCE_NOT_FOUND + || err == Aws::S3::S3Errors::NO_SUCH_BUCKET; +} + +inline void raise_if_unexpected_error(const Aws::S3::S3Error& err, const std::string& object_name) { + if (!is_expected_error_type(err.GetErrorType())) { + raise_s3_exception(err, object_name); + } +} + +template +void do_write_impl( + KeySegmentPair&& key_seg, + const std::string& root_folder, + const std::string& bucket_name, + S3ClientInterface& s3_client, + KeyBucketizer&& bucketizer) { + ARCTICDB_SAMPLE(S3StorageWrite, 0) + + auto key_type_dir = key_type_folder(root_folder, key_seg.key_type()); + ARCTICDB_TRACE(log::storage(), "S3 key_type_folder is {}", key_type_dir); + + ARCTICDB_SUBSAMPLE(S3StorageWriteValues, 0) + auto& k = key_seg.variant_key(); + auto s3_object_name = object_path(bucketizer.bucketize(key_type_dir, k), k); + auto& seg = key_seg.segment(); + + auto put_object_result = s3_client.put_object(s3_object_name, std::move(seg), bucket_name); + + if (!put_object_result.is_success()) { + auto& error = put_object_result.get_error(); + // No DuplicateKeyException is thrown because S3 overwrites the given key if it already exists. + raise_s3_exception(error, s3_object_name); + } +} + +template +void do_update_impl( + KeySegmentPair&& key_seg, + const std::string& root_folder, + const std::string& bucket_name, + S3ClientInterface& s3_client, + KeyBucketizer&& bucketizer) { + // s3 updates the key if it already exists. We skip the check for key not found to save a round-trip. + do_write_impl(std::move(key_seg), root_folder, bucket_name, s3_client, std::forward(bucketizer)); +} + +template +KeySegmentPair do_read_impl( + VariantKey&& variant_key, + const std::string& root_folder, + const std::string& bucket_name, + const S3ClientInterface& s3_client, + KeyBucketizer&& bucketizer, + KeyDecoder&& key_decoder, + ReadKeyOpts opts) { + ARCTICDB_SAMPLE(S3StorageRead, 0) + auto key_type_dir = key_type_folder(root_folder, variant_key_type(variant_key)); + auto s3_object_name = object_path(bucketizer.bucketize(key_type_dir, variant_key), variant_key); + auto get_object_result = s3_client.get_object(s3_object_name, bucket_name); + auto unencoded_key = key_decoder(std::move(variant_key)); + + if (get_object_result.is_success()) { + ARCTICDB_SUBSAMPLE(S3StorageVisitSegment, 0) + return {VariantKey{unencoded_key}, std::move(get_object_result.get_output())}; + ARCTICDB_DEBUG(log::storage(), "Read key {}: {}", variant_key_type(unencoded_key), variant_key_view(unencoded_key)); + } else { + auto& error = get_object_result.get_error(); + raise_if_unexpected_error(error, s3_object_name); + + log::storage().log( + opts.dont_warn_about_missing_key ? spdlog::level::debug : spdlog::level::warn, + "Failed to find segment for key '{}' {}: {}", + variant_key_view(unencoded_key), + error.GetExceptionName().c_str(), + error.GetMessage().c_str()); + + throw KeyNotFoundException(unencoded_key); + } + return KeySegmentPair{}; +} + +template +folly::Future do_async_read_impl( + VariantKey&& variant_key, + const std::string& root_folder, + const std::string& bucket_name, + const S3ClientInterface& s3_client, + KeyBucketizer&& bucketizer, + KeyDecoder&& key_decoder, + ReadKeyOpts) { + auto key_type_dir = key_type_folder(root_folder, variant_key_type(variant_key)); + auto s3_object_name = object_path(bucketizer.bucketize(key_type_dir, variant_key), variant_key); + return s3_client.get_object_async(s3_object_name, bucket_name).thenValue([vk=std::move(variant_key), decoder=std::forward(key_decoder)] (auto&& result) mutable -> KeySegmentPair { + if(result.is_success()) { + return KeySegmentPair(std::move(vk), std::move(result.get_output())); + } + else { + auto unencoded_key = decoder(std::move(vk)); + raise_s3_exception(result.get_error(), fmt::format("{}", unencoded_key)); + } + }); +} + +template +void do_read_impl( + VariantKey&& variant_key, + const ReadVisitor& visitor, + const std::string& root_folder, + const std::string& bucket_name, + const S3ClientInterface& s3_client, + KeyBucketizer&& bucketizer, + KeyDecoder&& key_decoder, + ReadKeyOpts opts) { + auto key_seg = do_read_impl(std::move(variant_key), root_folder, bucket_name, s3_client, std::forward(bucketizer), std::forward(key_decoder), opts); + visitor(key_seg.variant_key(), std::move(key_seg.segment())); +} + +struct FailedDelete { + VariantKey failed_key; + std::string error_message; + + FailedDelete(VariantKey&& failed_key, std::string&& error_message) : + failed_key(failed_key), + error_message(error_message) {} +}; + +template +void do_remove_impl( + std::span ks, + const std::string& root_folder, + const std::string& bucket_name, + S3ClientInterface& s3_client, + KeyBucketizer&& bucketizer) { + ARCTICDB_SUBSAMPLE(S3StorageDeleteBatch, 0) + auto fmt_db = [](auto&& k) { return variant_key_type(k); }; + std::vector to_delete; + boost::container::small_vector failed_deletes; + static const size_t delete_object_limit = + std::min(DELETE_OBJECTS_LIMIT, + static_cast(ConfigsMap::instance()->get_int("S3Storage.DeleteBatchSize", 1000))); + + to_delete.reserve(std::min(ks.size(), delete_object_limit)); + + (fg::from(ks) | fg::move | fg::groupBy(fmt_db)).foreach( + [&s3_client, &root_folder, &bucket_name, &to_delete, + b = std::forward(bucketizer), &failed_deletes](auto&& group) { + auto key_type_dir = key_type_folder(root_folder, group.key()); + for (auto k : folly::enumerate(group.values())) { + auto s3_object_name = object_path(b.bucketize(key_type_dir, *k), *k); + to_delete.emplace_back(std::move(s3_object_name)); + + if (to_delete.size() == delete_object_limit || k.index + 1 == group.size()) { + auto delete_object_result = s3_client.delete_objects(to_delete, bucket_name); + if (delete_object_result.is_success()) { + ARCTICDB_RUNTIME_DEBUG(log::storage(), "Deleted {} objects, one of which with key '{}'", + to_delete.size(), + variant_key_view(*k)); + for (auto& bad_key : delete_object_result.get_output().failed_deletes) { + auto bad_key_name = bad_key.s3_object_name.substr(key_type_dir.size(), + std::string::npos); + failed_deletes.emplace_back( + variant_key_from_bytes( + reinterpret_cast(bad_key_name.data()), + bad_key_name.size(), group.key()), + std::move(bad_key.error_message)); + } + } else { + auto& error = delete_object_result.get_error(); + std::string failed_objects = fmt::format("{}", fmt::join(to_delete, ", ")); + raise_s3_exception(error, failed_objects); + } + to_delete.clear(); + } } - - log::storage().error(error_message); - raise(error_message); - } - - inline bool is_expected_error_type(Aws::S3::S3Errors err) { - return err == Aws::S3::S3Errors::NO_SUCH_KEY || err == Aws::S3::S3Errors::RESOURCE_NOT_FOUND || err == Aws::S3::S3Errors::NO_SUCH_BUCKET; - } - - inline void raise_if_unexpected_error(const Aws::S3::S3Error& err, const std::string& object_name) { - if (!is_expected_error_type(err.GetErrorType())) { - raise_s3_exception(err, object_name); + }); + + util::check(to_delete.empty(), "Have {} segment that have not been removed", to_delete.size()); + if (!failed_deletes.empty()) { + auto failed_deletes_message = std::ostringstream(); + for (auto i = 0u; i < failed_deletes.size(); ++i) { + auto& failed = failed_deletes[i]; + failed_deletes_message << fmt::format("'{}' failed with '{}'", to_serialized_key(failed.failed_key), failed.error_message); + if (i != failed_deletes.size()) { + failed_deletes_message << ", "; } } - - template - void do_write_impl( - Composite &&kvs, - const std::string &root_folder, - const std::string &bucket_name, - S3ClientWrapper &s3_client, - KeyBucketizer &&bucketizer) { - ARCTICDB_SAMPLE(S3StorageWrite, 0) - auto fmt_db = [](auto &&kv) { return kv.key_type(); }; - - (fg::from(kvs.as_range()) | fg::move | fg::groupBy(fmt_db)).foreach( - [&s3_client, &bucket_name, &root_folder, b = std::move(bucketizer)](auto &&group) { - auto key_type_dir = key_type_folder(root_folder, group.key()); - ARCTICDB_TRACE(log::storage(), "S3 key_type_folder is {}", key_type_dir); - - ARCTICDB_SUBSAMPLE(S3StorageWriteValues, 0) - for (auto &kv: group.values()) { - auto &k = kv.variant_key(); - auto s3_object_name = object_path(b.bucketize(key_type_dir, k), k); - auto &seg = kv.segment(); - - auto put_object_result = s3_client.put_object(s3_object_name, std::move(seg), bucket_name); - - if (!put_object_result.is_success()) { - auto& error = put_object_result.get_error(); - // No DuplicateKeyException is thrown because S3 overwrites the given key if it already exists. - raise_s3_exception(error, s3_object_name); - } - } - }); - } - - template - void do_write_if_none_impl( + auto error_message = fmt::format("Failed to delete some of the objects: {}.", failed_deletes_message.str()); + raise(error_message); + } +} + +template +void do_remove_impl( + VariantKey&& variant_key, + const std::string& root_folder, + const std::string& bucket_name, + S3ClientInterface& s3_client, + KeyBucketizer&& bucketizer) { + std::array arr{std::move(variant_key)}; + do_remove_impl(std::span(arr), root_folder, bucket_name, s3_client, std::forward(bucketizer)); +} + +template +void do_write_if_none_impl( KeySegmentPair &&kv, const std::string &root_folder, const std::string &bucket_name, - S3ClientWrapper &s3_client, + S3ClientInterface &s3_client, KeyBucketizer &&bucketizer) { ARCTICDB_SAMPLE(S3StorageWriteIfNone, 0) auto key_type_dir = key_type_folder(root_folder, kv.key_type()); @@ -154,250 +314,123 @@ namespace s3 { } } - template - void do_update_impl( - Composite &&kvs, - const std::string &root_folder, - const std::string &bucket_name, - S3ClientWrapper &s3_client, - KeyBucketizer &&bucketizer) { - // s3 updates the key if it already exists. We skip the check for key not found to save a round-trip. - do_write_impl(std::move(kvs), root_folder, bucket_name, s3_client, std::move(bucketizer)); - } - - template - void do_read_impl(Composite &&ks, - const ReadVisitor &visitor, - folly::Function key_decoder, - const std::string &root_folder, - const std::string &bucket_name, - const S3ClientWrapper &s3_client, - KeyBucketizer &&bucketizer, - ReadKeyOpts opts) { - ARCTICDB_SAMPLE(S3StorageRead, 0) - auto fmt_db = [](auto &&k) { return variant_key_type(k); }; - std::vector keys_not_found; - - (fg::from(ks.as_range()) | fg::move | fg::groupBy(fmt_db)).foreach( - [&s3_client, &bucket_name, &root_folder, b = std::move(bucketizer), &visitor, &keys_not_found, - &key_decoder, opts = opts](auto &&group) { - - for (auto &k: group.values()) { - auto key_type_dir = key_type_folder(root_folder, variant_key_type(k)); - auto s3_object_name = object_path(b.bucketize(key_type_dir, k), k); - - auto get_object_result = s3_client.get_object( - s3_object_name, - bucket_name); - - auto unencoded_key = key_decoder(std::move(k)); - if (get_object_result.is_success()) { - ARCTICDB_SUBSAMPLE(S3StorageVisitSegment, 0) - - visitor(unencoded_key, std::move(get_object_result.get_output())); - - ARCTICDB_DEBUG(log::storage(), "Read key {}: {}", variant_key_type(unencoded_key), - variant_key_view(unencoded_key)); - } else { - auto &error = get_object_result.get_error(); - raise_if_unexpected_error(error, s3_object_name); - - log::storage().log( - opts.dont_warn_about_missing_key ? spdlog::level::debug : spdlog::level::warn, - "Failed to find segment for key '{}' {}: {}", - variant_key_view(unencoded_key), - error.GetExceptionName().c_str(), - error.GetMessage().c_str()); - - keys_not_found.push_back(unencoded_key); - } - } - }); - if (!keys_not_found.empty()) - throw KeyNotFoundException(Composite{std::move(keys_not_found)}); - } - - struct FailedDelete { - VariantKey failed_key; - std::string error_message; - - FailedDelete(VariantKey&& failed_key, std::string&& error_message): - failed_key(failed_key), - error_message(error_message) {} - }; - - template - void do_remove_impl(Composite &&ks, - const std::string &root_folder, - const std::string &bucket_name, - S3ClientWrapper &s3_client, - KeyBucketizer &&bucketizer) { - ARCTICDB_SUBSAMPLE(S3StorageDeleteBatch, 0) - auto fmt_db = [](auto &&k) { return variant_key_type(k); }; - std::vector to_delete; - std::vector failed_deletes; - static const size_t delete_object_limit = - std::min(DELETE_OBJECTS_LIMIT, - static_cast(ConfigsMap::instance()->get_int("S3Storage.DeleteBatchSize", 1000))); - - (fg::from(ks.as_range()) | fg::move | fg::groupBy(fmt_db)).foreach( - [&s3_client, &root_folder, &bucket_name, &to_delete, b = std::move( - bucketizer), &failed_deletes](auto &&group) { - auto key_type_dir = key_type_folder(root_folder, group.key()); - for (auto k: folly::enumerate(group.values())) { - auto s3_object_name = object_path(b.bucketize(key_type_dir, *k), *k); - to_delete.emplace_back(std::move(s3_object_name)); - - if (to_delete.size() == delete_object_limit || k.index + 1 == group.size()) { - auto delete_object_result = s3_client.delete_objects(to_delete, bucket_name); - if (delete_object_result.is_success()) { - ARCTICDB_RUNTIME_DEBUG(log::storage(), "Deleted {} objects, one of which with key '{}'", - to_delete.size(), - variant_key_view(*k)); - for (auto& bad_key: delete_object_result.get_output().failed_deletes) { - auto bad_key_name = bad_key.s3_object_name.substr(key_type_dir.size(), - std::string::npos); - failed_deletes.emplace_back( - variant_key_from_bytes( - reinterpret_cast(bad_key_name.data()), - bad_key_name.size(), group.key()), - std::move(bad_key.error_message)); - } - } else { - auto& error = delete_object_result.get_error(); - std::string failed_objects = fmt::format("{}", fmt::join(to_delete, ", ")); - raise_s3_exception(error, failed_objects); - } - to_delete.clear(); - } - } - }); - - util::check(to_delete.empty(), "Have {} segment that have not been removed", - to_delete.size()); - if (!failed_deletes.empty()) { - auto failed_deletes_message = std::ostringstream(); - for (auto i=0u; i(error_message); - } - } - - inline auto default_prefix_handler() { - return [](const std::string &prefix, const std::string &key_type_dir, const KeyDescriptor &key_descriptor, - KeyType) { - return !prefix.empty() ? fmt::format("{}/{}*{}", key_type_dir, key_descriptor, prefix) : key_type_dir; - }; - } - - template - bool do_iterate_type_impl( - KeyType key_type, - const IterateTypePredicate &visitor, +template +void do_update_impl( + Composite &&kvs, const std::string &root_folder, const std::string &bucket_name, - const S3ClientWrapper &s3_client, - KeyBucketizer &&bucketizer, - PrefixHandler &&prefix_handler = default_prefix_handler(), - const std::string &prefix = std::string{}) { - ARCTICDB_SAMPLE(S3StorageIterateType, 0) - auto key_type_dir = key_type_folder(root_folder, key_type); - const auto path_to_key_size = key_type_dir.size() + 1 + bucketizer.bucketize_length(key_type); - // if prefix is empty, add / to avoid matching both log and logc when key_type_dir is {root_folder}/log - if (prefix.empty()) { - key_type_dir += "/"; - } - - // Generally we get the key descriptor from the AtomKey, but in the case of iterating version journals - // where we want to have a narrower prefix, we can use the info that it's a version journal and derive - // the Descriptor. - // TODO: Set the IndexDescriptorImpl correctly - KeyDescriptor key_descriptor(prefix, - is_ref_key_class(key_type) ? IndexDescriptorImpl::Type::UNKNOWN - : IndexDescriptorImpl::Type::TIMESTAMP, - FormatType::TOKENIZED); - auto key_prefix = prefix_handler(prefix, key_type_dir, key_descriptor, key_type); - ARCTICDB_RUNTIME_DEBUG(log::storage(), "Searching for objects in bucket {} with prefix {}", bucket_name, - key_prefix); - - auto continuation_token = std::optional(); - do { - auto list_objects_result = s3_client.list_objects(key_prefix, bucket_name, continuation_token); - - if (list_objects_result.is_success()) { - auto& output = list_objects_result.get_output(); - - ARCTICDB_RUNTIME_DEBUG(log::storage(), "Received object list"); - - for (auto& s3_object_name: output.s3_object_names) { - auto key = s3_object_name.substr(path_to_key_size); - ARCTICDB_TRACE(log::version(), "Got object_list: {}, key: {}", s3_object_name, key); - auto k = variant_key_from_bytes( - reinterpret_cast(key.data()), - key.size(), - key_type); - - ARCTICDB_DEBUG(log::storage(), "Iterating key {}: {}", variant_key_type(k), - variant_key_view(k)); - ARCTICDB_SUBSAMPLE(S3StorageVisitKey, 0) - if (visitor(std::move(k))) { - return true; - } - ARCTICDB_SUBSAMPLE(S3StorageCursorNext, 0) - } - - continuation_token = output.next_continuation_token; - } else { - const auto &error = list_objects_result.get_error(); - log::storage().warn("Failed to iterate key type with key '{}' {}: {}", - key_type, - error.GetExceptionName().c_str(), - error.GetMessage().c_str()); - // We don't raise on expected errors like NoSuchKey because we want to return an empty list - // instead of raising. - raise_if_unexpected_error(error, key_prefix); - return false; + S3ClientInterface& s3_client, + KeyBucketizer &&bucketizer) { + // s3 updates the key if it already exists. We skip the check for key not found to save a round-trip. + do_write_impl(std::move(kvs), root_folder, bucket_name, s3_client, std::forward(bucketizer)); +} + +inline auto default_prefix_handler() { + return [](const std::string& prefix, const std::string& key_type_dir, const KeyDescriptor& key_descriptor, + KeyType) { + return !prefix.empty() ? fmt::format("{}/{}*{}", key_type_dir, key_descriptor, prefix) : key_type_dir; + }; +} + +template +bool do_iterate_type_impl( + KeyType key_type, + const IterateTypePredicate& visitor, + const std::string& root_folder, + const std::string& bucket_name, + const S3ClientInterface& s3_client, + KeyBucketizer&& bucketizer, + PrefixHandler&& prefix_handler = default_prefix_handler(), + const std::string& prefix = std::string{}) { + ARCTICDB_SAMPLE(S3StorageIterateType, 0) + auto key_type_dir = key_type_folder(root_folder, key_type); + const auto path_to_key_size = key_type_dir.size() + 1 + bucketizer.bucketize_length(key_type); + // if prefix is empty, add / to avoid matching both 'log' and 'logc' when key_type_dir is {root_folder}/log + if (prefix.empty()) { + key_type_dir += "/"; + } + + // Generally we get the key descriptor from the AtomKey, but in the case of iterating version journals + // where we want to have a narrower prefix, we can use the info that it's a version journal and derive + // the Descriptor. + // TODO: Set the IndexDescriptorImpl correctly + KeyDescriptor key_descriptor(prefix, + is_ref_key_class(key_type) ? IndexDescriptorImpl::Type::UNKNOWN + : IndexDescriptorImpl::Type::TIMESTAMP, + FormatType::TOKENIZED); + auto key_prefix = prefix_handler(prefix, key_type_dir, key_descriptor, key_type); + ARCTICDB_RUNTIME_DEBUG(log::storage(), "Searching for objects in bucket {} with prefix {}", bucket_name, + key_prefix); + + auto continuation_token = std::optional(); + do { + auto list_objects_result = s3_client.list_objects(key_prefix, bucket_name, continuation_token); + if (list_objects_result.is_success()) { + auto& output = list_objects_result.get_output(); + + ARCTICDB_RUNTIME_DEBUG(log::storage(), "Received object list"); + + for (auto& s3_object_name : output.s3_object_names) { + auto key = s3_object_name.substr(path_to_key_size); + ARCTICDB_TRACE(log::version(), "Got object_list: {}, key: {}", s3_object_name, key); + auto k = variant_key_from_bytes( + reinterpret_cast(key.data()), + key.size(), + key_type); + + ARCTICDB_DEBUG(log::storage(), "Iterating key {}: {}", variant_key_type(k), + variant_key_view(k)); + ARCTICDB_SUBSAMPLE(S3StorageVisitKey, 0) + if (visitor(std::move(k))) { + return true; } - } while (continuation_token.has_value()); - return false; - } - - template - bool do_key_exists_impl( - const VariantKey &key, - const std::string &root_folder, - const std::string &bucket_name, - const S3ClientWrapper &s3_client, - KeyBucketizer &&b - ) { - auto key_type_dir = key_type_folder(root_folder, variant_key_type(key)); - auto s3_object_name = object_path(b.bucketize(key_type_dir, key), key); - - auto head_object_result = s3_client.head_object( - s3_object_name, - bucket_name); - - if (!head_object_result.is_success()) { - auto &error = head_object_result.get_error(); - raise_if_unexpected_error(error, s3_object_name); - - ARCTICDB_DEBUG(log::storage(), "Head object returned false for key {} {} {}:{}", - variant_key_view(key), - int(error.GetErrorType()), - error.GetExceptionName().c_str(), - error.GetMessage().c_str()); + ARCTICDB_SUBSAMPLE(S3StorageCursorNext, 0) } - - return head_object_result.is_success(); + continuation_token = output.next_continuation_token; + } else { + const auto& error = list_objects_result.get_error(); + log::storage().warn("Failed to iterate key type with key '{}' {}: {}", + key_type, + error.GetExceptionName().c_str(), + error.GetMessage().c_str()); + // We don't raise on expected errors like NoSuchKey because we want to return an empty list + // instead of raising. + raise_if_unexpected_error(error, key_prefix); + return false; } - - } // namespace detail - } // namespace s3 -} // namespace arcticdb::storage \ No newline at end of file + } while (continuation_token.has_value()); + return false; +} + +template +bool do_key_exists_impl( + const VariantKey& key, + const std::string& root_folder, + const std::string& bucket_name, + const S3ClientInterface& s3_client, + KeyBucketizer&& b +) { + auto key_type_dir = key_type_folder(root_folder, variant_key_type(key)); + auto s3_object_name = object_path(b.bucketize(key_type_dir, key), key); + + auto head_object_result = s3_client.head_object( + s3_object_name, + bucket_name); + + if (!head_object_result.is_success()) { + auto& error = head_object_result.get_error(); + raise_if_unexpected_error(error, s3_object_name); + + ARCTICDB_DEBUG(log::storage(), "Head object returned false for key {} {} {}:{}", + variant_key_view(key), + int(error.GetErrorType()), + error.GetExceptionName().c_str(), + error.GetMessage().c_str()); + } + + return head_object_result.is_success(); +} + +} // namespace detail +} // namespace s3 +} // namespace arcticdb::storage diff --git a/cpp/arcticdb/storage/s3/nfs_backed_storage.cpp b/cpp/arcticdb/storage/s3/nfs_backed_storage.cpp index f065b966cd..2ebd300ccb 100644 --- a/cpp/arcticdb/storage/s3/nfs_backed_storage.cpp +++ b/cpp/arcticdb/storage/s3/nfs_backed_storage.cpp @@ -6,12 +6,11 @@ */ #include - -#include +#include #include -#include -#include -#include +#include +#include +#include namespace arcticdb::storage::nfs_backed { @@ -93,7 +92,6 @@ std::string get_root_folder(const std::string& root_folder, const VariantKey& vk }); } - std::string NfsBucketizer::bucketize(const std::string& root_folder, const VariantKey& vk) { return get_root_folder(root_folder, vk); } @@ -103,20 +101,19 @@ size_t NfsBucketizer::bucketize_length(KeyType key_type) { } VariantKey unencode_object_id(const VariantKey& key) { - return util::variant_match(key, - [] (const AtomKey& k) { - auto decoded_id = decode_item(k.id(), false); - auto start_index = decode_item(k.start_index(), false); - auto end_index = decode_item(k.end_index(), false); - return VariantKey{atom_key_builder().version_id(k.version_id()).start_index(start_index) - .end_index(end_index).creation_ts(k.creation_ts()).content_hash(k.content_hash()) - .build(decoded_id, k.type())}; - }, - [](const RefKey& r) { - auto decoded_id = decode_item(r.id(), true); - return VariantKey{RefKey{decoded_id, r.type(), r.is_old_type()}}; - }); + [] (const AtomKey& k) { + auto decoded_id = decode_item(k.id(), false); + auto start_index = decode_item(k.start_index(), false); + auto end_index = decode_item(k.end_index(), false); + return VariantKey{atom_key_builder().version_id(k.version_id()).start_index(start_index) + .end_index(end_index).creation_ts(k.creation_ts()).content_hash(k.content_hash()) + .build(decoded_id, k.type())}; + }, + [](const RefKey& r) { + auto decoded_id = decode_item(r.id(), true); + return VariantKey{RefKey{decoded_id, r.type(), r.is_old_type()}}; + }); } NfsBackedStorage::NfsBackedStorage(const LibraryPath &library_path, OpenMode mode, const Config &conf) : @@ -130,7 +127,7 @@ NfsBackedStorage::NfsBackedStorage(const LibraryPath &library_path, OpenMode mod log::storage().warn("Using Mock S3 storage for NfsBackedStorage"); s3_client_ = std::make_unique(); } else { - s3_client_ = std::make_unique(s3::get_aws_credentials(conf), s3::get_s3_config(conf), Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Never, false); + s3_client_ = std::make_unique(s3::get_aws_credentials(conf), s3::get_s3_config(conf), Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Never, false); } if (conf.prefix().empty()) { @@ -156,33 +153,41 @@ std::string NfsBackedStorage::name() const { return fmt::format("nfs_backed_storage-{}/{}/{}", region_, bucket_name_, root_folder_); } -void NfsBackedStorage::do_write(Composite&& kvs) { - auto enc = kvs.transform([] (auto&& key_seg) { - return KeySegmentPair{encode_object_id(key_seg.variant_key()), key_seg.segment_ptr()}; - }); +void NfsBackedStorage::do_write(KeySegmentPair&& key_seg) { + auto enc = KeySegmentPair{encode_object_id(key_seg.variant_key()), key_seg.segment_ptr()}; s3::detail::do_write_impl(std::move(enc), root_folder_, bucket_name_, *s3_client_, NfsBucketizer{}); } -void NfsBackedStorage::do_update(Composite&& kvs, UpdateOpts) { - auto enc = kvs.transform([] (auto&& key_seg) { - return KeySegmentPair{encode_object_id(key_seg.variant_key()), key_seg.segment_ptr()}; - }); +void NfsBackedStorage::do_update(KeySegmentPair&& key_seg, UpdateOpts) { + auto enc = KeySegmentPair{encode_object_id(key_seg.variant_key()), key_seg.segment_ptr()}; s3::detail::do_update_impl(std::move(enc), root_folder_, bucket_name_, *s3_client_, NfsBucketizer{}); } -void NfsBackedStorage::do_read(Composite&& ks, const ReadVisitor& visitor, ReadKeyOpts opts) { - auto enc = ks.transform([] (auto&& key) { - return encode_object_id(key); - }); +void NfsBackedStorage::do_read(VariantKey&& variant_key, const ReadVisitor& visitor, ReadKeyOpts opts) { + auto encoded_key = encode_object_id(variant_key); + auto decoder = [] (auto&& k) { return unencode_object_id(std::move(k)); }; + s3::detail::do_read_impl(std::move(variant_key), visitor, root_folder_, bucket_name_, *s3_client_, NfsBucketizer{}, std::move(decoder), opts); +} - s3::detail::do_read_impl(std::move(enc), visitor, unencode_object_id, root_folder_, bucket_name_, *s3_client_, NfsBucketizer{}, opts); +KeySegmentPair NfsBackedStorage::do_read(VariantKey&& variant_key, ReadKeyOpts opts) { + auto encoded_key = encode_object_id(variant_key); + auto decoder = [] (auto&& k) { return unencode_object_id(std::move(k)); }; + auto key_seg = s3::detail::do_read_impl(std::move(encoded_key), root_folder_, bucket_name_, *s3_client_, NfsBucketizer{}, std::move(decoder), opts); + return {unencode_object_id(key_seg.variant_key()), std::move(key_seg.segment())}; } -void NfsBackedStorage::do_remove(Composite&& ks, RemoveOpts) { - auto enc = ks.transform([] (auto&& key) { +void NfsBackedStorage::do_remove(VariantKey&& variant_key, RemoveOpts) { + auto enc = encode_object_id(variant_key); + s3::detail::do_remove_impl(std::move(enc), root_folder_, bucket_name_, *s3_client_, NfsBucketizer{}); +} + +void NfsBackedStorage::do_remove(std::span variant_keys, RemoveOpts) { + std::vector enc; + enc.reserve(variant_keys.size()); + std::transform(std::begin(variant_keys), std::end(variant_keys), std::back_inserter(enc), [] (auto&& key) { return encode_object_id(key); }); - s3::detail::do_remove_impl(std::move(enc), root_folder_, bucket_name_, *s3_client_, NfsBucketizer{}); + s3::detail::do_remove_impl(std::span(enc), root_folder_, bucket_name_, *s3_client_, NfsBucketizer{}); } bool NfsBackedStorage::do_iterate_type_until_match(KeyType key_type, const IterateTypePredicate& visitor, const std::string& prefix) { diff --git a/cpp/arcticdb/storage/s3/nfs_backed_storage.hpp b/cpp/arcticdb/storage/s3/nfs_backed_storage.hpp index d9b72f7ed0..1b3af5839c 100644 --- a/cpp/arcticdb/storage/s3/nfs_backed_storage.hpp +++ b/cpp/arcticdb/storage/s3/nfs_backed_storage.hpp @@ -19,7 +19,7 @@ #include #include -#include +#include #include namespace arcticdb::storage::nfs_backed { @@ -33,17 +33,21 @@ class NfsBackedStorage final : public Storage { std::string name() const final; private: - void do_write(Composite&& kvs) final; + void do_write(KeySegmentPair&& key_seg) final; void do_write_if_none(KeySegmentPair&& kv [[maybe_unused]]) final { storage::raise("Atomic operations are only supported for s3 backend"); }; - void do_update(Composite&& kvs, UpdateOpts opts) final; + void do_update(KeySegmentPair&& key_seg, UpdateOpts opts) final; - void do_read(Composite&& ks, const ReadVisitor& visitor, ReadKeyOpts opts) final; + void do_read(VariantKey&& variant_key, const ReadVisitor& visitor, ReadKeyOpts opts) final; - void do_remove(Composite&& ks, RemoveOpts opts) final; + KeySegmentPair do_read(VariantKey&& variant_key, ReadKeyOpts opts) final; + + void do_remove(VariantKey&& variant_key, RemoveOpts opts) final; + + void do_remove(std::span variant_keys, RemoveOpts opts) final; bool do_iterate_type_until_match(KeyType key_type, const IterateTypePredicate& visitor, const std::string &prefix) final; @@ -69,7 +73,7 @@ class NfsBackedStorage final : public Storage { const std::string& region() const { return region_; } std::shared_ptr s3_api_; - std::unique_ptr s3_client_; + std::unique_ptr s3_client_; std::string root_folder_; std::string bucket_name_; std::string region_; diff --git a/cpp/arcticdb/storage/s3/s3_api.cpp b/cpp/arcticdb/storage/s3/s3_api.cpp index 675c7c9b59..feeb86b1af 100644 --- a/cpp/arcticdb/storage/s3/s3_api.cpp +++ b/cpp/arcticdb/storage/s3/s3_api.cpp @@ -22,6 +22,7 @@ S3ApiInstance::S3ApiInstance(Aws::Utils::Logging::LogLevel log_level) : // Use correct URI encoding rather than legacy compat one in AWS SDK. PURE S3 needs this to handle symbol names // that have special characters (eg ':'). options_.httpOptions.compliantRfc3986Encoding = true; + //options_.httpOptions.httpClientFactory_create_fn = [](){ return Aws::MakeShared("ARCTICDB_HTTP_CLIENT"); }; if(log_level_ > Aws::Utils::Logging::LogLevel::Off) { Aws::Utils::Logging::InitializeAWSLogging( diff --git a/cpp/arcticdb/storage/s3/s3_real_client.cpp b/cpp/arcticdb/storage/s3/s3_client_impl.cpp similarity index 71% rename from cpp/arcticdb/storage/s3/s3_real_client.cpp rename to cpp/arcticdb/storage/s3/s3_client_impl.cpp index 2f5d6e79de..83b990b77d 100644 --- a/cpp/arcticdb/storage/s3/s3_real_client.cpp +++ b/cpp/arcticdb/storage/s3/s3_client_impl.cpp @@ -5,8 +5,8 @@ * 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 #include @@ -38,7 +38,7 @@ using namespace object_store_utils; namespace s3 { -S3Result RealS3Client::head_object( +S3Result S3ClientImpl::head_object( const std::string& s3_object_name, const std::string &bucket_name) const { @@ -111,11 +111,11 @@ Aws::IOStreamFactory S3StreamFactory() { return [=]() { return Aws::New(""); }; } -S3Result RealS3Client::get_object( +S3Result S3ClientImpl::get_object( const std::string &s3_object_name, const std::string &bucket_name) const { - ARCTICDB_RUNTIME_DEBUG(log::storage(), "Looking for object {}", s3_object_name); + auto start = util::SysClock::coarse_nanos_since_epoch(); Aws::S3::Model::GetObjectRequest request; request.WithBucket(bucket_name.c_str()).WithKey(s3_object_name.c_str()); request.SetResponseStreamFactory(S3StreamFactory()); @@ -124,13 +124,59 @@ S3Result RealS3Client::get_object( if (!outcome.IsSuccess()) { return {outcome.GetError()}; } - auto &retrieved = dynamic_cast(outcome.GetResult().GetBody()); - ARCTICDB_RUNTIME_DEBUG(log::storage(), "Returning object {}", s3_object_name); + auto &retrieved = dynamic_cast(outcome.GetResult().GetBody()); + auto nanos = util::SysClock::coarse_nanos_since_epoch() - start; + auto time_taken = double(nanos) / BILLION; + ARCTICDB_RUNTIME_DEBUG(log::storage(), "Returning object {} in {}s", s3_object_name, time_taken); return {Segment::from_buffer(retrieved.get_buffer())}; } -S3Result RealS3Client::put_object( +struct GetObjectAsyncHandler { + std::shared_ptr>> promise_; + timestamp start_; + + GetObjectAsyncHandler(std::shared_ptr>>&& promise) : + promise_(std::move(promise)), + start_(util::SysClock::coarse_nanos_since_epoch()){ + } + + ARCTICDB_MOVE_COPY_DEFAULT(GetObjectAsyncHandler) + + void operator()( + const Aws::S3::S3Client*, + const Aws::S3::Model::GetObjectRequest& request, + const Aws::S3::Model::GetObjectOutcome& outcome, + const std::shared_ptr&) { + if (outcome.IsSuccess()) { + auto& body = const_cast(outcome).GetResultWithOwnership().GetBody(); + auto& stream = dynamic_cast(body); + auto nanos = util::SysClock::coarse_nanos_since_epoch() - start_; + auto time_taken = double(nanos) / BILLION; + //log::storage().info("Returning object in {} seconds {}", time_taken, request.GetKey()); + ARCTICDB_RUNTIME_DEBUG(log::storage(), "Returning object {} in {}", request.GetKey(), time_taken); + promise_->setValue>({Segment::from_buffer(stream.get_buffer())}); + } else { + //log::storage().warn("Got S3 Error: {}", outcome.GetError().GetMessage()); + promise_->setValue>({outcome.GetError()}); + } +} +}; + +folly::Future> S3ClientImpl::get_object_async( + const std::string &s3_object_name, + const std::string &bucket_name) const { + auto promise = std::make_shared>>(); + auto future = promise->getFuture().via(&async::io_executor()); + Aws::S3::Model::GetObjectRequest request; + request.WithBucket(bucket_name.c_str()).WithKey(s3_object_name.c_str()); + request.SetResponseStreamFactory(S3StreamFactory()); + //log::version().info("Scheduling async read of {}", s3_object_name.c_str()); + s3_client.GetObjectAsync(request, GetObjectAsyncHandler{std::move(promise)}); + return future; +} + +S3Result S3ClientImpl::put_object( const std::string &s3_object_name, Segment &&segment, const std::string &bucket_name, @@ -146,27 +192,23 @@ S3Result RealS3Client::put_object( ARCTICDB_RUNTIME_DEBUG(log::storage(), "Set s3 key {}", request.GetKey().c_str()); auto [dst, write_size, buffer] = segment.serialize_header(); - auto body = std::make_shared( - reinterpret_cast(dst), write_size); + auto body = std::make_shared(reinterpret_cast(dst), write_size); util::check(body->good(), "Overflow of bufferstream with size {}", write_size); request.SetBody(body); ARCTICDB_SUBSAMPLE(S3StoragePutObject, 0) auto outcome = s3_client.PutObject(request); - if (!outcome.IsSuccess()) { return {outcome.GetError()}; } - ARCTICDB_RUNTIME_DEBUG(log::storage(), "Wrote key '{}', with {} bytes of data", - s3_object_name, - segment.size()); + + ARCTICDB_RUNTIME_DEBUG(log::storage(), "Wrote key '{}', with {} bytes of data", s3_object_name,segment.size()); return {std::monostate()}; } -S3Result RealS3Client::delete_objects( +S3Result S3ClientImpl::delete_objects( const std::vector& s3_object_names, const std::string& bucket_name) { - Aws::S3::Model::DeleteObjectsRequest request; request.WithBucket(bucket_name.c_str()); Aws::S3::Model::Delete del_objects; @@ -179,10 +221,10 @@ S3Result RealS3Client::delete_objects( request.SetDelete(del_objects); auto outcome = s3_client.DeleteObjects(request); - if (!outcome.IsSuccess()) { return {outcome.GetError()}; } + // AN-256: Per AWS S3 documentation, deleting non-exist objects is not an error, so not handling // RemoveOpts.ignores_missing_key_ std::vector failed_deletes; @@ -191,14 +233,13 @@ S3Result RealS3Client::delete_objects( } DeleteOutput result = {failed_deletes}; - return {result}; } -S3Result RealS3Client::list_objects( +S3Result S3ClientImpl::list_objects( const std::string& name_prefix, const std::string& bucket_name, - const std::optional continuation_token) const { + const std::optional& continuation_token) const { ARCTICDB_RUNTIME_DEBUG(log::storage(), "Searching for objects in bucket {} with prefix {}", bucket_name, name_prefix); diff --git a/cpp/arcticdb/storage/s3/s3_real_client.hpp b/cpp/arcticdb/storage/s3/s3_client_impl.hpp similarity index 57% rename from cpp/arcticdb/storage/s3/s3_real_client.hpp rename to cpp/arcticdb/storage/s3/s3_client_impl.hpp index 414178e507..31dc77cda4 100644 --- a/cpp/arcticdb/storage/s3/s3_real_client.hpp +++ b/cpp/arcticdb/storage/s3/s3_client_impl.hpp @@ -9,7 +9,7 @@ #include -#include +#include #include #include @@ -23,17 +23,25 @@ #include #include -namespace arcticdb::storage::s3{ +namespace arcticdb::storage::s3 { -// A real S3ClientWrapper around Aws::S3::Client, which executes actual requests to S3 storage. -class RealS3Client : public S3ClientWrapper { +class S3ClientImpl : public S3ClientInterface { public: template - RealS3Client(Args && ...args):s3_client(std::forward(args)...){}; + S3ClientImpl(Args&& ...args) : + s3_client(std::forward(args)...) {}; - S3Result head_object(const std::string& s3_object_name, const std::string& bucket_name) const override; + S3Result head_object( + const std::string& s3_object_name, + const std::string& bucket_name) const override; - S3Result get_object(const std::string& s3_object_name, const std::string& bucket_name) const override; + S3Result get_object( + const std::string& s3_object_name, + const std::string& bucket_name) const override; + + folly::Future> get_object_async( + const std::string& s3_object_name, + const std::string& bucket_name) const override; S3Result put_object( const std::string& s3_object_name, @@ -42,13 +50,13 @@ class RealS3Client : public S3ClientWrapper { PutHeader header = PutHeader::NONE) override; S3Result delete_objects( - const std::vector& s3_object_names, - const std::string& bucket_name) override; + const std::vector& s3_object_names, + const std::string& bucket_name) override; S3Result list_objects( - const std::string& prefix, - const std::string& bucket_name, - const std::optional continuation_token) const override; + const std::string& prefix, + const std::string& bucket_name, + const std::optional& continuation_token) const override; private: Aws::S3::S3Client s3_client; }; diff --git a/cpp/arcticdb/storage/s3/s3_client_wrapper.hpp b/cpp/arcticdb/storage/s3/s3_client_interface.hpp similarity index 55% rename from cpp/arcticdb/storage/s3/s3_client_wrapper.hpp rename to cpp/arcticdb/storage/s3/s3_client_interface.hpp index 830629f44f..4b9d428965 100644 --- a/cpp/arcticdb/storage/s3/s3_client_wrapper.hpp +++ b/cpp/arcticdb/storage/s3/s3_client_interface.hpp @@ -16,7 +16,7 @@ #include #include -#include +#include #include #include #include @@ -26,7 +26,23 @@ namespace arcticdb::storage{ using namespace object_store_utils; -namespace s3{ +namespace s3 { + +template +struct StorageResult { + std::variant result; + + [[nodiscard]] bool is_success() const { + return std::holds_alternative(result); + } + + Error& get_error() { + return std::get(result); + } + Output& get_output() { + return std::get(result); + } +}; template using S3Result = StorageResult; @@ -53,28 +69,36 @@ enum class PutHeader{ // An abstract class, which is responsible for sending the requests and parsing the responses from S3. // It can be derived as either a real connection to S3 or a mock used for unit tests. -class S3ClientWrapper { +class S3ClientInterface { public: - virtual S3Result head_object(const std::string& s3_object_name, const std::string& bucket_name) const = 0; + [[nodiscard]] virtual S3Result head_object( + const std::string& s3_object_name, + const std::string& bucket_name) const = 0; + + [[nodiscard]] virtual S3Result get_object( + const std::string& s3_object_name, + const std::string& bucket_name) const = 0; - virtual S3Result get_object(const std::string& s3_object_name, const std::string& bucket_name) const = 0; + [[nodiscard]] virtual folly::Future> get_object_async( + const std::string& s3_object_name, + const std::string& bucket_name) const = 0; virtual S3Result put_object( - const std::string& s3_object_name, - Segment&& segment, - const std::string& bucket_name, - PutHeader header = PutHeader::NONE) = 0; + const std::string& s3_object_name, + Segment&& segment, + const std::string& bucket_name, + PutHeader header = PutHeader::NONE) = 0; virtual S3Result delete_objects( - const std::vector& s3_object_names, - const std::string& bucket_name) = 0; + const std::vector& s3_object_names, + const std::string& bucket_name) = 0; - virtual S3Result list_objects( - const std::string& prefix, - const std::string& bucket_name, - const std::optional continuation_token) const = 0; + [[nodiscard]] virtual S3Result list_objects( + const std::string& prefix, + const std::string& bucket_name, + const std::optional& continuation_token) const = 0; - virtual ~S3ClientWrapper() = default; + virtual ~S3ClientInterface() = default; }; } diff --git a/cpp/arcticdb/storage/s3/s3_settings.hpp b/cpp/arcticdb/storage/s3/s3_settings.hpp index c529fd86ee..ae6cd901d0 100644 --- a/cpp/arcticdb/storage/s3/s3_settings.hpp +++ b/cpp/arcticdb/storage/s3/s3_settings.hpp @@ -44,6 +44,7 @@ class S3Settings { aws_auth_(aws_auth), aws_profile_(aws_profile) { } + explicit S3Settings(const arcticc::pb2::s3_storage_pb2::Config& config) { update(config); } diff --git a/cpp/arcticdb/storage/s3/s3_storage.cpp b/cpp/arcticdb/storage/s3/s3_storage.cpp index 9867ce8c79..6c55ebbcb0 100644 --- a/cpp/arcticdb/storage/s3/s3_storage.cpp +++ b/cpp/arcticdb/storage/s3/s3_storage.cpp @@ -20,11 +20,9 @@ #include #include #include -#include - #include -#include -#include +#include +#include #include #undef GetMessage @@ -35,8 +33,6 @@ using namespace object_store_utils; namespace s3 { -namespace fg = folly::gen; - std::string S3Storage::name() const { return fmt::format("s3_storage-{}/{}/{}", region_, bucket_name_, root_folder_); } @@ -49,24 +45,47 @@ std::string S3Storage::get_key_path(const VariantKey& key) const { // to most of them } -void S3Storage::do_write(Composite&& kvs) { - detail::do_write_impl(std::move(kvs), root_folder_, bucket_name_, *s3_client_, FlatBucketizer{}); +void S3Storage::do_write(KeySegmentPair&& key_seg) { + detail::do_write_impl(std::move(key_seg), root_folder_, bucket_name_, client(), FlatBucketizer{}); } void S3Storage::do_write_if_none(KeySegmentPair&& kv) { detail::do_write_if_none_impl(std::move(kv), root_folder_, bucket_name_, *s3_client_, FlatBucketizer{}); } -void S3Storage::do_update(Composite&& kvs, UpdateOpts) { - detail::do_update_impl(std::move(kvs), root_folder_, bucket_name_, *s3_client_, FlatBucketizer{}); +void S3Storage::do_update(KeySegmentPair&& key_seg, UpdateOpts) { + detail::do_update_impl(std::move(key_seg), root_folder_, bucket_name_, client(), FlatBucketizer{}); +} + +void S3Storage::do_read(VariantKey&& variant_key, const ReadVisitor& visitor, ReadKeyOpts opts) { + auto identity = [](auto&& k) { return k; }; + detail::do_read_impl(std::move(variant_key), visitor, root_folder_, bucket_name_, client(), FlatBucketizer{}, std::move(identity), opts); +} + +KeySegmentPair S3Storage::do_read(VariantKey&& variant_key, ReadKeyOpts opts) { + auto identity = [](auto&& k) { return k; }; + return detail::do_read_impl(std::move(variant_key), root_folder_, bucket_name_, client(), FlatBucketizer{}, std::move(identity), opts); +} + +folly::Future S3Storage::do_async_read(entity::VariantKey&& variant_key, const ReadVisitor& visitor, ReadKeyOpts opts) { + auto identity = [](auto&& k) { return k; }; + return detail::do_async_read_impl(std::move(variant_key), root_folder_, bucket_name_, client(), FlatBucketizer{}, std::move(identity), opts).thenValue([&visitor] (auto&& key_seg) { + visitor(key_seg.variant_key(), std::move(key_seg.segment())); + return folly::Unit{}; + }); } -void S3Storage::do_read(Composite&& ks, const ReadVisitor& visitor, ReadKeyOpts opts) { - detail::do_read_impl(std::move(ks), visitor, folly::identity, root_folder_, bucket_name_, *s3_client_, FlatBucketizer{}, opts); +folly::Future S3Storage::do_async_read(entity::VariantKey&& variant_key, ReadKeyOpts opts) { + auto identity = [](auto&& k) { return k; }; + return detail::do_async_read_impl(std::move(variant_key), root_folder_, bucket_name_, client(), FlatBucketizer{}, std::move(identity), opts); } -void S3Storage::do_remove(Composite&& ks, RemoveOpts) { - detail::do_remove_impl(std::move(ks), root_folder_, bucket_name_, *s3_client_, FlatBucketizer{}); +void S3Storage::do_remove(std::span variant_keys, RemoveOpts) { + detail::do_remove_impl(variant_keys, root_folder_, bucket_name_, client(), FlatBucketizer{}); +} + +void S3Storage::do_remove(VariantKey&& variant_key, RemoveOpts) { + detail::do_remove_impl(std::move(variant_key), root_folder_, bucket_name_, client(), FlatBucketizer{}); } bool S3Storage::do_iterate_type_until_match(KeyType key_type, const IterateTypePredicate& visitor, const std::string& prefix) { @@ -74,27 +93,19 @@ bool S3Storage::do_iterate_type_until_match(KeyType key_type, const IterateTypeP return !prefix.empty() ? fmt::format("{}/{}*{}", key_type_dir, key_descriptor, prefix) : key_type_dir; }; - return detail::do_iterate_type_impl(key_type, visitor, root_folder_, bucket_name_, *s3_client_, FlatBucketizer{}, std::move(prefix_handler), prefix); + return detail::do_iterate_type_impl(key_type, visitor, root_folder_, bucket_name_, client(), FlatBucketizer{}, std::move(prefix_handler), prefix); } bool S3Storage::do_key_exists(const VariantKey& key) { - return detail::do_key_exists_impl(key, root_folder_, bucket_name_, *s3_client_, FlatBucketizer{}); + return detail::do_key_exists_impl(key, root_folder_, bucket_name_, client(), FlatBucketizer{}); } } // namespace s3 } // namespace arcticdb::storage - namespace arcticdb::storage::s3 { -S3Storage::S3Storage(const LibraryPath &library_path, OpenMode mode, const S3Settings &conf) : - Storage(library_path, mode), - s3_api_(S3ApiInstance::instance()), // make sure we have an initialized AWS SDK - root_folder_(object_store_utils::get_root_folder(library_path)), - bucket_name_(conf.bucket_name()), - region_(conf.region()) { - auto creds = get_aws_credentials(conf); - +void S3Storage::create_s3_client(const S3Settings &conf, const Aws::Auth::AWSCredentials& creds) { if (conf.use_mock_storage_for_testing()){ ARCTICDB_RUNTIME_DEBUG(log::storage(), "Using Mock S3 storage"); s3_client_ = std::make_unique(); @@ -109,20 +120,31 @@ S3Storage::S3Storage(const LibraryPath &library_path, OpenMode mode, const S3Set return sts_client_.get(); }; auto cred_provider = Aws::MakeShared( - "DefaultAWSCredentialsProviderChain", - conf.aws_profile(), - std::chrono::minutes(static_cast(ConfigsMap::instance()->get_int("S3Storage.STSTokenExpiryMin", 60))), - sts_client_factory - ); - s3_client_ = std::make_unique(cred_provider, client_config, Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Never, conf.use_virtual_addressing()); + "DefaultAWSCredentialsProviderChain", + conf.aws_profile(), + std::chrono::minutes(static_cast(ConfigsMap::instance()->get_int("S3Storage.STSTokenExpiryMin", 60))), + sts_client_factory + ); + s3_client_ = std::make_unique(cred_provider, client_config, Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Never, conf.use_virtual_addressing()); } else if (creds.GetAWSAccessKeyId() == USE_AWS_CRED_PROVIDERS_TOKEN && creds.GetAWSSecretKey() == USE_AWS_CRED_PROVIDERS_TOKEN){ ARCTICDB_RUNTIME_DEBUG(log::storage(), "Using AWS auth mechanisms"); - s3_client_ = std::make_unique(get_s3_config(conf), Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Never, conf.use_virtual_addressing()); + s3_client_ = std::make_unique(get_s3_config(conf), Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Never, conf.use_virtual_addressing()); } else { ARCTICDB_RUNTIME_DEBUG(log::storage(), "Using provided auth credentials"); - s3_client_ = std::make_unique(creds, get_s3_config(conf), Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Never, conf.use_virtual_addressing()); + s3_client_ = std::make_unique(creds, get_s3_config(conf), Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Never, conf.use_virtual_addressing()); } +} + +S3Storage::S3Storage(const LibraryPath &library_path, OpenMode mode, const S3Settings &conf) : + Storage(library_path, mode), + s3_api_(S3ApiInstance::instance()), // make sure we have an initialized AWS SDK + root_folder_(object_store_utils::get_root_folder(library_path)), + bucket_name_(conf.bucket_name()), + region_(conf.region()) { + auto creds = get_aws_credentials(conf); + + create_s3_client(conf, creds); if (conf.prefix().empty()) { ARCTICDB_DEBUG(log::version(), "prefix not found, will use {}", root_folder_); diff --git a/cpp/arcticdb/storage/s3/s3_storage.hpp b/cpp/arcticdb/storage/s3/s3_storage.hpp index 50e7faff07..f5f9588b63 100644 --- a/cpp/arcticdb/storage/s3/s3_storage.hpp +++ b/cpp/arcticdb/storage/s3/s3_storage.hpp @@ -16,8 +16,8 @@ #include #include #include -#include #include +#include #include #include #include @@ -31,28 +31,41 @@ namespace arcticdb::storage::s3 { const std::string USE_AWS_CRED_PROVIDERS_TOKEN = "_RBAC_"; -class S3Storage final : public Storage { +class S3Storage final : public Storage, AsyncStorage { public: S3Storage(const LibraryPath &lib, OpenMode mode, const S3Settings &conf); - /** - * Full object path in S3 bucket. - */ std::string get_key_path(const VariantKey& key) const; std::string name() const final; + bool has_async_api() const final { + return ConfigsMap::instance()->get_int("S3.Async", 0) == 1; + } + + AsyncStorage* async_api() { + return dynamic_cast(this); + } + private: - void do_write(Composite&& kvs) final; + void do_write(KeySegmentPair&& key_seg) final; void do_write_if_none(KeySegmentPair&& kv) final; - void do_update(Composite&& kvs, UpdateOpts opts) final; + void do_update(KeySegmentPair&& key_seg, UpdateOpts opts) final; + + void do_read(VariantKey&& variant_key, const ReadVisitor& visitor, ReadKeyOpts opts) final; - void do_read(Composite&& ks, const ReadVisitor& visitor, ReadKeyOpts opts) final; + KeySegmentPair do_read(VariantKey&& variant_key, ReadKeyOpts opts) final; - void do_remove(Composite&& ks, RemoveOpts opts) final; + folly::Future do_async_read(entity::VariantKey&& variant_key, const ReadVisitor& visitor, ReadKeyOpts opts) final; + + folly::Future do_async_read(entity::VariantKey&& variant_key, ReadKeyOpts opts) final; + + void do_remove(VariantKey&& variant_key, RemoveOpts opts) final; + + void do_remove(std::span variant_keys, RemoveOpts opts) final; bool do_iterate_type_until_match(KeyType key_type, const IterateTypePredicate& visitor, const std::string &prefix) final; @@ -73,18 +86,19 @@ class S3Storage final : public Storage { return false; } + void create_s3_client(const S3Settings &conf, const Aws::Auth::AWSCredentials& creds); + std::string do_key_path(const VariantKey& key) const final { return get_key_path(key); }; - auto& client() { return s3_client_; } + S3ClientInterface& client() { return *s3_client_; } const std::string& bucket_name() const { return bucket_name_; } const std::string& root_folder() const { return root_folder_; } std::shared_ptr s3_api_; - std::unique_ptr s3_client_; + std::unique_ptr s3_client_; //aws sdk annoyingly requires raw pointer being passed in the sts client factory to the s3 client //thus sts_client_ should have same life span as s3_client_ std::unique_ptr sts_client_; - std::string root_folder_; std::string bucket_name_; std::string region_; @@ -247,11 +261,8 @@ auto get_s3_config(const ConfigType& conf) { client_configuration.caFile = conf.ca_cert_path(); client_configuration.caPath = conf.ca_cert_dir(); } - client_configuration.maxConnections = conf.max_connections() == 0 ? - ConfigsMap::instance()->get_int("VersionStore.NumIOThreads", 16) : - conf.max_connections(); - + client_configuration.maxConnections = ConfigsMap::instance()->get_int("S3Storage.MaxConnections", async::TaskScheduler::instance()->io_thread_count()); client_configuration.connectTimeoutMs = ConfigsMap::instance()->get_int("S3Storage.ConnectTimeoutMs", conf.connect_timeout() == 0 ? 30000 : conf.connect_timeout()); client_configuration.httpRequestTimeoutMs = ConfigsMap::instance()->get_int("S3Storage.HttpRequestTimeoutMs", 0); diff --git a/cpp/arcticdb/storage/storage.hpp b/cpp/arcticdb/storage/storage.hpp index 27abfb5b23..81c9320b24 100644 --- a/cpp/arcticdb/storage/storage.hpp +++ b/cpp/arcticdb/storage/storage.hpp @@ -5,154 +5,74 @@ #include #include #include +#include #include #include #include #include -#include -namespace arcticdb::storage { - -using ReadVisitor = std::function; - -class DuplicateKeyException : public ArcticSpecificException { -public: - explicit DuplicateKeyException(std::string message) : - ArcticSpecificException(message) { } - - explicit DuplicateKeyException(VariantKey key) : - ArcticSpecificException(std::string(variant_key_view(key))), - key_(std::move(key)) {} +#include - [[nodiscard]] const VariantKey &key() const { - return key_; - } -private: - VariantKey key_; -}; - -class NoDataFoundException : public ArcticCategorizedException { -public: - explicit NoDataFoundException(VariantId key) : - ArcticCategorizedException(std::visit([](const auto &key) { return fmt::format("{}", key); }, key)), - key_(key){ - } +#include - explicit NoDataFoundException(const std::string& msg) : - ArcticCategorizedException(msg) { - } - - explicit NoDataFoundException(const char* msg) : - ArcticCategorizedException(std::string(msg)) { - } - - [[nodiscard]] const VariantId &key() const { - util::check(static_cast(key_), "Key not found"); - return *key_; - } -private: - std::optional key_; -}; - -class KeyNotFoundException : public ArcticSpecificException { -public: - explicit KeyNotFoundException(std::string message) : - ArcticSpecificException(message) { - } - - explicit KeyNotFoundException(Composite&& keys) : - ArcticSpecificException(fmt::format("Not found: {}", keys)), - keys_(std::make_shared>(std::move(keys))) { - } - - explicit KeyNotFoundException(Composite&& keys, std::string err_output) : - ArcticSpecificException(err_output), - keys_(std::make_shared>(std::move(keys))) { - } - - explicit KeyNotFoundException(const VariantKey& single_key): - KeyNotFoundException(Composite{VariantKey{single_key}}) {} - - explicit KeyNotFoundException(const VariantKey& single_key, std::string err_output): - KeyNotFoundException(Composite{VariantKey{single_key}}, err_output) {} - - - Composite& keys() { - return *keys_; - } -private: - std::shared_ptr> keys_; - mutable std::string msg_; -}; +namespace arcticdb::storage { class Storage { public: - Storage(LibraryPath library_path, OpenMode mode) : lib_path_(std::move(library_path)), mode_(mode) {} virtual ~Storage() = default; + Storage(const Storage&) = delete; Storage& operator=(const Storage&) = delete; Storage(Storage&&) = default; Storage& operator=(Storage&&) = delete; - void write(Composite &&kvs) { + void write(KeySegmentPair&& key_seg) { ARCTICDB_SAMPLE(StorageWrite, 0) - return do_write(std::move(kvs)); - } - - void write(KeySegmentPair &&kv) { - return write(Composite{std::move(kv)}); + return do_write(std::move(key_seg)); } void write_if_none(KeySegmentPair&& kv) { return do_write_if_none(std::move(kv)); } - void update(Composite &&kvs, UpdateOpts opts) { + void update(KeySegmentPair&& key_seg, UpdateOpts opts) { ARCTICDB_SAMPLE(StorageUpdate, 0) - return do_update(std::move(kvs), opts); + return do_update(std::move(key_seg), opts); } - void update(KeySegmentPair &&kv, UpdateOpts opts) { - return update(Composite{std::move(kv)}, opts); + void read(VariantKey&& variant_key, const ReadVisitor& visitor, ReadKeyOpts opts) { + return do_read(std::move(variant_key), visitor, opts); } - void read(Composite &&ks, const ReadVisitor& visitor, ReadKeyOpts opts) { - return do_read(std::move(ks), visitor, opts); + KeySegmentPair read(VariantKey&& variant_key, ReadKeyOpts opts) { + return do_read(std::move(variant_key), opts); } - void read(VariantKey&& key, const ReadVisitor& visitor, ReadKeyOpts opts) { - return read(Composite{std::move(key)}, visitor, opts); + [[nodiscard]] virtual bool has_async_api() const { + return false; } - template - KeySegmentPair read(KeyType&& key, ReadKeyOpts opts) { - KeySegmentPair key_seg; - const ReadVisitor& visitor = [&key_seg](const VariantKey & vk, Segment&& value) { - key_seg.set_key(vk); - key_seg.segment() = std::move(value); - }; - - read(std::forward(key), visitor, opts); - return key_seg; + virtual AsyncStorage* async_api() { + util::raise_rte("Request for async API on non-async storage"); } - void remove(Composite &&ks, RemoveOpts opts) { - do_remove(std::move(ks), opts); + void remove(VariantKey&& variant_key, RemoveOpts opts) { + do_remove(std::move(variant_key), opts); } - void remove(VariantKey&& key, RemoveOpts opts) { - return remove(Composite{std::move(key)}, opts); + void remove(std::span variant_keys, RemoveOpts opts) { + return do_remove(variant_keys, opts); } - bool supports_prefix_matching() const { + [[nodiscard]] bool supports_prefix_matching() const { return do_supports_prefix_matching(); } - bool supports_atomic_writes() const { + [[nodiscard]] bool supports_atomic_writes() const { return do_supports_atomic_writes(); } @@ -178,29 +98,33 @@ class Storage { do_iterate_type_until_match(key_type, predicate_visitor, prefix); } - std::string key_path(const VariantKey& key) const { + [[nodiscard]] std::string key_path(const VariantKey& key) const { return do_key_path(key); } - bool is_path_valid(const std::string_view path) const { + [[nodiscard]] bool is_path_valid(std::string_view path) const { return do_is_path_valid(path); } [[nodiscard]] const LibraryPath &library_path() const { return lib_path_; } [[nodiscard]] OpenMode open_mode() const { return mode_; } - virtual std::string name() const = 0; + [[nodiscard]] virtual std::string name() const = 0; private: - virtual void do_write(Composite&& kvs) = 0; + virtual void do_write(KeySegmentPair&& key_seg) = 0; virtual void do_write_if_none(KeySegmentPair&& kv) = 0; - virtual void do_update(Composite&& kvs, UpdateOpts opts) = 0; + virtual void do_update(KeySegmentPair&& key_seg, UpdateOpts opts) = 0; + + virtual void do_read(VariantKey&& variant_key, const ReadVisitor& visitor, ReadKeyOpts opts) = 0; + + virtual KeySegmentPair do_read(VariantKey&& variant_key, ReadKeyOpts opts) = 0; - virtual void do_read(Composite&& ks, const ReadVisitor& visitor, ReadKeyOpts opts) = 0; + virtual void do_remove(VariantKey&& variant_key, RemoveOpts opts) = 0; - virtual void do_remove(Composite&& ks, RemoveOpts opts) = 0; + virtual void do_remove(std::span variant_keys, RemoveOpts opts) = 0; virtual bool do_key_exists(const VariantKey& key) = 0; @@ -214,9 +138,9 @@ class Storage { // the predicate. virtual bool do_iterate_type_until_match(KeyType key_type, const IterateTypePredicate& visitor, const std::string & prefix) = 0; - virtual std::string do_key_path(const VariantKey& key) const = 0; + [[nodiscard]] virtual std::string do_key_path(const VariantKey& key) const = 0; - virtual bool do_is_path_valid(const std::string_view) const { return true; } + [[nodiscard]] virtual bool do_is_path_valid(std::string_view) const { return true; } LibraryPath lib_path_; OpenMode mode_; diff --git a/cpp/arcticdb/storage/storage_exceptions.hpp b/cpp/arcticdb/storage/storage_exceptions.hpp index 49f9e38478..e56c5f2cce 100644 --- a/cpp/arcticdb/storage/storage_exceptions.hpp +++ b/cpp/arcticdb/storage/storage_exceptions.hpp @@ -11,15 +11,101 @@ #include #include +#include + namespace arcticdb::storage { +class DuplicateKeyException : public ArcticSpecificException { +public: + explicit DuplicateKeyException(const std::string& message) : + ArcticSpecificException(message) {} + + explicit DuplicateKeyException(VariantKey key) : + ArcticSpecificException(std::string(variant_key_view(key))), + key_(std::move(key)) {} + + [[nodiscard]] const VariantKey& key() const { + return key_; + } +private: + VariantKey key_; +}; + +class NoDataFoundException : public ArcticCategorizedException { +public: + explicit NoDataFoundException(VariantId key) : + ArcticCategorizedException(std::visit([](const auto& key) { + return fmt::format("{}", + key); + }, key)), + key_(key) { + } + + explicit NoDataFoundException(const std::string& msg) : + ArcticCategorizedException(msg) { + } + + explicit NoDataFoundException(const char *msg) : + ArcticCategorizedException(std::string(msg)) { + } + + [[nodiscard]] const VariantId& key() const { + util::check(static_cast(key_), "Key not found"); + return *key_; + } +private: + std::optional key_; +}; + +class KeyNotFoundException : public ArcticSpecificException { +public: + explicit KeyNotFoundException(const std::string& message) : + ArcticSpecificException(message) { + } + + explicit KeyNotFoundException(std::vector&& keys) : + ArcticSpecificException(fmt::format("Not found: {}", keys)), + keys_(std::make_shared>(std::move(keys))) { + } + + explicit KeyNotFoundException(std::vector&& keys, const std::string& err_output) : + ArcticSpecificException(err_output), + keys_(std::make_shared>(std::move(keys))) { + } + + explicit KeyNotFoundException(const VariantKey& single_key) : + KeyNotFoundException(std::vector{single_key}) { + } + + explicit KeyNotFoundException(const VariantKey& single_key, const std::string& err_output) : + KeyNotFoundException(std::vector{single_key}, err_output) { + } + + explicit KeyNotFoundException(boost::container::small_vector& keys) : + ArcticSpecificException(fmt::format("Not found: {}", keys)), + keys_(std::make_shared>(std::make_move_iterator(keys.begin()), std::make_move_iterator(keys.end()))) { + } + + explicit KeyNotFoundException(boost::container::small_vector& keys, const std::string& err_output) : + ArcticSpecificException(err_output), + keys_(std::make_shared>(std::make_move_iterator(keys.begin()), std::make_move_iterator(keys.end()))) { + } + + std::vector& keys() { + return *keys_; + } +private: + std::shared_ptr> keys_; + mutable std::string msg_; +}; + class LibraryPermissionException : public PermissionException { - public: - LibraryPermissionException(const LibraryPath &path, OpenMode mode, std::string_view operation) : +public: + LibraryPermissionException(const LibraryPath& path, OpenMode mode, std::string_view operation) : PermissionException(fmt::format("{} not permitted. lib={}, mode={}", operation, path, mode)), lib_path_(path), mode_(mode) {} - const LibraryPath &library_path() const { + const LibraryPath& library_path() const { return lib_path_; } @@ -27,7 +113,7 @@ class LibraryPermissionException : public PermissionException { return mode_; } - private: +private: LibraryPath lib_path_; OpenMode mode_; }; diff --git a/cpp/arcticdb/storage/storages.hpp b/cpp/arcticdb/storage/storages.hpp index 9b393cda4f..ae0dd21094 100644 --- a/cpp/arcticdb/storage/storages.hpp +++ b/cpp/arcticdb/storage/storages.hpp @@ -41,21 +41,21 @@ class Storages { storages_(std::move(storages)), mode_(mode) { } - void write(Composite&& kvs) { + void write(KeySegmentPair&& key_seg) { ARCTICDB_SAMPLE(StoragesWrite, 0) - primary().write(std::move(kvs)); + primary().write(std::move(key_seg)); } void write_if_none(KeySegmentPair&& kv) { primary().write_if_none(std::move(kv)); } - void update(Composite&& kvs, storage::UpdateOpts opts) { + void update(KeySegmentPair&& key_seg, storage::UpdateOpts opts) { ARCTICDB_SAMPLE(StoragesUpdate, 0) - primary().update(std::move(kvs), opts); + primary().update(std::move(key_seg), opts); } - bool supports_prefix_matching() const { + [[nodiscard]] bool supports_prefix_matching() const { return primary().supports_prefix_matching(); } @@ -75,30 +75,90 @@ class Storages { return primary().key_exists(key); } - bool is_path_valid(const std::string_view path) const { + [[nodiscard]] bool is_path_valid(const std::string_view path) const { return primary().is_path_valid(path); } - auto read(Composite&& ks, const ReadVisitor& visitor, ReadKeyOpts opts, bool primary_only=true) { - ARCTICDB_RUNTIME_SAMPLE(StoragesRead, 0) - if(primary_only) - return primary().read(std::move(ks), visitor, opts); - - if(auto rg = ks.as_range(); !std::all_of(std::begin(rg), std::end(rg), [] (const auto& vk) { - return variant_key_type(vk) == KeyType::TABLE_DATA; - })) { - return primary().read(std::move(ks), visitor, opts); + void read_sync_fallthrough(const VariantKey& variant_key, const ReadVisitor& visitor, ReadKeyOpts opts) { + for (const auto &storage : storages_) { + try { + return storage->read(VariantKey{variant_key}, visitor, opts); + } catch (typename storage::KeyNotFoundException&) { + ARCTICDB_DEBUG(log::version(), "Keys not found in storage, continuing to next storage"); + } } + throw storage::KeyNotFoundException(variant_key); + } + KeySegmentPair read_sync_fallthrough(const VariantKey& variant_key) { for(const auto& storage : storages_) { try { - return storage->read(std::move(ks), visitor, opts); - } catch (typename storage::KeyNotFoundException& ex) { + return storage->read(VariantKey{variant_key}, ReadKeyOpts{}); + } catch (typename storage::KeyNotFoundException&) { ARCTICDB_DEBUG(log::version(), "Keys not found in storage, continuing to next storage"); - ks = std::move(ex.keys()); } } - throw storage::KeyNotFoundException(std::move(ks)); + throw storage::KeyNotFoundException(variant_key); + } + + void read_sync(const VariantKey& variant_key, const ReadVisitor& visitor, ReadKeyOpts opts, bool primary_only=true) { + ARCTICDB_RUNTIME_SAMPLE(StoragesRead, 0) + if(primary_only || variant_key_type(variant_key) != KeyType::TABLE_DATA) + return primary().read(VariantKey{variant_key}, visitor, opts); + + read_sync_fallthrough(variant_key, visitor, opts); + } + + KeySegmentPair read_sync(const VariantKey& variant_key, ReadKeyOpts opts, bool primary_only=true) { + ARCTICDB_RUNTIME_SAMPLE(StoragesRead, 0) + if(primary_only || variant_key_type(variant_key) != KeyType::TABLE_DATA) + return primary().read(VariantKey{variant_key}, opts); + + return read_sync_fallthrough(variant_key); + } + + static folly::Future async_read(Storage& storage, VariantKey&& variant_key, const ReadVisitor& visitor, ReadKeyOpts opts) { + if(storage.has_async_api()) { + return storage.async_api()->async_read(std::move(variant_key), visitor, opts); + } else { + storage.read(std::move(variant_key), visitor, opts); + return folly::makeFuture(); + } + } + + static folly::Future async_read(Storage& storage, VariantKey&& variant_key, ReadKeyOpts opts) { + if(storage.has_async_api()) { + return storage.async_api()->async_read(std::move(variant_key), opts); + } else { + auto key_seg = storage.read(std::move(variant_key), opts); + return folly::makeFuture(std::move(key_seg)); + } + } + + folly::Future read(VariantKey&& variant_key, const ReadVisitor& visitor, ReadKeyOpts opts, bool primary_only=true) { + ARCTICDB_RUNTIME_SAMPLE(StoragesRead, 0) + if(primary_only || variant_key_type(variant_key) != KeyType::TABLE_DATA) + return async_read(primary(), std::move(variant_key), visitor, opts); + + // Not supporting async fall-through at the moment as we would need to ensure that + // visitation was idempotent. Could be achieved with a mutex/call once wrapper around + // the visitor for simultaneous async reads, or with a window of size 1 and cancellation + // token. + read_sync_fallthrough(variant_key, visitor, opts); + return folly::makeFuture(); + } + + folly::Future read(VariantKey&& variant_key, ReadKeyOpts opts, bool primary_only=true) { + ARCTICDB_RUNTIME_SAMPLE(StoragesRead, 0) + if(primary_only || variant_key_type(variant_key) != KeyType::TABLE_DATA) + return async_read(primary(), std::move(variant_key), opts); + + // Not supporting async fall-through at the moment as we would need to ensure that + // visitation was idempotent. Could be achieved with a mutex/call once wrapper around + // the visitor for simultaneous async reads, or with a window of size 1 and cancellation + // token. + auto res = read_sync_fallthrough(variant_key); + return folly::makeFuture(std::move(res)); } void iterate_type(KeyType key_type, const IterateTypeVisitor& visitor, const std::string &prefix=std::string{}, bool primary_only=true) { @@ -124,12 +184,16 @@ class Storages { } /** Calls Storage::do_key_path on the primary storage. Remember to check the open mode. */ - std::string key_path(const VariantKey& key) const { + [[nodiscard]] std::string key_path(const VariantKey& key) const { return primary().key_path(key); } - void remove(Composite&& ks, storage::RemoveOpts opts) { - primary().remove(std::move(ks), opts); + void remove(VariantKey&& variant_key, storage::RemoveOpts opts) { + primary().remove(std::move(variant_key), opts); + } + + void remove(std::span variant_keys, storage::RemoveOpts opts) { + primary().remove(variant_keys, opts); } [[nodiscard]] OpenMode open_mode() const { return mode_; } @@ -156,14 +220,14 @@ class Storages { source.iterate_type(key_type, visitor); } - std::optional> get_single_file_storage() const { + [[nodiscard]] std::optional> get_single_file_storage() const { if (dynamic_cast(storages_[0].get()) != nullptr) { return std::dynamic_pointer_cast(storages_[0]); } else { return std::nullopt; } } - std::string name() const { + [[nodiscard]] std::string name() const { return primary().name(); } @@ -173,7 +237,7 @@ class Storages { return *storages_[0]; } - const Storage& primary() const { + [[nodiscard]] const Storage& primary() const { util::check(!storages_.empty(), "No storages configured"); return *storages_[0]; } diff --git a/cpp/arcticdb/storage/test/common.hpp b/cpp/arcticdb/storage/test/common.hpp index 7a9cdeebed..3668bfbf1b 100644 --- a/cpp/arcticdb/storage/test/common.hpp +++ b/cpp/arcticdb/storage/test/common.hpp @@ -14,7 +14,7 @@ namespace arcticdb::storage { -inline VariantKey get_test_key(std::string name, entity::KeyType key_type = entity::KeyType::TABLE_DATA) { +inline VariantKey get_test_key(const std::string& name, entity::KeyType key_type = entity::KeyType::TABLE_DATA) { auto builder = arcticdb::atom_key_builder(); return builder.build(name, key_type); } @@ -25,22 +25,22 @@ inline Segment get_test_segment() { return encode_dispatch(std::move(segment_in_memory), codec_opts, arcticdb::EncodingVersion::V2); } -inline void write_in_store(Storage &store, std::string symbol, entity::KeyType key_type = entity::KeyType::TABLE_DATA) { +inline void write_in_store(Storage &store, const std::string& symbol, entity::KeyType key_type = entity::KeyType::TABLE_DATA) { auto variant_key = get_test_key(symbol, key_type); store.write(KeySegmentPair(std::move(variant_key), get_test_segment())); } -inline void update_in_store(Storage &store, std::string symbol, entity::KeyType key_type = entity::KeyType::TABLE_DATA) { +inline void update_in_store(Storage &store, const std::string& symbol, entity::KeyType key_type = entity::KeyType::TABLE_DATA) { auto variant_key = get_test_key(symbol, key_type); store.update(KeySegmentPair(std::move(variant_key), get_test_segment()), arcticdb::storage::UpdateOpts{}); } -inline bool exists_in_store(Storage &store, std::string symbol, entity::KeyType key_type = entity::KeyType::TABLE_DATA) { +inline bool exists_in_store(Storage &store, const std::string& symbol, entity::KeyType key_type = entity::KeyType::TABLE_DATA) { auto variant_key = get_test_key(symbol, key_type); return store.key_exists(variant_key); } -inline std::string read_in_store(Storage &store, std::string symbol, entity::KeyType key_type = entity::KeyType::TABLE_DATA) { +inline std::string read_in_store(Storage &store, const std::string& symbol, entity::KeyType key_type = entity::KeyType::TABLE_DATA) { auto variant_key = get_test_key(symbol, key_type); auto opts = ReadKeyOpts{}; auto result = store.read(std::move(variant_key), opts); @@ -53,7 +53,7 @@ inline void remove_in_store(Storage &store, const std::vector& symb to_remove.emplace_back(get_test_key(symbol, key_type)); } auto opts = RemoveOpts(); - store.remove(Composite(std::move(to_remove)), opts); + store.remove(std::span(to_remove), opts); } inline std::set list_in_store(Storage &store, entity::KeyType key_type = entity::KeyType::TABLE_DATA) { diff --git a/cpp/arcticdb/storage/test/in_memory_store.hpp b/cpp/arcticdb/storage/test/in_memory_store.hpp index da78be3c28..99c6707de2 100644 --- a/cpp/arcticdb/storage/test/in_memory_store.hpp +++ b/cpp/arcticdb/storage/test/in_memory_store.hpp @@ -203,7 +203,7 @@ namespace arcticdb { [&] (const RefKey& ref_key) { auto it = seg_by_ref_key_.find(ref_key); if (it == seg_by_ref_key_.end()) - throw storage::KeyNotFoundException(Composite(ref_key)); + throw storage::KeyNotFoundException(ref_key); ARCTICDB_DEBUG(log::storage(), "Mock store returning ref key {}", ref_key); std::pair res = {it->first, it->second->clone()}; return res; @@ -211,7 +211,7 @@ namespace arcticdb { [&] (const AtomKey& atom_key) { auto it = seg_by_atom_key_.find(atom_key); if (it == seg_by_atom_key_.end()) - throw storage::KeyNotFoundException(Composite(atom_key)); + throw storage::KeyNotFoundException(atom_key); ARCTICDB_DEBUG(log::storage(), "Mock store returning atom key {}", atom_key); std::pair res = {it->first, it->second->clone()}; //seg_by_atom_key_.erase(it); @@ -235,14 +235,14 @@ namespace arcticdb { [&] (const RefKey& ref_key) { auto it = seg_by_ref_key_.find(ref_key); if (it == seg_by_ref_key_.end()) - throw storage::KeyNotFoundException(Composite(ref_key)); + throw storage::KeyNotFoundException(ref_key); ARCTICDB_DEBUG(log::storage(), "Mock store returning compressed ref key {}", ref_key); return it->second->clone(); }, [&] (const AtomKey& atom_key) { auto it = seg_by_atom_key_.find(atom_key); if (it == seg_by_atom_key_.end()) - throw storage::KeyNotFoundException(Composite(atom_key)); + throw storage::KeyNotFoundException(atom_key); ARCTICDB_DEBUG(log::storage(), "Mock store returning compressed atom key {}", atom_key); return it->second->clone(); }); @@ -267,11 +267,11 @@ namespace arcticdb { StorageFailureSimulator::instance()->go(FailureType::DELETE); std::lock_guard lock{mutex_}; size_t removed = util::variant_match(key, - [&](const AtomKey &ak) { return seg_by_atom_key_.erase(ak); }, - [&](const RefKey &rk) { return seg_by_ref_key_.erase(rk); }); + [&](const AtomKey &atom_key) { return seg_by_atom_key_.erase(atom_key); }, + [&](const RefKey &ref_key) { return seg_by_ref_key_.erase(ref_key); }); ARCTICDB_DEBUG(log::storage(), "Mock store removed {} {}", removed, key); if (removed == 0 && !opts.ignores_missing_key_) { - throw storage::KeyNotFoundException(Composite(VariantKey(key))); + throw storage::KeyNotFoundException(VariantKey(key)); } return {}; } @@ -405,20 +405,20 @@ namespace arcticdb { read_metadata(const entity::VariantKey &key, storage::ReadKeyOpts) override { return util::variant_match(key, - [&](const AtomKey &ak) { - auto it = seg_by_atom_key_.find(ak); - // util::check_rte(it != seg_by_atom_key_.end(), "atom key {} not found in remove", ak); + [&](const AtomKey &atom_key) { + auto it = seg_by_atom_key_.find(atom_key); + // util::check_rte(it != seg_by_atom_key_.end(), "atom key {} not found in remove", atom_key); if (it == seg_by_atom_key_.end()) - throw storage::KeyNotFoundException(Composite(ak)); - ARCTICDB_DEBUG(log::storage(), "Mock store removing data for atom key {}", ak); + throw storage::KeyNotFoundException(atom_key); + ARCTICDB_DEBUG(log::storage(), "Mock store removing data for atom key {}", atom_key); return std::make_pair(std::make_optional(key), std::make_optional(*it->second->metadata())); }, - [&](const RefKey &rk) { - auto it = seg_by_ref_key_.find(rk); - // util::check_rte(it != seg_by_ref_key_.end(), "ref key {} not found in remove", rk); + [&](const RefKey &ref_key) { + auto it = seg_by_ref_key_.find(ref_key); + // util::check_rte(it != seg_by_ref_key_.end(), "ref key {} not found in remove", ref_key); if (it == seg_by_ref_key_.end()) - throw storage::KeyNotFoundException(Composite(rk)); - ARCTICDB_DEBUG(log::storage(), "Mock store removing data for ref key {}", rk); + throw storage::KeyNotFoundException(ref_key); + ARCTICDB_DEBUG(log::storage(), "Mock store removing data for ref key {}", ref_key); return std::make_pair(std::make_optional(key), std::make_optional(*it->second->metadata())); }); } @@ -427,20 +427,20 @@ namespace arcticdb { read_metadata_and_descriptor( const entity::VariantKey& key, storage::ReadKeyOpts) override { auto components = util::variant_match(key, - [&](const AtomKey &ak) { - auto it = seg_by_atom_key_.find(ak); - // util::check_rte(it != seg_by_atom_key_.end(), "atom key {} not found in remove", ak); + [&](const AtomKey &atom_key) { + auto it = seg_by_atom_key_.find(atom_key); + // util::check_rte(it != seg_by_atom_key_.end(), "atom key {} not found in remove", atom_key); if (it == seg_by_atom_key_.end()) - throw storage::KeyNotFoundException(Composite(ak)); - ARCTICDB_DEBUG(log::storage(), "Mock store removing data for atom key {}", ak); + throw storage::KeyNotFoundException(atom_key); + ARCTICDB_DEBUG(log::storage(), "Mock store removing data for atom key {}", atom_key); return std::make_tuple(key, std::make_optional(*it->second->metadata()), it->second->descriptor()); }, - [&](const RefKey &rk) { - auto it = seg_by_ref_key_.find(rk); - // util::check_rte(it != seg_by_ref_key_.end(), "ref key {} not found in remove", rk); + [&](const RefKey &ref_key) { + auto it = seg_by_ref_key_.find(ref_key); + // util::check_rte(it != seg_by_ref_key_.end(), "ref key {} not found in remove", ref_key); if (it == seg_by_ref_key_.end()) - throw storage::KeyNotFoundException(Composite(rk)); - ARCTICDB_DEBUG(log::storage(), "Mock store removing data for ref key {}", rk); + throw storage::KeyNotFoundException(ref_key); + ARCTICDB_DEBUG(log::storage(), "Mock store removing data for ref key {}", ref_key); return std::make_tuple(key, std::make_optional(*it->second->metadata()), it->second->descriptor()); }); return folly::makeFuture(std::move(components)); @@ -449,18 +449,18 @@ namespace arcticdb { folly::Future, arcticdb::TimeseriesDescriptor>> read_timeseries_descriptor(const entity::VariantKey& key, storage::ReadKeyOpts /*opts*/) override { - return util::variant_match(key, [&](const AtomKey &ak) { - auto it = seg_by_atom_key_.find(ak); + return util::variant_match(key, [&](const AtomKey &atom_key) { + auto it = seg_by_atom_key_.find(atom_key); if (it == seg_by_atom_key_.end()) - throw storage::KeyNotFoundException(Composite(ak)); - ARCTICDB_DEBUG(log::storage(), "Mock store removing data for atom key {}", ak); + throw storage::KeyNotFoundException(atom_key); + ARCTICDB_DEBUG(log::storage(), "Mock store removing data for atom key {}", atom_key); return std::make_pair(key, it->second->index_descriptor()); }, - [&](const RefKey &rk) { - auto it = seg_by_ref_key_.find(rk); + [&](const RefKey &ref_key) { + auto it = seg_by_ref_key_.find(ref_key); if (it == seg_by_ref_key_.end()) - throw storage::KeyNotFoundException(Composite(rk)); - ARCTICDB_DEBUG(log::storage(), "Mock store removing data for ref key {}", rk); + throw storage::KeyNotFoundException(ref_key); + ARCTICDB_DEBUG(log::storage(), "Mock store removing data for ref key {}", ref_key); return std::make_pair(key, it->second->index_descriptor()); }); } @@ -490,7 +490,6 @@ namespace arcticdb { seg_by_ref_key_[key] = std::make_unique(std::move(seg)); } - protected: std::recursive_mutex mutex_; // Allow iterate_type() to be re-entrant std::unordered_map> seg_by_atom_key_; diff --git a/cpp/arcticdb/storage/test/test_azure_storage.cpp b/cpp/arcticdb/storage/test/test_azure_storage.cpp index 92e56ee2a6..bda8b51f7b 100644 --- a/cpp/arcticdb/storage/test/test_azure_storage.cpp +++ b/cpp/arcticdb/storage/test/test_azure_storage.cpp @@ -10,7 +10,7 @@ #include #include -#include +#include #include #include diff --git a/cpp/arcticdb/storage/test/test_s3_storage.cpp b/cpp/arcticdb/storage/test/test_s3_storage.cpp index fb7177c33f..2e9665e8bb 100644 --- a/cpp/arcticdb/storage/test/test_s3_storage.cpp +++ b/cpp/arcticdb/storage/test/test_s3_storage.cpp @@ -10,7 +10,7 @@ #include #include #include -#include +#include #include #include #include @@ -31,7 +31,7 @@ struct EnvFunctionShim : ::testing::Test { #endif } - virtual ~EnvFunctionShim() { + ~EnvFunctionShim() override { for (const char* envname : env_vars_to_unset) { #if (WIN32) _putenv_s(envname, ""); @@ -285,7 +285,7 @@ TEST_P(S3AndNfsStorageFixture, test_read_missing_key_in_exception){ read_in_store(store, "snap-not-present", KeyType::SNAPSHOT_REF); FAIL(); } catch (KeyNotFoundException& e) { - auto keys = e.keys().as_range(); + auto keys = e.keys(); ASSERT_EQ(keys.size(), 1); const auto& key = keys.at(0); ASSERT_EQ(variant_key_id(key), StreamId{"snap-not-present"}); diff --git a/cpp/arcticdb/storage/test/test_storage_exceptions.cpp b/cpp/arcticdb/storage/test/test_storage_exceptions.cpp index 20a888538f..bccec61344 100644 --- a/cpp/arcticdb/storage/test/test_storage_exceptions.cpp +++ b/cpp/arcticdb/storage/test/test_storage_exceptions.cpp @@ -10,14 +10,14 @@ #include #include #include -#include +#include #include #include -#include +#include #include -#include +#include #include -#include +#include #include #include diff --git a/cpp/arcticdb/stream/append_map.cpp b/cpp/arcticdb/stream/append_map.cpp index c61267b122..aea3f261f7 100644 --- a/cpp/arcticdb/stream/append_map.cpp +++ b/cpp/arcticdb/stream/append_map.cpp @@ -378,7 +378,7 @@ std::pair> get_descriptor_a const std::shared_ptr& store, const AtomKey& k, bool load_data, - storage::ReadKeyOpts opts = storage::ReadKeyOpts{}) { + storage::ReadKeyOpts opts) { if(load_data) { auto seg = store->read_sync(k, opts).second; return std::make_pair(seg.index_descriptor(), std::make_optional(seg)); diff --git a/cpp/arcticdb/stream/stream_source.hpp b/cpp/arcticdb/stream/stream_source.hpp index 5745f82686..6279e49db9 100644 --- a/cpp/arcticdb/stream/stream_source.hpp +++ b/cpp/arcticdb/stream/stream_source.hpp @@ -28,7 +28,8 @@ struct StreamSource { virtual std::pair read_sync( const entity::VariantKey &key, - storage::ReadKeyOpts opts = storage::ReadKeyOpts{}) = 0; + storage::ReadKeyOpts opts = storage::ReadKeyOpts{}) + = 0; virtual folly::Future read_compressed( const entity::VariantKey &key, @@ -36,7 +37,7 @@ struct StreamSource { virtual storage::KeySegmentPair read_compressed_sync( const entity::VariantKey& key, - storage::ReadKeyOpts opts + storage::ReadKeyOpts opts = storage::ReadKeyOpts{} ) = 0; virtual void iterate_type( diff --git a/cpp/arcticdb/stream/stream_utils.hpp b/cpp/arcticdb/stream/stream_utils.hpp index 2dacb9bf3f..a4cda09fac 100644 --- a/cpp/arcticdb/stream/stream_utils.hpp +++ b/cpp/arcticdb/stream/stream_utils.hpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include #include diff --git a/cpp/arcticdb/stream/test/stream_test_common.hpp b/cpp/arcticdb/stream/test/stream_test_common.hpp index 764e5f633b..9606da1389 100644 --- a/cpp/arcticdb/stream/test/stream_test_common.hpp +++ b/cpp/arcticdb/stream/test/stream_test_common.hpp @@ -26,8 +26,6 @@ #include #include -namespace fg = folly::gen; - namespace arcticdb { template diff --git a/cpp/arcticdb/toolbox/library_tool.cpp b/cpp/arcticdb/toolbox/library_tool.cpp index ecd0dcc57b..65a12bb2af 100644 --- a/cpp/arcticdb/toolbox/library_tool.cpp +++ b/cpp/arcticdb/toolbox/library_tool.cpp @@ -49,7 +49,7 @@ ReadResult LibraryTool::read(const VariantKey& key) { } Segment LibraryTool::read_to_segment(const VariantKey& key) { - auto kv = store()->read_compressed_sync(key, storage::ReadKeyOpts{}); + auto kv = store()->read_compressed_sync(key); util::check(kv.has_segment(), "Failed to read key: {}", key); kv.segment().force_own_buffer(); return std::move(kv.segment()); diff --git a/cpp/arcticdb/util/allocation_tracing.cpp b/cpp/arcticdb/util/allocation_tracing.cpp new file mode 100644 index 0000000000..3bf4905022 --- /dev/null +++ b/cpp/arcticdb/util/allocation_tracing.cpp @@ -0,0 +1,58 @@ +#include + +#ifdef ARCTICDB_COUNT_ALLOCATIONS + +namespace arcticdb { + +bool AllocationTracker::started_ = false; +thread_local bool top_level_ = true; + +std::shared_ptr AllocationTracker::instance() { + std::call_once(AllocationTracker::init_flag_, &AllocationTracker::init); + return AllocationTracker::instance_; +} + +void AllocationTracker::destroy_instance() { + if (instance_) + instance_->print(); + + instance_.reset(); +} + +void AllocationTracker::trace() { + if(top_level_) { + top_level_ = false; + auto trace = unwind_stack(num_levels_); + { + std::lock_guard lock{mutex_}; + ++data_[trace]; + } + top_level_ = true; + } +} + +void AllocationTracker::init() { + instance_ = std::make_shared(); +} + +std::shared_ptr AllocationTracker::instance_; +std::once_flag AllocationTracker::init_flag_; + +} // namespace arcticdb + +void* operator new(std::size_t sz){ + void* ptr = std::malloc(sz); + if(arcticdb::AllocationTracker::started()) + arcticdb::AllocationTracker::instance()->trace(); + return ptr; +} + +void operator delete(void* ptr) noexcept{ + std::free(ptr); +} + +void operator delete(void* ptr, std::size_t) noexcept{ + std::free(ptr); +} + +#endif \ No newline at end of file diff --git a/cpp/arcticdb/util/allocation_tracing.hpp b/cpp/arcticdb/util/allocation_tracing.hpp new file mode 100644 index 0000000000..90c6e7b4b7 --- /dev/null +++ b/cpp/arcticdb/util/allocation_tracing.hpp @@ -0,0 +1,68 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#include +#include + +#ifdef ARCTICDB_COUNT_ALLOCATIONS + +namespace arcticdb { + +class AllocationTracker { + static std::shared_ptr instance_; + static std::once_flag init_flag_; + static bool started_; + + static void init(); + + std::unordered_map data_; + std::recursive_mutex mutex_; + constexpr static int num_levels_ = 3; + +public: + static std::shared_ptr instance(); + static void destroy_instance(); + + AllocationTracker() = default; + ~AllocationTracker() { + print(); + } + + ARCTICDB_NO_MOVE_OR_COPY(AllocationTracker) + + void trace(); + + static void start() { + started_ = true; + } + + static bool started() { + return started_; + } + + void print() { + for(const auto& [key, value] : data_) + if(value > 100) + std::cout << value << ": " << key << std::endl; + } +}; + +} + +void* operator new(std::size_t sz); + +void operator delete(void* ptr) noexcept; + +void operator delete(void* ptr, std::size_t) noexcept; + +#endif \ No newline at end of file diff --git a/cpp/arcticdb/util/global_lifetimes.cpp b/cpp/arcticdb/util/global_lifetimes.cpp index fbc5c8092a..47302c72a6 100644 --- a/cpp/arcticdb/util/global_lifetimes.cpp +++ b/cpp/arcticdb/util/global_lifetimes.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #if defined(_MSC_VER) && defined(_DEBUG) #include @@ -23,6 +24,9 @@ namespace arcticdb { ModuleData::~ModuleData() { +#ifdef ARCTICDB_COUNT_ALLOCATIONS + AllocationTracker::destroy_instance(); +#endif BufferPool::destroy_instance(); TracingData::destroy_instance(); Allocator::destroy_instance(); diff --git a/cpp/arcticdb/util/trace.cpp b/cpp/arcticdb/util/trace.cpp index 5b1c6991a0..25adf49b01 100644 --- a/cpp/arcticdb/util/trace.cpp +++ b/cpp/arcticdb/util/trace.cpp @@ -6,21 +6,62 @@ */ #include +#include #ifndef _WIN32 #include #endif +#ifdef ARCTICDB_COUNT_ALLOCATIONS +#include +#endif + namespace arcticdb { - std::string get_type_name(const std::type_info& ti){ +std::string get_type_name(const std::type_info& ti) { #ifndef _WIN32 - char* demangled = abi::__cxa_demangle(ti.name(), nullptr, nullptr, nullptr); - std::string ret = demangled; - free(demangled); - return ret; + char *demangled = abi::__cxa_demangle(ti.name(), nullptr, nullptr, nullptr); + std::string ret = demangled; + free(demangled); + return ret; #else - return ti.name(); + return ti.name(); #endif +} + +#ifdef ARCTICDB_COUNT_ALLOCATIONS + +std::string get_trace() { + return cpptrace::generate_trace(5, 10).to_string(); +} + +std::string_view removePrefix(std::string_view input, std::string_view prefix) { + auto pos = input.rfind(prefix); + if (pos != std::string_view::npos) { + return {input.data() + prefix.size(), input.size() - prefix.size()}; } + return input; +} + +std::string unwind_stack(int) { + return get_trace(); } + +std::string unwind_stack(int max_depth) { + void *buffer[max_depth]; + int num_frames = backtrace(buffer, max_depth); + char **symbols = backtrace_symbols(buffer, num_frames); + + thread_local std::ostringstream oss; + for (int i = 0; i < num_frames; ++i) { + auto filtered = removePrefix(symbols[i], "/opt/arcticdb/arcticdb_link/python/arcticdb_ext.cpython-38-x86_64-linux-gnu.so"); + oss << filtered << " "; + } + + free(symbols); + return oss.str(); +} + +#endif + +} // namespace arcticdb \ No newline at end of file diff --git a/cpp/arcticdb/util/trace.hpp b/cpp/arcticdb/util/trace.hpp index 146c9c9c91..88bf939713 100644 --- a/cpp/arcticdb/util/trace.hpp +++ b/cpp/arcticdb/util/trace.hpp @@ -14,4 +14,7 @@ namespace arcticdb { std::string get_type_name(const std::type_info& type_info); -} +#ifdef ARCTICDB_COUNT_ALLOCATIONS +std::string unwind_stack(int max_depth); +#endif +} // namespace arcticdb diff --git a/cpp/arcticdb/version/local_versioned_engine.cpp b/cpp/arcticdb/version/local_versioned_engine.cpp index 3c35ad92ff..99c1a245af 100644 --- a/cpp/arcticdb/version/local_versioned_engine.cpp +++ b/cpp/arcticdb/version/local_versioned_engine.cpp @@ -20,6 +20,7 @@ #include #include #include +#include namespace arcticdb::version_store { @@ -54,6 +55,10 @@ void LocalVersionedEngine::initialize(const std::shared_ptr& l async::TaskScheduler::reattach_instance(); } (void)async::TaskScheduler::instance(); +#ifdef ARCTICDB_COUNT_ALLOCATIONS + (void)AllocationTracker::instance(); + AllocationTracker::start(); +#endif } template LocalVersionedEngine::LocalVersionedEngine(const std::shared_ptr& library, const util::SysClock&); @@ -1079,9 +1084,15 @@ std::vector> LocalVersionedEngine::ba "ReadOptions::batch_throw_on_error_ should always be set here"); auto opt_index_key_futs = batch_get_versions_async(store(), version_map(), stream_ids, version_queries); std::vector> read_versions_futs; - for (auto&& [idx, opt_index_key_fut] : folly::enumerate(opt_index_key_futs)) { + + const auto max_batch_size = ConfigsMap::instance()->get_int("BatchRead.MaxConcurrency", 50); + ARCTICDB_RUNTIME_DEBUG(log::inmem(), "Running batch read with a maximum concurrency of {}", max_batch_size); + std::vector> all_results; + all_results.reserve(opt_index_key_futs.size()); + size_t batch_count = 0UL; + for (auto idx = 0UL; idx < opt_index_key_futs.size(); ++idx) { read_versions_futs.emplace_back( - std::move(opt_index_key_fut).thenValue([store = store(), + std::move(opt_index_key_futs[idx]).thenValue([store = store(), idx, &stream_ids, &version_queries, @@ -1103,11 +1114,22 @@ std::vector> LocalVersionedEngine::ba return read_frame_for_version(store, version_info, read_query, read_options, handler_data); }) ); + if(++batch_count == static_cast(max_batch_size)) { + auto read_versions = folly::collectAll(read_versions_futs).get(); + all_results.insert(all_results.end(), std::make_move_iterator(read_versions.begin()), std::make_move_iterator(read_versions.end())); + read_versions_futs.clear(); + batch_count = 0UL; + } } - auto read_versions = folly::collectAll(read_versions_futs).get(); + + if(!read_versions_futs.empty()) { + auto read_versions = folly::collectAll(read_versions_futs).get(); + all_results.insert(all_results.end(), std::make_move_iterator(read_versions.begin()), std::make_move_iterator(read_versions.end())); + } + std::vector> read_versions_or_errors; - read_versions_or_errors.reserve(read_versions.size()); - for (auto&& [idx, read_version]: folly::enumerate(read_versions)) { + read_versions_or_errors.reserve(all_results.size()); + for (auto&& [idx, read_version]: folly::enumerate(all_results)) { if (read_version.hasValue()) { read_versions_or_errors.emplace_back(std::move(read_version.value())); } else { diff --git a/cpp/arcticdb/version/snapshot.cpp b/cpp/arcticdb/version/snapshot.cpp index 50359d0f03..4a51dcf93f 100644 --- a/cpp/arcticdb/version/snapshot.cpp +++ b/cpp/arcticdb/version/snapshot.cpp @@ -70,8 +70,7 @@ void tombstone_snapshot( void tombstone_snapshot( const std::shared_ptr& store, storage::KeySegmentPair&& key_segment_pair, - bool log_changes - ) { + bool log_changes) { store->remove_key(key_segment_pair.ref_key()).get(); // Make the snapshot "disappear" to normal APIs if (log_changes) { log_delete_snapshot(store, key_segment_pair.ref_key().id()); @@ -106,10 +105,9 @@ void iterate_snapshots(const std::shared_ptr& store, folly::Function& store, folly::Function row_id_for_stream_in_snapshot_segment( - SegmentInMemory &seg, - bool using_ref_key, - const StreamId& stream_id) { + SegmentInMemory &seg, + bool using_ref_key, + const StreamId& stream_id) { if (using_ref_key) { // With ref keys we are sure the snapshot segment has the index atom keys sorted by stream_id. auto lb = std::lower_bound(std::begin(seg), std::end(seg), stream_id, diff --git a/cpp/arcticdb/version/symbol_list.cpp b/cpp/arcticdb/version/symbol_list.cpp index 42ebca31a4..3afbc2818f 100644 --- a/cpp/arcticdb/version/symbol_list.cpp +++ b/cpp/arcticdb/version/symbol_list.cpp @@ -147,10 +147,10 @@ MaybeCompaction last_compaction(const std::vector& keys) { // provided in SegmentInMemory, because the symbol list structure is the only place where columns can have more entries // than the segment has rows. Hence, we need to bypass the checks inside SegmentInMemory's function and directly call the // Column's string_at and scalar_at. -std::string string_at(const SegmentInMemory& seg, position_t row, position_t col){ +std::string_view string_at(const SegmentInMemory& seg, position_t row, position_t col){ auto offset = seg.column(col).scalar_at(row); util::check(offset.has_value(), "Symbol list trying to call string_at for missing row {}, column {}", row, col); - return std::string(seg.string_pool_ptr()->get_view(offset.value())); + return seg.string_pool_ptr()->get_view(offset.value()); } template @@ -172,7 +172,7 @@ StreamId stream_id_from_segment( } else { auto sym = string_at(seg, row_id, column); ARCTICDB_DEBUG(log::symbol(), "Reading string symbol '{}'", sym); - return {std::move(sym)}; + return StringId{sym}; } } diff --git a/cpp/arcticdb/version/test/test_version_common.hpp b/cpp/arcticdb/version/test/test_version_common.hpp index 1b6bc8ba3b..7c08e19293 100644 --- a/cpp/arcticdb/version/test/test_version_common.hpp +++ b/cpp/arcticdb/version/test/test_version_common.hpp @@ -12,7 +12,6 @@ using namespace arcticdb::storage; using namespace arcticdb::stream; using namespace arcticdb; -namespace fg = folly::gen; const uint64_t NumVersions = 10; const uint64_t NumValues = 10; diff --git a/cpp/arcticdb/version/version_core.cpp b/cpp/arcticdb/version/version_core.cpp index 56113f9851..99e69abe3b 100644 --- a/cpp/arcticdb/version/version_core.cpp +++ b/cpp/arcticdb/version/version_core.cpp @@ -69,7 +69,7 @@ VersionedItem write_dataframe_impl( ARCTICDB_SUBSAMPLE_DEFAULT(WaitForWriteCompletion) ARCTICDB_DEBUG(log::version(), "write_dataframe_impl stream_id: {} , version_id: {}, {} rows", frame->desc.id(), version_id, frame->num_rows); auto atom_key_fut = async_write_dataframe_impl(store, version_id, frame, options, de_dup_map, sparsify_floats, validate_index); - return VersionedItem(std::move(atom_key_fut).get()); + return {std::move(atom_key_fut).get()}; } folly::Future async_write_dataframe_impl( @@ -174,22 +174,6 @@ bool is_after(const IndexRange& a, const IndexRange& b) { return a.end_ > b.end_; } -template - void ensure_keys_line_up(const KeyContainer& slice_and_keys) { - std::optional start; - std::optional end; - SliceAndKey prev{}; - for(const auto& sk : slice_and_keys) { - util::check(!start || sk.slice_.row_range.first == end.value(), - "Can't update as there is a sorting mismatch at key {} relative to previous key {} - expected index {} got {}", - sk, prev, end.value(), start.value_or(0)); - - start = sk.slice_.row_range.first; - end = sk.slice_.row_range.second; - prev = sk; - } -} - std::vector filter_existing_slices(std::vector>&& maybe_slices) { std::vector result; for (auto& maybe_slice : maybe_slices) { @@ -506,17 +490,6 @@ VersionedItem update_impl( return versioned_item; } -void set_row_id_for_empty_columns_set( - const ReadQuery& read_query, - const PipelineContext& pipeline_context, - SegmentInMemory& frame, - size_t row_id) { - if (read_query.columns && read_query.columns->empty() && - pipeline_context.descriptor().index().type() == IndexDescriptor::Type::ROWCOUNT) { - frame.set_row_id(row_id); - } -} - folly::Future read_multi_key( const std::shared_ptr& store, const SegmentInMemory& index_key_seg, @@ -539,8 +512,24 @@ folly::Future read_multi_key( }); } -size_t generate_scheduling_iterations(const std::vector>& clauses) { - size_t res{1}; +void add_slice_to_component_manager( + EntityId entity_id, + pipelines::SegmentAndSlice& segment_and_slice, + std::shared_ptr component_manager, + size_t fetch_count) { + ARCTICDB_DEBUG(log::memory(), "Adding entity id {}", entity_id); + component_manager->add_entity( + entity_id, + std::make_shared(std::move(segment_and_slice.segment_in_memory_)), + std::make_shared(std::move(segment_and_slice.ranges_and_key_.row_range_)), + std::make_shared(std::move(segment_and_slice.ranges_and_key_.col_range_)), + std::make_shared(std::move(segment_and_slice.ranges_and_key_.key_)), + fetch_count + ); +} + +size_t num_scheduling_iterations(const std::vector>& clauses) { + size_t res = 1UL; auto it = std::next(clauses.cbegin()); while (it != clauses.cend()) { auto prev_it = std::prev(it); @@ -549,11 +538,13 @@ size_t generate_scheduling_iterations(const std::vector> } ++it; } + ARCTICDB_DEBUG(log::memory(), "Processing pipeline has {} scheduling stages after the initial read and process", res); return res; } void remove_processed_clauses(std::vector>& clauses) { // Erase all the clauses we have already scheduled to run + ARCTICDB_SAMPLE_DEFAULT(RemoveProcessedClauses) auto it = std::next(clauses.cbegin()); while (it != clauses.cend()) { auto prev_it = std::prev(it); @@ -566,109 +557,140 @@ void remove_processed_clauses(std::vector>& clauses) { clauses.erase(clauses.cbegin(), it); } -folly::Future> schedule_clause_processing( - std::shared_ptr component_manager, - std::vector>&& segment_and_slice_futures, - std::vector>&& processing_unit_indexes, - std::shared_ptr>> clauses) { - // All the shared pointers as arguments to this function and created within it are to ensure that resources are - // correctly kept alive after this function returns its future - auto num_segments = segment_and_slice_futures.size(); - auto segment_and_slice_future_splitters = split_futures(std::move(segment_and_slice_futures)); +std::pair>, std::shared_ptr>> get_entity_ids_and_position_map( + std::shared_ptr& component_manager, + size_t num_segments, + std::vector>&& processing_unit_indexes) { + // Map from entity id to position in segment_and_slice_futures + auto id_to_pos = std::make_shared>(); + id_to_pos->reserve(num_segments); - // Map from index in segment_and_slice_future_splitters to the number of calls to process in the first clause that - // will require that segment - auto segment_fetch_counts = generate_segment_fetch_counts(processing_unit_indexes, num_segments); // Map from position in segment_and_slice_future_splitters to entity ids std::vector pos_to_id; - // Map from entity id to position in segment_and_slice_futures - auto id_to_pos = std::make_shared>(); pos_to_id.reserve(num_segments); + auto ids = component_manager->get_new_entity_ids(num_segments); for (auto&& [idx, id]: folly::enumerate(ids)) { pos_to_id.emplace_back(id); id_to_pos->emplace(id, idx); } - std::vector> entity_ids_vec; - entity_ids_vec.reserve(processing_unit_indexes.size()); + std::vector> entity_work_units; + entity_work_units.reserve(processing_unit_indexes.size()); for (const auto& indexes: processing_unit_indexes) { - entity_ids_vec.emplace_back(); - entity_ids_vec.back().reserve(indexes.size()); + entity_work_units.emplace_back(); + entity_work_units.back().reserve(indexes.size()); for (auto index: indexes) { - entity_ids_vec.back().emplace_back(pos_to_id[index]); + entity_work_units.back().emplace_back(pos_to_id[index]); } } + return std::make_pair(std::move(entity_work_units), std::move(id_to_pos)); +} + +std::shared_ptr>>> schedule_first_iteration( + std::shared_ptr& component_manager, + size_t num_segments, + std::vector>&& entities_by_work_unit, + std::shared_ptr>&& segment_fetch_counts, + std::vector&& segment_and_slice_future_splitters, + std::shared_ptr>&& id_to_pos, + std::shared_ptr>>& clauses) { // Used to make sure each entity is only added into the component manager once auto slice_added_mtx = std::make_shared>(num_segments); auto slice_added = std::make_shared>(num_segments, false); auto futures = std::make_shared>>>(); - for (auto&& entity_ids: entity_ids_vec) { + for (auto&& entity_ids: entities_by_work_unit) { std::vector> local_futs; local_futs.reserve(entity_ids.size()); for (auto id: entity_ids) { - local_futs.emplace_back(segment_and_slice_future_splitters[id_to_pos->at(id)].getFuture()); + const auto pos = id_to_pos->at(id); + auto& future_or_splitter = segment_and_slice_future_splitters[pos]; + // Some of the entities for this unit of work may be shared with other units of work + util::variant_match(future_or_splitter, + [&local_futs] (folly::Future& fut) { + local_futs.emplace_back(std::move(fut)); + }, + [&local_futs] (folly::FutureSplitter& splitter) { + local_futs.emplace_back(splitter.getFuture()); + }); } + futures->emplace_back( - folly::collect(local_futs) - .via(&async::cpu_executor()) - .thenValue([component_manager, - segment_fetch_counts, - id_to_pos, - slice_added_mtx, - slice_added, - clauses, - entity_ids = std::move(entity_ids)](std::vector&& segment_and_slices) mutable { - for (auto&& [idx, segment_and_slice]: folly::enumerate(segment_and_slices)) { - auto entity_id = entity_ids[idx]; - auto pos = id_to_pos->at(entity_id); - std::lock_guard lock((*slice_added_mtx)[pos]); - if (!(*slice_added)[pos]) { - component_manager->add_entity( - entity_id, - std::make_shared(std::move(segment_and_slice.segment_in_memory_)), - std::make_shared(std::move(segment_and_slice.ranges_and_key_.row_range_)), - std::make_shared(std::move(segment_and_slice.ranges_and_key_.col_range_)), - std::make_shared(std::move(segment_and_slice.ranges_and_key_.key_)), - (*segment_fetch_counts)[pos] - ); - (*slice_added)[pos] = true; - } - } - return async::MemSegmentProcessingTask(*clauses, std::move(entity_ids))(); - })); + folly::collect(local_futs) + .via(&async::io_executor()) // Stay on the same executor as the read so that we can inline if possible + .thenValueInline([component_manager, segment_fetch_counts, id_to_pos, slice_added_mtx, slice_added, clauses,entity_ids = std::move(entity_ids)] + (std::vector&& segment_and_slices) mutable { + for (auto&& [idx, segment_and_slice]: folly::enumerate(segment_and_slices)) { + auto entity_id = entity_ids[idx]; + auto pos = id_to_pos->at(entity_id); + std::lock_guard lock((*slice_added_mtx)[pos]); + if (!(*slice_added)[pos]) { + add_slice_to_component_manager(entity_id, segment_and_slice, component_manager, (*segment_fetch_counts)[pos]); + (*slice_added)[pos] = true; + } + } + return async::MemSegmentProcessingTask(*clauses, std::move(entity_ids))(); + })); } + return futures; +} - auto entity_ids_vec_fut = folly::Future>>::makeEmpty(); - // The number of iterations we need to pass through the following loop to get all the work scheduled - auto scheduling_iterations = generate_scheduling_iterations(*clauses); - for (size_t i=0; i work_scheduled(folly::Unit{}); - if (i > 0) { - work_scheduled = entity_ids_vec_fut.via(&async::cpu_executor()).thenValue([clauses, futures](std::vector>&& entity_ids_vec) { - futures->clear(); - for (auto&& entity_ids: entity_ids_vec) { - futures->emplace_back(async::submit_cpu_task(async::MemSegmentProcessingTask(*clauses, std::move(entity_ids)))); - } - return folly::Unit{}; - }); - } +folly::Future> schedule_clause_processing( + std::shared_ptr component_manager, + std::vector>&& segment_and_slice_futures, + std::vector>&& processing_unit_indexes, + std::shared_ptr>> clauses) { + // All the shared pointers as arguments to this function and created within it are to ensure that resources are + // correctly kept alive after this function returns its future + const auto num_segments = segment_and_slice_futures.size(); - entity_ids_vec_fut = work_scheduled.via(&async::cpu_executor()).thenValue([clauses, futures](auto&&) { - return folly::collect(*futures).via(&async::cpu_executor()).thenValue([clauses](std::vector>&& entity_ids_vec) { - remove_processed_clauses(*clauses); - if (clauses->empty()) { - return entity_ids_vec; - } else { - return clauses->front()->structure_for_processing(std::move(entity_ids_vec)); - } - }); + // Map from index in segment_and_slice_future_splitters to the number of calls to process in the first clause that + // will require that segment + auto segment_fetch_counts = generate_segment_fetch_counts(processing_unit_indexes, num_segments); + + auto segment_and_slice_future_splitters = split_futures(std::move(segment_and_slice_futures), *segment_fetch_counts); + + auto [entities_by_work_unit, entity_id_to_segment_pos] = get_entity_ids_and_position_map(component_manager, num_segments, std::move(processing_unit_indexes)); + + // At this point we have a set of entity ids grouped by the work units produced by the original structure_for_processing, + // and a map of those ids to the position in the vector of futures or future-splitters (which is the same order as + // originally generated from the index via the pipeline_context and ranges_and_keys), so we can add each entity id and + // its components to the component manager and schedule the first stage of work (i.e. from the beginning until either + // the end of the pipeline or the next required structure_for_processing + auto futures = schedule_first_iteration( + component_manager, + num_segments, + std::move(entities_by_work_unit), + std::move(segment_fetch_counts), + std::move(segment_and_slice_future_splitters), + std::move(entity_id_to_segment_pos), + clauses); + + auto entity_ids_vec_fut = folly::collect(*futures).via(&async::io_executor()); + + const auto scheduling_iterations = num_scheduling_iterations(*clauses); + for (auto i = 1UL; i < scheduling_iterations; ++i) { + entity_ids_vec_fut = std::move(entity_ids_vec_fut).thenValue([clauses, scheduling_iterations, i] (std::vector>&& entity_id_vectors) { + ARCTICDB_RUNTIME_DEBUG(log::memory(), "Scheduling iteration {} of {}", i, scheduling_iterations); + + util::check(!clauses->empty(), "Scheduling iteration {} has no clauses to process", scheduling_iterations); + remove_processed_clauses(*clauses); + auto next_units_of_work = clauses->front()->structure_for_processing(std::move(entity_id_vectors)); + + std::vector>> work_futures; + for(auto&& unit_of_work : next_units_of_work) { + ARCTICDB_RUNTIME_DEBUG(log::memory(), "Scheduling work for entity ids: {}", unit_of_work); + work_futures.emplace_back(async::submit_cpu_task(async::MemSegmentProcessingTask{*clauses, std::move(unit_of_work)})); + } + + return folly::collect(work_futures).via(&async::io_executor()); }); } - return entity_ids_vec_fut.via(&async::cpu_executor()).thenValue([](std::vector>&& entity_ids_vec) { - return flatten_entities(std::move(entity_ids_vec)); + + return std::move(entity_ids_vec_fut).thenValueInline([](std::vector>&& entity_id_vectors) { + return flatten_entities(std::move(entity_id_vectors)); }); } @@ -714,7 +736,7 @@ void set_output_descriptors( } } std::optional new_stream_descriptor; - if (proc.segments_.has_value() && proc.segments_->size() > 0) { + if (proc.segments_.has_value() && !proc.segments_->empty()) { new_stream_descriptor = std::make_optional(); new_stream_descriptor->set_index(proc.segments_->at(0)->descriptor().index()); for (size_t idx = 0; idx < new_stream_descriptor->index().field_count(); idx++) { @@ -798,25 +820,31 @@ std::vector generate_ranges_and_keys(PipelineContext& pipeline_con return res; } -std::vector> generate_segment_and_slice_futures( - const std::shared_ptr &store, - const std::shared_ptr &pipeline_context, - const ProcessingConfig &processing_config, - const std::vector& all_ranges - ) { - std::vector> res; - auto ranges_copy = all_ranges; - auto segment_and_slice_futures = store->batch_read_uncompressed(std::move(ranges_copy), columns_to_decode(pipeline_context)); - auto pipeline_desc = pipeline_context->descriptor(); +util::BitSet get_incompletes_bitset(const std::vector& all_ranges) { + util::BitSet output(all_ranges.size()); + util::BitSet::bulk_insert_iterator it(output); + for(auto&& [index, range] : folly::enumerate(all_ranges)) { + if(range.is_incomplete()) + it = index; + } + it.flush(); + return output; +} +std::vector> add_schema_check( + const std::shared_ptr &pipeline_context, + std::vector>&& segment_and_slice_futures, + util::BitSet&& incomplete_bitset, + const ProcessingConfig &processing_config) { + std::vector> res; + res.reserve(segment_and_slice_futures.size()); for (size_t i = 0; i < segment_and_slice_futures.size(); ++i) { auto&& fut = segment_and_slice_futures.at(i); - bool is_incomplete = all_ranges.at(i).is_incomplete(); + const bool is_incomplete = incomplete_bitset[i]; if (is_incomplete) { res.push_back( std::move(fut) - .via(&async::cpu_executor()) - .thenValue([pipeline_desc, processing_config](SegmentAndSlice &&read_result) { + .thenValueInline([pipeline_desc=pipeline_context->descriptor(), processing_config](SegmentAndSlice &&read_result) { if (!processing_config.dynamic_schema_) { auto check = check_schema_matches_incomplete(read_result.segment_in_memory_.descriptor(), pipeline_desc); if (std::holds_alternative(check)) { @@ -829,10 +857,19 @@ std::vector> generate_segment_and_slic res.push_back(std::move(fut)); } } - return res; } +std::vector> generate_segment_and_slice_futures( + const std::shared_ptr &store, + const std::shared_ptr &pipeline_context, + const ProcessingConfig &processing_config, + std::vector&& all_ranges) { + auto incomplete_bitset = get_incompletes_bitset(all_ranges); + auto segment_and_slice_futures = store->batch_read_uncompressed(std::move(all_ranges), columns_to_decode(pipeline_context)); + return add_schema_check(pipeline_context, std::move(segment_and_slice_futures), std::move(incomplete_bitset), processing_config); +} + /* * Processes the slices in the given pipeline_context. * @@ -865,15 +902,15 @@ folly::Future> read_and_process( std::vector> processing_unit_indexes = read_query->clauses_[0]->structure_for_processing(ranges_and_keys); // Start reading as early as possible - auto segment_and_slice_futures = generate_segment_and_slice_futures(store, pipeline_context, processing_config, ranges_and_keys); + auto segment_and_slice_futures = generate_segment_and_slice_futures(store, pipeline_context, processing_config, std::move(ranges_and_keys)); - return schedule_clause_processing(component_manager, - std::move(segment_and_slice_futures), - std::move(processing_unit_indexes), - std::make_shared>>( - read_query->clauses_)) + return schedule_clause_processing( + component_manager, + std::move(segment_and_slice_futures), + std::move(processing_unit_indexes), + std::make_shared>>(read_query->clauses_)) .via(&async::cpu_executor()) - .thenValue([component_manager, read_query, pipeline_context](auto&& processed_entity_ids) { + .thenValue([component_manager, read_query, pipeline_context](std::vector&& processed_entity_ids) { auto proc = gather_entities, std::shared_ptr, std::shared_ptr>(*component_manager, std::move(processed_entity_ids)); if (std::any_of(read_query->clauses_.begin(), read_query->clauses_.end(), [](const std::shared_ptr& clause) { @@ -1100,7 +1137,7 @@ void check_incompletes_index_ranges_dont_overlap(const std::shared_ptr unique_timestamp_ranges; for (auto it = pipeline_context->incompletes_begin(); it!= pipeline_context->end(); it++) { if (it->slice_and_key().slice().rows().diff() == 0) { @@ -1121,7 +1158,8 @@ void check_incompletes_index_ranges_dont_overlap(const std::shared_ptr do_direct_read_or_process( ARCTICDB_SAMPLE(RunPipelineAndOutput, 0) util::check_rte(!pipeline_context->is_pickled(),"Cannot filter pickled data"); return read_and_process(store, pipeline_context, read_query, read_options) - .thenValue([store, pipeline_context, &read_options, &handler_data](auto&& segs) { + .thenValue([store, pipeline_context, &read_options, &handler_data](std::vector&& segs) { return prepare_output_frame(std::move(segs), pipeline_context, store, read_options, handler_data); }); } else { @@ -1542,8 +1580,8 @@ DeleteIncompleteKeysOnExit::DeleteIncompleteKeysOnExit( std::shared_ptr pipeline_context, std::shared_ptr store, bool via_iteration) - : context_(pipeline_context), - store_(store), + : context_(std::move(pipeline_context)), + store_(std::move(store)), via_iteration_(via_iteration) { } @@ -1905,7 +1943,7 @@ void set_row_id_if_index_only( if (read_query.columns && read_query.columns->empty() && pipeline_context.descriptor().index().type() == IndexDescriptor::Type::ROWCOUNT) { - frame.set_row_id(pipeline_context.rows_ - 1); + frame.set_row_id(static_cast(pipeline_context.rows_ - 1)); } } diff --git a/cpp/arcticdb/version/version_core.hpp b/cpp/arcticdb/version/version_core.hpp index 7a26c69946..cbc8b4724f 100644 --- a/cpp/arcticdb/version/version_core.hpp +++ b/cpp/arcticdb/version/version_core.hpp @@ -132,10 +132,10 @@ folly::Future read_multi_key( std::any& handler_data); folly::Future> schedule_clause_processing( - std::shared_ptr component_manager, - std::vector>&& segment_and_slice_futures, - std::vector>&& processing_unit_indexes, - std::shared_ptr>> clauses); + std::shared_ptr component_manager, + std::vector>&& segment_and_slice_futures, + std::vector>&& processing_unit_indexes, + std::shared_ptr>> clauses); FrameAndDescriptor read_segment_impl( const std::shared_ptr& store, diff --git a/cpp/arcticdb/version/version_utils.hpp b/cpp/arcticdb/version/version_utils.hpp index 22925db0c7..73e1cc3153 100644 --- a/cpp/arcticdb/version/version_utils.hpp +++ b/cpp/arcticdb/version/version_utils.hpp @@ -131,21 +131,21 @@ std::shared_ptr build_version_map_entry_with_predicate_iteratio std::vector read_keys; for (auto key_type : key_types) { store->iterate_type(key_type, - [&predicate, &read_keys, &store, &output, &perform_read_segment_with_keys](VariantKey &&vk) { - const auto &key = to_atom(std::move(vk)); - if (!predicate(key)) - return; - - read_keys.push_back(key); - ARCTICDB_DEBUG(log::storage(), "Version map iterating key {}", key); - if (perform_read_segment_with_keys) { - auto [kv, seg] = store->read_sync(to_atom(key)); - LoadProgress load_progress; - (void)read_segment_with_keys(seg, output, load_progress); - } - }, - prefix); - } + [&predicate, &read_keys, &store, &output, &perform_read_segment_with_keys](VariantKey &&vk) { + const auto &key = to_atom(std::move(vk)); + if (!predicate(key)) + return; + + read_keys.push_back(key); + ARCTICDB_DEBUG(log::storage(), "Version map iterating key {}", key); + if (perform_read_segment_with_keys) { + auto [kv, seg] = store->read_sync(to_atom(key)); + LoadProgress load_progress; + (void)read_segment_with_keys(seg, output, load_progress); + } + }, + prefix); +} if (!perform_read_segment_with_keys) { output->keys_.insert(output->keys_.end(), std::move_iterator(read_keys.begin()), @@ -176,13 +176,11 @@ inline void read_symbol_ref(const std::shared_ptr& store, const St std::pair key_seg_pair; // Trying to read a missing ref key is expected e.g. when writing a previously missing symbol. // If the ref key is missing we keep the entry empty and should not raise warnings. - auto read_opts = storage::ReadKeyOpts{}; - read_opts.dont_warn_about_missing_key=true; try { - key_seg_pair = store->read_sync(RefKey{stream_id, KeyType::VERSION_REF}, read_opts); + key_seg_pair = store->read_sync(RefKey{stream_id, KeyType::VERSION_REF}); } catch (const storage::KeyNotFoundException&) { try { - key_seg_pair = store->read_sync(RefKey{stream_id, KeyType::VERSION, true}, read_opts); + key_seg_pair = store->read_sync(RefKey{stream_id, KeyType::VERSION, true}); } catch (const storage::KeyNotFoundException&) { return; } diff --git a/python/arcticdb/version_store/processing.py b/python/arcticdb/version_store/processing.py index 5f77cda66d..f58944376f 100644 --- a/python/arcticdb/version_store/processing.py +++ b/python/arcticdb/version_store/processing.py @@ -585,7 +585,6 @@ def agg(self, aggregations: Dict[str, Union[str, Tuple[str, str]]]): self._python_clauses[-1].aggregations = aggregations return self - def resample( self, rule: Union[str, pd.DateOffset], @@ -780,7 +779,6 @@ def resample( self._python_clauses = self._python_clauses + [PythonResampleClause(rule=rule, closed=boundary_map[closed], label=boundary_map[label], offset=offset_ns, origin=origin)] return self - # TODO: specify type of other must be QueryBuilder with from __future__ import annotations once only Python 3.7+ # supported def then(self, other):