From 27893149a504e3ee1a7094bdf44da359c086d095 Mon Sep 17 00:00:00 2001 From: William Dealtry Date: Fri, 9 Feb 2024 18:10:02 +0000 Subject: [PATCH] WIP descriptor changes --- cpp/arcticdb/CMakeLists.txt | 21 +- cpp/arcticdb/async/task_scheduler.hpp | 6 +- cpp/arcticdb/async/tasks.cpp | 13 +- cpp/arcticdb/async/tasks.hpp | 23 +- cpp/arcticdb/async/test/test_async.cpp | 4 +- cpp/arcticdb/codec/codec-inl.hpp | 51 +- cpp/arcticdb/codec/codec.cpp | 370 ++++++----- cpp/arcticdb/codec/codec.hpp | 52 +- cpp/arcticdb/codec/core.hpp | 29 +- cpp/arcticdb/codec/encode_common.hpp | 67 +- cpp/arcticdb/codec/encode_v1.cpp | 68 +- cpp/arcticdb/codec/encode_v2.cpp | 607 ++++++++++-------- cpp/arcticdb/codec/encoded_field.cpp | 19 + cpp/arcticdb/codec/encoded_field.hpp | 420 ++++++------ .../codec/encoded_field_collection.hpp | 147 ++++- cpp/arcticdb/codec/encoding_sizes.cpp | 24 - cpp/arcticdb/codec/encoding_sizes.hpp | 74 +-- cpp/arcticdb/codec/lz4.hpp | 8 +- cpp/arcticdb/codec/magic_words.hpp | 6 +- cpp/arcticdb/codec/passthrough.hpp | 37 +- cpp/arcticdb/codec/protobuf_mappings.cpp | 154 +++++ cpp/arcticdb/codec/protobuf_mappings.hpp | 73 +++ cpp/arcticdb/codec/python_bindings.cpp | 6 +- cpp/arcticdb/codec/segment.cpp | 416 +++++++----- cpp/arcticdb/codec/segment.hpp | 300 ++++----- cpp/arcticdb/codec/segment_header.cpp | 94 +++ cpp/arcticdb/codec/segment_header.hpp | 270 ++++++++ cpp/arcticdb/codec/segment_identifier.hpp | 66 ++ cpp/arcticdb/codec/test/test_codec.cpp | 226 ++++--- .../test/test_encode_field_collection.cpp | 33 + .../codec/test/test_encoded_field.cpp | 139 ++++ .../codec/test/test_segment_header.cpp | 83 +++ .../codec/typed_block_encoder_impl.hpp | 41 +- .../variant_encoded_field_collection.cpp | 41 -- .../variant_encoded_field_collection.hpp | 25 - cpp/arcticdb/codec/zstd.hpp | 6 +- cpp/arcticdb/column_store/chunked_buffer.hpp | 4 +- cpp/arcticdb/column_store/column.cpp | 8 +- cpp/arcticdb/column_store/column.hpp | 16 +- cpp/arcticdb/column_store/column_data.hpp | 30 +- cpp/arcticdb/column_store/memory_segment.hpp | 40 +- .../column_store/memory_segment_impl.cpp | 29 +- .../column_store/memory_segment_impl.hpp | 44 +- cpp/arcticdb/column_store/string_pool.cpp | 4 + cpp/arcticdb/column_store/string_pool.hpp | 6 + .../test/ingestion_stress_test.cpp | 4 - .../column_store/test/test_column.cpp | 2 +- .../test/test_index_filtering.cpp | 8 +- cpp/arcticdb/entity/descriptor_item.hpp | 6 +- cpp/arcticdb/entity/field_collection.hpp | 5 +- .../entity/field_collection_proto.cpp | 3 +- .../entity/field_collection_proto.hpp | 4 +- cpp/arcticdb/entity/merge_descriptors.cpp | 8 +- cpp/arcticdb/entity/merge_descriptors.hpp | 8 +- cpp/arcticdb/entity/protobuf_mappings.cpp | 140 ++++ cpp/arcticdb/entity/protobuf_mappings.hpp | 68 +- cpp/arcticdb/entity/protobufs.hpp | 2 +- cpp/arcticdb/entity/serialized_key.hpp | 20 +- cpp/arcticdb/entity/stream_descriptor.hpp | 208 +++--- cpp/arcticdb/entity/test/test_atom_key.cpp | 4 +- cpp/arcticdb/entity/test/test_ref_key.cpp | 2 +- cpp/arcticdb/entity/timeseries_descriptor.hpp | 203 ++++-- cpp/arcticdb/entity/type_utils.cpp | 16 - cpp/arcticdb/entity/types-inl.hpp | 28 +- cpp/arcticdb/entity/types.hpp | 224 +++++-- cpp/arcticdb/entity/types_proto.cpp | 199 +++--- cpp/arcticdb/entity/types_proto.hpp | 166 +---- cpp/arcticdb/log/log.cpp | 4 +- cpp/arcticdb/memory_layout.hpp | 292 +++++++++ cpp/arcticdb/pipeline/column_stats.cpp | 2 + cpp/arcticdb/pipeline/frame_utils.cpp | 53 +- cpp/arcticdb/pipeline/frame_utils.hpp | 2 +- .../pipeline/index_segment_reader.cpp | 26 +- .../pipeline/index_segment_reader.hpp | 12 +- cpp/arcticdb/pipeline/index_utils.cpp | 12 +- cpp/arcticdb/pipeline/index_utils.hpp | 39 +- cpp/arcticdb/pipeline/index_writer.hpp | 12 +- cpp/arcticdb/pipeline/pipeline_context.cpp | 12 +- cpp/arcticdb/pipeline/pipeline_context.hpp | 2 +- cpp/arcticdb/pipeline/query.hpp | 14 +- cpp/arcticdb/pipeline/read_frame.cpp | 167 ++--- cpp/arcticdb/pipeline/read_pipeline.hpp | 26 +- cpp/arcticdb/pipeline/write_frame.cpp | 8 +- cpp/arcticdb/processing/aggregation.cpp | 4 +- cpp/arcticdb/processing/clause.cpp | 11 +- cpp/arcticdb/processing/clause.hpp | 6 - .../processing/operation_dispatch_unary.hpp | 2 +- cpp/arcticdb/python/normalization_checks.cpp | 2 +- cpp/arcticdb/python/python_handlers.cpp | 403 ++++++------ cpp/arcticdb/python/python_handlers.hpp | 22 +- .../python/python_to_tensor_frame.cpp | 8 +- .../storage/azure/azure_mock_client.cpp | 2 +- .../storage/azure/azure_real_client.cpp | 15 +- cpp/arcticdb/storage/azure/azure_storage.cpp | 2 +- cpp/arcticdb/storage/azure/azure_storage.hpp | 1 + .../storage/coalesced/multi_segment_utils.hpp | 12 +- .../storage/file/mapped_file_storage.cpp | 16 +- .../storage/file/mapped_file_storage.hpp | 8 +- cpp/arcticdb/storage/library.hpp | 15 +- cpp/arcticdb/storage/library_manager.cpp | 1 + cpp/arcticdb/storage/library_path.hpp | 3 - .../storage/lmdb/lmdb_mock_client.cpp | 6 +- .../storage/lmdb/lmdb_real_client.cpp | 7 +- cpp/arcticdb/storage/lmdb/lmdb_storage.cpp | 5 +- cpp/arcticdb/storage/lmdb/lmdb_storage.hpp | 2 +- .../storage/memory/memory_storage.cpp | 9 +- .../storage/memory/memory_storage.hpp | 1 + cpp/arcticdb/storage/mongo/mongo_client.cpp | 13 +- cpp/arcticdb/storage/mongo/mongo_storage.hpp | 1 + .../storage/rocksdb/rocksdb_storage.cpp | 6 +- .../storage/rocksdb/rocksdb_storage.hpp | 3 +- cpp/arcticdb/storage/s3/detail-inl.hpp | 6 +- cpp/arcticdb/storage/s3/s3_mock_client.cpp | 2 +- cpp/arcticdb/storage/s3/s3_real_client.cpp | 14 +- cpp/arcticdb/storage/storage.hpp | 2 +- cpp/arcticdb/storage/test/test_embedded.cpp | 25 +- .../storage/test/test_mongo_storage.cpp | 11 +- .../storage/test/test_storage_exceptions.cpp | 8 +- cpp/arcticdb/stream/aggregator-inl.hpp | 3 +- cpp/arcticdb/stream/append_map.cpp | 52 +- cpp/arcticdb/stream/index.hpp | 59 +- cpp/arcticdb/stream/merge.hpp | 2 +- cpp/arcticdb/stream/merge_utils.hpp | 4 +- cpp/arcticdb/stream/protobuf_mappings.cpp | 83 +++ cpp/arcticdb/stream/protobuf_mappings.hpp | 58 +- cpp/arcticdb/stream/python_bindings.cpp | 88 +-- cpp/arcticdb/stream/row_builder.hpp | 1 + cpp/arcticdb/stream/stream_reader.hpp | 2 +- cpp/arcticdb/stream/stream_utils.hpp | 8 +- cpp/arcticdb/stream/test/test_append_map.cpp | 4 +- cpp/arcticdb/toolbox/library_tool.cpp | 4 +- cpp/arcticdb/toolbox/library_tool.hpp | 4 +- cpp/arcticdb/util/buffer.hpp | 114 +++- cpp/arcticdb/util/buffer_holder.hpp | 4 +- cpp/arcticdb/util/cursored_buffer.hpp | 1 - cpp/arcticdb/util/lock_table.hpp | 2 +- cpp/arcticdb/util/magic_num.hpp | 2 +- cpp/arcticdb/util/pb_util.hpp | 8 +- cpp/arcticdb/util/ref_counted_map.hpp | 55 -- cpp/arcticdb/util/test/generators.hpp | 2 +- cpp/arcticdb/util/test/gtest_utils.hpp | 2 +- .../util/test/rapidcheck_generators.hpp | 4 +- .../util/test/test_slab_allocator.cpp | 3 +- cpp/arcticdb/util/type_handler.hpp | 9 +- .../version/local_versioned_engine.cpp | 19 +- cpp/arcticdb/version/python_bindings.cpp | 13 +- cpp/arcticdb/version/schema_checks.hpp | 6 +- cpp/arcticdb/version/symbol_list.cpp | 9 +- cpp/arcticdb/version/symbol_list.hpp | 26 +- .../version/test/test_version_store.cpp | 2 +- cpp/arcticdb/version/version_core-inl.hpp | 2 +- cpp/arcticdb/version/version_core.cpp | 52 +- cpp/arcticdb/version/version_tasks.hpp | 7 +- cpp/arcticdb/version/version_utils.cpp | 10 +- cpp/arcticdb/version/version_utils.hpp | 1 + cpp/proto/arcticc/pb2/descriptors.proto | 8 + cpp/proto/arcticc/pb2/encoding.proto | 5 +- 157 files changed, 4614 insertions(+), 2989 deletions(-) create mode 100644 cpp/arcticdb/codec/encoded_field.cpp delete mode 100644 cpp/arcticdb/codec/encoding_sizes.cpp create mode 100644 cpp/arcticdb/codec/protobuf_mappings.cpp create mode 100644 cpp/arcticdb/codec/protobuf_mappings.hpp create mode 100644 cpp/arcticdb/codec/segment_header.cpp create mode 100644 cpp/arcticdb/codec/segment_header.hpp create mode 100644 cpp/arcticdb/codec/segment_identifier.hpp create mode 100644 cpp/arcticdb/codec/test/test_encode_field_collection.cpp create mode 100644 cpp/arcticdb/codec/test/test_encoded_field.cpp create mode 100644 cpp/arcticdb/codec/test/test_segment_header.cpp delete mode 100644 cpp/arcticdb/codec/variant_encoded_field_collection.cpp delete mode 100644 cpp/arcticdb/codec/variant_encoded_field_collection.hpp create mode 100644 cpp/arcticdb/entity/protobuf_mappings.cpp create mode 100644 cpp/arcticdb/memory_layout.hpp create mode 100644 cpp/arcticdb/stream/protobuf_mappings.cpp delete mode 100644 cpp/arcticdb/util/ref_counted_map.hpp diff --git a/cpp/arcticdb/CMakeLists.txt b/cpp/arcticdb/CMakeLists.txt index fc34d7c0d1..b93a0ff15e 100644 --- a/cpp/arcticdb/CMakeLists.txt +++ b/cpp/arcticdb/CMakeLists.txt @@ -219,7 +219,6 @@ set(arcticdb_srcs entity/type_conversion.hpp entity/types.hpp entity/type_utils.hpp - entity/types_proto.hpp entity/types-inl.hpp entity/variant_key.hpp entity/versioned_item.hpp @@ -310,7 +309,7 @@ set(arcticdb_srcs stream/index.hpp stream/merge.hpp stream/merge.hpp - stream/merge.hpp util/ref_counted_map.hpp + stream/merge.hpp stream/protobuf_mappings.hpp stream/row_builder.hpp stream/schema.hpp @@ -353,8 +352,6 @@ set(arcticdb_srcs util/preconditions.hpp util/preprocess.hpp util/ranges_from_future.hpp - util/ref_counted_map.hpp - util/ref_counted_map.hpp util/regex_filter.hpp util/simple_string_hash.hpp util/slab_allocator.hpp @@ -390,9 +387,7 @@ set(arcticdb_srcs codec/codec.cpp codec/encode_v1.cpp codec/encode_v2.cpp - codec/encoding_sizes.cpp codec/segment.cpp - codec/variant_encoded_field_collection.cpp column_store/chunked_buffer.cpp column_store/column.cpp column_store/column_data.cpp @@ -408,7 +403,6 @@ set(arcticdb_srcs entity/performance_tracing.cpp entity/types.cpp entity/type_utils.cpp - entity/types_proto.cpp log/log.cpp pipeline/column_mapping.cpp pipeline/column_stats.cpp @@ -487,7 +481,12 @@ set(arcticdb_srcs version/symbol_list.cpp version/version_map_batch_methods.cpp storage/s3/ec2_utils.cpp -) + codec/segment_header.hpp + codec/protobuf_mappings.hpp + codec/encoded_field.cpp + memory_layout.hpp + codec/protobuf_mappings.cpp + entity/protobuf_mappings.cpp entity/types_proto.hpp entity/types_proto.cpp stream/protobuf_mappings.cpp codec/segment_header.cpp codec/segment_identifier.hpp) if(${ARCTICDB_INCLUDE_ROCKSDB}) list (APPEND arcticdb_srcs @@ -831,7 +830,7 @@ if(${TEST}) version/test/test_sorting_info_state_machine.cpp version/test/version_map_model.hpp storage/test/common.hpp - ) + codec/test/test_encode_field_collection.cpp codec/test/test_segment_header.cpp codec/test/test_encoded_field.cpp) set(EXECUTABLE_PERMS OWNER_WRITE OWNER_READ OWNER_EXECUTE GROUP_READ GROUP_EXECUTE WORLD_READ WORLD_EXECUTE) # 755 @@ -906,7 +905,7 @@ if(${TEST}) ) endif() - gtest_discover_tests(test_unit_arcticdb PROPERTIES DISCOVERY_TIMEOUT 60) + # gtest_discover_tests(test_unit_arcticdb PROPERTIES DISCOVERY_TIMEOUT 60) set(benchmark_srcs stream/test/stream_test_common.cpp @@ -992,5 +991,5 @@ if(${TEST}) ) endif() - gtest_discover_tests(arcticdb_rapidcheck_tests PROPERTIES DISCOVERY_TIMEOUT 60) + #gtest_discover_tests(arcticdb_rapidcheck_tests PROPERTIES DISCOVERY_TIMEOUT 60) endif() diff --git a/cpp/arcticdb/async/task_scheduler.hpp b/cpp/arcticdb/async/task_scheduler.hpp index 91a7a18222..6ff5374984 100644 --- a/cpp/arcticdb/async/task_scheduler.hpp +++ b/cpp/arcticdb/async/task_scheduler.hpp @@ -152,14 +152,16 @@ class TaskScheduler { explicit TaskScheduler(const std::optional& cpu_thread_count = std::nullopt, const std::optional& io_thread_count = std::nullopt) : cpu_thread_count_(cpu_thread_count ? *cpu_thread_count : ConfigsMap::instance()->get_int("VersionStore.NumCPUThreads", get_default_num_cpus())), - io_thread_count_(io_thread_count ? *io_thread_count : ConfigsMap::instance()->get_int("VersionStore.NumIOThreads", std::min(100, (int) (cpu_thread_count_ * 1.5)))), + io_thread_count_(io_thread_count ? *io_thread_count : ConfigsMap::instance()->get_int("VersionStore.NumIOThreads", (int) (cpu_thread_count_ * 1.5))), cpu_exec_(cpu_thread_count_, std::make_shared("CPUPool")) , io_exec_(io_thread_count_, std::make_shared("IOPool")){ util::check(cpu_thread_count_ > 0 && io_thread_count_ > 0, "Zero IO or CPU threads: {} {}", io_thread_count_, cpu_thread_count_); ARCTICDB_RUNTIME_DEBUG(log::schedule(), "Task scheduler created with {:d} {:d}", cpu_thread_count_, io_thread_count_); } - ~TaskScheduler() = default; + ~TaskScheduler() { + log::version().info("#### Destroying task scheduler"); + } template auto submit_cpu_task(Task &&t) { diff --git a/cpp/arcticdb/async/tasks.cpp b/cpp/arcticdb/async/tasks.cpp index b99bcf91e6..2a8d4e8731 100644 --- a/cpp/arcticdb/async/tasks.cpp +++ b/cpp/arcticdb/async/tasks.cpp @@ -27,10 +27,10 @@ namespace arcticdb::async { } } - return index_descriptor(StreamDescriptor::id_from_proto(desc.proto()), idx, fields); + return index_descriptor_from_range(desc.id(), idx, fields); } else { - return index_descriptor(StreamDescriptor::id_from_proto(desc.proto()), idx, desc.fields()); + return index_descriptor_from_range(desc.id(), idx, desc.fields()); } }); } @@ -39,10 +39,10 @@ namespace arcticdb::async { auto key = std::move(key_segment_pair.atom_key()); auto seg = std::move(key_segment_pair.release_segment()); ARCTICDB_DEBUG(log::storage(), "ReadAndDecodeAtomTask decoding segment of size {} with key {}", - seg.total_segment_size(), + seg.size(), key); auto &hdr = seg.header(); - auto desc = StreamDescriptor(std::make_shared(std::move(*hdr.mutable_stream_descriptor())), seg.fields_ptr()); + const auto& desc = seg.descriptor(); auto descriptor = async::get_filtered_descriptor(desc, columns_to_decode_); ranges_and_key_.col_range_.second = ranges_and_key_.col_range_.first + (descriptor.field_count() - descriptor.index().field_count()); ARCTICDB_TRACE(log::codec(), "Creating segment"); @@ -53,12 +53,11 @@ namespace arcticdb::async { 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 of size {} with key {}", - seg.total_segment_size(), + ARCTICDB_DEBUG(log::storage(), "ReadAndDecodeAtomTask decoding segment with key {}", variant_key_view(sk.key())); auto &hdr = seg.header(); - auto desc = StreamDescriptor(std::make_shared(std::move(*hdr.mutable_stream_descriptor())), seg.fields_ptr()); + 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()); diff --git a/cpp/arcticdb/async/tasks.hpp b/cpp/arcticdb/async/tasks.hpp index 5e870a3eae..21d07e6e5c 100644 --- a/cpp/arcticdb/async/tasks.hpp +++ b/cpp/arcticdb/async/tasks.hpp @@ -207,7 +207,9 @@ struct KeySegmentContinuation { }; 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) { return lib->read(key, opts); }); + return util::variant_match(variant_key, [&lib, &opts](const auto &key) { + return lib->read(key, opts); + }); } template @@ -321,8 +323,7 @@ struct DecodeSegmentTask : BaseTask { ARCTICDB_SAMPLE(DecodeAtomTask, 0) auto key_seg = std::move(ks); - ARCTICDB_DEBUG(log::storage(), "ReadAndDecodeAtomTask decoding segment of size {} with key {}", - key_seg.segment().total_segment_size(), + ARCTICDB_DEBUG(log::storage(), "ReadAndDecodeAtomTask decoding segment with key {}", variant_key_view(key_seg.variant_key())); return {key_seg.variant_key(), decode_segment(std::move(key_seg.segment()))}; @@ -365,7 +366,7 @@ struct DecodeSlicesTask : BaseTask { ARCTICDB_SAMPLE(DecodeSlicesTask, 0) auto sk_pairs = std::move(skp); return sk_pairs.transform([that=this] (auto&& ssp){ - auto seg_slice_pair = std::forward(ssp); + auto seg_slice_pair = std::move(ssp); ARCTICDB_DEBUG(log::version(), "Decoding slice {}", seg_slice_pair.second.key()); return that->decode_into_slice(std::move(seg_slice_pair)); }); @@ -440,8 +441,7 @@ struct DecodeMetadataTask : BaseTask { std::pair, std::optional> operator()(storage::KeySegmentPair &&ks) const { ARCTICDB_SAMPLE(ReadMetadataTask, 0) auto key_seg = std::move(ks); - ARCTICDB_DEBUG(log::storage(), "ReadAndDecodeMetadataTask decoding segment of size {} with key {}", - key_seg.segment().total_segment_size(), variant_key_view(key_seg.variant_key())); + ARCTICDB_DEBUG(log::storage(), "ReadAndDecodeMetadataTask decoding segment with key {}", variant_key_view(key_seg.variant_key())); auto meta = decode_metadata_from_segment(key_seg.segment()); std::pair> output; @@ -461,18 +461,14 @@ struct DecodeTimeseriesDescriptorTask : BaseTask { std::pair operator()(storage::KeySegmentPair &&ks) const { ARCTICDB_SAMPLE(DecodeTimeseriesDescriptorTask, 0) auto key_seg = std::move(ks); - ARCTICDB_DEBUG(log::storage(), "DecodeTimeseriesDescriptorTask decoding segment of size {} with key {}", - key_seg.segment().total_segment_size(), variant_key_view(key_seg.variant_key())); + ARCTICDB_DEBUG(log::storage(), "DecodeTimeseriesDescriptorTask decoding segment with key {}", variant_key_view(key_seg.variant_key())); auto maybe_desc = decode_timeseries_descriptor(key_seg.segment()); util::check(static_cast(maybe_desc), "Failed to decode timeseries descriptor"); return std::make_pair( std::move(key_seg.variant_key()), - TimeseriesDescriptor{ - std::make_shared(std::move(std::get<1>(*maybe_desc))), - std::make_shared(std::move(std::get<2>(*maybe_desc)))} - ); + std::move(*maybe_desc)); } }; @@ -484,8 +480,7 @@ struct DecodeMetadataAndDescriptorTask : BaseTask { std::tuple, StreamDescriptor> operator()(storage::KeySegmentPair &&ks) const { ARCTICDB_SAMPLE(ReadMetadataAndDescriptorTask, 0) auto key_seg = std::move(ks); - ARCTICDB_DEBUG(log::storage(), "DecodeMetadataAndDescriptorTask decoding segment of size {} with key {}", - key_seg.segment().total_segment_size(), variant_key_view(key_seg.variant_key())); + ARCTICDB_DEBUG(log::storage(), "DecodeMetadataAndDescriptorTask decoding segment with key {}", variant_key_view(key_seg.variant_key())); auto [any, descriptor] = decode_metadata_and_descriptor_fields(key_seg.segment()); return std::make_tuple( diff --git a/cpp/arcticdb/async/test/test_async.cpp b/cpp/arcticdb/async/test/test_async.cpp index 50111632ce..c639455fc1 100644 --- a/cpp/arcticdb/async/test/test_async.cpp +++ b/cpp/arcticdb/async/test/test_async.cpp @@ -43,7 +43,7 @@ TEST(Async, SinkBasic) { auto seg = ac::SegmentInMemory(); aa::EncodeAtomTask enc{ - ac::entity::KeyType::GENERATION, ac::entity::VersionId{6}, ac::entity::NumericId{123}, ac::entity::NumericId{456}, ac::timestamp{457}, {ac::entity::NumericIndex{999}}, std::move(seg), codec_opt, ac::EncodingVersion::V2 + ac::entity::KeyType::GENERATION, ac::entity::VersionId{6}, ac::NumericId{123}, ac::NumericId{456}, ac::timestamp{457}, ac::entity::NumericIndex{999}, std::move(seg), codec_opt, ac::EncodingVersion::V2 }; auto v = sched.submit_cpu_task(std::move(enc)).via(&aa::io_executor()).thenValue(aa::WriteSegmentTask{lib}).get(); @@ -52,7 +52,7 @@ TEST(Async, SinkBasic) { auto default_content_hash = h.digest(); ASSERT_EQ(ac::entity::atom_key_builder().gen_id(6).start_index(456).end_index(457).creation_ts(999) - .content_hash(default_content_hash).build(ac::entity::NumericId{123}, ac::entity::KeyType::GENERATION), + .content_hash(default_content_hash).build(ac::NumericId{123}, ac::entity::KeyType::GENERATION), to_atom(v) ); } diff --git a/cpp/arcticdb/codec/codec-inl.hpp b/cpp/arcticdb/codec/codec-inl.hpp index 4adcfc6f76..ce0aa54b54 100644 --- a/cpp/arcticdb/codec/codec-inl.hpp +++ b/cpp/arcticdb/codec/codec-inl.hpp @@ -15,8 +15,10 @@ #include #include #include - #include +#include +#include +#include #include @@ -32,23 +34,23 @@ void decode_block(const BlockType &block, const std::uint8_t *input, T *output) arcticdb::detail::PassthroughDecoder::decode_block(input, size_to_decode, output, decoded_size); } else { std::uint32_t encoder_version = block.encoder_version(); - switch (block.codec().codec_case()) { - case arcticdb::proto::encoding::VariantCodec::kZstd: - arcticdb::detail::ZstdDecoder::decode_block(encoder_version, - input, - size_to_decode, - output, - decoded_size); - break; - case arcticdb::proto::encoding::VariantCodec::kLz4: - arcticdb::detail::Lz4Decoder::decode_block(encoder_version, - input, - size_to_decode, - output, - decoded_size); - break; - default: - util::raise_error_msg("Unsupported block codec {}", block); + switch (block.codec().codec_type()) { + case arcticdb::Codec::ZSTD: + arcticdb::detail::ZstdDecoder::decode_block(encoder_version, + input, + size_to_decode, + output, + decoded_size); + break; + case arcticdb::Codec::LZ4: + arcticdb::detail::Lz4Decoder::decode_block(encoder_version, + input, + size_to_decode, + output, + decoded_size); + break; + default: + util::raise_rte("Unsupported block codec {}", codec_type_to_string(block.codec().codec_type())); } } } @@ -78,6 +80,7 @@ std::size_t decode_ndarray( EncodingVersion encoding_version ) { ARCTICDB_SUBSAMPLE_AGG(DecodeNdArray) + std::size_t read_bytes = 0; td.visit_tag([&](auto type_desc_tag) { using TD = std::decay_t; @@ -85,7 +88,7 @@ std::size_t decode_ndarray( const auto data_size = encoding_sizes::data_uncompressed_size(field); const bool is_empty_array = (data_size == 0) && type_desc_tag.dimension() > Dimension::Dim0; - // Empty array will not contain actual data, however, its sparse map should be loaded + // Empty array types will not contain actual data, however, its sparse map should be loaded // so that we can distinguish None from [] if(data_size == 0 && !is_empty_array) { util::check(type_desc_tag.data_type() == DataType::EMPTYVAL, @@ -152,25 +155,25 @@ std::size_t decode_ndarray( return read_bytes; } -template +template std::size_t decode_field( const TypeDescriptor &td, - const EncodedFieldType &field, + const EncodedFieldImpl &field, const std::uint8_t *input, DataSink &data_sink, std::optional& bv, EncodingVersion encoding_version) { size_t magic_size = 0u; - if constexpr(std::is_same_v) { + if (encoding_version != EncodingVersion::V1) { magic_size += sizeof(ColumnMagic); util::check_magic(input); } switch (field.encoding_case()) { - case EncodedFieldType::kNdarray: + case EncodedFieldType::NDARRAY: return decode_ndarray(td, field.ndarray(), input, data_sink, bv, encoding_version) + magic_size; default: - util::raise_error_msg("Unsupported encoding {}", field); + util::raise_rte("Unsupported encoding {}", field); } } diff --git a/cpp/arcticdb/codec/codec.cpp b/cpp/arcticdb/codec/codec.cpp index 2c21c6ad34..8ed544f0d9 100644 --- a/cpp/arcticdb/codec/codec.cpp +++ b/cpp/arcticdb/codec/codec.cpp @@ -8,42 +8,26 @@ #include #include #include +#include #include #include #include +#include +#include +#include - -#include #include - -#include - +#include namespace arcticdb { -Segment encode_v2( - SegmentInMemory&& in_mem_seg, - const arcticdb::proto::encoding::VariantCodec& codec_opts -); - -Segment encode_v1( - SegmentInMemory&& in_mem_seg, - const arcticdb::proto::encoding::VariantCodec& codec_opts -); - constexpr TypeDescriptor metadata_type_desc() { return TypeDescriptor{ DataType::UINT8, Dimension::Dim1 }; } -constexpr TypeDescriptor encoded_blocks_type_desc() { - return TypeDescriptor{ - DataType::UINT8, Dimension::Dim1 - }; -} - SizeResult max_compressed_size_dispatch( const SegmentInMemory& in_mem_seg, const arcticdb::proto::encoding::VariantCodec &codec_opts, @@ -106,15 +90,17 @@ class MetaBuffer { } std::optional decode_metadata( - const arcticdb::proto::encoding::SegmentHeader& hdr, + const SegmentHeader& hdr, const uint8_t*& data, const uint8_t* begin ARCTICDB_UNUSED ) { if (hdr.has_metadata_field()) { + hdr.metadata_field().validate(); auto meta_type_desc = metadata_type_desc(); MetaBuffer meta_buf; std::optional bv; - data += decode_field(meta_type_desc, hdr.metadata_field(), data, meta_buf, bv, to_encoding_version(hdr.encoding_version())); + ARCTICDB_DEBUG(log::codec(), "Decoding metadata at position {}: {}", data - begin, dump_bytes(data, 10)); + data += decode_ndarray(meta_type_desc, hdr.metadata_field().ndarray(), data, meta_buf, bv, hdr.encoding_version()); ARCTICDB_TRACE(log::codec(), "Decoded metadata to position {}", data - begin); google::protobuf::io::ArrayInputStream ais(meta_buf.buffer().data(), static_cast(meta_buf.buffer().bytes())); @@ -128,13 +114,17 @@ std::optional decode_metadata( } void decode_metadata( - const arcticdb::proto::encoding::SegmentHeader& hdr, + const SegmentHeader& hdr, const uint8_t*& data, const uint8_t* begin ARCTICDB_UNUSED, SegmentInMemory& res) { auto maybe_any = decode_metadata(hdr, data, begin); - if(maybe_any) + if(maybe_any) { + log::version().info("Found metadata on segment"); res.set_metadata(std::move(*maybe_any)); + } else { + log::version().info("No metadata on segment"); + } } std::optional decode_metadata_from_segment(const Segment &segment) { @@ -148,44 +138,68 @@ std::optional decode_metadata_from_segment(const Segment return decode_metadata(hdr, data, begin); } -Buffer decode_encoded_fields( - const arcticdb::proto::encoding::SegmentHeader& hdr, - const uint8_t* data, - const uint8_t* begin ARCTICDB_UNUSED) { - ARCTICDB_TRACE(log::codec(), "Decoding encoded fields"); - MetaBuffer meta_buffer; - std::optional bv; - if(hdr.has_column_fields()) { - constexpr auto type_desc = encoded_blocks_type_desc(); - decode_field(type_desc, hdr.column_fields(), data, meta_buffer, bv, to_encoding_version(hdr.encoding_version())); - } - ARCTICDB_TRACE(log::codec(), "Decoded encoded fields at position {}", data-begin); - return meta_buffer.detach_buffer(); +EncodedFieldCollection decode_encoded_fields( + const SegmentHeader& hdr, + const uint8_t* data, + const uint8_t* begin ARCTICDB_UNUSED) { + ARCTICDB_TRACE(log::codec(), "Decoding encoded fields"); + + util::check(hdr.has_column_fields() && hdr.column_fields().has_ndarray(), "Expected encoded field description to be set in header"); + std::optional bv; + const auto uncompressed_size = encoding_sizes::uncompressed_size(hdr.column_fields()); + constexpr auto type_desc = encoded_fields_type_desc(); + Column encoded_column(type_desc, uncompressed_size, false, false); + decode_ndarray(type_desc, hdr.column_fields().ndarray(), data, encoded_column, bv, hdr.encoding_version()); + + ARCTICDB_TRACE(log::codec(), "Decoded encoded fields at position {}", data-begin); + return {std::move(encoded_column.release_buffer()), std::move(encoded_column.release_shapes())}; +} + +std::shared_ptr extract_frame_metadata( + SegmentInMemory& res) { + auto output = std::make_shared(); + util::check(res.has_metadata(), "Cannot extract frame metadata as it is null"); + res.metadata()->UnpackTo(output.get()); + return output; +} + +FrameDescriptorImpl read_frame_descriptor( + const uint8_t*& data) { + auto* frame_descriptor = reinterpret_cast(data); + data += sizeof(FrameDescriptorImpl); + return *frame_descriptor; +} + +SegmentDescriptorImpl read_segment_descriptor( + const uint8_t*& data) { + util::check_magic(data); + auto* frame_descriptor = reinterpret_cast(data); + data += sizeof(SegmentDescriptorImpl); + return *frame_descriptor; } -std::optional decode_index_fields( - const arcticdb::proto::encoding::SegmentHeader& hdr, +std::shared_ptr decode_index_fields( + const SegmentHeader& hdr, const uint8_t*& data, const uint8_t* begin ARCTICDB_UNUSED, const uint8_t* end) { - //TODO append map sets the field but it's empty + auto fields = std::make_shared(); if(hdr.has_index_descriptor_field() && hdr.index_descriptor_field().has_ndarray()) { ARCTICDB_TRACE(log::codec(), "Decoding index fields"); util::check(data!=end, "Reached end of input block with index descriptor fields to decode"); std::optional bv; - FieldCollection fields; - data += decode_field(FieldCollection::type(), - hdr.index_descriptor_field(), + + data += decode_ndarray(FieldCollection::type(), + hdr.index_descriptor_field().ndarray(), data, - fields, + *fields, bv, - to_encoding_version(hdr.encoding_version())); + hdr.encoding_version()); ARCTICDB_TRACE(log::codec(), "Decoded index descriptor to position {}", data-begin); - return std::make_optional(std::move(fields)); - } else { - return std::nullopt; } + fields->regenerate_offsets(); + return fields; } namespace { @@ -194,10 +208,16 @@ inline arcticdb::proto::descriptors::TimeSeriesDescriptor timeseries_descriptor_ any.UnpackTo(&tsd); return tsd; } + +inline arcticdb::proto::descriptors::FrameMetadata frame_metadata_from_any(const google::protobuf::Any& any) { + arcticdb::proto::descriptors::FrameMetadata frame_meta; + any.UnpackTo(&frame_meta); + return frame_meta; +} } std::optional decode_descriptor_fields( - const arcticdb::proto::encoding::SegmentHeader& hdr, + const SegmentHeader& hdr, const uint8_t*& data, const uint8_t* begin ARCTICDB_UNUSED, const uint8_t* end) { @@ -211,7 +231,7 @@ std::optional decode_descriptor_fields( data, fields, bv, - to_encoding_version(hdr.encoding_version())); + hdr.encoding_version()); ARCTICDB_TRACE(log::codec(), "Decoded descriptor fields to position {}", data-begin); return std::make_optional(std::move(fields)); @@ -220,44 +240,74 @@ std::optional decode_descriptor_fields( } } -std::optional> decode_timeseries_descriptor( - const arcticdb::proto::encoding::SegmentHeader& hdr, +TimeseriesDescriptor unpack_timeseries_descriptor_from_proto( + const google::protobuf::Any& any) { + + auto tsd = timeseries_descriptor_from_any(any); + auto frame_meta = std::make_shared(); + exchange_timeseries_proto(tsd, *frame_meta); + + auto segment_desc = std::make_shared(segment_descriptor_from_proto((tsd.stream_descriptor()))); + auto frame_desc = std::make_shared(frame_descriptor_from_proto(tsd)); + const auto& desc = tsd.stream_descriptor(); + auto old_fields = std::make_shared(fields_from_proto(tsd.stream_descriptor())); + StreamId stream_id = desc.id_case() == desc.kNumId ? StreamId(desc.num_id()) : StreamId(desc.str_id()); + return {frame_desc, segment_desc, frame_meta, old_fields, stream_id}; +} + +std::optional decode_timeseries_descriptor_v1( + const SegmentHeader& hdr, + const uint8_t* data, + const uint8_t* begin, + const uint8_t* end ARCTICDB_UNUSED) { + auto maybe_any = decode_metadata(hdr, data, begin); + if(!maybe_any) + return std::nullopt; + + return unpack_timeseries_descriptor_from_proto(*maybe_any); +} + +std::optional decode_timeseries_descriptor_v2( + const SegmentHeader& hdr, const uint8_t* data, const uint8_t* begin, const uint8_t* end) { - util::check(data != nullptr, "Got null data ptr from segment"); - const auto has_magic_numbers = EncodingVersion(hdr.encoding_version()) == EncodingVersion::V2; - if(has_magic_numbers) util::check_magic(data); auto maybe_any = decode_metadata(hdr, data, begin); if(!maybe_any) return std::nullopt; - auto tsd = timeseries_descriptor_from_any(*maybe_any); - - if(has_magic_numbers) - util::check_magic(data); + auto frame_meta = std::make_shared(frame_metadata_from_any(*maybe_any)); + util::check_magic(data); if(hdr.has_descriptor_field() && hdr.descriptor_field().has_ndarray()) data += encoding_sizes::ndarray_field_compressed_size(hdr.descriptor_field().ndarray()); - if(has_magic_numbers) - util::check_magic(data); - - auto maybe_fields = decode_index_fields(hdr, data, begin, end); - if(!maybe_fields) { - auto old_fields = fields_from_proto(tsd.stream_descriptor()); - return std::make_optional(std::make_tuple(std::move(*maybe_any), std::move(tsd), std::move(old_fields))); - } + util::check_magic(data); + auto frame_desc = std::make_shared(read_frame_descriptor(data)); + auto segment_desc = std::make_shared(read_segment_descriptor(data)); + auto segment_id = read_identifier(data); + auto index_fields = decode_index_fields(hdr, data, begin, end); + return std::make_optional(frame_desc, segment_desc, frame_meta, std::move(index_fields), segment_id); +} - maybe_fields->regenerate_offsets(); - return std::make_tuple(std::move(*maybe_any), std::move(tsd), std::move(*maybe_fields)); +std::optional decode_timeseries_descriptor( + const SegmentHeader& hdr, + const uint8_t* data, + const uint8_t* begin, + const uint8_t* end) { + util::check(data != nullptr, "Got null data ptr from segment"); + auto encoding_version = EncodingVersion(hdr.encoding_version()); + if (encoding_version == EncodingVersion::V1) + return decode_timeseries_descriptor_v1(hdr, data, begin, end); + else + return decode_timeseries_descriptor_v2(hdr, data, begin, end); } -std::optional> decode_timeseries_descriptor( +std::optional decode_timeseries_descriptor( Segment& segment) { - auto &hdr = segment.header(); + const auto &hdr = segment.header(); const uint8_t* data = segment.buffer().data(); util::check(data != nullptr, "Got null data ptr from segment"); @@ -267,27 +317,6 @@ std::optional, StreamDescriptor> decode_metadata_and_descriptor_fields( - arcticdb::proto::encoding::SegmentHeader& hdr, - const uint8_t* data, - const uint8_t* begin, - const uint8_t* end) { - util::check(data != nullptr, "Got null data ptr from segment"); - if(EncodingVersion(hdr.encoding_version()) == EncodingVersion::V2) - util::check_magic(data); - - auto maybe_any = decode_metadata(hdr, data, begin); - if(EncodingVersion(hdr.encoding_version()) == EncodingVersion::V2) - util::check_magic(data); - - auto maybe_fields = decode_descriptor_fields(hdr, data, begin, end); - if(!maybe_fields) { - auto old_fields = std::make_shared(fields_from_proto(hdr.stream_descriptor())); - return std::make_pair(std::move(maybe_any),StreamDescriptor{std::make_shared(std::move(*hdr.mutable_stream_descriptor())), old_fields}); - } - return std::make_pair(std::move(maybe_any),StreamDescriptor{std::make_shared(std::move(*hdr.mutable_stream_descriptor())), std::make_shared(std::move(*maybe_fields))}); -} - std::pair, StreamDescriptor> decode_metadata_and_descriptor_fields( Segment& segment) { auto &hdr = segment.header(); @@ -295,51 +324,71 @@ std::pair, StreamDescriptor> decode_metadat util::check(data != nullptr, "Got null data ptr from segment"); const uint8_t* begin = data; - const uint8_t* end = data + segment.buffer().bytes(); + if(EncodingVersion(hdr.encoding_version()) == EncodingVersion::V2) + util::check_magic(data); + + auto maybe_any = decode_metadata(hdr, data, begin); + if(EncodingVersion(hdr.encoding_version()) == EncodingVersion::V2) + util::check_magic(data); - return decode_metadata_and_descriptor_fields(hdr, data, begin, end); + return std::make_pair(std::move(maybe_any), segment.descriptor()); } -void decode_string_pool( const arcticdb::proto::encoding::SegmentHeader& hdr, - const uint8_t*& data, - const uint8_t* begin ARCTICDB_UNUSED, - const uint8_t* end, - SegmentInMemory& res) { +void decode_string_pool( + const SegmentHeader& hdr, + const uint8_t*& data, + const uint8_t* begin ARCTICDB_UNUSED, + const uint8_t* end, + SegmentInMemory& res) { if (hdr.has_string_pool_field()) { ARCTICDB_TRACE(log::codec(), "Decoding string pool"); util::check(data!=end, "Reached end of input block with string pool fields to decode"); std::optional bv; - data += decode_field(string_pool_descriptor().type(), + data += decode_ndarray(string_pool_descriptor().type(), hdr.string_pool_field(), data, res.string_pool(), bv, - to_encoding_version(hdr.encoding_version())); + hdr.encoding_version()); ARCTICDB_TRACE(log::codec(), "Decoded string pool to position {}", data-begin); } } -void decode_v2( - const Segment& segment, - arcticdb::proto::encoding::SegmentHeader& hdr, - SegmentInMemory& res, - const StreamDescriptor& desc -) { +void skip_descriptor(const uint8_t*& data, const SegmentHeader& hdr) { + util::check_magic(data); + data += sizeof(SegmentDescriptor); + skip_identifier(data); + util::check(hdr.has_descriptor_field(), "Expected descriptor field in v2 encoding"); + util::check_magic(data); + if(hdr.has_descriptor_field() && hdr.descriptor_field().has_ndarray()) + data += encoding_sizes::field_compressed_size(hdr.descriptor_field()); + +} + +void decode_v2(const Segment& segment, + const SegmentHeader& hdr, + SegmentInMemory& res, + const StreamDescriptor& desc) { ARCTICDB_SAMPLE(DecodeSegment, 0) const auto [begin, end] = get_segment_begin_end(segment, hdr); auto encoded_fields_ptr = end; auto data = begin; util::check_magic(data); decode_metadata(hdr, data, begin, res); - util::check(hdr.has_descriptor_field(), "Expected descriptor field in v2 encoding"); - util::check_magic(data); - if(hdr.has_descriptor_field() && hdr.descriptor_field().has_ndarray()) - data += encoding_sizes::field_compressed_size(hdr.descriptor_field()); + skip_descriptor(data, hdr); util::check_magic(data); - if(auto index_fields = decode_index_fields(hdr, data, begin, end); index_fields) - res.set_index_fields(std::make_shared(std::move(*index_fields))); + if(hdr.has_index_descriptor_field()) { + auto index_frame_descriptor = std::make_shared(read_frame_descriptor(data)); + auto frame_metadata = extract_frame_metadata(res); + auto index_segment_descriptor = std::make_shared(read_segment_descriptor(data)); + auto index_segment_identifier = read_identifier(data); + auto index_fields = decode_index_fields(hdr, data, begin, end); + TimeseriesDescriptor tsd{std::move(index_frame_descriptor), std::move(index_segment_descriptor), std::move(frame_metadata), std::move(index_fields), index_segment_identifier}; + res.set_timeseries_descriptor(tsd); + res.reset_metadata(); + } util::check(hdr.has_column_fields(), "Expected column fields in v2 encoding"); util::check_magic(encoded_fields_ptr); @@ -348,20 +397,23 @@ void decode_v2( const auto fields_size = desc.fields().size(); const auto start_row = res.row_count(); EncodedFieldCollection encoded_fields(std::move(encoded_fields_buffer)); - const auto seg_row_count = fields_size ? ssize_t(encoded_fields.at(0).ndarray().items_count()) : 0L; + auto encoded_field = encoded_fields.begin(); + const auto seg_row_count = fields_size ? ssize_t(encoded_field->ndarray().items_count()) : 0L; res.init_column_map(); for (std::size_t i = 0; i < fields_size; ++i) { - const auto& encoded_field = encoded_fields.at(i); +#ifdef DUMP_BYTES + log::version().debug("{}", dump_bytes(begin, (data - begin) + encoding_sizes::field_compressed_size(*encoded_field), 100u)); +#endif const auto& field_name = desc.fields(i).name(); util::check(data!=end, "Reached end of input block with {} fields to decode", fields_size-i); if(auto col_index = res.column_index(field_name)) { auto& col = res.column(static_cast(*col_index)); - data += decode_field(res.field(*col_index).type(), encoded_field, data, col, col.opt_sparse_map(), to_encoding_version(hdr.encoding_version())); + data += decode_field(res.field(*col_index).type(), *encoded_field, data, col, col.opt_sparse_map(), hdr.encoding_version()); } else { - data += encoding_sizes::field_compressed_size(encoded_field) + sizeof(ColumnMagic); + data += encoding_sizes::field_compressed_size(*encoded_field) + sizeof(ColumnMagic); } - + ++encoded_field; ARCTICDB_TRACE(log::codec(), "Decoded column {} to position {}", i, data-begin); } @@ -373,79 +425,77 @@ void decode_v2( } } -void decode_v1( - const Segment& segment, - const arcticdb::proto::encoding::SegmentHeader& hdr, - SegmentInMemory& res, - StreamDescriptor::Proto& desc -) { +void decode_v1(const Segment& segment, + const SegmentHeader& hdr, + SegmentInMemory& res, + const StreamDescriptor& desc) { ARCTICDB_SAMPLE(DecodeSegment, 0) const uint8_t* data = segment.buffer().data(); util::check(data != nullptr, "Got null data ptr from segment"); const uint8_t* begin = data; const uint8_t* end = begin + segment.buffer().bytes(); decode_metadata(hdr, data, begin, res); + if(res.has_metadata() && res.metadata()->Is()) { + log::version().info("Unpacking timeseries descriptor from metadata"); + auto tsd = unpack_timeseries_descriptor_from_proto(*res.metadata()); + res.set_timeseries_descriptor(tsd); + res.reset_metadata(); + } - if (data!=end) { + if (data != end) { const auto fields_size = desc.fields().size(); - util::check(fields_size == hdr.fields_size(), "Mismatch between descriptor and header field size: {} != {}", fields_size, hdr.fields_size()); - const auto start_row = res.row_count(); - - const auto seg_row_count = fields_size ? ssize_t(hdr.fields(0).ndarray().items_count()) : 0LL; + const auto &column_fields = hdr.body_fields(); + util::check(fields_size == segment.fields_size(), + "Mismatch between descriptor and header field size: {} != {}", + fields_size, + column_fields.size()); + const auto start_row = res.row_count(); + const auto seg_row_count = fields_size ? ssize_t(column_fields.at(0).ndarray().items_count()) : 0LL; res.init_column_map(); - for (int i = 0; i < fields_size; ++i) { - const auto& field = hdr.fields(i); - const auto& field_name = desc.fields(i).name(); + for (std::size_t i = 0; i < fields_size; ++i) { + const auto &field = column_fields.at(i); + const auto &field_name = desc.fields(i).name(); + util::check(data != end, "Reached end of input block with {} fields to decode", fields_size - i); if (auto col_index = res.column_index(field_name)) { - auto& col = res.column(static_cast(*col_index)); - util::check( - data != end || is_empty_type(col.type().data_type()), - "Reached end of input block with {} fields to decode", - fields_size - i - ); + auto &col = res.column(static_cast(*col_index)); data += decode_field( res.field(*col_index).type(), field, data, col, col.opt_sparse_map(), - to_encoding_version(hdr.encoding_version()) + hdr.encoding_version() ); + ARCTICDB_DEBUG(log::codec(), "Decoded column {} to position {}", i, data - begin); } else { - util::check(data != end, "Reached end of input block with {} fields to decode", fields_size - i); data += encoding_sizes::field_compressed_size(field); + ARCTICDB_DEBUG(log::codec(), "Skipped column {}, at position {}", i, data - begin); } - ARCTICDB_TRACE(log::codec(), "Decoded column {} to position {}", i, data - begin); } - decode_string_pool(hdr, data, begin, end, res); - res.set_row_data(static_cast(start_row + seg_row_count-1)); + res.set_row_data(static_cast(start_row + seg_row_count - 1)); res.set_compacted(segment.header().compacted()); } } void decode_into_memory_segment( const Segment& segment, - arcticdb::proto::encoding::SegmentHeader& hdr, + SegmentHeader& hdr, SegmentInMemory& res, - StreamDescriptor& desc) + const StreamDescriptor& desc) { if(EncodingVersion(segment.header().encoding_version()) == EncodingVersion::V2) decode_v2(segment, hdr, res, desc); else - decode_v1(segment, hdr, res, desc.mutable_proto()); + decode_v1(segment, hdr, res, desc); } SegmentInMemory decode_segment(Segment&& s) { auto segment = std::move(s); auto &hdr = segment.header(); - ARCTICDB_TRACE(log::codec(), "Decoding descriptor: {}", segment.header().stream_descriptor().DebugString()); - StreamDescriptor descriptor(std::make_shared(std::move(*segment.header().mutable_stream_descriptor())), segment.fields_ptr()); - - if(EncodingVersion(segment.header().encoding_version()) != EncodingVersion::V2) - descriptor.fields() = field_collection_from_proto(std::move(*descriptor.mutable_proto().mutable_fields())); - + ARCTICDB_TRACE(log::codec(), "Decoding descriptor: {}", segment.descriptor()); + auto descriptor = segment.descriptor(); descriptor.fields().regenerate_offsets(); ARCTICDB_TRACE(log::codec(), "Creating segment"); SegmentInMemory res(std::move(descriptor)); @@ -455,7 +505,7 @@ SegmentInMemory decode_segment(Segment&& s) { return res; } -static void hash_field(const arcticdb::proto::encoding::EncodedField &field, HashAccum &accum) { +static void hash_field(const EncodedFieldImpl &field, HashAccum &accum) { auto &n = field.ndarray(); for(auto i = 0; i < n.shapes_size(); ++i) { auto v = n.shapes(i).hash(); @@ -468,17 +518,15 @@ static void hash_field(const arcticdb::proto::encoding::EncodedField &field, Has } } -HashedValue hash_segment_header(const arcticdb::proto::encoding::SegmentHeader &hdr) { +HashedValue hash_segment_header(const SegmentHeader &hdr) { HashAccum accum; if (hdr.has_metadata_field()) { hash_field(hdr.metadata_field(), accum); } - for (int i = 0; i < hdr.fields_size(); ++i) { - hash_field(hdr.fields(i), accum); - } if(hdr.has_string_pool_field()) { hash_field(hdr.string_pool_field(), accum); } + return accum.digest(); } @@ -518,7 +566,7 @@ void add_bitmagic_compressed_size( void encode_sparse_map( ColumnData& column_data, - std::variant variant_field, + EncodedFieldImpl& field, Buffer& out, std::ptrdiff_t& pos ) { @@ -526,9 +574,7 @@ void encode_sparse_map( util::check(!is_empty_type(column_data.type().data_type()), "Empty typed columns should not have sparse maps"); ARCTICDB_DEBUG(log::codec(), "Sparse map count = {} pos = {}", column_data.bit_vector()->count(), pos); const size_t sparse_bm_bytes = encode_bitmap(*column_data.bit_vector(), out, pos); - util::variant_match(variant_field, [sparse_bm_bytes](auto field) { - field->mutable_ndarray()->set_sparse_map_bytes(static_cast(sparse_bm_bytes)); - }); + field.mutable_ndarray()->set_sparse_map_bytes(static_cast(sparse_bm_bytes)); } } } // namespace arcticdb diff --git a/cpp/arcticdb/codec/codec.hpp b/cpp/arcticdb/codec/codec.hpp index 28bc60c806..e157bb99dc 100644 --- a/cpp/arcticdb/codec/codec.hpp +++ b/cpp/arcticdb/codec/codec.hpp @@ -8,28 +8,51 @@ #pragma once #include -#include -#include #include #include +#include +#include +#include namespace arcticdb { -using ShapesBlockTDT = TypeDescriptorTag, DimensionTag>; +class Segment; +class SegmentInMemory; + +using ShapesBlockTDT = entity::TypeDescriptorTag, entity::DimensionTag>; Segment encode_dispatch( SegmentInMemory&& in_mem_seg, const arcticdb::proto::encoding::VariantCodec &codec_opts, EncodingVersion encoding_version); +Segment encode_v2( + SegmentInMemory&& in_mem_seg, + const arcticdb::proto::encoding::VariantCodec& codec_opts +); + +Segment encode_v1( + SegmentInMemory&& in_mem_seg, + const arcticdb::proto::encoding::VariantCodec& codec_opts +); + +void decode_v1(const Segment& segment, + const SegmentHeader& hdr, + SegmentInMemory& res, + const StreamDescriptor& desc); + +void decode_v2(const Segment& segment, + const SegmentHeader& hdr, + SegmentInMemory& res, + const StreamDescriptor& desc); SizeResult max_compressed_size_dispatch( const SegmentInMemory& in_mem_seg, const arcticdb::proto::encoding::VariantCodec &codec_opts, EncodingVersion encoding_version); -Buffer decode_encoded_fields( - const arcticdb::proto::encoding::SegmentHeader& hdr, +EncodedFieldCollection decode_encoded_fields( + const SegmentHeader& hdr, const uint8_t* data, const uint8_t* begin ARCTICDB_UNUSED); @@ -37,14 +60,14 @@ SegmentInMemory decode_segment(Segment&& segment); void decode_into_memory_segment( const Segment& segment, - arcticdb::proto::encoding::SegmentHeader& hdr, + SegmentHeader& hdr, SegmentInMemory& res, - StreamDescriptor& desc); + const entity::StreamDescriptor& desc); -template +template std::size_t decode_field( - const TypeDescriptor &td, - const EncodedFieldType &field, + const entity::TypeDescriptor &td, + const EncodedFieldImpl &field, const uint8_t *input, DataSink &data_sink, std::optional& bv, @@ -56,10 +79,15 @@ std::optional decode_metadata_from_segment( std::pair, StreamDescriptor> decode_metadata_and_descriptor_fields( Segment& segment); -std::optional> decode_timeseries_descriptor( +std::optional decode_timeseries_descriptor( Segment& segment); -HashedValue hash_segment_header(const arcticdb::proto::encoding::SegmentHeader &hdr); +HashedValue hash_segment_header(const SegmentHeader &hdr); + +SegmentDescriptorImpl read_segment_descriptor(const uint8_t*& data); + +TimeseriesDescriptor unpack_timeseries_descriptor_from_proto(const google::protobuf::Any& any); + } // namespace arcticdb #define ARCTICDB_SEGMENT_ENCODER_H_ diff --git a/cpp/arcticdb/codec/core.hpp b/cpp/arcticdb/codec/core.hpp index 5462c95300..d640908173 100644 --- a/cpp/arcticdb/codec/core.hpp +++ b/cpp/arcticdb/codec/core.hpp @@ -14,13 +14,14 @@ #include #include #include +#include #include namespace arcticdb::detail { using namespace arcticdb::entity; -struct BlockProtobufHelper { +struct BlockDataHelper { std::size_t count_; std::size_t bytes_; @@ -39,8 +40,8 @@ struct BlockProtobufHelper { struct NdArrayBlock { std::size_t item_count_; - BlockProtobufHelper shapes_; - BlockProtobufHelper values_; + BlockDataHelper shapes_; + BlockDataHelper values_; template void update_field_size(EncodedFieldType &field) const { @@ -103,12 +104,12 @@ class CodecHelper { return v; } - static BlockProtobufHelper scalar_block(std::size_t elt) { - return {elt, elt * sizeof(T)}; + static BlockDataHelper scalar_block(std::size_t row_count) { + return {row_count, row_count * sizeof(T)}; } - static NdArrayBlock nd_array_block(std::size_t elt, const shape_t *shape) { - std::size_t shape_count = static_cast(dim) * elt; + static NdArrayBlock nd_array_block(std::size_t row_count, const shape_t *shape) { + std::size_t shape_count = static_cast(dim) * row_count; std::size_t total_values_count = 0; if constexpr (dim == Dimension::Dim1) { for (std::size_t i = 0; i < shape_count; ++i, ++shape) { @@ -122,7 +123,7 @@ class CodecHelper { } std::size_t shape_bytes = shape_count * sizeof(shape_t); std::size_t data_bytes = total_values_count * sizeof(T); - return NdArrayBlock{elt, {shape_count, shape_bytes}, {total_values_count, data_bytes}}; + return NdArrayBlock{row_count, {shape_count, shape_bytes}, {total_values_count, data_bytes}}; } }; @@ -145,7 +146,7 @@ struct ShapeEncodingFromBlock { template static std::size_t encode_block( const T *in, - BlockProtobufHelper &block_utils, + BlockDataHelper &block_utils, HashAccum &hasher, T *out, std::size_t out_capacity, @@ -214,11 +215,10 @@ struct GenericBlockEncoder { } } - template static void encode( const typename EncoderType::Opts &opts, const BlockType& block, - EncodedFieldType& field, + EncodedFieldImpl& field, Buffer& out, std::ptrdiff_t& pos ) { @@ -245,7 +245,7 @@ struct GenericBlockEncoder { auto t_out = reinterpret_cast(out.data() + pos); const auto total_items_count = field_nd_array->items_count() + block_row_count; field_nd_array->set_items_count(total_items_count); - auto value_pb = field_nd_array->add_values(); + auto value_pb = field_nd_array->add_values(EncodingVersion::V1); const auto compressed_size = EncoderType::encode_block(opts, block.data(), helper_scalar_block, @@ -254,6 +254,7 @@ struct GenericBlockEncoder { max_compressed_size, pos, *value_pb->mutable_codec()); + helper_scalar_block.set_block_data(*value_pb, helper.hasher_.digest(), compressed_size); helper_scalar_block.set_version(*value_pb, EncoderType::VERSION); } else { @@ -281,7 +282,7 @@ struct GenericBlockEncoder { HashedValue shape_hash = helper.get_digest_and_reset(); // write values - auto value_pb = field_nd_array->add_values(); + auto value_pb = field_nd_array->add_values(EncodingVersion::V1); auto t_out = reinterpret_cast(out.data() + pos); const auto values_comp_size = EncoderType::encode_block( opts, @@ -339,7 +340,7 @@ struct GenericBlockEncoderV2 { Helper helper; helper.hasher_.reset(helper.seed); - auto helper_scalar_block = BlockProtobufHelper{block.nbytes() / sizeof(T), block.nbytes()}; + auto helper_scalar_block = BlockDataHelper{block.nbytes() / sizeof(T), block.nbytes()}; ARCTICDB_TRACE(log::codec(), "Generic block encode writing scalar of {} elements", block.row_count()); const auto uncompressed_size = helper_scalar_block.bytes_; diff --git a/cpp/arcticdb/codec/encode_common.hpp b/cpp/arcticdb/codec/encode_common.hpp index 10e25479a1..f2c17cba74 100644 --- a/cpp/arcticdb/codec/encode_common.hpp +++ b/cpp/arcticdb/codec/encode_common.hpp @@ -33,6 +33,14 @@ struct EncodingPolicyType { using ColumnEncoder = VersionedColumnEncoder; }; +template +size_t calc_num_blocks(const ColumnData& column_data) { + if constexpr (EncodingPolicyType::version == EncodingVersion::V1) + return column_data.num_blocks() + (column_data.num_blocks() * !column_data.shapes()->empty()); + else + return column_data.num_blocks() + !column_data.shapes()->empty(); +} + template struct BytesEncoder { using Encoder = TypedBlockEncoderImpl; @@ -45,7 +53,7 @@ struct BytesEncoder { const arcticdb::proto::encoding::VariantCodec &codec_opts, Buffer &out_buffer, std::ptrdiff_t &pos, - EncodedFieldType *encoded_field + EncodedFieldType& encoded_field ) { if constexpr (EncodingPolicyType::version == EncodingVersion::V1) { const auto bytes_count = static_cast(data.bytes()); @@ -55,11 +63,9 @@ struct BytesEncoder { bytes_count, 1u, data.block_and_offset(0).block_); - Encoder::encode(codec_opts, typed_block, *encoded_field, out_buffer, pos); + Encoder::encode(codec_opts, typed_block, encoded_field, out_buffer, pos); } else if constexpr (EncodingPolicyType::version == EncodingVersion::V2) { - // On Man's Mac build servers size_t and ssize_t are long rather than long long but the shape TDT - // expects int64 (long long). - const size_t row_count = 1; + const shape_t row_count = 1; //TOD this seems weird and wrong. Why is it hardcoded to 1? const auto shapes_data = static_cast(data.bytes()); auto shapes_block = TypedBlockData(&shapes_data, nullptr, @@ -72,18 +78,18 @@ struct BytesEncoder { static_cast(bytes_count), row_count, data.block_and_offset(0).block_); - ShapesEncoder::encode_shapes(codec::default_shapes_codec(), shapes_block, *encoded_field, out_buffer, pos); - Encoder::encode_values(codec_opts, data_block, *encoded_field, out_buffer, pos); - auto *field_nd_array = encoded_field->mutable_ndarray(); + ShapesEncoder::encode_shapes(codec::default_shapes_codec(), shapes_block, encoded_field, out_buffer, pos); + Encoder::encode_values(codec_opts, data_block, encoded_field, out_buffer, pos); + auto *field_nd_array = encoded_field.mutable_ndarray(); const auto total_items_count = field_nd_array->items_count() + row_count; field_nd_array->set_items_count(total_items_count); } else { static_assert(std::is_same_v, "Unknown encoding version"); } + encoded_field.validate(); } - static size_t - max_compressed_size(const arcticdb::proto::encoding::VariantCodec &codec_opts, shape_t data_size) { + static size_t max_compressed_size(const arcticdb::proto::encoding::VariantCodec &codec_opts, shape_t data_size) { const shape_t shapes_bytes = sizeof(shape_t); const auto values_block = BytesBlock(data_size, &data_size); if constexpr (EncodingPolicyType::version == EncodingVersion::V1) { @@ -98,6 +104,10 @@ struct BytesEncoder { static_assert(std::is_same_v, "Unknown encoding version"); } } + + static size_t num_encoded_blocks(const ChunkedBuffer& buffer) { + return buffer.num_blocks() + 1; + } }; struct SizeResult { @@ -108,10 +118,9 @@ struct SizeResult { template void calc_metadata_size( - const SegmentInMemory &in_mem_seg, - const arcticdb::proto::encoding::VariantCodec &codec_opts, - SizeResult &result -) { + const SegmentInMemory &in_mem_seg, + const arcticdb::proto::encoding::VariantCodec &codec_opts, + SizeResult &result) { if (in_mem_seg.metadata()) { const auto metadata_bytes = static_cast(in_mem_seg.metadata()->ByteSizeLong()); result.uncompressed_bytes_ += metadata_bytes + sizeof(shape_t); @@ -162,30 +171,32 @@ void calc_string_pool_size( template void encode_metadata( - const SegmentInMemory &in_mem_seg, - arcticdb::proto::encoding::SegmentHeader &segment_header, - const arcticdb::proto::encoding::VariantCodec &codec_opts, - Buffer &out_buffer, - std::ptrdiff_t &pos -) { + const SegmentInMemory& in_mem_seg, + SegmentHeader& segment_header, + const arcticdb::proto::encoding::VariantCodec& codec_opts, + Buffer &out_buffer, + std::ptrdiff_t& pos) { if (in_mem_seg.metadata()) { const auto bytes_count = static_cast(in_mem_seg.metadata()->ByteSizeLong()); - ARCTICDB_TRACE(log::codec(), "Encoding {} bytes of metadata", bytes_count); - auto encoded_field = segment_header.mutable_metadata_field(); + log::codec().info("Encoding {} bytes of metadata", bytes_count); + constexpr int max_stack_alloc = 1 << 11; bool malloced{false}; - uint8_t *meta_ptr{nullptr}; + uint8_t* meta_ptr; if (bytes_count > max_stack_alloc) { - meta_ptr = reinterpret_cast(malloc(bytes_count)); + meta_ptr = reinterpret_cast(malloc(bytes_count)); malloced = true; } else { - meta_ptr = reinterpret_cast(alloca(bytes_count)); + meta_ptr = reinterpret_cast(alloca(bytes_count)); } ChunkedBuffer meta_buffer; meta_buffer.add_external_block(meta_ptr, bytes_count, 0u); + const auto num_encoded_fields = BytesEncoder::num_encoded_blocks(meta_buffer); + auto& encoded_field = segment_header.mutable_metadata_field(num_encoded_fields); google::protobuf::io::ArrayOutputStream aos(&meta_buffer[0], static_cast(bytes_count)); in_mem_seg.metadata()->SerializeToZeroCopyStream(&aos); + ARCTICDB_DEBUG(log::codec(), "Encoding metadata to position {}", pos); BytesEncoder::encode(meta_buffer, codec_opts, out_buffer, pos, encoded_field); ARCTICDB_DEBUG(log::codec(), "Encoded metadata to position {}", pos); if (malloced) @@ -198,17 +209,17 @@ void encode_metadata( template void encode_string_pool( const SegmentInMemory &in_mem_seg, - arcticdb::proto::encoding::SegmentHeader &segment_header, + SegmentHeader &segment_header, const arcticdb::proto::encoding::VariantCodec &codec_opts, Buffer &out_buffer, std::ptrdiff_t &pos ) { if (in_mem_seg.has_string_pool()) { ARCTICDB_TRACE(log::codec(), "Encoding string pool to position {}", pos); - auto *encoded_field = segment_header.mutable_string_pool_field(); auto col = in_mem_seg.string_pool_data(); + auto& encoded_field = segment_header.mutable_string_pool_field(calc_num_blocks(col)); EncodingPolicyType::ColumnEncoder::encode(codec_opts, col, encoded_field, out_buffer, pos); - ARCTICDB_TRACE(log::codec(), "Encoded string pool to position {}", pos); + ARCTICDB_DEBUG(log::codec(), "Encoded string pool to position {}", pos); } } diff --git a/cpp/arcticdb/codec/encode_v1.cpp b/cpp/arcticdb/codec/encode_v1.cpp index bd057bd8b7..70a5ab71db 100644 --- a/cpp/arcticdb/codec/encode_v1.cpp +++ b/cpp/arcticdb/codec/encode_v1.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include namespace arcticdb { @@ -18,7 +19,7 @@ namespace arcticdb { void encode_sparse_map( ColumnData& column_data, - std::variant variant_field, + EncodedFieldImpl& variant_field, Buffer& out, std::ptrdiff_t& pos ); @@ -32,7 +33,7 @@ namespace arcticdb { static void encode( const arcticdb::proto::encoding::VariantCodec &codec_opts, ColumnData& column_data, - std::variant variant_field, + EncodedFieldImpl& variant_field, Buffer& out, std::ptrdiff_t& pos); }; @@ -58,7 +59,7 @@ namespace arcticdb { // might be non-zero. max_compressed_bytes += Encoder::max_compressed_size(codec_opts, *block); } - add_bitmagic_compressed_size(column_data, uncompressed_bytes, max_compressed_bytes); + add_bitmagic_compressed_size(column_data, max_compressed_bytes, uncompressed_bytes); return std::make_pair(uncompressed_bytes, max_compressed_bytes); }); } @@ -66,11 +67,11 @@ namespace arcticdb { void ColumnEncoderV1::encode( const arcticdb::proto::encoding::VariantCodec& codec_opts, ColumnData& column_data, - std::variant variant_field, + EncodedFieldImpl& field, Buffer& out, std::ptrdiff_t& pos ) { - column_data.type().visit_tag([&](auto type_desc_tag) { + column_data.type().visit_tag([&codec_opts, &column_data, &field, &out, &pos](auto type_desc_tag) { using TDT = decltype(type_desc_tag); using Encoder = TypedBlockEncoderImpl; ARCTICDB_TRACE(log::codec(), "Column data has {} blocks", column_data.num_blocks()); @@ -78,12 +79,10 @@ namespace arcticdb { if constexpr(must_contain_data(static_cast(type_desc_tag))) { util::check(block.value().nbytes() > 0, "Zero-sized block"); } - std::visit([&](auto field){ - Encoder::encode(codec_opts, block.value(), *field, out, pos); - }, variant_field); + Encoder::encode(codec_opts, block.value(), field, out, pos); } }); - encode_sparse_map(column_data, variant_field, out, pos); + encode_sparse_map(column_data, field, out, pos); } using EncodingPolicyV1 = EncodingPolicyType; @@ -111,53 +110,58 @@ namespace arcticdb { */ ARCTICDB_SAMPLE(EncodeSegment, 0) auto in_mem_seg = std::move(s); - auto arena = std::make_unique(); - auto segment_header = google::protobuf::Arena::CreateMessage(arena.get()); - *segment_header->mutable_stream_descriptor() = in_mem_seg.descriptor().copy_to_proto(); - segment_header->set_compacted(in_mem_seg.compacted()); + SegmentHeader segment_header{EncodingVersion::V1}; + segment_header.set_compacted(in_mem_seg.compacted()); + + if(in_mem_seg.has_index_descriptor()) { + log::version().info("Memory segment has index descriptor, encoding to protobuf"); + util::check(!in_mem_seg.has_metadata(), "Metadata already set when trying to set index descriptor"); + auto proto = copy_time_series_descriptor_to_proto(in_mem_seg.index_descriptor()); + google::protobuf::Any any; + any.PackFrom(proto); + in_mem_seg.set_metadata(std::move(any)); + } else { + log::version().info("Memory segment has no index descriptor"); + } + std::ptrdiff_t pos = 0; static auto block_to_header_ratio = ConfigsMap::instance()->get_int("Codec.EstimatedHeaderRatio", 75); const auto preamble = in_mem_seg.num_blocks() * block_to_header_ratio; - auto [max_compressed_size, uncompressed_size, encoded_blocks_bytes] = max_compressed_size_v1(in_mem_seg, codec_opts); + auto [max_compressed_size, uncompressed_size, encoded_buffer_size] = max_compressed_size_v1(in_mem_seg, codec_opts); ARCTICDB_TRACE(log::codec(), "Estimated max buffer requirement: {}", max_compressed_size); auto out_buffer = std::make_shared(max_compressed_size, preamble); ColumnEncoderV1 encoder; - ARCTICDB_TRACE(log::codec(), "Encoding descriptor: {}", segment_header->stream_descriptor().DebugString()); - auto *tsd = segment_header->mutable_stream_descriptor(); - tsd->set_in_bytes(uncompressed_size); - - encode_metadata(in_mem_seg, *segment_header, codec_opts, *out_buffer, pos); + encode_metadata(in_mem_seg, segment_header, codec_opts, *out_buffer, pos); + ARCTICDB_TRACE(log::codec(), "Encoding descriptor: {}", in_mem_seg.descriptor()); + auto descriptor_data = in_mem_seg.descriptor().data_ptr(); + descriptor_data->uncompressed_bytes_ = uncompressed_size; + EncodedFieldCollection encoded_fields(encoded_buffer_size, in_mem_seg.num_columns()); if(in_mem_seg.row_count() > 0) { ARCTICDB_TRACE(log::codec(), "Encoding fields"); for (std::size_t column_index = 0; column_index < in_mem_seg.num_columns(); ++column_index) { auto column_data = in_mem_seg.column_data(column_index); - auto *encoded_field = segment_header->mutable_fields()->Add(); + auto* column_field = encoded_fields.add_field(column_data.num_blocks()); if(column_data.num_blocks() > 0) { - encoder.encode(codec_opts, column_data, encoded_field, *out_buffer, pos); - ARCTICDB_TRACE(log::codec(), + encoder.encode(codec_opts, column_data, *column_field, *out_buffer, pos); + ARCTICDB_DEBUG(log::codec(), "Encoded column {}: ({}) to position {}", column_index, in_mem_seg.descriptor().fields(column_index).name(), pos); } else { util::check(!must_contain_data(column_data.type()), "Column {} of type {} contains no blocks", column_index, column_data.type()); - auto* ndarray = encoded_field->mutable_ndarray(); + auto* ndarray = column_field->mutable_ndarray(); ndarray->set_items_count(0); } } - encode_string_pool(in_mem_seg, *segment_header, codec_opts, *out_buffer, pos); + encode_string_pool(in_mem_seg, segment_header, codec_opts, *out_buffer, pos); } + segment_header.set_body_fields(EncodedFieldCollection(std::move(encoded_fields))); ARCTICDB_DEBUG(log::codec(), "Setting buffer bytes to {}", pos); out_buffer->set_bytes(pos); - tsd->set_out_bytes(pos); - ARCTICDB_TRACE(log::codec(), "Encoded header: {}", tsd->DebugString()); - if(!segment_header->has_metadata_field()) - ARCTICDB_DEBUG(log::codec(), "No metadata field"); - ARCTICDB_DEBUG(log::codec(), "Block count {} header size {} ratio {}", - in_mem_seg.num_blocks(), segment_header->ByteSizeLong(), - in_mem_seg.num_blocks() ? segment_header->ByteSizeLong() / in_mem_seg.num_blocks() : 0); - return {std::move(arena), segment_header, std::move(out_buffer), in_mem_seg.descriptor().fields_ptr()}; + descriptor_data->compressed_bytes_ = pos; + return {std::move(segment_header), std::move(out_buffer), descriptor_data, in_mem_seg.descriptor().fields_ptr(), in_mem_seg.descriptor().id(), std::nullopt}; } } diff --git a/cpp/arcticdb/codec/encode_v2.cpp b/cpp/arcticdb/codec/encode_v2.cpp index 657e4f7ca7..508e22352a 100644 --- a/cpp/arcticdb/codec/encode_v2.cpp +++ b/cpp/arcticdb/codec/encode_v2.cpp @@ -11,345 +11,386 @@ #include #include #include - +#include namespace arcticdb { - void add_bitmagic_compressed_size( - const ColumnData& column_data, - size_t& max_compressed_bytes, - size_t& uncompressed_bytes - ); +void add_bitmagic_compressed_size( + const ColumnData& column_data, + size_t& max_compressed_bytes, + size_t& uncompressed_bytes +); + +void encode_sparse_map( + ColumnData& column_data, + EncodedFieldImpl& field, + Buffer& out, + std::ptrdiff_t& pos +); + +template +void write_magic(Buffer& buffer, std::ptrdiff_t& pos) { + new (buffer.data() + pos) MagicType{}; + pos += sizeof(MagicType); +} + +void write_frame_data(Buffer& buffer, std::ptrdiff_t& pos, const FrameDescriptor& frame_desc) { + auto ptr = new (buffer.data() + pos) FrameDescriptor{}; + *ptr = frame_desc; + pos += sizeof(FrameDescriptor); +} + +void write_segment_descriptor(Buffer& buffer, std::ptrdiff_t& pos, const SegmentDescriptorImpl& segment_desc) { + auto ptr = new (buffer.data() + pos) SegmentDescriptorImpl{}; + *ptr = segment_desc; + pos += sizeof(SegmentDescriptorImpl); +} - void encode_sparse_map( +/// @brief Utility class used to encode and compute the max encoding size for regular data columns for V2 encoding +/// What differs this from the already existing ColumnEncoder is that ColumnEncoder encodes the shapes of +/// multidimensional data as part of each block. ColumnEncoder2 uses a better strategy and encodes the shapes for the +/// whole column upfront (before all blocks). +/// @note Although ArcticDB did not support multidimensional user data prior to creating ColumnEncoder2 some of the +/// internal data was multidimensional and used ColumnEncoder. More specifically: string pool and metadata. +/// @note This should be used for V2 encoding. V1 encoding can't use it as there is already data written the other +/// way and it will be hard to distinguish both. +struct ColumnEncoderV2 { +public: + static void encode( + const arcticdb::proto::encoding::VariantCodec &codec_opts, ColumnData& column_data, - std::variant variant_field, + EncodedFieldImpl& field, Buffer& out, - std::ptrdiff_t& pos - ); + std::ptrdiff_t& pos); - template - static void write_magic(Buffer& buffer, std::ptrdiff_t& pos) { - new (buffer.data() + pos) MagicType{}; - pos += sizeof(MagicType); - } + static std::pair max_compressed_size( + const arcticdb::proto::encoding::VariantCodec& codec_opts, + ColumnData& column_data); +private: + static void encode_shapes( + const ColumnData& column_data, + EncodedFieldImpl& field, + Buffer& out, + std::ptrdiff_t& pos_in_buffer); - /// @brief Utility class used to encode and compute the max encoding size for regular data columns for V2 encoding - /// What differs this from the already existing ColumnEncoder is that ColumnEncoder encodes the shapes of - /// multidimensional data as part of each block. ColumnEncoder2 uses a better strategy and encodes the shapes for the - /// whole column upfront (before all blocks). - /// @note Although ArcticDB did not support multidimensional user data prior to creating ColumnEncoder2 some of the - /// internal data was multidimensional and used ColumnEncoder. More specifically: string pool and metadata. - /// @note This should be used for V2 encoding. V1 encoding can't use it as there is already data written the other - /// way and it will be hard to distinguish both. - struct ColumnEncoderV2 { - public: - static void encode( - const arcticdb::proto::encoding::VariantCodec &codec_opts, - ColumnData& column_data, - std::variant variant_field, - Buffer& out, - std::ptrdiff_t& pos); - static std::pair max_compressed_size( - const arcticdb::proto::encoding::VariantCodec& codec_opts, - ColumnData& column_data); - private: - static void encode_shapes( - const ColumnData& column_data, - std::variant variant_field, - Buffer& out, - std::ptrdiff_t& pos_in_buffer); - static void encode_blocks( - const arcticdb::proto::encoding::VariantCodec &codec_opts, - ColumnData& column_data, - std::variant variant_field, - Buffer& out, - std::ptrdiff_t& pos); - }; - - - [[nodiscard]] static TypedBlockData create_shapes_typed_block(const ColumnData& column_data) { - static_assert(sizeof(ssize_t) == sizeof(int64_t)); - const size_t row_count = column_data.shapes()->bytes() / sizeof(shape_t); - return {reinterpret_cast(column_data.shapes()->data()), - nullptr, - column_data.shapes()->bytes(), - row_count, - nullptr}; - } + static void encode_blocks( + const arcticdb::proto::encoding::VariantCodec &codec_opts, + ColumnData& column_data, + EncodedFieldImpl& field, + Buffer& out, + std::ptrdiff_t& pos); +}; + + +[[nodiscard]] static TypedBlockData create_shapes_typed_block(const ColumnData& column_data) { + static_assert(sizeof(ssize_t) == sizeof(int64_t)); + const size_t row_count = column_data.shapes()->bytes() / sizeof(shape_t); + return {reinterpret_cast(column_data.shapes()->data()), + nullptr, + column_data.shapes()->bytes(), + row_count, + nullptr}; +} - void ColumnEncoderV2::encode( +void ColumnEncoderV2::encode( const arcticdb::proto::encoding::VariantCodec& codec_opts, ColumnData& column_data, - std::variant variant_field, + EncodedFieldImpl& field, Buffer& out, - std::ptrdiff_t& pos - ) { - ARCTICDB_DEBUG(log::codec(), "Encoding field with codec {}", codec_opts.DebugString()); - encode_shapes(column_data, variant_field, out, pos); - encode_blocks(codec_opts, column_data, variant_field, out, pos); - encode_sparse_map(column_data, variant_field, out, pos); - } + std::ptrdiff_t& pos) { + encode_shapes(column_data, field, out, pos); + encode_blocks(codec_opts, column_data, field, out, pos); + encode_sparse_map(column_data, field, out, pos); + field.validate(); +} - void ColumnEncoderV2::encode_shapes( +void ColumnEncoderV2::encode_shapes( const ColumnData& column_data, - std::variant variant_field, + EncodedFieldImpl& field, Buffer& out, - std::ptrdiff_t& pos_in_buffer - ) { - // There is no need to store the shapes for a column of empty type as they will be all 0. The type handler will - // assign 0 for the shape upon reading. There is one edge case - when we have None in the column, as it should not - // have shape at all (since it's not an array). This is handled by the sparse map. - if(column_data.type().dimension() != Dimension::Dim0 && !is_empty_type(column_data.type().data_type())) { - TypedBlockData shapes_block = create_shapes_typed_block(column_data); - util::variant_match(variant_field, [&](auto field){ - using ShapesEncoder = TypedBlockEncoderImpl; - ShapesEncoder::encode_shapes(codec::default_shapes_codec(), shapes_block, *field, out, pos_in_buffer); - }); - } + std::ptrdiff_t& pos_in_buffer) { + // There is no need to store the shapes for a column of empty type as they will be all 0. The type handler will + // assign 0 for the shape upon reading. There is one edge case - when we have None in the column, as it should not + // have shape at all (since it's not an array). This is handled by the sparse map. + if(column_data.type().dimension() != Dimension::Dim0 && !is_empty_type(column_data.type().data_type())) { + TypedBlockData shapes_block = create_shapes_typed_block(column_data); + using ShapesEncoder = TypedBlockEncoderImpl; + ShapesEncoder::encode_shapes(codec::default_shapes_codec(), shapes_block, field, out, pos_in_buffer); } +} - void ColumnEncoderV2::encode_blocks( +void ColumnEncoderV2::encode_blocks( const arcticdb::proto::encoding::VariantCodec &codec_opts, ColumnData& column_data, - std::variant variant_field, + EncodedFieldImpl& field, Buffer& out, - std::ptrdiff_t& pos - ) { - column_data.type().visit_tag([&](auto type_desc_tag) { - using TDT = decltype(type_desc_tag); - using Encoder = TypedBlockEncoderImpl; - ARCTICDB_TRACE(log::codec(), "Column data has {} blocks", column_data.num_blocks()); - while (auto block = column_data.next()) { - if constexpr(must_contain_data(static_cast(type_desc_tag))) { - util::check(block.value().nbytes() > 0, "Zero-sized block"); - } - util::variant_match(variant_field, [&](auto field) { - Encoder::encode_values(codec_opts, block.value(), *field, out, pos); - }); + std::ptrdiff_t& pos) { + column_data.type().visit_tag([&codec_opts, &column_data, &field, &out, &pos](auto type_desc_tag) { + using TDT = decltype(type_desc_tag); + using Encoder = TypedBlockEncoderImpl; + ARCTICDB_TRACE(log::codec(), "Column data has {} blocks", column_data.num_blocks()); + while (auto block = column_data.next()) { + if constexpr(must_contain_data(static_cast(type_desc_tag))) { + util::check(block->nbytes() > 0, "Zero-sized block"); + Encoder::encode_values(codec_opts, block.value(), field, out, pos); + } else { + if(block->nbytes() > 0) + Encoder::encode_values(codec_opts, block.value(), field, out, pos); } - }); - } + } + }); +} - std::pair ColumnEncoderV2::max_compressed_size( +std::pair ColumnEncoderV2::max_compressed_size( const arcticdb::proto::encoding::VariantCodec& codec_opts, - ColumnData& column_data - ) { - return column_data.type().visit_tag([&codec_opts, &column_data](auto type_desc_tag) { - size_t max_compressed_bytes = 0; - size_t uncompressed_bytes = 0; - using TDT = decltype(type_desc_tag); - using Encoder = TypedBlockEncoderImpl; - using ShapesEncoder = TypedBlockEncoderImpl; - ARCTICDB_TRACE(log::codec(), "Column data has {} blocks", column_data.num_blocks()); - const size_t shapes_byte_count = column_data.shapes()->bytes(); - const TypedBlockData shapes_block = create_shapes_typed_block(column_data); - max_compressed_bytes += ShapesEncoder::max_compressed_size(codec::default_shapes_codec(), shapes_block); - uncompressed_bytes += shapes_byte_count; - while (auto block = column_data.next()) { - const auto nbytes = block.value().nbytes(); - if constexpr(must_contain_data(static_cast(type_desc_tag))) { - util::check(nbytes > 0, "Zero-sized block"); - } + ColumnData& column_data) { + return column_data.type().visit_tag([&codec_opts, &column_data](auto type_desc_tag) { + size_t max_compressed_bytes = 0; + size_t uncompressed_bytes = 0; + using TDT = decltype(type_desc_tag); + using Encoder = TypedBlockEncoderImpl; + using ShapesEncoder = TypedBlockEncoderImpl; + ARCTICDB_TRACE(log::codec(), "Column data has {} blocks", column_data.num_blocks()); + const size_t shapes_byte_count = column_data.shapes()->bytes(); + const TypedBlockData shapes_block = create_shapes_typed_block(column_data); + max_compressed_bytes += ShapesEncoder::max_compressed_size(codec::default_shapes_codec(), shapes_block); + uncompressed_bytes += shapes_byte_count; + while (auto block = column_data.next()) { + const auto nbytes = block.value().nbytes(); + if constexpr(must_contain_data(static_cast(type_desc_tag))) { + util::check(nbytes > 0, "Zero-sized block"); uncompressed_bytes += nbytes; - // For the empty type the column will contain 0 size of user data however the encoder might need add some - // encoder specific data to the buffer, thus the uncompressed size will be 0 but the max_compressed_bytes - // might be non-zero. max_compressed_bytes += Encoder::max_compressed_size(codec_opts, block.value()); + } else { + if(nbytes > 0) { + uncompressed_bytes += nbytes; + max_compressed_bytes += Encoder::max_compressed_size(codec_opts, block.value()); + } } - add_bitmagic_compressed_size(column_data, uncompressed_bytes, max_compressed_bytes); - return std::make_pair(uncompressed_bytes, max_compressed_bytes); - }); - } + } + add_bitmagic_compressed_size(column_data, max_compressed_bytes, uncompressed_bytes); + return std::make_pair(uncompressed_bytes, max_compressed_bytes); + }); +} - using EncodingPolicyV2 = EncodingPolicyType; +using EncodingPolicyV2 = EncodingPolicyType; - static void encode_field_descriptors( +static void encode_field_descriptors( const SegmentInMemory& in_mem_seg, - arcticdb::proto::encoding::SegmentHeader& segment_header, + SegmentHeader& segment_header, const arcticdb::proto::encoding::VariantCodec& codec_opts, Buffer& out_buffer, - std::ptrdiff_t& pos - ) { - ARCTICDB_TRACE(log::codec(), "Encoding field descriptors to position {}", pos); - auto *encoded_field = segment_header.mutable_descriptor_field(); - auto col = in_mem_seg.descriptor().fields().column_data(); - ColumnEncoderV2::encode(codec_opts, col, encoded_field, out_buffer, pos); - ARCTICDB_TRACE(log::codec(), "Encoded field descriptors to position {}", pos); - - write_magic(out_buffer, pos); - if (in_mem_seg.index_fields()) { - ARCTICDB_TRACE(log::codec(), "Encoding index fields descriptors to position {}", pos); - auto *index_field = segment_header.mutable_index_descriptor_field(); - auto index_col = in_mem_seg.index_fields()->column_data(); - ColumnEncoderV2::encode(codec_opts, index_col, index_field, out_buffer, pos); - ARCTICDB_TRACE(log::codec(), "Encoded index field descriptors to position {}", pos); - } + std::ptrdiff_t& pos) { + ARCTICDB_TRACE(log::codec(), "Encoding field descriptors to position {}", pos); + auto col = in_mem_seg.descriptor().fields().column_data(); + auto& encoded_field = segment_header.mutable_descriptor_field(calc_num_blocks(col)); + + ColumnEncoderV2::encode(codec_opts, col, encoded_field, out_buffer, pos); + ARCTICDB_TRACE(log::codec(), "Encoded field descriptors to position {}", pos); +} + +static void encode_index_descriptors( + const SegmentInMemory& in_mem_seg, + SegmentHeader& segment_header, + const arcticdb::proto::encoding::VariantCodec& codec_opts, + Buffer& out_buffer, + std::ptrdiff_t& pos) { + ARCTICDB_TRACE(log::codec(), "Encoding index descriptors to position {}", pos); + + if (in_mem_seg.has_index_descriptor()) { + const auto& tsd = in_mem_seg.index_descriptor(); + write_frame_data(out_buffer, pos, *tsd.frame_data_); + write_magic(out_buffer, pos); + write_segment_descriptor(out_buffer, pos, *tsd.segment_desc_); + write_identifier(out_buffer, pos, tsd.stream_id_); + + ARCTICDB_TRACE(log::codec(), "Encoding index fields descriptors to position {}", pos); + auto index_field_data = tsd.fields().column_data(); + auto& index_field = segment_header.mutable_index_descriptor_field(calc_num_blocks(index_field_data)); + + ColumnEncoderV2::encode(codec_opts, index_field_data, index_field, out_buffer, pos); + ARCTICDB_TRACE(log::codec(), "Encoded index field descriptors to position {}", pos); } +} - [[nodiscard]] size_t calc_column_blocks_size(const Column& col) { - size_t bytes = EncodedField::Size; - if(col.type().dimension() != entity::Dimension::Dim0) - bytes += sizeof(EncodedBlock); +[[nodiscard]] size_t calc_column_blocks_size(const Column& col) { + size_t bytes = EncodedFieldImpl::Size; + if(col.type().dimension() != entity::Dimension::Dim0) + bytes += sizeof(EncodedBlock); - bytes += sizeof(EncodedBlock) * col.num_blocks(); - ARCTICDB_TRACE(log::version(), "Encoded block size: {} + shapes({}) + {} * {} = {}", - EncodedField::Size, - col.type().dimension() != entity::Dimension::Dim0 ? sizeof(EncodedBlock) : 0u, - sizeof(EncodedBlock), - col.num_blocks(), - bytes); + bytes += sizeof(EncodedBlock) * col.num_blocks(); + ARCTICDB_TRACE(log::version(), "Encoded block size: {} + shapes({}) + {} * {} = {}", + EncodedFieldImpl::Size, + col.type().dimension() != entity::Dimension::Dim0 ? sizeof(EncodedBlock) : 0u, + sizeof(EncodedBlock), + col.num_blocks(), + bytes); - return bytes; - } + return bytes; +} - [[nodiscard]] static size_t encoded_blocks_size(const SegmentInMemory& in_mem_seg) { - size_t bytes = 0; - for (std::size_t c = 0; c < in_mem_seg.num_columns(); ++c) { - const auto& col = in_mem_seg.column(position_t(c)); - bytes += calc_column_blocks_size(col); - } - return bytes; +[[nodiscard]] static size_t encoded_blocks_size(const SegmentInMemory& in_mem_seg) { + size_t bytes = 0; + for (std::size_t c = 0; c < in_mem_seg.num_columns(); ++c) { + const auto& col = in_mem_seg.column(position_t(c)); + bytes += calc_column_blocks_size(col); } + bytes += sizeof(EncodedBlock); + return bytes; +} - static void calc_encoded_blocks_size( +static void calc_encoded_blocks_size( const SegmentInMemory& in_mem_seg, const arcticdb::proto::encoding::VariantCodec& codec_opts, - SizeResult& result - ) { - result.encoded_blocks_bytes_ = static_cast(encoded_blocks_size(in_mem_seg)); - result.uncompressed_bytes_ += result.encoded_blocks_bytes_; - result.max_compressed_bytes_ += BytesEncoder::max_compressed_size(codec_opts, result.encoded_blocks_bytes_); - } + SizeResult& result) { + result.encoded_blocks_bytes_ = static_cast(encoded_blocks_size(in_mem_seg)); + result.uncompressed_bytes_ += result.encoded_blocks_bytes_; + result.max_compressed_bytes_ += BytesEncoder::max_compressed_size(codec_opts, result.encoded_blocks_bytes_); +} + +static void add_stream_descriptor_data_size(SizeResult& result, const StreamId& stream_id) { + result.max_compressed_bytes_ += sizeof(FrameDescriptor); + result.uncompressed_bytes_ += sizeof(FrameDescriptor); + const auto identifier_size = identifier_bytes(stream_id); + result.max_compressed_bytes_ += identifier_size; + result.uncompressed_bytes_ += identifier_size; +} - static void calc_stream_descriptor_fields_size( +static void calc_stream_descriptor_fields_size( const SegmentInMemory& in_mem_seg, const arcticdb::proto::encoding::VariantCodec& codec_opts, - SizeResult& result - ) { - auto segment_fields = in_mem_seg.descriptor().fields().column_data(); - const auto [uncompressed, required] = ColumnEncoderV2::max_compressed_size(codec_opts, segment_fields); - result.uncompressed_bytes_ += uncompressed; - result.max_compressed_bytes_ += required; - - // Calculate index fields size - if(in_mem_seg.index_fields()) { - auto index_field_data = in_mem_seg.index_fields()->column_data(); - const auto [idx_uncompressed, idx_required] = ColumnEncoderV2::max_compressed_size(codec_opts, index_field_data); - result.uncompressed_bytes_ += idx_uncompressed; - result.max_compressed_bytes_ += idx_required; - } + SizeResult& result) { + auto segment_fields = in_mem_seg.descriptor().fields().column_data(); + const auto [uncompressed, required] = ColumnEncoderV2::max_compressed_size(codec_opts, segment_fields); + result.uncompressed_bytes_ += uncompressed; + result.max_compressed_bytes_ += required; + add_stream_descriptor_data_size(result, in_mem_seg.descriptor().id()); + + if(in_mem_seg.has_index_descriptor()) { + const auto& tsd = in_mem_seg.index_descriptor(); + auto index_field_data = tsd.fields().column_data(); + const auto [idx_uncompressed, idx_required] = ColumnEncoderV2::max_compressed_size(codec_opts, index_field_data); + result.uncompressed_bytes_ += idx_uncompressed; + result.max_compressed_bytes_ += idx_required; + add_stream_descriptor_data_size(result, tsd.stream_id_); } +} - [[nodiscard]] SizeResult max_compressed_size_v2( +[[nodiscard]] SizeResult max_compressed_size_v2( const SegmentInMemory& in_mem_seg, - const arcticdb::proto::encoding::VariantCodec& codec_opts - ) { - ARCTICDB_SAMPLE(GetSegmentCompressedSize, 0) - SizeResult result{}; - result.max_compressed_bytes_ += sizeof(MetadataMagic); - calc_metadata_size(in_mem_seg, codec_opts, result); - result.max_compressed_bytes_ += sizeof(DescriptorMagic); - result.max_compressed_bytes_ += sizeof(IndexMagic); - calc_stream_descriptor_fields_size(in_mem_seg, codec_opts, result); - result.max_compressed_bytes_ += sizeof(EncodedMagic); - calc_encoded_blocks_size(in_mem_seg, codec_opts, result); - - // Calculate fields collection size - if(in_mem_seg.row_count() > 0) { - result.max_compressed_bytes_ += sizeof(ColumnMagic) * in_mem_seg.descriptor().field_count(); - calc_columns_size(in_mem_seg, codec_opts, result); - result.max_compressed_bytes_ += sizeof(StringPoolMagic); - calc_string_pool_size(in_mem_seg, codec_opts, result); - } - ARCTICDB_TRACE(log::codec(), "Max compressed size {}", result.max_compressed_bytes_); - return result; + const arcticdb::proto::encoding::VariantCodec& codec_opts) { + ARCTICDB_SAMPLE(GetSegmentCompressedSize, 0) + SizeResult result{}; + result.max_compressed_bytes_ += sizeof(MetadataMagic); + calc_metadata_size(in_mem_seg, codec_opts, result); + result.max_compressed_bytes_ += sizeof(DescriptorFieldsMagic); + result.max_compressed_bytes_ += sizeof(IndexMagic); + calc_stream_descriptor_fields_size(in_mem_seg, codec_opts, result); + result.max_compressed_bytes_ += sizeof(EncodedMagic); + calc_encoded_blocks_size(in_mem_seg, codec_opts, result); + + // Calculate fields collection size + if(in_mem_seg.row_count() > 0) { + result.max_compressed_bytes_ += sizeof(ColumnMagic) * in_mem_seg.descriptor().field_count(); + calc_columns_size(in_mem_seg, codec_opts, result); + result.max_compressed_bytes_ += sizeof(StringPoolMagic); + calc_string_pool_size(in_mem_seg, codec_opts, result); } + ARCTICDB_TRACE(log::codec(), "Max compressed size {}", result.max_compressed_bytes_); + return result; +} - static void encode_encoded_fields( - arcticdb::proto::encoding::SegmentHeader& segment_header, +static void encode_encoded_fields( + SegmentHeader& segment_header, const arcticdb::proto::encoding::VariantCodec& codec_opts, Buffer& out_buffer, std::ptrdiff_t& pos, - const ChunkedBuffer& encoded_blocks_buffer - ) { + EncodedFieldCollection&& encoded_fields) { + log::codec().info("Encoding encoded blocks to position {}", pos); + + segment_header.set_footer_offset(pos); + write_magic(out_buffer, pos); + Column encoded_fields_column(encoded_fields_type_desc(), false, encoded_fields.release_data()); + auto data = encoded_fields_column.data(); + auto& encoded_field = segment_header.mutable_column_fields(calc_num_blocks(data)); + ColumnEncoderV2::encode(codec_opts, data, encoded_field, out_buffer, pos); + log::codec().info("Encoded encoded blocks to position {}", pos); +} - ARCTICDB_TRACE(log::codec(), "Encoding encoded blocks to position {}", pos); - auto encoded_field = segment_header.mutable_column_fields(); - encoded_field->set_offset(static_cast(pos)); - write_magic(out_buffer, pos); - if(!encoded_blocks_buffer.empty()) - BytesEncoder::encode(encoded_blocks_buffer, codec_opts, out_buffer, pos, encoded_field); +[[nodiscard]] Segment encode_v2( + SegmentInMemory&& s, + const arcticdb::proto::encoding::VariantCodec &codec_opts) { + ARCTICDB_SAMPLE(EncodeSegment, 0) + auto in_mem_seg = std::move(s); - ARCTICDB_TRACE(log::codec(), "Encoded encoded blocks to position {}", pos); + if(in_mem_seg.has_index_descriptor()) { + google::protobuf::Any any; + util::pack_to_any(in_mem_seg.index_descriptor().proto(), any); + in_mem_seg.set_metadata(std::move(any)); } - [[nodiscard]] Segment encode_v2(SegmentInMemory&& s, const arcticdb::proto::encoding::VariantCodec &codec_opts) { - ARCTICDB_SAMPLE(EncodeSegment, 0) - - auto in_mem_seg = std::move(s); - auto arena = std::make_unique(); - auto segment_header = google::protobuf::Arena::CreateMessage(arena.get()); - auto& seg_descriptor = in_mem_seg.descriptor(); - *segment_header->mutable_stream_descriptor() = std::move(seg_descriptor.mutable_proto()); - segment_header->set_compacted(in_mem_seg.compacted()); - segment_header->set_encoding_version(static_cast(EncodingVersion::V2)); - - std::ptrdiff_t pos = 0; - static auto block_to_header_ratio = ConfigsMap::instance()->get_int("Codec.EstimatedHeaderRatio", 75); - const auto preamble = in_mem_seg.num_blocks() * block_to_header_ratio; - auto [max_compressed_size, uncompressed_size, encoded_buffer_size] = max_compressed_size_v2(in_mem_seg, codec_opts); - ARCTICDB_TRACE(log::codec(), "Estimated max buffer requirement: {}", max_compressed_size); - auto out_buffer = std::make_shared(max_compressed_size + encoded_buffer_size, preamble); - ARCTICDB_TRACE(log::codec(), "Encoding descriptor: {}", segment_header->stream_descriptor().DebugString()); - auto *tsd = segment_header->mutable_stream_descriptor(); - tsd->set_in_bytes(uncompressed_size); - - write_magic(*out_buffer, pos); - encode_metadata(in_mem_seg, *segment_header, codec_opts, *out_buffer, pos); - write_magic(*out_buffer, pos); - encode_field_descriptors(in_mem_seg, *segment_header, codec_opts, *out_buffer, pos); - - auto encoded_fields_buffer = ChunkedBuffer::presized(static_cast(encoded_buffer_size)); - auto encoded_field_pos = 0u; - ColumnEncoderV2 encoder; - if(in_mem_seg.row_count() > 0) { - ARCTICDB_TRACE(log::codec(), "Encoding fields"); - for (std::size_t column_index = 0; column_index < in_mem_seg.num_columns(); ++column_index) { - write_magic(*out_buffer, pos); - auto column_field = new(encoded_fields_buffer.data() + encoded_field_pos) EncodedField; - ARCTICDB_TRACE(log::codec(),"Beginning encoding of column {}: ({}) to position {}", column_index, in_mem_seg.descriptor().field(column_index).name(), pos); - auto column_data = in_mem_seg.column_data(column_index); - if(column_data.num_blocks() > 0) { - encoder.encode(codec_opts, column_data, column_field, *out_buffer, pos); - ARCTICDB_TRACE(log::codec(), - "Encoded column {}: ({}) to position {}", - column_index, - in_mem_seg.descriptor().field(column_index).name(), - pos); - } else { - util::check(!must_contain_data(column_data.type()), "Column {} of type {} contains no blocks", column_index, column_data.type()); - auto* ndarray = column_field->mutable_ndarray(); - ndarray->set_items_count(0); - } - encoded_field_pos += encoded_field_bytes(*column_field); - util::check(encoded_field_pos <= encoded_fields_buffer.bytes(), - "Encoded field buffer overflow {} > {}", - encoded_field_pos, - encoded_fields_buffer.bytes()); - + SegmentHeader segment_header{EncodingVersion::V2}; + segment_header.set_compacted(in_mem_seg.compacted()); + + std::ptrdiff_t pos = 0; + auto [max_compressed_size, uncompressed_size, encoded_buffer_size] = max_compressed_size_v2(in_mem_seg, codec_opts); + ARCTICDB_TRACE(log::codec(), "Estimated max buffer requirement: {}", max_compressed_size); + const auto preamble = segment_header.required_bytes(in_mem_seg); + auto out_buffer = std::make_shared(max_compressed_size + encoded_buffer_size, preamble); + ARCTICDB_TRACE(log::codec(), "Encoding descriptor: {}", in_mem_seg.descriptor()); + + auto descriptor_data = in_mem_seg.descriptor().data_ptr(); + descriptor_data->uncompressed_bytes_ = uncompressed_size; + + write_magic(*out_buffer, pos); + encode_metadata(in_mem_seg, segment_header, codec_opts, *out_buffer, pos); + segment_header.validate(); + write_magic(*out_buffer, pos); + write_segment_descriptor(*out_buffer, pos, in_mem_seg.descriptor().data()); + write_identifier(*out_buffer, pos, in_mem_seg.descriptor().id()); + write_magic(*out_buffer, pos); + segment_header.validate(); + encode_field_descriptors(in_mem_seg, segment_header, codec_opts, *out_buffer, pos); + segment_header.validate(); + write_magic(*out_buffer, pos); + encode_index_descriptors(in_mem_seg, segment_header, codec_opts, *out_buffer, pos); + + EncodedFieldCollection encoded_fields(encoded_buffer_size, in_mem_seg.num_columns()); + ColumnEncoderV2 encoder; + segment_header.validate(); + if(in_mem_seg.row_count() > 0) { + ARCTICDB_TRACE(log::codec(), "Encoding fields"); + for (std::size_t column_index = 0; column_index < in_mem_seg.num_columns(); ++column_index) { + write_magic(*out_buffer, pos); + auto column_data = in_mem_seg.column_data(column_index); + auto* column_field = encoded_fields.add_field(column_data.num_blocks()); + ARCTICDB_TRACE(log::codec(),"Beginning encoding of column {}: ({}) to position {}", column_index, in_mem_seg.descriptor().field(column_index).name(), pos); + + if(column_data.num_blocks() > 0) { + encoder.encode(codec_opts, column_data, *column_field, *out_buffer, pos); + ARCTICDB_TRACE(log::codec(), + "Encoded column {}: ({}) to position {}", + column_index, + in_mem_seg.descriptor().field(column_index).name(), + pos); + } else { + util::check(!must_contain_data(column_data.type()), "Column {} of type {} contains no blocks", column_index, column_data.type()); + auto* ndarray = column_field->mutable_ndarray(); + ndarray->set_items_count(0); } - write_magic(*out_buffer, pos); - encode_string_pool(in_mem_seg, *segment_header, codec_opts, *out_buffer, pos); + segment_header.validate(); } + write_magic(*out_buffer, pos); + encode_string_pool(in_mem_seg, segment_header, codec_opts, *out_buffer, pos); + encode_encoded_fields(segment_header, codec_opts, *out_buffer, pos, std::move(encoded_fields)); + } - encode_encoded_fields(*segment_header, codec_opts, *out_buffer, pos, encoded_fields_buffer); - out_buffer->set_bytes(pos); - tsd->set_out_bytes(pos); + out_buffer->set_bytes(pos); + descriptor_data->compressed_bytes_ = pos; + segment_header.validate(); - ARCTICDB_TRACE(log::codec(), "Encoded header: {}", tsd->DebugString()); - ARCTICDB_DEBUG(log::codec(), "Block count {} header size {} ratio {}", - in_mem_seg.num_blocks(), segment_header->ByteSizeLong(), - in_mem_seg.num_blocks() ? segment_header->ByteSizeLong() / in_mem_seg.num_blocks() : 0); - return {std::move(arena), segment_header, std::move(out_buffer), seg_descriptor.fields_ptr()}; - } + ARCTICDB_TRACE(log::codec(), "Encoded header: {}", segment_header); + const auto& desc = in_mem_seg.descriptor(); + return {std::move(segment_header), std::move(out_buffer), descriptor_data, desc.fields_ptr(), desc.id(), std::nullopt}; } + +} //namespace arcticdb diff --git a/cpp/arcticdb/codec/encoded_field.cpp b/cpp/arcticdb/codec/encoded_field.cpp new file mode 100644 index 0000000000..07aff7e786 --- /dev/null +++ b/cpp/arcticdb/codec/encoded_field.cpp @@ -0,0 +1,19 @@ +#include +#include +#include + +namespace arcticdb { + +std::pair get_segment_begin_end( + const Segment &segment, + const SegmentHeader &hdr) { + const uint8_t *data = segment.buffer().data(); + util::check(data != nullptr, "Got null data ptr from segment"); + const uint8_t *begin = data; + + const auto fields_offset = hdr.footer_offset(); + const auto end = begin + fields_offset; + return {begin, end}; +} + +} \ No newline at end of file diff --git a/cpp/arcticdb/codec/encoded_field.hpp b/cpp/arcticdb/codec/encoded_field.hpp index 01e5ec5004..3e3087daf5 100644 --- a/cpp/arcticdb/codec/encoded_field.hpp +++ b/cpp/arcticdb/codec/encoded_field.hpp @@ -7,180 +7,120 @@ #pragma once -#include - -#pragma pack(push) -#pragma pack(1) - +#include +#include +#include +#include +#include namespace arcticdb { -inline std::pair get_segment_begin_end(const Segment &segment, - const arcticdb::proto::encoding::SegmentHeader &hdr) { - const uint8_t *data = segment.buffer().data(); - util::check(data != nullptr, "Got null data ptr from segment"); - const uint8_t *begin = data; - - const auto fields_offset = hdr.column_fields().offset(); - const auto end = begin + fields_offset; - return {begin, end}; -} - -constexpr size_t encoding_size = 6; -enum class Codec : uint16_t { - Unknown = 0, - Zstd, - TurboPfor, - Lz4, - Passthrough -}; - -struct ZstdCodec { - static constexpr Codec type_ = Codec::Zstd; - - void MergeFrom(const arcticdb::proto::encoding::VariantCodec::Zstd &zstd) { - level_ = zstd.level(); - is_streaming = zstd.is_streaming(); +class Segment; +class SegmentHeader; + +std::pair get_segment_begin_end( + const Segment &segment, + const SegmentHeader& hdr); + +constexpr std::string_view codec_type_to_string(Codec codec) { + switch(codec) { + case Codec::LZ4: + return "LZ4"; + case Codec::ZSTD: + return "ZSTD"; + case Codec::PFOR: + return "PFOR"; + case Codec::PASS: + return "PASS"; + default: + return "Unknown"; } +} - int32_t level_ = 0; - bool is_streaming = false; - uint8_t padding_ = 0; -}; - -static_assert(sizeof(ZstdCodec) == encoding_size); - -struct TurboPforCodec { - static constexpr Codec type_ = Codec::TurboPfor; - - void MergeFrom(const arcticdb::proto::encoding::VariantCodec::TurboPfor &tp4) { - sub_codec_ = SubCodec(tp4.sub_codec()); +struct BlockCodecImpl : public BlockCodec { + uint8_t* data() { + return &data_[0]; } - enum class SubCodec : uint32_t { - UNKNOWN = 0, - P4 = 16, - P4_DELTA = 17, - P4_DELTA_RLE = 18, - P4_ZZ = 20, - - FP_DELTA = 32, // fpp - FP_DELTA2_ZZ = 33, // fpzz - FP_GORILLA_RLE = 34, // fpg - FP_ZZ = 36, // bvz - FP_ZZ_DELTA = 40, // bvz - }; - - SubCodec sub_codec_ = SubCodec::UNKNOWN; - uint16_t padding_ = 0; -}; - -static_assert(sizeof(TurboPforCodec) == encoding_size); - -struct Lz4Codec { - static constexpr Codec type_ = Codec::Lz4; - - void MergeFrom(const arcticdb::proto::encoding::VariantCodec::Lz4& lz4) { - acceleration_ = lz4.acceleration(); + [[nodiscard]] Codec codec_type() const { + return codec_; } - int32_t acceleration_ = 1; - int16_t padding_ = 0; -}; - -static_assert(sizeof(Lz4Codec) == encoding_size); - -struct PassthroughCodec { - static constexpr Codec type_ = Codec::Passthrough; - - uint32_t unused_ = 0; - uint16_t padding_ = 0; -}; - -static_assert(sizeof(PassthroughCodec) == encoding_size); - -struct BlockCodec { - Codec codec_ = Codec::Unknown; - constexpr static size_t DataSize = 24; - std::array data_; - - uint8_t* data() { + [[nodiscard]] const uint8_t* data() const { return &data_[0]; } - BlockCodec() { + BlockCodecImpl() { memset(data(), 0, DataSize); } ZstdCodec *mutable_zstd() { - codec_ = Codec::Zstd; + codec_ = Codec::ZSTD; auto zstd = new(data()) ZstdCodec{}; return zstd; } Lz4Codec *mutable_lz4() { - codec_ = Codec::Lz4; + codec_ = Codec::LZ4; auto lz4 = new(data()) Lz4Codec{}; return lz4; } - TurboPforCodec *mutable_turbopfor() { - codec_ = Codec::TurboPfor; - auto pfor = new(data()) TurboPforCodec{}; + PforCodec *mutable_pfor() { + codec_ = Codec::PFOR; + auto pfor = new(data()) PforCodec{}; return pfor; } PassthroughCodec *mutable_passthrough() { - codec_ = Codec::Passthrough; + codec_ = Codec::PASS; auto pass = new(data()) PassthroughCodec{}; return pass; } - arcticdb::proto::encoding::VariantCodec::CodecCase codec_case() const { - switch (codec_) { - case Codec::Zstd:return arcticdb::proto::encoding::VariantCodec::kLz4; - case Codec::Lz4:return arcticdb::proto::encoding::VariantCodec::kLz4; - case Codec::TurboPfor:return arcticdb::proto::encoding::VariantCodec::kTp4; - case Codec::Passthrough:return arcticdb::proto::encoding::VariantCodec::kPassthrough; - default:util::raise_rte("Unknown codec"); - } + [[nodiscard]] const ZstdCodec& zstd() const { + util::check(codec_ == Codec::ZSTD, "Not a zstd codec"); + return *reinterpret_cast(data()); } - template - explicit BlockCodec(const CodecType &codec) : - codec_(CodecType::type) { - memcpy(data_, &codec, encoding_size); + [[nodiscard]] const Lz4Codec& lz4() const { + util::check(codec_ == Codec::LZ4, "Not an lz4 codec"); + return *reinterpret_cast(data()); } -}; -struct EncodedBlock { - uint32_t in_bytes_ = 0; - uint32_t out_bytes_ = 0; - uint64_t hash_ = 0; - uint16_t encoder_version_ = 0; - bool is_shape_ = false; - uint8_t pad_ = 0; - BlockCodec codec_; + [[nodiscard]] const PforCodec& pfor() const { + util::check(codec_ == Codec::PFOR, "Not a pfor codec"); + return *reinterpret_cast(data()); + } - EncodedBlock() = default; + [[nodiscard]] const PassthroughCodec& passthrough() const { + util::check(codec_ == Codec::PASS, "Not a passthrough codec"); + return *reinterpret_cast(data()); + } - explicit EncodedBlock(bool is_shape) : - is_shape_(is_shape) { + template + explicit BlockCodecImpl(const CodecType &codec) { + codec_ = CodecType::type; + memcpy(data_, &codec, encoding_size); } +}; - std::string DebugString() const { - return ""; +struct EncodedBlock : Block { + explicit EncodedBlock(bool is_shape) { + is_shape_ = is_shape; } - bool has_codec() const { - return codec_.codec_ != Codec::Passthrough; + EncodedBlock() = default; + + [[nodiscard]] bool has_codec() const { + return codecs_[0].codec_ != Codec::PASS; } - auto encoder_version() const { + [[nodiscard]] auto encoder_version() const { return encoder_version_; } - auto codec() const { - return codec_; + [[nodiscard]] auto codec() const { + return *reinterpret_cast(&codecs_[0]); } void set_in_bytes(uint32_t bytes) { @@ -203,57 +143,42 @@ struct EncodedBlock { return hash_; } - uint32_t out_bytes() const { + [[nodiscard]] uint32_t out_bytes() const { return out_bytes_; } - uint32_t in_bytes() const { + [[nodiscard]] uint32_t in_bytes() const { return in_bytes_; } - BlockCodec *mutable_codec() { - return &codec_; + BlockCodecImpl *mutable_codec() { + return reinterpret_cast(&codecs_[0]); } }; -struct EncodedField { - - enum class EncodedFieldType : uint8_t { - Unknown, - kNdarray, - Dictionary - }; - - EncodedFieldType type_ = EncodedFieldType::Unknown; - google::protobuf::uint8 shapes_count_ = 0u; - uint16_t values_count_ = 0u; - uint32_t sparse_map_bytes_ = 0u; - uint64_t items_count_ = 0u; - std::array blocks_; - - static constexpr size_t MinimumSize = sizeof(type_) + sizeof(shapes_count_) + sizeof(values_count_) + sizeof(sparse_map_bytes_) + sizeof(items_count_); - - static constexpr EncodedFieldType kNdarray = EncodedFieldType::kNdarray; - +struct EncodedFieldImpl : public EncodedField { static constexpr size_t Size = sizeof(type_) + sizeof(shapes_count_) + sizeof(values_count_) + sizeof(sparse_map_bytes_) + - sizeof(items_count_); + sizeof(items_count_) + + sizeof(format_); + + EncodedFieldImpl() = default; - EncodedField() = default; + ARCTICDB_NO_MOVE_OR_COPY(EncodedFieldImpl) EncodedBlock *blocks() { - return &blocks_[0]; + return reinterpret_cast(&blocks_[0]); } - const EncodedBlock* blocks() const { - return &blocks_[0]; + [[nodiscard]] const EncodedBlock* blocks() const { + return reinterpret_cast(&blocks_[0]); } struct EncodedBlockCollection { - EncodedBlockCollection(const EncodedField &field, bool is_shapes) : + EncodedBlockCollection(const EncodedFieldImpl &field, bool is_shapes) : field_(field), is_shapes_(is_shapes) { } @@ -263,19 +188,22 @@ struct EncodedField { ValueType, boost::forward_traversal_tag> { public: - explicit EncodedBlockCollectionIterator(EncodedBlock *blocks) : - blocks_(blocks) {} + EncodedBlockCollectionIterator(EncodedBlock *blocks, size_t increment) : + blocks_(blocks), + increment_(increment){} ~EncodedBlockCollectionIterator() = default; - explicit EncodedBlockCollectionIterator(EncodedBlock *blocks, size_t pos) : + EncodedBlockCollectionIterator(EncodedBlock *blocks, size_t pos, size_t increment) : blocks_(blocks), - pos_(pos) {} + pos_(pos), + increment_(increment){} template explicit EncodedBlockCollectionIterator(const EncodedBlockCollectionIterator &other) : blocks_(other.blocks_), - pos_(other.pos_) {} + pos_(other.pos_), + increment_(other.increment_){} EncodedBlockCollectionIterator() = default; @@ -283,6 +211,7 @@ struct EncodedField { if (&other != this) { pos_ = other.pos_; blocks_ = other.blocks_; + increment_ = other.increment_; } return *this; @@ -290,16 +219,17 @@ struct EncodedField { EncodedBlockCollectionIterator(const EncodedBlockCollectionIterator &other) : blocks_(other.blocks_), - pos_(other.pos_) { + pos_(other.pos_), + increment_(other.increment_) { } template - bool equal(const EncodedBlockCollectionIterator &other) const { - return pos_ == other.pos_ && blocks_ == other.blocks_; + [[nodiscard]] bool equal(const EncodedBlockCollectionIterator &other) const { + return pos_ == other.pos_ && blocks_ == other.blocks_ && increment_ == other.increment_; } void increment() { - ++pos_; + pos_ += increment_; } [[nodiscard]] ValueType &dereference() const { @@ -308,81 +238,125 @@ struct EncodedField { EncodedBlock *blocks_ = nullptr; size_t pos_ = 0; + size_t increment_ = 1; }; - EncodedBlock *blocks() const { - return const_cast(field_).blocks(); + [[nodiscard]] EncodedBlock *blocks() const { + return const_cast(field_).blocks(); + } + + [[nodiscard]] size_t increment() const { + return field_.is_scalar() || !field_.is_old_style_shapes() ? 1 : 2; } [[nodiscard]] auto begin() { - return EncodedBlockCollectionIterator(blocks(), first()); + return EncodedBlockCollectionIterator(blocks(), first(), increment()); } [[nodiscard]] auto end() { - return EncodedBlockCollectionIterator(blocks(), last()); + return EncodedBlockCollectionIterator(blocks(), last(), increment()); } [[nodiscard]] auto begin() const { - return EncodedBlockCollectionIterator(blocks(), first()); + return EncodedBlockCollectionIterator(blocks(), first(), increment()); } [[nodiscard]] auto end() const { - return EncodedBlockCollectionIterator(blocks(), last()); + return EncodedBlockCollectionIterator(blocks(), last(), increment()); + } + + [[nodiscard]] size_t shape_value_offset() const { + return is_shapes_ || field_.is_scalar() ? 0U : 1U; } - size_t first() const { - return is_shapes_ ? 0u : field_.shapes_count_; + [[nodiscard]] size_t first() const { + return shape_value_offset(); } - size_t last() const { - return is_shapes_ ? field_.shapes_count_ : field_.shapes_count_ + field_.values_count_; + [[nodiscard]] size_t last() const { + if(field_.is_scalar()) + return is_shapes_ ? 0 : field_.values_count_; + + if(field_.is_old_style_shapes()) + return field_.values_count_ + field_.shapes_count_ + shape_value_offset(); + else + return is_shapes_ ? field_.shapes_count_ : field_.shapes_count_ + field_.values_count_; } [[nodiscard]] const EncodedBlock& operator[](const size_t idx) const { - // Shape blocks are located before values blocks in the field. In case this is a collection of value blocks - // we have to skip all shape blocks. In case this is a collection of shapes we can start from 0 index. - const size_t shape_offset = !is_shapes_ * field_.shapes_count_; + const size_t shape_offset = is_shapes_ ? 0 : field_.shapes_count_; return field_.blocks()[shape_offset + idx]; } - const EncodedField &field_; + + const EncodedFieldImpl& field_; bool is_shapes_; }; - EncodedFieldType encoding_case() const { + bool is_scalar() const { + return shapes_count_ == 0; + } + + bool is_old_style_shapes() const { + return shapes_size() == values_size(); + } + + [[nodiscard]] EncodedFieldType encoding_case() const { return type_; } - const EncodedBlock& shapes(size_t n) const { - util::check(n == 0, "Expected only one shape"); + [[nodiscard]] const EncodedBlock& shapes(size_t n) const { util::check(shapes_count_ != 0, "No shape allocated"); - return blocks_[0]; + if(!is_old_style_shapes()) + return *reinterpret_cast(&blocks_[0]); + else + return *reinterpret_cast(&blocks_[n * 2]); } - const EncodedBlock &values(size_t n) const { + [[nodiscard]] const EncodedBlock &values(size_t n) const { util::check(n < values_count_ + shapes_count_, "Cannot return block {} from {} blocks ({} shapes)", n, values_count_, shapes_count_); - return blocks()[shapes_count_ + n]; + if(is_scalar() || !is_old_style_shapes()) + return blocks()[shapes_count_ + n]; + else + return blocks()[(n * 2) + 1]; } - EncodedBlockCollection shapes() const { + [[nodiscard]] EncodedBlockCollection shapes() const { return {*this, true}; } - EncodedBlockCollection values() const { + [[nodiscard]] EncodedBlockCollection values() const { return {*this, false}; } + void validate() const { + size_t shapes_count = 0; + for(const auto& shape : shapes()) { + util::check(shape.is_shape_, "Expected shape to have is_shape_set"); + util::check(shape.codecs_[0].codec_ != Codec::UNKNOWN, "Unknown shape codec"); + ++shapes_count; + } + util::check(shapes_count == static_cast(shapes_size()), "Shape size mismatch: {} != {}", shapes_count, shapes_size()); + + size_t values_count = 0; + for(const auto& value : values()) { + util::check(value.is_shape_ == false, "Value has is_shape set"); + util::check(value.codec().codec_type() != Codec::UNKNOWN, "Unknown codec in block {}", values_count); + ++values_count; + } + util::check(values_count == static_cast(values_size()), "Shape size mismatch: {} != {}", values_count, values_size()); + } + EncodedBlock *add_shapes() { - util::check(shapes_count_ == 0, "Expected single shapes block"); - auto block = new(blocks() + items_count()) EncodedBlock{true}; + auto block = new(blocks() + (shapes_count_ * 2)) EncodedBlock{true}; ++shapes_count_; return block; } - int shapes_size() const { + [[nodiscard]] int shapes_size() const { return shapes_count_; } - int values_size() const { + [[nodiscard]] int values_size() const { return values_count_; } @@ -390,26 +364,33 @@ struct EncodedField { sparse_map_bytes_ = bytes; } - EncodedBlock *add_values() { - auto block = new(static_cast(blocks() + shapes_count_ + values_count_)) EncodedBlock{false}; + EncodedBlock *add_values(EncodingVersion encoding_version) { + const bool old_style = encoding_version == EncodingVersion::V1; + size_t pos; + if(!old_style || is_scalar()) + pos = shapes_count_ + values_count_; + else + pos = (values_count_ * 2) + 1; + + auto block = new(static_cast(blocks() + pos)) EncodedBlock{false}; ++values_count_; return block; } - EncodedField *mutable_ndarray() { - type_ = EncodedFieldType::kNdarray; + EncodedFieldImpl *mutable_ndarray() { + type_ = EncodedFieldType::NDARRAY; return this; } - const EncodedField &ndarray() const { + [[nodiscard]] const EncodedFieldImpl &ndarray() const { return *this; } - bool has_ndarray() const { - return type_ == EncodedFieldType::kNdarray; + [[nodiscard]] bool has_ndarray() const { + return type_ == EncodedFieldType::NDARRAY; } - std::string DebugString() const { + [[nodiscard]] std::string DebugString() const { return ""; } @@ -417,22 +398,49 @@ struct EncodedField { return items_count_; } - size_t sparse_map_bytes() const { + [[nodiscard]] size_t sparse_map_bytes() const { return sparse_map_bytes_; } - void set_items_count(size_t count) { + void set_items_count(uint32_t count) { items_count_ = count; } }; -static_assert(sizeof(EncodedField) - sizeof(EncodedBlock) == EncodedField::Size); +static_assert(EncodedFieldImpl::Size == sizeof(EncodedFieldImpl) - sizeof(EncodedBlock)); -inline size_t encoded_field_bytes(const EncodedField &encoded_field) { - return sizeof(EncodedField) - + (sizeof(EncodedBlock) * ((encoded_field.shapes_count_ + encoded_field.values_count_) - 1)); +inline size_t calc_field_bytes(size_t num_blocks) { + return EncodedFieldImpl::Size + (sizeof(EncodedBlock) * num_blocks); } -#pragma pack(pop) +inline size_t encoded_field_bytes(const EncodedField &encoded_field) { + return calc_field_bytes(encoded_field.shapes_count_ + encoded_field.values_count_); +} } //namespace arcticc + +namespace fmt { +template<> +struct formatter { + template + constexpr auto parse(ParseContext &ctx) { return ctx.begin(); } + + template + auto format(arcticdb::BlockCodecImpl codec, FormatContext &ctx) const { + + return format_to(ctx.out(), "{}", arcticdb::codec_type_to_string(codec.codec_type())); + } +}; + +template<> +struct formatter { + template + constexpr auto parse(ParseContext &ctx) { return ctx.begin(); } + + template + auto format(const arcticdb::EncodedFieldImpl& field, FormatContext &ctx) const { + return format_to(ctx.out(), "{}", field.has_ndarray() ? "NDARRAY" : "DICT"); //TODO better formatting + } +}; + +} // namespace fmt diff --git a/cpp/arcticdb/codec/encoded_field_collection.hpp b/cpp/arcticdb/codec/encoded_field_collection.hpp index e3028fa677..1f760b55ca 100644 --- a/cpp/arcticdb/codec/encoded_field_collection.hpp +++ b/cpp/arcticdb/codec/encoded_field_collection.hpp @@ -7,55 +7,164 @@ #pragma once +#include #include +#include #include namespace arcticdb { +using namespace arcticdb::entity; + +constexpr TypeDescriptor encoded_fields_type_desc() { + using namespace arcticdb::entity; + return TypeDescriptor{ + DataType::UINT8, Dimension::Dim0 + }; +} + class EncodedFieldCollection { - Buffer buffer_; - std::vector offsets_; + ChunkedBuffer data_; + Buffer offsets_; + size_t count_ = 0U; + size_t offset_ = 0U; public: - explicit EncodedFieldCollection(Buffer &&buffer) : - buffer_(std::move(buffer)) { - regenerate_offsets(); + struct EncodedFieldCollectionIterator { + size_t pos_ = 0UL; + ChunkedBuffer* buffer_ = nullptr; + + EncodedFieldCollectionIterator(ChunkedBuffer* buffer) : + buffer_(buffer) { + } + + EncodedFieldImpl& current() { + return *reinterpret_cast(buffer_->ptr_cast(pos_, EncodedFieldImpl::Size)); + } + + EncodedFieldImpl& operator*() { + return current(); + } + + void operator++() { + pos_ += encoded_field_bytes(current()); + } + + EncodedFieldImpl* operator->() { + return &(current()); + } + }; + + EncodedFieldCollection(ChunkedBuffer&& data, Buffer&& offsets) : + data_(std::move(data)), + offsets_(std::move(offsets)) { + } + + EncodedFieldCollection(size_t bytes, size_t num_fields) : + data_(bytes), + offsets_(num_fields * sizeof(uint64_t)){ } EncodedFieldCollection() = default; + [[nodiscard]] EncodedFieldCollection clone() const { + return {data_.clone(), offsets_.clone()}; + } + ARCTICDB_MOVE_ONLY_DEFAULT(EncodedFieldCollection) + [[nodiscard]] EncodedFieldCollectionIterator begin() const { + return {const_cast(&data_)}; + } + + size_t num_blocks() const { + return data_.num_blocks(); + } + [[nodiscard]] bool empty() const { - return buffer_.empty(); + return data_.empty(); } - [[nodiscard]] size_t get_offset(size_t pos) const { - util::check(pos < offsets_.size(), "Offset {} exceeds offsets size {}", pos, offsets_.size()); - return offsets_[pos]; + [[nodiscard]] size_t data_bytes() const { + return data_.bytes(); } - [[nodiscard]] const EncodedField &at(size_t pos) const { - return *(buffer_.ptr_cast(get_offset(pos), EncodedField::MinimumSize)); + [[nodiscard]] const uint8_t* data_buffer() const { + return data_.data(); } - [[nodiscard]] EncodedField &at(size_t pos) { - return *(buffer_.ptr_cast(get_offset(pos), EncodedField::MinimumSize)); + [[nodiscard]] size_t offset_bytes() const { + return offsets_.bytes(); + } + + [[nodiscard]] const uint8_t* offsets_buffer() const { + return offsets_.data(); + } + + [[nodiscard]] uint64_t get_offset(size_t pos) const { + const auto offset = *offsets_.ptr_cast(pos * sizeof(uint64_t), sizeof(uint64_t)); + return offset; + } + + void write_offset(size_t pos, uint64_t value) { + *offsets_.ptr_cast(pos * sizeof(uint64_t), sizeof(uint64_t)) = value; + } + + [[nodiscard]] const EncodedFieldImpl& to_field(size_t bytes_pos) const { + return *reinterpret_cast(data_.ptr_cast(bytes_pos, EncodedFieldImpl::Size)); + } + + [[nodiscard]] EncodedFieldImpl& to_field(size_t bytes_pos) { + return *reinterpret_cast(data_.ptr_cast(bytes_pos, EncodedFieldImpl::Size)); + } + + [[nodiscard]] const EncodedFieldImpl& at(size_t pos) const { + return to_field(get_offset(pos)); + } + + [[nodiscard]] EncodedFieldImpl &at(size_t pos) { + return to_field(get_offset(pos)); } [[nodiscard]] size_t size() const { - return offsets_.size(); + return offsets_.bytes() / sizeof(uint64_t); } void regenerate_offsets() { - if (!offsets_.empty()) + if(!offsets_.empty()) return; - auto field_pos = 0u; - while (field_pos < buffer_.bytes()) { - offsets_.push_back(field_pos); - field_pos += encoded_field_bytes(*reinterpret_cast(buffer_.data() + field_pos)); + auto pos = 0UL; + count_ = 0UL; + while(pos < data_.bytes()) { + const auto& field = to_field(pos); + offsets_.ensure((count_ + 1) * sizeof(uint64_t)); + write_offset(count_, pos); + ++count_; + pos += encoded_field_bytes(field); } + util::check(pos == data_.bytes(), "Size mismatch in regenerate_offsets, {} != {}", pos, data_.bytes()); + } + + [[nodiscard]] EncodedFieldImpl* add_field(size_t num_blocks) { + offsets_.ensure((count_ + 1) * sizeof(uint64_t)); + write_offset(count_, offset_); + const auto required_bytes = calc_field_bytes(num_blocks); + util::check(required_bytes >= EncodedFieldImpl::Size, "Unexpectedly small allocation size: {}", required_bytes); + data_.ensure(offset_ + required_bytes); + auto* field = new (data_.ptr_cast(offset_, required_bytes)) EncodedFieldImpl; + log::codec().info("Adding encoded field with {} blocks at position {}, {} bytes required", num_blocks, offset_, required_bytes); + ++count_; + offset_ += required_bytes; + return field; + } + + Buffer&& release_offsets() { + return std::move(offsets_); + } + + ChunkedBuffer&& release_data() { + return std::move(data_); } }; diff --git a/cpp/arcticdb/codec/encoding_sizes.cpp b/cpp/arcticdb/codec/encoding_sizes.cpp deleted file mode 100644 index 393b3ba8f1..0000000000 --- a/cpp/arcticdb/codec/encoding_sizes.cpp +++ /dev/null @@ -1,24 +0,0 @@ -/* Copyright 2023 Man Group Operations Limited - * - * Use of this software is governed by the Business Source License 1.1 included in the file licenses/BSL.txt. - * - * As of the Change Date specified in that file, in accordance with the Business Source License, use of this software will be governed by the Apache License, version 2.0. - */ - -#include -#include -#include - -namespace arcticdb::encoding_sizes { - -std::size_t represented_size(const arcticdb::proto::encoding::SegmentHeader& sh, size_t total_rows) { - std::size_t total = 0; - - for(const auto& field : sh.stream_descriptor().fields()) { - total += total_rows * get_type_size(entity::data_type_from_proto(field.type_desc())); - } - - return total; -} - -} \ No newline at end of file diff --git a/cpp/arcticdb/codec/encoding_sizes.hpp b/cpp/arcticdb/codec/encoding_sizes.hpp index ca29b6bf05..c6194d06b3 100644 --- a/cpp/arcticdb/codec/encoding_sizes.hpp +++ b/cpp/arcticdb/codec/encoding_sizes.hpp @@ -11,54 +11,55 @@ #include #include #include +#include #include namespace arcticdb::encoding_sizes { template std::size_t shape_compressed_size(const NDArrayEncodedFieldType &nda) { - return std::accumulate(std::begin(nda.shapes()), std::end(nda.shapes()), size_t(0), - [] (size_t a, const auto& block) { return a + block.out_bytes(); }); - } + return std::accumulate(std::begin(nda.shapes()), std::end(nda.shapes()), size_t(0), + [] (size_t a, const auto& block) { return a + block.out_bytes(); }); +} - template std::size_t data_compressed_size(const NDArrayEncodedFieldType &nda) { - return std::accumulate(std::begin(nda.values()), std::end(nda.values()), size_t(0), - [] (size_t a, const auto& block) { return a + block.out_bytes(); }); - } +template std::size_t data_compressed_size(const NDArrayEncodedFieldType &nda) { + return std::accumulate(std::begin(nda.values()), std::end(nda.values()), size_t(0), + [] (size_t a, const auto& block) { return a + block.out_bytes(); }); +} - template std::size_t shape_uncompressed_size(const NDArrayEncodedFieldType &nda) { - return std::accumulate(std::begin(nda.shapes()), std::end(nda.shapes()), size_t(0), - [] (size_t a, const auto& block) { return a + block.in_bytes(); }); - } +template std::size_t shape_uncompressed_size(const NDArrayEncodedFieldType &nda) { + return std::accumulate(std::begin(nda.shapes()), std::end(nda.shapes()), size_t(0), + [] (size_t a, const auto& block) { return a + block.in_bytes(); }); +} - template - std::size_t data_uncompressed_size(const NDArrayEncodedFieldType &nda) { - return std::accumulate(std::begin(nda.values()), std::end(nda.values()), size_t(0), - [] (size_t a, const auto& block) { return a + block.in_bytes(); }); - } +template +std::size_t data_uncompressed_size(const NDArrayEncodedFieldType &nda) { + return std::accumulate(std::begin(nda.values()), std::end(nda.values()), size_t(0), + [] (size_t a, const auto& block) { return a + block.in_bytes(); }); +} - template - std::size_t bitmap_serialized_size(const NDArrayEncodedFieldType &nda) { - return nda.sparse_map_bytes(); - } +template +std::size_t bitmap_serialized_size(const NDArrayEncodedFieldType &nda) { + return nda.sparse_map_bytes(); +} - template - std::size_t ndarray_field_compressed_size(const NDArrayEncodedFieldType &nda) { - return shape_compressed_size(nda) + data_compressed_size(nda) + bitmap_serialized_size(nda); - } +template +std::size_t ndarray_field_compressed_size(const NDArrayEncodedFieldType &nda) { + return shape_compressed_size(nda) + data_compressed_size(nda) + bitmap_serialized_size(nda); +} - template - std::size_t uncompressed_size(const NDArrayEncodedFieldType &nda) { - return shape_uncompressed_size(nda) + data_uncompressed_size(nda) + bitmap_serialized_size(nda); - } +template +std::size_t uncompressed_size(const NDArrayEncodedFieldType &nda) { + return shape_uncompressed_size(nda) + data_uncompressed_size(nda) + bitmap_serialized_size(nda); +} + +inline std::size_t field_compressed_size(const EncodedFieldImpl &field) { +switch (field.encoding_case()) { + case EncodedFieldType::NDARRAY: + return ndarray_field_compressed_size(field.ndarray()); + default: + util::raise_rte("Unsupported encoding {}", field.DebugString()); - template - std::size_t field_compressed_size(const EncodedFieldType &field) { - switch (field.encoding_case()) { - case EncodedFieldType::kNdarray: - return ndarray_field_compressed_size(field.ndarray()); - default: - util::raise_error_msg("Unsupported encoding {}", field); - } +} } template @@ -103,7 +104,6 @@ std::size_t segment_uncompressed_size(const FieldCollectionType &fields) { return total; } - std::size_t represented_size(const arcticdb::proto::encoding::SegmentHeader& sh, size_t total_rows); } // namespace encoding_sizes diff --git a/cpp/arcticdb/codec/lz4.hpp b/cpp/arcticdb/codec/lz4.hpp index 61c261c243..33d485cc60 100644 --- a/cpp/arcticdb/codec/lz4.hpp +++ b/cpp/arcticdb/codec/lz4.hpp @@ -34,7 +34,7 @@ struct Lz4BlockEncoder { static std::size_t encode_block( const Opts& opts, const T *in, - BlockProtobufHelper &block_utils, + BlockDataHelper &block_utils, HashAccum &hasher, T *out, std::size_t out_capacity, @@ -50,10 +50,10 @@ struct Lz4BlockEncoder { util::check_arg(compressed_bytes > 0 || (compressed_bytes == 0 && block_utils.bytes_ == 0), "expected compressed bytes >= 0, actual {}", compressed_bytes); - ARCTICDB_TRACE(log::storage(), "Block of size {} compressed to {} bytes", block_utils.bytes_, compressed_bytes); + ARCTICDB_DEBUG(log::storage(), "Block of size {} compressed to {} bytes", block_utils.bytes_, compressed_bytes); hasher(in, block_utils.count_); pos += ssize_t(compressed_bytes); - out_codec.mutable_lz4()->MergeFrom(opts); + copy_codec(*out_codec.mutable_lz4(), opts); return std::size_t(compressed_bytes); } }; @@ -74,7 +74,7 @@ struct Lz4Decoder { std::size_t out_bytes ) { - ARCTICDB_TRACE(log::codec(), "Lz4 decoder reading block: {} {}", in_bytes, out_bytes); + ARCTICDB_DEBUG(log::codec(), "Lz4 decoder reading block: {} {}", in_bytes, out_bytes); // Decompressed size < 0 means an error occurred in LZ4 during the decompression. In case it's negative // the specific value is somewhat random and does not mean anything. Decompressed size of 0 is allowed and means // 0 bytes were passed for compression. In that case t_out is allowed to be null since it's not used at all. diff --git a/cpp/arcticdb/codec/magic_words.hpp b/cpp/arcticdb/codec/magic_words.hpp index c654c7e92c..a7b8f9ed81 100644 --- a/cpp/arcticdb/codec/magic_words.hpp +++ b/cpp/arcticdb/codec/magic_words.hpp @@ -7,11 +7,15 @@ #pragma once +#include + namespace arcticdb { - using DescriptorMagic = util::MagicNum<'D','e','s','c'>; + using DescriptorFieldsMagic = util::MagicNum<'D','e','s','c'>; using EncodedMagic = util::MagicNum<'E','n','c','d'>; using StringPoolMagic = util::MagicNum<'S','t','r','p'>; using MetadataMagic = util::MagicNum<'M','e','t','a'>; using IndexMagic = util::MagicNum<'I','n','d','x'>; using ColumnMagic = util::MagicNum<'C','l','m','n'>; + using FrameMetadataMagic = util::MagicNum<'F','r', 'a', 'm'>; + using SegmentDescriptorMagic = util::MagicNum<'S','D', 's', 'c'>; } diff --git a/cpp/arcticdb/codec/passthrough.hpp b/cpp/arcticdb/codec/passthrough.hpp index 0660a05651..5fc84f7d30 100644 --- a/cpp/arcticdb/codec/passthrough.hpp +++ b/cpp/arcticdb/codec/passthrough.hpp @@ -19,7 +19,6 @@ namespace arcticdb::detail { template class BlockType, class TD> struct PassthroughEncoderV1 { - using Opts = arcticdb::proto::encoding::VariantCodec::Passthrough; static size_t max_compressed_size(const BlockType &block ) { @@ -35,7 +34,12 @@ struct PassthroughEncoderV1 { } template - static void encode(const Opts&, const BlockType& block, EncodedFieldType& field, Buffer& out, std::ptrdiff_t& pos) { + static void encode( + const Opts&, + const BlockType& block, + EncodedFieldType& field, + Buffer& out, + std::ptrdiff_t& pos) { using namespace arcticdb::entity; using Helper = CodecHelper; using T = typename Helper::T; @@ -46,19 +50,21 @@ struct PassthroughEncoderV1 { if constexpr (Helper::dim == entity::Dimension::Dim0) { // Only store data, no shapes since dimension is 0 - auto v_block = Helper::scalar_block(block_row_count); - helper.ensure_buffer(out, pos, v_block.bytes_); + auto scalar_block = Helper::scalar_block(block_row_count); + helper.ensure_buffer(out, pos, scalar_block.bytes_); // doing copy + hash in one pass, this might have a negative effect on perf // since the hashing is path dependent. This is a toy example though so not critical - T *t_out = out.ptr_cast(pos, v_block.bytes_); - encode_block(d, v_block, helper.hasher_, t_out, pos); + T *t_out = out.ptr_cast(pos, scalar_block.bytes_); + encode_block(d, scalar_block, helper.hasher_, t_out, pos); auto *nd_array = field.mutable_ndarray(); auto total_row_count = nd_array->items_count() + block_row_count; nd_array->set_items_count(total_row_count); - auto values_pb = nd_array->add_values(); - v_block.set_block_data(*values_pb, helper.hasher_.digest(), v_block.bytes_); + auto values = nd_array->add_values(EncodingVersion::V1); + (void)values->mutable_codec()->mutable_passthrough(); + scalar_block.set_block_data(*values, helper.hasher_.digest(), scalar_block.bytes_); + } else { auto helper_array_block = Helper::nd_array_block(block_row_count, block.shapes()); helper.ensure_buffer(out, pos, helper_array_block.shapes_.bytes_ + helper_array_block.values_.bytes_); @@ -73,12 +79,16 @@ struct PassthroughEncoderV1 { encode_block(d, helper_array_block.values_, helper.hasher_, t_out, pos); auto field_nd_array = field.mutable_ndarray(); // Important: In case V2 EncodedField is used shapes must be added before values. - auto shapes_pb = field_nd_array->add_shapes(); - auto values_pb = field_nd_array->add_values(); + auto shapes = field_nd_array->add_shapes(); + (void)shapes->mutable_codec()->mutable_passthrough(); + + auto values = field_nd_array->add_values(EncodingVersion::V1); + (void)values->mutable_codec()->mutable_passthrough(); + helper_array_block.update_field_size(*field_nd_array); helper_array_block.set_block_data( - shapes_pb, - values_pb, + shapes, + values, shape_hash, helper_array_block.shapes_.bytes_, helper.hasher_.digest(), @@ -87,7 +97,7 @@ struct PassthroughEncoderV1 { } private: template - static void encode_block(const T *in, BlockProtobufHelper &block_utils, HashAccum &hasher, T *out, std::ptrdiff_t &pos) { + static void encode_block(const T *in, BlockDataHelper &block_utils, HashAccum &hasher, T *out, std::ptrdiff_t &pos) { memcpy(out, in, block_utils.bytes_); hasher(in, block_utils.bytes_ / sizeof(T)); pos += static_cast(block_utils.bytes_); @@ -131,6 +141,7 @@ struct PassthroughEncoderV2 { encoded_block->set_in_bytes(data_byte_size); encoded_block->set_out_bytes(data_byte_size); encoded_block->set_hash(helper.hasher_.digest()); + (void)encoded_block->mutable_codec()->mutable_passthrough(); } private: template diff --git a/cpp/arcticdb/codec/protobuf_mappings.cpp b/cpp/arcticdb/codec/protobuf_mappings.cpp new file mode 100644 index 0000000000..134076b245 --- /dev/null +++ b/cpp/arcticdb/codec/protobuf_mappings.cpp @@ -0,0 +1,154 @@ +// +// Created by root on 2/16/24. +// +#include +#include +#include +#include +#include +#include +#include + +namespace arcticdb { + +void block_from_proto(const arcticdb::proto::encoding::Block& input, EncodedBlock& output, bool is_shape) { + output.set_in_bytes(input.in_bytes()); + output.set_out_bytes(input.out_bytes()); + output.set_hash(input.hash()); + output.set_encoder_version(static_cast(input.encoder_version())); + output.is_shape_ = is_shape; + switch (input.codec().codec_case()) { + case arcticdb::proto::encoding::VariantCodec::kZstd: { + set_codec(input.codec().zstd(), *output.mutable_codec()->mutable_zstd()); + break; + } + case arcticdb::proto::encoding::VariantCodec::kLz4: { + set_codec(input.codec().lz4(), *output.mutable_codec()->mutable_lz4()); + break; + } + case arcticdb::proto::encoding::VariantCodec::kPassthrough : { + set_codec(input.codec().passthrough(), *output.mutable_codec()->mutable_passthrough()); + break; + } + default: + util::raise_rte("Unrecognized_codec"); + } +} + +void proto_from_block(const EncodedBlock& input, arcticdb::proto::encoding::Block& output) { + output.set_in_bytes(input.in_bytes()); + output.set_out_bytes(input.out_bytes()); + output.set_hash(input.hash()); + output.set_encoder_version(input.encoder_version()); + + switch (input.codec().codec_) { + case Codec::ZSTD: { + set_zstd(input.codec().zstd(), *output.mutable_codec()->mutable_zstd()); + break; + } + case Codec::LZ4: { + set_lz4(input.codec().lz4(), *output.mutable_codec()->mutable_lz4()); + break; + } + case Codec::PASS: { + set_passthrough(input.codec().passthrough(), *output.mutable_codec()->mutable_passthrough()); + break; + } + default: + util::raise_rte("Unrecognized_codec"); + } +} + +void encoded_field_from_proto(const arcticdb::proto::encoding::EncodedField& input, EncodedFieldImpl& output) { + util::check(input.has_ndarray(), "Only ndarray fields supported for v1 encoding"); + const auto& input_ndarray = input.ndarray(); + auto* output_ndarray = output.mutable_ndarray(); + output_ndarray->set_items_count(input_ndarray.items_count()); + output_ndarray->set_sparse_map_bytes(input_ndarray.sparse_map_bytes()); + + for(auto i = 0; i < input_ndarray.shapes_size(); ++i) { + auto* shape_block = output_ndarray->add_shapes(); + block_from_proto(input_ndarray.shapes(i), *shape_block, true); + } + + for(auto i = 0; i < input_ndarray.values_size(); ++i) { + auto* value_block = output_ndarray->add_values(EncodingVersion::V1); + block_from_proto(input_ndarray.values(i), *value_block, false); + } +} + +void copy_encoded_field_to_proto(const EncodedFieldImpl& input, arcticdb::proto::encoding::EncodedField& output) { + util::check(input.has_ndarray(), "Only ndarray fields supported for v1 encoding"); + const auto& input_ndarray = input.ndarray(); + auto* output_ndarray = output.mutable_ndarray(); + output_ndarray->set_items_count(input_ndarray.items_count()); + output_ndarray->set_sparse_map_bytes(input_ndarray.sparse_map_bytes()); + + for(auto i = 0; i < input_ndarray.shapes_size(); ++i) { + auto* shape_block = output_ndarray->add_shapes(); + proto_from_block(input_ndarray.shapes(i), *shape_block); + } + + for(auto i = 0; i < input_ndarray.values_size(); ++i) { + auto* value_block = output_ndarray->add_values(); + proto_from_block(input_ndarray.values(i), *value_block); + } +} + +size_t num_blocks(const arcticdb::proto::encoding::EncodedField& field) { + util::check(field.has_ndarray(), "Expected ndarray in segment header"); + return field.ndarray().shapes_size() + field.ndarray().values_size(); +} + +SegmentHeader deserialize_segment_header_from_proto(const arcticdb::proto::encoding::SegmentHeader& header) { + SegmentHeader output; + output.set_encoding_version(EncodingVersion(header.encoding_version())); + output.set_compacted(header.compacted()); + + if(header.has_metadata_field()) + encoded_field_from_proto(header.metadata_field(), output.mutable_metadata_field(num_blocks(header.metadata_field()))); + + if(header.has_string_pool_field()) + encoded_field_from_proto(header.string_pool_field(), output.mutable_string_pool_field(num_blocks(header.string_pool_field()))); + + auto fields_from_proto = encoded_fields_from_proto(header); + output.set_body_fields(std::move(fields_from_proto)); + return output; +} + +size_t calc_proto_encoded_blocks_size(const arcticdb::proto::encoding::SegmentHeader& hdr) { + size_t bytes{}; + for(const auto& field : hdr.fields()) { + bytes += EncodedFieldImpl::Size; + if(field.has_ndarray()) { + const auto& ndarray = field.ndarray(); + const auto shapes_size = sizeof(EncodedBlock) * ndarray.shapes_size(); + const auto values_size = sizeof(EncodedBlock) * ndarray.values_size(); + bytes += shapes_size + values_size; + } + } + return bytes; +} + +EncodedFieldCollection encoded_fields_from_proto(const arcticdb::proto::encoding::SegmentHeader& hdr) { + const auto encoded_buffer_size = calc_proto_encoded_blocks_size(hdr); + EncodedFieldCollection encoded_fields(encoded_buffer_size, hdr.fields_size()); + auto buffer = ChunkedBuffer::presized(encoded_buffer_size); + for(auto&& [index, in_field] : folly::enumerate(hdr.fields())) { + auto* out_field = encoded_fields.add_field(num_blocks(in_field)); + encoded_field_from_proto(in_field, *out_field); + } + return encoded_fields; +} + +void copy_encoded_fields_to_proto(const EncodedFieldCollection& fields, arcticdb::proto::encoding::SegmentHeader& hdr) { + auto& proto_fields = *hdr.mutable_fields(); + auto field = fields.begin(); + for(auto i = 0U; i < fields.size(); ++i) { + auto* proto_field = proto_fields.Add(); + copy_encoded_field_to_proto(field.current(), *proto_field); + ++field; + } +} + +} //namespace arcticdb diff --git a/cpp/arcticdb/codec/protobuf_mappings.hpp b/cpp/arcticdb/codec/protobuf_mappings.hpp new file mode 100644 index 0000000000..8d29421369 --- /dev/null +++ b/cpp/arcticdb/codec/protobuf_mappings.hpp @@ -0,0 +1,73 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace arcticdb { + +template +void copy_codec(T& out_codec, const U& in_codec) { + out_codec.MergeFrom(in_codec); +} + +inline void copy_codec(ZstdCodec& codec, const arcticdb::proto::encoding::VariantCodec::Zstd& zstd) { + codec.level_ = zstd.level(); + codec.is_streaming_ = zstd.is_streaming(); +} + +inline void copy_codec(Lz4Codec& codec, const arcticdb::proto::encoding::VariantCodec::Lz4& lz4) { + codec.acceleration_ = lz4.acceleration(); +} + +inline void copy_codec(PassthroughCodec&, const arcticdb::proto::encoding::VariantCodec::Passthrough&) { + // No data in passthrough +} + +[[nodiscard]] inline arcticdb::proto::encoding::VariantCodec::CodecCase codec_case(Codec codec) { + switch (codec) { + case Codec::ZSTD:return arcticdb::proto::encoding::VariantCodec::kZstd; + case Codec::LZ4:return arcticdb::proto::encoding::VariantCodec::kLz4; + case Codec::PFOR:return arcticdb::proto::encoding::VariantCodec::kTp4; + case Codec::PASS:return arcticdb::proto::encoding::VariantCodec::kPassthrough; + default:util::raise_rte("Unknown codec"); + } +} + +template +void set_codec(Input& in, Output& out) { + copy_codec(out, in); +} + +void block_from_proto(const arcticdb::proto::encoding::Block& input, EncodedBlock& output, bool is_shape); + +inline void set_lz4(const Lz4Codec& lz4_in, arcticdb::proto::encoding::VariantCodec::Lz4& lz4_out) { + lz4_out.set_acceleration(lz4_in.acceleration_); +} + +inline void set_zstd(const ZstdCodec& zstd_in, arcticdb::proto::encoding::VariantCodec::Zstd& zstd_out) { + zstd_out.set_is_streaming(zstd_in.is_streaming_); + zstd_out.set_level(zstd_in.level_); +} + +inline void set_passthrough(const PassthroughCodec& passthrough_in, arcticdb::proto::encoding::VariantCodec::Passthrough& passthrough_out) { + passthrough_out.set_mark(passthrough_in.unused_); +} + +void proto_from_block(const EncodedBlock& input, arcticdb::proto::encoding::Block& output); + +void encoded_field_from_proto(const arcticdb::proto::encoding::EncodedField& input, EncodedFieldImpl& output); + +void copy_encoded_field_to_proto(const EncodedFieldImpl& input, arcticdb::proto::encoding::EncodedField& output); + +SegmentHeader deserialize_segment_header_from_proto(const arcticdb::proto::encoding::SegmentHeader& header); + +size_t calc_proto_encoded_blocks_size(const arcticdb::proto::encoding::SegmentHeader& hdr); + +EncodedFieldCollection encoded_fields_from_proto(const arcticdb::proto::encoding::SegmentHeader& hdr); + +void copy_encoded_fields_to_proto(const EncodedFieldCollection& fields, arcticdb::proto::encoding::SegmentHeader& hdr); + +} //namespace arcticdb \ No newline at end of file diff --git a/cpp/arcticdb/codec/python_bindings.cpp b/cpp/arcticdb/codec/python_bindings.cpp index eeddaffd12..403ce2e783 100644 --- a/cpp/arcticdb/codec/python_bindings.cpp +++ b/cpp/arcticdb/codec/python_bindings.cpp @@ -102,7 +102,7 @@ Segment encode_segment(SegmentInMemory segment_in_memory, const py::object &opts return encode_dispatch(std::move(segment_in_memory), opts_cpp, encoding_version); } -SegmentInMemory decode_python_segment(Segment segment) { +SegmentInMemory decode_python_segment(Segment& segment) { return decode_segment(std::move(segment)); } @@ -188,8 +188,8 @@ void register_codec(py::module &m) { .def("fields_size", &Segment::fields_size) .def("fields", &Segment::fields_vector) .def_property_readonly("header", [](const Segment& self) { - return pb_to_python(self.header()); - }) + return self.header().clone(); + }, py::return_value_policy::move) .def_property_readonly("bytes", [](const Segment& self) { return py::bytes(reinterpret_cast(self.buffer().data()), self.buffer().bytes()); }); diff --git a/cpp/arcticdb/codec/segment.cpp b/cpp/arcticdb/codec/segment.cpp index 5eeab44b0d..857953e607 100644 --- a/cpp/arcticdb/codec/segment.cpp +++ b/cpp/arcticdb/codec/segment.cpp @@ -8,247 +8,341 @@ #include #include #include +#include -#include #include #include -#include +#include namespace arcticdb { + +arcticdb::proto::encoding::SegmentHeader generate_proto_header(const SegmentHeader& header, const StreamDescriptor& desc) { + arcticdb::proto::encoding::SegmentHeader segment_header; + if(header.has_metadata_field()) + copy_encoded_field_to_proto(header.metadata_field(), *segment_header.mutable_metadata_field()); + + if(header.has_string_pool_field()) + copy_encoded_field_to_proto(header.string_pool_field(), *segment_header.mutable_string_pool_field()); + + copy_stream_descriptor_to_proto(desc, *segment_header.mutable_stream_descriptor()); + copy_encoded_fields_to_proto(header.body_fields(), segment_header); + + segment_header.set_compacted(header.compacted()); + segment_header.set_encoding_version(static_cast(header.encoding_version())); + ARCTICDB_TRACE(log::codec(), "Encoded segment header {}", segment_header.DebugString()); + return segment_header; +} + namespace segment_size { -std::tuple compressed(const arcticdb::proto::encoding::SegmentHeader &seg_hdr) { + +SegmentCompressedSize compressed(const SegmentHeader &seg_hdr, const std::optional& proto_wrapper) { size_t string_pool_size = 0; if (seg_hdr.has_string_pool_field()) string_pool_size = encoding_sizes::ndarray_field_compressed_size(seg_hdr.string_pool_field().ndarray()); - std::size_t buffer_size = 0; - if (EncodingVersion(seg_hdr.encoding_version()) == EncodingVersion::V1) { - size_t metadata_size = 0; - // If we have metadata it is part of the buffer size, otherwise the allocated buffer is much too small - if (seg_hdr.has_metadata_field()) - metadata_size = encoding_sizes::ndarray_field_compressed_size(seg_hdr.metadata_field().ndarray()); - - buffer_size = encoding_sizes::segment_compressed_size(seg_hdr.fields()) + metadata_size + string_pool_size; + size_t metadata_size = 0; + if (seg_hdr.has_metadata_field()) + metadata_size = encoding_sizes::ndarray_field_compressed_size(seg_hdr.metadata_field().ndarray()); + + size_t buffer_size; + size_t body_size; + if(seg_hdr.encoding_version() == EncodingVersion::V1) { + const auto fields_size = encoding_sizes::segment_compressed_size(proto_wrapper->proto().fields()); + ARCTICDB_DEBUG(log::codec(), "Calculating total size: {} fields + {} metadata + {} string pool = {}", fields_size, metadata_size, string_pool_size, fields_size + metadata_size + string_pool_size); + buffer_size = fields_size + metadata_size + string_pool_size; + body_size = buffer_size; + } else { + buffer_size = seg_hdr.footer_offset() + sizeof(EncodedMagic) + encoding_sizes::ndarray_field_compressed_size(seg_hdr.column_fields().ndarray()); + body_size = seg_hdr.footer_offset(); } - else - buffer_size = seg_hdr.column_fields().offset() + sizeof(EncodedMagic) + encoding_sizes::ndarray_field_compressed_size(seg_hdr.column_fields().ndarray()); - return {string_pool_size, buffer_size}; + return {string_pool_size, buffer_size, body_size}; } } -FieldCollection decode_fields( - const arcticdb::proto::encoding::SegmentHeader& hdr, - const uint8_t* data) { - const auto begin ARCTICDB_UNUSED = data; +FieldCollection decode_descriptor_fields( + const SegmentHeader& hdr, + const uint8_t* data, + const uint8_t* begin) { FieldCollection fields; if (hdr.has_descriptor_field()) { - ARCTICDB_TRACE(log::codec(), "Decoding string pool"); + ARCTICDB_TRACE(log::codec(), "Decoding descriptor"); std::optional bv; - data += decode_field(FieldCollection::type(), + util::check(hdr.descriptor_field().has_ndarray(), "Expected descriptor field to be ndarray"); + (void)decode_ndarray(FieldCollection::type(), hdr.descriptor_field(), data, fields, bv, - to_encoding_version(hdr.encoding_version())); + hdr.encoding_version()); - ARCTICDB_TRACE(log::codec(), "Decoded string pool to position {}", data-begin); + ARCTICDB_TRACE(log::codec(), "Decoded descriptor to position {}", data-begin); } fields.regenerate_offsets(); return fields; } -std::optional decode_index_fields( - const arcticdb::proto::encoding::SegmentHeader& hdr, - const uint8_t*& data, - const uint8_t* const begin ARCTICDB_UNUSED - ) { - if(hdr.has_index_descriptor_field()) { - FieldCollection fields; - ARCTICDB_TRACE(log::codec(), "Decoding string pool"); - std::optional bv; - data += decode_field(FieldCollection::type(), - hdr.index_descriptor_field(), - data, - fields, - bv, - to_encoding_version(hdr.encoding_version())); - ARCTICDB_TRACE(log::codec(), "Decoded string pool to position {}", data-begin); - return std::make_optional(std::move(fields)); +SegmentHeaderProtoWrapper decode_protobuf_header(const uint8_t* data, size_t header_bytes_size) { + google::protobuf::io::ArrayInputStream ais(data, static_cast(header_bytes_size)); + + auto arena = std::make_unique(); + auto seg_hdr = google::protobuf::Arena::CreateMessage(arena.get()); + seg_hdr->ParseFromZeroCopyStream(&ais); + log::codec().info("Serialized header bytes output: {}", dump_bytes(data, header_bytes_size)); + ARCTICDB_TRACE(log::codec(), "Decoded protobuf header: {}", seg_hdr->DebugString()); + return {seg_hdr, std::move(arena)}; +} + +void skip_metadata_field(const uint8_t*& src, const SegmentHeader& seg_hdr) { + util::check_magic(src); + if(seg_hdr.has_metadata_field()) { + const auto metadata_size = encoding_sizes::field_compressed_size(seg_hdr.metadata_field()); + log::codec().info("Skipping {} bytes of metadata", metadata_size); + src += metadata_size; } else { - return std::nullopt; + log::codec().info("No metadata to skip"); } } -Segment Segment::from_bytes(const std::uint8_t* src, std::size_t readable_size, bool copy_data /* = false */) { - ARCTICDB_SAMPLE(SegmentFromBytes, 0) - auto* fixed_hdr = reinterpret_cast(src); - util::check_arg(fixed_hdr->magic_number == MAGIC_NUMBER, "expected first 2 bytes: {}, actual {}", fixed_hdr->magic_number, MAGIC_NUMBER); +FieldCollection deserialize_descriptor_fields_collection(const uint8_t* src, const SegmentHeader& seg_hdr) { + FieldCollection fields; + util::check_magic(src); + if(seg_hdr.has_descriptor_field() && seg_hdr.descriptor_field().has_ndarray()) + fields = decode_descriptor_fields(seg_hdr, src, src); - ARCTICDB_SUBSAMPLE(ReadHeaderAndSegment, 0) - auto header_bytes ARCTICDB_UNUSED = arcticdb::Segment::FIXED_HEADER_SIZE + fixed_hdr->header_bytes; - ARCTICDB_DEBUG(log::codec(), "Reading header: {} + {} = {}", - arcticdb::Segment::FIXED_HEADER_SIZE, - fixed_hdr->header_bytes, - header_bytes); - google::protobuf::io::ArrayInputStream ais(src + arcticdb::Segment::FIXED_HEADER_SIZE, static_cast(fixed_hdr->header_bytes)); - auto arena = std::make_unique(); - auto seg_hdr = google::protobuf::Arena::CreateMessage(arena.get()); - seg_hdr->ParseFromZeroCopyStream(&ais); - if(!seg_hdr->has_metadata_field()) - ARCTICDB_DEBUG(log::storage(), "Segment has no medatadata field"); + return fields; +} +EncodedFieldCollection deserialize_body_fields(const SegmentHeader& hdr, const uint8_t* data) { + const auto* encoded_fields_ptr = data; + util::check(hdr.has_column_fields(), "Expected column fields in v2 encoding"); + util::check_magic(encoded_fields_ptr); - src += arcticdb::Segment::FIXED_HEADER_SIZE + fixed_hdr->header_bytes; + return EncodedFieldCollection{decode_encoded_fields(hdr, encoded_fields_ptr, data)}; +} - auto version = EncodingVersion(seg_hdr->encoding_version()); - util::check(version == EncodingVersion::V1 || version == EncodingVersion::V2 , - "expected encoding_version < 2, actual {}", - seg_hdr->encoding_version()); +struct DeserializedSegmentData { + SegmentHeader segment_header_; + std::shared_ptr fields_; + std::shared_ptr segment_desc_; + std::optional proto_wrapper_; + StreamId stream_id_; +}; - FieldCollection fields; - if(version == EncodingVersion::V1) - fields = fields_from_proto(seg_hdr->stream_descriptor()); - else { - const auto* fields_ptr = src; - util::check_magic(fields_ptr); - if(seg_hdr->has_metadata_field()) - fields_ptr += encoding_sizes::field_compressed_size(seg_hdr->metadata_field()); - - util::check_magic(fields_ptr); - if(seg_hdr->has_descriptor_field() && seg_hdr->descriptor_field().has_ndarray()) - fields = decode_fields(*seg_hdr, fields_ptr); +DeserializedSegmentData decode_header_and_fields(const uint8_t*& src) { + auto* fixed_hdr = reinterpret_cast(src); + ARCTICDB_DEBUG(log::codec(), "Reading header: {} + {} = {}", + FIXED_HEADER_SIZE, + fixed_hdr->header_bytes, + FIXED_HEADER_SIZE + fixed_hdr->header_bytes); + + util::check_arg(fixed_hdr->magic_number == MAGIC_NUMBER, "expected first 2 bytes: {}, actual {}", fixed_hdr->magic_number, MAGIC_NUMBER); + std::optional proto_wrapper; + + const auto* header_ptr = src + FIXED_HEADER_SIZE; + if(const auto header_version = fixed_hdr->encoding_version; header_version == HEADER_VERSION_V1) { + proto_wrapper = decode_protobuf_header(header_ptr, fixed_hdr->header_bytes); + auto data = std::make_shared(segment_descriptor_from_proto(proto_wrapper->proto().stream_descriptor())); + auto segment_header = deserialize_segment_header_from_proto(proto_wrapper->proto()); + util::check(segment_header.encoding_version() == EncodingVersion::V1, "Expected v1 header to contain legacy encoding version"); + auto fields = std::make_shared(field_collection_from_proto(proto_wrapper->proto().stream_descriptor().fields())); + src += FIXED_HEADER_SIZE + fixed_hdr->header_bytes; + auto stream_id = stream_id_from_proto(proto_wrapper->proto().stream_descriptor()); + return {std::move(segment_header), std::move(fields), std::move(data), std::move(proto_wrapper), stream_id}; + } else { + SegmentHeader segment_header; + const auto* fields_ptr = header_ptr + fixed_hdr->header_bytes; + segment_header.deserialize_from_bytes(header_ptr); + skip_metadata_field(fields_ptr, segment_header); + auto segment_desc = std::make_shared(read_segment_descriptor(fields_ptr)); + auto stream_id = read_identifier(fields_ptr); + util::check(segment_header.encoding_version() == EncodingVersion::V2, "Expected V2 encoding in binary header"); + auto fields = std::make_shared(deserialize_descriptor_fields_collection(fields_ptr, segment_header)); + src += FIXED_HEADER_SIZE + fixed_hdr->header_bytes; + return {std::move(segment_header), std::move(fields), std::move(segment_desc), std::move(proto_wrapper), stream_id}; } +} + +void check_encoding(EncodingVersion encoding_version) { + util::check(encoding_version == EncodingVersion::V1 || encoding_version == EncodingVersion::V2 , + "expected encoding_version < 2, actual {}", + encoding_version); +} - const auto[string_pool_size, buffer_bytes] = segment_size::compressed(*seg_hdr); - ARCTICDB_DEBUG(log::codec(), "Reading string pool {} header {} + {} and buffer bytes {}", string_pool_size, arcticdb::Segment::FIXED_HEADER_SIZE, fixed_hdr->header_bytes, buffer_bytes); - util::check(arcticdb::Segment::FIXED_HEADER_SIZE + fixed_hdr->header_bytes + buffer_bytes <= readable_size, +void check_size(const FixedHeader* fixed_hdr, size_t buffer_bytes, size_t readable_size, size_t string_pool_size) { + util::check(FIXED_HEADER_SIZE + fixed_hdr->header_bytes + buffer_bytes <= readable_size, "Size disparity, fixed header size {} + variable header size {} + buffer size {} (string pool size {}) >= total size {}", - arcticdb::Segment::FIXED_HEADER_SIZE, + FIXED_HEADER_SIZE, fixed_hdr->header_bytes, buffer_bytes, string_pool_size, - readable_size - ); + readable_size); +} + +void set_body_fields(SegmentHeader& seg_hdr, const uint8_t* src) { + if(seg_hdr.has_column_fields()) { + auto encoded_fields = deserialize_body_fields(seg_hdr, src + seg_hdr.footer_offset()); + seg_hdr.set_body_fields(std::move(encoded_fields)); + } +} +Segment Segment::from_bytes(const std::uint8_t* src, std::size_t readable_size, bool copy_data /* = false */) { + ARCTICDB_SAMPLE(SegmentFromBytes, 0) + util::check(src != nullptr, "Got null data ptr from segment"); + auto* fixed_hdr = reinterpret_cast(src); + auto [seg_hdr, fields, desc_data, proto_wrapper, stream_id] = decode_header_and_fields(src); + check_encoding(seg_hdr.encoding_version()); + const auto[string_pool_size, buffer_bytes, body_bytes] = segment_size::compressed(seg_hdr, proto_wrapper); + check_size(fixed_hdr, buffer_bytes, readable_size, string_pool_size); + ARCTICDB_DEBUG(log::codec(), "Reading string pool {} header {} + {} and buffer bytes {}", string_pool_size, FIXED_HEADER_SIZE, fixed_hdr->header_bytes, buffer_bytes); ARCTICDB_SUBSAMPLE(CreateBufferView, 0) + VariantBuffer variant_buffer; if (copy_data) { auto buf = std::make_shared(); - buf->ensure(buffer_bytes); - memcpy(buf->data(), src, buffer_bytes); - return {std::move(arena), seg_hdr, std::move(buf), std::make_shared(std::move(fields))}; + buf->ensure(body_bytes); + memcpy(buf->data(), src, body_bytes); + variant_buffer = std::move(buf); } else { - BufferView bv{const_cast(src), buffer_bytes}; - return {std::move(arena), seg_hdr, std::move(bv), std::make_shared(std::move(fields))}; + variant_buffer = BufferView{const_cast(src), buffer_bytes}; } -} + set_body_fields(seg_hdr, src); + return {std::move(seg_hdr), std::move(variant_buffer), std::move(desc_data), std::move(fields), stream_id, readable_size}; +} -Segment Segment::from_buffer(std::shared_ptr&& buffer) { - ARCTICDB_SAMPLE(SegmentFromBytes, 0) - auto* fixed_hdr = reinterpret_cast(buffer->data()); +Segment Segment::from_buffer(const std::shared_ptr& buffer) { + ARCTICDB_SAMPLE(SegmentFromBuffer, 0) + auto* fixed_hdr = reinterpret_cast(buffer->data()); auto readable_size = buffer->bytes(); - util::check_arg(fixed_hdr->magic_number == MAGIC_NUMBER, "expected first 2 bytes: {}, actual {}", - MAGIC_NUMBER, fixed_hdr->magic_number); - util::check_arg(fixed_hdr->encoding_version == HEADER_VERSION_V1, - "expected encoding_version {}, actual {}", - HEADER_VERSION_V1 , fixed_hdr->encoding_version); + const auto* src = buffer->data(); + auto [seg_hdr, fields, desc_data, proto_wrapper, stream_id] = decode_header_and_fields(src); + check_encoding(seg_hdr.encoding_version()); ARCTICDB_SUBSAMPLE(ReadHeaderAndSegment, 0) - auto header_bytes ARCTICDB_UNUSED = arcticdb::Segment::FIXED_HEADER_SIZE + fixed_hdr->header_bytes; + auto header_bytes ARCTICDB_UNUSED = FIXED_HEADER_SIZE + fixed_hdr->header_bytes; ARCTICDB_DEBUG(log::codec(), "Reading header: {} + {} = {}", - arcticdb::Segment::FIXED_HEADER_SIZE, + FIXED_HEADER_SIZE, fixed_hdr->header_bytes, header_bytes); - google::protobuf::io::ArrayInputStream ais(buffer->data() + arcticdb::Segment::FIXED_HEADER_SIZE, fixed_hdr->header_bytes); - auto arena = std::make_unique(); - auto seg_hdr = google::protobuf::Arena::CreateMessage(arena.get()); - seg_hdr->ParseFromZeroCopyStream(&ais); - const auto[string_pool_size, buffer_bytes] = segment_size::compressed(*seg_hdr); + const auto[string_pool_size, buffer_bytes, body_bytes] = segment_size::compressed(seg_hdr, proto_wrapper); ARCTICDB_DEBUG(log::codec(), "Reading string pool {} and buffer bytes {}", string_pool_size, buffer_bytes); - util::check(arcticdb::Segment::FIXED_HEADER_SIZE + fixed_hdr->header_bytes + buffer_bytes <= readable_size, - "Size disparity, fixed header size {} + variable header size {} + buffer size {} (string pool size {}) >= total size {}", - arcticdb::Segment::FIXED_HEADER_SIZE, - fixed_hdr->header_bytes, - buffer_bytes, - string_pool_size, - readable_size - ); - - auto version = EncodingVersion(seg_hdr->encoding_version()); - util::check(version == EncodingVersion::V1 || version == EncodingVersion::V2, - "expected encoding_version < 2, actual {}", - seg_hdr->encoding_version()); + check_size(fixed_hdr, buffer_bytes, readable_size, string_pool_size); + set_body_fields(seg_hdr, src); + buffer->set_preamble(FIXED_HEADER_SIZE + fixed_hdr->header_bytes); + ARCTICDB_SUBSAMPLE(CreateSegment, 0) + return{std::move(seg_hdr), buffer, std::move(desc_data), std::move(fields), stream_id, readable_size}; +} - auto preamble_size = arcticdb::Segment::FIXED_HEADER_SIZE + fixed_hdr->header_bytes; +size_t Segment::write_proto_header(uint8_t* dst) { + const auto& header = generate_header_proto(); + const auto hdr_size = proto_size(); + FixedHeader hdr = {MAGIC_NUMBER, HEADER_VERSION_V1, std::uint32_t(hdr_size)}; + write_fixed_header(dst, hdr); - FieldCollection fields; - if(version == EncodingVersion::V1) { - fields = fields_from_proto(seg_hdr->stream_descriptor()); - } - else { - const auto* fields_ptr = buffer->data() + preamble_size; - util::check_magic(fields_ptr); - if(seg_hdr->has_metadata_field()) - fields_ptr += encoding_sizes::field_compressed_size(seg_hdr->metadata_field()); - - util::check_magic(fields_ptr); - if(seg_hdr->has_descriptor_field() && seg_hdr->descriptor_field().has_ndarray()) - fields = decode_fields(*seg_hdr, fields_ptr); - } + google::protobuf::io::ArrayOutputStream aos(dst + FIXED_HEADER_SIZE, static_cast(hdr_size)); + header.SerializeToZeroCopyStream(&aos); + log::codec().info("Serialized header bytes: {}", dump_bytes(dst + FIXED_HEADER_SIZE, hdr_size)); + return hdr_size; +} - buffer->set_preamble(arcticdb::Segment::FIXED_HEADER_SIZE + fixed_hdr->header_bytes); - ARCTICDB_SUBSAMPLE(CreateSegment, 0) - return{std::move(arena), seg_hdr, std::move(buffer), std::make_shared(std::move(fields))}; +size_t Segment::write_binary_header(uint8_t* dst) const { + auto bytes_written = header_.serialize_to_bytes(dst + sizeof(FixedHeader)); + FixedHeader hdr = {MAGIC_NUMBER, HEADER_VERSION_V2, std::uint32_t(bytes_written)}; + write_fixed_header(dst, hdr); + return bytes_written; +} +std::pair Segment::serialize_header_v2(size_t expected_bytes) { + const auto header_bytes = header_.bytes() + sizeof(FixedHeader); + FixedHeader hdr = {MAGIC_NUMBER, HEADER_VERSION_V2, std::uint32_t(expected_bytes)}; + util::check(header_bytes == buffer_.preamble_bytes(), "Expected v2 header of size {} to fit exactly into buffer preamble of size {}", header_.bytes(), buffer_.preamble_bytes()); + const auto &buffer = buffer_.get_owning_buffer(); + auto* dst = buffer->preamble(); + write_fixed_header(dst, hdr); + header_.serialize_to_bytes(dst + FIXED_HEADER_SIZE, expected_bytes); + return std::make_pair(buffer->preamble(), calculate_size()); } -void Segment::write_header(uint8_t* dst, size_t hdr_size) const { - FixedHeader hdr = {MAGIC_NUMBER, HEADER_VERSION_V1, std::uint32_t(hdr_size)}; - hdr.write(dst); - if(!header_->has_metadata_field()) - ARCTICDB_DEBUG(log::codec(), "Expected metadata field"); +std::pair Segment::serialize_v1_header_in_place(size_t total_hdr_size) { + const auto &buffer = buffer_.get_owning_buffer(); + auto base_ptr = buffer->preamble() + (buffer->preamble_bytes() - total_hdr_size); + util::check(base_ptr + total_hdr_size == buffer->data(), "Expected base ptr to align with data ptr, {} != {}",fmt::ptr(base_ptr + total_hdr_size),fmt::ptr(buffer->data())); + write_proto_header(base_ptr); + ARCTICDB_TRACE(log::storage(), "Header fits in internal buffer {:x} with {} bytes space: {}", intptr_t (base_ptr), buffer->preamble_bytes() - total_hdr_size,dump_bytes(buffer->data(), buffer->bytes(), 100u)); + return std::make_pair(base_ptr, calculate_size()); +} - google::protobuf::io::ArrayOutputStream aos(dst + FIXED_HEADER_SIZE, static_cast(hdr_size)); - header_->SerializeToZeroCopyStream(&aos); +std::tuple> Segment::serialize_v1_header_to_buffer(size_t hdr_size) { + auto tmp = std::make_unique(); + ARCTICDB_DEBUG(log::storage(), "Header doesn't fit in internal buffer, needed {} bytes but had {}, writing to temp buffer at {:x}", hdr_size, buffer_.preamble_bytes(),uintptr_t(tmp->data())); + tmp->ensure(calculate_size()); + auto* dst = tmp->preamble(); + write_proto_header(dst); + std::memcpy(dst + FIXED_HEADER_SIZE + hdr_size, + buffer().data(), + buffer().bytes()); + return std::make_tuple(tmp->preamble(), calculate_size(), std::move(tmp)); } -std::pair Segment::try_internal_write(std::shared_ptr& tmp, size_t hdr_size) { +std::tuple> Segment::serialize_header_v1() { + auto proto_header = generate_proto_header(header_, desc_); + const auto hdr_size = proto_header.ByteSizeLong(); auto total_hdr_size = hdr_size + FIXED_HEADER_SIZE; - if(std::holds_alternative>(buffer_) && std::get>(buffer_)->preamble_bytes() >= total_hdr_size) { - auto& buffer = std::get>(buffer_); - auto base_ptr = buffer->preamble() + (buffer->preamble_bytes() - total_hdr_size); - util::check(base_ptr + total_hdr_size == buffer->data(), "Expected base ptr to align with data ptr, {} != {}", fmt::ptr(base_ptr + total_hdr_size), fmt::ptr(buffer->data())); - ARCTICDB_TRACE(log::codec(), "Buffer contents before header write: {}", dump_bytes(buffer->data(), buffer->bytes(), 100u)); - write_header(base_ptr, hdr_size); - ARCTICDB_TRACE(log::storage(), "Header fits in internal buffer {:x} with {} bytes space: {}", uintptr_t (base_ptr), buffer->preamble_bytes() - total_hdr_size, dump_bytes(buffer->data(), buffer->bytes(), 100u)); - return std::make_pair(base_ptr, total_segment_size(hdr_size)); + + if (buffer_.is_owning() && buffer_.preamble_bytes() >= total_hdr_size) { + auto [dst, size] = serialize_v1_header_in_place(total_hdr_size); + return std::make_tuple(dst, size, std::unique_ptr()); + } else { + return serialize_v1_header_to_buffer(hdr_size); } +} + +std::tuple> Segment::serialize_header() { + if (header_.encoding_version() == EncodingVersion::V1) + return serialize_header_v1(); else { - tmp = std::make_shared(); - ARCTICDB_DEBUG(log::storage(), "Header doesn't fit in internal buffer, needed {} bytes but had {}, writing to temp buffer at {:x}", hdr_size, std::get>(buffer_)->preamble_bytes(), uintptr_t(tmp->data())); - tmp->ensure(total_segment_size(hdr_size)); - write_to(tmp->preamble(), hdr_size); - return std::make_pair(tmp->preamble(), total_segment_size(hdr_size)); + auto [dst, size] = serialize_header_v2(buffer_.preamble_bytes() - FIXED_HEADER_SIZE); + return std::make_tuple(dst, size, std::unique_ptr()); } } -void Segment::write_to(std::uint8_t* dst, std::size_t hdr_sz) { +[[nodiscard]] std::shared_ptr Segment::fields_ptr() const { + return desc_.fields_ptr(); +} + +[[nodiscard]] size_t Segment::fields_size() const { + return desc_.field_count(); +} + +[[nodiscard]] const Field& Segment::fields(size_t pos) const { + return desc_.fields(pos); +} + +const arcticdb::proto::encoding::SegmentHeader& Segment::generate_header_proto() { + if(!proto_) + proto_ = std::make_unique(generate_proto_header(header_, desc_)); + + return *proto_; +} + +void Segment::write_to(std::uint8_t* dst) { ARCTICDB_SAMPLE(SegmentWriteToStorage, RMTSF_Aggregate) ARCTICDB_SUBSAMPLE(SegmentWriteHeader, RMTSF_Aggregate) - write_header(dst, hdr_sz); - ARCTICDB_SUBSAMPLE(SegmentWriteBody, RMTSF_Aggregate) - ARCTICDB_DEBUG(log::codec(), "Writing {} bytes to body at offset {}", - buffer().bytes(), - arcticdb::Segment::FIXED_HEADER_SIZE + hdr_sz); - std::memcpy(dst + arcticdb::Segment::FIXED_HEADER_SIZE + hdr_sz, - buffer().data(), - buffer().bytes()); + size_t header_size = 0U; + if(header_.encoding_version() == EncodingVersion::V1) + header_size = write_proto_header(dst); + else + header_size = write_binary_header(dst); + + ARCTICDB_SUBSAMPLE(SegmentWriteBody, RMTSF_Aggregate) + ARCTICDB_DEBUG(log::codec(), "Writing {} bytes to body at offset {}", buffer().bytes(), FIXED_HEADER_SIZE + header_size); + std::memcpy(dst + FIXED_HEADER_SIZE + header_size, buffer().data(), buffer().bytes()); + log::codec().info("Wrote segment {} header {} body ({} bytes)", header_size + FIXED_HEADER_SIZE, buffer().bytes(), header_size + buffer().bytes() + FIXED_HEADER_SIZE); } } //namespace arcticdb diff --git a/cpp/arcticdb/codec/segment.hpp b/cpp/arcticdb/codec/segment.hpp index 2e8b344843..b8d42ca484 100644 --- a/cpp/arcticdb/codec/segment.hpp +++ b/cpp/arcticdb/codec/segment.hpp @@ -7,13 +7,14 @@ #pragma once -#include "util/buffer.hpp" #include -#include +#include +#include +#include +#include + #include #include -#include -#include #include #include @@ -22,31 +23,40 @@ namespace arcticdb { namespace segment_size { -std::tuple compressed(const arcticdb::proto::encoding::SegmentHeader& seg_hdr); -} -enum class EncodingVersion : uint16_t { - V1 = 0, - V2 = 1, - COUNT +struct SegmentCompressedSize { + size_t string_pool_size_ = 0U; + size_t total_buffer_size_ = 0U; + size_t body_size_ = 0U; }; -template>> -inline constexpr EncodingVersion to_encoding_version(T encoding_version) { - util::check(encoding_version >= 0 && encoding_version < uint16_t(EncodingVersion::COUNT), "Invalid encoding version"); - return static_cast(encoding_version); +SegmentCompressedSize compressed(const arcticdb::proto::encoding::SegmentHeader& seg_hdr); } +struct SegmentHeaderProtoWrapper { + arcticdb::proto::encoding::SegmentHeader* header_; + std::unique_ptr arena_; + + [[nodiscard]] const auto& proto() const { return *header_; } + + [[nodiscard]] auto& proto() { return *header_; } +}; + +SegmentHeaderProtoWrapper decode_protobuf_header(const uint8_t* data, size_t header_bytes_size); + +arcticdb::proto::encoding::SegmentHeader generate_proto_header(const SegmentHeader& header, const StreamDescriptor& desc); + static constexpr uint16_t HEADER_VERSION_V1 = 1; +static constexpr uint16_t HEADER_VERSION_V2 = 2; inline EncodingVersion encoding_version(const storage::LibraryDescriptor::VariantStoreConfig& cfg) { return util::variant_match(cfg, - [](const arcticdb::proto::storage::VersionStoreConfig &version_config) { - return EncodingVersion(version_config.encoding_version()); - }, - [](std::monostate) { - return EncodingVersion::V1; - } + [](const arcticdb::proto::storage::VersionStoreConfig &version_config) { + return EncodingVersion(version_config.encoding_version()); + }, + [](std::monostate) { + return EncodingVersion::V1; + } ); } @@ -58,98 +68,69 @@ inline EncodingVersion encoding_version(const storage::LibraryDescriptor::Varian */ class Segment { public: - constexpr static uint16_t MAGIC_NUMBER = 0xFA57; - - struct FixedHeader { - std::uint16_t magic_number; - std::uint16_t encoding_version; - std::uint32_t header_bytes; - - void write(std::uint8_t *dst) const { - ARCTICDB_DEBUG(log::codec(), "Writing header with size {}", header_bytes); - auto h = reinterpret_cast(dst); - *h = *this; - } - - void write(std::ostream &dst){ - dst.write(reinterpret_cast(this), sizeof(FixedHeader)); - } - }; - - constexpr static std::size_t FIXED_HEADER_SIZE = sizeof(FixedHeader); - - Segment() : - header_(google::protobuf::Arena::CreateMessage(arena_.get())) { - } - - Segment(std::unique_ptr&& arena, arcticdb::proto::encoding::SegmentHeader* header, std::shared_ptr buffer, std::shared_ptr fields) : - arena_(std::move(arena)), - header_(header), - buffer_(std::move(buffer)), - fields_(std::move(fields)){ + Segment() = default; + + Segment( + SegmentHeader&& header, + std::shared_ptr buffer, + std::shared_ptr data, + std::shared_ptr fields, + StreamId stream_id, + size_t size) : + header_(std::move(header)), + buffer_(std::move(buffer)), + desc_(std::move(data), std::move(fields), std::move(stream_id)), + size_(size) { } - Segment(std::unique_ptr&& arena, arcticdb::proto::encoding::SegmentHeader* header, BufferView &&buffer, std::shared_ptr fields) : - arena_(std::move(arena)), - header_(header), - buffer_(buffer), - fields_(std::move(fields)){} - - // for rvo only, go to solution should be to move - Segment(const Segment &that) : - header_(google::protobuf::Arena::CreateMessage(arena_.get())), - keepalive_(that.keepalive_) { - header_->CopyFrom(*that.header_); - auto b = std::make_shared(); - util::variant_match(that.buffer_, - [] (const std::monostate&) {/* Uninitialized buffer */}, - [&b](const BufferView& buf) { buf.copy_to(*b); }, - [&b](const std::shared_ptr& buf) { buf->copy_to(*b); } - ); - buffer_ = std::move(b); - if(that.fields_) - fields_ = std::make_shared(that.fields_->clone()); + Segment( + SegmentHeader&& header, + BufferView buffer, + std::shared_ptr data, + std::shared_ptr fields, + StreamId stream_id, + const std::optional& size) : + header_(std::move(header)), + buffer_(buffer), + desc_(std::move(data), std::move(fields), std::move(stream_id)), + size_(size) { } - Segment &operator=(const Segment &that) { - if(this == &that) - return *this; - - header_->CopyFrom(*that.header_); - auto b = std::make_shared(); - util::variant_match(that.buffer_, - [] (const std::monostate&) {/* Uninitialized buffer */}, - [&b](const BufferView& buf) { buf.copy_to(*b); }, - [&b](const std::shared_ptr& buf) { buf->copy_to(*b); } - ); - buffer_ = std::move(b); - fields_ = that.fields_; - keepalive_ = that.keepalive_; - return *this; + Segment( + SegmentHeader&& header, + VariantBuffer &&buffer, + std::shared_ptr data, + std::shared_ptr fields, + StreamId stream_id, + const std::optional size) : + header_(std::move(header)), + buffer_(std::move(buffer)), + desc_(std::move(data), std::move(fields), std::move(stream_id)), + size_(size) { } Segment(Segment &&that) noexcept { using std::swap; swap(header_, that.header_); - swap(arena_, that.arena_); - swap(fields_, that.fields_); + swap(desc_, that.desc_); swap(keepalive_, that.keepalive_); - move_buffer(std::move(that)); + swap(size_, that.size_); + buffer_.move_buffer(std::move(that.buffer_)); } Segment &operator=(Segment &&that) noexcept { using std::swap; swap(header_, that.header_); - swap(arena_, that.arena_); - swap(fields_, that.fields_); + swap(desc_, that.desc_); swap(keepalive_, that.keepalive_); - move_buffer(std::move(that)); + swap(size_, that.size_); + buffer_.move_buffer(std::move(that.buffer_)); return *this; } ~Segment() = default; - static Segment from_buffer(std::shared_ptr&& buf); + static Segment from_buffer(const std::shared_ptr& buf); void set_buffer(VariantBuffer&& buffer) { buffer_ = std::move(buffer); @@ -157,122 +138,117 @@ class Segment { static Segment from_bytes(const std::uint8_t *src, std::size_t readable_size, bool copy_data = false); - void write_to(std::uint8_t *dst, std::size_t hdr_sz); + void write_to(std::uint8_t *dst); - std::pair try_internal_write(std::shared_ptr& tmp, size_t hdr_size); + std::tuple> serialize_header(); - void write_header(uint8_t* dst, size_t hdr_size) const; + size_t write_proto_header(uint8_t* dst); - [[nodiscard]] std::size_t total_segment_size() const { - return total_segment_size(segment_header_bytes_size()); + [[nodiscard]] std::size_t size() const { + util::check(size_.has_value(), "Segment size has not been set"); + return *size_; } - [[nodiscard]] std::size_t total_segment_size(std::size_t hdr_size) const { - auto total = FIXED_HEADER_SIZE + hdr_size + buffer_bytes(); - ARCTICDB_TRACE(log::storage(), "Total segment size {} + {} + {} = {}", FIXED_HEADER_SIZE, hdr_size, buffer_bytes(), total); - return total; - } + [[nodiscard]] std::size_t calculate_size() { + if(!size_.has_value()) + size_ = FIXED_HEADER_SIZE + segment_header_bytes_size() + buffer_bytes(); - [[nodiscard]] std::size_t segment_header_bytes_size() const { - return header_->ByteSizeLong(); + return *size_; } - [[nodiscard]] std::size_t buffer_bytes() const { - std::size_t s = 0; - util::variant_match(buffer_, - [] (const std::monostate&) { /* Uninitialized buffer */}, - [&s](const BufferView& b) { s = b.bytes(); }, - [&s](const std::shared_ptr& b) { s = b->bytes(); }); + const arcticdb::proto::encoding::SegmentHeader& generate_header_proto(); - return s; - } + [[nodiscard]] size_t proto_size() { + util::check(static_cast(proto_), "Proto has not been generated"); - arcticdb::proto::encoding::SegmentHeader &header() { - return *header_; + return proto_->ByteSizeLong(); } - [[nodiscard]] const arcticdb::proto::encoding::SegmentHeader &header() const { - return *header_; + [[nodiscard]] std::size_t segment_header_bytes_size() { + if(header_.encoding_version() == EncodingVersion::V1) { + generate_header_proto(); + return proto_size(); + } + else + return header_.bytes(); } - [[nodiscard]] BufferView buffer() const { - if (std::holds_alternative>(buffer_)) { - return std::get>(buffer_)->view(); - } else { - return std::get(buffer_); - } + [[nodiscard]] std::size_t buffer_bytes() const { + return buffer_.bytes(); } - [[nodiscard]] bool is_uninitialized() const { - return std::holds_alternative(buffer_); + SegmentHeader &header() { + return header_; } - [[nodiscard]] bool is_empty() const { - return is_uninitialized() || (buffer().bytes() == 0 && header_->ByteSizeLong() == 0); + [[nodiscard]] const SegmentHeader &header() const { + return header_; } - [[nodiscard]] bool is_owning_buffer() const { - return std::holds_alternative>(buffer_); + [[nodiscard]] BufferView buffer() const { + return buffer_.view(); } - [[nodiscard]] std::shared_ptr fields_ptr() const { - return fields_; + [[nodiscard]] bool is_empty() const { + return buffer_.is_uninitialized() || (buffer().bytes() == 0 && header_.empty()); } - [[nodiscard]] size_t fields_size() const { - return fields_->size(); + [[nodiscard]] std::shared_ptr fields_ptr() const; + + [[nodiscard]] size_t fields_size() const; + + [[nodiscard]] const Field& fields(size_t pos) const; + + void force_own_buffer() { + buffer_.force_own_buffer(); + keepalive_.reset(); } // For external language tools, not efficient [[nodiscard]] std::vector fields_vector() const { std::vector fields; - for(const auto& field : *fields_) + for(const auto& field : desc_.fields()) fields.push_back(field.name()); return fields; } - void force_own_buffer() { - if (!is_owning_buffer()) { - auto b = std::make_shared(); - std::get(buffer_).copy_to(*b); - buffer_ = std::move(b); - } - keepalive_.reset(); - } - void set_keepalive(std::any&& keepalive) { keepalive_ = std::move(keepalive); } - const std::any& keepalive() const { + [[nodiscard]] const std::any& keepalive() const { return keepalive_; } + [[nodiscard]] const StreamDescriptor& descriptor() const { + return desc_; + } + + Segment clone() const { + return Segment{header_.clone(), buffer_.clone(), desc_.clone(), size_}; + } + private: - void move_buffer(Segment &&that) { - if(is_uninitialized() || that.is_uninitialized()) { - std::swap(buffer_, that.buffer_); - } else if (!(is_owning_buffer() ^ that.is_owning_buffer())) { - if (is_owning_buffer()) { - swap(*std::get>(buffer_), *std::get>(that.buffer_)); - } else { - swap(std::get(buffer_), std::get(that.buffer_)); - } - } else if (is_owning_buffer()) { - log::storage().info("Copying segment"); - // data of segment being moved is not owned, moving it is dangerous, copying instead - that.buffer().copy_to(*std::get>(buffer_)); - } else { - // data of this segment is a view, but the move data is moved - buffer_ = std::move(std::get>(that.buffer_)); - } + Segment(SegmentHeader&& header, VariantBuffer&& buffer, StreamDescriptor&& desc, const std::optional size) : + header_(std::move(header)), + buffer_(std::move(buffer)), + desc_(std::move(desc)), + size_(size) { } - std::unique_ptr arena_ = std::make_unique(); - arcticdb::proto::encoding::SegmentHeader* header_ = nullptr; + + std::tuple> serialize_v1_header_to_buffer(size_t total_hdr_size); + std::pair serialize_v1_header_in_place(size_t total_header_size); + std::tuple> serialize_header_v1(); + std::pair serialize_header_v2(size_t expected_bytes); + size_t write_binary_header(uint8_t* dst) const; + + SegmentHeader header_; VariantBuffer buffer_; - std::shared_ptr fields_; + StreamDescriptor desc_; std::any keepalive_; + std::unique_ptr proto_; + std::optional size_; }; } //namespace arcticdb diff --git a/cpp/arcticdb/codec/segment_header.cpp b/cpp/arcticdb/codec/segment_header.cpp new file mode 100644 index 0000000000..942588ae99 --- /dev/null +++ b/cpp/arcticdb/codec/segment_header.cpp @@ -0,0 +1,94 @@ +#include +#include + +namespace arcticdb { + +size_t field_collection_encoded_field_bytes(const FieldCollection& fields) { + return calc_field_bytes(fields.num_blocks() + 1); //Field collection always has shapes buffer +} + + +size_t SegmentHeader::serialize_to_bytes(uint8_t* dst, std::optional expected_bytes) const { + const auto* begin = dst; + data_.field_buffer_.fields_bytes_ = static_cast(header_fields_.data_bytes()); + data_.field_buffer_.offset_bytes_ = static_cast(header_fields_.offset_bytes()); + memcpy(dst, &data_, sizeof(HeaderData)); + dst += sizeof(HeaderData); + log::codec().info("Wrote header data in {} bytes", dst - begin); + memcpy(dst, header_fields_.data_buffer(), header_fields_.data_bytes()); + dst += header_fields_.data_bytes(); + memcpy(dst, header_fields_.offsets_buffer(), header_fields_.offset_bytes()); + dst += header_fields_.offset_bytes(); + memcpy(dst, &offset_, sizeof(offset_)); + log::codec().info("Wrote header fields in {} bytes", dst - begin); + dst += sizeof(offset_); + log::codec().info("Wrote offsets in {} bytes", dst - begin); + size_t bytes_written = dst - begin; + util::check(!expected_bytes || bytes_written == *expected_bytes, "Mismatch between actual and expected bytes: {} != {}", dst - begin, *expected_bytes); + log::codec().info("Wrote V2 header with {} bytes ({} expected)", bytes_written, expected_bytes.value_or(0)); + return bytes_written; +} + +size_t calc_required_header_fields_bytes(const SegmentInMemory& in_mem_seg) { + size_t required = 0UL; + if(in_mem_seg.has_index_descriptor()) { + const auto index_descriptor_size = field_collection_encoded_field_bytes(in_mem_seg.index_descriptor().fields()) + sizeof(uint64_t); + required += index_descriptor_size; + log::codec().info("Index descriptor size {}", index_descriptor_size); + } + + if(in_mem_seg.has_string_pool()) { + const auto string_pool_size = calc_field_bytes(in_mem_seg.const_string_pool().num_blocks() + 1) + sizeof(uint64_t); //String pool has a custom shapes buffer + required += string_pool_size; + log::codec().info("String pool size {}", string_pool_size); + } + + if(!in_mem_seg.descriptor().empty()) { + const auto descriptor_size = field_collection_encoded_field_bytes(in_mem_seg.descriptor().fields()) + sizeof(uint64_t); + required += descriptor_size; + log::codec().info("Descriptor size {}", descriptor_size); + } + + // Metadata and column fields are allocated in one contiguous buffer with dimension 1 + if(in_mem_seg.metadata()) { + const auto metadata_size = calc_field_bytes(2) + sizeof(uint64_t); + required += metadata_size; + log::codec().info("Metadata size {}", metadata_size); + } + + if(!in_mem_seg.empty()) { + const auto column_fields_size = calc_field_bytes(2) + sizeof(uint64_t); + log::codec().info("Column fields size {}", column_fields_size); + required += column_fields_size; + } + + return required; +} + +void SegmentHeader::deserialize_from_bytes(const uint8_t* data) { + memcpy(&data_, data, sizeof(HeaderData)); + data += sizeof(HeaderData); + ChunkedBuffer fields_buffer; + fields_buffer.add_external_block(data, data_.field_buffer_.fields_bytes_, 0UL); + data += data_.field_buffer_.fields_bytes_; + Buffer offsets_buffer{data_.field_buffer_.offset_bytes_}; + memcpy(offsets_buffer.data(), data, data_.field_buffer_.offset_bytes_); + data += offsets_buffer.bytes(); + header_fields_ = EncodedFieldCollection{std::move(fields_buffer), std::move(offsets_buffer)}; + auto* offsets = reinterpret_cast(data); + for(auto i = 0UL; i < offset_.size(); ++i) + offset_[i] = *offsets++; +} + +size_t SegmentHeader::required_bytes(const SegmentInMemory& in_mem_seg) { + size_t required = 0UL; + + required += FIXED_HEADER_SIZE; + required += sizeof(HeaderData); + required += sizeof(offset_); + log::codec().info("Overhead size {} + {} + {} = {}", FIXED_HEADER_SIZE, sizeof(HeaderData), sizeof(offset_), required); + required += calc_required_header_fields_bytes(in_mem_seg); + log::codec().info("Total calculated header size: {}", required); + return required; +} +} //namespace arcticdb diff --git a/cpp/arcticdb/codec/segment_header.hpp b/cpp/arcticdb/codec/segment_header.hpp new file mode 100644 index 0000000000..ee0d729cda --- /dev/null +++ b/cpp/arcticdb/codec/segment_header.hpp @@ -0,0 +1,270 @@ +#pragma once + +#include +#include +//#include +#include + +namespace arcticdb { + +class SegmentInMemory; + +static constexpr std::array offset_names_ = { + "METADATA", + "STRING_POOL", + "DESCRIPTOR", + "INDEX", + "COLUMN" +}; + +inline void write_fixed_header(std::uint8_t *dst, const FixedHeader& hdr) { + ARCTICDB_DEBUG(log::codec(), "Writing header with size {}", hdr.header_bytes); + auto h = reinterpret_cast(dst); + *h = hdr; +} + +class SegmentHeader { + HeaderData data_; + EncodedFieldCollection header_fields_; + EncodedFieldCollection body_fields_; + std::array offset_ = {}; + size_t field_count_ = 0U; + +public: + explicit SegmentHeader(EncodingVersion encoding_version) { + data_.encoding_version_ = encoding_version; + } + + ARCTICDB_MOVE_ONLY_DEFAULT(SegmentHeader) + + SegmentHeader() = default; + + SegmentHeader clone() const { + SegmentHeader output(data_.encoding_version_); + output.data_ = data_; + output.header_fields_ = header_fields_.clone(); + output.body_fields_ = body_fields_.clone(); + output.offset_ = offset_; + return output; + } + + [[nodiscard]] bool empty() const { + return header_fields_.empty(); + } + + static constexpr uint8_t flag_mask(HeaderFlag flag) { + return 1 << static_cast(flag); + } + + void set_offset(FieldOffset field, uint32_t offset) { + util::check(offset < 5, "Out of bounds offset {}", offset); + offset_[as_pos(field)] = offset; + } + + template + void set_flag(bool value) { + constexpr auto mask = flag_mask(flag); + if(value) + data_.flags_ |= mask; + else + data_.flags_ &= ~mask; + } + + template + [[nodiscard]] bool get_flag() const { + return data_.flags_ & flag_mask(flag); + } + + [[nodiscard]] bool compacted() const { + return get_flag(); + } + + void set_compacted(bool value) { + set_flag(value); + } + + [[nodiscard]] size_t bytes() const { + const auto total_bytes = sizeof(HeaderData) + header_fields_.data_bytes() + header_fields_.offset_bytes() + sizeof(offset_); + log::codec().info("Header bytes {} + {} + {} + {} = {}",sizeof(HeaderData), header_fields_.data_bytes(), header_fields_.offset_bytes(), sizeof(offset_), total_bytes); + return sizeof(HeaderData) + header_fields_.data_bytes() + header_fields_.offset_bytes() + sizeof(offset_); + } + + [[nodiscard]] static constexpr size_t as_pos(FieldOffset field_offset) { + return static_cast(field_offset); + } + + [[nodiscard]] int32_t get_offset(FieldOffset field_offset) const { + return offset_[as_pos(field_offset)]; + } + + [[nodiscard]] static constexpr std::string_view offset_name(FieldOffset field_offset) { + return offset_names_[as_pos(field_offset)]; + } + + [[nodiscard]] bool has_metadata_field() const { + return has_field(FieldOffset::METADATA); + } + + [[nodiscard]] bool has_string_pool_field() const { + return has_field(FieldOffset::STRING_POOL); + } + + [[nodiscard]] bool has_descriptor_field() const { + return has_field(FieldOffset::DESCRIPTOR); + } + + [[nodiscard]] bool has_index_descriptor_field() const { + return has_field(FieldOffset::INDEX); + } + + [[nodiscard]] bool has_column_fields() const { + return has_field(FieldOffset::COLUMN); + } + + template + [[nodiscard]] const EncodedFieldImpl& get_field() const { + util::check(has_field(field_offset), "Field {} has not been set", offset_name(field_offset)); + return header_fields_.at(get_offset(field_offset)); + } + + [[nodiscard]] const EncodedFieldImpl& metadata_field() const { + return get_field(); + } + + [[nodiscard]] const EncodedFieldImpl& string_pool_field() const { + return get_field(); + } + [[nodiscard]] const EncodedFieldImpl& descriptor_field() const { + return get_field(); + } + + [[nodiscard]] const EncodedFieldImpl& index_descriptor_field() const { + return get_field(); + } + + [[nodiscard]] const EncodedFieldImpl& column_fields() const { + return get_field(); + } + + void validate() const { + for(auto i = 0U; i < static_cast(FieldOffset::COUNT); ++i) { + auto offset = FieldOffset(i); + if(has_field(offset)) + header_fields_.at(get_offset(offset)).validate(); + } + } + + template + EncodedFieldImpl& create_field(size_t num_blocks) { + log::codec().info("Header adding field with {} blocks", num_blocks); + auto new_field = header_fields_.add_field(num_blocks); + set_offset(field_offset, field_count_++); + set_field(field_offset); + return *new_field; + } + + template + [[nodiscard]] EncodedFieldImpl& get_mutable_field(size_t num_blocks) { + if(has_field(field_offset)) { + return header_fields_.at(get_offset(field_offset)); + } else { + return create_field(num_blocks); + } + } + + [[nodiscard]] EncodedFieldImpl& mutable_metadata_field(size_t num_blocks) { + return get_mutable_field(num_blocks); + } + + [[nodiscard]] EncodedFieldImpl& mutable_string_pool_field(size_t num_blocks) { + return get_mutable_field(num_blocks); + } + + [[nodiscard]] EncodedFieldImpl& mutable_descriptor_field(size_t num_blocks) { + return get_mutable_field(num_blocks); + } + + [[nodiscard]] EncodedFieldImpl& mutable_index_descriptor_field(size_t num_blocks) { + return get_mutable_field(num_blocks); + } + + [[nodiscard]] EncodedFieldImpl& mutable_column_fields(size_t num_blocks) { + return get_mutable_field(num_blocks); + } + + size_t required_bytes(const SegmentInMemory& in_mem_seg); + + [[nodiscard]] EncodingVersion encoding_version() const { + return data_.encoding_version_; + } + + void set_encoding_version(EncodingVersion encoding_version) { + data_.encoding_version_ = encoding_version; + } + + void set_footer_offset(uint64_t offset) { + log::codec().info("Setting footer offset at {}", offset); + data_.footer_offset_ = offset; + } + + [[nodiscard]] uint64_t footer_offset() const { + return data_.footer_offset_; + } + + size_t serialize_to_bytes(uint8_t* dst, std::optional expected_bytes = std::nullopt) const; + + static constexpr uint16_t field_mask(FieldOffset field_offset) { + return 1U << static_cast(field_offset); + } + + void set_field(FieldOffset field_offset) { + data_.fields_ |= field_mask(field_offset); + } + + [[nodiscard]] bool has_field(FieldOffset field_offset) const { + return data_.fields_ & field_mask(field_offset); + } + + void deserialize_from_bytes(const uint8_t* data); + + [[nodiscard]] const EncodedFieldCollection& body_fields() const { + return body_fields_; + } + + void set_body_fields(EncodedFieldCollection&& body_fields) { + body_fields_ = std::move(body_fields); + body_fields_.regenerate_offsets(); + } +}; + +} //namespace arcticdb + +namespace fmt { +template<> +struct formatter { + template + constexpr auto parse(ParseContext &ctx) { return ctx.begin(); } + + template + auto format(const arcticdb::SegmentHeader &header, FormatContext &ctx) const { + fmt::format_to(ctx.out(), fmt::runtime("Segment header: encoding {}: {} bytes { \n"), header.encoding_version(), header.bytes()); + + if(header.has_descriptor_field()) + fmt::format_to(ctx.out(), "Descriptor: {}\n", header.descriptor_field()); + + if(header.has_metadata_field()) + fmt::format_to(ctx.out(), "Metadata: {}\n", header.metadata_field()); + + if(header.has_index_descriptor_field()) + fmt::format_to(ctx.out(), "Index: {}\n", header.index_descriptor_field()); + + if(header.has_string_pool_field()) + fmt::format_to(ctx.out(), "String pool: {}\n", header.string_pool_field()); + + if(header.has_column_fields()) + fmt::format_to(ctx.out(), "Columns: {}\n", header.column_fields()); + + return fmt::format_to(ctx.out(), fmt::runtime("}\n")); + } +}; +} \ No newline at end of file diff --git a/cpp/arcticdb/codec/segment_identifier.hpp b/cpp/arcticdb/codec/segment_identifier.hpp new file mode 100644 index 0000000000..ec7c21e090 --- /dev/null +++ b/cpp/arcticdb/codec/segment_identifier.hpp @@ -0,0 +1,66 @@ +#pragma once + +#include +#include + +namespace arcticdb { + +static constexpr size_t SegmentIdentifierSize = sizeof(SegmentIdentifierHeader); + +struct SegmentIdentifier { + SegmentIdentifierHeader header_; + std::array data_; +}; + +[[nodiscard]] inline size_t identifier_bytes(const StreamId& stream_id) { + return util::variant_match(stream_id, + [] (const NumericId&) { return SegmentIdentifierSize; }, + [] (const StringId& str_id) { return SegmentIdentifierSize + str_id.size(); }); +} + +inline void write_identifier(Buffer& buffer, std::ptrdiff_t& pos, const StreamId& stream_id) { + auto data = new (buffer.data() + pos) SegmentDescriptorImpl{}; + util::variant_match(stream_id, + [data, &pos] (const NumericId& num_id) { + SegmentIdentifierHeader header{IdentifierType::NUMERIC, static_cast(num_id)}; + *reinterpret_cast(data) = header; + pos += SegmentIdentifierSize; + }, + [data, &pos] (const StringId& str_id) { + auto* identifier_impl = reinterpret_cast(data); + identifier_impl->header_.type_ = IdentifierType::STRING; + identifier_impl->header_.size_ = static_cast(str_id.size()); + memcpy(&identifier_impl->data_[0], str_id.data(), str_id.size()); + pos += SegmentIdentifierSize + str_id.size(); + }); +} + +inline StreamId read_identifier(const uint8_t*& data) { + auto* identifier = reinterpret_cast(data); + switch(identifier->header_.type_) { + case IdentifierType::STRING: + data += SegmentIdentifierSize + identifier->header_.size_; + return StringId(&identifier->data_[0], identifier->header_.size_); + case IdentifierType::NUMERIC: + data += SegmentIdentifierSize; + return NumericId(identifier->header_.size_); + default: + util::raise_rte("Unknown identifier type in read_identifier"); + } +} + +inline void skip_identifier(const uint8_t*& data) { + auto* identifier = reinterpret_cast(data); + switch(identifier->header_.type_) { + case IdentifierType::STRING: + data += SegmentIdentifierSize + identifier->header_.size_; + break; + case IdentifierType::NUMERIC: + data += SegmentIdentifierSize; + break; + default: + util::raise_rte("Unknown identifier type in skip_identifier"); + } +} + +} //namespace arcticdb \ No newline at end of file diff --git a/cpp/arcticdb/codec/test/test_codec.cpp b/cpp/arcticdb/codec/test/test_codec.cpp index 5cd20bddd2..9b2ff41ed8 100644 --- a/cpp/arcticdb/codec/test/test_codec.cpp +++ b/cpp/arcticdb/codec/test/test_codec.cpp @@ -26,7 +26,7 @@ namespace arcticdb { static void encode( const arcticdb::proto::encoding::VariantCodec &codec_opts, ColumnData& column_data, - std::variant variant_field, + EncodedFieldImpl& variant_field, Buffer& out, std::ptrdiff_t& pos); }; @@ -36,7 +36,7 @@ namespace arcticdb { static void encode( const arcticdb::proto::encoding::VariantCodec &codec_opts, ColumnData& column_data, - std::variant variant_field, + EncodedFieldImpl& variant_field, Buffer& out, std::ptrdiff_t& pos); static std::pair max_compressed_size( @@ -45,14 +45,14 @@ namespace arcticdb { private: static void encode_shapes( const ColumnData& column_data, - std::variant variant_field, + EncodedFieldImpl& variant_field, Buffer& out, std::ptrdiff_t& pos_in_buffer); static void encode_blocks( const arcticdb::proto::encoding::VariantCodec &codec_opts, ColumnData& column_data, - std::variant variant_field, + EncodedFieldImpl& variant_field, Buffer& out, std::ptrdiff_t& pos); }; @@ -62,7 +62,7 @@ namespace arcticdb { using namespace arcticdb; -using EncoginVersions = ::testing::Types< +using EncodingVersions = ::testing::Types< std::integral_constant, std::integral_constant>; @@ -80,7 +80,7 @@ class FieldEncoderTestDim0Base : public testing::Test { template class FieldEncoderTestDim0 : public FieldEncoderTestDim0Base{}; -using EncodedFieldsType = ::testing::Types; +using EncodedFieldsType = ::testing::Types; TYPED_TEST_SUITE(FieldEncoderTestDim0, EncodedFieldsType); TYPED_TEST(FieldEncoderTestDim0, Passthrough_v1) { @@ -133,10 +133,8 @@ class FieldEncoderTestFromColumnDim0 : public FieldEncoderTestDim0Base{}; /// @brief Cartesian product between the type of the encoded field and the encoding version. /// (EncodedField, arcticdb::proto::encoding::EncodedField) x (EncodingVersion::V1, EncodingVersion::V2) using FieldVersionT = ::testing::Types< - std::pair, - std::pair, - std::pair, - std::pair>; + std::pair, + std::pair>; TYPED_TEST_SUITE(FieldEncoderTestFromColumnDim0, FieldVersionT); TYPED_TEST(FieldEncoderTestFromColumnDim0, Passthrough) { @@ -156,7 +154,7 @@ TYPED_TEST(FieldEncoderTestFromColumnDim0, Passthrough) { column_data); Buffer out(max_compressed_size); column_data.reset(); - ColumnEncoder::encode(TestFixture::passthorugh_encoding_options, column_data, &field, out, pos); + ColumnEncoder::encode(TestFixture::passthorugh_encoding_options, column_data, field, out, pos); auto& nd = field.ndarray(); ASSERT_EQ(nd.items_count(), TestFixture::values.size()); ASSERT_EQ(nd.shapes_size(), 0); @@ -180,34 +178,6 @@ class FieldEncoderTestDim1 : public testing::Test { arcticdb::proto::encoding::VariantCodec passthorugh_encoding_options; }; -TEST_F(FieldEncoderTestDim1, PassthroughV1ProtoField) { - using Encoder = TypedBlockEncoderImpl; - const TypedBlockData block( - values.data(), - shapes.data(), - values_byte_size, - shapes.size(), - nullptr); - arcticdb::proto::encoding::EncodedField field; - Buffer out(Encoder::max_compressed_size(passthorugh_encoding_options, block)); - std::ptrdiff_t pos = 0; - Encoder::encode(passthorugh_encoding_options, block, field, out, pos); - - const auto& nd = field.ndarray(); - ASSERT_EQ(nd.items_count(), shapes.size()); - - const auto& shapes = nd.shapes(); - ASSERT_EQ(shapes[0].in_bytes(), shapes_byte_size); - ASSERT_EQ(shapes[0].out_bytes(), shapes_byte_size); - ASSERT_NE(0, shapes[0].hash()); - - const auto& vals = nd.values(); - ASSERT_EQ(vals[0].in_bytes(), values_expected_bytes); - ASSERT_EQ(vals[0].out_bytes(), values_expected_bytes); - ASSERT_NE(0, vals[0].hash()); - ASSERT_EQ(pos, values_expected_bytes + shapes_byte_size); -} - TEST_F(FieldEncoderTestDim1, PassthroughV1NativeField) { using Encoder = TypedBlockEncoderImpl; const TypedBlockData block( @@ -217,9 +187,9 @@ TEST_F(FieldEncoderTestDim1, PassthroughV1NativeField) { shapes.size(), nullptr); // one block for shapes and one for values - constexpr size_t encoded_field_size = EncodedField::Size + 2 * sizeof(EncodedBlock); + constexpr size_t encoded_field_size = EncodedFieldImpl::Size + 2 * sizeof(EncodedBlock); std::array encoded_field_memory; - EncodedField* field = new(encoded_field_memory.data()) EncodedField; + EncodedFieldImpl* field = new(encoded_field_memory.data()) EncodedFieldImpl; Buffer out(Encoder::max_compressed_size(passthorugh_encoding_options, block)); std::ptrdiff_t pos = 0; @@ -240,47 +210,6 @@ TEST_F(FieldEncoderTestDim1, PassthroughV1NativeField) { ASSERT_EQ(pos, values_expected_bytes + shapes_byte_size); } -TEST_F(FieldEncoderTestDim1, PassthroughV2ProtoField) { - using Encoder = TypedBlockEncoderImpl; - using ShapesEncoder = TypedBlockEncoderImpl; - const TypedBlockData values_block( - values.data(), - shapes.data(), - values_byte_size, - shapes.size(), - nullptr); - const TypedBlockData shapes_block( - shapes.data(), - nullptr, - shapes_byte_size, - 0, - nullptr); - const size_t values_max_compressed_size = Encoder::max_compressed_size(passthorugh_encoding_options, - values_block); - const size_t shapes_max_compressed_size = ShapesEncoder::max_compressed_size(passthorugh_encoding_options, - shapes_block); - const size_t total_max_compressed_size = values_max_compressed_size + shapes_max_compressed_size; - arcticdb::proto::encoding::EncodedField field; - Buffer out(total_max_compressed_size); - std::ptrdiff_t pos = 0; - ShapesEncoder::encode_shapes(passthorugh_encoding_options, shapes_block, field, out, pos); - Encoder::encode_values(passthorugh_encoding_options, values_block, field, out, pos); - - const auto& nd = field.ndarray(); - ASSERT_EQ(nd.items_count(), shapes.size()); - - const auto& shapes = nd.shapes(); - ASSERT_EQ(shapes[0].in_bytes(), shapes_byte_size); - ASSERT_EQ(shapes[0].out_bytes(), shapes_byte_size); - ASSERT_NE(0, shapes[0].hash()); - - const auto& vals = nd.values(); - ASSERT_EQ(vals[0].in_bytes(), values_expected_bytes); - ASSERT_EQ(vals[0].out_bytes(), values_expected_bytes); - ASSERT_NE(0, vals[0].hash()); - ASSERT_EQ(pos, values_expected_bytes + shapes_byte_size); -} - TEST_F(FieldEncoderTestDim1, PassthroughV2NativeField) { using Encoder = TypedBlockEncoderImpl; using ShapesEncoder = TypedBlockEncoderImpl; @@ -302,9 +231,9 @@ TEST_F(FieldEncoderTestDim1, PassthroughV2NativeField) { shapes_block); const size_t total_max_compressed_size = values_max_compressed_size + shapes_max_compressed_size; // one block for shapes and one for values - constexpr size_t encoded_field_size = EncodedField::Size + 2 * sizeof(EncodedBlock); + constexpr size_t encoded_field_size = EncodedFieldImpl::Size + 2 * sizeof(EncodedBlock); std::array encoded_field_memory; - EncodedField* field = new(encoded_field_memory.data()) EncodedField; + EncodedFieldImpl* field = new(encoded_field_memory.data()) EncodedFieldImpl; Buffer out(total_max_compressed_size); std::ptrdiff_t pos = 0; ShapesEncoder::encode_shapes(passthorugh_encoding_options, shapes_block, *field, out, pos); @@ -356,31 +285,17 @@ class TestMultiblockData_Dim1 : public testing::Test { Buffer shapes_buffer; }; -TEST_F(TestMultiblockData_Dim1, EncodingVersion_1) { - arcticdb::proto::encoding::EncodedField encoded_field; - ColumnData column_data(&data_buffer, &shapes_buffer, type_descriptor, nullptr); - const auto [_, max_compressed_size] = ColumnEncoderV1::max_compressed_size(passthorugh_encoding_options, column_data); - Buffer out(max_compressed_size); - ptrdiff_t out_pos = 0; - column_data.reset(); - ColumnEncoderV1::encode(passthorugh_encoding_options, column_data, &encoded_field, out, out_pos); - const auto ndarray = encoded_field.ndarray(); - ASSERT_EQ(ndarray.shapes_size(), 2); - ASSERT_EQ(ndarray.values_size(), 2); - ASSERT_EQ(ndarray.items_count(), shapes_data.size()); -} - TEST_F(TestMultiblockData_Dim1, EncodingVersion_2) { - constexpr size_t encoded_field_size = EncodedField::Size + 3 * sizeof(EncodedBlock); + constexpr size_t encoded_field_size = EncodedFieldImpl::Size + 3 * sizeof(EncodedBlock); std::array encoded_field_owner; - EncodedField* encoded_field = new(encoded_field_owner.data()) EncodedField; + EncodedFieldImpl* encoded_field = new(encoded_field_owner.data()) EncodedFieldImpl; ColumnData column_data(&data_buffer, &shapes_buffer, type_descriptor, nullptr); const auto [_, max_compressed_size] = ColumnEncoderV2::max_compressed_size(passthorugh_encoding_options, column_data); Buffer out(max_compressed_size); ptrdiff_t out_pos = 0; column_data.reset(); - ColumnEncoderV2::encode(passthorugh_encoding_options, column_data, encoded_field, out, out_pos); - const auto ndarray = encoded_field->ndarray(); + ColumnEncoderV2::encode(passthorugh_encoding_options, column_data, *encoded_field, out, out_pos); + const auto& ndarray = encoded_field->ndarray(); ASSERT_EQ(ndarray.shapes_size(), 1); ASSERT_EQ(ndarray.values_size(), 2); ASSERT_EQ(ndarray.items_count(), shapes_data.size()); @@ -389,7 +304,7 @@ TEST_F(TestMultiblockData_Dim1, EncodingVersion_2) { template class SegmentStringEncodingTest : public testing::Test{}; -TYPED_TEST_SUITE(SegmentStringEncodingTest, EncoginVersions); +TYPED_TEST_SUITE(SegmentStringEncodingTest, EncodingVersions); TYPED_TEST(SegmentStringEncodingTest, EncodeSingleString) { const auto tsd = create_tsd, Dimension::Dim0>("thing", 1); @@ -485,15 +400,114 @@ bool TransactionalThing::destroyed = false; TEST(Segment, KeepAlive) { { + auto buf = std::make_shared(); Segment segment; + segment.set_buffer(std::move(buf)); segment.set_keepalive(std::any(TransactionalThing{})); auto seg1 = std::move(segment); Segment seg2{std::move(seg1)}; - auto seg3 = seg2; - Segment seg4{seg3}; + auto seg3 = seg2.clone(); + Segment seg4{seg3.clone()}; - std::any_cast(seg4.keepalive()).magic_.check(); + std::any_cast(seg2.keepalive()).magic_.check(); } ASSERT_EQ(TransactionalThing::destroyed, true); } + +TEST(Segment, RoundtripTimeseriesDescriptorV1) { + const auto stream_desc = stream_descriptor(StreamId{"thing"}, RowCountIndex{}, {scalar_field(DataType::UINT8, "ints")}); + SegmentInMemory in_mem_seg{stream_desc.clone()}; + in_mem_seg.set_scalar(0, 23); + in_mem_seg.end_row(); + TimeseriesDescriptor tsd; + tsd.set_total_rows(23); + tsd.set_stream_descriptor(stream_desc); + in_mem_seg.set_timeseries_descriptor(tsd); + auto copy = in_mem_seg.clone(); + auto seg = encode_v1(std::move(in_mem_seg), codec::default_lz4_codec()); + SegmentInMemory decoded{stream_desc.clone()}; + decode_v1(seg, seg.header(), decoded, seg.descriptor()); + ASSERT_EQ(decoded.index_descriptor().total_rows(), 23); + ASSERT_EQ(decoded, copy); +} + +TEST(Segment, RoundtripTimeseriesDescriptorWriteToBufferV1) { + const auto stream_desc = stream_descriptor(StreamId{"thing"}, RowCountIndex{}, {scalar_field(DataType::UINT8, "ints")}); + SegmentInMemory in_mem_seg{stream_desc.clone()}; + in_mem_seg.set_scalar(0, 23); + in_mem_seg.end_row(); + TimeseriesDescriptor tsd; + tsd.set_total_rows(23); + tsd.set_stream_descriptor(stream_desc); + in_mem_seg.set_timeseries_descriptor(tsd); + auto copy = in_mem_seg.clone(); + auto seg = encode_v1(std::move(in_mem_seg), codec::default_lz4_codec()); + std::vector vec; + const auto bytes = seg.calculate_size(); + vec.resize(bytes); + seg.write_to(vec.data()); + auto unserialized = Segment::from_bytes(vec.data(), bytes); + SegmentInMemory decoded{stream_desc.clone()}; + decode_v1(unserialized, unserialized.header(), decoded, unserialized.descriptor()); + ASSERT_EQ(decoded.index_descriptor().total_rows(), 23); + ASSERT_EQ(decoded, copy); +} + +TEST(Segment, RoundtripStringsWriteToBufferV1) { + const auto stream_desc = stream_descriptor(StreamId{"thing"}, RowCountIndex{}, {scalar_field(DataType::UTF_DYNAMIC64, "ints")}); + SegmentInMemory in_mem_seg{stream_desc.clone()}; + in_mem_seg.set_string(0, "kismet"); + in_mem_seg.end_row(); + auto copy = in_mem_seg.clone(); + auto seg = encode_v1(std::move(in_mem_seg), codec::default_lz4_codec()); + std::vector vec; + const auto bytes = seg.calculate_size(); + vec.resize(bytes); + seg.write_to(vec.data()); + auto unserialized = Segment::from_bytes(vec.data(), bytes); + SegmentInMemory decoded{stream_desc.clone()}; + decode_v1(unserialized, unserialized.header(), decoded, unserialized.descriptor()); + ASSERT_EQ(decoded.string_at(0, 0).value(), "kismet"); + ASSERT_EQ(decoded, copy); +} + +TEST(Segment, RoundtripTimeseriesDescriptorV2) { + const auto stream_desc = stream_descriptor(StreamId{"thing"}, RowCountIndex{}, {scalar_field(DataType::UINT8, "ints")}); + SegmentInMemory in_mem_seg{stream_desc.clone()}; + in_mem_seg.set_scalar(0, 23); + in_mem_seg.end_row(); + TimeseriesDescriptor tsd; + tsd.set_total_rows(23); + tsd.set_stream_descriptor(stream_desc); + in_mem_seg.set_timeseries_descriptor(tsd); + auto copy = in_mem_seg.clone(); + auto seg = encode_v2(std::move(in_mem_seg), codec::default_lz4_codec()); + SegmentInMemory decoded{stream_desc.clone()}; + decode_v2(seg, seg.header(), decoded, seg.descriptor()); + ASSERT_EQ(decoded.index_descriptor().total_rows(), 23); + ASSERT_EQ(decoded, copy); +} + +TEST(Segment, RoundtripTimeseriesDescriptorWriteToBufferV2) { + const auto stream_desc = stream_descriptor(StreamId{"thing"}, RowCountIndex{}, {scalar_field(DataType::UINT8, "ints")}); + SegmentInMemory in_mem_seg{stream_desc.clone()}; + in_mem_seg.set_scalar(0, 23); + in_mem_seg.end_row(); + TimeseriesDescriptor tsd; + tsd.set_total_rows(23); + tsd.set_stream_descriptor(stream_desc); + in_mem_seg.set_timeseries_descriptor(tsd); + auto copy = in_mem_seg.clone(); + auto seg = encode_v2(std::move(in_mem_seg), codec::default_lz4_codec()); + std::vector vec; + const auto bytes = seg.calculate_size(); + log::codec().info("## Resizing buffer to {} bytes", bytes); + vec.resize(bytes); + seg.write_to(vec.data()); + auto unserialized = Segment::from_bytes(vec.data(), bytes); + SegmentInMemory decoded{stream_desc.clone()}; + decode_v2(unserialized, unserialized.header(), decoded, unserialized.descriptor()); + ASSERT_EQ(decoded.index_descriptor().total_rows(), 23); + ASSERT_EQ(decoded, copy); +} \ No newline at end of file diff --git a/cpp/arcticdb/codec/test/test_encode_field_collection.cpp b/cpp/arcticdb/codec/test/test_encode_field_collection.cpp new file mode 100644 index 0000000000..7ba28f8c00 --- /dev/null +++ b/cpp/arcticdb/codec/test/test_encode_field_collection.cpp @@ -0,0 +1,33 @@ +#include +#include + +TEST(EncodedFieldCollection, AddField) { + using namespace arcticdb; + EncodedFieldCollection fields; + auto* field1 = fields.add_field(3); + auto* block1 = field1->mutable_ndarray()->add_shapes(); + block1->set_in_bytes(1); + auto* block2 = field1->mutable_ndarray()->add_values(EncodingVersion::V2); + block2->set_in_bytes(2); + auto* block3 = field1->mutable_ndarray()->add_values(EncodingVersion::V2); + block3->set_in_bytes(3); + auto* field2 = fields.add_field(2); + auto* block4 = field2->mutable_ndarray()->add_values(EncodingVersion::V2); + block4->set_in_bytes(4); + auto* block5 = field2->mutable_ndarray()->add_values(EncodingVersion::V2); + block5->set_in_bytes(5); + + const auto& read1 = fields.at(0); + auto b1 = read1.shapes(0); + ASSERT_EQ(b1.in_bytes(), 1); + auto b2 = read1.values(0); + ASSERT_EQ(b2.in_bytes(), 2); + auto b3 = read1.values(1); + ASSERT_EQ(b3.in_bytes(), 3); + + const auto& read2 = fields.at(1); + auto b4 = read2.values(0); + ASSERT_EQ(b4.in_bytes(), 4); + auto b5 = read2.values(1); + ASSERT_EQ(b5.in_bytes(), 5); +} \ No newline at end of file diff --git a/cpp/arcticdb/codec/test/test_encoded_field.cpp b/cpp/arcticdb/codec/test/test_encoded_field.cpp new file mode 100644 index 0000000000..9aa5696148 --- /dev/null +++ b/cpp/arcticdb/codec/test/test_encoded_field.cpp @@ -0,0 +1,139 @@ +#include +#include +#include + +TEST(EncodedField, ScalarBlocks) { + using namespace arcticdb; + EncodedFieldCollection coll(calc_field_bytes(4), 1); + auto* field_ptr = coll.add_field(4); + auto& field = *field_ptr; + auto* v1 = field.add_values(EncodingVersion::V1); + v1->mutable_codec()->mutable_lz4()->acceleration_ = 1; + auto* v2 = field.add_values(EncodingVersion::V1); + v2->mutable_codec()->mutable_lz4()->acceleration_ = 2; + auto* v3 = field.add_values(EncodingVersion::V1); + v3->mutable_codec()->mutable_lz4()->acceleration_ = 3; + auto* v4 = field.add_values(EncodingVersion::V1); + v4 ->mutable_codec()->mutable_lz4()->acceleration_ = 4; + + ASSERT_EQ(field.values_size(), 4); + ASSERT_EQ(field.shapes_size(), 0); + + auto expected = 1; + for(const auto& value : field.values()) { + ASSERT_EQ(value.codec().lz4().acceleration_, expected); + ++expected; + } + field.validate(); +} + +TEST(EncodedField, OldStyleShapes) { + using namespace arcticdb; + EncodedFieldCollection coll(calc_field_bytes(8), 1); + auto* field_ptr = coll.add_field(8); + auto& field = *field_ptr; + auto* s1 = field.add_shapes(); + s1->mutable_codec()->mutable_lz4()->acceleration_ = 1; + auto* v1 = field.add_values(EncodingVersion::V1); + v1->mutable_codec()->mutable_lz4()->acceleration_ = 2; + auto* s2 = field.add_shapes(); + s2->mutable_codec()->mutable_lz4()->acceleration_ = 3; + auto* v2 = field.add_values(EncodingVersion::V1); + v2->mutable_codec()->mutable_lz4()->acceleration_ = 4; + auto* s3 = field.add_shapes(); + s3->mutable_codec()->mutable_lz4()->acceleration_ = 5; + auto* v3 = field.add_values(EncodingVersion::V1); + v3->mutable_codec()->mutable_lz4()->acceleration_ = 6; + auto* s4 = field.add_shapes(); + s4->mutable_codec()->mutable_lz4()->acceleration_ = 7; + auto* v4 = field.add_values(EncodingVersion::V1); + v4->mutable_codec()->mutable_lz4()->acceleration_ = 8; + + ASSERT_EQ(field.values_size(), 4); + ASSERT_EQ(field.shapes_size(), 4); + + auto expected = 2; + for(const auto& value : field.values()) { + ASSERT_EQ(value.codec().lz4().acceleration_, expected); + expected += 2; + } + + expected = 1; + for(const auto& shape : field.shapes()) { + ASSERT_EQ(shape.codec().lz4().acceleration_, expected); + expected += 2; + } + field.validate(); +} + +TEST(EncodedField, OldStyleShapesEnterShapesFirst) { + using namespace arcticdb; + EncodedFieldCollection coll(calc_field_bytes(8), 1); + auto* field_ptr = coll.add_field(8); + auto& field = *field_ptr; + auto* s1 = field.add_shapes(); + s1->mutable_codec()->mutable_lz4()->acceleration_ = 1; + auto* s2 = field.add_shapes(); + s2->mutable_codec()->mutable_lz4()->acceleration_ = 3; + auto* s3 = field.add_shapes(); + s3->mutable_codec()->mutable_lz4()->acceleration_ = 5; + auto* s4 = field.add_shapes(); + s4->mutable_codec()->mutable_lz4()->acceleration_ = 7; + + + auto* v1 = field.add_values(EncodingVersion::V1); + v1->mutable_codec()->mutable_lz4()->acceleration_ = 2; + auto* v2 = field.add_values(EncodingVersion::V1); + v2->mutable_codec()->mutable_lz4()->acceleration_ = 4; + auto* v3 = field.add_values(EncodingVersion::V1); + v3->mutable_codec()->mutable_lz4()->acceleration_ = 6; + auto* v4 = field.add_values(EncodingVersion::V1); + v4->mutable_codec()->mutable_lz4()->acceleration_ = 8; + + ASSERT_EQ(field.values_size(), 4); + ASSERT_EQ(field.shapes_size(), 4); + + auto expected = 2; + for(const auto& value : field.values()) { + ASSERT_EQ(value.codec().lz4().acceleration_, expected); + expected += 2; + } + + expected = 1; + for(const auto& shape : field.shapes()) { + ASSERT_EQ(shape.codec().lz4().acceleration_, expected); + expected += 2; + } + field.validate(); +} + +TEST(EncodedField, NewStyleShapes) { + using namespace arcticdb; + EncodedFieldCollection coll(calc_field_bytes(5), 1); + auto* field_ptr = coll.add_field(5); + auto& field = *field_ptr; + auto* s1 = field.add_shapes(); + s1->mutable_codec()->mutable_lz4()->acceleration_ = 1; + auto* v1 = field.add_values(EncodingVersion::V2); + v1->mutable_codec()->mutable_lz4()->acceleration_ = 2; + auto* v2 = field.add_values(EncodingVersion::V2); + v2->mutable_codec()->mutable_lz4()->acceleration_ = 3; + auto* v3 = field.add_values(EncodingVersion::V2); + v3->mutable_codec()->mutable_lz4()->acceleration_ = 4; + auto* v4 = field.add_values(EncodingVersion::V2); + v4->mutable_codec()->mutable_lz4()->acceleration_ = 5; + + ASSERT_EQ(field.values_size(), 4); + ASSERT_EQ(field.shapes_size(), 1); + + auto expected = 2; + for(const auto& value : field.values()) { + ASSERT_EQ(value.codec().lz4().acceleration_, expected); + ++expected; + } + + for (const auto& shape: field.shapes()) { + ASSERT_EQ(shape.codec().lz4().acceleration_, 1); + } + field.validate(); +} \ No newline at end of file diff --git a/cpp/arcticdb/codec/test/test_segment_header.cpp b/cpp/arcticdb/codec/test/test_segment_header.cpp new file mode 100644 index 0000000000..df0310bd0d --- /dev/null +++ b/cpp/arcticdb/codec/test/test_segment_header.cpp @@ -0,0 +1,83 @@ +#include +#include +#include +#include + +TEST(SegmentHeader, WriteAndReadFields) { + using namespace arcticdb; + SegmentHeader header{EncodingVersion::V1}; + auto& string_pool_field = header.mutable_string_pool_field(2); + auto* values = string_pool_field.mutable_ndarray()->add_values(EncodingVersion::V2); + values->set_in_bytes(23); + values = string_pool_field.mutable_ndarray()->add_values(EncodingVersion::V2); + values->set_in_bytes(47); + + const auto& read_string_pool = header.string_pool_field(); + auto& read_values1 = read_string_pool.values(0); + ASSERT_EQ(read_values1.in_bytes(), 23); + auto&read_values2 = read_string_pool.values(1); + ASSERT_EQ(read_values2.in_bytes(), 47); +} + +TEST(SegmentHeader, HasFields) { + using namespace arcticdb; + SegmentHeader header{EncodingVersion::V1}; + ASSERT_EQ(header.has_index_descriptor_field(), false); + ASSERT_EQ(header.has_metadata_field(), false); + ASSERT_EQ(header.has_column_fields(), false); + ASSERT_EQ(header.has_descriptor_field(), false); + ASSERT_EQ(header.has_string_pool_field(), false); + + (void)header.mutable_string_pool_field(2); + ASSERT_EQ(header.has_index_descriptor_field(), false); + ASSERT_EQ(header.has_metadata_field(), false); + ASSERT_EQ(header.has_column_fields(), false); + ASSERT_EQ(header.has_descriptor_field(), false); + ASSERT_EQ(header.has_string_pool_field(), true); + + (void)header.mutable_descriptor_field(2); + ASSERT_EQ(header.has_index_descriptor_field(), false); + ASSERT_EQ(header.has_metadata_field(), false); + ASSERT_EQ(header.has_column_fields(), false); + ASSERT_EQ(header.has_descriptor_field(), true); + ASSERT_EQ(header.has_string_pool_field(), true); + + (void)header.mutable_column_fields(2); + ASSERT_EQ(header.has_index_descriptor_field(), false); + ASSERT_EQ(header.has_metadata_field(), false); + ASSERT_EQ(header.has_column_fields(), true); + ASSERT_EQ(header.has_descriptor_field(), true); + ASSERT_EQ(header.has_string_pool_field(), true); +} + +TEST(SegmentHeader, SerializeUnserializeV1) { + using namespace arcticdb; + SegmentHeader header{EncodingVersion::V1}; + auto& string_pool_field = header.mutable_string_pool_field(10); + for(auto i = 0U; i < 5; ++i) { + auto *shapes = string_pool_field.mutable_ndarray()->add_shapes(); + shapes->set_in_bytes(i + 1); + shapes->mutable_codec()->mutable_lz4()->acceleration_ = 1; + auto *values = string_pool_field.mutable_ndarray()->add_values(EncodingVersion::V1); + values->set_in_bytes(i + 1); + values->mutable_codec()->mutable_lz4()->acceleration_ = 1; + } + + auto desc = stream_descriptor(StreamId{"thing"}, stream::RowCountIndex{}, {scalar_field(DataType::UINT8, "ints")}); + + auto proto = generate_proto_header(header, desc); + const auto header_size = proto.ByteSizeLong(); + std::vector vec(header_size); + auto read_header = decode_protobuf_header(vec.data(), header_size); + + const auto& string_pool = read_header.proto().string_pool_field(); + auto expected = 1U; + for(const auto& value : string_pool.ndarray().values()) { + ASSERT_EQ(value.in_bytes(), expected++); + } + + expected = 1U; + for(const auto& shape : string_pool.ndarray().shapes()) { + ASSERT_EQ(shape.in_bytes(), expected++); + } +} \ No newline at end of file diff --git a/cpp/arcticdb/codec/typed_block_encoder_impl.hpp b/cpp/arcticdb/codec/typed_block_encoder_impl.hpp index 34ea13b2fb..d022212a7a 100644 --- a/cpp/arcticdb/codec/typed_block_encoder_impl.hpp +++ b/cpp/arcticdb/codec/typed_block_encoder_impl.hpp @@ -8,6 +8,7 @@ #include namespace arcticdb { + /// @todo Split this class. This class does too much. Supports encoding via V1 and V2 but in a clunky way. The /// interface for encoding is different (V2 requires EncodedBlock to be passed, thus encode_values and /// encode_shapes were added). @@ -51,6 +52,35 @@ namespace arcticdb { }); } + template + static void encode_to_values( + const arcticdb::proto::encoding::VariantCodec& codec_opts, + const TypedBlockType& typed_block, + Buffer& out, + std::ptrdiff_t& pos, + NDArrayType& ndarray + ) { + if constexpr (encoder_version == EncodingVersion::V2) { + auto *values_encoded_block = ndarray->add_values(encoder_version); + visit_encoder(codec_opts, [&](auto encoder_tag) { + decltype(encoder_tag)::Encoder::encode(get_opts(codec_opts, encoder_tag), + typed_block, + out, + pos, + values_encoded_block); + }); + } else { + auto *values_encoded_block = ndarray->add_values(); + visit_encoder(codec_opts, [&](auto encoder_tag) { + decltype(encoder_tag)::Encoder::encode(get_opts(codec_opts, encoder_tag), + typed_block, + out, + pos, + values_encoded_block); + }); + } + } + template static void encode_values( const arcticdb::proto::encoding::VariantCodec& codec_opts, @@ -66,14 +96,9 @@ namespace arcticdb { ARCTICDB_TRACE(log::codec(), "Encoder got values of size 0. Noting to encode."); return; } - auto* values_encoded_block = ndarray->add_values(); - visit_encoder(codec_opts, [&](auto encoder_tag) { - decltype(encoder_tag)::Encoder::encode(get_opts(codec_opts, encoder_tag), - typed_block, - out, - pos, - values_encoded_block); - }); + + encode_to_values, decltype(ndarray)>(codec_opts, typed_block, out, pos, ndarray); + const auto existing_items_count = ndarray->items_count(); ndarray->set_items_count(existing_items_count + typed_block.row_count()); } diff --git a/cpp/arcticdb/codec/variant_encoded_field_collection.cpp b/cpp/arcticdb/codec/variant_encoded_field_collection.cpp deleted file mode 100644 index df3da27cd2..0000000000 --- a/cpp/arcticdb/codec/variant_encoded_field_collection.cpp +++ /dev/null @@ -1,41 +0,0 @@ -/* Copyright 2023 Man Group Operations Limited - * - * Use of this software is governed by the Business Source License 1.1 included in the file licenses/BSL.txt. - * - * As of the Change Date specified in that file, in accordance with the Business Source License, use of this software will be governed by the Apache License, version 2.0. - */ - -#include -#include -#include - -namespace arcticdb { - -VariantEncodedFieldCollection::VariantEncodedFieldCollection(const Segment& segment) { - if(EncodingVersion(segment.header().encoding_version()) == EncodingVersion::V2) { - const auto& hdr = segment.header(); - auto [begin, encoded_fields_ptr] = get_segment_begin_end(segment, segment.header()); - util::check_magic(encoded_fields_ptr); - auto encoded_fields_buffer = decode_encoded_fields(hdr, encoded_fields_ptr, begin); - fields_ = EncodedFieldCollection{std::move(encoded_fields_buffer)}; - } else { - is_proto_ = true; - header_ = &segment.header(); - } -} - -VariantField VariantEncodedFieldCollection::at(size_t pos) const { - if (is_proto_) - return &header_->fields(static_cast(pos)); - else - return &fields_.at(pos); -} - -size_t VariantEncodedFieldCollection::size() const { - if (is_proto_) - return header_->fields_size(); - else - return fields_.size(); -} - -} //namespace arcticdb diff --git a/cpp/arcticdb/codec/variant_encoded_field_collection.hpp b/cpp/arcticdb/codec/variant_encoded_field_collection.hpp deleted file mode 100644 index 41288cab85..0000000000 --- a/cpp/arcticdb/codec/variant_encoded_field_collection.hpp +++ /dev/null @@ -1,25 +0,0 @@ -/* Copyright 2023 Man Group Operations Limited - * - * Use of this software is governed by the Business Source License 1.1 included in the file licenses/BSL.txt. - * - * As of the Change Date specified in that file, in accordance with the Business Source License, use of this software will be governed by the Apache License, version 2.0. - */ - -#pragma once - -#include - -namespace arcticdb { - -using VariantField = std::variant; - -struct VariantEncodedFieldCollection { - EncodedFieldCollection fields_; - const arcticdb::proto::encoding::SegmentHeader *header_ = nullptr; - bool is_proto_ = false; - explicit VariantEncodedFieldCollection(const Segment &segment); - [[nodiscard]] VariantField at(size_t pos) const; - [[nodiscard]] size_t size() const; -}; - -} // namespace arcticdb diff --git a/cpp/arcticdb/codec/zstd.hpp b/cpp/arcticdb/codec/zstd.hpp index 4f69ff07d2..34b08113f4 100644 --- a/cpp/arcticdb/codec/zstd.hpp +++ b/cpp/arcticdb/codec/zstd.hpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -35,7 +36,7 @@ struct ZstdBlockEncoder { static std::size_t encode_block( const Opts &opts, const T* in, - BlockProtobufHelper &block_utils, + BlockDataHelper &block_utils, HashAccum& hasher, T *out, std::size_t out_capacity, @@ -44,7 +45,8 @@ struct ZstdBlockEncoder { std::size_t compressed_bytes = ZSTD_compress(out, out_capacity, in, block_utils.bytes_, opts.level()); hasher(in, block_utils.count_); pos += compressed_bytes; - out_codec.mutable_zstd()->MergeFrom(opts); + copy_codec(*out_codec.mutable_zstd(), opts); + return compressed_bytes; } }; diff --git a/cpp/arcticdb/column_store/chunked_buffer.hpp b/cpp/arcticdb/column_store/chunked_buffer.hpp index 4daa91b1ca..91f333d854 100644 --- a/cpp/arcticdb/column_store/chunked_buffer.hpp +++ b/cpp/arcticdb/column_store/chunked_buffer.hpp @@ -287,7 +287,7 @@ class ChunkedBufferImpl { void check_bytes(size_t pos_bytes, size_t required_bytes) const { if (pos_bytes + required_bytes > bytes()) { std::string err = fmt::format("Cursor overflow in chunked_buffer ptr_cast, cannot read {} bytes from a buffer of size {} with cursor " - "at {}, as it would required {} bytes. ", + "at {}, as it would require {} bytes. ", required_bytes, bytes(), pos_bytes, @@ -332,7 +332,7 @@ class ChunkedBufferImpl { bytes_ += size; } - bool empty() const { return bytes_ == 0; } + [[nodiscard]] bool empty() const { return bytes_ == 0; } void clear() { bytes_ = 0; diff --git a/cpp/arcticdb/column_store/column.cpp b/cpp/arcticdb/column_store/column.cpp index 40e35609ee..b32709a5c4 100644 --- a/cpp/arcticdb/column_store/column.cpp +++ b/cpp/arcticdb/column_store/column.cpp @@ -129,6 +129,10 @@ Column Column::clone() const { return output; } +bool Column::empty() const { + return row_count() == 0; +} + bool Column::is_sparse() const { if(last_logical_row_ != last_physical_row_) { util::check(static_cast(sparse_map_), "Expected sparse map in column with logical row {} and physical row {}", last_logical_row_, last_physical_row_); @@ -745,10 +749,6 @@ void Column::set_sparse_bit_for_row(size_t sparse_location) { sparse_map()[bv_size(sparse_location)] = true; } -bool Column::empty() const { - return row_count() == 0; -} - void Column::regenerate_offsets() const { if (ARCTICDB_LIKELY(is_scalar() || !offsets_.empty())) return; diff --git a/cpp/arcticdb/column_store/column.hpp b/cpp/arcticdb/column_store/column.hpp index 7d7a11cdbe..f81afdd253 100644 --- a/cpp/arcticdb/column_store/column.hpp +++ b/cpp/arcticdb/column_store/column.hpp @@ -86,9 +86,7 @@ void initialise_output_column(const Column& left_input_column, const Column& rig void initialise_output_bitset(const util::BitSet& input_bitset, bool sparse_missing_value_output, util::BitSet& output_bitset); class Column { - public: - template class TypedColumnIterator : public boost::iterator_facade, ValueType, boost::random_access_traversal_tag> { using RawType = std::decay_t; @@ -221,6 +219,13 @@ class Column { allow_sparse_(allow_sparse) { } + Column(TypeDescriptor type, bool allow_sparse, ChunkedBuffer&& buffer, Buffer&& shapes) : + data_(std::move(buffer)), + shapes_(std::move(shapes)), + type_(type), + allow_sparse_(allow_sparse) { + } + Column( TypeDescriptor type, size_t expected_rows, @@ -243,6 +248,8 @@ class Column { Column clone() const; + bool empty() const; + bool is_sparse() const; bool sparse_permitted() const; @@ -592,9 +599,8 @@ class Column { return data_.buffer(); } - //TODO this will need to be more efficient - index each block? template - std::optional index_of(T val) const { + std::optional search_unsorted(T val) const { util::check_arg(is_scalar(), "Cannot index on multidimensional values"); for (position_t i = 0; i < row_count(); ++i) { if (val == *ptr_cast(i, sizeof(T))) @@ -891,7 +897,6 @@ class Column { } private: - position_t last_offset() const; void update_offsets(size_t nbytes); bool is_scalar() const; @@ -902,7 +907,6 @@ class Column { size_t inflated_row_count() const; size_t num_shapes() const; void set_sparse_bit_for_row(size_t sparse_location); - bool empty() const; void regenerate_offsets() const; // Permutes the physical column storage based on the given sorted_pos. diff --git a/cpp/arcticdb/column_store/column_data.hpp b/cpp/arcticdb/column_store/column_data.hpp index 5a1955e2a7..2ed4d6160e 100644 --- a/cpp/arcticdb/column_store/column_data.hpp +++ b/cpp/arcticdb/column_store/column_data.hpp @@ -24,7 +24,7 @@ struct TypedBlockData { template class TypedColumnBlockIterator : public boost::iterator_facade, ValueType, boost::random_access_traversal_tag> { public: - TypedColumnBlockIterator(ValueType* ptr) + explicit TypedColumnBlockIterator(ValueType* ptr) : ptr_(ptr) { } TypedColumnBlockIterator(const TypedColumnBlockIterator& other) @@ -38,7 +38,9 @@ struct TypedBlockData { : ptr_(nullptr) { } TypedColumnBlockIterator& operator=(const TypedColumnBlockIterator& other) { - ptr_ = other.ptr_; + if(&other != this) + ptr_ = other.ptr_; + return *this; } @@ -92,12 +94,12 @@ struct TypedBlockData { block_(nullptr) {} - std::size_t nbytes() const { return nbytes_; } - std::size_t row_count() const { return row_count_; } - TypeDescriptor type() const { return static_cast(TDT()); } - const shape_t *shapes() const { return shapes_; } - const raw_type *data() const { return data_; } - const MemBlock *mem_block() const { return block_; } + [[nodiscard]] std::size_t nbytes() const { return nbytes_; } + [[nodiscard]] std::size_t row_count() const { return row_count_; } + [[nodiscard]] TypeDescriptor type() const { return static_cast(TDT()); } + [[nodiscard]] const shape_t *shapes() const { return shapes_; } + [[nodiscard]] const raw_type *data() const { return data_; } + [[nodiscard]] const MemBlock *mem_block() const { return block_; } raw_type operator[](size_t pos) const { return reinterpret_cast(block_->data())[pos]; @@ -111,7 +113,7 @@ struct TypedBlockData { return TypedColumnBlockIterator(data_ + row_count_); } - size_t offset() const { + [[nodiscard]] size_t offset() const { return block_->offset_; } @@ -148,7 +150,7 @@ struct ColumnData { ssize_t idx_{0}; RawType* ptr_{nullptr}; - inline ssize_t idx() const { + [[nodiscard]] inline ssize_t idx() const { return idx_; } @@ -321,15 +323,15 @@ struct ColumnData { return ColumnDataIterator(this, end_ptr); } - TypeDescriptor type() const { + [[nodiscard]] TypeDescriptor type() const { return type_; } - const ChunkedBuffer &buffer() const { + [[nodiscard]] const ChunkedBuffer &buffer() const { return *data_; } - const util::BitMagic* bit_vector() const { + [[nodiscard]] const util::BitMagic* bit_vector() const { return bit_vector_; } @@ -343,7 +345,7 @@ struct ColumnData { return shape; } - size_t num_blocks() const { + [[nodiscard]] size_t num_blocks() const { return data_->blocks().size(); } diff --git a/cpp/arcticdb/column_store/memory_segment.hpp b/cpp/arcticdb/column_store/memory_segment.hpp index 73e6d587c7..afae80f9d8 100644 --- a/cpp/arcticdb/column_store/memory_segment.hpp +++ b/cpp/arcticdb/column_store/memory_segment.hpp @@ -99,28 +99,20 @@ class SegmentInMemory { return impl_->column_index(name); } - std::shared_ptr index_fields() const { - return impl_->index_fields(); - } - - bool has_index_fields() const { - return impl_->has_index_fields(); - } - - TimeseriesDescriptor index_descriptor() { + const TimeseriesDescriptor& index_descriptor() const { return impl_->index_descriptor(); } - FieldCollection&& detach_index_fields() { - return impl_->detach_index_fields(); + TimeseriesDescriptor& mutable_index_descriptor() { + return impl_->mutable_index_descriptor(); } - std::shared_ptr timeseries_proto() { - return impl_->timeseries_proto(); + bool has_index_descriptor() const { + return impl_->has_index_descriptor(); } - void set_index_fields(std::shared_ptr fields) { - impl_->set_index_fields(std::move(fields)); + TimeseriesDescriptor&& detach_index_descriptor() { + return impl_->detach_index_descriptor(); } void init_column_map() const { @@ -240,9 +232,9 @@ class SegmentInMemory { return impl_->string_array_at(row, col); } - void set_timeseries_descriptor(TimeseriesDescriptor&& tsd) { + void set_timeseries_descriptor(const TimeseriesDescriptor& tsd) { util::check(!tsd.proto_is_null(), "Got null timeseries descriptor in set_timeseries_descriptor"); - impl_->set_timeseries_descriptor(std::move(tsd)); + impl_->set_timeseries_descriptor(tsd); } size_t num_columns() const { return impl_->num_columns(); } @@ -337,6 +329,10 @@ class SegmentInMemory { return impl_->string_pool_ptr(); } + void reset_metadata() { + impl_->reset_metadata(); + } + void set_metadata(google::protobuf::Any &&meta) { impl_->set_metadata(std::move(meta)); } @@ -345,10 +341,6 @@ class SegmentInMemory { return impl_->has_metadata(); } - void override_metadata(google::protobuf::Any &&meta) { - impl_->override_metadata(std::move(meta)); - } - ssize_t get_row_id() { return impl_->get_row_id(); } @@ -473,10 +465,6 @@ class SegmentInMemory { return output; } - StreamId get_index_col_name() const{ - return impl_->get_index_col_name(); - } - private: explicit SegmentInMemory(std::shared_ptr impl) : impl_(std::move(impl)) {} @@ -484,4 +472,4 @@ class SegmentInMemory { std::shared_ptr impl_; }; -} +} //namespace arcticdb diff --git a/cpp/arcticdb/column_store/memory_segment_impl.cpp b/cpp/arcticdb/column_store/memory_segment_impl.cpp index bf69e721d7..56b2d5b493 100644 --- a/cpp/arcticdb/column_store/memory_segment_impl.cpp +++ b/cpp/arcticdb/column_store/memory_segment_impl.cpp @@ -154,6 +154,9 @@ SegmentInMemoryImpl SegmentInMemoryImpl::clone() const { } output.allow_sparse_ = allow_sparse_; output.compacted_ = compacted_; + if(tsd_) + output.set_timeseries_descriptor(tsd_->clone()); + return output; } @@ -326,14 +329,6 @@ std::shared_ptr SegmentInMemoryImpl::filter(util::BitSet&& return output; } -std::shared_ptr SegmentInMemoryImpl::timeseries_proto() { - if(!tsd_) { - tsd_ = std::make_shared(); - metadata_->UnpackTo(tsd_.get()); - } - return tsd_; -} - std::shared_ptr SegmentInMemoryImpl::get_output_segment(size_t num_values, bool pre_allocate) const { std::shared_ptr output; if (is_sparse()) { @@ -635,22 +630,16 @@ void SegmentInMemoryImpl::sort(position_t idx) { } } -void SegmentInMemoryImpl::set_timeseries_descriptor(TimeseriesDescriptor&& tsd) { - index_fields_ = tsd.fields_ptr(); - tsd_ = tsd.proto_ptr(); - util::check(!tsd_->has_stream_descriptor() || tsd_->stream_descriptor().has_index(), "Stream descriptor without index in set_timeseries_descriptor"); - google::protobuf::Any any; - any.PackFrom(tsd.proto()); - set_metadata(std::move(any)); +void SegmentInMemoryImpl::set_timeseries_descriptor(const TimeseriesDescriptor& tsd) { + tsd_ = tsd; } -void SegmentInMemoryImpl::set_metadata(google::protobuf::Any&& meta) { - util::check_arg(!metadata_, "Cannot override previously set metadata"); - if (meta.ByteSizeLong()) - metadata_ = std::make_unique(std::move(meta)); +void SegmentInMemoryImpl::reset_metadata() { + metadata_.reset(); } -void SegmentInMemoryImpl::override_metadata(google::protobuf::Any&& meta) { +void SegmentInMemoryImpl::set_metadata(google::protobuf::Any&& meta) { + util::check_arg(!metadata_, "Cannot override previously set metadata"); if (meta.ByteSizeLong()) metadata_ = std::make_unique(std::move(meta)); } diff --git a/cpp/arcticdb/column_store/memory_segment_impl.hpp b/cpp/arcticdb/column_store/memory_segment_impl.hpp index 9747be880b..029af2861e 100644 --- a/cpp/arcticdb/column_store/memory_segment_impl.hpp +++ b/cpp/arcticdb/column_store/memory_segment_impl.hpp @@ -32,7 +32,7 @@ namespace arcticdb { class SegmentInMemoryImpl; namespace { -inline std::shared_ptr allocate_sparse_segment(const StreamId& id, const IndexDescriptor& index); +inline std::shared_ptr allocate_sparse_segment(const StreamId& id, const IndexDescriptorImpl& index); inline std::shared_ptr allocate_dense_segment(const StreamDescriptor& descriptor, size_t row_count); @@ -51,7 +51,7 @@ inline void check_output_bitset(const arcticdb::util::BitSet& output, "Mismatch in output bitset in filter_segment"); } } -} // namespace anon +} // namespace class SegmentInMemoryImpl { @@ -417,12 +417,14 @@ class SegmentInMemoryImpl { row_id_++; } - std::shared_ptr index_fields() const { - return index_fields_; + const TimeseriesDescriptor& index_descriptor() const { + util::check(tsd_.has_value(), "Index descriptor requested but not set"); + return *tsd_; } - void set_index_fields(std::shared_ptr index_fields) { - index_fields_ = std::move(index_fields); + TimeseriesDescriptor& mutable_index_descriptor() { + util::check(tsd_.has_value(), "Index descriptor requested but not set"); + return *tsd_; } void end_block_write(ssize_t size) { @@ -717,8 +719,10 @@ class SegmentInMemoryImpl { StringPool &string_pool() { return *string_pool_; } //TODO protected + void reset_metadata(); + void set_metadata(google::protobuf::Any&& meta); - void override_metadata(google::protobuf::Any&& meta); + bool has_metadata() const; const google::protobuf::Any* metadata() const; @@ -762,21 +766,16 @@ class SegmentInMemoryImpl { bool filter_down_stringpool=false, bool validate=false) const; - std::shared_ptr timeseries_proto(); - - TimeseriesDescriptor index_descriptor() { - return {timeseries_proto(), index_fields_}; + bool has_index_descriptor() const { + return tsd_.has_value(); } - bool has_index_fields() const { - return static_cast(index_fields_); + TimeseriesDescriptor&& detach_index_descriptor() { + util::check(tsd_.has_value(), "No index descriptor on segment"); + return std::move(*tsd_); } - FieldCollection&& detach_index_fields() { - return std::move(*index_fields_); - } - - void set_timeseries_descriptor(TimeseriesDescriptor&& tsd); + void set_timeseries_descriptor(const TimeseriesDescriptor& tsd); /// @brief Construct a copy of the segment containing only rows in [start_row; end_row) /// @param start_row Start of the row range (inclusive) @@ -800,10 +799,6 @@ class SegmentInMemoryImpl { std::vector> split(size_t rows) const; - StreamId get_index_col_name() const{ - return descriptor().id(); - } - private: ssize_t row_id_ = -1; std::shared_ptr descriptor_ = std::make_shared(); @@ -816,12 +811,11 @@ class SegmentInMemoryImpl { bool allow_sparse_ = false; bool compacted_ = false; util::MagicNum<'M', 'S', 'e', 'g'> magic_; - std::shared_ptr index_fields_; - std::shared_ptr tsd_; + std::optional tsd_; }; namespace { -inline std::shared_ptr allocate_sparse_segment(const StreamId& id, const IndexDescriptor& index) { +inline std::shared_ptr allocate_sparse_segment(const StreamId& id, const IndexDescriptorImpl& index) { return std::make_shared(StreamDescriptor{id, index}, 0, false, true); } diff --git a/cpp/arcticdb/column_store/string_pool.cpp b/cpp/arcticdb/column_store/string_pool.cpp index c16d3bc61b..c96c436e4f 100644 --- a/cpp/arcticdb/column_store/string_pool.cpp +++ b/cpp/arcticdb/column_store/string_pool.cpp @@ -135,6 +135,10 @@ void StringPool::set_allow_sparse(bool) { // Not used } +size_t StringPool::num_blocks() const { + return block_.num_blocks(); +} + OffsetString StringPool::get(std::string_view s, bool deduplicate) { if(deduplicate) { if (auto it = map_.find(s); it != map_.end()) diff --git a/cpp/arcticdb/column_store/string_pool.hpp b/cpp/arcticdb/column_store/string_pool.hpp index 2f4ef6b8a8..2146aea3d0 100644 --- a/cpp/arcticdb/column_store/string_pool.hpp +++ b/cpp/arcticdb/column_store/string_pool.hpp @@ -101,6 +101,10 @@ class StringBlock { uint8_t * pos_data(size_t required_size); + [[nodiscard]] size_t num_blocks() { + return data_.buffer().num_blocks(); + } + StringHead* head_at(position_t pos) { auto data = data_.buffer().ptr_cast(pos, sizeof(StringHead)); return reinterpret_cast(data); @@ -167,6 +171,8 @@ class StringPool { size_t size() const; + [[nodiscard]] size_t num_blocks() const; + py::buffer_info as_buffer_info() const; std::optional get_offset_for_column(std::string_view str, const Column& column); diff --git a/cpp/arcticdb/column_store/test/ingestion_stress_test.cpp b/cpp/arcticdb/column_store/test/ingestion_stress_test.cpp index 5204eafdf2..a5fb2ac006 100644 --- a/cpp/arcticdb/column_store/test/ingestion_stress_test.cpp +++ b/cpp/arcticdb/column_store/test/ingestion_stress_test.cpp @@ -172,7 +172,6 @@ TEST_F(IngestionStressStore, ScalarIntDynamicSchema) { auto new_descriptor = index.create_stream_descriptor(symbol, columns_second.clone()); // Now write again. - for (timestamp i = 0; i < NumRows; ++i) { agg.start_row(timestamp{i + NumRows})([&](auto &rb) { for (uint64_t j = 1u; j < NumColumnsSecondWrite; ++j) @@ -199,10 +198,7 @@ TEST_F(IngestionStressStore, ScalarIntDynamicSchema) { }); } - agg.commit(); - - for(auto &seg : sink.segments_) { log::version().info("Writing to symbol: {}", symbol); arcticdb::append_incomplete_segment(test_store_->_test_get_store(), symbol, std::move(seg)); diff --git a/cpp/arcticdb/column_store/test/test_column.cpp b/cpp/arcticdb/column_store/test/test_column.cpp index bd652428a5..ccb27b5c78 100644 --- a/cpp/arcticdb/column_store/test/test_column.cpp +++ b/cpp/arcticdb/column_store/test/test_column.cpp @@ -53,7 +53,7 @@ void test_column_type(size_t num_values = 20, size_t num_tests = 50) { ASSERT_EQ(*column.ptr_cast(index, sizeof(raw_type)), start); if constexpr (dimensions == Dimension::Dim0) { - ASSERT_EQ(column.index_of(start).value(), index); + ASSERT_EQ(column.search_unsorted(start).value(), index); auto s = column.scalar_at(j); ASSERT_FALSE(s == std::nullopt); ASSERT_EQ(s.value(), start); diff --git a/cpp/arcticdb/column_store/test/test_index_filtering.cpp b/cpp/arcticdb/column_store/test/test_index_filtering.cpp index a2469d23e7..4dc1e5bf16 100644 --- a/cpp/arcticdb/column_store/test/test_index_filtering.cpp +++ b/cpp/arcticdb/column_store/test/test_index_filtering.cpp @@ -13,6 +13,7 @@ #include #include #include +#include namespace arcticdb { using namespace arcticdb::pipelines; @@ -20,7 +21,7 @@ using namespace arcticdb::pipelines; std::pair> get_sample_slice_and_key(StreamId stream_id, VersionId version_id, size_t col_slices = 1, size_t row_slices = 10) { StreamDescriptor stream_desc{ stream_id, - IndexDescriptor{1, IndexDescriptor::TIMESTAMP} + IndexDescriptorImpl{1, IndexDescriptorImpl::Type::TIMESTAMP} }; stream_desc.add_field(scalar_field(DataType::NANOSECONDS_UTC64, "time")); @@ -77,11 +78,11 @@ TEST(IndexFilter, Static) { const auto stream_id = StreamId{"thing"}; const auto version_id = VersionId{0}; - auto [metadata, slice_and_keys] = get_sample_slice_and_key(stream_id, version_id); + auto [tsd, slice_and_keys] = get_sample_slice_and_key(stream_id, version_id); const IndexPartialKey& partial_key{stream_id, version_id}; auto mock_store = std::make_shared(); - index::IndexWriter writer(mock_store, partial_key, std::move(metadata)); + index::IndexWriter writer(mock_store, partial_key, std::move(tsd)); for (auto &slice_and_key : slice_and_keys) { writer.add(slice_and_key.key(), slice_and_key.slice()); @@ -89,6 +90,7 @@ TEST(IndexFilter, Static) { auto key_fut = writer.commit(); auto key = std::move(key_fut).get(); auto seg = mock_store->read(key, storage::ReadKeyOpts{}).get(); + pipelines::index::IndexSegmentReader isr{std::move(seg.second)}; auto pipeline_context = std::make_shared(StreamDescriptor{isr.tsd().as_stream_descriptor()}); diff --git a/cpp/arcticdb/entity/descriptor_item.hpp b/cpp/arcticdb/entity/descriptor_item.hpp index bcf81f27aa..21136364fe 100644 --- a/cpp/arcticdb/entity/descriptor_item.hpp +++ b/cpp/arcticdb/entity/descriptor_item.hpp @@ -18,7 +18,7 @@ struct DescriptorItem { entity::AtomKey &&key, std::optional start_index, std::optional end_index, - std::optional timeseries_descriptor) : + std::optional timeseries_descriptor) : key_(std::move(key)), start_index_(start_index), end_index_(end_index), @@ -30,13 +30,13 @@ struct DescriptorItem { entity::AtomKey key_; std::optional start_index_; std::optional end_index_; - std::optional timeseries_descriptor_; + std::optional timeseries_descriptor_; std::string symbol() const { return fmt::format("{}", key_.id()); } uint64_t version() const { return key_.version_id(); } timestamp creation_ts() const { return key_.creation_ts(); } std::optional start_index() const { return start_index_; } std::optional end_index() const { return end_index_; } - std::optional timeseries_descriptor() const { return timeseries_descriptor_; } + std::optional timeseries_descriptor() const { return timeseries_descriptor_; } }; } \ No newline at end of file diff --git a/cpp/arcticdb/entity/field_collection.hpp b/cpp/arcticdb/entity/field_collection.hpp index 891e5a975d..21e9bc8191 100644 --- a/cpp/arcticdb/entity/field_collection.hpp +++ b/cpp/arcticdb/entity/field_collection.hpp @@ -179,6 +179,10 @@ class FieldCollection { return {&buffer_.buffer(), &shapes_.buffer(), type_, nullptr}; } + size_t num_blocks() const { + return buffer_.buffer().num_blocks(); + } + const Field& operator[](size_t pos) const { return at(pos); } @@ -202,5 +206,4 @@ FieldCollection fields_from_range(const RangeType& fields) { return output; } - } //namespace arcticdb diff --git a/cpp/arcticdb/entity/field_collection_proto.cpp b/cpp/arcticdb/entity/field_collection_proto.cpp index 1e4f8d494a..2cf066b47c 100644 --- a/cpp/arcticdb/entity/field_collection_proto.cpp +++ b/cpp/arcticdb/entity/field_collection_proto.cpp @@ -6,10 +6,10 @@ */ #include +#include namespace arcticdb { - FieldCollection fields_from_proto(const arcticdb::proto::descriptors::StreamDescriptor& desc) { FieldCollection output; for (const auto& field : desc.fields()) @@ -18,7 +18,6 @@ FieldCollection fields_from_proto(const arcticdb::proto::descriptors::StreamDesc return output; } - void proto_from_fields(const FieldCollection& fields, arcticdb::proto::descriptors::StreamDescriptor& desc) { for (const auto& field : fields) { auto new_field = desc.add_fields(); diff --git a/cpp/arcticdb/entity/field_collection_proto.hpp b/cpp/arcticdb/entity/field_collection_proto.hpp index 9f3554669f..fc71055437 100644 --- a/cpp/arcticdb/entity/field_collection_proto.hpp +++ b/cpp/arcticdb/entity/field_collection_proto.hpp @@ -8,12 +8,12 @@ #pragma once #include -#include +#include namespace arcticdb { FieldCollection fields_from_proto(const arcticdb::proto::descriptors::StreamDescriptor& desc); -void proto_from_fields(const FieldCollection& fields, arcticdb::proto::descriptors::StreamDescriptor& desc); +void proto_from_fields(const FieldCollection& fields, arcticdb::proto::descriptors::StreamDescriptor& desc); } //namespace arcticdb diff --git a/cpp/arcticdb/entity/merge_descriptors.cpp b/cpp/arcticdb/entity/merge_descriptors.cpp index ed8ea636cf..ebb904923a 100644 --- a/cpp/arcticdb/entity/merge_descriptors.cpp +++ b/cpp/arcticdb/entity/merge_descriptors.cpp @@ -15,7 +15,7 @@ StreamDescriptor merge_descriptors( const StreamDescriptor &original, const std::vector> &entries, const std::unordered_set &filtered_set, - const std::optional& default_index) { + const std::optional& default_index) { using namespace arcticdb::stream; std::vector merged_fields; std::unordered_map merged_fields_map; @@ -87,7 +87,7 @@ StreamDescriptor merge_descriptors( const StreamDescriptor &original, const std::vector> &entries, const std::vector &filtered_columns, - const std::optional& default_index) { + const std::optional& default_index) { std::unordered_set filtered_set(filtered_columns.begin(), filtered_columns.end()); return merge_descriptors(original, entries, filtered_set, default_index); } @@ -96,7 +96,7 @@ StreamDescriptor merge_descriptors( const StreamDescriptor &original, const std::vector &entries, const std::vector &filtered_columns, - const std::optional& default_index) { + const std::optional& default_index) { std::vector> fields; for (const auto &entry : entries) { fields.push_back(std::make_shared(entry.slice_.desc()->fields().clone())); @@ -109,7 +109,7 @@ StreamDescriptor merge_descriptors( const StreamDescriptor &original, const std::vector &entries, const std::unordered_set &filtered_set, - const std::optional& default_index) { + const std::optional& default_index) { std::vector> fields; for (const auto &entry : entries) { fields.push_back(std::make_shared(entry.segment(store).descriptor().fields().clone())); diff --git a/cpp/arcticdb/entity/merge_descriptors.hpp b/cpp/arcticdb/entity/merge_descriptors.hpp index be491c8f50..4058748499 100644 --- a/cpp/arcticdb/entity/merge_descriptors.hpp +++ b/cpp/arcticdb/entity/merge_descriptors.hpp @@ -13,24 +13,24 @@ StreamDescriptor merge_descriptors( const StreamDescriptor &original, const std::vector> &entries, const std::unordered_set &filtered_set, - const std::optional& default_index); + const std::optional& default_index); entity::StreamDescriptor merge_descriptors( const entity::StreamDescriptor &original, const std::vector> &entries, const std::vector &filtered_columns, - const std::optional& default_index = std::nullopt); + const std::optional& default_index = std::nullopt); entity::StreamDescriptor merge_descriptors( const entity::StreamDescriptor &original, const std::vector &entries, const std::vector &filtered_columns, - const std::optional& default_index = std::nullopt); + const std::optional& default_index = std::nullopt); entity::StreamDescriptor merge_descriptors( const std::shared_ptr& store, const entity::StreamDescriptor &original, const std::vector &entries, const std::unordered_set &filtered_set, - const std::optional& default_index = std::nullopt); + const std::optional& default_index = std::nullopt); } \ No newline at end of file diff --git a/cpp/arcticdb/entity/protobuf_mappings.cpp b/cpp/arcticdb/entity/protobuf_mappings.cpp new file mode 100644 index 0000000000..68b80e4159 --- /dev/null +++ b/cpp/arcticdb/entity/protobuf_mappings.cpp @@ -0,0 +1,140 @@ +/* Copyright 2023 Man Group Operations Limited + * + * Use of this software is governed by the Business Source License 1.1 included in the file licenses/BSL.txt. + * + * As of the Change Date specified in that file, in accordance with the Business Source License, use of this software will be governed by the Apache License, version 2.0. + */ + + +#include +#include +#include +#include +#include +#include + +namespace arcticdb { + +using namespace arcticdb::entity; + + +inline arcticdb::proto::descriptors::SortedValue sorted_value_to_proto(SortedValue sorted) { + switch (sorted) { + case SortedValue::UNSORTED: + return arcticdb::proto::descriptors::SortedValue::UNSORTED; + case SortedValue::DESCENDING: + return arcticdb::proto::descriptors::SortedValue::DESCENDING; + case SortedValue::ASCENDING: + return arcticdb::proto::descriptors::SortedValue::ASCENDING; + default: + return arcticdb::proto::descriptors::SortedValue::UNKNOWN; + } +} + +inline SortedValue sorted_value_from_proto(arcticdb::proto::descriptors::SortedValue sorted_proto) { + switch (sorted_proto) { + case arcticdb::proto::descriptors::SortedValue::UNSORTED: + return SortedValue::UNSORTED; + case arcticdb::proto::descriptors::SortedValue::DESCENDING: + return SortedValue::DESCENDING; + case arcticdb::proto::descriptors::SortedValue::ASCENDING: + return SortedValue::ASCENDING; + default: + return SortedValue::UNKNOWN; + } +} + +[[nodiscard]] arcticdb::proto::descriptors::IndexDescriptor index_descriptor_to_proto(const IndexDescriptorImpl& index_descriptor) { + arcticdb::proto::descriptors::IndexDescriptor proto; + proto.set_kind(static_cast(index_descriptor.type_)); + proto.set_field_count(index_descriptor.field_count_); + return proto; +} + +[[nodiscard]] IndexDescriptorImpl index_descriptor_from_proto(const arcticdb::proto::descriptors::IndexDescriptor index_descriptor) { + IndexDescriptorImpl output; + output.set_type(IndexDescriptor::Type(index_descriptor.kind())); + output.set_field_count(index_descriptor.field_count()); + return output; +} + +arcticdb::proto::descriptors::AtomKey encode_key(const AtomKey &key) { + arcticdb::proto::descriptors::AtomKey output; + util::variant_match(key.id(), + [&](const StringId &id) { output.set_string_id(id); }, + [&](const NumericId &id) { output.set_numeric_id(id); }); + output.set_version_id(key.version_id()); + output.set_creation_ts(key.creation_ts()); + output.set_content_hash(key.content_hash()); + + util::variant_match(key.start_index(), + [&](const StringId &id) { output.set_string_start(id); }, + [&](const NumericId &id) { output.set_numeric_start(id); }); + util::variant_match(key.end_index(), + [&](const StringId &id) { output.set_string_end(id); }, + [&](const NumericId &id) { output.set_numeric_end(id); }); + + output.set_key_type(arcticdb::proto::descriptors::KeyType (int(key.type()))); + return output; +} + +AtomKey decode_key(const arcticdb::proto::descriptors::AtomKey& input) { + StreamId stream_id = input.id_case() == input.kNumericId ? StreamId(input.numeric_id()) : StreamId(input.string_id()); + IndexValue index_start = input.index_start_case() == input.kNumericStart ? IndexValue(input.numeric_start()) : IndexValue(input.string_start()); + IndexValue index_end = input.index_end_case() == input.kNumericEnd ? IndexValue(input.numeric_end() ): IndexValue(input.string_end()); + + return atom_key_builder() + .version_id(input.version_id()) + .creation_ts(timestamp(input.creation_ts())) + .content_hash(input.content_hash()) + .start_index(index_start) + .end_index(index_end) + .build(stream_id, KeyType(input.key_type())); +} + +void copy_stream_descriptor_to_proto(const StreamDescriptor& desc, arcticdb::proto::descriptors::StreamDescriptor& proto) { + proto.set_in_bytes(desc.uncompressed_bytes()); + proto.set_out_bytes(desc.compressed_bytes()); + proto.set_sorted(arcticdb::proto::descriptors::SortedValue(desc.sorted())); + *proto.mutable_index() = index_descriptor_to_proto(desc.index()); //TODO pass as parameter + util::variant_match(desc.id(), + [&proto] (const StringId& str) { proto.set_str_id(str); }, + [&proto] (const NumericId& n) { proto.set_num_id(n); }); + + proto.mutable_fields()->Clear(); + for(const auto& field : desc.fields()) { + auto new_field = proto.mutable_fields()->Add(); + new_field->set_name(std::string(field.name())); + new_field->mutable_type_desc()->set_dimension(static_cast(field.type().dimension())); + set_data_type(field.type().data_type(), *new_field->mutable_type_desc()); + } + util::check(proto.index().kind() != arcticdb::proto::descriptors::IndexDescriptor::UNKNOWN, "Unknown index type in protobuf conversion"); +} + +arcticdb::proto::descriptors::TimeSeriesDescriptor copy_time_series_descriptor_to_proto(const TimeseriesDescriptor& tsd) { + arcticdb::proto::descriptors::TimeSeriesDescriptor output; + + output.set_total_rows(tsd.total_rows()); + if(tsd.column_groups()) + output.mutable_column_groups()->set_enabled(true); + + exchange_timeseries_proto(tsd.proto(), output); + + auto index_stream_descriptor = tsd.as_stream_descriptor(); + copy_stream_descriptor_to_proto(index_stream_descriptor, *output.mutable_stream_descriptor()); + return output; +} + +inline void set_id(arcticdb::proto::descriptors::StreamDescriptor& pb_desc, StreamId id) { + std::visit([&pb_desc](const auto& arg) { + using IdType = std::decay_t; + if constexpr (std::is_same_v) + pb_desc.set_num_id(arg); + else if constexpr (std::is_same_v) + pb_desc.set_str_id(arg); + else + util::raise_rte("Encoding unknown descriptor type"); + }, id); +} + +} //namespace arcticdb diff --git a/cpp/arcticdb/entity/protobuf_mappings.hpp b/cpp/arcticdb/entity/protobuf_mappings.hpp index f5d969889d..cd05a5ad66 100644 --- a/cpp/arcticdb/entity/protobuf_mappings.hpp +++ b/cpp/arcticdb/entity/protobuf_mappings.hpp @@ -9,45 +9,43 @@ #include #include -#include - +#include namespace arcticdb { -using namespace arcticdb::entity; - -inline arcticdb::proto::descriptors::AtomKey encode_key(const AtomKey &key) { - arcticdb::proto::descriptors::AtomKey output; - util::variant_match(key.id(), - [&](const StringId &id) { output.set_string_id(id); }, - [&](const NumericId &id) { output.set_numeric_id(id); }); - output.set_version_id(key.version_id()); - output.set_creation_ts(key.creation_ts()); - output.set_content_hash(key.content_hash()); - - util::variant_match(key.start_index(), - [&](const StringId &id) { output.set_string_start(id); }, - [&](const NumericId &id) { output.set_numeric_start(id); }); - util::variant_match(key.end_index(), - [&](const StringId &id) { output.set_string_end(id); }, - [&](const NumericId &id) { output.set_numeric_end(id); }); - - output.set_key_type(arcticdb::proto::descriptors::KeyType (int(key.type()))); - return output; -} +namespace entity { +struct StreamDescriptor; +} //namespace arcticdb::entity + +struct TimeseriesDescriptor; + +arcticdb::proto::descriptors::AtomKey encode_key(const entity::AtomKey &key); + +entity::AtomKey decode_key(const arcticdb::proto::descriptors::AtomKey& input); + +void copy_stream_descriptor_to_proto(const entity::StreamDescriptor& desc, arcticdb::proto::descriptors::StreamDescriptor& proto); + +arcticdb::proto::descriptors::TimeSeriesDescriptor copy_time_series_descriptor_to_proto(const TimeseriesDescriptor& tsd); + +inline void set_id(arcticdb::proto::descriptors::StreamDescriptor& pb_desc, StreamId id); + +[[nodiscard]] arcticdb::proto::descriptors::IndexDescriptor index_descriptor_to_proto(const entity::IndexDescriptorImpl& index_descriptor); + +[[nodiscard]] entity::IndexDescriptorImpl index_descriptor_from_proto(const arcticdb::proto::descriptors::IndexDescriptor index_descriptor); + +template +void exchange_timeseries_proto(const SourceType& source, DestType& destination) { + if (source.has_normalization()) + *destination.mutable_normalization() = source.normalization(); + + if (source.has_user_meta()) + *destination.mutable_user_meta() = source.user_meta(); + + if (source.has_next_key()) + *destination.mutable_next_key() = source.next_key(); -inline AtomKey decode_key(const arcticdb::proto::descriptors::AtomKey& input) { - StreamId stream_id = input.id_case() == input.kNumericId ? StreamId(input.numeric_id()) : StreamId(input.string_id()); - IndexValue index_start = input.index_start_case() == input.kNumericStart ? IndexValue(input.numeric_start()) : IndexValue(input.string_start()); - IndexValue index_end = input.index_end_case() == input.kNumericEnd ? IndexValue(input.numeric_end() ): IndexValue(input.string_end()); - - return atom_key_builder() - .version_id(input.version_id()) - .creation_ts(timestamp(input.creation_ts())) - .content_hash(input.content_hash()) - .start_index(index_start) - .end_index(index_end) - .build(stream_id, KeyType(input.key_type())); + if (source.has_multi_key_meta()) + *destination.mutable_multi_key_meta() = source.multi_key_meta(); } } //namespace arcticdb \ No newline at end of file diff --git a/cpp/arcticdb/entity/protobufs.hpp b/cpp/arcticdb/entity/protobufs.hpp index 91e1159fec..70ed5cb474 100644 --- a/cpp/arcticdb/entity/protobufs.hpp +++ b/cpp/arcticdb/entity/protobufs.hpp @@ -34,4 +34,4 @@ namespace arcticdb::proto { namespace nfs_backed_storage = arcticc::pb2::nfs_backed_storage_pb2; namespace utils = arcticc::pb2::utils_pb2; -} //namespace arcticdb +} //namespace arcticdb::proto diff --git a/cpp/arcticdb/entity/serialized_key.hpp b/cpp/arcticdb/entity/serialized_key.hpp index 43271bed6c..dca06a5ca9 100644 --- a/cpp/arcticdb/entity/serialized_key.hpp +++ b/cpp/arcticdb/entity/serialized_key.hpp @@ -46,12 +46,12 @@ inline VariantId variant_id_from_token(std::string_view strv, VariantType varian } } -inline VariantType variant_type_from_index_type(IndexDescriptor::Type index_type) { +inline VariantType variant_type_from_index_type(IndexDescriptorImpl::Type index_type) { switch (index_type) { - case IndexDescriptor::TIMESTAMP: - case IndexDescriptor::ROWCOUNT: + case IndexDescriptorImpl::Type::TIMESTAMP: + case IndexDescriptorImpl::Type::ROWCOUNT: return VariantType::NUMERIC_TYPE; - case IndexDescriptor::STRING: + case IndexDescriptorImpl::Type::STRING: return VariantType::STRING_TYPE; default: return VariantType::UNKNOWN_TYPE; @@ -79,7 +79,7 @@ inline AtomKey key_from_old_style_bytes(const uint8_t *data, size_t size, KeyTyp auto cursor = std::string_view(reinterpret_cast(data), size); auto arr = util::split_to_array(cursor, OldKeyDelimiter); auto id_variant_type = variant_type_from_key_type(key_type); - auto index_type = IndexDescriptor::Type(util::num_from_strv(arr[int(OldKeyField::index_type)])); + auto index_type = IndexDescriptorImpl::Type(util::num_from_strv(arr[int(OldKeyField::index_type)])); auto index_variant_type = variant_type_from_index_type(index_type); return atom_key_from_tokens(arr, id_variant_type, index_variant_type, key_type); } @@ -162,8 +162,8 @@ inline size_t max_id_size(const VariantId& id) { } inline size_t max_index_size(const IndexDescriptor& index) { - switch(index.type()) { - case IndexDescriptor::STRING: + switch(index.type_) { + case IndexDescriptor::Type::STRING: return max_string_size(); default: return sizeof(uint64_t); @@ -178,7 +178,7 @@ struct KeyDescriptor { format_type(format_type) { } - KeyDescriptor(const StringId& id, IndexDescriptor::Type index_type, FormatType format_type) : + KeyDescriptor(const StringId& id, IndexDescriptorImpl::Type index_type, FormatType format_type) : identifier(SerializedKeyIdentifier), id_type(variant_type_from_id(id)), index_type(to_type_char(index_type)), @@ -187,13 +187,13 @@ struct KeyDescriptor { KeyDescriptor(const RefKey &key, FormatType format_type) : identifier(SerializedKeyIdentifier), id_type(variant_type_from_id(key.id())), - index_type(to_type_char(IndexDescriptor::UNKNOWN)), + index_type(to_type_char(IndexDescriptorImpl::Type::UNKNOWN)), format_type(format_type) { } char identifier; VariantType id_type; - IndexDescriptor::TypeChar index_type; + IndexDescriptorImpl::TypeChar index_type; FormatType format_type; }; diff --git a/cpp/arcticdb/entity/stream_descriptor.hpp b/cpp/arcticdb/entity/stream_descriptor.hpp index 25a9a1a71c..4b80e14946 100644 --- a/cpp/arcticdb/entity/stream_descriptor.hpp +++ b/cpp/arcticdb/entity/stream_descriptor.hpp @@ -7,78 +7,105 @@ #pragma once - -#include -#include -#include +#include +#include #include #include +#include namespace arcticdb::entity { -struct StreamDescriptor { - using Proto = arcticdb::proto::descriptors::StreamDescriptor; +struct SegmentDescriptorImpl : public SegmentDescriptor { + SegmentDescriptorImpl() = default; + + ARCTICDB_MOVE_COPY_DEFAULT(SegmentDescriptorImpl) + + [[nodiscard]] const IndexDescriptorImpl& index() const { + return static_cast(index_); + } + + IndexDescriptorImpl& index() { + return static_cast(index_); + } + + [[nodiscard]] SegmentDescriptorImpl clone() const { + return *this; + } +}; + +inline bool operator==(const SegmentDescriptorImpl& l, const SegmentDescriptorImpl& r) { + return l.sorted_ == r.sorted_ && + l.index() == r.index() && + l.compressed_bytes_ == r.compressed_bytes_ && + l.uncompressed_bytes_ == r.uncompressed_bytes_; +} - std::shared_ptr data_ = std::make_shared(); +struct StreamDescriptor { + std::shared_ptr segment_desc_ = std::make_shared(); std::shared_ptr fields_ = std::make_shared(); - ; + StreamId stream_id_; StreamDescriptor() = default; ~StreamDescriptor() = default; - [[nodiscard]] Proto copy_to_proto() const { - Proto proto; - proto.CopyFrom(*data_); - proto.mutable_fields()->Clear(); - for(const auto& field : *fields_) { - auto new_field = proto.mutable_fields()->Add(); - new_field->set_name(std::string(field.name())); - new_field->mutable_type_desc()->set_dimension(static_cast(field.type().dimension())); - set_data_type(field.type().data_type(), *new_field->mutable_type_desc()); - } - return proto; + StreamDescriptor(std::shared_ptr data, std::shared_ptr fields) : + segment_desc_(std::move(data)), + fields_(std::move(fields)) { } - void set_id(const StreamId& id) { - util::variant_match(id, - [this] (const StringId& str) { data_->set_str_id(str); }, - [this] (const NumericId& n) { data_->set_num_id(n); }); + StreamDescriptor(std::shared_ptr data, std::shared_ptr fields, StreamId stream_id) : + segment_desc_(std::move(data)), + fields_(std::move(fields)), + stream_id_(std::move(stream_id)) { } - static StreamId id_from_proto(const Proto& proto) { - if(proto.id_case() == arcticdb::proto::descriptors::StreamDescriptor::kNumId) - return NumericId(proto.num_id()); - else - return proto.str_id(); + [[nodiscard]] const SegmentDescriptorImpl& data() const { + return *segment_desc_; + } + + void set_id(const StreamId& id) { + stream_id_ = id; } [[nodiscard]] StreamId id() const { - return id_from_proto(*data_); + return stream_id_; + } + + [[nodiscard]] uint64_t uncompressed_bytes() const { + return segment_desc_->uncompressed_bytes_; } - [[nodiscard]] IndexDescriptor index() const { - return IndexDescriptor(data_->index()); + [[nodiscard]] uint64_t compressed_bytes() const { + return segment_desc_->compressed_bytes_; + } + + [[nodiscard]] SortedValue sorted() const { + return segment_desc_->sorted_; + } + + [[nodiscard]] IndexDescriptorImpl index() const { + return static_cast(segment_desc_->index_); } void set_sorted(SortedValue sorted) { - data_->set_sorted(sorted_value_to_proto(sorted)); + segment_desc_->sorted_ = sorted; } - SortedValue get_sorted() { - return sorted_value_from_proto(data_->sorted()); + void set_index(const IndexDescriptorImpl& idx) { + segment_desc_->index_ = idx; } - void set_index(const IndexDescriptor& idx) { - data_->mutable_index()->CopyFrom(idx.data_); + IndexDescriptorImpl& index() { + return static_cast(segment_desc_->index_); } - void set_index_type(const IndexDescriptor::Type type) { - data_->mutable_index()->set_kind(type); + void set_index_type(const IndexDescriptorImpl::Type type) { + index().set_type(type); } void set_index_field_count(size_t size) { - data_->mutable_index()->set_field_count(size); + index().set_field_count(size); } explicit StreamDescriptor(const StreamId& id) { @@ -89,30 +116,18 @@ struct StreamDescriptor { fields_->add_field(TypeDescriptor{data_type, Dimension::Dim0}, name); } - StreamDescriptor(const StreamId& id, const IndexDescriptor &idx, std::shared_ptr fields) { + StreamDescriptor(const StreamId& id, const IndexDescriptorImpl &idx, std::shared_ptr fields) { set_id(id); set_index(idx); util::check(static_cast(fields), "Creating field collection with null pointer"); fields_ = std::move(fields); } - StreamDescriptor(std::shared_ptr data, std::shared_ptr fields) : - data_(std::move(data)), - fields_(std::move(fields)) { - util::check(static_cast(data_), "Creating stream descriptor with null protobuf pointer"); - util::check(static_cast(fields_), "Creating stream descriptor with null fields pointer"); - } - - - StreamDescriptor(const StreamId& id, const IndexDescriptor &idx) { + StreamDescriptor(const StreamId& id, const IndexDescriptorImpl &idx) { set_id(id); set_index(idx); } - StreamDescriptor(std::shared_ptr data) : - data_(std::move(data)) { - } - StreamDescriptor(const StreamDescriptor& other) = default; StreamDescriptor& operator=(const StreamDescriptor& other) = default; @@ -122,11 +137,12 @@ struct StreamDescriptor { if(&left == &right) return; - swap(left.data_, right.data_); + swap(left.stream_id_, right.stream_id_); + swap(left.segment_desc_, right.segment_desc_); swap(left.fields_, right.fields_); } - StreamDescriptor& operator=(StreamDescriptor&& other) { + StreamDescriptor& operator=(StreamDescriptor&& other) noexcept { swap(*this, other); return *this; } @@ -137,9 +153,7 @@ struct StreamDescriptor { } [[nodiscard]] StreamDescriptor clone() const { - Proto proto; - proto.CopyFrom(*data_); - return StreamDescriptor{std::make_shared(std::move(proto)), std::make_shared(fields_->clone())}; + return StreamDescriptor{std::make_shared(segment_desc_->clone()), std::make_shared(fields_->clone()), stream_id_}; }; [[nodiscard]] const FieldCollection& fields() const { @@ -172,10 +186,14 @@ struct StreamDescriptor { return fields_->add(field); } - std::shared_ptr fields_ptr() const { + [[nodiscard]] std::shared_ptr fields_ptr() const { return fields_; } + [[nodiscard]] std::shared_ptr data_ptr() const { + return segment_desc_; + } + decltype(auto) begin() { return fields().begin(); } @@ -184,11 +202,11 @@ struct StreamDescriptor { return fields().end(); } - decltype(auto) begin() const { + [[nodiscard]] decltype(auto) begin() const { return fields().begin(); } - decltype(auto) end() const { + [[nodiscard]] decltype(auto) end() const { return fields().end(); } @@ -196,11 +214,11 @@ struct StreamDescriptor { return fields().size(); } - bool empty() const { + [[nodiscard]] bool empty() const { return fields().empty(); } - std::optional find_field(std::string_view view) const { + [[nodiscard]] std::optional find_field(std::string_view view) const { auto it = std::find_if(begin(), end(), [&](const auto& field) { return field.name() == view; }); @@ -210,8 +228,7 @@ struct StreamDescriptor { } friend bool operator==(const StreamDescriptor& left, const StreamDescriptor& right) { - google::protobuf::util::MessageDifferencer diff; - if(!diff.Compare(*left.data_, *right.data_)) + if(*left.segment_desc_ != *right.segment_desc_) return false; return *left.fields_ == *right.fields_; @@ -237,53 +254,38 @@ struct StreamDescriptor { const Field& field(size_t pos) { return fields_->at(pos); } - - [[nodiscard]] const Proto& proto() const { - return *data_; - } - - Proto& mutable_proto() { - return *data_; - } - - void print_proto_debug_str() const { - data_->PrintDebugString(); - } }; template -inline void set_index(arcticdb::proto::descriptors::StreamDescriptor &stream_desc) { - auto& pb_desc = *stream_desc.mutable_index(); - pb_desc.set_field_count(std::uint32_t(IndexType::field_count())); - pb_desc.set_kind(static_cast( - static_cast(IndexType::type()))); +inline void set_index(StreamDescriptor &stream_desc) { + stream_desc.set_index_field_count(std::uint32_t(IndexType::field_count())); + stream_desc.set_index_type(IndexType::type()); } template -StreamDescriptor index_descriptor(const StreamId& stream_id, IndexType, const RangeType& fields) { - arcticdb::proto::descriptors::StreamDescriptor desc; - set_id(desc, stream_id); +StreamDescriptor index_descriptor_from_range(const StreamId& stream_id, IndexType, const RangeType& fields) { + StreamDescriptor desc; + desc.set_id(stream_id); set_index(desc); - auto out_fields = std::make_shared(); + auto out_fields = desc.fields_ptr(); for(const auto& field : fields) { out_fields->add({field.type(), field.name()}); } - return StreamDescriptor(std::make_shared(std::move(desc)), std::move(out_fields)); + return desc; } template -StreamDescriptor index_descriptor(StreamId stream_id, IndexType index_type, - std::initializer_list fields) { - return index_descriptor(stream_id, index_type, folly::gen::from(fields) | folly::gen::as()); +StreamDescriptor index_descriptor(StreamId stream_id, IndexType index_type, std::initializer_list fields) { + return index_descriptor_from_range(stream_id, index_type, fields); } template -StreamDescriptor stream_descriptor(const StreamId& stream_id, IndexType idx, RangeType fields) { +StreamDescriptor stream_descriptor_from_range(const StreamId& stream_id, IndexType idx, RangeType fields) { StreamDescriptor output; output.set_id(stream_id); - set_index(*output.data_); + set_index(output); for(auto i = 0u; i < IndexType::field_count(); ++i) { const auto& field = idx.field(i); output.add_field(FieldRef{field.type(), field.name()}); @@ -297,23 +299,15 @@ StreamDescriptor stream_descriptor(const StreamId& stream_id, IndexType idx, Ran } template -StreamDescriptor stream_descriptor(StreamId stream_id, IndexType index_type, - std::initializer_list fields) { - std::vector vec{fields}; - return stream_descriptor(stream_id, index_type, folly::range(vec)); -} - -inline TypeDescriptor stream_id_descriptor(const StreamId &stream_id) { - return std::holds_alternative(stream_id) ? - TypeDescriptor(DataType::UINT64, 0) : - TypeDescriptor(DataType::ASCII_DYNAMIC64, 0); +StreamDescriptor stream_descriptor(StreamId stream_id, IndexType index_type, std::initializer_list fields) { + return stream_descriptor_from_range(stream_id, index_type, fields); } inline DataType stream_id_data_type(const StreamId &stream_id) { return std::holds_alternative(stream_id) ? DataType::UINT64 : DataType::ASCII_DYNAMIC64; } -inline FieldCollection field_collection_from_proto(google::protobuf::RepeatedPtrField&& fields) { +inline FieldCollection field_collection_from_proto(const google::protobuf::RepeatedPtrField& fields) { FieldCollection output; for(const auto& field : fields) { output.add_field(type_desc_from_proto(field.type_desc()), field.name()); @@ -321,7 +315,7 @@ inline FieldCollection field_collection_from_proto(google::protobuf::RepeatedPtr return output; } -} //namespace arcticdb +} //namespace arcticdb::entity namespace fmt { template<> @@ -339,13 +333,13 @@ struct formatter { }; template<> -struct formatter { +struct formatter { template constexpr auto parse(ParseContext &ctx) { return ctx.begin(); } template - auto format(const arcticdb::entity::StreamDescriptor::Proto &sd, FormatContext &ctx) const { - return fmt::format_to(ctx.out(), "{}", sd.DebugString()); + auto format(const arcticdb::proto::descriptors::StreamDescriptor &sd, FormatContext &ctx) const { + return format_to(ctx.out(), "{}", sd.DebugString()); } }; diff --git a/cpp/arcticdb/entity/test/test_atom_key.cpp b/cpp/arcticdb/entity/test/test_atom_key.cpp index 874a88ca60..18c7361e11 100644 --- a/cpp/arcticdb/entity/test/test_atom_key.cpp +++ b/cpp/arcticdb/entity/test/test_atom_key.cpp @@ -15,9 +15,11 @@ #include +using namespace arcticdb; using namespace arcticdb::entity; TEST(Key, Basic) { + using namespace arcticdb; using namespace arcticdb::entity; using namespace arcticdb::storage; @@ -108,7 +110,7 @@ struct AlternativeFormat { TEST(Key, Formatting) { AtomKey k{ - StreamId{NumericId{999}}, + arcticdb::StreamId{NumericId{999}}, VersionId(123), timestamp(123000000LL), 0x789456321ULL, diff --git a/cpp/arcticdb/entity/test/test_ref_key.cpp b/cpp/arcticdb/entity/test/test_ref_key.cpp index 9d1ea557b2..f3b5df81f8 100644 --- a/cpp/arcticdb/entity/test/test_ref_key.cpp +++ b/cpp/arcticdb/entity/test/test_ref_key.cpp @@ -11,6 +11,6 @@ TEST(RefKey, Basic) { using namespace arcticdb::entity; RefKey rk{ "HelloWorld", KeyType::STORAGE_INFO}; - ASSERT_EQ(rk.id(), VariantId("HelloWorld")); + ASSERT_EQ(rk.id(), arcticdb::VariantId("HelloWorld")); ASSERT_EQ(rk.type(), KeyType::STORAGE_INFO); } \ No newline at end of file diff --git a/cpp/arcticdb/entity/timeseries_descriptor.hpp b/cpp/arcticdb/entity/timeseries_descriptor.hpp index cd3b080419..ed686b21bb 100644 --- a/cpp/arcticdb/entity/timeseries_descriptor.hpp +++ b/cpp/arcticdb/entity/timeseries_descriptor.hpp @@ -12,73 +12,144 @@ namespace arcticdb { +struct FrameDescriptorImpl : public FrameDescriptor { + FrameDescriptorImpl() = default; + + ARCTICDB_MOVE_COPY_DEFAULT(FrameDescriptorImpl) + + [[nodiscard]] FrameDescriptorImpl clone() const { + return *this; + } +}; + struct TimeseriesDescriptor { - using Proto = arcticdb::proto::descriptors::TimeSeriesDescriptor; - - std::shared_ptr proto_ = std::make_shared(); - std::shared_ptr fields_ = std::make_shared(); - TimeseriesDescriptor() = default; - - TimeseriesDescriptor(std::shared_ptr proto, std::shared_ptr fields) : - proto_(std::move(proto)), - fields_(std::move(fields)) { - } - - [[nodiscard]] std::shared_ptr fields_ptr() const { - return fields_; - } - - [[nodiscard]] std::shared_ptr proto_ptr() const { - return proto_; - } - - [[nodiscard]] bool proto_is_null() const { - return !proto_; - } - - void set_stream_descriptor(const StreamDescriptor& desc) { - fields_ = std::make_shared(desc.fields().clone()); - proto_ = std::make_shared(); - proto_->mutable_stream_descriptor()->CopyFrom(desc.proto()); - } - - [[nodiscard]] const FieldCollection& fields() const { - return *fields_; - } - - [[nodiscard]] FieldCollection& mutable_fields() { - return *fields_; - } - - [[nodiscard]] Proto& mutable_proto() { - return *proto_; - } - - [[nodiscard]] const Proto& proto() const { - return *proto_; - } - - [[nodiscard]] TimeseriesDescriptor clone() const { - auto proto = std::make_shared(); - proto->CopyFrom(*proto_); - return {std::move(proto), std::make_shared(fields_->clone())}; - } - - [[nodiscard]] StreamDescriptor as_stream_descriptor() const { - auto stream_descriptor = std::make_shared(); - stream_descriptor->CopyFrom(proto_->stream_descriptor()); - return StreamDescriptor(stream_descriptor, fields_); - } - - void copy_to_self_proto() { - proto_->mutable_stream_descriptor()->mutable_fields()->Clear(); - for(const auto& field : *fields_) { - auto new_field = proto_->mutable_stream_descriptor()->mutable_fields()->Add(); - new_field->set_name(std::string(field.name())); - new_field->mutable_type_desc()->set_dimension(static_cast(field.type().dimension())); - set_data_type(field.type().data_type(), *new_field->mutable_type_desc()); - } - } + using Proto = arcticdb::proto::descriptors::FrameMetadata; + + std::shared_ptr frame_data_ = std::make_shared(); + std::shared_ptr segment_desc_ = std::make_shared(); + std::shared_ptr proto_ = std::make_shared(); + std::shared_ptr fields_ = std::make_shared(); + StreamId stream_id_; + + TimeseriesDescriptor() = default; + + TimeseriesDescriptor( + std::shared_ptr frame_desc, + std::shared_ptr segment_desc, + std::shared_ptr proto, + std::shared_ptr fields, + StreamId stream_id) : + frame_data_(std::move(frame_desc)), + segment_desc_(segment_desc), + proto_(std::move(proto)), + fields_(std::move(fields)), + stream_id_(stream_id) { + } + + [[nodiscard]] const FrameDescriptorImpl &frame_descriptor() const { + return *frame_data_; + } + + [[nodiscard]] IndexDescriptorImpl index() const { + return segment_desc_->index_; + } + + void set_stream_descriptor(const StreamDescriptor &desc) { + segment_desc_ = desc.data_ptr(); + fields_ = desc.fields_ptr(); + } + + void set_total_rows(uint64_t rows) { + frame_data_->total_rows_ = rows; + } + + [[nodiscard]] uint64_t total_rows() const { + return frame_data_->total_rows_; + } + + [[nodiscard]] SortedValue sorted() const { + return segment_desc_->sorted_; + } + + void set_sorted(SortedValue sorted) { + segment_desc_->sorted_ = sorted; + } + + const arcticdb::proto::descriptors::UserDefinedMetadata& user_metadata() const { + return proto_->user_meta(); + } + + const arcticdb::proto::descriptors::NormalizationMetadata normalization() const { + return proto_->normalization(); + } + + arcticdb::proto::descriptors::UserDefinedMetadata &&detach_user_metadata() { + return std::move(*proto_->mutable_user_meta()); + } + + arcticdb::proto::descriptors::NormalizationMetadata &&detach_normalization_metadata() { + return std::move(*proto_->mutable_normalization()); + } + + arcticdb::proto::descriptors::UserDefinedMetadata &&detach_multi_key_metadata() { + return std::move(*proto_->mutable_multi_key_meta()); + } + + void set_user_metadata(arcticdb::proto::descriptors::UserDefinedMetadata &&user_meta) { + *proto_->mutable_user_meta() = std::move(user_meta); + } + + void set_normalization_metadata(arcticdb::proto::descriptors::NormalizationMetadata &&norm_meta) { + *proto_->mutable_normalization() = std::move(norm_meta); + } + + void set_multi_key_metadata(arcticdb::proto::descriptors::UserDefinedMetadata &&multi_key_meta) { + *proto_->mutable_multi_key_meta() = std::move(multi_key_meta); + } + + [[nodiscard]] std::shared_ptr fields_ptr() const { + return fields_; + } + + [[nodiscard]] std::shared_ptr proto_ptr() const { + return proto_; + } + + [[nodiscard]] bool proto_is_null() const { + return !proto_; + } + + [[nodiscard]] const FieldCollection &fields() const { + return *fields_; + } + + [[nodiscard]] FieldCollection &mutable_fields() { + return *fields_; + } + + [[nodiscard]] Proto &mutable_proto() { + return *proto_; + } + + [[nodiscard]] const Proto &proto() const { + return *proto_; + } + + [[nodiscard]] TimeseriesDescriptor clone() const { + auto proto = std::make_shared(); + proto->CopyFrom(*proto_); + auto frame_desc = std::make_shared(frame_data_->clone()); + auto segment_desc = std::make_shared(segment_desc_->clone()); + return {std::move(frame_desc), std::move(segment_desc), std::move(proto), std::make_shared(fields_->clone()), stream_id_}; + } + + [[nodiscard]] bool column_groups() const { + return frame_data_->column_groups_; + } + + [[nodiscard]] StreamDescriptor as_stream_descriptor() const { + return {segment_desc_, fields_, stream_id_}; + } }; } //namespace arcticdb diff --git a/cpp/arcticdb/entity/type_utils.cpp b/cpp/arcticdb/entity/type_utils.cpp index 83df12de1d..23ca1a1a8f 100644 --- a/cpp/arcticdb/entity/type_utils.cpp +++ b/cpp/arcticdb/entity/type_utils.cpp @@ -8,7 +8,6 @@ #include #include -#include namespace arcticdb { bool trivially_compatible_types(const entity::TypeDescriptor& left, const entity::TypeDescriptor& right) { @@ -131,13 +130,6 @@ namespace arcticdb { return target; } - std::optional has_valid_type_promotion( - const proto::descriptors::TypeDescriptor& source, - const proto::descriptors::TypeDescriptor& target - ) { - return has_valid_type_promotion(entity::type_desc_from_proto(source), entity::type_desc_from_proto(target)); - } - std::optional has_valid_common_type( const entity::TypeDescriptor& left, const entity::TypeDescriptor& right @@ -178,12 +170,4 @@ namespace arcticdb { } return maybe_common_type; } - - std::optional has_valid_common_type( - const proto::descriptors::TypeDescriptor& left, - const proto::descriptors::TypeDescriptor& right - ) { - return has_valid_common_type(entity::type_desc_from_proto(left), entity::type_desc_from_proto(right)); - } - } \ No newline at end of file diff --git a/cpp/arcticdb/entity/types-inl.hpp b/cpp/arcticdb/entity/types-inl.hpp index 8c939a0aed..edb9bcc2fc 100644 --- a/cpp/arcticdb/entity/types-inl.hpp +++ b/cpp/arcticdb/entity/types-inl.hpp @@ -126,12 +126,36 @@ struct formatter { }; template<> -struct formatter { +struct formatter { template constexpr auto parse(ParseContext &ctx) { return ctx.begin(); } template - constexpr auto format(const arcticdb::entity::StreamId &tsid, FormatContext &ctx) const { + auto format(const arcticdb::entity::Field &fd, FormatContext &ctx) const { + if (!fd.name().empty()) + return format_to(ctx.out(), "FD", fd.name(), fd.type()); + else + return format_to(ctx.out(), "FD", fd.type()); + } +}; + +template<> +struct formatter { + template + constexpr auto parse(ParseContext &ctx) { return ctx.begin(); } + + template + auto format(const arcticdb::entity::IndexDescriptorImpl &idx, FormatContext &ctx) const { + return format_to(ctx.out(), "IDX", idx.field_count(), static_cast(idx.type())); + } +}; +template<> +struct formatter { + template + constexpr auto parse(ParseContext &ctx) { return ctx.begin(); } + + template + constexpr auto format(const arcticdb::StreamId &tsid, FormatContext &ctx) const { return std::visit([&ctx](auto &&val) { return fmt::format_to(ctx.out(), "{}", val); }, tsid); diff --git a/cpp/arcticdb/entity/types.hpp b/cpp/arcticdb/entity/types.hpp index b158ae91a4..4db2b90a2c 100644 --- a/cpp/arcticdb/entity/types.hpp +++ b/cpp/arcticdb/entity/types.hpp @@ -9,6 +9,10 @@ #include #include +#include +#include +#include +#include #include #include @@ -16,7 +20,6 @@ #include #include - #ifdef _WIN32 // `ssize_t` is defined in `sys/types.h` but it is not ISO C (it simply is POSIX), hence its is not defined natively by MSVC. // See: https://learn.microsoft.com/en-us/windows/win32/winprog/windows-data-types @@ -24,20 +27,22 @@ using ssize_t = SSIZE_T; #endif -namespace arcticdb::entity { +#include -enum class SortedValue : uint8_t { - UNKNOWN = 0, - UNSORTED = 1, - ASCENDING = 2, - DESCENDING = 3, -}; +namespace arcticdb::proto { + namespace descriptors = arcticc::pb2::descriptors_pb2; +} + +namespace arcticdb { using NumericId = int64_t; using UnsignedId = uint64_t; using StringId = std::string; using VariantId = std::variant; using StreamId = VariantId; + +namespace entity { + using SnapshotId = VariantId; using VersionId = uint64_t; using SignedVersionId = int64_t; @@ -49,7 +54,8 @@ using position_t = int64_t; /** The VariantId holds int64 (NumericId) but is also used to store sizes up to uint64, so needs safe conversion */ inline NumericId safe_convert_to_numeric_id(uint64_t input) { - util::check(input <= static_cast(std::numeric_limits::max()), "Numeric symbol greater than 2^63 is not supported."); + util::check(input <= static_cast(std::numeric_limits::max()), + "Numeric symbol greater than 2^63 is not supported."); return static_cast(input); } @@ -98,34 +104,34 @@ enum class ValueType : uint8_t { }; // Sequence types are composed of more than one element -constexpr bool is_sequence_type(ValueType v){ +constexpr bool is_sequence_type(ValueType v) { return uint8_t(v) >= uint8_t(ValueType::ASCII_FIXED) && uint8_t(v) <= uint8_t(ValueType::ASCII_DYNAMIC); } -constexpr bool is_numeric_type(ValueType v){ +constexpr bool is_numeric_type(ValueType v) { return v == ValueType::NANOSECONDS_UTC || (uint8_t(v) >= uint8_t(ValueType::UINT) && uint8_t(v) <= uint8_t(ValueType::FLOAT)); } -constexpr bool is_floating_point_type(ValueType v){ +constexpr bool is_floating_point_type(ValueType v) { return uint8_t(v) == uint8_t(ValueType::FLOAT); } -constexpr bool is_time_type(ValueType v){ +constexpr bool is_time_type(ValueType v) { return uint8_t(v) == uint8_t(ValueType::NANOSECONDS_UTC); } -constexpr bool is_integer_type(ValueType v){ +constexpr bool is_integer_type(ValueType v) { return uint8_t(v) == uint8_t(ValueType::INT) || uint8_t(v) == uint8_t(ValueType::UINT); } -constexpr bool is_fixed_string_type(ValueType v){ +constexpr bool is_fixed_string_type(ValueType v) { return v == ValueType::ASCII_FIXED || v == ValueType::UTF8_FIXED; } -constexpr bool is_dynamic_string_type(ValueType v){ +constexpr bool is_dynamic_string_type(ValueType v) { return is_sequence_type(v) && !is_fixed_string_type(v); } constexpr bool is_utf_type(ValueType v) { @@ -155,7 +161,7 @@ constexpr SizeBits get_size_bits(uint8_t size) { } [[nodiscard]] constexpr int get_byte_count(SizeBits size_bits) { - switch(size_bits) { + switch (size_bits) { case SizeBits::S8: return 1; case SizeBits::S16: return 2; case SizeBits::S32: return 4; @@ -164,7 +170,7 @@ constexpr SizeBits get_size_bits(uint8_t size) { } } -namespace detail{ +namespace detail { constexpr uint8_t combine_val_bits(ValueType v, SizeBits b = SizeBits::UNKNOWN_SIZE_BITS) { return (static_cast(v) << 3u) | static_cast(b); @@ -202,13 +208,13 @@ constexpr DataType combine_data_type(ValueType v, SizeBits b = SizeBits::UNKNOWN } // Constructs the corresponding DataType from a given primitive arithmetic type (u/int8_t, float, or double) -template +template constexpr DataType data_type_from_raw_type() { static_assert(std::is_arithmetic_v); if constexpr (std::is_floating_point_v) { return combine_data_type(ValueType::FLOAT, get_size_bits(sizeof(T))); } - if constexpr(std::is_signed_v) { + if constexpr (std::is_signed_v) { return combine_data_type(ValueType::INT, get_size_bits(sizeof(T))); } return combine_data_type(ValueType::UINT, get_size_bits(sizeof(T))); @@ -233,11 +239,11 @@ constexpr size_t get_type_size(DataType dt) noexcept { return size_t(1) << (size_t(s) - 1); } -constexpr bool is_sequence_type(DataType v){ +constexpr bool is_sequence_type(DataType v) { return is_sequence_type(slice_value_type(v)); } -constexpr bool is_numeric_type(DataType v){ +constexpr bool is_numeric_type(DataType v) { return is_numeric_type(slice_value_type(v)); } @@ -257,31 +263,31 @@ constexpr bool is_signed_type(DataType dt) { return slice_value_type(dt) == ValueType::INT; } -constexpr bool is_floating_point_type(DataType v){ +constexpr bool is_floating_point_type(DataType v) { return is_floating_point_type(slice_value_type(v)); } -constexpr bool is_time_type(DataType v){ +constexpr bool is_time_type(DataType v) { return is_time_type(slice_value_type(v)); } -constexpr bool is_integer_type(DataType v){ +constexpr bool is_integer_type(DataType v) { return is_integer_type(slice_value_type(v)); } -constexpr bool is_fixed_string_type(DataType v){ +constexpr bool is_fixed_string_type(DataType v) { return is_fixed_string_type(slice_value_type(v)); } -constexpr bool is_dynamic_string_type(DataType v){ +constexpr bool is_dynamic_string_type(DataType v) { return is_dynamic_string_type(slice_value_type(v)); } -constexpr bool is_utf_type(DataType v){ +constexpr bool is_utf_type(DataType v) { return is_utf_type(slice_value_type(v)); } -constexpr bool is_empty_type(DataType v){ +constexpr bool is_empty_type(DataType v) { return is_empty_type(slice_value_type(v)); } @@ -289,8 +295,8 @@ static_assert(slice_value_type(DataType::UINT16) == ValueType(1)); static_assert(get_type_size(DataType::UINT32) == 4); static_assert(get_type_size(DataType::UINT64) == 8); -constexpr ValueType get_value_type(char specifier) noexcept { - switch(specifier){ +constexpr ValueType get_value_type(char specifier) noexcept { + switch (specifier) { case 'u': return ValueType::UINT; // unsigned integer case 'i': return ValueType::INT; // signed integer case 'f': return ValueType::FLOAT; // floating-point @@ -311,10 +317,10 @@ constexpr ValueType get_value_type(char specifier) noexcept { } } -constexpr char get_dtype_specifier(ValueType vt){ - switch(vt){ +constexpr char get_dtype_specifier(ValueType vt) { + switch (vt) { case ValueType::UINT: return 'u'; - case ValueType::INT: return 'i'; + case ValueType::INT: return 'i'; case ValueType::FLOAT: return 'f'; case ValueType::BOOL: return 'b'; // NOTE: this is safe as of Pandas < 2.0 because `datetime64` _always_ has been using nanosecond resolution, @@ -329,12 +335,11 @@ constexpr char get_dtype_specifier(ValueType vt){ case ValueType::ASCII_FIXED: return 'S'; case ValueType::BYTES: return 'O'; case ValueType::EMPTY: return 'O'; - default: - return 'x'; + default:return 'x'; } } -constexpr char get_dtype_specifier(DataType dt){ +constexpr char get_dtype_specifier(DataType dt) { return get_dtype_specifier(slice_value_type(dt)); } @@ -356,7 +361,6 @@ struct DataTypeTag : public DataTypeTagBase { \ }; \ using TAG_##__DT__ = DataTypeTag; - DATA_TYPE_TAG(UINT8, std::uint8_t) DATA_TYPE_TAG(UINT16, std::uint16_t) DATA_TYPE_TAG(UINT32, std::uint32_t) @@ -403,7 +407,7 @@ Dimension as_dim_checked(uint8_t d); struct TypeDescriptor; -inline void set_data_type(DataType data_type, TypeDescriptor& type_desc); +inline void set_data_type(DataType data_type, TypeDescriptor &type_desc); struct TypeDescriptor { DataType data_type_; @@ -450,7 +454,6 @@ struct TypeDescriptor { } }; - /// @brief Check if the type must contain data /// Some types are allowed not to have any data, e.g. empty arrays or the empty type (which by design denotes the /// lack of data). @@ -467,11 +470,11 @@ constexpr bool is_numpy_array(TypeDescriptor td) { } constexpr bool is_pyobject_type(TypeDescriptor td) { - return is_dynamic_string_type(slice_value_type(td.data_type())) || is_bool_object_type(td.data_type()) || - is_numpy_array(td); + return is_dynamic_string_type(slice_value_type(td.data_type())) || is_bool_object_type(td.data_type()) || + is_numpy_array(td); } -inline void set_data_type(DataType data_type, TypeDescriptor& type_desc) { +inline void set_data_type(DataType data_type, TypeDescriptor &type_desc) { type_desc.data_type_ = data_type; } @@ -502,7 +505,7 @@ struct TypeDescriptorTag { } }; -template +template using ScalarTagType = TypeDescriptorTag>; template @@ -512,6 +515,66 @@ struct ScalarTypeInfo { using RawType = typename TDT::DataTypeTag::raw_type; }; +struct IndexDescriptorImpl : public IndexDescriptor { + using TypeChar = char; + + IndexDescriptorImpl() = default; + + IndexDescriptorImpl(uint32_t field_count, Type type) : + IndexDescriptor(type, field_count) { + } + + IndexDescriptorImpl(const IndexDescriptor& idx) : + IndexDescriptor(idx) { + } + + [[nodiscard]] bool uninitialized() const { + return field_count() == 0 && type_ == Type::UNKNOWN; + } + + [[nodiscard]] uint32_t field_count() const { + return field_count_; + } + + [[nodiscard]] Type type() const { + return type_; + } + + void set_type(Type type) { + type_ = type; + } + + void set_field_count(uint32_t field_count) { + field_count_ = field_count; + } + + ARCTICDB_MOVE_COPY_DEFAULT(IndexDescriptorImpl) + + friend bool operator==(const IndexDescriptorImpl &left, const IndexDescriptorImpl &right) { + return left.type() == right.type() && left.field_count_ == right.field_count_; + } +}; + +constexpr IndexDescriptorImpl::TypeChar to_type_char(IndexDescriptorImpl::Type type) { + switch (type) { + case IndexDescriptorImpl::Type::TIMESTAMP:return 'T'; + case IndexDescriptorImpl::Type::ROWCOUNT:return 'R'; + case IndexDescriptorImpl::Type::STRING:return 'S'; + case IndexDescriptorImpl::Type::UNKNOWN:return 'U'; + default:util::raise_rte("Unknown index type: {}", int(type)); + } +} + +constexpr IndexDescriptorImpl::Type from_type_char(IndexDescriptorImpl::TypeChar type) { + switch (type) { + case 'T': return IndexDescriptorImpl::Type::TIMESTAMP; + case 'R': return IndexDescriptorImpl::Type::ROWCOUNT; + case 'S': return IndexDescriptorImpl::Type::STRING; + case 'U': return IndexDescriptorImpl::Type::UNKNOWN; + default:util::raise_rte("Unknown index type: {}", int(type)); + } +} + struct FieldRef { TypeDescriptor type_; std::string_view name_; @@ -524,12 +587,11 @@ struct FieldRef { return name_; } - friend bool operator==(const FieldRef& left, const FieldRef& right) { + friend bool operator==(const FieldRef &left, const FieldRef &right) { return left.type_ == right.type_ && left.name_ == right.name_; } }; - struct Field { uint32_t size_ = 0; TypeDescriptor type_; @@ -538,10 +600,8 @@ struct Field { ARCTICDB_NO_MOVE_OR_COPY(Field) - - private: - explicit Field(const FieldRef& ref) { + explicit Field(const FieldRef &ref) { set(ref.type_, ref.name_); } @@ -549,8 +609,8 @@ struct Field { set(type, name); } public: - static void emplace(TypeDescriptor type, std::string_view name, void* ptr) { - new (ptr) Field(type, name); + static void emplace(TypeDescriptor type, std::string_view name, void *ptr) { + new(ptr) Field(type, name); } static size_t calc_size(std::string_view name) { @@ -561,15 +621,15 @@ struct Field { return {name_, size_}; } - [[nodiscard]] const TypeDescriptor& type() const { + [[nodiscard]] const TypeDescriptor &type() const { return type_; } - [[nodiscard]] TypeDescriptor* mutable_type_desc() { + [[nodiscard]] TypeDescriptor *mutable_type_desc() { return &type_; } - TypeDescriptor& mutable_type() { + TypeDescriptor &mutable_type() { return type_; } @@ -603,13 +663,21 @@ struct FieldWrapper { mutable_field().set(type, name); } - const Field& field() const { - return *reinterpret_cast(data_.data()); + const Field &field() const { + return *reinterpret_cast(data_.data()); + } + + const TypeDescriptor& type() const { + return field().type(); + } + + const std::string_view name() const { + return field().name(); } private: - Field& mutable_field() { - return *reinterpret_cast(data_.data()); + Field &mutable_field() { + return *reinterpret_cast(data_.data()); } }; @@ -617,29 +685,32 @@ inline FieldRef scalar_field(DataType type, std::string_view name) { return {TypeDescriptor{type, Dimension::Dim0}, name}; } -template -auto visit_field(const Field& field, Callable&& c) { +template +auto visit_field(const Field &field, Callable &&c) { return field.type().visit_tag(std::forward(c)); } -inline bool operator==(const Field& l, const Field& r) { +inline bool operator==(const Field &l, const Field &r) { return l.type() == r.type() && l.name() == r.name(); } -inline bool operator!=(const Field& l, const Field& r) { +inline bool operator!=(const Field &l, const Field &r) { return !(l == r); } -std::size_t sizeof_datatype(const TypeDescriptor& td); -} // namespace arcticdb::entity +std::size_t sizeof_datatype(const TypeDescriptor &td); + +} // namespace entity + +} // namespace arcticdb // StreamId ordering - numbers before strings namespace std { -template<> -struct less { - bool operator()(const arcticdb::entity::StreamId &left, const arcticdb::entity::StreamId &right) const { - using namespace arcticdb::entity; +template<> +struct less { + bool operator()(const arcticdb::StreamId &left, const arcticdb::StreamId &right) const { + using namespace arcticdb; if (std::holds_alternative(left)) { if (std::holds_alternative(right)) return left < right; @@ -653,7 +724,8 @@ struct less { } } }; -} + +} // namespace std namespace fmt { @@ -671,7 +743,19 @@ struct formatter { } }; -} +template<> +struct formatter { + + template + constexpr auto parse(ParseContext &ctx) { return ctx.begin(); } + + template + auto format(const FieldWrapper& f, FormatContext &ctx) const { + return fmt::format_to(ctx.out(), "{}: {}", f.type(), f.name()); + } +}; + +} //namespace fmt #define ARCTICDB_TYPES_H_ #include "types-inl.hpp" \ No newline at end of file diff --git a/cpp/arcticdb/entity/types_proto.cpp b/cpp/arcticdb/entity/types_proto.cpp index 5fd69d7e8a..2530429e83 100644 --- a/cpp/arcticdb/entity/types_proto.cpp +++ b/cpp/arcticdb/entity/types_proto.cpp @@ -5,122 +5,121 @@ * 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 namespace arcticdb::entity { - bool operator==(const FieldProto& left, const FieldProto& right) { - google::protobuf::util::MessageDifferencer diff; - return diff.Compare(left, right); +bool operator==(const FieldProto& left, const FieldProto& right) { + google::protobuf::util::MessageDifferencer diff; + return diff.Compare(left, right); +} + +bool operator<(const FieldProto& left, const FieldProto& right) { + return left.name() < right.name(); +} + +arcticdb::proto::descriptors::SortedValue sorted_value_to_proto(SortedValue sorted) { + switch (sorted) { + case SortedValue::UNSORTED: + return arcticdb::proto::descriptors::SortedValue::UNSORTED; + case SortedValue::DESCENDING: + return arcticdb::proto::descriptors::SortedValue::DESCENDING; + case SortedValue::ASCENDING: + return arcticdb::proto::descriptors::SortedValue::ASCENDING; + default: + return arcticdb::proto::descriptors::SortedValue::UNKNOWN; } - - bool operator<(const FieldProto& left, const FieldProto& right) { - return left.name() < right.name(); +} + +SortedValue sorted_value_from_proto(arcticdb::proto::descriptors::SortedValue sorted_proto) { + switch (sorted_proto) { + case arcticdb::proto::descriptors::SortedValue::UNSORTED: + return SortedValue::UNSORTED; + case arcticdb::proto::descriptors::SortedValue::DESCENDING: + return SortedValue::DESCENDING; + case arcticdb::proto::descriptors::SortedValue::ASCENDING: + return SortedValue::ASCENDING; + default: + return SortedValue::UNKNOWN; } +} +void set_data_type(DataType data_type, arcticdb::proto::descriptors::TypeDescriptor& type_desc) { + type_desc.set_size_bits( + static_cast( + static_cast(slice_bit_size(data_type)))); + type_desc.set_value_type( + static_cast( + static_cast(slice_value_type(data_type)))); +} - arcticdb::proto::descriptors::SortedValue sorted_value_to_proto(SortedValue sorted) { - switch (sorted) { - case SortedValue::UNSORTED: - return arcticdb::proto::descriptors::SortedValue::UNSORTED; - case SortedValue::DESCENDING: - return arcticdb::proto::descriptors::SortedValue::DESCENDING; - case SortedValue::ASCENDING: - return arcticdb::proto::descriptors::SortedValue::ASCENDING; - default: - return arcticdb::proto::descriptors::SortedValue::UNKNOWN; - } - } - - SortedValue sorted_value_from_proto(arcticdb::proto::descriptors::SortedValue sorted_proto) { - switch (sorted_proto) { - case arcticdb::proto::descriptors::SortedValue::UNSORTED: - return SortedValue::UNSORTED; - case arcticdb::proto::descriptors::SortedValue::DESCENDING: - return SortedValue::DESCENDING; - case arcticdb::proto::descriptors::SortedValue::ASCENDING: - return SortedValue::ASCENDING; - default: - return SortedValue::UNKNOWN; - } - } +[[nodiscard]] +auto to_proto(const TypeDescriptor& desc) +-> arcticdb::proto::descriptors::TypeDescriptor +{ + arcticdb::proto::descriptors::TypeDescriptor output; + output.set_dimension(static_cast(desc.dimension_)); + set_data_type(desc.data_type_, output); - void set_data_type(DataType data_type, arcticdb::proto::descriptors::TypeDescriptor& type_desc) { - type_desc.set_size_bits( - static_cast( - static_cast(slice_bit_size(data_type)))); - type_desc.set_value_type( - static_cast( - static_cast(slice_value_type(data_type)))); - } + return output; +} - [[nodiscard]] - auto to_proto(const TypeDescriptor& desc) - -> arcticdb::proto::descriptors::TypeDescriptor - { - arcticdb::proto::descriptors::TypeDescriptor output; - output.set_dimension(static_cast(desc.dimension_)); - set_data_type(desc.data_type_, output); - return output; - } +DataType get_data_type(const arcticdb::proto::descriptors::TypeDescriptor& type_desc) { + return combine_data_type( + static_cast(static_cast(type_desc.value_type())), + static_cast(static_cast(type_desc.size_bits())) + ); +} - - - DataType get_data_type(const arcticdb::proto::descriptors::TypeDescriptor& type_desc) { - return combine_data_type( +TypeDescriptor type_desc_from_proto(const arcticdb::proto::descriptors::TypeDescriptor& type_desc) { + return { + combine_data_type( static_cast(static_cast(type_desc.value_type())), static_cast(static_cast(type_desc.size_bits())) - ); - } - - TypeDescriptor type_desc_from_proto(const arcticdb::proto::descriptors::TypeDescriptor& type_desc) { - return { - combine_data_type( - static_cast(static_cast(type_desc.value_type())), - static_cast(static_cast(type_desc.size_bits())) - ), - static_cast(static_cast(type_desc.dimension())) - }; - } - - DataType data_type_from_proto(const arcticdb::proto::descriptors::TypeDescriptor& type_desc) { - return type_desc_from_proto(type_desc).data_type(); - } - - - arcticdb::proto::descriptors::StreamDescriptor_FieldDescriptor field_proto(DataType dt, Dimension dim, std::string_view name) { - arcticdb::proto::descriptors::StreamDescriptor_FieldDescriptor output; - if (!name.empty()) - output.set_name(name.data(), name.size()); - - auto output_desc = output.mutable_type_desc(); - output_desc->set_dimension(static_cast(dim)); - output_desc->set_size_bits(static_cast( - static_cast(slice_bit_size(dt)))); - - output_desc->set_value_type( - static_cast( - static_cast(slice_value_type(dt)))); - - return output; - } - - void set_id(arcticdb::proto::descriptors::StreamDescriptor& pb_desc, StreamId id) { - std::visit([&pb_desc](auto&& arg) { - using IdType = std::decay_t; - if constexpr (std::is_same_v) - pb_desc.set_num_id(arg); - else if constexpr (std::is_same_v) - pb_desc.set_str_id(arg); - else - util::raise_rte("Encoding unknown descriptor type"); - }, id); - } - -} // namespace arcticdb + ), + static_cast(static_cast(type_desc.dimension())) + }; +} + +DataType data_type_from_proto(const arcticdb::proto::descriptors::TypeDescriptor& type_desc) { + return type_desc_from_proto(type_desc).data_type(); +} + + +arcticdb::proto::descriptors::StreamDescriptor_FieldDescriptor field_proto(DataType dt, Dimension dim, std::string_view name) { + arcticdb::proto::descriptors::StreamDescriptor_FieldDescriptor output; + if (!name.empty()) + output.set_name(name.data(), name.size()); + + auto output_desc = output.mutable_type_desc(); + output_desc->set_dimension(static_cast(dim)); + output_desc->set_size_bits(static_cast( + static_cast(slice_bit_size(dt)))); + + output_desc->set_value_type( + static_cast( + static_cast(slice_value_type(dt)))); + + return output; +} + +void set_id(arcticdb::proto::descriptors::StreamDescriptor& pb_desc, StreamId id) { + std::visit([&pb_desc](auto&& arg) { + using IdType = std::decay_t; + if constexpr (std::is_same_v) + pb_desc.set_num_id(arg); + else if constexpr (std::is_same_v) + pb_desc.set_str_id(arg); + else + util::raise_rte("Encoding unknown descriptor type"); + }, id); +} + +} // namespace arcticdb \ No newline at end of file diff --git a/cpp/arcticdb/entity/types_proto.hpp b/cpp/arcticdb/entity/types_proto.hpp index ed54518239..5a484de9bf 100644 --- a/cpp/arcticdb/entity/types_proto.hpp +++ b/cpp/arcticdb/entity/types_proto.hpp @@ -5,166 +5,36 @@ * As of the Change Date specified in that file, in accordance with the Business Source License, use of this software will be governed by the Apache License, version 2.0. */ -#pragma once - -#include - #include +#include +#include namespace arcticdb::proto { - namespace descriptors = arcticc::pb2::descriptors_pb2; -} +namespace descriptors = arcticc::pb2::descriptors_pb2; +} //namespace arcticdb::proto namespace arcticdb::entity { - using FieldProto = arcticdb::proto::descriptors::StreamDescriptor_FieldDescriptor; - - bool operator==(const FieldProto& left, const FieldProto& right); - bool operator<(const FieldProto& left, const FieldProto& right); - - - - arcticdb::proto::descriptors::SortedValue sorted_value_to_proto(SortedValue sorted); - - SortedValue sorted_value_from_proto(arcticdb::proto::descriptors::SortedValue sorted_proto); - - - void set_data_type(DataType data_type, arcticdb::proto::descriptors::TypeDescriptor& type_desc); - - - DataType get_data_type(const arcticdb::proto::descriptors::TypeDescriptor& type_desc); - - TypeDescriptor type_desc_from_proto(const arcticdb::proto::descriptors::TypeDescriptor& type_desc); - - DataType data_type_from_proto(const arcticdb::proto::descriptors::TypeDescriptor& type_desc); - - - arcticdb::proto::descriptors::StreamDescriptor_FieldDescriptor field_proto(DataType dt, Dimension dim, std::string_view name); - - - - struct IndexDescriptor { - using Proto = arcticdb::proto::descriptors::IndexDescriptor; - - Proto data_; - using Type = arcticdb::proto::descriptors::IndexDescriptor::Type; - - static const Type UNKNOWN = arcticdb::proto::descriptors::IndexDescriptor_Type_UNKNOWN; - static const Type ROWCOUNT = arcticdb::proto::descriptors::IndexDescriptor_Type_ROWCOUNT; - static const Type STRING = arcticdb::proto::descriptors::IndexDescriptor_Type_STRING; - static const Type TIMESTAMP = arcticdb::proto::descriptors::IndexDescriptor_Type_TIMESTAMP; - - using TypeChar = char; - - IndexDescriptor() = default; - IndexDescriptor(size_t field_count, Type type) { - data_.set_kind(type); - data_.set_field_count(static_cast(field_count)); - } - - explicit IndexDescriptor(arcticdb::proto::descriptors::IndexDescriptor data) - : data_(std::move(data)) { - } - - bool uninitialized() const { - return data_.field_count() == 0 && data_.kind() == Type::IndexDescriptor_Type_UNKNOWN; - } - - const Proto& proto() const { - return data_; - } - - size_t field_count() const { - return static_cast(data_.field_count()); - } - - Type type() const { - return data_.kind(); - } - - void set_type(Type type) { - data_.set_kind(type); - } - - ARCTICDB_MOVE_COPY_DEFAULT(IndexDescriptor) - - friend bool operator==(const IndexDescriptor& left, const IndexDescriptor& right) { - return left.type() == right.type(); - } - }; - - constexpr IndexDescriptor::TypeChar to_type_char(IndexDescriptor::Type type) { - switch (type) { - case IndexDescriptor::TIMESTAMP:return 'T'; - case IndexDescriptor::ROWCOUNT:return 'R'; - case IndexDescriptor::STRING:return 'S'; - case IndexDescriptor::UNKNOWN:return 'U'; - default:util::raise_rte("Unknown index type: {}", int(type)); - } - } - - constexpr IndexDescriptor::Type from_type_char(IndexDescriptor::TypeChar type) { - switch (type) { - case 'T': return IndexDescriptor::TIMESTAMP; - case 'R': return IndexDescriptor::ROWCOUNT; - case 'S': return IndexDescriptor::STRING; - case 'U': return IndexDescriptor::UNKNOWN; - default:util::raise_rte("Unknown index type: {}", int(type)); - } - } - - void set_id(arcticdb::proto::descriptors::StreamDescriptor& pb_desc, StreamId id); - -} // namespace arcticdb::entity - +using FieldProto = arcticdb::proto::descriptors::StreamDescriptor_FieldDescriptor; -namespace fmt { +bool operator==(const FieldProto &left, const FieldProto &right); +bool operator<(const FieldProto &left, const FieldProto &right); - template<> - struct formatter { - template - constexpr auto parse(ParseContext& ctx) { return ctx.begin(); } +arcticdb::proto::descriptors::SortedValue sorted_value_to_proto(SortedValue sorted); - template - auto format(const arcticdb::proto::descriptors::TypeDescriptor& type_desc, FormatContext& ctx) const { - auto td = arcticdb::entity::type_desc_from_proto(type_desc); - return fmt::format_to(ctx.out(), "{}", td); - } - }; +SortedValue sorted_value_from_proto(arcticdb::proto::descriptors::SortedValue sorted_proto); - template<> - struct formatter { - template - constexpr auto parse(ParseContext& ctx) { return ctx.begin(); } +void set_data_type(DataType data_type, arcticdb::proto::descriptors::TypeDescriptor &type_desc); - template - auto format(const arcticdb::proto::descriptors::StreamDescriptor_FieldDescriptor& field_desc, FormatContext& ctx) const { - return fmt::format_to(ctx.out(), "{}: {}", field_desc.name(), field_desc.type_desc()); - } - }; +DataType get_data_type(const arcticdb::proto::descriptors::TypeDescriptor &type_desc); - template<> - struct formatter { - template - constexpr auto parse(ParseContext& ctx) { return ctx.begin(); } +TypeDescriptor type_desc_from_proto(const arcticdb::proto::descriptors::TypeDescriptor &type_desc); - template - auto format(const arcticdb::entity::IndexDescriptor& idx, FormatContext& ctx) const { - return fmt::format_to(ctx.out(), "IDX", idx.field_count(), static_cast(idx.type())); - } - }; +DataType data_type_from_proto(const arcticdb::proto::descriptors::TypeDescriptor &type_desc); - template<> - struct formatter { - template - constexpr auto parse(ParseContext& ctx) { return ctx.begin(); } +arcticdb::proto::descriptors::StreamDescriptor_FieldDescriptor field_proto( + DataType dt, + Dimension dim, + std::string_view name); - template - auto format(const arcticdb::entity::Field& fd, FormatContext& ctx) const { - if (!fd.name().empty()) - return fmt::format_to(ctx.out(), "FD", fd.name(), fd.type()); - else - return fmt::format_to(ctx.out(), "FD", fd.type()); - } - }; -} +} // namespace arcticdb::entity \ No newline at end of file diff --git a/cpp/arcticdb/log/log.cpp b/cpp/arcticdb/log/log.cpp index 205e3a9129..c4a28ec52e 100644 --- a/cpp/arcticdb/log/log.cpp +++ b/cpp/arcticdb/log/log.cpp @@ -72,7 +72,7 @@ struct Loggers::Impl constexpr auto get_default_log_level() { - return spdlog::level::info; + return spdlog::level::debug; } spdlog::logger &storage() { @@ -287,7 +287,7 @@ bool Loggers::configure(const arcticdb::proto::logger::LoggersConfig &conf, bool util::as_opt(sink_conf.daily_file().utc_rotation_minute()).value_or(0) )); break; - default:util::raise_error_msg("Unsupported sink_conf {}", sink_conf); + default:util::raise_rte("Unsupported sink_conf {}", sink_conf.DebugString()); } } diff --git a/cpp/arcticdb/memory_layout.hpp b/cpp/arcticdb/memory_layout.hpp new file mode 100644 index 0000000000..3cd1cc4651 --- /dev/null +++ b/cpp/arcticdb/memory_layout.hpp @@ -0,0 +1,292 @@ +#pragma once + +#include +#include + +namespace arcticdb { + +#pragma pack(push) +#pragma pack(1) + +constexpr size_t encoding_size = 6; +enum class Codec : uint16_t { + UNKNOWN = 0, + ZSTD, + PFOR, + LZ4, + PASS, + RLE, + FSST, + GORILLA_RLE, + CONSTANT +}; + +struct ZstdCodec { + static constexpr Codec type_ = Codec::ZSTD; + + int32_t level_ = 0; + bool is_streaming_ = false; + uint8_t padding_ = 0; +}; + +static_assert(sizeof(ZstdCodec) == encoding_size); + +struct Lz4Codec { + static constexpr Codec type_ = Codec::LZ4; + + int32_t acceleration_ = 1; + int16_t padding_ = 0; +}; + +static_assert(sizeof(Lz4Codec) == encoding_size); + +struct PassthroughCodec { + static constexpr Codec type_ = Codec::PASS; + + uint32_t unused_ = 0; + uint16_t padding_ = 0; +}; + +struct PforCodec { + static constexpr Codec type_ = Codec::PFOR; + + uint32_t unused_ = 0; + uint16_t padding_ = 0; + +}; + +struct BlockCodec { + Codec codec_ = Codec::UNKNOWN; + constexpr static size_t DataSize = 24; + std::array data_ = {}; +}; + +struct Block { + uint32_t in_bytes_ = 0; + uint32_t out_bytes_ = 0; + uint64_t hash_ = 0; + uint16_t encoder_version_ = 0; + bool is_shape_ = false; + uint8_t num_codecs_ = 0; + std::array codecs_; + + Block() = default; +}; + +static_assert(sizeof(Block) == 46); + +enum class EncodedFieldType : uint8_t { + UNKNOWN, + NDARRAY, + DICTIONARY +}; + +enum class BitmapFormat : uint8_t { + UNKNOWN, + DENSE, + BITMAGIC, + ROARING +}; + +struct EncodedField { + EncodedFieldType type_ = EncodedFieldType::UNKNOWN; + uint8_t shapes_count_ = 0u; + uint16_t values_count_ = 0u; + uint32_t sparse_map_bytes_ = 0u; + uint32_t items_count_ = 0u; + BitmapFormat format_ = BitmapFormat::UNKNOWN; + std::array blocks_; +}; + +static_assert(sizeof(EncodedField) == 59); + +enum class EncodingVersion : uint16_t { + V1 = 0, + V2 = 1, + COUNT = 2 +}; + +constexpr static uint16_t MAGIC_NUMBER = 0xFA57; + +struct FixedHeader { + std::uint16_t magic_number; + std::uint16_t encoding_version; + std::uint32_t header_bytes; +}; + +constexpr static std::size_t FIXED_HEADER_SIZE = sizeof(FixedHeader); + +enum class HeaderFlag : uint8_t { + COMPACTED, +}; + +struct FieldBuffer { + mutable uint32_t fields_bytes_ = 0U; + mutable uint32_t offset_bytes_ = 0U; +}; + +struct HeaderData { ; + EncodingVersion encoding_version_ = EncodingVersion::V1; + uint16_t fields_ = 0U; + uint8_t flags_ = 0U; + uint64_t footer_offset_ = 0U; + FieldBuffer field_buffer_; +}; + +enum class SortedValue : uint8_t { + UNKNOWN = 0, + UNSORTED = 1, + ASCENDING = 2, + DESCENDING = 3, +}; + +enum class SchemaType : uint8_t { + STATIC, + DYNAMIC +}; + +struct IndexDescriptor { + enum class Type : int32_t { + UNKNOWN = 0, + ROWCOUNT = 82, + STRING = 83, + TIMESTAMP = 84 + }; + + IndexDescriptor() = default; + + IndexDescriptor(Type type, uint32_t field_count) : + type_(type), + field_count_(field_count) { + } + + Type type_ = Type::UNKNOWN; + uint32_t field_count_ = 0U; +}; + +enum class FrameMetadataEncoding : uint8_t { + PROTOBUF = 0 +}; + +struct FrameDescriptor { + uint64_t total_rows_ = 0UL; + bool column_groups_ = false; + FrameMetadataEncoding metadata_encoding_ = FrameMetadataEncoding::PROTOBUF; +}; + +struct SegmentDescriptor { + SortedValue sorted_ = SortedValue::UNKNOWN; + uint64_t compressed_bytes_ = 0UL; + uint64_t uncompressed_bytes_ = 0UL; + IndexDescriptor index_; +}; + +enum class IdentifierType : uint8_t { + NUMERIC = 0, + STRING = 1 +}; + +struct SegmentIdentifierHeader { + IdentifierType type_ = IdentifierType::NUMERIC; + uint32_t size_ = 0; +}; + +// A segment header contains a set of optional fields that describe the contents of a given segment +enum class FieldOffset : uint8_t { + METADATA, // Opaque field for user and normalization metadata + STRING_POOL, // Deduplicated compressed field of string data + DESCRIPTOR, // Collection of field names and types for the current segment + INDEX, // Optional additional set of fields used when this segment indexes a dataframe + COLUMN, // Set of encoded fields that represent the body (user) data of the segment + COUNT +}; + +/* + * Note. The structures below contain variable-length fields (represented by named structures) and should not be + * used to implement the data format directly. They are intended as a syntactically-correct representation + * of the storage format only. + */ +template +struct Optional { + // Denotes a field that may or may not be present +}; + +struct FieldList { + // A list of field descriptors containing the name, type and dimensionality of a column of data +}; + +struct EncodedFieldList { + // A list of encoded fields that describes the contents of other fields. An encoded field is a list of blocks + // with a specific set of compression types +}; + +struct OpaqueField { + // An opaque field to be filled with user-determined content, used for things like + // language-specific normalization data +}; + +struct ColumnField { + ColumnMagic column_magic_; + // A data field described by an EncodedField, consists of a set of compressed blocks that may represent + // shapes and values, and an optional sparse bitmap +}; + +template +struct CompressedField { + // A compressed block of data containing some other structure. A compressed field is represented by an EncodedField + // which contains a set of Block objects describing the compression stype +}; + +template +struct RepeatedField { + // A set of fields that are repeated, whose number corresponds to a unary field describing this set. For example, the + // number of repeated column fields should correspond to the number of entries in the descriptor (which describes the + // user-facing information about a column's contents, and the number of EncodedFields in the body fields, which describe + // the block structure and compression +}; + +// Binrary representation of a segment header. Contains positioning information about the structure of the segment, +// and the list of fields representing the segment metadata fields +struct SegmentHeaderData { + HeaderData data_; + EncodedFieldList header_fields_; // Header fields containing the fields described by FieldOffsets + std::array offset_ = {}; // Maps the entries in the FieldOffset enumeration to the header field entries +}; + +// The overall memory layout of an ArcticDB segment +struct MemoryLayout { + FixedHeader fixed_header_; + SegmentHeaderData variable_header_; + + MetadataMagic metadata_magic_; + Optional metadata_; + + SegmentDescriptorMagic segment_descriptor_magic_; + SegmentDescriptor segment_descriptor_; + SegmentIdentifierHeader identifier_header_; + Optional identifier_data_; + DescriptorFieldsMagic descriptor_magic_; + CompressedField descriptor_fields_; + + IndexMagic index_magic_; + // Optional fields present if this segment refers to a complete dataframe, i.e. if it is a primary index + Optional index_frame_descriptor_; + Optional index_segment_descriptor_magic_; + Optional index_segment_descriptor_; + Optional index_identifier_header_; + Optional index_identifier_data_; + Optional> index_descriptor_fields_; + + RepeatedField columns_; + + StringPoolMagic string_pool_magic_; + Optional string_pool_field_; + + EncodedFieldList body_fields_; // Encoded field list representing the user data fields (columns) +}; + +#pragma pack(pop) + + + +} //namespace arcticdb \ No newline at end of file diff --git a/cpp/arcticdb/pipeline/column_stats.cpp b/cpp/arcticdb/pipeline/column_stats.cpp index 8bbc2459c4..5fddcb14c7 100644 --- a/cpp/arcticdb/pipeline/column_stats.cpp +++ b/cpp/arcticdb/pipeline/column_stats.cpp @@ -14,6 +14,7 @@ namespace arcticdb { SegmentInMemory merge_column_stats_segments(const std::vector& segments) { SegmentInMemory merged; merged.init_column_map(); + merged.descriptor().set_index(IndexDescriptorImpl{0, IndexDescriptor::Type::ROWCOUNT}); // Maintain the order of the columns in the input segments ankerl::unordered_dense::map field_name_to_index; @@ -22,6 +23,7 @@ SegmentInMemory merge_column_stats_segments(const std::vector& for (auto &segment : segments) { for (const auto &field: segment.descriptor().fields()) { auto new_type = field.type(); + if (auto it = field_name_to_index.find(std::string{field.name()}); it != field_name_to_index.end()) { auto &merged_type = type_descriptors.at(field_name_to_index.at(std::string{field.name()})); auto opt_common_type = has_valid_common_type(merged_type, new_type); diff --git a/cpp/arcticdb/pipeline/frame_utils.cpp b/cpp/arcticdb/pipeline/frame_utils.cpp index abd47f87bd..a90bcb683b 100644 --- a/cpp/arcticdb/pipeline/frame_utils.cpp +++ b/cpp/arcticdb/pipeline/frame_utils.cpp @@ -13,49 +13,36 @@ namespace arcticdb { TimeseriesDescriptor make_timeseries_descriptor( size_t total_rows, - StreamDescriptor&& desc, + const StreamDescriptor& desc, arcticdb::proto::descriptors::NormalizationMetadata&& norm_meta, std::optional&& um, std::optional&& prev_key, std::optional&& next_key, bool bucketize_dynamic ) { - arcticdb::proto::descriptors::TimeSeriesDescriptor time_series_descriptor; - time_series_descriptor.set_total_rows(total_rows); - *time_series_descriptor.mutable_stream_descriptor() = std::move(desc.proto()); - time_series_descriptor.mutable_normalization()->CopyFrom(norm_meta); + + auto frame_desc = std::make_shared(); + frame_desc->total_rows_ = total_rows; + frame_desc->column_groups_ = bucketize_dynamic; + + auto segment_desc = std::make_shared(); + segment_desc->index_ = desc.index(); + segment_desc->sorted_ = desc.sorted(); + + auto proto = std::make_shared(); + proto->mutable_normalization()->CopyFrom(norm_meta); auto user_meta = std::move(um); if(user_meta) - *time_series_descriptor.mutable_user_meta() = std::move(*user_meta); + *proto->mutable_user_meta() = std::move(*user_meta); if(prev_key) - *time_series_descriptor.mutable_next_key() = encode_key(prev_key.value()); + proto->mutable_next_key()->CopyFrom(encode_key(prev_key.value())); if(next_key) - time_series_descriptor.mutable_next_key()->CopyFrom(encode_key(next_key.value())); - - if(bucketize_dynamic) - time_series_descriptor.mutable_column_groups()->set_enabled(true); + proto->mutable_next_key()->CopyFrom(encode_key(next_key.value())); //TODO maybe need ensure_norm_meta? - return TimeseriesDescriptor{std::make_shared(std::move(time_series_descriptor)), desc.fields_ptr()}; -} - - -TimeseriesDescriptor timseries_descriptor_from_index_segment( - size_t total_rows, - pipelines::index::IndexSegmentReader&& index_segment_reader, - std::optional&& prev_key, - bool bucketize_dynamic -) { - return make_timeseries_descriptor( - total_rows, - StreamDescriptor{std::make_shared(std::move(*index_segment_reader.mutable_tsd().mutable_proto().mutable_stream_descriptor())),index_segment_reader.mutable_tsd().fields_ptr()}, - std::move(*index_segment_reader.mutable_tsd().mutable_proto().mutable_normalization()), - std::move(*index_segment_reader.mutable_tsd().mutable_proto().mutable_user_meta()), - std::move(prev_key), - std::nullopt, - bucketize_dynamic); + return TimeseriesDescriptor{std::move(frame_desc), std::move(segment_desc), std::move(proto), desc.fields_ptr(), desc.id()}; } TimeseriesDescriptor timeseries_descriptor_from_pipeline_context( @@ -64,8 +51,7 @@ TimeseriesDescriptor timeseries_descriptor_from_pipeline_context( bool bucketize_dynamic) { return make_timeseries_descriptor( pipeline_context->total_rows_, - StreamDescriptor{std::make_shared(std::move(pipeline_context->desc_->mutable_proto())), - pipeline_context->desc_->fields_ptr()}, + pipeline_context->descriptor(), std::move(*pipeline_context->norm_meta_), pipeline_context->user_meta_ ? std::make_optional(std::move(*pipeline_context->user_meta_)) : std::nullopt, std::move(prev_key), @@ -81,8 +67,7 @@ TimeseriesDescriptor index_descriptor_from_frame( ) { return make_timeseries_descriptor( frame->num_rows + existing_rows, - StreamDescriptor{std::make_shared(std::move(frame->desc.mutable_proto())), - frame->desc.fields_ptr()}, + frame->desc, std::move(frame->norm_meta), std::move(frame->user_meta), std::move(prev_key), @@ -149,7 +134,7 @@ std::pair offset_and_row_count(const std::shared_ptr& frame) { - return !std::holds_alternative(frame->index) || frame->desc.get_sorted() == SortedValue::ASCENDING; + return !std::holds_alternative(frame->index) || frame->desc.sorted() == SortedValue::ASCENDING; } } diff --git a/cpp/arcticdb/pipeline/frame_utils.hpp b/cpp/arcticdb/pipeline/frame_utils.hpp index 58e8f8c0ef..4f50c90128 100644 --- a/cpp/arcticdb/pipeline/frame_utils.hpp +++ b/cpp/arcticdb/pipeline/frame_utils.hpp @@ -50,7 +50,7 @@ inline size_t get_max_string_size(const pipelines::PipelineContextRow& context_r TimeseriesDescriptor make_timeseries_descriptor( size_t total_rows, - StreamDescriptor&& desc, + const StreamDescriptor& desc, arcticdb::proto::descriptors::NormalizationMetadata&& norm_meta, std::optional&& um, std::optional&& prev_key, diff --git a/cpp/arcticdb/pipeline/index_segment_reader.cpp b/cpp/arcticdb/pipeline/index_segment_reader.cpp index b55b617144..8781053ed3 100644 --- a/cpp/arcticdb/pipeline/index_segment_reader.cpp +++ b/cpp/arcticdb/pipeline/index_segment_reader.cpp @@ -24,20 +24,8 @@ IndexSegmentReader get_index_reader(const AtomKey &prev_index, const std::shared return index::IndexSegmentReader{std::move(seg)}; } -IndexSegmentReader::IndexSegmentReader(SegmentInMemory&& s) : seg_(std::move(s)) { - seg_.metadata()->UnpackTo(&tsd_.mutable_proto()); - if(seg_.has_index_fields()) { - tsd_.mutable_fields() = seg_.detach_index_fields(); - tsd_.mutable_fields().regenerate_offsets(); - } else { - TimeseriesDescriptor::Proto tsd; - if(seg_.metadata()->UnpackTo(&tsd)) { - tsd_.mutable_fields() = fields_from_proto(tsd.stream_descriptor()); - } else { - util::raise_rte("Unable to unpack index fields"); - } - } - ARCTICDB_DEBUG(log::version(), "Decoded index segment descriptor: {}", tsd_.proto().DebugString()); +IndexSegmentReader::IndexSegmentReader(SegmentInMemory&& s) : + seg_(std::move(s)) { } const Column &IndexSegmentReader::column(Fields field) const { @@ -59,7 +47,7 @@ IndexRange get_index_segment_range( } bool IndexSegmentReader::bucketize_dynamic() const { - return tsd().proto().has_column_groups() && tsd().proto().column_groups().enabled(); + return tsd().column_groups(); } SliceAndKey IndexSegmentReader::row(std::size_t r) const { @@ -106,11 +94,11 @@ IndexSegmentIterator IndexSegmentReader::last() const { } bool IndexSegmentReader::is_pickled() const { - return tsd_.proto().normalization().input_type_case() == arcticdb::proto::descriptors::NormalizationMetadata::InputTypeCase::kMsgPackFrame; + return tsd().proto().normalization().input_type_case() == arcticdb::proto::descriptors::NormalizationMetadata::InputTypeCase::kMsgPackFrame; } bool IndexSegmentReader::has_timestamp_index() const { - return tsd_.proto().stream_descriptor().index().kind() == arcticdb::proto::descriptors::IndexDescriptor::Type::IndexDescriptor_Type_TIMESTAMP; + return tsd().index().type_ == IndexDescriptor::Type::TIMESTAMP; } void check_column_and_date_range_filterable(const pipelines::index::IndexSegmentReader& index_segment_reader, const ReadQuery& read_query) { @@ -119,8 +107,8 @@ void check_column_and_date_range_filterable(const pipelines::index::IndexSegment "The data for this symbol is pickled and does not support column stats, date_range, row_range, or column queries"); util::check(index_segment_reader.has_timestamp_index() || !std::holds_alternative(read_query.row_filter), "Cannot apply date range filter to symbol with non-timestamp index"); - sorting::check(index_segment_reader.get_sorted() == SortedValue::UNKNOWN || - index_segment_reader.get_sorted() == SortedValue::ASCENDING || + sorting::check(index_segment_reader.sorted() == SortedValue::UNKNOWN || + index_segment_reader.sorted() == SortedValue::ASCENDING || !std::holds_alternative(read_query.row_filter), "When filtering data using date_range, the symbol must be sorted in ascending order. ArcticDB believes it is not sorted in ascending order and cannot therefore filter the data using date_range."); } diff --git a/cpp/arcticdb/pipeline/index_segment_reader.hpp b/cpp/arcticdb/pipeline/index_segment_reader.hpp index 63496e80fd..14ad963df2 100644 --- a/cpp/arcticdb/pipeline/index_segment_reader.hpp +++ b/cpp/arcticdb/pipeline/index_segment_reader.hpp @@ -37,7 +37,6 @@ struct IndexSegmentReader { using std::swap; swap(left.seg_, right.seg_); - swap(left.tsd_, right.tsd_); } ARCTICDB_MOVE_ONLY_DEFAULT(IndexSegmentReader) @@ -64,26 +63,25 @@ struct IndexSegmentReader { bool bucketize_dynamic() const; - SortedValue get_sorted() const { - return sorted_value_from_proto(tsd().proto().stream_descriptor().sorted()); + SortedValue sorted() const { + return tsd().sorted(); } void set_sorted(SortedValue sorted) { - mutable_tsd().mutable_proto().mutable_stream_descriptor()->set_sorted(sorted_value_to_proto(sorted)); + mutable_tsd().set_sorted(sorted); } const TimeseriesDescriptor& tsd() const { - return tsd_; + return seg_.index_descriptor(); } TimeseriesDescriptor& mutable_tsd() { - return tsd_; + return seg_.mutable_index_descriptor(); } private: mutable std::unordered_map, AxisRange::Hasher> descriptor_by_col_group_; SegmentInMemory seg_; - TimeseriesDescriptor tsd_; }; struct IndexSegmentIterator { diff --git a/cpp/arcticdb/pipeline/index_utils.cpp b/cpp/arcticdb/pipeline/index_utils.cpp index 6ea4656559..6248198824 100644 --- a/cpp/arcticdb/pipeline/index_utils.cpp +++ b/cpp/arcticdb/pipeline/index_utils.cpp @@ -16,7 +16,7 @@ namespace arcticdb::pipelines::index { template folly::Future write_index( - TimeseriesDescriptor &&metadata, + const TimeseriesDescriptor& metadata, std::vector &&sk, const IndexPartialKey &partial_key, const std::shared_ptr &sink @@ -31,14 +31,14 @@ folly::Future write_index( folly::Future write_index( const stream::Index& index, - TimeseriesDescriptor &&metadata, + const TimeseriesDescriptor& metadata, std::vector &&sk, const IndexPartialKey &partial_key, const std::shared_ptr &sink ) { return util::variant_match(index, [&] (auto idx) { using IndexType = decltype(idx); - return write_index(std::move(metadata), std::move(sk), partial_key, sink); + return write_index(metadata, std::move(sk), partial_key, sink); }); } @@ -51,7 +51,7 @@ folly::Future write_index( auto offset = frame->offset; auto index = stream::index_type_from_descriptor(frame->desc); auto timeseries_desc = index_descriptor_from_frame(frame, offset); - return write_index(index, std::move(timeseries_desc), std::move(slice_and_keys), partial_key, sink); + return write_index(index, timeseries_desc, std::move(slice_and_keys), partial_key, sink); } folly::Future write_index( @@ -86,7 +86,7 @@ TimeseriesDescriptor get_merged_tsd( const std::shared_ptr& new_frame) { auto existing_descriptor = existing_tsd.as_stream_descriptor(); auto merged_descriptor = existing_descriptor; - if (existing_tsd.proto().total_rows() == 0){ + if (existing_tsd.total_rows() == 0){ // If the existing dataframe is empty, we use the descriptor of the new_frame merged_descriptor = new_frame->desc; } @@ -116,7 +116,7 @@ TimeseriesDescriptor get_merged_tsd( } } } - merged_descriptor.set_sorted(deduce_sorted(existing_descriptor.get_sorted(), new_frame->desc.get_sorted())); + merged_descriptor.set_sorted(deduce_sorted(existing_descriptor.sorted(), new_frame->desc.sorted())); return make_timeseries_descriptor( row_count, std::move(merged_descriptor), diff --git a/cpp/arcticdb/pipeline/index_utils.hpp b/cpp/arcticdb/pipeline/index_utils.hpp index 97dba5de9f..bf36d507ba 100644 --- a/cpp/arcticdb/pipeline/index_utils.hpp +++ b/cpp/arcticdb/pipeline/index_utils.hpp @@ -35,14 +35,14 @@ inline std::vector unfiltered_index(const index::IndexSegmentReader } template -std::optional index_value_from_row(const RowType &row, IndexDescriptor::Type index_type, int field_num) { +std::optional index_value_from_row(const RowType &row, IndexDescriptorImpl::Type index_type, int field_num) { std::optional index_value; switch (index_type) { - case IndexDescriptor::TIMESTAMP: - case IndexDescriptor::ROWCOUNT: + case IndexDescriptorImpl::Type::TIMESTAMP: + case IndexDescriptorImpl::Type::ROWCOUNT: index_value = row.template scalar_at(field_num); break; - case IndexDescriptor::STRING: { + case IndexDescriptorImpl::Type::STRING: { auto opt = row.string_at(field_num); index_value = opt ? std::make_optional(std::string(opt.value())) : std::nullopt; break; @@ -54,25 +54,26 @@ std::optional index_value_from_row(const RowType &row, IndexDescript } template -std::optional index_start_from_row(const RowType &row, IndexDescriptor::Type index_type) { +std::optional index_start_from_row(const RowType &row, IndexDescriptorImpl::Type index_type) { return index_value_from_row(row, index_type, 0); } template - IndexValue index_value_from_segment(const SegmentType &seg, size_t row_id, FieldType field) { +IndexValue index_value_from_segment(const SegmentType &seg, size_t row_id, FieldType field) { auto index_type = seg.template scalar_at(row_id, int(FieldType::index_type)); IndexValue index_value; - switch (index_type.value()) { - case IndexDescriptor::TIMESTAMP: - case IndexDescriptor::ROWCOUNT: - index_value = seg.template scalar_at(row_id, int(field)).value(); - break; - case IndexDescriptor::STRING: - index_value = std::string(seg.string_at(row_id, int(field)).value()); - break; - default: - util::raise_rte("Unknown index type {} for column {} and row {}", - uint32_t(index_type.value()), uint32_t(field), row_id); + auto type = IndexDescriptor::Type(index_type.value()); + switch (type) { + case IndexDescriptorImpl::Type::TIMESTAMP: + case IndexDescriptorImpl::Type::ROWCOUNT: + index_value = seg.template scalar_at(row_id, int(field)).value(); + break; + case IndexDescriptorImpl::Type::STRING: + index_value = std::string(seg.string_at(row_id, int(field)).value()); + break; + default: + util::raise_rte("Unknown index type {} for column {} and row {}", + uint32_t(index_type.value()), uint32_t(field), row_id); } return index_value; } @@ -89,14 +90,14 @@ IndexValue index_end_from_segment(const SegmentType &seg, size_t row_id) { template folly::Future write_index( - TimeseriesDescriptor&& metadata, + const TimeseriesDescriptor& metadata, std::vector&& slice_and_keys, const IndexPartialKey& partial_key, const std::shared_ptr& sink); folly::Future write_index( const stream::Index& index, - TimeseriesDescriptor &&metadata, + const TimeseriesDescriptor& metadata, std::vector &&sk, const IndexPartialKey &partial_key, const std::shared_ptr &sink); diff --git a/cpp/arcticdb/pipeline/index_writer.hpp b/cpp/arcticdb/pipeline/index_writer.hpp index 4ee9104ae3..fdc1585bb0 100644 --- a/cpp/arcticdb/pipeline/index_writer.hpp +++ b/cpp/arcticdb/pipeline/index_writer.hpp @@ -28,10 +28,9 @@ class IndexWriter { public: ARCTICDB_MOVE_ONLY_DEFAULT(IndexWriter) - IndexWriter(std::shared_ptr sink, IndexPartialKey partial_key, TimeseriesDescriptor &&meta, const std::optional& key_type = std::nullopt) : - bucketize_columns_(meta.proto().has_column_groups() && meta.proto().column_groups().enabled()), + IndexWriter(std::shared_ptr sink, IndexPartialKey partial_key, const TimeseriesDescriptor &tsd, const std::optional& key_type = std::nullopt) : + bucketize_columns_(tsd.column_groups()), partial_key_(std::move(partial_key)), - meta_(std::move(meta)), agg_(Desc::schema(partial_key_.id, bucketize_columns_), [&](auto &&segment) { on_segment(std::forward(segment)); @@ -40,11 +39,7 @@ class IndexWriter { sink_(std::move(sink)), key_being_committed_(folly::Future::makeEmpty()), key_type_(key_type) { - static const auto encoding = ConfigsMap::instance()->get_int("VersionStore.Encoding", 1); - if(encoding == 1) { - meta_.copy_to_self_proto(); - } - agg_.segment().set_timeseries_descriptor(std::move(meta_)); //TODO very weird, why this short-lived member? + agg_.segment().set_timeseries_descriptor(tsd); } void add(const arcticdb::entity::AtomKey &key, const FrameSlice &slice) { @@ -127,7 +122,6 @@ class IndexWriter { bool bucketize_columns_ = false; IndexPartialKey partial_key_; - TimeseriesDescriptor meta_; SliceAggregator agg_; std::shared_ptr sink_; folly::Future key_being_committed_; diff --git a/cpp/arcticdb/pipeline/pipeline_context.cpp b/cpp/arcticdb/pipeline/pipeline_context.cpp index 0b47842613..3defa60d49 100644 --- a/cpp/arcticdb/pipeline/pipeline_context.cpp +++ b/cpp/arcticdb/pipeline/pipeline_context.cpp @@ -28,16 +28,12 @@ PipelineContext::PipelineContext(SegmentInMemory& frame, const AtomKey& key) : map->set_from_descriptor(frame.descriptor()); auto descriptor = std::make_shared(frame.descriptor()); - segment_descriptors_[0] = (std::move(descriptor)); + segment_descriptors_[0] = std::move(descriptor); } void PipelineContext::set_selected_columns(const std::vector& columns) { util::check(static_cast(desc_), "Descriptor not set in set_selected_columns"); - selected_columns_ = requested_column_bitset_including_index(desc_->proto(), columns); -} - -bool PipelineContextRow::selected_columns(size_t n) const { - return !parent_->selected_columns_ || parent_->selected_columns_.value()[n]; + selected_columns_ = requested_column_bitset_including_index(*desc_, columns); } const std::optional& PipelineContextRow::get_selected_columns() const { @@ -90,6 +86,10 @@ void PipelineContextRow::set_descriptor(std::shared_ptr&& desc parent_->segment_descriptors_[index_] = std::move(desc); } +void PipelineContextRow::set_descriptor(const StreamDescriptor& desc) { + parent_->segment_descriptors_[index_] = std::make_shared(desc); +} + void PipelineContextRow::set_descriptor(const std::shared_ptr& desc) { parent_->segment_descriptors_[index_] = desc; } diff --git a/cpp/arcticdb/pipeline/pipeline_context.hpp b/cpp/arcticdb/pipeline/pipeline_context.hpp index e15f18dd76..f8b8983bdb 100644 --- a/cpp/arcticdb/pipeline/pipeline_context.hpp +++ b/cpp/arcticdb/pipeline/pipeline_context.hpp @@ -39,10 +39,10 @@ struct PipelineContextRow { [[nodiscard]] const SliceAndKey& slice_and_key() const; SliceAndKey& slice_and_key(); [[nodiscard]] const std::optional& get_selected_columns() const; - bool selected_columns(size_t n) const; bool fetch_index(); [[nodiscard]] const StreamDescriptor& descriptor() const; void set_descriptor(StreamDescriptor&& desc); + void set_descriptor(const StreamDescriptor& desc); void set_descriptor(std::shared_ptr&& desc); void set_descriptor(const std::shared_ptr& desc); void set_compacted(bool val); diff --git a/cpp/arcticdb/pipeline/query.hpp b/cpp/arcticdb/pipeline/query.hpp index 5d20744500..39b0194f10 100644 --- a/cpp/arcticdb/pipeline/query.hpp +++ b/cpp/arcticdb/pipeline/query.hpp @@ -272,20 +272,20 @@ template inline FilterQuery create_index_filter(const IndexRange &range, bool dynamic_schema, bool column_groups) { static_assert(std::is_same_v); return [rg = range, dynamic_schema, column_groups](const ContainerType &container, std::unique_ptr&& input) mutable { - auto index_type = container.seg().template scalar_at(0u, int(index::Fields::index_type)); - - switch (index_type.value()) { - case IndexDescriptor::TIMESTAMP: { + auto maybe_index_type = container.seg().template scalar_at(0u, int(index::Fields::index_type)); + const auto index_type = IndexDescriptor::Type(maybe_index_type.value()); + switch (index_type) { + case IndexDescriptorImpl::Type::TIMESTAMP: { return build_bitset_for_index(container, rg, dynamic_schema, column_groups, std::move(input)); } - case IndexDescriptor::STRING: { + case IndexDescriptorImpl::Type::STRING: { return build_bitset_for_index(container, rg, dynamic_schema, column_groups, std::move(input)); } - default:util::raise_rte("Unknown index type {} in create_index_filter", uint32_t(index_type.value())); + default:util::raise_rte("Unknown index type {} in create_index_filter", uint32_t(index_type)); } }; } @@ -342,7 +342,7 @@ inline std::vector> build_update_query_filters( // be appended to, the type of the frame being appended, and the specified range, if supplied. std::vector> queries; util::variant_match(range, - [&](const RowRange &row_range) { + [&](const RowRange &row_range) { util::check(std::holds_alternative(index), "Cannot partition by row count when a timeseries-indexed frame was supplied"); queries.emplace_back( create_row_filter(RowRange{row_range.first, row_range.second})); diff --git a/cpp/arcticdb/pipeline/read_frame.cpp b/cpp/arcticdb/pipeline/read_frame.cpp index 7fc44cd37c..946d97e091 100644 --- a/cpp/arcticdb/pipeline/read_frame.cpp +++ b/cpp/arcticdb/pipeline/read_frame.cpp @@ -23,9 +23,10 @@ #include #include #include -#include -#include #include +#include + +#include #include #include #include @@ -58,7 +59,7 @@ StreamDescriptor get_filtered_descriptor(StreamDescriptor&& descriptor, const st auto index = stream::index_type_from_descriptor(desc); return util::variant_match(index, [&desc, &filter_columns] (const auto& idx) { const std::shared_ptr& fields = filter_columns ? filter_columns : desc.fields_ptr(); - return StreamDescriptor{index_descriptor(desc.id(), idx, *fields)}; + return StreamDescriptor{index_descriptor_from_range(desc.id(), idx, *fields)}; }); } @@ -94,8 +95,8 @@ size_t get_index_field_count(const SegmentInMemory& frame) { return frame.descriptor().index().field_count(); } -const uint8_t* skip_heading_fields(const arcticdb::proto::encoding::SegmentHeader & hdr, const uint8_t*& data) { - const auto has_magic_numbers = EncodingVersion(hdr.encoding_version()) == EncodingVersion::V2; +const uint8_t* skip_heading_fields(const SegmentHeader & hdr, const uint8_t*& data) { + const auto has_magic_numbers = hdr.encoding_version() == EncodingVersion::V2; if(has_magic_numbers) util::check_magic(data); @@ -105,8 +106,12 @@ const uint8_t* skip_heading_fields(const arcticdb::proto::encoding::SegmentHeade data += metadata_size; } - if(has_magic_numbers) - util::check_magic(data); + if(has_magic_numbers) { + util::check_magic(data); + data += sizeof(SegmentDescriptor); + skip_identifier(data); + util::check_magic(data); + } if(hdr.has_descriptor_field()) { auto descriptor_field_size = encoding_sizes::ndarray_field_compressed_size(hdr.descriptor_field().ndarray()); @@ -125,7 +130,7 @@ const uint8_t* skip_heading_fields(const arcticdb::proto::encoding::SegmentHeade return data; } -void decode_string_pool(const arcticdb::proto::encoding::SegmentHeader & hdr, const uint8_t*& data, const uint8_t *begin ARCTICDB_UNUSED, const uint8_t* end, PipelineContextRow &context) { +void decode_string_pool(const SegmentHeader& hdr, const uint8_t*& data, const uint8_t *begin ARCTICDB_UNUSED, const uint8_t* end, PipelineContextRow &context) { if (hdr.has_string_pool_field()) { ARCTICDB_DEBUG(log::codec(), "Decoding string pool at position: {}", data - begin); util::check(data != end, "Reached end of input block with string pool fields to decode"); @@ -136,21 +141,21 @@ void decode_string_pool(const arcticdb::proto::encoding::SegmentHeader & hdr, co if(EncodingVersion(hdr.encoding_version()) == EncodingVersion::V2) util::check_magic(data); - data += decode_field(string_pool_descriptor().type(), - hdr.string_pool_field(), + util::check(hdr.string_pool_field().has_ndarray(), "Expected string pool field to be ndarray"); + data += decode_ndarray(string_pool_descriptor().type(), + hdr.string_pool_field().ndarray(), data, context.string_pool(), bv, - to_encoding_version(hdr.encoding_version())); + hdr.encoding_version()); ARCTICDB_TRACE(log::codec(), "Decoded string pool to position {}", data - begin); } } -template -void decode_index_field_impl( +void decode_index_field( SegmentInMemory &frame, - const EncodedFieldType& field, + const EncodedFieldImpl& field, const uint8_t*& data, const uint8_t *begin ARCTICDB_UNUSED, const uint8_t* end ARCTICDB_UNUSED, @@ -188,40 +193,16 @@ void decode_index_field_impl( } } -void decode_index_field( - SegmentInMemory& frame, - VariantField variant_field, - const uint8_t*& data, - const uint8_t* begin ARCTICDB_UNUSED, - const uint8_t* end ARCTICDB_UNUSED, - PipelineContextRow& context, - EncodingVersion encoding_version -) { - util::variant_match(variant_field, [&](auto field) { - decode_index_field_impl(frame, *field, data, begin, end, context, encoding_version); - }); -} - -template -void decode_or_expand_impl( +void decode_or_expand( const uint8_t*& data, uint8_t* dest, - const EncodedFieldType& encoded_field_info, + const EncodedFieldImpl& encoded_field_info, size_t dest_bytes, std::shared_ptr buffers, - EncodingVersion encding_version, - const ColumnMapping& m -) { - if (auto handler = TypeHandlerRegistry::instance()->get_handler(m.source_type_desc_); handler) { - handler->handle_type( - data, - dest, - VariantField{&encoded_field_info}, - dest_bytes, - std::move(buffers), - encding_version, - m - ); + EncodingVersion encoding_version, + const ColumnMapping& m) { + if(auto handler = TypeHandlerRegistry::instance()->get_handler(m.source_type_desc_); handler) { + handler->handle_type(data, dest, encoded_field_info, m.source_type_desc_, dest_bytes, std::move(buffers), encoding_version, m); } else { std::optional bv; if (encoded_field_info.has_ndarray() && encoded_field_info.ndarray().sparse_map_bytes() > 0) { @@ -229,7 +210,7 @@ void decode_or_expand_impl( const auto bytes = encoding_sizes::data_uncompressed_size(ndarray); ChunkedBuffer sparse{bytes}; SliceDataSink sparse_sink{sparse.data(), bytes}; - data += decode_field(m.source_type_desc_, encoded_field_info, data, sparse_sink, bv, encding_version); + data += decode_field(m.source_type_desc_, encoded_field_info, data, sparse_sink, bv, encoding_version); m.source_type_desc_.visit_tag([dest, dest_bytes, &bv, &sparse](const auto tdt) { using TagType = decltype(tdt); using RawType = typename TagType::DataTypeTag::raw_type; @@ -245,58 +226,35 @@ void decode_or_expand_impl( util::default_initialize(dest + bytes, dest_bytes - bytes); }); } - data += decode_field(m.source_type_desc_, encoded_field_info, data, sink, bv, encding_version); + data += decode_field(m.source_type_desc_, encoded_field_info, data, sink, bv, encoding_version); } } } -size_t get_field_range_compressed_size(size_t start_idx, size_t num_fields, - const arcticdb::proto::encoding::SegmentHeader& hdr, - const VariantEncodedFieldCollection& fields) { +size_t get_field_range_compressed_size( + size_t start_idx, + size_t num_fields, + const SegmentHeader& hdr, + const EncodedFieldCollection& fields) { size_t total = 0ULL; const size_t magic_num_size = EncodingVersion(hdr.encoding_version()) == EncodingVersion::V2 ? sizeof(ColumnMagic) : 0u; ARCTICDB_DEBUG(log::version(), "Skipping between {} and {}", start_idx, start_idx + num_fields); for(auto i = start_idx; i < start_idx + num_fields; ++i) { - util::variant_match(fields.at(i), [&total, magic_num_size] (const auto& field) { - ARCTICDB_DEBUG(log::version(), "Adding {}", encoding_sizes::ndarray_field_compressed_size(field->ndarray()) + magic_num_size); - total += encoding_sizes::ndarray_field_compressed_size(field->ndarray()) + magic_num_size; - }); + const auto& field = fields.at(i); + ARCTICDB_DEBUG(log::version(), "Adding {}", encoding_sizes::ndarray_field_compressed_size(field.ndarray()) + magic_num_size); + total += encoding_sizes::ndarray_field_compressed_size(field.ndarray()) + magic_num_size; } ARCTICDB_DEBUG(log::version(), "Fields {} to {} contain {} bytes", start_idx, start_idx + num_fields, total); return total; } -void decode_or_expand( - const uint8_t*& data, - uint8_t* dest, - const VariantField& variant_field, - size_t dest_bytes, - std::shared_ptr buffers, - EncodingVersion encoding_version, - const ColumnMapping& m -) { - util::variant_match(variant_field, [&](auto field) { - decode_or_expand_impl( - data, - dest, - *field, - dest_bytes, - buffers, - encoding_version, - m - ); - }); -} - void advance_field_size( - const VariantField& variant_field, + const EncodedFieldImpl& field, const uint8_t*& data, bool has_magic_numbers ) { - util::variant_match(variant_field, [&data, has_magic_numbers] (auto field) { const size_t magic_num_size = has_magic_numbers ? sizeof(ColumnMagic) : 0ULL; - data += encoding_sizes::ndarray_field_compressed_size(field->ndarray()) + magic_num_size; - }); + data += encoding_sizes::ndarray_field_compressed_size(field.ndarray()) + magic_num_size; } void advance_skipped_cols( @@ -305,8 +263,8 @@ void advance_skipped_cols( size_t source_col, size_t first_col_offset, size_t index_fieldcount, - const VariantEncodedFieldCollection& fields, - const arcticdb::proto::encoding::SegmentHeader& hdr) { + const EncodedFieldCollection& fields, + const SegmentHeader& hdr) { const auto next_col = prev_col_offset + 1; auto skipped_cols = source_col - next_col; if(skipped_cols) { @@ -341,17 +299,17 @@ void decode_into_frame_static( auto &hdr = seg.header(); auto index_fieldcount = get_index_field_count(frame); data = skip_heading_fields(hdr, data); - context.set_descriptor(StreamDescriptor{ std::make_shared(std::move(*hdr.mutable_stream_descriptor())), seg.fields_ptr() }); + context.set_descriptor(seg.descriptor()); context.set_compacted(hdr.compacted()); - ARCTICDB_DEBUG(log::version(), "Num fields: {}", seg.header().fields_size()); - const EncodingVersion encoding_version = EncodingVersion(hdr.encoding_version()); + ARCTICDB_DEBUG(log::version(), "Num fields: {}", seg.descriptor().field_count()); + const auto encoding_version = hdr.encoding_version(); const bool has_magic_nums = encoding_version == EncodingVersion::V2; - VariantEncodedFieldCollection fields(seg); + const auto& fields = hdr.body_fields(); // data == end in case we have empty data types (e.g. {EMPTYVAL, Dim0}, {EMPTYVAL, Dim1}) for which we store nothing // in storage as they can be reconstructed in the type handler on the read path. - if (data != end || fields.size() > 0) { - auto index_field = fields.at(0u); + if (data != end || !fields.empty()) { + auto& index_field = fields.at(0u); decode_index_field(frame, index_field, data, begin, end, context, encoding_version); StaticColumnMappingIterator it(context, index_fieldcount); @@ -363,7 +321,7 @@ void decode_into_frame_static( if(has_magic_nums) util::check_magic_in_place(data); - auto encoded_field = fields.at(it.source_field_pos()); + auto& encoded_field = fields.at(it.source_field_pos()); util::check(it.source_field_pos() < size_t(fields.size()), "Field index out of range: {} !< {}", it.source_field_pos(), fields.size()); auto field_name = context.descriptor().fields(it.source_field_pos()).name(); auto& buffer = frame.column(static_cast(it.dest_col())).data().buffer(); @@ -432,22 +390,21 @@ void decode_into_frame_dynamic( auto &hdr = seg.header(); auto index_fieldcount = get_index_field_count(frame); data = skip_heading_fields(hdr, data); - context.set_descriptor(StreamDescriptor{std::make_shared(std::move(*hdr.mutable_stream_descriptor())), seg.fields_ptr()}); + context.set_descriptor(std::make_shared(seg.descriptor())); context.set_compacted(hdr.compacted()); - const EncodingVersion encdoing_version = EncodingVersion(hdr.encoding_version()); - const bool has_magic_numbers = encdoing_version == EncodingVersion::V2; - VariantEncodedFieldCollection fields(seg); - // data == end in case we have empty data types (e.g. {EMPTYVAL, Dim0}, {EMPTYVAL, Dim1}) for which we store nothing - // in storage as they can be reconstructed in the type handler on the read path. - if (data != end || fields.size() > 0) { - auto index_field = fields.at(0u); - decode_index_field(frame, index_field, data, begin, end, context, encdoing_version); + const auto encoding_version = hdr.encoding_version(); + const bool has_magic_numbers = encoding_version == EncodingVersion::V2; + + if (data != end) { + const auto& fields = hdr.body_fields(); + auto& index_field = fields.at(0u); + decode_index_field(frame, index_field, data, begin, end, context, encoding_version); auto field_count = context.slice_and_key().slice_.col_range.diff() + index_fieldcount; for (auto field_col = index_fieldcount; field_col < field_count; ++field_col) { auto field_name = context.descriptor().fields(field_col).name(); - auto encoded_field = fields.at(field_col); + auto& encoded_field = fields.at(field_col); auto frame_loc_opt = frame.column_index(field_name); if (!frame_loc_opt) { // Column is not selected in the output frame. @@ -458,6 +415,7 @@ void decode_into_frame_dynamic( auto dst_col = *frame_loc_opt; auto& buffer = frame.column(static_cast(dst_col)).data().buffer(); ColumnMapping m{frame, dst_col, field_col, context}; + util::check( static_cast(has_valid_type_promotion(m.source_type_desc_, m.dest_type_desc_)), "Can't promote type {} to type {} in field {}", @@ -484,15 +442,14 @@ void decode_into_frame_dynamic( encoded_field, m.dest_bytes_, buffers, - encdoing_version, + encoding_version, m ); - // decode_or_expand will invoke the empty type handler which will do backfilling with the default value depending on the - // destination type. + if (!trivially_compatible_types(m.source_type_desc_, m.dest_type_desc_) && !source_is_empty) { - m.dest_type_desc_.visit_tag([&buffer, &m, &data, encoded_field, buffers, encdoing_version] (auto dest_desc_tag) { + m.dest_type_desc_.visit_tag([&buffer, &m, buffers] (auto dest_desc_tag) { using DestinationType = typename decltype(dest_desc_tag)::DataTypeTag::raw_type; - m.source_type_desc_.visit_tag([&buffer, &m, &data, &encoded_field, &buffers, encdoing_version] (auto src_desc_tag ) { + m.source_type_desc_.visit_tag([&buffer, &m] (auto src_desc_tag ) { using SourceType = typename decltype(src_desc_tag)::DataTypeTag::raw_type; if constexpr(std::is_arithmetic_v && std::is_arithmetic_v) { // If the source and destination types are different, then sizeof(destination type) >= sizeof(source type) @@ -649,7 +606,6 @@ class EmptyDynamicStringReducer { protected: Column& column_; SegmentInMemory frame_; - const Field& frame_field_; size_t row_ ; ChunkedBuffer& src_buffer_; size_t column_width_; @@ -663,12 +619,11 @@ class EmptyDynamicStringReducer { EmptyDynamicStringReducer( Column& column, SegmentInMemory frame, - const Field& frame_field, + const Field&, size_t alloc_width, std::shared_ptr spinlock) : column_(column), frame_(std::move(frame)), - frame_field_(frame_field), row_(0), src_buffer_(column.data().buffer()), column_width_(alloc_width), diff --git a/cpp/arcticdb/pipeline/read_pipeline.hpp b/cpp/arcticdb/pipeline/read_pipeline.hpp index b0dc0349d4..705b87e40a 100644 --- a/cpp/arcticdb/pipeline/read_pipeline.hpp +++ b/cpp/arcticdb/pipeline/read_pipeline.hpp @@ -84,7 +84,7 @@ std::vector filter_index(const ContainerType &container, std::optio return output; } -inline util::BitSet build_column_bitset(const StreamDescriptor::Proto &desc, const folly::F14FastSet& columns) { +inline util::BitSet build_column_bitset(const StreamDescriptor &desc, const folly::F14FastSet& columns) { util::BitSet col_bitset(static_cast(desc.fields().size())); for (std::size_t c = 0; c < static_cast(desc.fields().size()); ++c) { auto& f = desc.fields(static_cast(c)); @@ -95,24 +95,24 @@ inline util::BitSet build_column_bitset(const StreamDescriptor::Proto &desc, con return col_bitset; } -inline util::BitSet build_column_bitset(const StreamDescriptor::Proto&desc, const std::vector& columns) { +inline util::BitSet build_column_bitset(const StreamDescriptor &desc, const std::vector& columns) { folly::F14FastSet col_set{columns.begin(), columns.end()}; return build_column_bitset(desc, col_set); } -inline bool contains_index_column(const std::vector& columns, const StreamDescriptor::Proto& desc) { +inline auto add_index_column(const std::vector& columns, const StreamDescriptor& desc) { + std::vector columns_with_index{columns}; + columns_with_index.push_back(std::string{desc.fields(0).name()}); + return columns_with_index; +} + +inline bool contains_index_column(const std::vector& columns, const StreamDescriptor& desc) { return desc.index().field_count() == 0 || std::find(std::begin(columns), std::end(columns), desc.fields(0).name()) != std::end(columns); } -inline auto add_index_column(const std::vector& columns, const StreamDescriptor::Proto& desc) { - std::vector columns_with_index{columns}; - columns_with_index.push_back(desc.fields(0).name()); - return columns_with_index; -} - -inline std::optional requested_column_bitset_including_index(const StreamDescriptor::Proto& desc, const std::vector& columns) { +inline std::optional requested_column_bitset_including_index(const StreamDescriptor& desc, const std::vector& columns) { // Add the index column if it's not there if (!columns.empty()) { if(!contains_index_column(columns, desc)) { @@ -125,7 +125,7 @@ inline std::optional requested_column_bitset_including_index(const return std::nullopt; } -inline std::optional clause_column_bitset(const StreamDescriptor::Proto& desc, +inline std::optional clause_column_bitset(const StreamDescriptor& desc, const std::vector>& clauses) { folly::F14FastSet column_set; for (const auto& clause: clauses) { @@ -145,7 +145,7 @@ inline std::optional clause_column_bitset(const StreamDescriptor:: // Returns std::nullopt if all columns are required, which is the case if requested_columns is std::nullopt // Otherwise augment the requested_columns bitset with columns that are required by any of the clauses -inline std::optional overall_column_bitset(const StreamDescriptor::Proto& desc, +inline std::optional overall_column_bitset(const StreamDescriptor& desc, const std::vector>& clauses, const std::optional& requested_columns) { // std::all_of returns true if the range is empty @@ -202,7 +202,7 @@ inline std::vector> get_column_bitset_and_query_funct if(!dynamic_schema || column_groups) { pipeline_context->set_selected_columns(query.columns); - pipeline_context->overall_column_bitset_ = overall_column_bitset(pipeline_context->descriptor().proto(), + pipeline_context->overall_column_bitset_ = overall_column_bitset(pipeline_context->descriptor(), query.clauses_, pipeline_context->selected_columns_); } diff --git a/cpp/arcticdb/pipeline/write_frame.cpp b/cpp/arcticdb/pipeline/write_frame.cpp index cf7af8031b..689c5066ea 100644 --- a/cpp/arcticdb/pipeline/write_frame.cpp +++ b/cpp/arcticdb/pipeline/write_frame.cpp @@ -183,6 +183,9 @@ folly::Future> slice_and_write( bool sparsify_floats) { ARCTICDB_SUBSAMPLE_DEFAULT(SliceFrame) auto slices = slice(*frame, slicing); + if(slices.empty()) + return folly::makeFuture(std::vector{}); + ARCTICDB_SUBSAMPLE_DEFAULT(SliceAndWrite) return write_slices(frame, std::move(slices), slicing, std::move(key), sink, de_dup_map, sparsify_floats); } @@ -200,6 +203,9 @@ write_frame( // Write the keys of the slices into an index segment ARCTICDB_SUBSAMPLE_DEFAULT(WriteIndex) return std::move(fut_slice_keys).thenValue([frame=frame, key = std::move(key), &store](auto&& slice_keys) mutable { + // if(slice_keys.empty()) + // return folly::makeFuture(AtomKey{}); + return index::write_index(frame, std::forward(slice_keys), key, store); }); } @@ -221,7 +227,7 @@ folly::Future append_frame( auto& frame_index = frame->index_tensor.value(); util::check(frame_index.data_type() == DataType::NANOSECONDS_UTC64, "Expected timestamp index in append, got type {}", frame_index.data_type()); - if (index_segment_reader.tsd().proto().total_rows() != 0 && frame_index.size() != 0) { + if (index_segment_reader.tsd().total_rows() != 0 && frame_index.size() != 0) { auto first_index = NumericIndex{*frame_index.ptr_cast(0)}; auto prev = std::get(index_segment_reader.last()->key().end_index()); util::check(ignore_sort_order || prev - 1 <= first_index, diff --git a/cpp/arcticdb/processing/aggregation.cpp b/cpp/arcticdb/processing/aggregation.cpp index 2d27449960..40145180ec 100644 --- a/cpp/arcticdb/processing/aggregation.cpp +++ b/cpp/arcticdb/processing/aggregation.cpp @@ -72,11 +72,11 @@ namespace } } - inline util::BitMagic::enumerator::value_type deref(util::BitMagic::enumerator iter) { + [[maybe_unused]] inline util::BitMagic::enumerator::value_type deref(util::BitMagic::enumerator iter) { return *iter; } - inline std::size_t deref(std::size_t index) { + [[maybe_unused]] inline std::size_t deref(std::size_t index) { return index; } diff --git a/cpp/arcticdb/processing/clause.cpp b/cpp/arcticdb/processing/clause.cpp index 8e4a050a0c..78922de488 100644 --- a/cpp/arcticdb/processing/clause.cpp +++ b/cpp/arcticdb/processing/clause.cpp @@ -497,8 +497,9 @@ Composite AggregationClause::process(Composite&& entity_id SegmentInMemory seg; auto index_col = std::make_shared(make_scalar_type(grouping_data_type), grouping_map.size(), true, false); + seg.add_column(scalar_field(grouping_data_type, grouping_column_), index_col); - seg.descriptor().set_index(IndexDescriptor(0, IndexDescriptor::ROWCOUNT)); + seg.descriptor().set_index(IndexDescriptorImpl(0, IndexDescriptorImpl::Type::ROWCOUNT)); details::visit_type(grouping_data_type, [&grouping_map, &index_col](auto data_type_tag) { using col_type_info = ScalarTypeInfo; @@ -622,7 +623,7 @@ void merge_impl( FieldCollection new_fields{}; (void)new_fields.add(fields[0].ref()); - auto index_desc = index_descriptor(stream_id, index, new_fields); + auto index_desc = index_descriptor_from_range(stream_id, index, new_fields); auto desc = StreamDescriptor{index_desc}; AggregatorType agg{ @@ -643,9 +644,9 @@ Composite MergeClause::process(Composite&& entity_ids) con [](const std::unique_ptr &left, const std::unique_ptr &right) { const auto left_index = index::index_value_from_row(left->row(), - IndexDescriptor::TIMESTAMP, 0); + IndexDescriptorImpl::Type::TIMESTAMP, 0); const auto right_index = index::index_value_from_row(right->row(), - IndexDescriptor::TIMESTAMP, 0); + IndexDescriptorImpl::Type::TIMESTAMP, 0); return left_index > right_index; }; @@ -749,7 +750,7 @@ Composite ColumnStatsGenerationClause::process(Composite&& end_index_col->set_row_data(0); SegmentInMemory seg; - seg.descriptor().set_index(IndexDescriptor(0, IndexDescriptor::ROWCOUNT)); + seg.descriptor().set_index(IndexDescriptorImpl(0, IndexDescriptorImpl::Type::ROWCOUNT)); seg.add_column(scalar_field(DataType::NANOSECONDS_UTC64, start_index_column_name), start_index_col); seg.add_column(scalar_field(DataType::NANOSECONDS_UTC64, end_index_column_name), end_index_col); for (const auto& agg_data: folly::enumerate(aggregators_data)) { diff --git a/cpp/arcticdb/processing/clause.hpp b/cpp/arcticdb/processing/clause.hpp index d646740fe8..435fe24129 100644 --- a/cpp/arcticdb/processing/clause.hpp +++ b/cpp/arcticdb/processing/clause.hpp @@ -344,12 +344,6 @@ struct PartitionClause { } }; -inline StreamDescriptor empty_descriptor(arcticdb::proto::descriptors::IndexDescriptor::Type type = arcticdb::proto::descriptors::IndexDescriptor::ROWCOUNT, const StreamId &id = "merged") { - const auto index = stream::variant_index_from_type(type); - const auto field_count = util::variant_match(index, [] (const auto& idx) { return idx.field_count(); }); - return StreamDescriptor{StreamId{id}, IndexDescriptor{field_count, type}, std::make_shared()}; -} - struct NamedAggregator { std::string aggregation_operator_; std::string input_column_name_; diff --git a/cpp/arcticdb/processing/operation_dispatch_unary.hpp b/cpp/arcticdb/processing/operation_dispatch_unary.hpp index c8018d7ba6..a1c0cf977a 100644 --- a/cpp/arcticdb/processing/operation_dispatch_unary.hpp +++ b/cpp/arcticdb/processing/operation_dispatch_unary.hpp @@ -115,7 +115,7 @@ VariantData unary_comparator(const Column& col, Func&& func) { constexpr auto sparse_missing_value_output = std::is_same_v; details::visit_type(col.type().data_type(), [&](auto col_tag) { using type_info = ScalarTypeInfo; - Column::transform(col, output_bitset, sparse_missing_value_output, [&func](auto input_value) -> bool { + Column::transform(col, output_bitset, sparse_missing_value_output, [&](auto input_value) -> bool { if constexpr (is_floating_point_type(type_info::data_type)) { return func.apply(input_value); } else if constexpr (is_sequence_type(type_info::data_type)) { diff --git a/cpp/arcticdb/python/normalization_checks.cpp b/cpp/arcticdb/python/normalization_checks.cpp index d34b806fbd..45c6d05e4f 100644 --- a/cpp/arcticdb/python/normalization_checks.cpp +++ b/cpp/arcticdb/python/normalization_checks.cpp @@ -195,7 +195,7 @@ void fix_normalization_or_throw( auto &old_norm = existing_isr.tsd().proto().normalization(); auto &new_norm = new_frame.norm_meta; - if (check_pandas_like(old_norm, new_norm, existing_isr.tsd().proto().total_rows())) + if (check_pandas_like(old_norm, new_norm, existing_isr.tsd().total_rows())) return; if (is_append) { if (check_ndarray_append(old_norm, new_norm)) diff --git a/cpp/arcticdb/python/python_handlers.cpp b/cpp/arcticdb/python/python_handlers.cpp index 4e9aca7f33..bb36d60efe 100644 --- a/cpp/arcticdb/python/python_handlers.cpp +++ b/cpp/arcticdb/python/python_handlers.cpp @@ -9,226 +9,225 @@ #include #include #include +#include namespace arcticdb { - /// @brief Generate numpy.dtype object from ArcticDB type descriptor - /// The dtype is used as type specifier for numpy arrays stored as column elements - /// @note There is special handling for ArcticDB's empty type - /// When numpy creates an empty array its type is float64. We want to mimic this because: - /// i) There is no equivalent to empty value - /// ii) We want input dataframes to be exact match of the output and that includes the type - [[nodiscard]] static inline py::dtype generate_python_dtype(const TypeDescriptor& td, stride_t type_byte_size) { - if(is_empty_type(td.data_type())) { - return py::dtype{"f8"}; - } - return py::dtype{fmt::format("{}{:d}", get_dtype_specifier(td.data_type()), type_byte_size)}; +/// @brief Generate numpy.dtype object from ArcticDB type descriptor +/// The dtype is used as type specifier for numpy arrays stored as column elements +/// @note There is special handling for ArcticDB's empty type +/// When numpy creates an empty array its type is float64. We want to mimic this because: +/// i) There is no equivalent to empty value +/// ii) We want input dataframes to be exact match of the output and that includes the type +[[nodiscard]] static inline py::dtype generate_python_dtype(const TypeDescriptor &td, stride_t type_byte_size) { + if (is_empty_type(td.data_type())) { + return py::dtype{"f8"}; } + return py::dtype{fmt::format("{}{:d}", get_dtype_specifier(td.data_type()), type_byte_size)}; +} - /// @important This calls pybind's initialize array function which is NOT thread safe. Moreover, numpy arrays can - /// be created only by the thread holding the GIL. In practice we can get away with allocating arrays only from - /// a single thread (even if it's not the one holding the GIL). This, however, is not guaranteed to work. - /// @todo Allocate numpy arrays only from the thread holding the GIL - [[nodiscard]] static inline PyObject* initialize_array( - const pybind11::dtype& descr, - const shape_t shapes, - const stride_t strides, - const void* source_ptr, - std::shared_ptr owner, - std::mutex& creation_mutex - ) { - std::lock_guard creation_guard{creation_mutex}; - // TODO: Py capsule can take only void ptr as input. We need a better way to handle destruction - // Allocating shared ptr on the heap is sad. - auto* object = new std::shared_ptr(std::move(owner)); - auto arr = py::array(descr, {shapes}, {strides}, source_ptr, py::capsule(object, [](void* obj){ - delete reinterpret_cast*>(obj); - })); - return arr.release().ptr(); - } +/// @important This calls pybind's initialize array function which is NOT thread safe. Moreover, numpy arrays can +/// be created only by the thread holding the GIL. In practice we can get away with allocating arrays only from +/// a single thread (even if it's not the one holding the GIL). This, however, is not guaranteed to work. +/// @todo Allocate numpy arrays only from the thread holding the GIL +[[nodiscard]] static inline PyObject *initialize_array( + const pybind11::dtype &descr, + const shape_t shapes, + const stride_t strides, + const void *source_ptr, + std::shared_ptr owner, + std::mutex &creation_mutex +) { + std::lock_guard creation_guard{creation_mutex}; + // TODO: Py capsule can take only void ptr as input. We need a better way to handle destruction + // Allocating shared ptr on the heap is sad. + auto *object = new std::shared_ptr(std::move(owner)); + auto arr = py::array(descr, {shapes}, {strides}, source_ptr, py::capsule(object, [](void *obj) { + delete reinterpret_cast *>(obj); + })); + return arr.release().ptr(); +} - static inline const PyObject** fill_with_none(const PyObject** dest, size_t count) { - auto none = py::none(); - std::generate_n(dest, count, [&none]() { return none.inc_ref().ptr(); }); - return dest + count; +static inline const PyObject **fill_with_none(const PyObject **dest, size_t count) { + auto none = py::none(); + std::generate_n(dest, count, [&none]() { return none.inc_ref().ptr(); }); + return dest + count; +} + +void EmptyHandler::handle_type( + const uint8_t *&input, + uint8_t *dest, + const EncodedFieldImpl &field, + const entity::TypeDescriptor &, + size_t, + std::shared_ptr, + EncodingVersion encoding_version, + const ColumnMapping &m +) { + ARCTICDB_SAMPLE(HandleEmpty, 0) + util::check(dest != nullptr, "Got null destination pointer"); + ARCTICDB_TRACE( + log::version(), + "Empty type handler invoked for source type: {}, destination type: {}, num rows: {}", + m.source_type_desc_, + m.dest_type_desc_, + m.num_rows_ + ); + static_assert(get_type_size(DataType::EMPTYVAL) == sizeof(PyObject *)); + + if (encoding_version == EncodingVersion::V2) + util::check_magic(input); + + if (field.encoding_case() == EncodedFieldType::NDARRAY) { + const auto &ndarray_field = field.ndarray(); + const auto num_blocks = ndarray_field.values_size(); + util::check(num_blocks <= 1, "Unexpected number of empty type blocks: {}", num_blocks); + for (auto block_num = 0; block_num < num_blocks; ++block_num) { + const auto &block_info = ndarray_field.values(block_num); + input += block_info.out_bytes(); + } + } else { + util::raise_rte("Unsupported encoding {}", field); } +} - void EmptyHandler::handle_type( - const uint8_t*& input, - uint8_t* dest, - const VariantField& variant_field, - size_t dest_bytes, - std::shared_ptr, - EncodingVersion, - const ColumnMapping& m - ) { - ARCTICDB_SAMPLE(HandleEmpty, 0) - util::check(dest != nullptr, "Got null destination pointer"); - ARCTICDB_TRACE( - log::version(), - "Empty type handler invoked for source type: {}, destination type: {}, num rows: {}", - m.source_type_desc_, - m.dest_type_desc_, - m.num_rows_ - ); - static_assert(get_type_size(DataType::EMPTYVAL) == sizeof(PyObject*)); - - m.dest_type_desc_.visit_tag([&](auto tag) { - util::default_initialize(dest, dest_bytes); - }); +int EmptyHandler::type_size() const { + return sizeof(PyObject *); +} - util::variant_match(variant_field, [&input](const auto& field) { - using EncodedFieldType = std::decay_t; - if constexpr (std::is_same_v) - util::check_magic(input); - - if (field->encoding_case() == EncodedFieldType::kNdarray) { - const auto& ndarray_field = field->ndarray(); - const auto num_blocks = ndarray_field.values_size(); - util::check(num_blocks <= 1, "Unexpected number of empty type blocks: {}", num_blocks); - for (auto block_num = 0; block_num < num_blocks; ++block_num) { - const auto& block_info = ndarray_field.values(block_num); - input += block_info.out_bytes(); - } - } else { - util::raise_error_msg("Unsupported encoding {}", *field); - } +void EmptyHandler::default_initialize(void *dest, size_t byte_size) const { + fill_with_none(reinterpret_cast(dest), byte_size / type_size()); +} + +void BoolHandler::handle_type( + const uint8_t *&data, + uint8_t *dest, + const EncodedFieldImpl &field, + const entity::TypeDescriptor &, + size_t, + std::shared_ptr, + EncodingVersion encoding_version, + const ColumnMapping &m) { + ARCTICDB_SAMPLE(HandleBool, 0) + util::check(dest != nullptr, "Got null destination pointer"); + util::check(field.has_ndarray(), "Bool handler expected array"); + ARCTICDB_DEBUG(log::version(), "Bool handler got encoded field: {}", field.DebugString()); + auto ptr_dest = reinterpret_cast(dest); + const auto &ndarray = field.ndarray(); + const auto bytes = encoding_sizes::data_uncompressed_size(ndarray); + ChunkedBuffer decoded_data = ChunkedBuffer::presized(bytes); + SliceDataSink decoded_data_sink{decoded_data.data(), bytes}; + std::optional sparse_map; + data += decode_field(m.source_type_desc_, field, data, decoded_data_sink, sparse_map, encoding_version); + const auto num_bools = sparse_map.has_value() ? sparse_map->count() : m.num_rows_; + auto ptr_src = decoded_data.template ptr_cast(0, num_bools * sizeof(uint8_t)); + if (sparse_map.has_value()) { + ARCTICDB_TRACE(log::codec(), "Bool handler using a sparse map"); + unsigned last_row = 0u; + for (auto en = sparse_map->first(); en < sparse_map->end(); ++en, last_row++) { + const auto current_pos = *en; + ptr_dest = fill_with_none(ptr_dest, current_pos - last_row); + last_row = current_pos; + *ptr_dest++ = py::bool_(static_cast(*ptr_src++)).release().ptr(); + } + fill_with_none(ptr_dest, m.num_rows_ - last_row); + } else { + ARCTICDB_TRACE(log::codec(), "Bool handler didn't find a sparse map. Assuming dense array."); + std::transform(ptr_src, ptr_src + num_bools, ptr_dest, [](uint8_t value) { + return py::bool_(static_cast(value)).release().ptr(); }); } +} - int EmptyHandler::type_size() const { - return sizeof(PyObject*); - } +int BoolHandler::type_size() const { + return sizeof(PyObject *); +} - void EmptyHandler::default_initialize(void* dest, size_t byte_size) const { - fill_with_none(reinterpret_cast(dest), byte_size / type_size()); - } +void BoolHandler::default_initialize(void *dest, size_t byte_size) const { + fill_with_none(reinterpret_cast(dest), byte_size / type_size()); +} - void BoolHandler::handle_type( - const uint8_t*& data, - uint8_t* dest, - const VariantField& encoded_field_info, - size_t, - std::shared_ptr, - EncodingVersion encding_version, - const ColumnMapping& m - ) { - std::visit([&](const auto& field){ - ARCTICDB_SAMPLE(HandleBool, 0) - util::check(dest != nullptr, "Got null destination pointer"); - util::check(field->has_ndarray(), "Bool handler expected array"); - ARCTICDB_DEBUG(log::version(), "Bool handler got encoded field: {}", field->DebugString()); - auto ptr_dest = reinterpret_cast(dest); - const auto& ndarray = field->ndarray(); - const auto bytes = encoding_sizes::data_uncompressed_size(ndarray); - ChunkedBuffer decoded_data = ChunkedBuffer::presized(bytes); - SliceDataSink decoded_data_sink{decoded_data.data(), bytes}; - std::optional sparse_map; - data += decode_field(m.source_type_desc_, *field, data, decoded_data_sink, sparse_map, encding_version); - const auto num_bools = sparse_map.has_value() ? sparse_map->count() : m.num_rows_; - auto ptr_src = decoded_data.template ptr_cast(0, num_bools * sizeof(uint8_t)); - if (sparse_map.has_value()) { - ARCTICDB_TRACE(log::codec(), "Bool handler using a sparse map"); - unsigned last_row = 0u; - for (auto en = sparse_map->first(); en < sparse_map->end(); ++en, last_row++) { - const auto current_pos = *en; - ptr_dest = fill_with_none(ptr_dest, current_pos - last_row); - last_row = current_pos; - *ptr_dest++ = py::bool_(static_cast(*ptr_src++)).release().ptr(); - } - fill_with_none(ptr_dest, m.num_rows_ - last_row); - } else { - ARCTICDB_TRACE(log::codec(), "Bool handler didn't find a sparse map. Assuming dense array."); - std::transform(ptr_src, ptr_src + num_bools, ptr_dest, [](uint8_t value) { - return py::bool_(static_cast(value)).release().ptr(); - }); - } - }, encoded_field_info); +std::mutex ArrayHandler::initialize_array_mutex; + +void ArrayHandler::handle_type( + const uint8_t *&data, + uint8_t *dest, + const EncodedFieldImpl &field, + const entity::TypeDescriptor &, + size_t, + std::shared_ptr buffers, + EncodingVersion encoding_version, + const ColumnMapping &m +) { + ARCTICDB_SAMPLE(HandleArray, 0) + util::check(field.has_ndarray(), "Expected ndarray in array object handler"); + + auto ptr_dest = reinterpret_cast(dest); + if (!field.ndarray().sparse_map_bytes()) { + log::version().info("Array handler has no values"); + fill_with_none(ptr_dest, m.num_rows_); + return; } + std::shared_ptr column = buffers->get_buffer(m.source_type_desc_, true); + column->check_magic(); + log::version().info("Column got buffer at {}", uintptr_t(column.get())); + auto bv = std::make_optional(util::BitSet{}); + data += decode_field(m.source_type_desc_, field, data, *column, bv, encoding_version); + + auto last_row = 0u; + ARCTICDB_SUBSAMPLE(InitArrayAcquireGIL, 0) + const auto strides = static_cast(get_type_size(m.source_type_desc_.data_type())); + const py::dtype py_dtype = generate_python_dtype(m.source_type_desc_, strides); + m.source_type_desc_.visit_tag([&](auto tdt) { + const auto &blocks = column->blocks(); + if (blocks.empty()) + return; + + auto block_it = blocks.begin(); + const auto *shapes = column->shape_ptr(); + auto block_pos = 0u; + const auto *ptr_src = (*block_it)->data(); + constexpr stride_t stride = static_cast(tdt).get_type_byte_size(); + for (auto en = bv->first(); en < bv->end(); ++en) { + const shape_t shape = shapes ? *shapes : 0; + const auto offset = *en; + ptr_dest = fill_with_none(ptr_dest, offset - last_row); + last_row = offset; + *ptr_dest++ = initialize_array(py_dtype, + shape, + stride, + ptr_src + block_pos, + column, + initialize_array_mutex); + block_pos += shape * stride; + if (shapes) { + ++shapes; + } + if (block_it != blocks.end() && block_pos == (*block_it)->bytes() && ++block_it != blocks.end()) { + ptr_src = (*block_it)->data(); + block_pos = 0; + } - int BoolHandler::type_size() const { - return sizeof(PyObject*); - } + ++last_row; + } + if (block_it != blocks.end() && block_pos == (*block_it)->bytes() && ++block_it != blocks.end()) { + ptr_src = (*block_it)->data(); + block_pos = 0; + } - void BoolHandler::default_initialize(void* dest, size_t byte_size) const { - fill_with_none(reinterpret_cast(dest), byte_size / type_size()); - } + ++last_row; + }); - std::mutex ArrayHandler::initialize_array_mutex; - - void ArrayHandler::handle_type( - const uint8_t*& data, - uint8_t* dest, - const VariantField& encoded_field_info, - size_t, - std::shared_ptr buffers, - EncodingVersion encoding_version, - const ColumnMapping& m - ) { - util::variant_match(encoded_field_info, [&](auto field){ - ARCTICDB_SAMPLE(HandleArray, 0) - util::check(field->has_ndarray(), "Expected ndarray in array object handler"); - - auto ptr_dest = reinterpret_cast(dest); - if(!field->ndarray().sparse_map_bytes()) { - log::version().info("Array handler has no values"); - fill_with_none(ptr_dest, m.num_rows_); - return; - } - std::shared_ptr column = buffers->get_buffer(m.source_type_desc_, true); - column->check_magic(); - log::version().info("Column got buffer at {}", uintptr_t(column.get())); - auto bv = std::make_optional(util::BitSet{}); - data += decode_field(m.source_type_desc_, *field, data, *column, bv, encoding_version); - - auto last_row = 0u; - ARCTICDB_SUBSAMPLE(InitArrayAcquireGIL, 0) - const auto strides = static_cast(get_type_size(m.source_type_desc_.data_type())); - const py::dtype py_dtype = generate_python_dtype(m.source_type_desc_, strides); - m.source_type_desc_.visit_tag([&] (auto tdt) { - const auto& blocks = column->blocks(); - if(blocks.empty()) - return; - - auto block_it = blocks.begin(); - const auto* shapes = column->shape_ptr(); - auto block_pos = 0u; - const auto* ptr_src = (*block_it)->data(); - constexpr stride_t stride = static_cast(tdt).get_type_byte_size(); - for (auto en = bv->first(); en < bv->end(); ++en) { - const shape_t shape = shapes ? *shapes : 0; - const auto offset = *en; - ptr_dest = fill_with_none(ptr_dest, offset - last_row); - last_row = offset; - *ptr_dest++ = initialize_array(py_dtype, - shape, - stride, - ptr_src + block_pos, - column, - initialize_array_mutex); - block_pos += shape * stride; - if(shapes) { - ++shapes; - } - if(block_it != blocks.end() && block_pos == (*block_it)->bytes() && ++block_it != blocks.end()) { - ptr_src = (*block_it)->data(); - block_pos = 0; - } - - ++last_row; - } - }); - - ARCTICDB_SUBSAMPLE(ArrayIncNones, 0) - fill_with_none(ptr_dest, m.num_rows_ - last_row); - }); - } + ARCTICDB_SUBSAMPLE(ArrayIncNones, 0) + fill_with_none(ptr_dest, m.num_rows_ - last_row); +} - int ArrayHandler::type_size() const { - return sizeof(PyObject*); - } +int ArrayHandler::type_size() const { + return sizeof(PyObject *); +} - void ArrayHandler::default_initialize(void* dest, size_t byte_size) const { - fill_with_none(reinterpret_cast(dest), byte_size / type_size()); - } +void ArrayHandler::default_initialize(void *dest, size_t byte_size) const { + fill_with_none(reinterpret_cast(dest), byte_size / type_size()); } + +} //namespace arcticdb \ No newline at end of file diff --git a/cpp/arcticdb/python/python_handlers.hpp b/cpp/arcticdb/python/python_handlers.hpp index 186c87cbd9..f65f8ba65a 100644 --- a/cpp/arcticdb/python/python_handlers.hpp +++ b/cpp/arcticdb/python/python_handlers.hpp @@ -17,7 +17,8 @@ namespace arcticdb { void handle_type( const uint8_t*& data, uint8_t* dest, - const VariantField& encoded_field, + const EncodedFieldImpl& encoded_field, + const entity::TypeDescriptor& type_descriptor, size_t dest_bytes, std::shared_ptr buffers, EncodingVersion encding_version, @@ -33,7 +34,8 @@ namespace arcticdb { void handle_type( const uint8_t *&data, uint8_t *dest, - const VariantField &encoded_field, + const EncodedFieldImpl &encoded_field, + const entity::TypeDescriptor &type_descriptor, size_t dest_bytes, std::shared_ptr buffers, EncodingVersion encding_version, @@ -45,13 +47,12 @@ namespace arcticdb { struct DecimalHandler { void handle_type( - const uint8_t*& data, - uint8_t* dest, - const VariantField& encoded_field, - size_t dest_bytes, - std::shared_ptr buffers, - EncodingVersion encding_version, - const ColumnMapping& m + const uint8_t*& data, + uint8_t* dest, + const EncodedFieldImpl& encoded_field, + const entity::TypeDescriptor& type_descriptor, + size_t dest_bytes, + std::shared_ptr buffers ); int type_size() const; }; @@ -61,7 +62,8 @@ namespace arcticdb { void handle_type( const uint8_t*& data, uint8_t* dest, - const VariantField& encoded_field, + const EncodedFieldImpl& encoded_field, + const entity::TypeDescriptor& type_descriptor, size_t dest_bytes, std::shared_ptr buffers, EncodingVersion encding_version, diff --git a/cpp/arcticdb/python/python_to_tensor_frame.cpp b/cpp/arcticdb/python/python_to_tensor_frame.cpp index b7952bcf59..60d3fc1681 100644 --- a/cpp/arcticdb/python/python_to_tensor_frame.cpp +++ b/cpp/arcticdb/python/python_to_tensor_frame.cpp @@ -223,7 +223,7 @@ std::shared_ptr py_ndf_to_frame( if (idx_names.empty()) { res->index = stream::RowCountIndex(); - res->desc.set_index_type(IndexDescriptor::ROWCOUNT); + res->desc.set_index_type(IndexDescriptorImpl::Type::ROWCOUNT); } else { util::check(idx_names.size() == 1, "Multi-indexed dataframes not handled"); auto index_tensor = obj_to_tensor(idx_vals[0].ptr(), empty_types); @@ -237,14 +237,14 @@ std::shared_ptr py_ndf_to_frame( if (index_tensor.data_type() == DataType::NANOSECONDS_UTC64 || is_empty_type(index_tensor.data_type())) { res->desc.set_index_field_count(1); - res->desc.set_index_type(IndexDescriptor::TIMESTAMP); + res->desc.set_index_type(IndexDescriptorImpl::Type::TIMESTAMP); res->desc.add_scalar_field(index_tensor.dt_, index_column_name); res->index = stream::TimeseriesIndex(index_column_name); res->index_tensor = std::move(index_tensor); } else { res->index = stream::RowCountIndex(); - res->desc.set_index_type(IndexDescriptor::ROWCOUNT); + res->desc.set_index_type(IndexDescriptorImpl::Type::ROWCOUNT); res->desc.add_scalar_field(index_tensor.dt_, index_column_name); res->field_tensors.push_back(std::move(index_tensor)); } @@ -285,7 +285,7 @@ std::shared_ptr py_none_to_frame() { // Fill index res->index = stream::RowCountIndex(); - res->desc.set_index_type(IndexDescriptor::ROWCOUNT); + res->desc.set_index_type(IndexDescriptorImpl::Type::ROWCOUNT); // Fill tensors auto col_name = "bytes"; diff --git a/cpp/arcticdb/storage/azure/azure_mock_client.cpp b/cpp/arcticdb/storage/azure/azure_mock_client.cpp index efafb98bd9..4f7a66d158 100644 --- a/cpp/arcticdb/storage/azure/azure_mock_client.cpp +++ b/cpp/arcticdb/storage/azure/azure_mock_client.cpp @@ -84,7 +84,7 @@ Segment MockAzureClient::read_blob( throw get_exception(message, error_code, Azure::Core::Http::HttpStatusCode::NotFound); } - return pos->second; + return std::move(pos->second); } void MockAzureClient::delete_blobs( diff --git a/cpp/arcticdb/storage/azure/azure_real_client.cpp b/cpp/arcticdb/storage/azure/azure_real_client.cpp index 60a8101d78..645423e002 100644 --- a/cpp/arcticdb/storage/azure/azure_real_client.cpp +++ b/cpp/arcticdb/storage/azure/azure_real_client.cpp @@ -46,24 +46,13 @@ void RealAzureClient::write_blob( const Azure::Storage::Blobs::UploadBlockBlobFromOptions& upload_option, unsigned int request_timeout) { - std::shared_ptr tmp; - auto hdr_size = segment.segment_header_bytes_size(); - auto [dst, write_size] = segment.try_internal_write(tmp, hdr_size); - util::check(arcticdb::Segment::FIXED_HEADER_SIZE + hdr_size + segment.buffer().bytes() <= write_size, - "Size disparity, fixed header size {} + variable header size {} + buffer size {} >= total size {}", - arcticdb::Segment::FIXED_HEADER_SIZE, - hdr_size, - segment.buffer().bytes(), - write_size); + auto [dst, write_size, buffer] = segment.serialize_header(); ARCTICDB_SUBSAMPLE(AzureStorageUploadObject, 0) auto blob_client = container_client.GetBlockBlobClient(blob_name); ARCTICDB_RUNTIME_DEBUG(log::storage(), "Writing key '{}' with {} bytes of data", blob_name, - segment.total_segment_size(hdr_size)); + write_size); blob_client.UploadFrom(dst, write_size, upload_option, get_context(request_timeout)); - ARCTICDB_RUNTIME_DEBUG(log::storage(), "Wrote key '{}' with {} bytes of data", - blob_name, - segment.total_segment_size(hdr_size)); } Segment RealAzureClient::read_blob( diff --git a/cpp/arcticdb/storage/azure/azure_storage.cpp b/cpp/arcticdb/storage/azure/azure_storage.cpp index 8b381c34a4..34d7fd3ecf 100644 --- a/cpp/arcticdb/storage/azure/azure_storage.cpp +++ b/cpp/arcticdb/storage/azure/azure_storage.cpp @@ -247,7 +247,7 @@ void do_iterate_type_impl(KeyType key_type, auto key_type_dir = key_type_folder(root_folder, key_type); KeyDescriptor key_descriptor(prefix, - is_ref_key_class(key_type) ? IndexDescriptor::UNKNOWN : IndexDescriptor::TIMESTAMP, FormatType::TOKENIZED); + 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 auto root_folder_size = key_type_dir.size() + 1 + bucketizer.bucketize_length(key_type); diff --git a/cpp/arcticdb/storage/azure/azure_storage.hpp b/cpp/arcticdb/storage/azure/azure_storage.hpp index 7dc04f28ed..354be06a10 100644 --- a/cpp/arcticdb/storage/azure/azure_storage.hpp +++ b/cpp/arcticdb/storage/azure/azure_storage.hpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include diff --git a/cpp/arcticdb/storage/coalesced/multi_segment_utils.hpp b/cpp/arcticdb/storage/coalesced/multi_segment_utils.hpp index e0923500f7..d19f0eaede 100644 --- a/cpp/arcticdb/storage/coalesced/multi_segment_utils.hpp +++ b/cpp/arcticdb/storage/coalesced/multi_segment_utils.hpp @@ -15,14 +15,14 @@ static constexpr uint64_t NumericFlag = uint64_t(1) << 31; static_assert(NumericFlag > NumericMask); template -uint64_t get_symbol_prefix(const entity::StreamId& stream_id) { +uint64_t get_symbol_prefix(const StreamId& stream_id) { using InternalType = uint64_t; static_assert(sizeof(StorageType) <= sizeof(InternalType)); constexpr size_t end = sizeof(InternalType); constexpr size_t begin = sizeof(InternalType) - sizeof(StorageType); StorageType data{}; util::variant_match(stream_id, - [&data] (const entity::StringId& string_id) { + [&data] (const StringId& string_id) { auto* target = reinterpret_cast(&data); for(size_t p = begin, i = 0; p < end && i < string_id.size(); ++p, ++i) { const auto c = string_id[i]; @@ -30,8 +30,8 @@ uint64_t get_symbol_prefix(const entity::StreamId& stream_id) { target[p] = c; } }, - [&data] (const entity::NumericId& numeric_id) { - util::check(numeric_id < static_cast(NumericMask), "Numeric id too large: {}", numeric_id); + [&data] (const NumericId& numeric_id) { + util::check(numeric_id < static_cast(NumericMask), "Numeric id too large: {}", numeric_id); data &= NumericFlag; data &= numeric_id; } @@ -49,7 +49,7 @@ struct TimeSymbol { IndexDataType data_ = 0UL; - TimeSymbol(const entity::StreamId& stream_id, entity::timestamp time) { + TimeSymbol(const StreamId& stream_id, entity::timestamp time) { set_data(stream_id, time); } @@ -62,7 +62,7 @@ struct TimeSymbol { } private: - void set_data(const entity::StreamId& stream_id, entity::timestamp time) { + void set_data(const StreamId& stream_id, entity::timestamp time) { time <<= 32; auto prefix = get_symbol_prefix(stream_id); data_ = time | prefix; diff --git a/cpp/arcticdb/storage/file/mapped_file_storage.cpp b/cpp/arcticdb/storage/file/mapped_file_storage.cpp index b15672605e..67ef786dc8 100644 --- a/cpp/arcticdb/storage/file/mapped_file_storage.cpp +++ b/cpp/arcticdb/storage/file/mapped_file_storage.cpp @@ -8,6 +8,7 @@ #include #include +#include #include #include #include @@ -41,7 +42,7 @@ void MappedFileStorage::init() { EncodingVersion{ static_cast(config_.encoding_version())}).max_compressed_bytes_; StreamId id = config_.has_str_id() ? StreamId{} : NumericId{}; - data_size += entity::max_key_size(id, IndexDescriptor{config_.index()}); + data_size += entity::max_key_size(id, index_descriptor_from_proto(config_.index())); file_.create_file(config_.path(), data_size); } else { ARCTICDB_DEBUG(log::storage(), "Opening existing mapped file storage at path {}", config_.path()); @@ -60,10 +61,10 @@ void MappedFileStorage::do_load_header(size_t header_offset, size_t header_size) multi_segment_header_.set_segment(std::move(header)); } -uint64_t MappedFileStorage::get_data_offset(const Segment& seg, size_t header_size) { +uint64_t MappedFileStorage::get_data_offset(const Segment& seg) { ARCTICDB_SAMPLE(MappedFileStorageGetOffset, 0) std::lock_guard lock{offset_mutex_}; - const auto segment_size = seg.total_segment_size(header_size); + const auto segment_size = seg.size(); ARCTICDB_DEBUG(log::storage(), "Mapped file storage returning offset {} and adding {} bytes", offset_, segment_size); const auto previous_offset = offset_; offset_ += segment_size; @@ -72,12 +73,11 @@ uint64_t MappedFileStorage::get_data_offset(const Segment& seg, size_t header_si uint64_t MappedFileStorage::write_segment(Segment&& seg) { auto segment = std::move(seg); - const auto header_size = segment.segment_header_bytes_size(); - auto offset = get_data_offset(segment, header_size); + auto offset = get_data_offset(segment); auto* data = file_.data() + offset; ARCTICDB_SUBSAMPLE(FileStorageMemCpy, 0) - ARCTICDB_DEBUG(log::storage(), "Mapped file storage writing segment of size {} at offset {}", segment.total_segment_size(header_size), offset); - segment.write_to(data, header_size); + segment.write_to(data); + ARCTICDB_DEBUG(log::storage(), "Mapped file storage wrote segment of size {} at offset {}", segment.size(), offset); return offset; } @@ -85,8 +85,8 @@ void MappedFileStorage::do_write(Composite&& kvs) { ARCTICDB_SAMPLE(MappedFileStorageWriteValues, 0) auto key_values = std::move(kvs); key_values.broadcast([this] (auto key_seg) { - const auto size = key_seg.segment().total_segment_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); }); } diff --git a/cpp/arcticdb/storage/file/mapped_file_storage.hpp b/cpp/arcticdb/storage/file/mapped_file_storage.hpp index aedc4359e5..5a2bc06b38 100644 --- a/cpp/arcticdb/storage/file/mapped_file_storage.hpp +++ b/cpp/arcticdb/storage/file/mapped_file_storage.hpp @@ -10,8 +10,10 @@ #include #include #include +#include #include #include +#include #include namespace fs = std::filesystem; @@ -53,7 +55,7 @@ class MappedFileStorage final : public SingleFileStorage { void do_finalize(KeyData key_data) override; - uint64_t get_data_offset(const Segment& seg, size_t header_size); + uint64_t get_data_offset(const Segment& seg); void do_load_header(size_t header_offset, size_t header_size) override; @@ -79,7 +81,7 @@ inline arcticdb::proto::storage::VariantStorage pack_config( size_t file_size, size_t items_count, const StreamId& id, - const IndexDescriptor& index_desc, + const IndexDescriptorImpl& index_desc, EncodingVersion encoding_version, const arcticdb::proto::encoding::VariantCodec& codec_opts) { arcticdb::proto::storage::VariantStorage output; @@ -90,7 +92,7 @@ inline arcticdb::proto::storage::VariantStorage pack_config( util::variant_match(id, [&cfg] (const StringId& str) { cfg.set_str_id(str); }, [&cfg] (const NumericId& n) { cfg.set_num_id(n); }); - cfg.mutable_index()->CopyFrom(index_desc.proto()), + cfg.mutable_index()->CopyFrom(index_descriptor_to_proto(index_desc)), cfg.set_encoding_version(static_cast(encoding_version)); cfg.mutable_codec_opts()->CopyFrom(codec_opts); util::pack_to_any(cfg, *output.mutable_config()); diff --git a/cpp/arcticdb/storage/library.hpp b/cpp/arcticdb/storage/library.hpp index 3e5122b636..e7b9d3907f 100644 --- a/cpp/arcticdb/storage/library.hpp +++ b/cpp/arcticdb/storage/library.hpp @@ -76,28 +76,15 @@ class Library { throw LibraryPermissionException(library_path_, open_mode(), "write"); } - [[maybe_unused]] const size_t total_size = kvs.fold( - [](size_t s, const KeySegmentPair& seg) { return s + seg.segment().total_segment_size(); }, - size_t(0) - ); - [[maybe_unused]] const auto kv_count = kvs.size(); storages_->write(std::move(kvs)); - ARCTICDB_TRACE(log::storage(), "{} kv written, {} bytes", kv_count, total_size); } void update(Composite&& kvs, storage::UpdateOpts opts) { ARCTICDB_SAMPLE(LibraryUpdate, 0) - if (open_mode() < OpenMode::WRITE) { + if (open_mode() < OpenMode::WRITE) throw LibraryPermissionException(library_path_, open_mode(), "update"); - } - [[maybe_unused]] const size_t total_size = kvs.fold( - [](size_t s, const KeySegmentPair& seg) { return s + seg.segment().total_segment_size(); }, - size_t(0) - ); - [[maybe_unused]] const auto kv_count = kvs.size(); storages_->update(std::move(kvs), opts); - ARCTICDB_TRACE(log::storage(), "{} kv updated, {} bytes", kv_count, total_size); } void read(Composite&& ks, const ReadVisitor& visitor, ReadKeyOpts opts) { diff --git a/cpp/arcticdb/storage/library_manager.cpp b/cpp/arcticdb/storage/library_manager.cpp index 098f48c300..8f4e28095f 100644 --- a/cpp/arcticdb/storage/library_manager.cpp +++ b/cpp/arcticdb/storage/library_manager.cpp @@ -85,6 +85,7 @@ LibraryManager::LibraryManager(const std::shared_ptr& library) void LibraryManager::write_library_config(const py::object& lib_cfg, const LibraryPath& path, const StorageOverride& storage_override, const bool validate) const { SegmentInMemory segment; + segment.descriptor().set_index({0UL, IndexDescriptor::Type::ROWCOUNT}); arcticdb::proto::storage::LibraryConfig lib_cfg_proto; google::protobuf::Any output = {}; diff --git a/cpp/arcticdb/storage/library_path.hpp b/cpp/arcticdb/storage/library_path.hpp index fcef9d314e..14702c5c6a 100644 --- a/cpp/arcticdb/storage/library_path.hpp +++ b/cpp/arcticdb/storage/library_path.hpp @@ -29,9 +29,6 @@ class DefaultStringViewable : public std::shared_ptr { std::make_shared(args...)), hash_(arcticdb::hash(std::string_view{*this})) {} - DefaultStringViewable(const DefaultStringViewable &that) : - std::shared_ptr::shared_ptr(that), hash_(that.hash_) {} - operator std::string_view() const { return *this->get(); } diff --git a/cpp/arcticdb/storage/lmdb/lmdb_mock_client.cpp b/cpp/arcticdb/storage/lmdb/lmdb_mock_client.cpp index 879089797c..be3888f684 100644 --- a/cpp/arcticdb/storage/lmdb/lmdb_mock_client.cpp +++ b/cpp/arcticdb/storage/lmdb/lmdb_mock_client.cpp @@ -86,18 +86,18 @@ std::optional MockLmdbClient::read(const std::string& db_name, std::str return std::nullopt; } - return lmdb_contents_.at(key); + return std::make_optional(lmdb_contents_.at(key).clone()); } void MockLmdbClient::write(const std::string& db_name, std::string& path, arcticdb::Segment&& segment, ::lmdb::txn&, ::lmdb::dbi&, int64_t) { LmdbKey key = {db_name, path}; - raise_if_has_failure_trigger(key, StorageOperation::WRITE); + raise_if_has_failure_trigger(key, StorageOperation::WRITE); if(has_key(key)) { raise_key_exists_error(lmdb_operation_string(StorageOperation::WRITE)); } else { - lmdb_contents_.insert({key, segment}); + lmdb_contents_.try_emplace(key, std::move(segment)); } } diff --git a/cpp/arcticdb/storage/lmdb/lmdb_real_client.cpp b/cpp/arcticdb/storage/lmdb/lmdb_real_client.cpp index 4eb35fcd92..75874dc3f2 100644 --- a/cpp/arcticdb/storage/lmdb/lmdb_real_client.cpp +++ b/cpp/arcticdb/storage/lmdb/lmdb_real_client.cpp @@ -35,7 +35,7 @@ std::optional RealLmdbClient::read(const std::string&, std::string& pat return std::nullopt; } - auto segment = Segment::from_bytes(reinterpret_cast(mdb_val.mv_data),mdb_val.mv_size); + auto segment = Segment::from_bytes(reinterpret_cast(mdb_val.mv_data), mdb_val.mv_size); return segment; } @@ -43,9 +43,8 @@ void RealLmdbClient::write(const std::string&, std::string& path, arcticdb::Segm ::lmdb::txn& txn, ::lmdb::dbi& dbi, int64_t overwrite_flag) { MDB_val mdb_key{path.size(), path.data()}; - std::size_t hdr_sz = seg.segment_header_bytes_size(); MDB_val mdb_val; - mdb_val.mv_size = seg.total_segment_size(hdr_sz); + mdb_val.mv_size = seg.calculate_size(); ARCTICDB_SUBSAMPLE(LmdbPut, 0) int rc = ::mdb_put(txn.handle(), dbi.handle(), &mdb_key, &mdb_val, MDB_RESERVE | overwrite_flag); @@ -55,7 +54,7 @@ void RealLmdbClient::write(const std::string&, std::string& path, arcticdb::Segm ARCTICDB_SUBSAMPLE(LmdbMemCpy, 0) // mdb_val now points to a reserved memory area we must write to - seg.write_to(reinterpret_cast(mdb_val.mv_data), hdr_sz); + seg.write_to(reinterpret_cast(mdb_val.mv_data)); } bool RealLmdbClient::remove(const std::string&, std::string& path, ::lmdb::txn& txn, ::lmdb::dbi& dbi) { diff --git a/cpp/arcticdb/storage/lmdb/lmdb_storage.cpp b/cpp/arcticdb/storage/lmdb/lmdb_storage.cpp index db1879cd2a..53f0d5a5f5 100644 --- a/cpp/arcticdb/storage/lmdb/lmdb_storage.cpp +++ b/cpp/arcticdb/storage/lmdb/lmdb_storage.cpp @@ -63,6 +63,7 @@ void LmdbStorage::do_write_internal(Composite&& kvs, ::lmdb::txn 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); @@ -127,9 +128,9 @@ void LmdbStorage::do_read(Composite&& ks, const ReadVisitor& visitor ARCTICDB_SUBSAMPLE(LmdbStorageVisitSegment, 0) std::any keepalive; segment.value().set_keepalive(std::any(std::move(txn))); - visitor(k, std::move(segment.value())); ARCTICDB_DEBUG(log::storage(), "Read key {}: {}, with {} bytes of data", variant_key_type(k), - variant_key_view(k), segment.value().total_segment_size()); + variant_key_view(k), segment.value().size()); + visitor(k, std::move(segment.value())); } else { ARCTICDB_DEBUG(log::storage(), "Failed to find segment for key {}", variant_key_view(k)); failed_reads.push_back(k); diff --git a/cpp/arcticdb/storage/lmdb/lmdb_storage.hpp b/cpp/arcticdb/storage/lmdb/lmdb_storage.hpp index 5629738084..e41081f0f6 100644 --- a/cpp/arcticdb/storage/lmdb/lmdb_storage.hpp +++ b/cpp/arcticdb/storage/lmdb/lmdb_storage.hpp @@ -9,7 +9,7 @@ #include #include - +#include #include #include diff --git a/cpp/arcticdb/storage/memory/memory_storage.cpp b/cpp/arcticdb/storage/memory/memory_storage.cpp index bf11171930..243849b81e 100644 --- a/cpp/arcticdb/storage/memory/memory_storage.cpp +++ b/cpp/arcticdb/storage/memory/memory_storage.cpp @@ -33,14 +33,14 @@ namespace arcticdb::storage::memory { key_vec.erase(it); } - key_vec.try_emplace(key, kv.segment()); + key_vec.try_emplace(key, std::move(kv.segment())); }, [&](const AtomKey &key) { if (key_vec.find(key) != key_vec.end()) { throw DuplicateKeyException(key); } - key_vec.try_emplace(key, kv.segment()); + key_vec.try_emplace(key, std::move(kv.segment())); } ); } @@ -66,7 +66,7 @@ namespace arcticdb::storage::memory { if(it != key_vec.end()) { key_vec.erase(it); } - key_vec.insert(std::make_pair(kv.variant_key(), kv.segment())); + key_vec.insert(std::make_pair(kv.variant_key(), kv.segment().clone())); } }); } @@ -82,8 +82,7 @@ namespace arcticdb::storage::memory { if(it != key_vec.end()) { ARCTICDB_DEBUG(log::storage(), "Read key {}: {}", variant_key_type(k), variant_key_view(k)); - auto seg = it->second; - visitor(k, std::move(seg)); + visitor(k, it->second.clone()); } else { throw KeyNotFoundException(std::move(ks)); } diff --git a/cpp/arcticdb/storage/memory/memory_storage.hpp b/cpp/arcticdb/storage/memory/memory_storage.hpp index a4b288bd62..48c36a5de0 100644 --- a/cpp/arcticdb/storage/memory/memory_storage.hpp +++ b/cpp/arcticdb/storage/memory/memory_storage.hpp @@ -14,6 +14,7 @@ #include #include #include +#include namespace arcticdb::storage::memory { diff --git a/cpp/arcticdb/storage/mongo/mongo_client.cpp b/cpp/arcticdb/storage/mongo/mongo_client.cpp index 608dd04b8a..b8ca904976 100644 --- a/cpp/arcticdb/storage/mongo/mongo_client.cpp +++ b/cpp/arcticdb/storage/mongo/mongo_client.cpp @@ -50,9 +50,9 @@ StreamId stream_id_from_document(DocType& doc, KeyType key_type) { template AtomKey atom_key_from_document(DocType &doc, KeyType key_type) { - auto index_type = IndexDescriptor::Type(doc["index_type"].get_int32().value); + auto index_type = IndexDescriptorImpl::Type(doc["index_type"].get_int32().value); IndexValue start_index, end_index; - if (index_type == IndexDescriptor::TIMESTAMP) { + if (index_type == IndexDescriptorImpl::Type::TIMESTAMP) { start_index = doc["start_time"].get_int64().value; end_index = doc["end_time"].get_int64().value; } else { @@ -117,7 +117,7 @@ void add_atom_key_values(bsoncxx::builder::basic::document& basic_builder, const auto index_type = arcticdb::stream::get_index_value_type(key); basic_builder.append(kvp("index_type", types::b_int32{static_cast(index_type)})); - if(index_type == IndexDescriptor::TIMESTAMP) { + if(index_type == IndexDescriptorImpl::Type::TIMESTAMP) { basic_builder.append(kvp("start_time", types::b_int64{int64_t(std::get(key.start_index()))})); basic_builder.append(kvp("end_time", types::b_int64{int64_t(std::get(key.end_index()))})); } else @@ -134,13 +134,12 @@ auto build_document(storage::KeySegmentPair &kv) { using builder::stream::document; const auto &key = kv.variant_key(); - const auto &segment = kv.segment(); - const auto hdr_size = segment.segment_header_bytes_size(); - const auto total_size = segment.total_segment_size(hdr_size); + auto &segment = kv.segment(); + const auto total_size = segment.calculate_size(); /*thread_local*/ std::vector buffer{}; buffer.resize(total_size); bsoncxx::types::b_binary data = {}; - kv.segment().write_to(buffer.data(), hdr_size); + kv.segment().write_to(buffer.data()); data.size = uint32_t(total_size); data.bytes = buffer.data(); diff --git a/cpp/arcticdb/storage/mongo/mongo_storage.hpp b/cpp/arcticdb/storage/mongo/mongo_storage.hpp index ced21689d8..e5279b0b46 100644 --- a/cpp/arcticdb/storage/mongo/mongo_storage.hpp +++ b/cpp/arcticdb/storage/mongo/mongo_storage.hpp @@ -12,6 +12,7 @@ #include #include #include +#include #include namespace arcticdb::storage::mongo { diff --git a/cpp/arcticdb/storage/rocksdb/rocksdb_storage.cpp b/cpp/arcticdb/storage/rocksdb/rocksdb_storage.cpp index 5a673f3fa7..aa5b6d60ea 100644 --- a/cpp/arcticdb/storage/rocksdb/rocksdb_storage.cpp +++ b/cpp/arcticdb/storage/rocksdb/rocksdb_storage.cpp @@ -20,6 +20,7 @@ #include #include #include +#include namespace arcticdb::storage::rocksdb { @@ -232,11 +233,10 @@ void RocksDBStorage::do_write_internal(Composite&& kvs) { auto k_str = to_serialized_key(kv.variant_key()); auto& seg = kv.segment(); - auto hdr_sz = seg.segment_header_bytes_size(); - auto total_sz = seg.total_segment_size(hdr_sz); + auto total_sz = seg.calculate_size(); std::string seg_data; seg_data.resize(total_sz); - seg.write_to(reinterpret_cast(seg_data.data()), hdr_sz); + seg.write_to(reinterpret_cast(seg_data.data())); auto allow_override = std::holds_alternative(kv.variant_key()); if (!allow_override && do_key_exists(kv.variant_key())) { throw DuplicateKeyException(kv.variant_key()); diff --git a/cpp/arcticdb/storage/rocksdb/rocksdb_storage.hpp b/cpp/arcticdb/storage/rocksdb/rocksdb_storage.hpp index b0c76460ef..4779dc1646 100644 --- a/cpp/arcticdb/storage/rocksdb/rocksdb_storage.hpp +++ b/cpp/arcticdb/storage/rocksdb/rocksdb_storage.hpp @@ -11,9 +11,10 @@ #include #include #include -#include +#include #include +#include #include namespace arcticdb::storage::rocksdb { diff --git a/cpp/arcticdb/storage/s3/detail-inl.hpp b/cpp/arcticdb/storage/s3/detail-inl.hpp index 04881c2e62..59a2854c47 100644 --- a/cpp/arcticdb/storage/s3/detail-inl.hpp +++ b/cpp/arcticdb/storage/s3/detail-inl.hpp @@ -288,10 +288,10 @@ namespace s3 { // 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 IndexDescriptor correctly + // TODO: Set the IndexDescriptorImpl correctly KeyDescriptor key_descriptor(prefix, - is_ref_key_class(key_type) ? IndexDescriptor::UNKNOWN - : IndexDescriptor::TIMESTAMP, + 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, diff --git a/cpp/arcticdb/storage/s3/s3_mock_client.cpp b/cpp/arcticdb/storage/s3/s3_mock_client.cpp index e0bff24f18..7eaf2f6834 100644 --- a/cpp/arcticdb/storage/s3/s3_mock_client.cpp +++ b/cpp/arcticdb/storage/s3/s3_mock_client.cpp @@ -76,7 +76,7 @@ S3Result MockS3Client::get_object( if (pos == s3_contents.end()){ return {not_found_error}; } - return {pos->second}; + return {pos->second.clone()}; } S3Result MockS3Client::put_object( diff --git a/cpp/arcticdb/storage/s3/s3_real_client.cpp b/cpp/arcticdb/storage/s3/s3_real_client.cpp index 37e18788e0..7a070027d9 100644 --- a/cpp/arcticdb/storage/s3/s3_real_client.cpp +++ b/cpp/arcticdb/storage/s3/s3_real_client.cpp @@ -95,7 +95,6 @@ struct S3StreamBuffer : public std::streambuf { } }; - struct S3IOStream : public std::iostream { S3StreamBuffer stream_buf_; @@ -142,16 +141,7 @@ S3Result RealS3Client::put_object( request.SetKey(s3_object_name.c_str()); ARCTICDB_RUNTIME_DEBUG(log::storage(), "Set s3 key {}", request.GetKey().c_str()); - std::shared_ptr tmp; - auto hdr_size = segment.segment_header_bytes_size(); - auto [dst, write_size] = segment.try_internal_write(tmp, hdr_size); - util::check(arcticdb::Segment::FIXED_HEADER_SIZE + hdr_size + segment.buffer().bytes() <= - write_size, - "Size disparity, fixed header size {} + variable header size {} + buffer size {} >= total size {}", - arcticdb::Segment::FIXED_HEADER_SIZE, - hdr_size, - segment.buffer().bytes(), - write_size); + auto [dst, write_size, buffer] = segment.serialize_header(); auto body = std::make_shared( reinterpret_cast(dst), write_size); util::check(body->good(), "Overflow of bufferstream with size {}", write_size); @@ -165,7 +155,7 @@ S3Result RealS3Client::put_object( } ARCTICDB_RUNTIME_DEBUG(log::storage(), "Wrote key '{}', with {} bytes of data", s3_object_name, - segment.total_segment_size(hdr_size)); + segment.size()); return {std::monostate()}; } diff --git a/cpp/arcticdb/storage/storage.hpp b/cpp/arcticdb/storage/storage.hpp index 9825d4415c..c09d8ffebe 100644 --- a/cpp/arcticdb/storage/storage.hpp +++ b/cpp/arcticdb/storage/storage.hpp @@ -129,7 +129,7 @@ class Storage { KeySegmentPair key_seg; const ReadVisitor& visitor = [&key_seg](const VariantKey & vk, Segment&& value) { key_seg.variant_key() = vk; - key_seg.segment() = value; + key_seg.segment() = std::move(value); }; read(std::forward(key), visitor, opts); diff --git a/cpp/arcticdb/storage/test/test_embedded.cpp b/cpp/arcticdb/storage/test/test_embedded.cpp index 60ae84f09c..afaaf0a06d 100644 --- a/cpp/arcticdb/storage/test/test_embedded.cpp +++ b/cpp/arcticdb/storage/test/test_embedded.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #ifdef ARCTICDB_INCLUDE_ROCKSDB #include @@ -96,9 +97,10 @@ TEST_P(SimpleTestSuite, Example) { std::unique_ptr storage = GetParam().new_backend(); ac::entity::AtomKey k = ac::entity::atom_key_builder().gen_id(1).build(NumericId{999}); - as::KeySegmentPair kv(k); - kv.segment().header().set_start_ts(1234); - kv.segment().set_buffer(std::make_shared()); + auto segment_in_memory = get_test_frame("symbol", {}, 10, 0).segment_; + auto codec_opts = proto::encoding::VariantCodec(); + auto segment = encode_dispatch(std::move(segment_in_memory), codec_opts, arcticdb::EncodingVersion::V2); + arcticdb::storage::KeySegmentPair kv(k, std::move(segment)); storage->write(std::move(kv)); @@ -110,10 +112,8 @@ TEST_P(SimpleTestSuite, Example) { res.segment() = std::move(seg); res.segment().force_own_buffer(); // necessary since the non-owning buffer won't survive the visit }, storage::ReadKeyOpts{}); - ASSERT_EQ(res.segment().header().start_ts(), 1234); res = storage->read(k, as::ReadKeyOpts{}); - ASSERT_EQ(res.segment().header().start_ts(), 1234); bool executed = false; storage->iterate_type(arcticdb::entity::KeyType::TABLE_DATA, @@ -123,9 +123,10 @@ TEST_P(SimpleTestSuite, Example) { }); ASSERT_TRUE(executed); - as::KeySegmentPair update_kv(k); - update_kv.segment().header().set_start_ts(4321); - update_kv.segment().set_buffer(std::make_shared()); + segment_in_memory = get_test_frame("symbol", {}, 10, 0).segment_; + codec_opts = proto::encoding::VariantCodec(); + segment = encode_dispatch(std::move(segment_in_memory), codec_opts, arcticdb::EncodingVersion::V2); + arcticdb::storage::KeySegmentPair update_kv(k, std::move(segment)); storage->update(std::move(update_kv), as::UpdateOpts{}); @@ -135,10 +136,8 @@ TEST_P(SimpleTestSuite, Example) { update_res.segment() = std::move(seg); update_res.segment().force_own_buffer(); // necessary since the non-owning buffer won't survive the visit }, as::ReadKeyOpts{}); - ASSERT_EQ(update_res.segment().header().start_ts(), 4321); update_res = storage->read(k, as::ReadKeyOpts{}); - ASSERT_EQ(update_res.segment().header().start_ts(), 4321); executed = false; storage->iterate_type(arcticdb::entity::KeyType::TABLE_DATA, @@ -167,8 +166,8 @@ TEST_P(SimpleTestSuite, Strings) { google::protobuf::Any any; arcticdb::TimeseriesDescriptor metadata; - metadata.mutable_proto().set_total_rows(12); - metadata.mutable_proto().mutable_stream_descriptor()->CopyFrom(s.descriptor().proto()); + metadata.set_total_rows(12); + metadata.set_stream_descriptor(s.descriptor()); any.PackFrom(metadata.proto()); s.set_metadata(std::move(any)); @@ -185,7 +184,6 @@ TEST_P(SimpleTestSuite, Strings) { ac::entity::AtomKey k = ac::entity::atom_key_builder().gen_id(1).build(NumericId{999}); auto save_k = k; as::KeySegmentPair kv(std::move(k), std::move(seg)); - kv.segment().header().set_start_ts(1234); storage->write(std::move(kv)); as::KeySegmentPair res; @@ -194,7 +192,6 @@ TEST_P(SimpleTestSuite, Strings) { res.segment() = std::move(seg); res.segment().force_own_buffer(); // necessary since the non-owning buffer won't survive the visit }, as::ReadKeyOpts{}); - ASSERT_EQ(res.segment().header().start_ts(), 1234); SegmentInMemory res_mem = decode_segment(std::move(res.segment())); ASSERT_EQ(s.string_at(0, 1), res_mem.string_at(0, 1)); diff --git a/cpp/arcticdb/storage/test/test_mongo_storage.cpp b/cpp/arcticdb/storage/test/test_mongo_storage.cpp index 87f560d180..3827fc0020 100644 --- a/cpp/arcticdb/storage/test/test_mongo_storage.cpp +++ b/cpp/arcticdb/storage/test/test_mongo_storage.cpp @@ -36,7 +36,6 @@ TEST(MongoStorage, ClientSession) { ac::entity::AtomKey k = ac::entity::atom_key_builder().gen_id(1).build("999"); as::KeySegmentPair kv(k); - kv.segment().header().set_start_ts(1234); storage.write(std::move(kv)); @@ -47,10 +46,8 @@ TEST(MongoStorage, ClientSession) { res.segment() = std::move(seg); res.segment().force_own_buffer(); // necessary since the non-owning buffer won't survive the visit }, as::ReadKeyOpts{}); - ASSERT_EQ(res.segment().header().start_ts(), 1234); res = storage.read(k, as::ReadKeyOpts{}); - ASSERT_EQ(res.segment().header().start_ts(), 1234); bool executed = false; storage.iterate_type(ac::entity::KeyType::TABLE_DATA, @@ -67,7 +64,6 @@ TEST(MongoStorage, ClientSession) { ASSERT_TRUE(executed); as::KeySegmentPair update_kv(k); - update_kv.segment().header().set_start_ts(4321); storage.update(std::move(update_kv), as::UpdateOpts{}); @@ -78,10 +74,8 @@ TEST(MongoStorage, ClientSession) { update_res.segment() = std::move(seg); update_res.segment().force_own_buffer(); // necessary since the non-owning buffer won't survive the visit }, as::ReadKeyOpts{}); - ASSERT_EQ(update_res.segment().header().start_ts(), 4321); update_res = storage.read(k, as::ReadKeyOpts{}); - ASSERT_EQ(update_res.segment().header().start_ts(), 4321); executed = false; storage.iterate_type(ac::entity::KeyType::TABLE_DATA, @@ -91,9 +85,8 @@ TEST(MongoStorage, ClientSession) { }); ASSERT_TRUE(executed); - ac::entity::AtomKey numeric_k = ac::entity::atom_key_builder().gen_id(1).build(ac::entity::NumericId{999}); + ac::entity::AtomKey numeric_k = ac::entity::atom_key_builder().gen_id(1).build(ac::NumericId{999}); as::KeySegmentPair numeric_kv(numeric_k); - numeric_kv.segment().header().set_start_ts(7890); storage.write(std::move(numeric_kv)); @@ -104,8 +97,6 @@ TEST(MongoStorage, ClientSession) { numeric_res.segment() = std::move(seg); numeric_res.segment().force_own_buffer(); // necessary since the non-owning buffer won't survive the visit }, as::ReadKeyOpts{}); - ASSERT_EQ(numeric_res.segment().header().start_ts(), 7890); numeric_res = storage.read(numeric_k, as::ReadKeyOpts{}); - ASSERT_EQ(numeric_res.segment().header().start_ts(), 7890); } diff --git a/cpp/arcticdb/storage/test/test_storage_exceptions.cpp b/cpp/arcticdb/storage/test/test_storage_exceptions.cpp index a7d422ec68..19c8a314ad 100644 --- a/cpp/arcticdb/storage/test/test_storage_exceptions.cpp +++ b/cpp/arcticdb/storage/test/test_storage_exceptions.cpp @@ -250,9 +250,11 @@ TEST_F(LMDBStorageTestBase, WriteMapFullError) { auto storage = factory.create(); arcticdb::entity::AtomKey k = arcticdb::entity::atom_key_builder().gen_id(0).build("sym"); - arcticdb::storage::KeySegmentPair kv(k); - kv.segment().header().set_start_ts(1234); - kv.segment().set_buffer(std::make_shared(40000)); + + auto segment_in_memory = get_test_frame("symbol", {}, 40000, 0).segment_; + auto codec_opts = proto::encoding::VariantCodec(); + auto segment = encode_dispatch(std::move(segment_in_memory), codec_opts, arcticdb::EncodingVersion::V2); + arcticdb::storage::KeySegmentPair kv(k, std::move(segment)); ASSERT_THROW({ storage->write(std::move(kv)); diff --git a/cpp/arcticdb/stream/aggregator-inl.hpp b/cpp/arcticdb/stream/aggregator-inl.hpp index d2f9e7a04b..9501b2cafb 100644 --- a/cpp/arcticdb/stream/aggregator-inl.hpp +++ b/cpp/arcticdb/stream/aggregator-inl.hpp @@ -32,8 +32,7 @@ inline void Aggregator::commit_i template inline void Aggregator::commit() { - if (ARCTICDB_LIKELY(segment_.row_count() > 0 || segment_.metadata())) { // LIKELY -// segment_.end_sparse_columns(); + if (ARCTICDB_LIKELY(segment_.row_count() > 0 || segment_.metadata()) || segment_.has_index_descriptor()) { // LIKELY commit_impl(); } } diff --git a/cpp/arcticdb/stream/append_map.cpp b/cpp/arcticdb/stream/append_map.cpp index d971499cca..f06a1e10bc 100644 --- a/cpp/arcticdb/stream/append_map.cpp +++ b/cpp/arcticdb/stream/append_map.cpp @@ -70,12 +70,6 @@ std::vector get_incomplete_append_slices_for_stream_id( bool via_iteration, bool load_data); -inline bool has_appends_key( - const std::shared_ptr& store, - const RefKey& ref_key) { - return store->key_exists(ref_key).get(); -} - inline std::vector load_via_iteration( const std::shared_ptr& store, const StreamId& stream_id, @@ -141,11 +135,7 @@ TimeseriesDescriptor pack_timeseries_descriptor( size_t total_rows, std::optional&& next_key, arcticdb::proto::descriptors::NormalizationMetadata&& norm_meta) { - util::check(descriptor.proto().has_index(), "Stream descriptor without index in pack_timeseries_descriptor"); auto tsd = make_timeseries_descriptor(total_rows, std::move(descriptor), std::move(norm_meta), std::nullopt, std::nullopt, std::move(next_key), false); - if(ConfigsMap::instance()->get_int("VersionStore.Encoding", 1) == 1) { - tsd.copy_to_self_proto(); - } return tsd; } @@ -173,10 +163,10 @@ SegmentInMemory incomplete_segment_from_frame( auto timeseries_desc = index_descriptor_from_frame(frame, existing_rows, std::move(prev_key)); util::check(!timeseries_desc.fields().empty(), "Expected fields not to be empty in incomplete segment"); auto norm_meta = timeseries_desc.proto().normalization(); - StreamDescriptor descriptor(std::make_shared(std::move(*timeseries_desc.mutable_proto().mutable_stream_descriptor())), timeseries_desc.fields_ptr()); + auto descriptor = timeseries_desc.as_stream_descriptor(); SingleSegmentAggregator agg{FixedSchema{descriptor, index}, [&](auto&& segment) { auto tsd = pack_timeseries_descriptor(std::move(descriptor), existing_rows + num_rows, std::move(prev_key), std::move(norm_meta)); - segment.set_timeseries_descriptor(std::move(tsd)); + segment.set_timeseries_descriptor(tsd); output = std::forward(segment); }}; @@ -279,8 +269,8 @@ void write_head(const std::shared_ptr& store, const AtomKey& next_key, si auto desc = stream_descriptor(next_key.id(), RowCountIndex{}, {}); SegmentInMemory segment(desc); auto tsd = pack_timeseries_descriptor(std::move(desc), total_rows, next_key, {}); - segment.set_timeseries_descriptor(std::move(tsd)); - store->write(KeyType::APPEND_REF, next_key.id(), std::move(segment)).get(); + segment.set_timeseries_descriptor(tsd); + store->write_sync(KeyType::APPEND_REF, next_key.id(), std::move(segment)); } void remove_incomplete_segments( @@ -316,18 +306,17 @@ std::vector load_via_list( std::pair, size_t> read_head(const std::shared_ptr& store, StreamId stream_id) { auto ref_key = RefKey{std::move(stream_id), KeyType::APPEND_REF}; auto output = std::make_pair, size_t>(std::nullopt, 0); - - if(!has_appends_key(store, ref_key)) - return output; - - auto fut = store->read(ref_key); - auto [key, seg] = std::move(fut).get(); - const auto& tsd = seg.index_descriptor(); - if(tsd.proto().has_next_key()) { - output.first = decode_key(tsd.proto().next_key()); + try { + auto [key, seg] = store->read_sync(ref_key); + const auto &tsd = seg.index_descriptor(); + if (tsd.proto().has_next_key()) + output.first = decode_key(tsd.proto().next_key()); + + output.second = tsd.total_rows(); + } catch (storage::KeyNotFoundException& ex) { + ARCTICDB_RUNTIME_DEBUG(log::version(), "Failed to get head of append list for {}: {}", ref_key, ex.what()); } - output.second = tsd.proto().total_rows(); return output; } @@ -338,18 +327,18 @@ std::pair> get_descriptor_a storage::ReadKeyOpts opts = storage::ReadKeyOpts{}) { if(load_data) { auto [key, seg] = store->read_sync(k, opts); - return std::make_pair(TimeseriesDescriptor{seg.timeseries_proto(), seg.index_fields()}, std::make_optional(seg)); + return std::make_pair(seg.index_descriptor(), std::make_optional(seg)); } else { auto [key, tsd] = store->read_timeseries_descriptor(k, opts).get(); return std::make_pair(std::move(tsd), std::nullopt); } } -AppendMapEntry create_entry(const arcticdb::proto::descriptors::TimeSeriesDescriptor& tsd) { +AppendMapEntry create_entry(const TimeseriesDescriptor& tsd) { AppendMapEntry entry; - if(tsd.has_next_key()) - entry.next_key_ = decode_key(tsd.next_key()); + if(tsd.proto().has_next_key()) + entry.next_key_ = decode_key(tsd.proto().next_key()); entry.total_rows_ = tsd.total_rows(); return entry; @@ -359,7 +348,7 @@ AppendMapEntry entry_from_key(const std::shared_ptr& store, const auto opts = storage::ReadKeyOpts{}; opts.dont_warn_about_missing_key = true; auto [tsd, seg] = get_descriptor_and_data(store, key, load_data, opts); - auto entry = create_entry(tsd.proto()); + auto entry = create_entry(tsd); auto descriptor = std::make_shared(); auto desc = std::make_shared(tsd.as_stream_descriptor()); auto index_field_count = desc->index().field_count(); @@ -408,8 +397,7 @@ void append_incomplete_segment( auto seg_row_count = seg.row_count(); auto tsd = pack_timeseries_descriptor(seg.descriptor().clone(), seg_row_count, std::move(next_key), {}); - seg.set_timeseries_descriptor(std::move(tsd)); - util::check(static_cast(seg.metadata()), "Expected metadata"); + seg.set_timeseries_descriptor(tsd); auto new_key = store->write( arcticdb::stream::KeyType::APPEND_DATA, 0, @@ -440,7 +428,7 @@ std::vector get_incomplete_append_slices_for_stream_id( if(!entries.empty()) { auto index_desc = entries[0].descriptor().index(); - if (index_desc.type() != IndexDescriptor::ROWCOUNT) { + if (index_desc.type() != IndexDescriptorImpl::Type::ROWCOUNT) { std::sort(std::begin(entries), std::end(entries)); } else { // Can't sensibly sort rowcount indexes, so you'd better have written them in the right order diff --git a/cpp/arcticdb/stream/index.hpp b/cpp/arcticdb/stream/index.hpp index 9edf08a60b..bc6115ea9a 100644 --- a/cpp/arcticdb/stream/index.hpp +++ b/cpp/arcticdb/stream/index.hpp @@ -22,8 +22,8 @@ namespace arcticdb::stream { using namespace arcticdb::entity; -inline IndexDescriptor::Type get_index_value_type(const AtomKey &key) { - return std::holds_alternative(key.start_index()) ? IndexDescriptor::TIMESTAMP : IndexDescriptor::STRING; +inline IndexDescriptorImpl::Type get_index_value_type(const AtomKey &key) { + return std::holds_alternative(key.start_index()) ? IndexDescriptorImpl::Type::TIMESTAMP : IndexDescriptorImpl::Type::STRING; } template @@ -31,7 +31,7 @@ class BaseIndex { public: template StreamDescriptor create_stream_descriptor(StreamId stream_id, RangeType&& fields) const { - return stream_descriptor(stream_id, *derived(), std::move(fields)); + return stream_descriptor_from_range(stream_id, *derived(), std::move(fields)); } [[nodiscard]] StreamDescriptor create_stream_descriptor( @@ -46,7 +46,7 @@ class BaseIndex { return static_cast(this); } - explicit operator IndexDescriptor() const { + explicit operator IndexDescriptorImpl() const { return {Derived::field_count(), Derived::type()}; } @@ -76,8 +76,8 @@ class TimeseriesIndex : public BaseIndex { return 1; } - static constexpr IndexDescriptor::Type type() { - return IndexDescriptor::TIMESTAMP; + static constexpr IndexDescriptorImpl::Type type() { + return IndexDescriptorImpl::Type::TIMESTAMP; } void check(const FieldCollection &fields) const { @@ -179,8 +179,8 @@ class TableIndex : public BaseIndex { return 1; } - static constexpr IndexDescriptor::Type type() { - return IndexDescriptor::STRING; + static constexpr IndexDescriptorImpl::Type type() { + return IndexDescriptorImpl::Type::STRING; } void check(const FieldCollection &fields) const { @@ -258,7 +258,7 @@ class RowCountIndex : public BaseIndex { static constexpr size_t field_count() { return 0; } - static constexpr IndexDescriptor::Type type() { return IndexDescriptor::ROWCOUNT; } + static constexpr IndexDescriptorImpl::Type type() { return IndexDescriptorImpl::Type::ROWCOUNT; } void check(const FieldCollection& ) const { // No index defined @@ -289,7 +289,7 @@ class RowCountIndex : public BaseIndex { // No index value } - RowCountIndex make_from_descriptor(const StreamDescriptor&) const { + [[nodiscard]] RowCountIndex make_from_descriptor(const StreamDescriptor&) const { return RowCountIndex::default_index(); } @@ -299,51 +299,48 @@ class RowCountIndex : public BaseIndex { using Index = std::variant; inline Index index_type_from_descriptor(const StreamDescriptor &desc) { - switch (desc.index().proto().kind()) { - case IndexDescriptor::TIMESTAMP: + switch (desc.index().type()) { + case IndexDescriptorImpl::Type::TIMESTAMP: return TimeseriesIndex::make_from_descriptor(desc); - case IndexDescriptor::STRING: + case IndexDescriptorImpl::Type::STRING: return TableIndex::make_from_descriptor(desc); - case IndexDescriptor::ROWCOUNT: + case IndexDescriptorImpl::Type::ROWCOUNT: return RowCountIndex{}; - default:util::raise_rte("Data obtained from storage refers to an index type that this build of ArcticDB doesn't understand ({}).", int(desc.index().proto().kind())); + default: + util::raise_rte("Data obtained from storage refers to an index type that this build of ArcticDB doesn't understand ({}).", int(desc.index().type())); } } -inline Index default_index_type_from_descriptor(const IndexDescriptor::Proto &desc) { - switch (desc.kind()) { - case IndexDescriptor::TIMESTAMP: +inline Index default_index_type_from_descriptor(const IndexDescriptorImpl &desc) { + switch (desc.type()) { + case IndexDescriptorImpl::Type::TIMESTAMP: return TimeseriesIndex::default_index(); - case IndexDescriptor::STRING: + case IndexDescriptorImpl::Type::STRING: return TableIndex::default_index(); - case IndexDescriptor::ROWCOUNT: + case IndexDescriptorImpl::Type::ROWCOUNT: return RowCountIndex::default_index(); default: - util::raise_rte("Unknown index type {} trying to generate index type", int(desc.kind())); + util::raise_rte("Unknown index type {} trying to generate index type", int(desc.type())); } } // Only to be used for visitation to get field count etc as the name is not set -inline Index variant_index_from_type(IndexDescriptor::Type type) { +inline Index variant_index_from_type(IndexDescriptorImpl::Type type) { switch (type) { - case IndexDescriptor::TIMESTAMP: + case IndexDescriptorImpl::Type::TIMESTAMP: return TimeseriesIndex{TimeseriesIndex::DefaultName}; - case IndexDescriptor::STRING: + case IndexDescriptorImpl::Type::STRING: return TableIndex{TableIndex::DefaultName}; - case IndexDescriptor::ROWCOUNT: + case IndexDescriptorImpl::Type::ROWCOUNT: return RowCountIndex{}; default: util::raise_rte("Unknown index type {} trying to generate index type", int(type)); } } -inline Index default_index_type_from_descriptor(const IndexDescriptor &desc) { - return default_index_type_from_descriptor(desc.proto()); -} - -inline IndexDescriptor get_descriptor_from_index(const Index& index) { +inline IndexDescriptorImpl get_descriptor_from_index(const Index& index) { return util::variant_match(index, [] (const auto& idx) { - return static_cast(idx); + return static_cast(idx); }); } diff --git a/cpp/arcticdb/stream/merge.hpp b/cpp/arcticdb/stream/merge.hpp index 32b19fd658..61a4ebeec7 100644 --- a/cpp/arcticdb/stream/merge.hpp +++ b/cpp/arcticdb/stream/merge.hpp @@ -19,7 +19,7 @@ void do_merge( while (!input_streams.empty()) { auto next = input_streams.pop_top(); - agg.start_row(pipelines::index::index_value_from_row(next->row(), IndexDescriptor::TIMESTAMP, 0).value()) ([&next, add_symbol_column](auto &rb) { + agg.start_row(pipelines::index::index_value_from_row(next->row(), IndexDescriptorImpl::Type::TIMESTAMP, 0).value()) ([&next, add_symbol_column](auto &rb) { if(add_symbol_column) rb.set_scalar_by_name("symbol", std::string_view(std::get(next->id())), DataType::UTF_DYNAMIC64); diff --git a/cpp/arcticdb/stream/merge_utils.hpp b/cpp/arcticdb/stream/merge_utils.hpp index 0e5c3ce338..d9efbbede7 100644 --- a/cpp/arcticdb/stream/merge_utils.hpp +++ b/cpp/arcticdb/stream/merge_utils.hpp @@ -78,10 +78,12 @@ inline void merge_segments( } } } - if (segment.row_count() && segment.descriptor().index().type() == IndexDescriptor::TIMESTAMP) { + + if (segment.row_count() && segment.descriptor().index().type() == IndexDescriptorImpl::Type::TIMESTAMP) { min_idx = std::min(min_idx, segment.begin()->begin()->value()); max_idx = std::max(max_idx, (segment.end() - 1)->begin()->value()); } + merge_string_columns(segment, merged.string_pool_ptr(), false); merged.append(segment); merged.set_compacted(true); diff --git a/cpp/arcticdb/stream/protobuf_mappings.cpp b/cpp/arcticdb/stream/protobuf_mappings.cpp new file mode 100644 index 0000000000..54304ec827 --- /dev/null +++ b/cpp/arcticdb/stream/protobuf_mappings.cpp @@ -0,0 +1,83 @@ +/* Copyright 2023 Man Group Operations Limited + * + * Use of this software is governed by the Business Source License 1.1 included in the file licenses/BSL.txt. + * + * As of the Change Date specified in that file, in accordance with the Business Source License, use of this software will be governed by the Apache License, version 2.0. + */ + +#include +#include +#include + +#include +#include + +namespace arcticdb { + +struct FrameDescriptorImpl; + +arcticdb::proto::descriptors::NormalizationMetadata make_timeseries_norm_meta(const StreamId& stream_id) { + using namespace arcticdb::proto::descriptors; + NormalizationMetadata norm_meta; + NormalizationMetadata_PandasDataFrame pandas; + auto id = std::get(stream_id); + pandas.mutable_common()->set_name(std::move(id)); + NormalizationMetadata_PandasIndex pandas_index; + pandas_index.set_name("time"); + pandas.mutable_common()->mutable_index()->CopyFrom(pandas_index); + norm_meta.mutable_df()->CopyFrom(pandas); + return norm_meta; +} + +arcticdb::proto::descriptors::NormalizationMetadata make_rowcount_norm_meta(const StreamId& stream_id) { + using namespace arcticdb::proto::descriptors; + NormalizationMetadata norm_meta; + NormalizationMetadata_PandasDataFrame pandas; + auto id = std::get(stream_id); + pandas.mutable_common()->set_name(std::move(id)); + NormalizationMetadata_PandasIndex pandas_index; + pandas_index.set_is_not_range_index(true); + pandas.mutable_common()->mutable_index()->CopyFrom(pandas_index); + norm_meta.mutable_df()->CopyFrom(pandas); + return norm_meta; +} + +/** + * Set the minimum defaults into norm_meta. Originally created to synthesize norm_meta for incomplete compaction. + */ +void ensure_timeseries_norm_meta(arcticdb::proto::descriptors::NormalizationMetadata& norm_meta, const StreamId& stream_id, bool set_tz) { + if(norm_meta.input_type_case() == arcticdb::proto::descriptors::NormalizationMetadata::INPUT_TYPE_NOT_SET) { + norm_meta.CopyFrom(make_timeseries_norm_meta(stream_id)); + } + + if(set_tz && norm_meta.df().common().index().tz().empty()) + norm_meta.mutable_df()->mutable_common()->mutable_index()->set_tz("UTC"); +} + +void ensure_rowcount_norm_meta(arcticdb::proto::descriptors::NormalizationMetadata& norm_meta, const StreamId& stream_id) { + if(norm_meta.input_type_case() == arcticdb::proto::descriptors::NormalizationMetadata::INPUT_TYPE_NOT_SET) { + norm_meta.CopyFrom(make_rowcount_norm_meta(stream_id)); + } +} + +FrameDescriptorImpl frame_descriptor_from_proto(arcticdb::proto::descriptors::TimeSeriesDescriptor& tsd) { + FrameDescriptorImpl output; + output.column_groups_ = tsd.has_column_groups() && tsd.column_groups().enabled(); + output.total_rows_ = tsd.total_rows(); + return output; +} + +SegmentDescriptorImpl segment_descriptor_from_proto(const arcticdb::proto::descriptors::StreamDescriptor& desc) { + SegmentDescriptorImpl output; + output.sorted_ = SortedValue(desc.sorted()); + output.compressed_bytes_ = desc.out_bytes(); + output.uncompressed_bytes_ = desc.in_bytes(); + output.index_ = IndexDescriptor(IndexDescriptor::Type(desc.index().kind()), desc.index().field_count()); + return output; +} + +StreamId stream_id_from_proto(const arcticdb::proto::descriptors::StreamDescriptor& desc) { + return desc.id_case() == desc.kNumId ? StreamId(desc.num_id()) : StreamId(desc.str_id()); +} + +} //namespace arcticdb \ No newline at end of file diff --git a/cpp/arcticdb/stream/protobuf_mappings.hpp b/cpp/arcticdb/stream/protobuf_mappings.hpp index 9fcb0fb567..33f544ec26 100644 --- a/cpp/arcticdb/stream/protobuf_mappings.hpp +++ b/cpp/arcticdb/stream/protobuf_mappings.hpp @@ -8,56 +8,32 @@ #pragma once #include -#include - -#include +#include #include namespace arcticdb { -inline arcticdb::proto::descriptors::NormalizationMetadata make_timeseries_norm_meta(const entity::StreamId& stream_id) { - using namespace arcticdb::proto::descriptors; - NormalizationMetadata norm_meta; - NormalizationMetadata_PandasDataFrame pandas; - auto id = std::get(stream_id); - pandas.mutable_common()->set_name(std::move(id)); - NormalizationMetadata_PandasIndex pandas_index; - pandas_index.set_name("time"); - pandas.mutable_common()->mutable_index()->CopyFrom(pandas_index); - norm_meta.mutable_df()->CopyFrom(pandas); - return norm_meta; -} +struct FrameDescriptorImpl; -inline arcticdb::proto::descriptors::NormalizationMetadata make_rowcount_norm_meta(const entity::StreamId& stream_id) { - using namespace arcticdb::proto::descriptors; - NormalizationMetadata norm_meta; - NormalizationMetadata_PandasDataFrame pandas; - auto id = std::get(stream_id); - pandas.mutable_common()->set_name(std::move(id)); - NormalizationMetadata_PandasIndex pandas_index; - pandas_index.set_is_not_range_index(true); - pandas.mutable_common()->mutable_index()->CopyFrom(pandas_index); - norm_meta.mutable_df()->CopyFrom(pandas); - return norm_meta; +namespace entity { +struct SegmentDescriptorImpl; } -/** - * Set the minimum defaults into norm_meta. Originally created to synthesize norm_meta for incomplete compaction. - */ -inline void ensure_timeseries_norm_meta(arcticdb::proto::descriptors::NormalizationMetadata& norm_meta, const entity::StreamId& stream_id, bool set_tz) { - if(norm_meta.input_type_case() == arcticdb::proto::descriptors::NormalizationMetadata::INPUT_TYPE_NOT_SET) { - norm_meta.CopyFrom(make_timeseries_norm_meta(stream_id)); - } +arcticdb::proto::descriptors::NormalizationMetadata make_timeseries_norm_meta(const StreamId& stream_id); - if(set_tz && norm_meta.df().common().index().tz().empty()) - norm_meta.mutable_df()->mutable_common()->mutable_index()->set_tz("UTC"); -} +arcticdb::proto::descriptors::NormalizationMetadata make_rowcount_norm_meta(const StreamId& stream_id); -inline void ensure_rowcount_norm_meta(arcticdb::proto::descriptors::NormalizationMetadata& norm_meta, const entity::StreamId& stream_id) { - if(norm_meta.input_type_case() == arcticdb::proto::descriptors::NormalizationMetadata::INPUT_TYPE_NOT_SET) { - norm_meta.CopyFrom(make_rowcount_norm_meta(stream_id)); - } -} +void ensure_timeseries_norm_meta(arcticdb::proto::descriptors::NormalizationMetadata& norm_meta, const StreamId& stream_id, bool set_tz); + +void ensure_rowcount_norm_meta(arcticdb::proto::descriptors::NormalizationMetadata& norm_meta, const StreamId& stream_id); + +FrameDescriptorImpl frame_descriptor_from_proto(arcticdb::proto::descriptors::TimeSeriesDescriptor& tsd); + +entity::SegmentDescriptorImpl segment_descriptor_from_proto(const arcticdb::proto::descriptors::StreamDescriptor& desc); + +StreamId stream_id_from_proto(const arcticdb::proto::descriptors::StreamDescriptor& desc); + +size_t num_blocks(const arcticdb::proto::encoding::EncodedField& field); } //namespace arcticdb \ No newline at end of file diff --git a/cpp/arcticdb/stream/python_bindings.cpp b/cpp/arcticdb/stream/python_bindings.cpp index 02cd52cfd1..f7b576a4ea 100644 --- a/cpp/arcticdb/stream/python_bindings.cpp +++ b/cpp/arcticdb/stream/python_bindings.cpp @@ -24,10 +24,10 @@ namespace py = pybind11; namespace arcticdb { using namespace arcticdb::python_util; -std::vector field_collection_to_ref_vector(const FieldCollection& fields){ - auto result = std::vector(); +std::vector field_collection_to_ref_vector(const FieldCollection& fields){ + auto result = std::vector{}; result.reserve(fields.size()); - std::transform(fields.begin(), fields.end(), std::back_inserter(result), [](const Field& field){return field.ref();}); + std::transform(fields.begin(), fields.end(), std::back_inserter(result), [](const Field& field){return FieldWrapper{field.type(), field.name()};}); return result; } @@ -48,9 +48,6 @@ void register_types(py::module &m) { DATA_TYPE(NANOSECONDS_UTC64) DATA_TYPE(ASCII_FIXED64) DATA_TYPE(ASCII_DYNAMIC64) - //DATA_TYPE(UTF8_STRING) - // DATA_TYPE(BYTES) - // DATA_TYPE(PICKLE) #undef DATA_TYPE ; @@ -62,46 +59,54 @@ void register_types(py::module &m) { m.def("as_dim_checked", &as_dim_checked, "Turns a uint8_t into a Dimension enum object"); python_util::add_repr(py::class_(m, "TypeDescriptor") - .def(py::init()) - .def("data_type", &TypeDescriptor::data_type) - .def("dimension", &TypeDescriptor::dimension) - ); - //TODO re-add this constructor + .def(py::init()) + .def("data_type", &TypeDescriptor::data_type) + .def("dimension", &TypeDescriptor::dimension)); + python_util::add_repr(py::class_(m, "FieldDescriptor") - .def(py::init()) - .def("type", &FieldRef::type) - .def("name", &FieldRef::name) - ); + .def(py::init()) + .def_property_readonly("type", &FieldRef::type) + .def_property_readonly("name", &FieldRef::name)); - py::enum_(m, "IndexKind") - .value("TIMESTAMP", IndexDescriptor::TIMESTAMP) - .value("STRING", IndexDescriptor::STRING) - .value("ROWCOUNT", IndexDescriptor::ROWCOUNT); + python_util::add_repr(py::class_(m, "FieldDescriptorWrapper") + .def_property_readonly("type", &FieldWrapper::type) + .def_property_readonly("name", &FieldWrapper::name)); - python_util::add_repr(py::class_(m, "IndexDescriptor") - .def(py::init()) - .def("field_count", &IndexDescriptor::field_count) - .def("kind", &IndexDescriptor::type) - ); + py::enum_(m, "IndexKind") + .value("TIMESTAMP", IndexDescriptorImpl::Type::TIMESTAMP) + .value("STRING", IndexDescriptorImpl::Type::STRING) + .value("ROWCOUNT", IndexDescriptorImpl::Type::ROWCOUNT); + + python_util::add_repr(py::class_(m, "IndexDescriptor") + .def(py::init()) + .def("field_count", &IndexDescriptorImpl::field_count) + .def("kind", &IndexDescriptorImpl::type)); python_util::add_repr(py::class_(m, "StreamDescriptor") - .def(py::init([](StreamId stream_id, IndexDescriptor idx_desc, const std::vector& fields) { - auto index = stream::default_index_type_from_descriptor(idx_desc.proto()); - return util::variant_match(index, [&stream_id, &fields] (auto idx_type){ - return StreamDescriptor{index_descriptor(stream_id, idx_type, fields_from_range(fields))}; - }); - })) - .def("id", &StreamDescriptor::id) - .def("fields", [](const StreamDescriptor& desc){ - return field_collection_to_ref_vector(desc.fields()); - }) + .def(py::init([](StreamId stream_id, IndexDescriptorImpl idx_desc, const std::vector& fields) { + auto index = stream::default_index_type_from_descriptor(idx_desc); + return util::variant_match(index, [&stream_id, &fields] (auto idx_type){ + return StreamDescriptor{index_descriptor_from_range(stream_id, idx_type, fields_from_range(fields))}; + }); + })) + .def("id", &StreamDescriptor::id) + .def("fields", [](const StreamDescriptor& desc){ + return field_collection_to_ref_vector(desc.fields()); + }) ); - python_util::add_repr(py::class_(m, "TimeseriesDescriptor") - .def("fields", [](const TimeseriesDescriptor& desc){ - return field_collection_to_ref_vector(desc.fields()); - }) - ); + py::class_(m, "TimeseriesDescriptor") + .def_property_readonly("fields", [](const TimeseriesDescriptor& desc){ + return field_collection_to_ref_vector(desc.fields()); + }).def_property_readonly("normalization", [](const TimeseriesDescriptor& self) { + return python_util::pb_to_python(self.normalization()); + }).def_property_readonly("sorted", [](const TimeseriesDescriptor& self) { + return self.sorted(); + }).def_property_readonly("index", [](const TimeseriesDescriptor& self) { + return self.index(); + }).def_property_readonly("total_rows", [](const TimeseriesDescriptor& self) { + return self.total_rows(); + }); py::class_(m, "TimestampRange") .def(py::init()) @@ -111,8 +116,7 @@ void register_types(py::module &m) { .def_property_readonly("start_nanos_utc", &PyTimestampRange::start_nanos_utc) .def_property_readonly("end_nanos_utc", &PyTimestampRange::end_nanos_utc); - m.def("create_timestamp_index_stream_descriptor", [](StreamId tsid, - const std::vector& fields) { + m.def("create_timestamp_index_stream_descriptor", [](StreamId tsid, const std::vector& fields) { auto rg = folly::range(fields.begin(), fields.end()); const auto index = stream::TimeseriesIndex::default_index(); return index.create_stream_descriptor(tsid, fields_from_range(rg)); @@ -126,9 +130,7 @@ struct SegmentHolder { SegmentInMemory segment; }; - void register_stream_bindings(py::module &m) { - using Agg = FixedTimestampAggregator; using FixedTickRowBuilder = typename Agg::RowBuilderType; diff --git a/cpp/arcticdb/stream/row_builder.hpp b/cpp/arcticdb/stream/row_builder.hpp index c0c79f44c0..ade422f531 100644 --- a/cpp/arcticdb/stream/row_builder.hpp +++ b/cpp/arcticdb/stream/row_builder.hpp @@ -21,6 +21,7 @@ #include #include +#include #include diff --git a/cpp/arcticdb/stream/stream_reader.hpp b/cpp/arcticdb/stream/stream_reader.hpp index 7bdd41d0c8..7bc5bbc901 100644 --- a/cpp/arcticdb/stream/stream_reader.hpp +++ b/cpp/arcticdb/stream/stream_reader.hpp @@ -55,7 +55,7 @@ class RowsFromSegIterator : public IndexRangeFilter { // Not filtering rows where we have a rowcount index - the assumption is that it's essentially an un-indexed blob // that we need to segment somehow. - auto accept = index_type == IndexDescriptor::ROWCOUNT || accept_index(pipelines::index::index_start_from_row(res.value(), index_type).value()); + auto accept = index_type == IndexDescriptorImpl::Type::ROWCOUNT || accept_index(pipelines::index::index_start_from_row(res.value(), index_type).value()); if (++row_id == seg_->row_count()) { prev_seg_ = seg_; seg_ = std::nullopt; diff --git a/cpp/arcticdb/stream/stream_utils.hpp b/cpp/arcticdb/stream/stream_utils.hpp index 95b15a8f5b..cbde36ab49 100644 --- a/cpp/arcticdb/stream/stream_utils.hpp +++ b/cpp/arcticdb/stream/stream_utils.hpp @@ -368,9 +368,9 @@ inline std::set filter_by_regex(const std::set& results, con return filtered_results; } -inline std::vector get_index_columns_from_descriptor(const TimeseriesDescriptor& descriptor) { - const auto& norm_info = descriptor.proto().normalization(); - const auto& stream_descriptor = descriptor.proto().stream_descriptor(); +inline std::vector get_index_columns_from_descriptor(const TimeseriesDescriptor& tsd) { + const auto& norm_info = tsd.proto().normalization(); + const auto& stream_descriptor = tsd.as_stream_descriptor(); // For explicit integer indexes, the index is actually present in the first column even though the field_count // is 0. ssize_t index_till; @@ -380,7 +380,7 @@ inline std::vector get_index_columns_from_descriptor(const Timeseri else index_till = 1 + common.multi_index().field_count(); //# The value of field_count is len(index) - 1 - std::vector index_columns; + std::vector index_columns; for(auto field_idx = 0; field_idx < index_till; ++field_idx) index_columns.push_back(stream_descriptor.fields(field_idx).name()); diff --git a/cpp/arcticdb/stream/test/test_append_map.cpp b/cpp/arcticdb/stream/test/test_append_map.cpp index fa7deef7d6..1301076af3 100644 --- a/cpp/arcticdb/stream/test/test_append_map.cpp +++ b/cpp/arcticdb/stream/test/test_append_map.cpp @@ -43,7 +43,7 @@ TEST(Append, MergeDescriptorsPromote) { using namespace arcticdb; StreamId id{"test_desc"}; - IndexDescriptor idx{1u, IndexDescriptor::TIMESTAMP}; + IndexDescriptorImpl idx{1u, IndexDescriptorImpl::Type::TIMESTAMP}; std::vector fields { scalar_field(DataType::NANOSECONDS_UTC64, "time"), @@ -85,7 +85,7 @@ TEST(Append, MergeDescriptorsNoPromote) { using namespace arcticdb; StreamId id{"test_desc"}; - IndexDescriptor idx{1u, IndexDescriptor::TIMESTAMP}; + IndexDescriptorImpl idx{1u, IndexDescriptorImpl::Type::TIMESTAMP}; std::vector fields { scalar_field(DataType::NANOSECONDS_UTC64, "time"), diff --git a/cpp/arcticdb/toolbox/library_tool.cpp b/cpp/arcticdb/toolbox/library_tool.cpp index 0ff5573a7b..14ca2ceffd 100644 --- a/cpp/arcticdb/toolbox/library_tool.cpp +++ b/cpp/arcticdb/toolbox/library_tool.cpp @@ -39,7 +39,7 @@ Segment LibraryTool::read_to_segment(const VariantKey& key) { auto kv = store_->read_compressed_sync(key, storage::ReadKeyOpts{}); util::check(kv.has_segment(), "Failed to read key: {}", key); kv.segment().force_own_buffer(); - return kv.segment(); + return std::move(kv.segment()); } std::optional LibraryTool::read_metadata(const VariantKey& key){ @@ -55,7 +55,7 @@ TimeseriesDescriptor LibraryTool::read_timeseries_descriptor(const VariantKey& k return store_->read_timeseries_descriptor(key).get().second; } -void LibraryTool::write(VariantKey key, Segment segment) { +void LibraryTool::write(VariantKey key, Segment& segment) { storage::KeySegmentPair kv{std::move(key), std::move(segment)}; store_->write_compressed_sync(std::move(kv)); } diff --git a/cpp/arcticdb/toolbox/library_tool.hpp b/cpp/arcticdb/toolbox/library_tool.hpp index abc4867b4c..ef769cf62a 100644 --- a/cpp/arcticdb/toolbox/library_tool.hpp +++ b/cpp/arcticdb/toolbox/library_tool.hpp @@ -43,7 +43,7 @@ class LibraryTool { [[nodiscard]] TimeseriesDescriptor read_timeseries_descriptor(const VariantKey& key); - void write(VariantKey key, Segment segment); + void write(VariantKey key, Segment& segment); void remove(VariantKey key); @@ -53,7 +53,7 @@ class LibraryTool { std::string get_key_path(const VariantKey& key); - std::vector find_keys_for_id(entity::KeyType kt, const entity::StreamId &stream_id); + std::vector find_keys_for_id(entity::KeyType kt, const StreamId &stream_id); int count_keys(entity::KeyType kt); diff --git a/cpp/arcticdb/util/buffer.hpp b/cpp/arcticdb/util/buffer.hpp index 905856c22c..ca7c27df42 100644 --- a/cpp/arcticdb/util/buffer.hpp +++ b/cpp/arcticdb/util/buffer.hpp @@ -69,6 +69,10 @@ struct Buffer : public BaseBuffer { check_invariants(); } + static auto presized(size_t size) { + return Buffer(size); + }; + Buffer &operator=(Buffer &&b) noexcept { deallocate(); using std::swap; @@ -150,7 +154,7 @@ struct Buffer : public BaseBuffer { check_invariants(); if (bytes_offset + required_bytes > bytes()) { std::string err = fmt::format("Cursor overflow in reallocating buffer ptr_cast, cannot read {} bytes from a buffer of size {} with cursor " - "at {}, as it would required {} bytes. ", + "at {}, as it would require {} bytes. ", required_bytes, bytes(), bytes_offset, @@ -169,8 +173,7 @@ struct Buffer : public BaseBuffer { } inline void ensure(size_t bytes) { - const size_t total_size = bytes + preamble_bytes_; - if(total_size > capacity_) { + if(const size_t total_size = bytes + preamble_bytes_; total_size > capacity_) { resize(total_size); } else { ARCTICDB_TRACE(log::version(), "Buffer {} has sufficient bytes for {}, ptr {} data {}, capacity {}", @@ -259,7 +262,110 @@ struct Buffer : public BaseBuffer { entity::timestamp ts_ = 0; }; -using VariantBuffer = std::variant, BufferView>; +class VariantBuffer { + using VariantType = std::variant, BufferView>; + + VariantType buffer_; +public: + VariantBuffer() = default; + + template + VariantBuffer(BufferType&& buf) : + buffer_(std::forward(buf)) { + } + + [[nodiscard]] VariantBuffer clone() const { + return util::variant_match(buffer_, + [] (const BufferView& bv) { auto b = std::make_shared(); bv.copy_to(*b); return VariantBuffer{std::move(b)}; }, + [] (const std::shared_ptr& buf) { return VariantBuffer{ std::make_shared(buf->clone())}; }, + [] (const std::monostate) -> VariantBuffer { util::raise_rte("Uninitialized buffer"); } + ); + } + + template + VariantBuffer& operator=(BufferType&& buf) { + buffer_ = std::forward(buf); + return *this; + } + + [[nodiscard]] bool is_owning() const { + return std::holds_alternative>(buffer_); + } + + [[nodiscard]] const std::shared_ptr& get_owning_buffer() const { + return std::get>(buffer_); + } + + uint8_t* data() { + return util::variant_match(buffer_, + [] (BufferView& bv) { return bv.data(); }, + [] (const std::shared_ptr& buf) { return buf->data(); }, + [] (const std::monostate) ->uint8_t* { util::raise_rte("Uninitialized buffer"); } + ); + } + + [[nodiscard]] size_t preamble_bytes() const { + if (std::holds_alternative>(buffer_)) { + return std::get>(buffer_)->preamble_bytes(); + } else { + return 0U; + } + } + + [[nodiscard]] BufferView view() const { + if (std::holds_alternative>(buffer_)) { + return std::get>(buffer_)->view(); + } else { + return std::get(buffer_); + } + } + + [[nodiscard]] std::size_t bytes() const { + std::size_t s = 0; + util::variant_match(buffer_, + [] (const std::monostate&) { /* Uninitialized buffer */}, + [&s](const BufferView& b) { s = b.bytes(); }, + [&s](const std::shared_ptr& b) { s = b->bytes(); }); + + return s; + } + + [[nodiscard]] bool is_uninitialized() const { + return std::holds_alternative(buffer_); + } + + void move_buffer(VariantBuffer &&that) { + if(is_uninitialized() || that.is_uninitialized()) { + std::swap(buffer_, that.buffer_); + } else if (!(is_owning_buffer() ^ that.is_owning_buffer())) { + if (is_owning_buffer()) { + swap(*std::get>(buffer_), *std::get>(that.buffer_)); + } else { + swap(std::get(buffer_), std::get(that.buffer_)); + } + } else if (is_owning_buffer()) { + log::storage().info("Copying segment"); + // data of segment being moved is not owned, moving it is dangerous, copying instead + std::get(that.buffer_).copy_to(*std::get>(buffer_)); + } else { + // data of this segment is a view, but the move data is moved + buffer_ = std::move(std::get>(that.buffer_)); + } + } + + [[nodiscard]] bool is_owning_buffer() const { + return std::holds_alternative>(buffer_); + } + + void force_own_buffer() { + if (!is_owning_buffer()) { + auto b = std::make_shared(); + std::get(buffer_).copy_to(*b); + buffer_ = std::move(b); + } + } +}; + } // namespace arcticdb diff --git a/cpp/arcticdb/util/buffer_holder.hpp b/cpp/arcticdb/util/buffer_holder.hpp index 603500eeea..8388a0f47d 100644 --- a/cpp/arcticdb/util/buffer_holder.hpp +++ b/cpp/arcticdb/util/buffer_holder.hpp @@ -7,7 +7,7 @@ #pragma once -#include +#include #include namespace arcticdb { @@ -22,4 +22,4 @@ struct BufferHolder { return column; } }; -} +} //namespace arcticdb diff --git a/cpp/arcticdb/util/cursored_buffer.hpp b/cpp/arcticdb/util/cursored_buffer.hpp index 38a2f97a3f..fbe1ff0aba 100644 --- a/cpp/arcticdb/util/cursored_buffer.hpp +++ b/cpp/arcticdb/util/cursored_buffer.hpp @@ -129,7 +129,6 @@ struct CursoredBuffer { template const T *ptr_cast(position_t t_pos, size_t required_bytes) const { -// return reinterpret_cast(buffer_.template ptr_cast(t_pos * sizeof(T), required_bytes)); return reinterpret_cast(buffer_.template ptr_cast(t_pos * sizeof(T), required_bytes)); } diff --git a/cpp/arcticdb/util/lock_table.hpp b/cpp/arcticdb/util/lock_table.hpp index 699594c3ac..b56efdb2e2 100644 --- a/cpp/arcticdb/util/lock_table.hpp +++ b/cpp/arcticdb/util/lock_table.hpp @@ -40,7 +40,7 @@ struct ScopedLock { }; class LockTable { - std::unordered_map> locks_; + std::unordered_map> locks_; std::mutex mutex_; public: LockTable() = default; diff --git a/cpp/arcticdb/util/magic_num.hpp b/cpp/arcticdb/util/magic_num.hpp index 07ad429d0f..192c754edd 100644 --- a/cpp/arcticdb/util/magic_num.hpp +++ b/cpp/arcticdb/util/magic_num.hpp @@ -27,7 +27,7 @@ struct MagicNum { void check() const { std::string_view expected(reinterpret_cast(&Magic), 4); - util::check(magic_ == Magic, "Magic number failure, expected {}({}) got {}({})", Magic, expected, magic_); + util::check(magic_ == Magic, "Magic number failure, expected {}({}) got {}", Magic, expected, magic_); } private: diff --git a/cpp/arcticdb/util/pb_util.hpp b/cpp/arcticdb/util/pb_util.hpp index 307e0d00cb..37cbebbcb1 100644 --- a/cpp/arcticdb/util/pb_util.hpp +++ b/cpp/arcticdb/util/pb_util.hpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include @@ -19,11 +20,6 @@ namespace arcticdb::util { -template -[[noreturn]] void raise_error_msg(const char *pattern, const Msg &msg) { - // google::protobuf::TextFormat::PrintToString(msg, &s); - throw ExcType(fmt::format(fmt::runtime(pattern), msg.DebugString())); -} namespace { constexpr char TYPE_URL[] = "cxx.arctic.org"; @@ -37,7 +33,7 @@ void pack_to_any(const Msg &msg, google::protobuf::Any &any) { inline folly::StringPiece get_arcticdb_pb_type_name(const google::protobuf::Any &any) { folly::StringPiece sp{any.type_url()}; if (!sp.startsWith(TYPE_URL)) { - raise_error_msg("Not a valid arcticc proto msg", any); + util::raise_rte("Not a valid arcticc proto msg", any.DebugString()); } return sp.subpiece(sizeof(TYPE_URL), sp.size()); } diff --git a/cpp/arcticdb/util/ref_counted_map.hpp b/cpp/arcticdb/util/ref_counted_map.hpp deleted file mode 100644 index addd5c66a8..0000000000 --- a/cpp/arcticdb/util/ref_counted_map.hpp +++ /dev/null @@ -1,55 +0,0 @@ -/* Copyright 2023 Man Group Operations Limited - * - * Use of this software is governed by the Business Source License 1.1 included in the file licenses/BSL.txt. - * - * As of the Change Date specified in that file, in accordance with the Business Source License, use of this software will be governed by the Apache License, version 2.0. - */ - -#pragma once - -#include -#include - -namespace arcticdb { - -class SegmentMap { - using ValueType = std::variant; - using ContainerType = std::unordered_map>; - std::atomic id_; - std::shared_ptr store_; - std::mutex mutex_; -public: - using const_iterator = ContainerType::const_iterator; - - SegmentMap(const std::shared_ptr& store) : - store_(store) { - } - - uint64_t insert(std::shared_ptr&& seg) { - const auto id = id_++; - std::shared_ptr value( - std::move(seg), - [this, id](ValueType* v) - map_.erase(id); - delete v; - } - ); - map_.emplace(id, value); - return id; - } - - int size() const { - return map_.size(); - } - - const_iterator begin() const { - return map_.begin(); - } - - const_iterator end() const { - return map_.end(); - } -private: - container_type map_; -}; -} \ No newline at end of file diff --git a/cpp/arcticdb/util/test/generators.hpp b/cpp/arcticdb/util/test/generators.hpp index c73c526842..27855679fb 100644 --- a/cpp/arcticdb/util/test/generators.hpp +++ b/cpp/arcticdb/util/test/generators.hpp @@ -408,7 +408,7 @@ struct SegmentToInputFrameAdapter { input_frame_->desc = segment_.descriptor(); input_frame_->num_rows = segment_.row_count(); size_t col{0}; - if (segment_.descriptor().index().type() != IndexDescriptor::ROWCOUNT) { + if (segment_.descriptor().index().type() != IndexDescriptorImpl::Type::ROWCOUNT) { for (size_t i = 0; i < segment_.descriptor().index().field_count(); ++i) { input_frame_->index_tensor = tensor_from_column(segment_.column(col)); ++col; diff --git a/cpp/arcticdb/util/test/gtest_utils.hpp b/cpp/arcticdb/util/test/gtest_utils.hpp index 0085533e34..a923929c84 100644 --- a/cpp/arcticdb/util/test/gtest_utils.hpp +++ b/cpp/arcticdb/util/test/gtest_utils.hpp @@ -18,7 +18,7 @@ template<> inline void PrintTo(const our_type&val, ::std::ostream* os) { fmt::pr MAKE_GTEST_FMT(arcticdb::entity::RefKey, "{}") MAKE_GTEST_FMT(arcticdb::entity::AtomKeyImpl, "{}") MAKE_GTEST_FMT(arcticdb::entity::VariantKey, "VariantKey({})") -MAKE_GTEST_FMT(arcticdb::entity::VariantId, "VariantId({})") +MAKE_GTEST_FMT(arcticdb::VariantId, "VariantId({})") // FUTURE (C++20): with capabilities, we can write a generic PrintTo that covers all fmt::format-able types that is // not ambiguous with the built-in diff --git a/cpp/arcticdb/util/test/rapidcheck_generators.hpp b/cpp/arcticdb/util/test/rapidcheck_generators.hpp index f7ad4afdaa..414511f29d 100644 --- a/cpp/arcticdb/util/test/rapidcheck_generators.hpp +++ b/cpp/arcticdb/util/test/rapidcheck_generators.hpp @@ -71,9 +71,9 @@ struct Arbitrary { for (const auto& field_name: field_names) { field_descriptors.add_field(arcticdb::entity::scalar_field(*gen_numeric_datatype(), field_name)); } - auto desc =stream_descriptor(arcticdb::entity::StreamId{id}, arcticdb::stream::RowCountIndex{}, arcticdb::fields_from_range(field_descriptors)); + auto desc = stream_descriptor_from_range(arcticdb::StreamId{id}, arcticdb::stream::RowCountIndex{}, std::move(field_descriptors)); return gen::build( - gen::set(&StreamDescriptor::data_, gen::just(desc.data_)), + gen::set(&StreamDescriptor::segment_desc_, gen::just(desc.segment_desc_)), gen::set(&StreamDescriptor::fields_, gen::just(desc.fields_)) ); } diff --git a/cpp/arcticdb/util/test/test_slab_allocator.cpp b/cpp/arcticdb/util/test/test_slab_allocator.cpp index 9ddb229bca..dc437b5df4 100644 --- a/cpp/arcticdb/util/test/test_slab_allocator.cpp +++ b/cpp/arcticdb/util/test/test_slab_allocator.cpp @@ -101,7 +101,7 @@ void run_test(MemoryChunk& mc, unsigned int K) std::cout << "Average execution time: " << avg / execution_times.size() << '\n'; } -TEST(SlabAlloc, CacheLine128) { +/*TEST(SlabAlloc, CacheLine128) { SKIP_WIN("Slab allocator not supported"); SlabAllocator mc128{num_blocks_per_thread * num_threads}; std::cout << "BEGIN: mc128 tests\n"; @@ -234,3 +234,4 @@ TEST(SlabAlloc, Callbacks) { mc.allocate(); ASSERT_FALSE(mc._get_cb_activated()); } +*/ \ No newline at end of file diff --git a/cpp/arcticdb/util/type_handler.hpp b/cpp/arcticdb/util/type_handler.hpp index 87fcb6296f..f3355c63e6 100644 --- a/cpp/arcticdb/util/type_handler.hpp +++ b/cpp/arcticdb/util/type_handler.hpp @@ -8,8 +8,8 @@ #pragma once #include -#include -#include +#include +#include #include @@ -19,6 +19,7 @@ namespace arcticdb { struct BufferHolder; +struct ColumnMapping; struct ITypeHandler { template @@ -31,7 +32,8 @@ struct ITypeHandler { void handle_type( const uint8_t*& source, uint8_t* dest, - const VariantField& encoded_field_info, + const EncodedFieldImpl& encoded_field_info, + const entity::TypeDescriptor& type_descriptor, size_t dest_bytes, std::shared_ptr buffers, EncodingVersion encoding_version, @@ -42,6 +44,7 @@ struct ITypeHandler { source, dest, encoded_field_info, + type_descriptor, dest_bytes, buffers, encoding_version, diff --git a/cpp/arcticdb/version/local_versioned_engine.cpp b/cpp/arcticdb/version/local_versioned_engine.cpp index 244b7c8fae..36362c171e 100644 --- a/cpp/arcticdb/version/local_versioned_engine.cpp +++ b/cpp/arcticdb/version/local_versioned_engine.cpp @@ -46,6 +46,7 @@ void LocalVersionedEngine::initialize(const std::shared_ptr& l async::TaskScheduler::set_forked(false); async::TaskScheduler::reattach_instance(); } + (void)async::TaskScheduler::instance(); } template LocalVersionedEngine::LocalVersionedEngine(const std::shared_ptr& library, const util::SysClock&); @@ -365,7 +366,11 @@ folly::Future LocalVersionedEngine::get_descriptor( .thenValue([](auto&& key_seg_pair) -> DescriptorItem { auto key_seg = std::move(std::get<0>(key_seg_pair)); auto seg = std::move(std::get<1>(key_seg_pair)); - auto timeseries_descriptor = seg.has_metadata() ? std::make_optional(*seg.metadata()) : std::nullopt; + std::optional timeseries_descriptor; + if (seg.has_index_descriptor()) + timeseries_descriptor.emplace(seg.index_descriptor()); + + //TODO this is bonkers - replace auto start_index = seg.column(position_t(index::Fields::start_index)).type().visit_tag([&](auto column_desc_tag) -> std::optional { using ColumnDescriptorType = std::decay_t; using ColumnTagType = typename ColumnDescriptorType::DataTypeTag; @@ -1748,9 +1753,9 @@ std::unordered_map LocalVersionedEngine::scan_object_size ++sizes_info.count; key_size_calculators.emplace_back(std::forward(k), [&sizes_info] (auto&& ks) { auto key_seg = std::move(ks); - sizes_info.compressed_size += key_seg.segment().total_segment_size(); - auto desc = key_seg.segment().header().stream_descriptor(); - sizes_info.uncompressed_size += desc.in_bytes(); + sizes_info.compressed_size += key_seg.segment().size(); + const auto& desc = key_seg.segment().descriptor(); + sizes_info.uncompressed_size += desc.uncompressed_bytes(); return key_seg.variant_key(); }); }); @@ -1781,9 +1786,9 @@ std::unordered_map> LocalVer auto key_seg = std::move(ks); auto variant_key = key_seg.variant_key(); auto stream_id = variant_key_id(variant_key); - auto compressed_size = key_seg.segment().total_segment_size(); - auto desc = key_seg.segment().header().stream_descriptor(); - auto uncompressed_size = desc.in_bytes(); + auto compressed_size = key_seg.segment().size(); + auto desc = key_seg.segment().descriptor(); + auto uncompressed_size = desc.uncompressed_bytes(); { std::lock_guard lock{mutex}; diff --git a/cpp/arcticdb/version/python_bindings.cpp b/cpp/arcticdb/version/python_bindings.cpp index 5bf1420ee7..0d57512f60 100644 --- a/cpp/arcticdb/version/python_bindings.cpp +++ b/cpp/arcticdb/version/python_bindings.cpp @@ -199,18 +199,7 @@ void register_bindings(py::module &version, py::exceptionUnpackTo(&tsd); - pyobj = python_util::pb_to_python(tsd); - } else { - pyobj = pybind11::none(); - } - return pyobj; - }); + .def_property_readonly("timeseries_descriptor", &DescriptorItem::timeseries_descriptor); py::class_>(version, "FrameSlice") .def_property_readonly("col_range", &pipelines::FrameSlice::columns) diff --git a/cpp/arcticdb/version/schema_checks.hpp b/cpp/arcticdb/version/schema_checks.hpp index 526d9b7fb7..3b7cc59d6f 100644 --- a/cpp/arcticdb/version/schema_checks.hpp +++ b/cpp/arcticdb/version/schema_checks.hpp @@ -36,17 +36,17 @@ inline void check_normalization_index_match(NormalizationOperation operation, bool new_is_timeseries = std::holds_alternative(frame.index); if (operation == UPDATE) { - util::check_rte(old_idx_kind == IndexDescriptor::TIMESTAMP && new_is_timeseries, + util::check_rte(old_idx_kind == IndexDescriptorImpl::Type::TIMESTAMP && new_is_timeseries, "Update will not work as expected with a non-timeseries index"); } else { // TODO: AN-722 if (new_is_timeseries) { - if (old_idx_kind != IndexDescriptor::TIMESTAMP) { + if (old_idx_kind != IndexDescriptorImpl::Type::TIMESTAMP) { log::version().warn("Appending a timeseries to a non-timeseries-indexed symbol may create a " "confusing index and cause problems later"); } } else { - if (old_idx_kind != IndexDescriptor::ROWCOUNT) { + if (old_idx_kind != IndexDescriptorImpl::Type::ROWCOUNT) { // Backwards compatibility log::version().warn("Appending a non-timeseries-indexed data to a timeseries symbol is highly " "likely to cause corruption/unexpected behaviour."); diff --git a/cpp/arcticdb/version/symbol_list.cpp b/cpp/arcticdb/version/symbol_list.cpp index 289882a50d..789e202d3a 100644 --- a/cpp/arcticdb/version/symbol_list.cpp +++ b/cpp/arcticdb/version/symbol_list.cpp @@ -32,7 +32,7 @@ using CollectionType = std::vector; constexpr std::string_view version_string = "_v2_"; constexpr NumericIndex version_identifier = std::numeric_limits::max(); -SymbolListData::SymbolListData(std::shared_ptr version_map, entity::StreamId type_indicator, uint32_t seed) : +SymbolListData::SymbolListData(std::shared_ptr version_map, StreamId type_indicator, uint32_t seed) : type_holder_(std::move(type_indicator)), seed_(seed), version_map_(std::move(version_map)){ @@ -374,8 +374,6 @@ ProblematicResult is_problematic( if(existing.reference_id_ < latest.reference_id_) return not_a_problem(); - - if(all_same_action) return not_a_problem(); @@ -677,11 +675,6 @@ SegmentInMemory write_entries_to_symbol_segment( SegmentInMemory create_empty_segment(const StreamId& stream_id) { SegmentInMemory output{StreamDescriptor{stream_id}}; - google::protobuf::Any any = {}; - arcticdb::proto::descriptors::SymbolListDescriptor metadata; - metadata.set_enabled(true); - any.PackFrom(metadata); - output.set_metadata(std::move(any)); return output; } diff --git a/cpp/arcticdb/version/symbol_list.hpp b/cpp/arcticdb/version/symbol_list.hpp index 2c7f7de874..08b8c0cdb5 100644 --- a/cpp/arcticdb/version/symbol_list.hpp +++ b/cpp/arcticdb/version/symbol_list.hpp @@ -27,12 +27,12 @@ struct LoadResult; class Store; struct SymbolListData { - entity::StreamId type_holder_; + StreamId type_holder_; uint32_t seed_; std::shared_ptr version_map_; std::atomic warned_expected_slowdown_ = false; - explicit SymbolListData(std::shared_ptr version_map, entity::StreamId type_indicator = entity::StringId(), + explicit SymbolListData(std::shared_ptr version_map, StreamId type_indicator = StringId(), uint32_t seed = 0); }; @@ -48,7 +48,7 @@ enum class ActionType : uint8_t { DELETE }; -inline entity::StreamId action_id(ActionType action) { +inline StreamId action_id(ActionType action) { switch (action) { case ActionType::ADD: return StringId{AddSymbol}; @@ -85,10 +85,10 @@ inline bool operator==(const SymbolEntryData& l, const SymbolEntryData& r) { } struct SymbolListEntry : public SymbolEntryData { - entity::StreamId stream_id_; + StreamId stream_id_; SymbolListEntry( - entity::StreamId stream_id, + StreamId stream_id, entity::VersionId reference_id, timestamp reference_time, ActionType action @@ -149,25 +149,25 @@ ProblematicResult is_problematic(const std::vector& updated, ti class SymbolList { SymbolListData data_; public: - explicit SymbolList(std::shared_ptr version_map, entity::StreamId type_indicator = entity::StringId(), + explicit SymbolList(std::shared_ptr version_map, StreamId type_indicator = StringId(), uint32_t seed = 0) : data_(std::move(version_map), std::move(type_indicator), seed) { } - std::set load(const std::shared_ptr& version_map, const std::shared_ptr& store, bool no_compaction); + std::set load(const std::shared_ptr& version_map, const std::shared_ptr& store, bool no_compaction); - std::vector get_symbols(const std::shared_ptr& store, bool no_compaction=false) { + std::vector get_symbols(const std::shared_ptr& store, bool no_compaction=false) { auto symbols = load(data_.version_map_, store, no_compaction); return {std::make_move_iterator(symbols.begin()), std::make_move_iterator(symbols.end())}; } - std::set get_symbol_set(const std::shared_ptr& store) { + std::set get_symbol_set(const std::shared_ptr& store) { return load(data_.version_map_, store, false); } - static void add_symbol(const std::shared_ptr& store, const entity::StreamId& symbol, entity::VersionId reference_id); + static void add_symbol(const std::shared_ptr& store, const StreamId& symbol, entity::VersionId reference_id); - static void remove_symbol(const std::shared_ptr& store, const entity::StreamId& symbol, entity::VersionId reference_id); + static void remove_symbol(const std::shared_ptr& store, const StreamId& symbol, entity::VersionId reference_id); static void clear(const std::shared_ptr& store); @@ -183,13 +183,13 @@ std::vector delete_keys( struct WriteSymbolTask : async::BaseTask { const std::shared_ptr store_; std::shared_ptr symbol_list_; - const entity::StreamId stream_id_; + const StreamId stream_id_; const entity::VersionId reference_id_; WriteSymbolTask( std::shared_ptr store, std::shared_ptr symbol_list, - entity::StreamId stream_id, + StreamId stream_id, entity::VersionId reference_id) : store_(std::move(store)), symbol_list_(std::move(symbol_list)), diff --git a/cpp/arcticdb/version/test/test_version_store.cpp b/cpp/arcticdb/version/test/test_version_store.cpp index 6910ce2154..c6bd334b19 100644 --- a/cpp/arcticdb/version/test/test_version_store.cpp +++ b/cpp/arcticdb/version/test/test_version_store.cpp @@ -769,7 +769,7 @@ TEST(VersionStore, TestWriteAppendMapHead) { auto key = atom_key_builder().version_id(0).creation_ts(PilotedClock::nanos_since_epoch()).content_hash(0).build(symbol, KeyType::APPEND_DATA); - auto descriptor = StreamDescriptor{symbol, IndexDescriptor{1u, IndexDescriptor::TIMESTAMP}, std::make_shared(fields_from_range(fields))}; + auto descriptor = StreamDescriptor{symbol, IndexDescriptorImpl{1u, IndexDescriptorImpl::Type::TIMESTAMP}, std::make_shared(fields_from_range(fields))}; write_head(version_store._test_get_store(), key, num_rows); auto [next_key, total_rows] = read_head(version_store._test_get_store(), symbol); ASSERT_EQ(next_key, key); diff --git a/cpp/arcticdb/version/version_core-inl.hpp b/cpp/arcticdb/version/version_core-inl.hpp index 1dfee5617d..68b8fd50c4 100644 --- a/cpp/arcticdb/version/version_core-inl.hpp +++ b/cpp/arcticdb/version/version_core-inl.hpp @@ -66,7 +66,7 @@ void merge_frames_for_keys_impl( auto compare = [=](const std::unique_ptr &left, const std::unique_ptr &right) { - return pipelines::index::index_value_from_row(left->row(), IndexDescriptor::TIMESTAMP, 0) > pipelines::index::index_value_from_row(right->row(), IndexDescriptor::TIMESTAMP, 0); + return pipelines::index::index_value_from_row(left->row(), IndexDescriptorImpl::Type::TIMESTAMP, 0) > pipelines::index::index_value_from_row(right->row(), IndexDescriptorImpl::Type::TIMESTAMP, 0); }; movable_priority_queue, std::vector>, decltype(compare)> input_streams{compare}; diff --git a/cpp/arcticdb/version/version_core.cpp b/cpp/arcticdb/version/version_core.cpp index 6c3beec712..0257d042f1 100644 --- a/cpp/arcticdb/version/version_core.cpp +++ b/cpp/arcticdb/version/version_core.cpp @@ -26,7 +26,6 @@ #include #include #include -#include #include #include #include @@ -109,12 +108,12 @@ folly::Future async_write_dataframe_impl( } namespace { -IndexDescriptor::Proto check_index_match(const arcticdb::stream::Index& index, const IndexDescriptor::Proto& desc) { +IndexDescriptorImpl check_index_match(const arcticdb::stream::Index& index, const IndexDescriptorImpl& desc) { if (std::holds_alternative(index)) - util::check(desc.kind() == IndexDescriptor::TIMESTAMP, + util::check(desc.type() == IndexDescriptorImpl::Type::TIMESTAMP, "Index mismatch, cannot update a non-timeseries-indexed frame with a timeseries"); else - util::check(desc.kind() == IndexDescriptor::ROWCOUNT, + util::check(desc.type() == IndexDescriptorImpl::Type::ROWCOUNT, "Index mismatch, cannot update a timeseries with a non-timeseries-indexed frame"); return desc; @@ -127,7 +126,7 @@ void sorted_data_check_append(const std::shared_ptr& frame, in } sorting::check( !std::holds_alternative(frame->index) || - index_segment_reader.mutable_tsd().mutable_proto().stream_descriptor().sorted() == arcticdb::proto::descriptors::SortedValue::ASCENDING, + index_segment_reader.tsd().sorted() == SortedValue::ASCENDING, "When calling append with validate_index enabled, the existing data must be sorted"); } @@ -143,7 +142,7 @@ folly::Future async_append_impl( ARCTICDB_DEBUG(log::version(), "append stream_id: {} , version_id: {}", stream_id, update_info.next_version_id_); auto index_segment_reader = index::get_index_reader(*(update_info.previous_index_key_), store); bool bucketize_dynamic = index_segment_reader.bucketize_dynamic(); - auto row_offset = index_segment_reader.tsd().proto().total_rows(); + auto row_offset = index_segment_reader.tsd().total_rows(); util::check_rte(!index_segment_reader.is_pickled(), "Cannot append to pickled data"); if (validate_index) { sorted_data_check_append(frame, index_segment_reader); @@ -286,11 +285,9 @@ VersionedItem delete_range_impl( auto flattened_slice_and_keys = flatten_and_fix_rows(groups, row_count); std::sort(std::begin(flattened_slice_and_keys), std::end(flattened_slice_and_keys)); - bool bucketize_dynamic = index_segment_reader.bucketize_dynamic(); - auto time_series = timseries_descriptor_from_index_segment(row_count, std::move(index_segment_reader), std::nullopt, bucketize_dynamic); - auto version_key_fut = util::variant_match(index, [&time_series, &flattened_slice_and_keys, &stream_id, &version_id, &store] (auto idx) { + auto version_key_fut = util::variant_match(index, [&index_segment_reader, &flattened_slice_and_keys, &stream_id, &version_id, &store] (auto idx) { using IndexType = decltype(idx); - return pipelines::index::write_index(std::move(time_series), std::move(flattened_slice_and_keys), IndexPartialKey{stream_id, version_id}, store); + return pipelines::index::write_index(index_segment_reader.tsd(), std::move(flattened_slice_and_keys), IndexPartialKey{stream_id, version_id}, store); }); auto version_key = std::move(version_key_fut).get(); auto versioned_item = VersionedItem(to_atom(std::move(version_key))); @@ -298,19 +295,19 @@ VersionedItem delete_range_impl( return versioned_item; } -void sorted_data_check_update(InputTensorFrame& frame, index::IndexSegmentReader& index_segment_reader){ +void sorted_data_check_update(InputTensorFrame& frame, const index::IndexSegmentReader& index_segment_reader){ bool is_time_series = std::holds_alternative(frame.index); sorting::check( is_time_series, "When calling update, the input data must be a time series."); - bool input_data_is_sorted = frame.desc.get_sorted() == SortedValue::ASCENDING || - frame.desc.get_sorted() == SortedValue::UNKNOWN; + bool input_data_is_sorted = frame.desc.sorted() == SortedValue::ASCENDING || + frame.desc.sorted() == SortedValue::UNKNOWN; // If changing this error message, the corresponding message in _normalization.py::restrict_data_to_date_range_only should also be updated sorting::check( input_data_is_sorted, "When calling update, the input data must be sorted."); - bool existing_data_is_sorted = index_segment_reader.get_sorted() == SortedValue::ASCENDING || - index_segment_reader.get_sorted() == SortedValue::UNKNOWN; + bool existing_data_is_sorted = index_segment_reader.sorted() == SortedValue::ASCENDING || + index_segment_reader.sorted() == SortedValue::UNKNOWN; sorting::check( existing_data_is_sorted, "When calling update, the existing data must be sorted."); @@ -328,8 +325,8 @@ VersionedItem update_impl( ARCTICDB_DEBUG(log::version(), "Update versioned dataframe for stream_id: {} , version_id = {}", stream_id, update_info.previous_index_key_->version_id()); auto index_segment_reader = index::get_index_reader(*(update_info.previous_index_key_), store); util::check_rte(!index_segment_reader.is_pickled(), "Cannot update pickled data"); - auto index_desc = check_index_match(frame->index, index_segment_reader.tsd().proto().stream_descriptor().index()); - util::check(index_desc.kind() == IndexDescriptor::TIMESTAMP, "Update not supported for non-timeseries indexes"); + auto index_desc = check_index_match(frame->index, index_segment_reader.tsd().index()); + util::check(index_desc.type() == IndexDescriptorImpl::Type::TIMESTAMP, "Update not supported for non-timeseries indexes"); sorted_data_check_update(*frame, index_segment_reader); bool bucketize_dynamic = index_segment_reader.bucketize_dynamic(); (void)check_and_mark_slices(index_segment_reader, dynamic_schema, false, std::nullopt, bucketize_dynamic); @@ -593,11 +590,13 @@ void set_output_descriptors( std::shared_ptr> columns_to_decode(const std::shared_ptr& pipeline_context) { std::shared_ptr> res; + ARCTICDB_DEBUG(log::version(), "Creating columns list with {} bits set", pipeline_context->overall_column_bitset_->count()); if(pipeline_context->overall_column_bitset_) { res = std::make_shared>(); auto en = pipeline_context->overall_column_bitset_->first(); auto en_end = pipeline_context->overall_column_bitset_->end(); while (en < en_end) { + ARCTICDB_DEBUG(log::version(), "Adding field {}", pipeline_context->desc_->field(*en).name()); res->insert(std::string(pipeline_context->desc_->field(*en++).name())); } } @@ -691,7 +690,7 @@ void add_index_columns_to_query(const ReadQuery& read_query, const TimeseriesDes std::vector index_columns_to_add; for(const auto& index_column : index_columns) { if(std::find(std::begin(read_query.columns), std::end(read_query.columns), index_column) == std::end(read_query.columns)) - index_columns_to_add.push_back(index_column); + index_columns_to_add.push_back(std::string(index_column)); } read_query.columns.insert(std::begin(read_query.columns), std::begin(index_columns_to_add), std::end(index_columns_to_add)); } @@ -700,8 +699,7 @@ void add_index_columns_to_query(const ReadQuery& read_query, const TimeseriesDes FrameAndDescriptor read_segment_impl( const std::shared_ptr& store, const VariantKey& key) { - auto fut_segment = store->read(key); - auto [_, seg] = std::move(fut_segment).get(); + auto [_, seg] = store->read_sync(key); return frame_and_descriptor_from_segment(std::move(seg)); } @@ -752,7 +750,7 @@ void read_indexed_keys_to_pipeline( add_index_columns_to_query(read_query, index_segment_reader.tsd()); const auto& tsd = index_segment_reader.tsd(); - read_query.calculate_row_filter(static_cast(tsd.proto().total_rows())); + read_query.calculate_row_filter(static_cast(tsd.total_rows())); bool bucketize_dynamic = index_segment_reader.bucketize_dynamic(); pipeline_context->desc_ = tsd.as_stream_descriptor(); @@ -765,7 +763,7 @@ void read_indexed_keys_to_pipeline( pipeline_context->slice_and_keys_ = filter_index(index_segment_reader, combine_filter_functions(queries)); pipeline_context->total_rows_ = pipeline_context->calc_rows(); - pipeline_context->rows_ = index_segment_reader.tsd().proto().total_rows(); + pipeline_context->rows_ = index_segment_reader.tsd().total_rows(); pipeline_context->norm_meta_ = std::make_unique(std::move(*index_segment_reader.mutable_tsd().mutable_proto().mutable_normalization())); pipeline_context->user_meta_ = std::make_unique(std::move(*index_segment_reader.mutable_tsd().mutable_proto().mutable_user_meta())); pipeline_context->bucketize_dynamic_ = bucketize_dynamic; @@ -831,7 +829,7 @@ void check_incompletes_index_ranges_dont_overlap(const std::shared_ptrdescriptor().index().type() == IndexDescriptor::TIMESTAMP) { + if (pipeline_context->descriptor().index().type() == IndexDescriptorImpl::Type::TIMESTAMP) { std::optional last_existing_index_value; // Beginning of incomplete segments == beginning of all segments implies all segments are incompletes, so we are // writing, not appending @@ -1091,7 +1089,7 @@ FrameAndDescriptor read_column_stats_impl( try { auto segment_in_memory = store->read(column_stats_key).get().second; TimeseriesDescriptor tsd; - tsd.mutable_proto().set_total_rows(segment_in_memory.row_count()); + tsd.set_total_rows(segment_in_memory.row_count()); tsd.set_stream_descriptor(segment_in_memory.descriptor()); return {SegmentInMemory(std::move(segment_in_memory)), tsd, {}, {}}; } catch (const std::exception& e) { @@ -1190,8 +1188,8 @@ VersionedItem collate_and_write( TimeseriesDescriptor tsd; tsd.set_stream_descriptor(pipeline_context->descriptor()); + tsd.set_total_rows(pipeline_context->total_rows_); auto& tsd_proto = tsd.mutable_proto(); - tsd_proto.set_total_rows(pipeline_context->total_rows_); tsd_proto.mutable_normalization()->CopyFrom(*pipeline_context->norm_meta_); if(user_meta) tsd_proto.mutable_user_meta()->CopyFrom(*user_meta); @@ -1232,7 +1230,7 @@ VersionedItem sort_merge_impl( std::optional previous_sorted_value; if(append && update_info.previous_index_key_.has_value()) { read_indexed_keys_to_pipeline(store, pipeline_context, *(update_info.previous_index_key_), read_query, ReadOptions{}); - previous_sorted_value.emplace(pipeline_context->desc_->get_sorted()); + previous_sorted_value.emplace(pipeline_context->desc_->sorted()); } auto num_versioned_rows = pipeline_context->total_rows_; @@ -1326,7 +1324,7 @@ VersionedItem compact_incomplete_impl( std::optional previous_sorted_value; if(append && update_info.previous_index_key_.has_value()) { read_indexed_keys_to_pipeline(store, pipeline_context, *(update_info.previous_index_key_), read_query, read_options); - previous_sorted_value.emplace(pipeline_context->desc_->get_sorted()); + previous_sorted_value.emplace(pipeline_context->desc_->sorted()); } auto prev_size = pipeline_context->slice_and_keys_.size(); diff --git a/cpp/arcticdb/version/version_tasks.hpp b/cpp/arcticdb/version/version_tasks.hpp index b44c2dfed9..e0b6f30323 100644 --- a/cpp/arcticdb/version/version_tasks.hpp +++ b/cpp/arcticdb/version/version_tasks.hpp @@ -37,12 +37,7 @@ struct UpdateMetadataTask : async::BaseTask { auto index_key = *(update_info_.previous_index_key_); auto segment = store_->read_sync(index_key).second; - auto tsd = segment.index_descriptor(); - google::protobuf::Any output = {}; - tsd.mutable_proto().mutable_user_meta()->CopyFrom(user_meta_); - output.PackFrom(tsd.proto()); - - segment.override_metadata(std::move(output)); + segment.mutable_index_descriptor().mutable_proto().mutable_user_meta()->CopyFrom(user_meta_); return to_atom(store_->write_sync(index_key.type(), update_info_.next_version_id_, index_key.id(), index_key.start_index(), index_key.end_index(), std::move(segment))); } diff --git a/cpp/arcticdb/version/version_utils.cpp b/cpp/arcticdb/version/version_utils.cpp index ba5b993210..a8f9067337 100644 --- a/cpp/arcticdb/version/version_utils.cpp +++ b/cpp/arcticdb/version/version_utils.cpp @@ -20,8 +20,8 @@ using namespace arcticdb::entity; using namespace arcticdb::stream; -std::unordered_map get_num_version_entries(const std::shared_ptr& store, size_t batch_size) { - std::unordered_map output; +std::unordered_map get_num_version_entries(const std::shared_ptr& store, size_t batch_size) { + std::unordered_map output; size_t max_blocks = ConfigsMap::instance()->get_int("VersionMap.MaxVersionBlocks", 5); store->iterate_type(entity::KeyType::VERSION, [&output, batch_size, max_blocks] (const VariantKey& key) { ++output[variant_key_id(key)]; @@ -48,10 +48,10 @@ std::unordered_map get_num_version_entries(const std:: FrameAndDescriptor frame_and_descriptor_from_segment(SegmentInMemory&& seg) { TimeseriesDescriptor tsd; auto& tsd_proto = tsd.mutable_proto(); - tsd_proto.set_total_rows(seg.row_count()); + tsd.set_total_rows(seg.row_count()); const auto& seg_descriptor = seg.descriptor(); - tsd_proto.mutable_stream_descriptor()->CopyFrom(seg_descriptor.proto()); - if (seg.descriptor().index().type() == IndexDescriptor::ROWCOUNT) + tsd.set_stream_descriptor(seg_descriptor); + if (seg_descriptor.index().type() == IndexDescriptor::Type::ROWCOUNT) ensure_rowcount_norm_meta(*tsd_proto.mutable_normalization(), seg_descriptor.id()); else ensure_timeseries_norm_meta(*tsd.mutable_proto().mutable_normalization(), seg_descriptor.id(), false); diff --git a/cpp/arcticdb/version/version_utils.hpp b/cpp/arcticdb/version/version_utils.hpp index e21229280a..2602ebaabf 100644 --- a/cpp/arcticdb/version/version_utils.hpp +++ b/cpp/arcticdb/version/version_utils.hpp @@ -391,6 +391,7 @@ inline SortedValue deduce_sorted(SortedValue existing_frame, SortedValue input_f return final_state; } + FrameAndDescriptor frame_and_descriptor_from_segment(SegmentInMemory&& seg); } // namespace arcticdb \ No newline at end of file diff --git a/cpp/proto/arcticc/pb2/descriptors.proto b/cpp/proto/arcticc/pb2/descriptors.proto index c1cf5a49dc..3abcec3ea0 100644 --- a/cpp/proto/arcticc/pb2/descriptors.proto +++ b/cpp/proto/arcticc/pb2/descriptors.proto @@ -309,6 +309,14 @@ message TimeSeriesDescriptor UserDefinedMetadata multi_key_meta = 7; } +message FrameMetadata +{ + NormalizationMetadata normalization = 1; + UserDefinedMetadata user_meta = 2; + AtomKey next_key = 3; + UserDefinedMetadata multi_key_meta = 4; +} + message SymbolListDescriptor { bool enabled = 1; diff --git a/cpp/proto/arcticc/pb2/encoding.proto b/cpp/proto/arcticc/pb2/encoding.proto index 4e50d66998..e286aeaca4 100644 --- a/cpp/proto/arcticc/pb2/encoding.proto +++ b/cpp/proto/arcticc/pb2/encoding.proto @@ -26,13 +26,10 @@ message SegmentHeader { ROWCOUNT = 0; XX_HASH = 1; } - HashType hashing_algo = 6; // defaults to none + HashType hashing_algo = 6; // unused EncodedField metadata_field = 7; // optional metadata encoding EncodedField string_pool_field = 8; // string pool bool compacted = 9; // is the segment the result of a compaction - EncodedField descriptor_field = 10; - EncodedField index_descriptor_field = 11; - EncodedField column_fields = 12; uint32 encoding_version = 13; }