diff --git a/.github/workflows/test.yml b/.github/workflows/test.yml index a70f39ce..644636a1 100644 --- a/.github/workflows/test.yml +++ b/.github/workflows/test.yml @@ -165,7 +165,12 @@ jobs: run: python -m pip install ".[testing]" - name: Test Python - run: python -m pytest -v -k test_stream_data_to_s3 + env: + ZARR_S3_ENDPOINT: ${{ env.MINIO_URL }} + ZARR_S3_BUCKET_NAME: ${{ env.MINIO_BUCKET }} + AWS_ACCESS_KEY_ID: ${{ env.MINIO_ACCESS_KEY }} + AWS_SECRET_ACCESS_KEY: ${{ env.MINIO_SECRET_KEY }} + run: python -m pytest -s -k test_stream_data_to_s3 test-python: diff --git a/benchmarks/benchmark.py b/benchmarks/benchmark.py index cb145ece..80e88433 100644 --- a/benchmarks/benchmark.py +++ b/benchmarks/benchmark.py @@ -148,6 +148,7 @@ def run_acquire_zarr_test( data_type=aqz.DataType.UINT16, ) ], + overwrite=True, ) # Create a ZarrStream for appending frames. @@ -156,15 +157,23 @@ def run_acquire_zarr_test( elapsed_times = [] total_start = time.perf_counter_ns() + chunk = np.empty((tchunk_size, 2048, 2048), dtype=np.uint16) for i in range(data.shape[0]): start_plane = time.perf_counter_ns() - stream.append(data[i]) + chunk_idx = i % tchunk_size + chunk[chunk_idx] = data[i] + if chunk_idx == tchunk_size - 1: + stream.append(chunk) elapsed = time.perf_counter_ns() - start_plane elapsed_times.append(elapsed) print(f"Acquire-zarr: Plane {i} written in {elapsed / 1e6:.3f} ms") # Close (or flush) the stream to finalize writes. - del stream + start_close = time.perf_counter_ns() + stream.close() + elapsed = time.perf_counter_ns() - start_close + elapsed_times.append(elapsed) + print(f"Acquire-zarr: Final close took {elapsed / 1e6:.3f} ms") total_elapsed = time.perf_counter_ns() - total_start tot_ms = total_elapsed / 1e6 print(f"Acquire-zarr: Total write time: {tot_ms:.3f} ms") diff --git a/python/acquire-zarr-py.cpp b/python/acquire-zarr-py.cpp index a1113a31..220781f5 100644 --- a/python/acquire-zarr-py.cpp +++ b/python/acquire-zarr-py.cpp @@ -888,7 +888,7 @@ class PyZarrStreamSettings settings_.overwrite = static_cast(overwrite_); if (s3_settings_) { - *(settings_.s3_settings) = *(s3_settings_->settings()); + settings_.s3_settings = s3_settings_->settings(); } // construct array lifetime props and set up arrays @@ -1087,7 +1087,7 @@ class PyZarrStream } auto buf = contiguous_data.request(); - auto* ptr = (uint8_t*)buf.ptr; + auto* ptr = static_cast(buf.ptr); py::gil_scoped_release release; diff --git a/python/tests/test_stream.py b/python/tests/test_stream.py index c8d7e108..66ab2e6d 100644 --- a/python/tests/test_stream.py +++ b/python/tests/test_stream.py @@ -378,7 +378,6 @@ def test_stream_data_to_filesystem( data[i, :, :] = i stream.append(data) - stream.close() # close the stream, flush the files chunk_size_bytes = data.dtype.itemsize @@ -394,40 +393,40 @@ def test_stream_data_to_filesystem( shard_size_bytes + table_size_bytes + 4 ) # 4 bytes for crc32c checksum + for x in range(settings.arrays[0].dimensions[-1].array_size_px): + for y in range(settings.arrays[0].dimensions[-2].array_size_px): + for z in range(settings.arrays[0].dimensions[-3].array_size_px): + shard_file = store_path / "test.zarr" / "0" / "c" / str(z) / str(y) / str(x) + assert shard_file.is_file() + if compression_codec is None: + assert shard_file.stat().st_size == shard_size_bytes + else: + size = shard_file.stat().st_size + assert table_size_bytes < size <= shard_size_bytes + group = zarr.open(settings.store_path, mode="r") array = group["0"] assert array.shape == data.shape for i in range(array.shape[0]): - assert np.array_equal(array[i, :, :], data[i, :, :]) + assert np.array_equal(array[i, :, :], data[i, :, :]), f"Data mismatch at index {i}" metadata = array.metadata + sharding_codec = metadata.codecs[0] if compression_codec is not None: cname = ( zblosc.BloscCname.lz4 if compression_codec == CompressionCodec.BLOSC_LZ4 else zblosc.BloscCname.zstd ) - blosc_codec = metadata.codecs[0].codecs[1] + + assert len(sharding_codec.codecs) == 2 + blosc_codec = sharding_codec.codecs[1] assert blosc_codec.cname == cname assert blosc_codec.clevel == 1 assert blosc_codec.shuffle == zblosc.BloscShuffle.shuffle - - assert ( - store_path / "test.zarr" / "0" / "c" / "0" / "0" / "0" - ).is_file() - assert ( - store_path / "test.zarr" / "0" / "c" / "0" / "0" / "0" - ).stat().st_size <= shard_size_bytes else: - assert len(metadata.codecs[0].codecs) == 1 - - assert ( - store_path / "test.zarr" / "0" / "c" / "0" / "0" / "0" - ).is_file() - assert ( - store_path / "test.zarr" / "0" / "c" / "0" / "0" / "0" - ).stat().st_size == shard_size_bytes + assert len(sharding_codec.codecs) == 1 # bytes codec @pytest.mark.parametrize( @@ -456,12 +455,12 @@ def test_stream_data_to_s3( pytest.skip("S3 settings not set") settings.store_path = f"{request.node.name}.zarr".replace("[", "").replace( - "]", "" + "]", "_" ) settings.s3 = s3_settings - settings.data_type = np.uint16 + settings.arrays[0].data_type = np.uint16 if compression_codec is not None: - settings.compression = CompressionSettings( + settings.arrays[0].compression = CompressionSettings( compressor=Compressor.BLOSC1, codec=compression_codec, level=1, @@ -501,18 +500,23 @@ def test_stream_data_to_s3( assert np.array_equal(array[i, :, :], data[i, :, :]) metadata = array.metadata + assert len(metadata.codecs) == 1 # sharding codec + sharding_codec = metadata.codecs[0] + if compression_codec is not None: cname = ( zblosc.BloscCname.lz4 if compression_codec == CompressionCodec.BLOSC_LZ4 else zblosc.BloscCname.zstd ) - blosc_codec = metadata.codecs[0].codecs[1] + assert len(sharding_codec.codecs) == 2 + + blosc_codec = sharding_codec.codecs[1] assert blosc_codec.cname == cname assert blosc_codec.clevel == 1 assert blosc_codec.shuffle == zblosc.BloscShuffle.shuffle else: - assert len(metadata.codecs[0].codecs) == 1 + assert len(sharding_codec.codecs) == 1 # bytes codec # cleanup s3 = s3fs.S3FileSystem( diff --git a/src/streaming/CMakeLists.txt b/src/streaming/CMakeLists.txt index fe1713a1..dbbf858e 100644 --- a/src/streaming/CMakeLists.txt +++ b/src/streaming/CMakeLists.txt @@ -11,8 +11,6 @@ add_library(${tgt} acquire.zarr.cpp array.dimensions.hh array.dimensions.cpp - locked.buffer.hh - locked.buffer.cpp frame.queue.hh frame.queue.cpp downsampler.hh @@ -29,19 +27,27 @@ add_library(${tgt} s3.connection.cpp file.handle.hh file.handle.cpp - sink.hh - sink.cpp - file.sink.hh - file.sink.cpp ${PLATFORM_CPP} - s3.sink.hh - s3.sink.cpp + s3.object.hh + s3.object.cpp array.base.hh array.base.cpp array.hh array.cpp multiscale.array.hh multiscale.array.cpp + fs.storage.hh + fs.storage.cpp + fs.array.hh + fs.array.cpp + fs.multiscale.array.hh + fs.multiscale.array.cpp + s3.storage.hh + s3.storage.cpp + s3.array.hh + s3.array.cpp + s3.multiscale.array.hh + s3.multiscale.array.cpp plate.hh plate.cpp $ diff --git a/src/streaming/acquire.zarr.cpp b/src/streaming/acquire.zarr.cpp index dac07584..460c36ee 100644 --- a/src/streaming/acquire.zarr.cpp +++ b/src/streaming/acquire.zarr.cpp @@ -4,6 +4,7 @@ #include "zarr.stream.hh" #include // bit_ceil +#include // memcpy #include // uint32_t #include #include diff --git a/src/streaming/array.base.cpp b/src/streaming/array.base.cpp index 391f7068..d2ef195c 100644 --- a/src/streaming/array.base.cpp +++ b/src/streaming/array.base.cpp @@ -6,18 +6,12 @@ #include "multiscale.array.hh" zarr::ArrayBase::ArrayBase(std::shared_ptr config, - std::shared_ptr thread_pool, - std::shared_ptr file_handle_pool, - std::shared_ptr s3_connection_pool) + std::shared_ptr thread_pool) : config_(config) , thread_pool_(thread_pool) - , s3_connection_pool_(s3_connection_pool) - , file_handle_pool_(file_handle_pool) { CHECK(config_); // required CHECK(thread_pool_); // required - EXPECT(s3_connection_pool_ != nullptr || file_handle_pool_ != nullptr, - "Either S3 connection pool or file handle pool must be provided."); } std::string @@ -31,94 +25,6 @@ zarr::ArrayBase::node_path_() const return key; } -bool -zarr::ArrayBase::make_metadata_sinks_() -{ - metadata_sinks_.clear(); - - try { - const auto sink_keys = metadata_keys_(); - for (const auto& key : sink_keys) { - const std::string path = node_path_() + "/" + key; - std::unique_ptr sink = - config_->bucket_name - ? make_s3_sink(*config_->bucket_name, path, s3_connection_pool_) - : make_file_sink(path, file_handle_pool_); - - if (sink == nullptr) { - LOG_ERROR("Failed to create metadata sink for ", key); - return false; - } - metadata_sinks_.emplace(key, std::move(sink)); - } - } catch (const std::exception& exc) { - LOG_ERROR("Failed to create metadata sinks: ", exc.what()); - return false; - } - - return true; -} - -bool -zarr::ArrayBase::write_metadata_() -{ - if (!make_metadata_()) { - LOG_ERROR("Failed to make metadata."); - return false; - } - - if (!make_metadata_sinks_()) { - LOG_ERROR("Failed to make metadata sinks."); - return false; - } - - for (const auto& [key, metadata] : metadata_strings_) { - const auto it = metadata_sinks_.find(key); - if (it == metadata_sinks_.end()) { - LOG_ERROR("Metadata sink not found for key: ", key); - return false; - } - - auto& sink = it->second; - if (!sink) { - LOG_ERROR("Metadata sink is null for key: ", key); - return false; - } - - std::span data{ reinterpret_cast(metadata.data()), - metadata.size() }; - if (!sink->write(0, data)) { - LOG_ERROR("Failed to write metadata for key: ", key); - return false; - } - } - - return true; -} - -std::unique_ptr -zarr::make_array(std::shared_ptr config, - std::shared_ptr thread_pool, - std::shared_ptr file_handle_pool, - std::shared_ptr s3_connection_pool) -{ - // create a multiscale array at the dataset root (node_key is empty) or if - // we have a genuine multiscale dataset - const auto multiscale = - config->node_key.empty() || config->downsampling_method.has_value(); - - std::unique_ptr array; - if (multiscale) { - array = std::make_unique( - config, thread_pool, file_handle_pool, s3_connection_pool); - } else { - array = std::make_unique( - config, thread_pool, file_handle_pool, s3_connection_pool); - } - - return array; -} - bool zarr::finalize_array(std::unique_ptr&& array) { diff --git a/src/streaming/array.base.hh b/src/streaming/array.base.hh index dfa4075a..2a78ec55 100644 --- a/src/streaming/array.base.hh +++ b/src/streaming/array.base.hh @@ -2,10 +2,6 @@ #include "array.dimensions.hh" #include "blosc.compression.params.hh" -#include "file.handle.hh" -#include "locked.buffer.hh" -#include "s3.connection.hh" -#include "sink.hh" #include "thread.pool.hh" #include "zarr.types.h" @@ -56,9 +52,7 @@ class ArrayBase { public: ArrayBase(std::shared_ptr config, - std::shared_ptr thread_pool, - std::shared_ptr file_handle_pool, - std::shared_ptr s3_connection_pool); + std::shared_ptr thread_pool); virtual ~ArrayBase() = default; /** @@ -78,31 +72,30 @@ class ArrayBase * @param data The data to write. * @return The number of bytes successfully written. */ - [[nodiscard]] virtual size_t write_frame(LockedBuffer& data) = 0; + [[nodiscard]] virtual size_t write_frame(std::vector& data) = 0; protected: std::shared_ptr config_; std::shared_ptr thread_pool_; - std::shared_ptr s3_connection_pool_; - std::shared_ptr file_handle_pool_; - std::unordered_map metadata_strings_; - std::unordered_map> metadata_sinks_; + std::string last_written_metadata_; std::string node_path_() const; - [[nodiscard]] virtual bool make_metadata_() = 0; - virtual std::vector metadata_keys_() const = 0; - [[nodiscard]] bool make_metadata_sinks_(); - [[nodiscard]] bool write_metadata_(); + [[nodiscard]] virtual bool make_metadata_(std::string& metadata_str) = 0; + [[nodiscard]] virtual bool write_metadata_() = 0; friend bool finalize_array(std::unique_ptr&& array); }; +template std::unique_ptr make_array(std::shared_ptr config, std::shared_ptr thread_pool, - std::shared_ptr file_handle_pool, - std::shared_ptr s3_connection_pool); + Args&&... args) +{ + return std::make_unique( + config, thread_pool, std::forward(args)...); +} [[nodiscard]] bool finalize_array(std::unique_ptr&& array); diff --git a/src/streaming/array.cpp b/src/streaming/array.cpp index 7d4be0e6..1456774f 100644 --- a/src/streaming/array.cpp +++ b/src/streaming/array.cpp @@ -1,10 +1,9 @@ #include "array.hh" #include "macros.hh" -#include "sink.hh" #include "zarr.common.hh" +#include #include -#include #include // std::fill #include @@ -63,19 +62,17 @@ shuffle_to_string(uint8_t shuffle) } // namespace zarr::Array::Array(std::shared_ptr config, - std::shared_ptr thread_pool, - std::shared_ptr file_handle_pool, - std::shared_ptr s3_connection_pool) - : ArrayBase(config, thread_pool, file_handle_pool, s3_connection_pool) + std::shared_ptr thread_pool) + : ArrayBase(config, thread_pool) , bytes_to_flush_{ 0 } , frames_written_{ 0 } - , append_chunk_index_{ 0 } - , current_layer_{ 0 } + , append_shard_index_{ 0 } , is_closing_{ false } + , current_layer_{ 0 } { const size_t n_chunks = config_->dimensions->number_of_chunks_in_memory(); EXPECT(n_chunks > 0, "Array has zero chunks in memory"); - chunk_buffers_ = std::vector(n_chunks); + chunk_buffers_ = std::vector>(n_chunks); const auto& dims = config_->dimensions; const auto number_of_shards = dims->number_of_shards(); @@ -89,7 +86,7 @@ zarr::Array::Array(std::shared_ptr config, std::ranges::fill(table, std::numeric_limits::max()); } - data_root_ = node_path_() + "/c/" + std::to_string(append_chunk_index_); + data_root_ = node_path_() + "/c/" + std::to_string(append_shard_index_); } size_t @@ -104,7 +101,7 @@ zarr::Array::memory_usage() const noexcept } size_t -zarr::Array::write_frame(LockedBuffer& data) +zarr::Array::write_frame(std::vector& data) { const auto nbytes_data = data.size(); const auto nbytes_frame = @@ -119,7 +116,7 @@ zarr::Array::write_frame(LockedBuffer& data) return 0; } - if (bytes_to_flush_ == 0) { // first frame, we need to init the buffers + if (frames_written_ == 0) { // first frame, we need to init the buffers fill_buffers_(); } @@ -137,30 +134,26 @@ zarr::Array::write_frame(LockedBuffer& data) bytes_to_flush_ += bytes_written; ++frames_written_; - if (should_flush_()) { - CHECK(compress_and_flush_data_()); + if (should_flush_layer_()) { + EXPECT(compress_and_flush_data_(), "Failed to flush chunk layer data"); + bytes_to_flush_ = 0; + + const auto& dims = config_->dimensions; + const auto lps = dims->chunk_layers_per_shard(); + current_layer_ = (current_layer_ + 1) % lps; if (should_rollover_()) { - rollover_(); + close_shards_(); CHECK(write_metadata_()); } - bytes_to_flush_ = 0; } return bytes_written; } -std::vector -zarr::Array::metadata_keys_() const -{ - return { "zarr.json" }; -} - bool -zarr::Array::make_metadata_() +zarr::Array::make_metadata_(std::string& metadata_str) { - metadata_strings_.clear(); - std::vector array_shape, chunk_shape, shard_shape; const auto& dims = config_->dimensions; @@ -235,7 +228,7 @@ zarr::Array::make_metadata_() crc32_codec, }); - configuration["index_location"] = "end"; + configuration["index_location"] = index_location_(); configuration["codecs"] = json::array({ codec }); if (config_->compression_params) { @@ -260,7 +253,7 @@ zarr::Array::make_metadata_() metadata["codecs"] = codecs; - metadata_strings_.emplace("zarr.json", metadata.dump(4)); + metadata_str = metadata.dump(4); return true; } @@ -272,21 +265,17 @@ zarr::Array::close_() is_closing_ = true; try { if (bytes_to_flush_ > 0) { - CHECK(compress_and_flush_data_()); - } else { - CHECK(close_impl_()); + if (!compress_and_flush_data_()) { + LOG_ERROR("Failed to flush remaining data on close"); + return false; + } + bytes_to_flush_ = 0; } - close_sinks_(); + finalize_append_shard_(); if (frames_written_ > 0) { CHECK(write_metadata_()); - for (auto& [key, sink] : metadata_sinks_) { - EXPECT(zarr::finalize_sink(std::move(sink)), - "Failed to finalize metadata sink ", - key); - } } - metadata_sinks_.clear(); retval = true; } catch (const std::exception& exc) { LOG_ERROR("Failed to finalize array writer: ", exc.what()); @@ -296,129 +285,52 @@ zarr::Array::close_() return retval; } -bool -zarr::Array::close_impl_() +void +zarr::Array::make_data_paths_() { - if (current_layer_ == 0) { - return true; - } - - // write the table - const auto& dims = config_->dimensions; - const auto n_shards = dims->number_of_shards(); - std::vector> futures; - - std::atomic all_successful = 1; - - for (auto shard_idx = 0; shard_idx < n_shards; ++shard_idx) { - const std::string data_path = data_paths_[shard_idx]; - auto* file_offset = shard_file_offsets_.data() + shard_idx; - auto* shard_table = shard_tables_.data() + shard_idx; - - auto promise = std::make_shared>(); - futures.emplace_back(promise->get_future()); - - auto job = [shard_idx, - data_path, - shard_table, - file_offset, - promise, - &all_successful, - this](std::string& err) { - bool success = true; - - try { - std::unique_ptr sink; - - if (data_sinks_.contains( - data_path)) { // sink already constructed - sink = std::move(data_sinks_[data_path]); - data_sinks_.erase(data_path); - } else { - sink = make_data_sink_(data_path); - } - - if (sink == nullptr) { - err = "Failed to create sink for " + data_path; - success = false; - } else { - const auto table_size = - shard_table->size() * sizeof(uint64_t); - std::vector table(table_size + sizeof(uint32_t)); - - // copy the table data - memcpy(table.data(), shard_table->data(), table_size); - const auto* table_ptr = table.data(); - - // compute crc32 checksum of the table - const uint32_t checksum = - crc32c::Crc32c(table_ptr, table_size); - memcpy( - table.data() + table_size, &checksum, sizeof(uint32_t)); - - if (!sink->write(*file_offset, table)) { - err = "Failed to write table and checksum to shard " + - std::to_string(shard_idx); - success = false; - } + if (data_paths_.empty()) { + const auto& dimensions = config_->dimensions; + + std::queue paths_queue; + paths_queue.emplace(data_root_); + + // create intermediate paths + for (auto i = 1; // skip the last dimension + i < dimensions->ndims() - 1; // skip the x dimension + ++i) { + const auto& dim = dimensions->at(i); + const auto n_parts = shards_along_dimension(dim); + CHECK(n_parts); + + auto n_paths = paths_queue.size(); + for (auto j = 0; j < n_paths; ++j) { + const auto path = paths_queue.front(); + paths_queue.pop(); + + for (auto k = 0; k < n_parts; ++k) { + const auto kstr = std::to_string(k); + paths_queue.push(path + (path.empty() ? kstr : "/" + kstr)); } - } catch (const std::exception& exc) { - err = "Failed to flush data: " + std::string(exc.what()); - success = false; } + } - all_successful.fetch_and(success); - promise->set_value(); - - return success; - }; - - // one thread is reserved for processing the frame queue and runs the - // entire lifetime of the stream - if (thread_pool_->n_threads() == 1 || !thread_pool_->push_job(job)) { - if (std::string err; !job(err)) { - LOG_ERROR(err); + // create final paths + data_paths_.reserve(paths_queue.size() * + shards_along_dimension(dimensions->width_dim())); + { + const auto& dim = dimensions->width_dim(); + const auto n_parts = shards_along_dimension(dim); + CHECK(n_parts); + + auto n_paths = paths_queue.size(); + for (auto i = 0; i < n_paths; ++i) { + const auto path = paths_queue.front(); + paths_queue.pop(); + for (auto j = 0; j < n_parts; ++j) + data_paths_.push_back(path + "/" + std::to_string(j)); } } } - - return all_successful; -} - -bool -zarr::Array::is_s3_array_() const -{ - return config_->bucket_name.has_value(); -} - -void -zarr::Array::make_data_paths_() -{ - if (data_paths_.empty()) { - data_paths_ = construct_data_paths( - data_root_, *config_->dimensions, shards_along_dimension); - } -} - -std::unique_ptr -zarr::Array::make_data_sink_(std::string_view path) -{ - const auto is_s3 = is_s3_array_(); - - std::unique_ptr sink; - - // create parent directories if needed - if (is_s3) { - const auto bucket_name = *config_->bucket_name; - sink = make_s3_sink(bucket_name, path, s3_connection_pool_); - } else { - const auto parent_paths = get_parent_paths(data_paths_); - CHECK(make_dirs(parent_paths, thread_pool_)); - - sink = make_file_sink(path, file_handle_pool_); - } - - return sink; } void @@ -429,12 +341,13 @@ zarr::Array::fill_buffers_() const auto n_bytes = config_->dimensions->bytes_per_chunk(); for (auto& buf : chunk_buffers_) { - buf.resize_and_fill(n_bytes, 0); + buf.resize(n_bytes); // no-op if already that size + std::ranges::fill(buf, 0); } } size_t -zarr::Array::write_frame_to_chunks_(LockedBuffer& data) +zarr::Array::write_frame_to_chunks_(std::vector& data) { // break the frame into tiles and write them to the chunk buffers const auto bytes_per_px = bytes_of_type(config_->dtype); @@ -472,401 +385,87 @@ zarr::Array::write_frame_to_chunks_(LockedBuffer& data) size_t bytes_written = 0; const auto n_tiles = n_tiles_x * n_tiles_y; - auto frame = data.take(); + std::vector frame = std::move(data); #pragma omp parallel for reduction(+ : bytes_written) for (auto tile = 0; tile < n_tiles; ++tile) { auto& chunk_buffer = chunk_buffers_[tile + group_offset]; - bytes_written += chunk_buffer.with_lock([chunk_offset, - frame_rows, - frame_cols, - tile_rows, - tile_cols, - tile, - n_tiles_x, - bytes_per_px, - bytes_per_row, - bytes_per_chunk, - &frame](auto& chunk_data) { - const auto* data_ptr = frame.data(); - const auto data_size = frame.size(); - - const auto chunk_start = chunk_data.data(); - - const auto tile_idx_y = tile / n_tiles_x; - const auto tile_idx_x = tile % n_tiles_x; - - auto chunk_pos = chunk_offset; - size_t bytes_written = 0; - - for (auto k = 0; k < tile_rows; ++k) { - const auto frame_row = tile_idx_y * tile_rows + k; - if (frame_row < frame_rows) { - const auto frame_col = tile_idx_x * tile_cols; - - const auto region_width = - std::min(frame_col + tile_cols, frame_cols) - frame_col; - - const auto region_start = - bytes_per_px * (frame_row * frame_cols + frame_col); - const auto nbytes = region_width * bytes_per_px; - - // copy region - EXPECT(region_start + nbytes <= data_size, - "Buffer overflow in framme. Region start: ", - region_start, - " nbytes: ", - nbytes, - " data size: ", - data_size); - EXPECT(chunk_pos + nbytes <= bytes_per_chunk, - "Buffer overflow in chunk. Chunk pos: ", - chunk_pos, - " nbytes: ", - nbytes, - " bytes per chunk: ", - bytes_per_chunk); - memcpy( - chunk_start + chunk_pos, data_ptr + region_start, nbytes); - bytes_written += nbytes; - } - chunk_pos += bytes_per_row; - } - - return bytes_written; - }); - } - - data.assign(std::move(frame)); - - return bytes_written; -} - -ByteVector -zarr::Array::consolidate_chunks_(uint32_t shard_index) -{ - const auto& dims = config_->dimensions; - CHECK(shard_index < dims->number_of_shards()); - - const auto chunks_per_shard = dims->chunks_per_shard(); - const auto chunks_in_mem = dims->number_of_chunks_in_memory(); - const auto n_layers = dims->chunk_layers_per_shard(); - - const auto chunks_per_layer = chunks_per_shard / n_layers; - const auto layer_offset = current_layer_ * chunks_per_layer; - const auto chunk_offset = current_layer_ * chunks_in_mem; - - auto& shard_table = shard_tables_[shard_index]; - const auto file_offset = shard_file_offsets_[shard_index]; - shard_table[2 * layer_offset] = file_offset; - - uint64_t last_chunk_offset = shard_table[2 * layer_offset]; - uint64_t last_chunk_size = shard_table[2 * layer_offset + 1]; - size_t shard_size = last_chunk_size; - - for (auto i = 1; i < chunks_per_layer; ++i) { - const auto offset_idx = 2 * (layer_offset + i); - const auto size_idx = offset_idx + 1; - if (shard_table[size_idx] == std::numeric_limits::max()) { - continue; - } - - shard_table[offset_idx] = last_chunk_offset + last_chunk_size; - last_chunk_offset = shard_table[offset_idx]; - last_chunk_size = shard_table[size_idx]; - shard_size += last_chunk_size; - } - - std::vector shard_layer(shard_size); - - const auto chunk_indices_this_layer = - dims->chunk_indices_for_shard_layer(shard_index, current_layer_); - - size_t offset = 0; - for (const auto& idx : chunk_indices_this_layer) { - // this clears the chunk data out of the LockedBuffer - const auto chunk = chunk_buffers_[idx - chunk_offset].take(); - std::copy(chunk.begin(), chunk.end(), shard_layer.begin() + offset); - - offset += chunk.size(); - } - - EXPECT(offset == shard_size, - "Consolidated shard size does not match expected: ", - offset, - " != ", - shard_size); - - return std::move(shard_layer); -} - -bool -zarr::Array::compress_and_flush_data_() -{ - // construct paths to shard sinks if they don't already exist - if (data_paths_.empty()) { - make_data_paths_(); - } - - // create parent directories if needed - const auto is_s3 = is_s3_array_(); - if (!is_s3) { - const auto parent_paths = get_parent_paths(data_paths_); - CHECK(make_dirs(parent_paths, thread_pool_)); // no-op if they exist - } - - const auto& dims = config_->dimensions; - - const auto n_shards = dims->number_of_shards(); - CHECK(data_paths_.size() == n_shards); - - const auto chunks_in_memory = chunk_buffers_.size(); - const auto n_layers = dims->chunk_layers_per_shard(); - CHECK(n_layers > 0); - - const auto chunk_group_offset = current_layer_ * chunks_in_memory; - - std::atomic all_successful = 1; - - auto write_table = is_closing_ || should_rollover_(); - - std::vector> futures; - - // queue jobs to compress all chunks - const auto bytes_of_raw_chunk = config_->dimensions->bytes_per_chunk(); - const auto bytes_per_px = bytes_of_type(config_->dtype); - - for (auto i = 0; i < chunks_in_memory; ++i) { - auto promise = std::make_shared>(); - futures.emplace_back(promise->get_future()); - - const auto chunk_idx = i + chunk_group_offset; - const auto shard_idx = dims->shard_index_for_chunk(chunk_idx); - const auto internal_idx = dims->shard_internal_index(chunk_idx); - auto* shard_table = shard_tables_.data() + shard_idx; - - if (config_->compression_params) { - const auto compression_params = config_->compression_params.value(); - - auto job = [&chunk_buffer = chunk_buffers_[i], - bytes_per_px, - compression_params, - shard_table, - shard_idx, - chunk_idx, - internal_idx, - promise, - &all_successful](std::string& err) { - bool success = false; - - try { - if (!chunk_buffer.compress(compression_params, - bytes_per_px)) { - err = "Failed to compress chunk " + - std::to_string(chunk_idx) + " (internal index " + - std::to_string(internal_idx) + " of shard " + - std::to_string(shard_idx) + ")"; - } - - // update shard table with size - shard_table->at(2 * internal_idx + 1) = chunk_buffer.size(); - success = true; - } catch (const std::exception& exc) { - err = exc.what(); - } - - promise->set_value(); - - all_successful.fetch_and(static_cast(success)); - return success; - }; - - // one thread is reserved for processing the frame queue and runs - // the entire lifetime of the stream - if (thread_pool_->n_threads() == 1 || - !thread_pool_->push_job(job)) { - std::string err; - if (!job(err)) { - LOG_ERROR(err); - } + const auto* data_ptr = frame.data(); + const auto data_size = frame.size(); + + const auto chunk_start = chunk_buffer.data(); + + const auto tile_idx_y = tile / n_tiles_x; + const auto tile_idx_x = tile % n_tiles_x; + + auto chunk_pos = chunk_offset; + + for (auto k = 0; k < tile_rows; ++k) { + const auto frame_row = tile_idx_y * tile_rows + k; + if (frame_row < frame_rows) { + const auto frame_col = tile_idx_x * tile_cols; + + const auto region_width = + std::min(frame_col + tile_cols, frame_cols) - frame_col; + + const auto region_start = + bytes_per_px * (frame_row * frame_cols + frame_col); + const auto nbytes = region_width * bytes_per_px; + + // copy region + EXPECT(region_start + nbytes <= data_size, + "Buffer overflow in framme. Region start: ", + region_start, + " nbytes: ", + nbytes, + " data size: ", + data_size); + EXPECT(chunk_pos + nbytes <= bytes_per_chunk, + "Buffer overflow in chunk. Chunk pos: ", + chunk_pos, + " nbytes: ", + nbytes, + " bytes per chunk: ", + bytes_per_chunk); + memcpy( + chunk_start + chunk_pos, data_ptr + region_start, nbytes); + bytes_written += nbytes; } - } else { - // no compression, just update shard table with size - shard_table->at(2 * internal_idx + 1) = bytes_of_raw_chunk; + chunk_pos += bytes_per_row; } } - // if we're not compressing, there aren't any futures to wait for - for (auto& future : futures) { - future.wait(); - } - futures.clear(); - - const auto bucket_name = config_->bucket_name; - auto connection_pool = s3_connection_pool_; - - // wait for the chunks in each shard to finish compressing, then defragment - // and write the shard - for (auto shard_idx = 0; shard_idx < n_shards; ++shard_idx) { - const std::string data_path = data_paths_[shard_idx]; - auto* file_offset = shard_file_offsets_.data() + shard_idx; - auto* shard_table = shard_tables_.data() + shard_idx; - - auto promise = std::make_shared>(); - futures.emplace_back(promise->get_future()); - - auto job = [shard_idx, - is_s3, - data_path, - shard_table, - file_offset, - write_table, - bucket_name, - connection_pool, - promise, - &all_successful, - this](std::string& err) { - bool success = true; - std::unique_ptr sink; - - try { - // consolidate chunks in shard - const auto shard_data = consolidate_chunks_(shard_idx); - - if (data_sinks_.contains(data_path)) { // S3 sink, constructed - sink = std::move(data_sinks_[data_path]); - data_sinks_.erase(data_path); - } else { - sink = make_data_sink_(data_path); - } - - if (sink == nullptr) { - err = "Failed to create sink for " + data_path; - success = false; - } else { - success = sink->write(*file_offset, shard_data); - if (!success) { - err = "Failed to write shard at path " + data_path; - } else { - *file_offset += shard_data.size(); - - if (write_table) { - const size_t table_size = - shard_table->size() * sizeof(uint64_t); - std::vector table( - table_size + sizeof(uint32_t), 0); - - memcpy( - table.data(), shard_table->data(), table_size); - - // compute crc32 checksum of the table - const uint32_t checksum = - crc32c::Crc32c(table.data(), table_size); - memcpy(table.data() + table_size, - &checksum, - sizeof(uint32_t)); - - if (!sink->write(*file_offset, table)) { - err = "Failed to write table and checksum to " - "shard " + - std::to_string(shard_idx); - success = false; - } - } - } - } - } catch (const std::exception& exc) { - err = "Failed to flush data: " + std::string(exc.what()); - success = false; - } - - if (sink != nullptr) { - data_sinks_.emplace(data_path, std::move(sink)); - } + data = std::move(frame); - all_successful.fetch_and(success); - promise->set_value(); - - return success; - }; - - // one thread is reserved for processing the frame queue and runs the - // entire lifetime of the stream - if (thread_pool_->n_threads() == 1 || !thread_pool_->push_job(job)) { - std::string err; - if (!job(err)) { - LOG_ERROR(err); - } - } - } - - // wait for all threads to finish - for (auto& future : futures) { - future.wait(); - } - - // reset shard tables and file offsets - if (write_table) { - for (auto& table : shard_tables_) { - std::fill( - table.begin(), table.end(), std::numeric_limits::max()); - } - - std::fill(shard_file_offsets_.begin(), shard_file_offsets_.end(), 0); - current_layer_ = 0; - } else { - ++current_layer_; - } - - return static_cast(all_successful); + return bytes_written; } bool -zarr::Array::should_flush_() const +zarr::Array::should_flush_layer_() const { const auto& dims = config_->dimensions; - size_t frames_before_flush = dims->final_dim().chunk_size_px; - for (auto i = 1; i < dims->ndims() - 2; ++i) { - frames_before_flush *= dims->at(i).array_size_px; - } - - CHECK(frames_before_flush > 0); - return frames_written_ % frames_before_flush == 0; + const size_t frames_per_layer = dims->frames_per_layer(); + return frames_written_ % frames_per_layer == 0; } bool zarr::Array::should_rollover_() const { const auto& dims = config_->dimensions; - const auto& append_dim = dims->final_dim(); - size_t frames_before_flush = - append_dim.chunk_size_px * append_dim.shard_size_chunks; - for (auto i = 1; i < dims->ndims() - 2; ++i) { - frames_before_flush *= dims->at(i).array_size_px; - } - - CHECK(frames_before_flush > 0); - return frames_written_ % frames_before_flush == 0; + const size_t frames_per_shard = dims->frames_per_shard(); + return frames_written_ % frames_per_shard == 0; } void -zarr::Array::rollover_() +zarr::Array::close_shards_() { LOG_DEBUG("Rolling over"); - close_sinks_(); - ++append_chunk_index_; - data_root_ = node_path_() + "/c/" + std::to_string(append_chunk_index_); -} - -void -zarr::Array::close_sinks_() -{ - data_paths_.clear(); + finalize_append_shard_(); - for (auto& [path, sink] : data_sinks_) { - EXPECT( - finalize_sink(std::move(sink)), "Failed to finalize sink at ", path); + // advance to the next shard index + if (!is_closing_) { + data_root_ = + node_path_() + "/c/" + std::to_string(++append_shard_index_); } - data_sinks_.clear(); } diff --git a/src/streaming/array.dimensions.cpp b/src/streaming/array.dimensions.cpp index 81203551..2c42ad04 100644 --- a/src/streaming/array.dimensions.cpp +++ b/src/streaming/array.dimensions.cpp @@ -11,9 +11,12 @@ ArrayDimensions::ArrayDimensions(std::vector&& dims, , bytes_per_chunk_(zarr::bytes_of_type(dtype)) , number_of_chunks_in_memory_(1) { - EXPECT(dims_.size() > 2, "Array must have at least three dimensions."); + const auto ndims = dims_.size(); + EXPECT(ndims > 2, "Array must have at least three dimensions."); - for (auto i = 0; i < dims_.size(); ++i) { + frames_per_layer_ = final_dim().chunk_size_px; + + for (auto i = 0; i < ndims; ++i) { const auto& dim = dims_[i]; bytes_per_chunk_ *= dim.chunk_size_px; chunks_per_shard_ *= dim.shard_size_chunks; @@ -21,9 +24,20 @@ ArrayDimensions::ArrayDimensions(std::vector&& dims, if (i > 0) { number_of_chunks_in_memory_ *= zarr::chunks_along_dimension(dim); number_of_shards_ *= zarr::shards_along_dimension(dim); + if (i < ndims - 2) { + frames_per_layer_ *= dim.array_size_px; + } } } + EXPECT(number_of_chunks_in_memory_ > 0, + "Array must have at least one chunk in memory."); + EXPECT(chunks_per_shard_ > 0, + "Array must have at least one chunk per shard."); + EXPECT(number_of_shards_ > 0, "Array must have at least one shard."); + EXPECT(frames_per_layer_ > 0, + "Array must have at least one frame before flush."); + chunk_indices_for_shard_.resize(number_of_shards_); for (auto i = 0; i < chunks_per_shard_ * number_of_shards_; ++i) { @@ -216,6 +230,18 @@ ArrayDimensions::shard_internal_index(uint32_t chunk_index) const return shard_internal_indices_.at(chunk_index); } +uint64_t +ArrayDimensions::frames_per_layer() const +{ + return frames_per_layer_; +} + +uint64_t +ArrayDimensions::frames_per_shard() const +{ + return frames_per_layer_ * dims_[0].shard_size_chunks; +} + uint32_t ArrayDimensions::shard_index_for_chunk_(uint32_t chunk_index) const { diff --git a/src/streaming/array.dimensions.hh b/src/streaming/array.dimensions.hh index 3a806c09..35ace61a 100644 --- a/src/streaming/array.dimensions.hh +++ b/src/streaming/array.dimensions.hh @@ -115,7 +115,8 @@ class ArrayDimensions uint32_t chunk_layers_per_shard() const; /** - * @brief Get the shard index for a given chunk index, given array dimensions. + * @brief Get the shard index for a given chunk index, given array + * dimensions. * @param chunk_index The index of the chunk. * @return The index of the shard containing the chunk. */ @@ -132,8 +133,9 @@ class ArrayDimensions /** * @brief Get the chunk indices for a specific layer within a shard. * @param shard_index The index of the shard. - * @param layer - * @return + * @param layer The layer within the shard. + * @return A vector of chunk indices for the specified layer within the + * shard. */ std::vector chunk_indices_for_shard_layer(uint32_t shard_index, uint32_t layer) const; @@ -145,6 +147,23 @@ class ArrayDimensions */ uint32_t shard_internal_index(uint32_t chunk_index) const; + /** + * @brief Get the number of frames in a shard layer, i.e., the number of + * frames that can be stored in one layer of chunks within a shard. + * @note This is used to determine when to flush chunk buffers to storage. + * @return The number of frames in a shard layer. + */ + uint64_t frames_per_layer() const; + + /** + * @brief Get the number of frames per full shard, i.e., the number of + * frames that are written before closing and rolling over to a new shard. + * @details This is just the product of the number of frames per layer and + * the number of chunk layers per shard. + * @return The number of frames per shard. + */ + uint64_t frames_per_shard() const; + private: std::vector dims_; ZarrDataType dtype_; @@ -159,6 +178,8 @@ class ArrayDimensions std::unordered_map shard_internal_indices_; std::vector> chunk_indices_for_shard_; + uint64_t frames_per_layer_; + uint32_t shard_index_for_chunk_(uint32_t chunk_index) const; uint32_t shard_internal_index_(uint32_t chunk_index) const; }; diff --git a/src/streaming/array.hh b/src/streaming/array.hh index 4010dae0..cf3bc576 100644 --- a/src/streaming/array.hh +++ b/src/streaming/array.hh @@ -1,11 +1,6 @@ #pragma once #include "array.base.hh" -#include "blosc.compression.params.hh" -#include "definitions.hh" -#include "file.sink.hh" -#include "locked.buffer.hh" -#include "s3.connection.hh" #include "thread.pool.hh" namespace zarr { @@ -15,54 +10,85 @@ class Array : public ArrayBase { public: Array(std::shared_ptr config, - std::shared_ptr thread_pool, - std::shared_ptr file_handle_pool, - std::shared_ptr s3_connection_pool); + std::shared_ptr thread_pool); size_t memory_usage() const noexcept override; - [[nodiscard]] size_t write_frame(LockedBuffer&) override; + [[nodiscard]] size_t write_frame(std::vector&) override; protected: - /// Buffering - std::vector chunk_buffers_; - - /// Filesystem + std::vector> chunk_buffers_; std::vector data_paths_; - std::unordered_map> data_sinks_; - /// Bookkeeping uint64_t bytes_to_flush_; uint32_t frames_written_; - uint32_t append_chunk_index_; + uint32_t append_shard_index_; std::string data_root_; bool is_closing_; - /// Sharding uint32_t current_layer_; std::vector shard_file_offsets_; std::vector> shard_tables_; - std::vector metadata_keys_() const override; - bool make_metadata_() override; + bool make_metadata_(std::string& metadata) override; [[nodiscard]] bool close_() override; - [[nodiscard]] bool close_impl_(); - - bool is_s3_array_() const; + /** + * @brief Construct the data paths for all shards in the array with the + * current append shard index. + */ void make_data_paths_(); - [[nodiscard]] std::unique_ptr make_data_sink_(std::string_view path); + + /** + * @brief Fill the chunk buffers with empty data, resizing as needed. + */ void fill_buffers_(); - bool should_flush_() const; + /** + * @brief Determine if we should flush the current chunk buffers to storage. + * @return True if we should flush, false otherwise. + */ + bool should_flush_layer_() const; + + /** + * @brief Determine if we should roll over to a new shard along the append + * dimension. + * @return True if we should roll over, false otherwise. + */ bool should_rollover_() const; - size_t write_frame_to_chunks_(LockedBuffer& data); - - [[nodiscard]] ByteVector consolidate_chunks_(uint32_t shard_index); - [[nodiscard]] bool compress_and_flush_data_(); - void rollover_(); - void close_sinks_(); + /** + * @brief Write the given frame data into the chunk buffers. + * @param data The frame data. + * @return The number of bytes written. + */ + size_t write_frame_to_chunks_(std::vector& data); + + /** + * @brief Finalize all current shard files and close their associated I/O + * streams. Update the data root to point to the next shard index. + */ + void close_shards_(); + + /** + * @brief Return the location of the shard index for this array ("start" or + * "end"). + * @return The index location. + */ + virtual std::string index_location_() const = 0; + + /** + * @brief Compress and flush all data currently in the chunk buffers to the + * underlying storage. + * @return True on success, false on failure. + */ + [[nodiscard]] virtual bool compress_and_flush_data_() = 0; + + /** + * @brief Ensure all tables are flushed and close all open IO streams + * associated with this array. + */ + virtual void finalize_append_shard_() = 0; friend class MultiscaleArray; }; diff --git a/src/streaming/definitions.hh b/src/streaming/definitions.hh deleted file mode 100644 index c9b7ed1e..00000000 --- a/src/streaming/definitions.hh +++ /dev/null @@ -1,10 +0,0 @@ -#pragma once - -#include // uint8_t -#include -#include - -using ByteVector = std::vector; - -using ByteSpan = std::span; -using ConstByteSpan = std::span; diff --git a/src/streaming/downsampler.cpp b/src/streaming/downsampler.cpp index d0441569..8150c567 100644 --- a/src/streaming/downsampler.cpp +++ b/src/streaming/downsampler.cpp @@ -136,8 +136,8 @@ max2(const T& a, const T& b) } template -[[nodiscard]] ByteVector -scale_image(ConstByteSpan src, +[[nodiscard]] std::vector +scale_image(std::span src, size_t& width, size_t& height, ZarrDownsamplingMethod method) @@ -177,7 +177,7 @@ scale_image(ConstByteSpan src, const auto h_pad = height + (height % downscale); const auto size_downscaled = w_pad * h_pad * bytes_of_type / factor; - ByteVector dst(size_downscaled, 0); + std::vector dst(size_downscaled, 0); auto* dst_as_T = reinterpret_cast(dst.data()); auto* src_as_T = reinterpret_cast(src.data()); @@ -206,8 +206,8 @@ scale_image(ConstByteSpan src, template void -average_two_frames(ByteVector& dst, - ConstByteSpan src, +average_two_frames(std::vector& dst, + std::span src, ZarrDownsamplingMethod method) { T (*average_fun)(const T&, const T&) = nullptr; @@ -303,111 +303,108 @@ zarr::Downsampler::Downsampler(std::shared_ptr config, } void -zarr::Downsampler::add_frame(LockedBuffer& frame) +zarr::Downsampler::add_frame(std::vector& frame) { const auto& base_dims = writer_configurations_[0]->dimensions; size_t frame_width = base_dims->width_dim().array_size_px; size_t frame_height = base_dims->height_dim().array_size_px; - frame.with_lock([&](const auto& data) { - ByteVector current_frame(data.begin(), data.end()); - ByteVector next_level_frame; - - for (auto level = 1; level < n_levels_(); ++level) { - const auto& prev_dims = - writer_configurations_[level - 1]->dimensions; - const auto prev_width = prev_dims->width_dim().array_size_px; - const auto prev_height = prev_dims->height_dim().array_size_px; - const auto prev_planes = - prev_dims->at(prev_dims->ndims() - 3).array_size_px; - - EXPECT(prev_width == frame_width && prev_height == frame_height, - "Frame dimensions do not match expected dimensions: ", - prev_width, - "x", - prev_height, - " vs. ", - frame_width, - "x", - frame_height); - - const auto& next_dims = writer_configurations_[level]->dimensions; - const auto next_width = next_dims->width_dim().array_size_px; - const auto next_height = next_dims->height_dim().array_size_px; - const auto next_planes = - next_dims->at(next_dims->ndims() - 3).array_size_px; - - // only downsample if this level's XY size is smaller than the last - if (next_width < prev_width || next_height < prev_height) { - next_level_frame = - scale_fun_(current_frame, frame_width, frame_height, method_); - } else { - next_level_frame.assign(current_frame.begin(), - current_frame.end()); - } + // frame.with_lock([&](const auto& data) { + std::vector current_frame(frame.begin(), frame.end()); + std::vector next_level_frame; + + for (auto level = 1; level < n_levels_(); ++level) { + const auto& prev_dims = writer_configurations_[level - 1]->dimensions; + const auto prev_width = prev_dims->width_dim().array_size_px; + const auto prev_height = prev_dims->height_dim().array_size_px; + const auto prev_planes = + prev_dims->at(prev_dims->ndims() - 3).array_size_px; + + EXPECT(prev_width == frame_width && prev_height == frame_height, + "Frame dimensions do not match expected dimensions: ", + prev_width, + "x", + prev_height, + " vs. ", + frame_width, + "x", + frame_height); + + const auto& next_dims = writer_configurations_[level]->dimensions; + const auto next_width = next_dims->width_dim().array_size_px; + const auto next_height = next_dims->height_dim().array_size_px; + const auto next_planes = + next_dims->at(next_dims->ndims() - 3).array_size_px; + + // only downsample if this level's XY size is smaller than the last + if (next_width < prev_width || next_height < prev_height) { + next_level_frame = + scale_fun_(current_frame, frame_width, frame_height, method_); + } else { + next_level_frame.assign(current_frame.begin(), current_frame.end()); + } - EXPECT(next_width == frame_width && next_height == frame_height, - "Downsampled dimensions do not match expected dimensions: ", - next_width, - "x", - next_height, - " vs. ", - frame_width, - "x", - frame_height); - - // if the Z dimension is spatial, and has an odd number of planes, - // and this is the last plane, we don't want to queue it up to be - // averaged with the first frame of the next timepoint - bool average_this_frame = next_planes < prev_planes; - if (prev_planes % 2 != 0 && - level_frame_count_.at(level - 1) % prev_planes == 0) { - average_this_frame = false; - } + EXPECT(next_width == frame_width && next_height == frame_height, + "Downsampled dimensions do not match expected dimensions: ", + next_width, + "x", + next_height, + " vs. ", + frame_width, + "x", + frame_height); + + // if the Z dimension is spatial, and has an odd number of planes, + // and this is the last plane, we don't want to queue it up to be + // averaged with the first frame of the next timepoint + bool average_this_frame = next_planes < prev_planes; + if (prev_planes % 2 != 0 && + level_frame_count_.at(level - 1) % prev_planes == 0) { + average_this_frame = false; + } - // only average if this level's Z size is smaller than the last - // and if we are not at the last frame of the previous level - if (average_this_frame) { - auto it = partial_scaled_frames_.find(level); - if (it != partial_scaled_frames_.end()) { - // average2_fun_ writes to next_level_frame - // swap here so that decimate2 can take it->second - next_level_frame.swap(it->second); - average2_fun_(next_level_frame, it->second, method_); - emplace_downsampled_frame_(level, next_level_frame); - - // clean up this LOD - partial_scaled_frames_.erase(it); - - // set up for next iteration - if (level + 1 < writer_configurations_.size()) { - current_frame.assign(next_level_frame.begin(), - next_level_frame.end()); - } - } else { - partial_scaled_frames_.emplace(level, next_level_frame); - break; - } - } else { - // no downsampling in Z, so we can just pass the data to the - // next level + // only average if this level's Z size is smaller than the last + // and if we are not at the last frame of the previous level + if (average_this_frame) { + auto it = partial_scaled_frames_.find(level); + if (it != partial_scaled_frames_.end()) { + // average2_fun_ writes to next_level_frame + // swap here so that decimate2 can take it->second + next_level_frame.swap(it->second); + average2_fun_(next_level_frame, it->second, method_); emplace_downsampled_frame_(level, next_level_frame); + // clean up this LOD + partial_scaled_frames_.erase(it); + + // set up for next iteration if (level + 1 < writer_configurations_.size()) { current_frame.assign(next_level_frame.begin(), next_level_frame.end()); } + } else { + partial_scaled_frames_.emplace(level, next_level_frame); + break; + } + } else { + // no downsampling in Z, so we can just pass the data to the + // next level + emplace_downsampled_frame_(level, next_level_frame); + + if (level + 1 < writer_configurations_.size()) { + current_frame.assign(next_level_frame.begin(), + next_level_frame.end()); } } - }); + } } bool -zarr::Downsampler::take_frame(int level, LockedBuffer& frame_data) +zarr::Downsampler::take_frame(int level, std::vector& frame_data) { auto it = downsampled_frames_.find(level); if (it != downsampled_frames_.end()) { - frame_data.assign(it->second); + frame_data.assign(it->second.begin(), it->second.end()); downsampled_frames_.erase(level); return true; } @@ -595,8 +592,9 @@ zarr::Downsampler::make_writer_configurations_( } void -zarr::Downsampler::emplace_downsampled_frame_(int level, - const ByteVector& frame_data) +zarr::Downsampler::emplace_downsampled_frame_( + int level, + const std::vector& frame_data) { downsampled_frames_.emplace(level, frame_data); ++level_frame_count_.at(level); diff --git a/src/streaming/downsampler.hh b/src/streaming/downsampler.hh index a90d1e5c..d43b0947 100644 --- a/src/streaming/downsampler.hh +++ b/src/streaming/downsampler.hh @@ -2,7 +2,6 @@ #include "array.hh" #include "array.dimensions.hh" -#include "definitions.hh" #include "nlohmann/json.hpp" @@ -21,7 +20,7 @@ class Downsampler * level, by calling take_frame(). * @param frame The full-resolution frame data. */ - void add_frame(LockedBuffer& frame); + void add_frame(std::vector& frame); /** * @brief Get the downsampled frame for the given level, removing it from @@ -32,7 +31,7 @@ class Downsampler * @param[out] frame_data The downsampled frame data. * @return True if the downsampled frame was found, false otherwise. */ - bool take_frame(int level, LockedBuffer& frame_data); + bool take_frame(int level, std::vector& frame_data); const std::unordered_map>& writer_configurations() const; @@ -41,10 +40,12 @@ class Downsampler nlohmann::json get_metadata() const; private: - using ScaleFunT = std::function< - ByteVector(ConstByteSpan, size_t&, size_t&, ZarrDownsamplingMethod)>; - using Average2FunT = - std::function; + using ScaleFunT = std::function(std::span, + size_t&, + size_t&, + ZarrDownsamplingMethod)>; + using Average2FunT = std::function< + void(std::vector&, std::span, ZarrDownsamplingMethod)>; ZarrDownsamplingMethod method_; @@ -53,13 +54,13 @@ class Downsampler std::unordered_map> writer_configurations_; - std::unordered_map downsampled_frames_; - std::unordered_map partial_scaled_frames_; + std::unordered_map> downsampled_frames_; + std::unordered_map> partial_scaled_frames_; std::unordered_map level_frame_count_; size_t n_levels_() const; void make_writer_configurations_(std::shared_ptr config); - void emplace_downsampled_frame_(int level, const ByteVector& frame_data); + void emplace_downsampled_frame_(int level, const std::vector& frame_data); }; } // namespace zarr \ No newline at end of file diff --git a/src/streaming/file.handle.cpp b/src/streaming/file.handle.cpp index 34671a4a..0f7eab03 100644 --- a/src/streaming/file.handle.cpp +++ b/src/streaming/file.handle.cpp @@ -1,7 +1,5 @@ -#include "definitions.hh" #include "file.handle.hh" - -#include +#include "macros.hh" void* init_handle(const std::string& filename, void* flags); @@ -33,32 +31,76 @@ zarr::FileHandle::get() const zarr::FileHandlePool::FileHandlePool() : max_active_handles_(get_max_active_handles()) - , n_active_handles_(0) { } -std::unique_ptr +zarr::FileHandlePool::~FileHandlePool() +{ + // wait until the pool has been drained + std::unique_lock lock(mutex_); + while (!handle_map_.empty()) { + if (!evict_idle_handle_()) { + cv_.wait(lock, [&] { return true; }); + } + } +} + +std::shared_ptr zarr::FileHandlePool::get_handle(const std::string& filename, void* flags) { std::unique_lock lock(mutex_); - if (n_active_handles_ >= max_active_handles_) { - cv_.wait(lock, - [this]() { return n_active_handles_ < max_active_handles_; }); + if (const auto it = handle_map_.find(filename); it != handle_map_.end()) { + if (auto handle = it->second->second.lock()) { + // move to front of list + handles_.splice(handles_.begin(), handles_, it->second); + return handle; + } + + // expired, remove from list and map + handles_.erase(it->second); + handle_map_.erase(it); } - ++n_active_handles_; - return std::make_unique(filename, flags); + cv_.wait(lock, [&] { return handles_.size() < max_active_handles_; }); + std::shared_ptr handle(init_handle(filename, flags), [](void* h) { + flush_file(h); + destroy_handle(h); + }); + + EXPECT(handle != nullptr, "Failed to create file handle for " + filename); + + handles_.emplace_front(filename, handle); + handle_map_.emplace(filename, handles_.begin()); + + return handle; } void -zarr::FileHandlePool::return_handle(std::unique_ptr&& handle) +zarr::FileHandlePool::close_handle(const std::string& filename) { std::unique_lock lock(mutex_); + if (const auto it = handle_map_.find(filename); it != handle_map_.end()) { + handles_.erase(it->second); + handle_map_.erase(it); + cv_.notify_all(); + } +} + +bool +zarr::FileHandlePool::evict_idle_handle_() +{ + bool evicted = false; + for (auto it = handles_.begin(); it != handles_.end();) { + if (it->second.expired()) { + handle_map_.erase(it->first); + it = handles_.erase(it); + evicted = true; + } + } - if (handle != nullptr && n_active_handles_ > 0) { - --n_active_handles_; + if (evicted) { + cv_.notify_all(); } - // handle will be destroyed when going out of scope - flush_file(handle->get()); + return evicted; } diff --git a/src/streaming/file.handle.hh b/src/streaming/file.handle.hh index ea2e94ee..acbf811e 100644 --- a/src/streaming/file.handle.hh +++ b/src/streaming/file.handle.hh @@ -1,9 +1,11 @@ #pragma once #include +#include #include // for std::unique_ptr #include #include +#include namespace zarr { /** @@ -44,7 +46,7 @@ class FileHandlePool { public: FileHandlePool(); - ~FileHandlePool() = default; + ~FileHandlePool(); /** * @brief Get a file handle for the specified filename. @@ -52,23 +54,29 @@ class FileHandlePool * been reached, until a handle is returned to the pool. * @param filename The path to the file to open. * @param flags Platform-specific flags for opening the file. - * @return A unique pointer to a FileHandle, or nullptr on failure. + * @return A shared pointer to a file handle, or nullptr on failure. */ - std::unique_ptr get_handle(const std::string& filename, - void* flags); + std::shared_ptr get_handle(const std::string& filename, void* flags); /** - * @brief Return a file handle to the pool. - * @details This function should be called when a file handle is no longer - * needed, to allow other threads to acquire a handle. - * @param handle The file handle to return. + * @brief Close the handle for the specified filename, if it exists in the + * pool. This will remove the handle from the pool and close the underlying + * file. + * @param filename The path to the file whose handle should be closed. */ - void return_handle(std::unique_ptr&& handle); + void close_handle(const std::string& filename); private: + using HandleEntry = std::pair>; + using HandleList = std::list; + const uint64_t max_active_handles_; - std::atomic n_active_handles_; + HandleList handles_; + std::unordered_map handle_map_; + std::mutex mutex_; std::condition_variable cv_; + + bool evict_idle_handle_(); }; } // namespace zarr \ No newline at end of file diff --git a/src/streaming/file.sink.cpp b/src/streaming/file.sink.cpp deleted file mode 100644 index 20fbeec3..00000000 --- a/src/streaming/file.sink.cpp +++ /dev/null @@ -1,76 +0,0 @@ -#include "file.sink.hh" -#include "macros.hh" - -#include - -void* -make_flags(); - -void -destroy_flags(void*); - -bool -seek_and_write(void* handle, size_t offset, ConstByteSpan data); - -bool -flush_file(void* handle); - -zarr::FileSink::FileSink(std::string_view filename, - std::shared_ptr file_handle_pool) - : file_handle_pool_(file_handle_pool) - , filename_(filename) - , flags_(make_flags()) -{ - EXPECT(file_handle_pool_ != nullptr, "File handle pool not provided."); -} - -zarr::FileSink::~FileSink() -{ - destroy_flags(flags_); - flags_ = nullptr; -} - -bool -zarr::FileSink::write(size_t offset, ConstByteSpan data) -{ - if (data.data() == nullptr || data.size() == 0) { - return true; - } - - auto handle = file_handle_pool_->get_handle(filename_, flags_); - if (handle == nullptr) { - LOG_ERROR("Failed to get file handle for ", filename_); - return false; - } - - bool retval = false; - try { - retval = seek_and_write(handle->get(), offset, data); - } catch (const std::exception& exc) { - LOG_ERROR("Failed to write to file ", filename_, ": ", exc.what()); - } - - file_handle_pool_->return_handle(std::move(handle)); - - return retval; -} - -bool -zarr::FileSink::flush_() -{ - auto handle = file_handle_pool_->get_handle(filename_, flags_); - if (handle == nullptr) { - LOG_ERROR("Failed to get file handle for ", filename_); - return false; - } - - bool retval = false; - try { - retval = flush_file(handle->get()); - } catch (const std::exception& exc) { - LOG_ERROR("Failed to flush file ", filename_, ": ", exc.what()); - } - file_handle_pool_->return_handle(std::move(handle)); - - return retval; -} \ No newline at end of file diff --git a/src/streaming/file.sink.hh b/src/streaming/file.sink.hh deleted file mode 100644 index 49b06f38..00000000 --- a/src/streaming/file.sink.hh +++ /dev/null @@ -1,28 +0,0 @@ -#pragma once - -#include "file.handle.hh" -#include "sink.hh" - -#include -#include - -namespace zarr { -class FileSink : public Sink -{ - public: - FileSink(std::string_view filename, - std::shared_ptr file_handle_pool); - ~FileSink() override; - - bool write(size_t offset, ConstByteSpan data) override; - - protected: - bool flush_() override; - - private: - std::shared_ptr file_handle_pool_; - - std::string filename_; - void* flags_; -}; -} // namespace zarr diff --git a/src/streaming/frame.queue.cpp b/src/streaming/frame.queue.cpp index 06742e0a..81af22fd 100644 --- a/src/streaming/frame.queue.cpp +++ b/src/streaming/frame.queue.cpp @@ -19,7 +19,7 @@ zarr::FrameQueue::FrameQueue(size_t num_frames, size_t avg_frame_size) } bool -zarr::FrameQueue::push(LockedBuffer& frame, const std::string& key) +zarr::FrameQueue::push(std::vector& frame, const std::string& key) { std::unique_lock lock(mutex_); size_t write_pos = write_pos_.load(std::memory_order_relaxed); @@ -39,7 +39,7 @@ zarr::FrameQueue::push(LockedBuffer& frame, const std::string& key) } bool -zarr::FrameQueue::pop(LockedBuffer& frame, std::string& key) +zarr::FrameQueue::pop(std::vector& frame, std::string& key) { std::unique_lock lock(mutex_); size_t read_pos = read_pos_.load(std::memory_order_relaxed); diff --git a/src/streaming/frame.queue.hh b/src/streaming/frame.queue.hh index 07a02428..c17370ed 100644 --- a/src/streaming/frame.queue.hh +++ b/src/streaming/frame.queue.hh @@ -1,8 +1,5 @@ #pragma once -#include "definitions.hh" -#include "locked.buffer.hh" - #include #include #include @@ -16,8 +13,8 @@ class FrameQueue explicit FrameQueue(size_t num_frames, size_t avg_frame_size); ~FrameQueue() = default; - bool push(LockedBuffer& frame, const std::string& key); - bool pop(LockedBuffer& frame, std::string& key); + bool push(std::vector& frame, const std::string& key); + bool pop(std::vector& frame, std::string& key); size_t size() const; size_t bytes_used() const; @@ -29,7 +26,7 @@ class FrameQueue struct Frame { std::string key; - LockedBuffer data; + std::vector data; std::atomic ready{ false }; }; diff --git a/src/streaming/fs.array.cpp b/src/streaming/fs.array.cpp new file mode 100644 index 00000000..79df421a --- /dev/null +++ b/src/streaming/fs.array.cpp @@ -0,0 +1,428 @@ +#include "fs.array.hh" +#include "macros.hh" +#include "zarr.common.hh" + +#include +#include + +#include // memcp +#include +#include +#include +#include + +void* +make_flags(); + +void +destroy_flags(void* flags); + +bool +seek_and_write(void* handle, size_t offset, std::span data); + +namespace fs = std::filesystem; + +namespace { +std::vector +get_parent_paths(const std::vector& file_paths) +{ + std::unordered_set unique_paths; + for (const auto& file_path : file_paths) { + unique_paths.emplace(fs::path(file_path).parent_path().string()); + } + + return { unique_paths.begin(), unique_paths.end() }; +} + +bool +make_dirs(const std::vector& dir_paths, + std::shared_ptr thread_pool) +{ + if (dir_paths.empty()) { + return true; + } + EXPECT(thread_pool, "Thread pool not provided."); + + std::atomic all_successful = 1; + const std::unordered_set unique_paths(dir_paths.begin(), dir_paths.end()); + + std::vector> futures; + + for (const auto& path : unique_paths) { + auto promise = std::make_shared>(); + futures.emplace_back(promise->get_future()); + + auto job = [path, promise, &all_successful](std::string& err) { + bool success = true; + try { + if (fs::is_directory(path) || path.empty()) { + promise->set_value(); + return success; + } + + std::error_code ec; + if (!fs::create_directories(path, ec) && + !fs::is_directory(path)) { + err = "Failed to create directory '" + path + + "': " + ec.message(); + success = false; + } + } catch (const std::exception& exc) { + err = + "Failed to create directory '" + path + "': " + exc.what(); + success = false; + } + + promise->set_value(); + all_successful.fetch_and(success); + return success; + }; + + if (thread_pool->n_threads() == 1 || !thread_pool->push_job(job)) { + if (std::string err; !job(err)) { + LOG_ERROR(err); + } + } + } + + // wait for all jobs to finish + for (auto& future : futures) { + future.wait(); + } + + return all_successful; +} +} // namespace + +bool +zarr::FSArray::ShardFile::close() +{ + // finish writing chunks + for (auto& future : chunk_futures) { + future.wait(); + } + chunk_futures.clear(); + + // compute table checksum and write it out + const size_t table_size = table.size() * sizeof(uint64_t); + const auto* table_data = reinterpret_cast(table.data()); + const uint32_t checksum = crc32c::Crc32c(table_data, table_size); + + const size_t table_buffer_size = table.size() * sizeof(uint64_t); + constexpr size_t checksum_size = sizeof(uint32_t); + + std::vector table_buffer(table_buffer_size + checksum_size); + memcpy(table_buffer.data(), table_data, table_size); + memcpy(table_buffer.data() + table_buffer_size, &checksum, checksum_size); + + if (!seek_and_write(handle.get(), 0, table_buffer)) { + LOG_ERROR("Failed to write table and checksum for shard at ", path); + return false; + } + + return true; +} + +zarr::FSArray::FSArray(std::shared_ptr config, + std::shared_ptr thread_pool, + std::shared_ptr file_handle_pool) + : Array(config, thread_pool) + , FSStorage(file_handle_pool) + , table_size_bytes_(config->dimensions->chunks_per_shard() * 2 * + sizeof(uint64_t) + + sizeof(uint32_t)) +{ + std::ranges::fill(shard_file_offsets_, table_size_bytes_); +} + +bool +zarr::FSArray::write_metadata_() +{ + std::string metadata; + if (!make_metadata_(metadata)) { + LOG_ERROR("Failed to make metadata."); + return false; + } + + if (last_written_metadata_ == metadata) { + return true; // no changes + } + const std::string path = node_path_() + "/zarr.json"; + + bool success; + if ((success = write_string(path, metadata, 0))) { + last_written_metadata_ = metadata; + } + + return success; +} + +std::string +zarr::FSArray::index_location_() const +{ + return "start"; +} + +bool +zarr::FSArray::compress_and_flush_data_() +{ + const auto& dims = config_->dimensions; + const uint32_t chunks_per_shard = dims->chunks_per_shard(); + const auto n_shards = dims->number_of_shards(); + + // construct paths to shard sinks if they don't already exist + if (data_paths_.empty()) { + make_data_paths_(); + CHECK(data_paths_.size() == n_shards); + + // create parent directories if needed + const auto parent_paths = get_parent_paths(data_paths_); + CHECK(make_dirs(parent_paths, thread_pool_)); // no-op if they exist + + // create shard files + std::unique_lock lock(shard_files_mutex_); + for (const auto& path : data_paths_) { + auto shard_file = std::make_shared(); + shard_file->path = path; + shard_file->handle = get_handle_(path); + shard_file->table = std::vector( + 2 * chunks_per_shard, std::numeric_limits::max()); + shard_file->file_offset = table_size_bytes_; + + shard_files_[path] = std::move(shard_file); + } + } + + const uint32_t chunks_in_mem = dims->number_of_chunks_in_memory(); + const uint32_t n_layers = dims->chunk_layers_per_shard(); + const uint32_t chunks_per_layer = chunks_per_shard / n_layers; + + const size_t bytes_per_px = bytes_of_type(config_->dtype); + + // this layer's entries in the shard table begin here + const uint32_t layer_offset = current_layer_ * chunks_per_layer; + + // this layer's entries in the (global) chunk grid begin here + const uint32_t chunk_offset = current_layer_ * chunks_in_mem; + + for (auto shard_idx = 0; shard_idx < n_shards; ++shard_idx) { + const std::string data_path = data_paths_[shard_idx]; + auto shard_file = shard_files_[data_path]; + + // chunk storage is at chunk_index - chunk_offset + const auto chunk_indices_this_layer = + dims->chunk_indices_for_shard_layer(shard_idx, current_layer_); + + const auto& params = config_->compression_params; + + const size_t future_offset = shard_file->chunk_futures.size(); + shard_file->chunk_futures.resize(shard_file->chunk_futures.size() + + chunk_indices_this_layer.size()); + +#pragma omp parallel for + for (auto i = 0; i < chunk_indices_this_layer.size(); ++i) { + const uint32_t chunk_idx = chunk_indices_this_layer[i]; + CHECK(chunk_idx >= chunk_offset); + uint32_t internal_index = dims->shard_internal_index(chunk_idx); + auto promise = + std::make_shared>(); // TODO (not a shared + // pointer and std::move?) + + auto& chunk_data = chunk_buffers_[chunk_idx - chunk_offset]; + const size_t bytes_of_chunk = chunk_data.size(); + + shard_file->chunk_futures[i + future_offset] = promise->get_future(); + // shard_file->chunk_futures.push_back(promise->get_future()); + + auto job = [chunk_data = std::move(chunk_data), + ¶ms, + shard_file, + bytes_per_px, + internal_index, + promise](std::string& err) { + bool success = true; + std::vector compressed; + const uint8_t* data_out = nullptr; + uint64_t chunk_size_out = 0; + + try { + // compress here + if (params) { + compressed.resize(chunk_data.size() + + BLOSC_MAX_OVERHEAD); + const auto n_bytes_compressed = + blosc_compress_ctx(params->clevel, + params->shuffle, + bytes_per_px, + chunk_data.size(), + chunk_data.data(), + compressed.data(), + compressed.size(), + params->codec_id.c_str(), + 0, + 1); + if (n_bytes_compressed <= 0) { + err = "blosc_compress_ctx failed with code " + + std::to_string(n_bytes_compressed) + + " for chunk " + + std::to_string(internal_index) + " of shard "; + success = false; + } + data_out = compressed.data(); + chunk_size_out = n_bytes_compressed; + } else { + data_out = chunk_data.data(); + chunk_size_out = chunk_data.size(); + } + EXPECT(success, err); + EXPECT(data_out != nullptr, err); + EXPECT(chunk_size_out != 0, err); + + uint64_t file_offset_local; + { + std::lock_guard lock(shard_file->offset_mutex); + file_offset_local = shard_file->file_offset; + shard_file->file_offset += chunk_size_out; + } + + // write data + success = + seek_and_write(shard_file->handle.get(), + file_offset_local, + std::span(data_out, chunk_size_out)); + EXPECT(success, + "Failed to write chunk data to ", + shard_file->path, + " internal index ", + internal_index); + + // write table entry + shard_file->table[2 * internal_index] = file_offset_local; + shard_file->table[2 * internal_index + 1] = chunk_size_out; + } catch (const std::exception& exc) { + err = "Failed to write chunk " + + std::to_string(internal_index) + " of shard at " + + shard_file->path + ": " + exc.what(); + success = false; + } + + promise->set_value(); + return success; + }; + + // one thread is reserved for processing the frame queue and runs + // the entire lifetime of the stream + if (thread_pool_->n_threads() == 1 || + !thread_pool_->push_job(job)) { + if (std::string err; !job(err)) { + LOG_ERROR(err); + } + } + + if (!is_closing_) { + auto& chunk = chunk_buffers_[chunk_idx - chunk_offset]; + chunk.resize(bytes_of_chunk); + std::ranges::fill(chunk, 0); + } + } + + // if we're about to roll over to a new append shard, signal that we're + // not going to add any more chunks and that we can wait to close + // if (current_layer_ == n_layers - 1) { + // auto job = [shard_file, this](std::string& err) -> bool { + // bool success; + // + // try { + // success = shard_file->close(); + // std::unique_lock lock(shard_files_mutex_); + // shard_files_.erase(shard_file->path); + // file_handle_pool_->close_handle(shard_file->path); + // shard_files_cv_.notify_all(); + // } catch (const std::exception& exc) { + // err = exc.what(); + // success = false; + // } + // + // return success; + // }; + // + // // one thread is reserved for processing the frame queue and runs + // // the entire lifetime of the stream + // if (thread_pool_->n_threads() == 1 || + // !thread_pool_->push_job(job)) { + // if (std::string err; !job(err)) { + // LOG_ERROR(err); + // } + // } + // } + } + + return true; +} + +void +zarr::FSArray::finalize_append_shard_() +{ + data_paths_.clear(); + + if (is_closing_) { + // close all shards + for (auto& shard_file : shard_files_ | std::views::values) { + EXPECT(shard_file->close(), + "Failed to close shard file at path ", + shard_file->path); + } + shard_files_.clear(); + // wait on all the shards to be written out + // std::unique_lock lock(shard_files_mutex_); + // shard_files_cv_.wait(lock, [this] { return shard_files_.empty(); }); + } +} + +std::shared_ptr +zarr::FSArray::get_handle_(const std::string& path) +{ + std::unique_lock lock(handles_mutex_); + if (const auto it = handles_.find(path); it != handles_.end()) { + return it->second; + } + + void* flags = make_flags(); + const auto handle = file_handle_pool_->get_handle(path, flags); + destroy_flags(flags); + + handles_.emplace(path, handle); + return handle; +} + +void +zarr::FSArray::write_table_entries_(uint32_t shard_idx) +{ + CHECK(shard_idx < shard_tables_.size()); + const auto& path = data_paths_[shard_idx]; + const auto handle = get_handle_(path); + + EXPECT( + handle != nullptr, "Failed to get file handle for finalizing ", path); + + // compute table checksum and write it out + auto& shard_table = shard_tables_[shard_idx]; + const size_t table_size = shard_table.size() * sizeof(uint64_t); + const auto* table_data = + reinterpret_cast(shard_table.data()); + const uint32_t checksum = crc32c::Crc32c(table_data, table_size); + + const size_t table_buffer_size = shard_table.size() * sizeof(uint64_t); + constexpr size_t checksum_size = sizeof(uint32_t); + + std::vector table_buffer(table_buffer_size + checksum_size); + memcpy(table_buffer.data(), table_data, table_size); + memcpy(table_buffer.data() + table_buffer_size, &checksum, checksum_size); + + EXPECT(seek_and_write(handle.get(), 0, table_buffer), + "Failed to write final checksum for shard at ", + path); + + std::ranges::fill(shard_table, std::numeric_limits::max()); + shard_file_offsets_[shard_idx] = table_size_bytes_; +} diff --git a/src/streaming/fs.array.hh b/src/streaming/fs.array.hh new file mode 100644 index 00000000..a18e50bf --- /dev/null +++ b/src/streaming/fs.array.hh @@ -0,0 +1,61 @@ +#pragma once + +#include "array.hh" +#include "fs.storage.hh" + +#include +#include + +namespace zarr { +class FSArray final + : public Array + , public FSStorage +{ + public: + FSArray(std::shared_ptr config, + std::shared_ptr thread_pool, + std::shared_ptr file_handle_pool); + + protected: + struct ShardFile + { + std::string path; + std::shared_ptr handle; + std::vector table; + std::mutex table_mutex; + uint64_t file_offset; + std::mutex offset_mutex; + std::vector> chunk_futures; + + [[nodiscard]] bool close(); + }; + + const size_t table_size_bytes_; + + std::unordered_map> shard_files_; + std::mutex shard_files_mutex_; + std::condition_variable shard_files_cv_; + + std::unordered_map> handles_; + std::mutex handles_mutex_; + + bool write_metadata_() override; + std::string index_location_() const override; + bool compress_and_flush_data_() override; + void finalize_append_shard_() override; + + /** + * @brief Get a file handle for the given path, creating it and adding it to + * the local handle pool if it does not already exist. + * @param path The file path. + * @return The file handle. + */ + std::shared_ptr get_handle_(const std::string& path); + + /** + * @brief Write the shard table entries for the given shard index. + * @param shard_idx The shard index. + */ + void write_table_entries_(uint32_t shard_idx); +}; +} // namespace zarr \ No newline at end of file diff --git a/src/streaming/fs.multiscale.array.cpp b/src/streaming/fs.multiscale.array.cpp new file mode 100644 index 00000000..539b7d01 --- /dev/null +++ b/src/streaming/fs.multiscale.array.cpp @@ -0,0 +1,64 @@ +#include "fs.multiscale.array.hh" +#include "macros.hh" + +zarr::FSMultiscaleArray::FSMultiscaleArray( + std::shared_ptr config, + std::shared_ptr thread_pool, + std::shared_ptr file_handle_pool) + : MultiscaleArray(config, thread_pool) + , FSStorage(file_handle_pool) +{ + // dimensions may be null in the case of intermediate groups, e.g., the + // A in A/1 + if (config_->dimensions) { + CHECK(FSMultiscaleArray::create_arrays_()); + } +} + +bool +zarr::FSMultiscaleArray::write_metadata_() +{ + std::string metadata; + if (!make_metadata_(metadata)) { + LOG_ERROR("Failed to make metadata."); + return false; + } + + if (last_written_metadata_ == metadata) { + return true; // no changes + } + const std::string path = node_path_() + "/zarr.json"; + + bool success; + if ((success = write_string(path, metadata, 0))) { + last_written_metadata_ = metadata; + } + + return success; +} + +bool +zarr::FSMultiscaleArray::create_arrays_() +{ + arrays_.clear(); + + try { + if (downsampler_) { + const auto& configs = downsampler_->writer_configurations(); + arrays_.resize(configs.size()); + + for (const auto& [lod, config] : configs) { + arrays_[lod] = std::make_unique( + config, thread_pool_, file_handle_pool_); + } + } else { + arrays_.push_back(std::make_unique( + make_base_array_config_(), thread_pool_, file_handle_pool_)); + } + } catch (const std::exception& e) { + LOG_ERROR(e.what()); + return false; + } + + return true; +} diff --git a/src/streaming/fs.multiscale.array.hh b/src/streaming/fs.multiscale.array.hh new file mode 100644 index 00000000..13b51bf6 --- /dev/null +++ b/src/streaming/fs.multiscale.array.hh @@ -0,0 +1,22 @@ +#pragma once + +#include "fs.array.hh" +#include "fs.storage.hh" +#include "multiscale.array.hh" + +namespace zarr { +class FSMultiscaleArray + : public MultiscaleArray + , public FSStorage +{ + public: + FSMultiscaleArray(std::shared_ptr config, + std::shared_ptr thread_pool, + std::shared_ptr file_handle_pool); + + protected: + bool write_metadata_() override; + + bool create_arrays_() override; +}; +} // namespace zarr \ No newline at end of file diff --git a/src/streaming/fs.storage.cpp b/src/streaming/fs.storage.cpp new file mode 100644 index 00000000..7f404cd2 --- /dev/null +++ b/src/streaming/fs.storage.cpp @@ -0,0 +1,42 @@ +#include "fs.storage.hh" +#include "macros.hh" + +#include + +void* +make_flags(); + +void +destroy_flags(void* flags); + +bool +seek_and_write(void* handle, size_t offset, std::span data); + +zarr::FSStorage::FSStorage(std::shared_ptr file_handle_pool) + : file_handle_pool_(file_handle_pool) +{ +} + +bool +zarr::FSStorage::write_string(const std::string& path, + const std::string& data, + size_t offset) const +{ + void* flags = make_flags(); + const auto handle = file_handle_pool_->get_handle(path, flags); + destroy_flags(flags); + + if (handle == nullptr) { + LOG_ERROR("Failed to get file handle for ", path); + return false; + } + + std::span span{ reinterpret_cast(data.data()), + data.size() }; + if (!seek_and_write(handle.get(), offset, span)) { + LOG_ERROR("Failed to write string to ", path); + return false; + } + + return true; +} diff --git a/src/streaming/fs.storage.hh b/src/streaming/fs.storage.hh new file mode 100644 index 00000000..d2b8637b --- /dev/null +++ b/src/streaming/fs.storage.hh @@ -0,0 +1,29 @@ +#pragma once + +#include "array.base.hh" +#include "file.handle.hh" + +#include + +namespace zarr { +class FSStorage +{ + public: + explicit FSStorage(std::shared_ptr file_handle_pool); + virtual ~FSStorage() = default; + + /** + * @brief Write a string to a path at the given offset. + * @param path The path to write to. + * @param data The string to write. + * @param offset The offset to write at. + * @return True if the write was successful, false otherwise. + */ + [[nodiscard]] bool write_string(const std::string& path, + const std::string& data, + size_t offset) const; + + protected: + std::shared_ptr file_handle_pool_; +}; +} // namespace zarr \ No newline at end of file diff --git a/src/streaming/locked.buffer.cpp b/src/streaming/locked.buffer.cpp deleted file mode 100644 index 59544775..00000000 --- a/src/streaming/locked.buffer.cpp +++ /dev/null @@ -1,130 +0,0 @@ -#include "locked.buffer.hh" -#include "macros.hh" - -#include - -zarr::LockedBuffer::LockedBuffer(std::vector&& data) - : data_(std::move(data)) -{ -} - -zarr::LockedBuffer::LockedBuffer(zarr::LockedBuffer&& other) noexcept - : data_(std::move(other.data_)) -{ -} - -zarr::LockedBuffer& -zarr::LockedBuffer::operator=(zarr::LockedBuffer&& other) noexcept -{ - if (this != &other) { - std::unique_lock lock1(mutex_, std::defer_lock); - std::unique_lock lock2(other.mutex_, std::defer_lock); - std::lock(lock1, lock2); // avoid deadlock - - data_ = std::move(other.data_); - } - - return *this; -} - -void -zarr::LockedBuffer::resize(size_t n) -{ - std::unique_lock lock(mutex_); - data_.resize(n); -} - -void -zarr::LockedBuffer::resize_and_fill(size_t n, uint8_t value) -{ - std::unique_lock lock(mutex_); - - data_.resize(n, value); - std::fill(data_.begin(), data_.end(), value); -} - -size_t -zarr::LockedBuffer::size() const -{ - std::unique_lock lock(mutex_); - return data_.size(); -} - -void -zarr::LockedBuffer::assign(ConstByteSpan data) -{ - std::unique_lock lock(mutex_); - data_.assign(data.begin(), data.end()); -} - -void -zarr::LockedBuffer::assign(ByteVector&& data) -{ - std::unique_lock lock(mutex_); - data_ = std::move(data); -} - -void -zarr::LockedBuffer::assign_at(size_t offset, ConstByteSpan data) -{ - std::unique_lock lock(mutex_); - if (offset + data.size() > data_.size()) { - data_.resize(offset + data.size()); - } - std::copy(data.begin(), data.end(), data_.begin() + offset); -} - -void -zarr::LockedBuffer::swap(zarr::LockedBuffer& other) -{ - std::unique_lock lock(mutex_); - other.with_lock([this](ByteVector& other_data) { data_.swap(other_data); }); -} - -void -zarr::LockedBuffer::clear() -{ - std::unique_lock lock(mutex_); - data_.clear(); -} - -std::vector -zarr::LockedBuffer::take() -{ - std::unique_lock lock(mutex_); - std::vector result = std::move(data_); - data_ = std::vector{}; // Fresh empty vector - return result; -} - -bool -zarr::LockedBuffer::compress(const zarr::BloscCompressionParams& params, - size_t type_size) -{ - std::unique_lock lock(mutex_); - if (data_.empty()) { - LOG_WARNING("Buffer is empty, not compressing."); - return false; - } - - std::vector compressed_data(data_.size() + BLOSC_MAX_OVERHEAD); - const auto n_bytes_compressed = blosc_compress_ctx(params.clevel, - params.shuffle, - type_size, - data_.size(), - data_.data(), - compressed_data.data(), - compressed_data.size(), - params.codec_id.c_str(), - 0, - 1); - - if (n_bytes_compressed <= 0) { - LOG_ERROR("blosc_compress_ctx failed with code ", n_bytes_compressed); - return false; - } - - compressed_data.resize(n_bytes_compressed); - data_ = compressed_data; - return true; -} \ No newline at end of file diff --git a/src/streaming/locked.buffer.hh b/src/streaming/locked.buffer.hh deleted file mode 100644 index 20f343dd..00000000 --- a/src/streaming/locked.buffer.hh +++ /dev/null @@ -1,99 +0,0 @@ -#pragma once - -#include "blosc.compression.params.hh" -#include "definitions.hh" - -#include -#include - -namespace zarr { -class LockedBuffer -{ - private: - mutable std::mutex mutex_; - std::vector data_; - - public: - LockedBuffer() = default; - LockedBuffer(std::vector&& data); - - LockedBuffer(const LockedBuffer& other) = delete; - LockedBuffer(LockedBuffer&& other) noexcept; - - LockedBuffer& operator=(const LockedBuffer&) = delete; - LockedBuffer& operator=(LockedBuffer&& other) noexcept; - - template - auto with_lock(F&& fun) -> decltype(fun(data_)) - { - std::unique_lock lock(mutex_); - return fun(data_); - } - - /** - * @brief Resize the buffer to @p n bytes, but keep existing data. - * @param n New size of the buffer. - */ - void resize(size_t n); - - /** - * @brief Resize the buffer to @p n bytes, filling new bytes with @p value. - * @param n New size of the buffer. - * @param value Value to fill new bytes with. - */ - void resize_and_fill(size_t n, uint8_t value); - - /** - * @brief Get the current size of the buffer. - * @return Size of the buffer in bytes. - */ - size_t size() const; - - /** - * @brief Assign new data to the buffer, replacing existing data. - * @param data Data to assign to the buffer. - */ - void assign(ConstByteSpan data); - - /** - * @brief Assign new data to the buffer, replacing existing data. - * @note Moves the data - * @param data Data to assign to the buffer. - */ - void assign(ByteVector&& data); - - /** - * @brief Assign new data to the buffer at offset @p offset, replacing - * existing data. - * @param offset - * @param data - */ - void assign_at(size_t offset, ConstByteSpan data); - - /** - * @brief Swap the contents of this buffer with another. - * @param other The other LockedBuffer to swap with. - */ - void swap(LockedBuffer& other); - - /** - * @brief Clear the buffer, removing all data. - */ - void clear(); - - /** - * @brief Take the contents of the buffer, leaving it empty. - * @return The contents of the buffer. - */ - std::vector take(); - - /** - * @brief Compress the buffer in place using Blosc with the given parameters. - * @param params Compression parameters. - * @param type_size Size of the data type being compressed (e.g., 1 for uint8, 2 for uint16). - * @return true if compression was successful, false otherwise. - */ - [[nodiscard]] bool compress(const zarr::BloscCompressionParams& params, - size_t type_size); -}; -} // namespace zarr \ No newline at end of file diff --git a/src/streaming/multiscale.array.cpp b/src/streaming/multiscale.array.cpp index bf9f8a77..2afeadaf 100644 --- a/src/streaming/multiscale.array.cpp +++ b/src/streaming/multiscale.array.cpp @@ -21,24 +21,15 @@ dimension_type_to_string(ZarrDimensionType type) } } // namespace -zarr::MultiscaleArray::MultiscaleArray( - std::shared_ptr config, - std::shared_ptr thread_pool, - std::shared_ptr file_handle_pool, - std::shared_ptr s3_connection_pool) - : ArrayBase(config, thread_pool, file_handle_pool, s3_connection_pool) +zarr::MultiscaleArray::MultiscaleArray(std::shared_ptr config, + std::shared_ptr thread_pool) + : ArrayBase(config, thread_pool) { bytes_per_frame_ = config_->dimensions == nullptr ? 0 : bytes_of_frame(*config_->dimensions, config_->dtype); EXPECT(create_downsampler_(), "Failed to create downsampler"); - - // dimensions may be null in the case of intermediate groups, e.g., the - // A in A/1 - if (config_->dimensions) { - CHECK(create_arrays_()); - } } size_t @@ -53,7 +44,7 @@ zarr::MultiscaleArray::memory_usage() const noexcept } size_t -zarr::MultiscaleArray::write_frame(LockedBuffer& data) +zarr::MultiscaleArray::write_frame(std::vector& data) { if (arrays_.empty()) { LOG_WARNING("Attempt to write to group with no arrays"); @@ -76,17 +67,9 @@ zarr::MultiscaleArray::write_frame(LockedBuffer& data) return n_bytes; } -std::vector -zarr::MultiscaleArray::metadata_keys_() const -{ - return { "zarr.json" }; -} - bool -zarr::MultiscaleArray::make_metadata_() +zarr::MultiscaleArray::make_metadata_(std::string& metadata_str) { - metadata_sinks_.clear(); - nlohmann::json metadata = { { "zarr_format", 3 }, { "consolidated_metadata", nullptr }, @@ -98,7 +81,7 @@ zarr::MultiscaleArray::make_metadata_() metadata["attributes"]["ome"] = get_ome_metadata_(); } - metadata_strings_.emplace("zarr.json", metadata.dump(4)); + metadata_str = metadata.dump(4); return true; } @@ -118,36 +101,7 @@ zarr::MultiscaleArray::close_() return false; } - for (auto& [key, sink] : metadata_sinks_) { - EXPECT(zarr::finalize_sink(std::move(sink)), - "Failed to finalize metadata sink ", - key); - } - arrays_.clear(); - metadata_sinks_.clear(); - - return true; -} - -bool -zarr::MultiscaleArray::create_arrays_() -{ - arrays_.clear(); - - if (downsampler_) { - const auto& configs = downsampler_->writer_configurations(); - arrays_.resize(configs.size()); - - for (const auto& [lod, config] : configs) { - arrays_[lod] = std::make_unique( - config, thread_pool_, file_handle_pool_, s3_connection_pool_); - } - } else { - const auto config = make_base_array_config_(); - arrays_.push_back(std::make_unique( - config, thread_pool_, file_handle_pool_, s3_connection_pool_)); - } return true; } @@ -279,7 +233,7 @@ zarr::MultiscaleArray::make_base_array_config_() const } void -zarr::MultiscaleArray::write_multiscale_frames_(LockedBuffer& data) +zarr::MultiscaleArray::write_multiscale_frames_(std::vector& data) { if (!downsampler_) { return; // no downsampler, nothing to do @@ -288,7 +242,7 @@ zarr::MultiscaleArray::write_multiscale_frames_(LockedBuffer& data) downsampler_->add_frame(data); for (auto i = 1; i < arrays_.size(); ++i) { - LockedBuffer downsampled_frame; + std::vector downsampled_frame; if (downsampler_->take_frame(i, downsampled_frame)) { const auto n_bytes = arrays_[i]->write_frame(downsampled_frame); EXPECT(n_bytes == downsampled_frame.size(), diff --git a/src/streaming/multiscale.array.hh b/src/streaming/multiscale.array.hh index 96a6d4b1..0afe6ed0 100644 --- a/src/streaming/multiscale.array.hh +++ b/src/streaming/multiscale.array.hh @@ -2,7 +2,6 @@ #include "array.hh" #include "downsampler.hh" -#include "sink.hh" #include "thread.pool.hh" #include @@ -14,9 +13,7 @@ class MultiscaleArray : public ArrayBase { public: MultiscaleArray(std::shared_ptr config, - std::shared_ptr thread_pool, - std::shared_ptr file_handle_pool, - std::shared_ptr s3_connection_pool); + std::shared_ptr thread_pool); size_t memory_usage() const noexcept override; @@ -28,7 +25,7 @@ class MultiscaleArray : public ArrayBase * @param data The frame data to write. * @return The number of bytes written of the full-resolution frame. */ - [[nodiscard]] size_t write_frame(LockedBuffer& data) override; + [[nodiscard]] size_t write_frame(std::vector& data) override; protected: std::unique_ptr downsampler_; @@ -36,12 +33,11 @@ class MultiscaleArray : public ArrayBase size_t bytes_per_frame_; - std::vector metadata_keys_() const override; - bool make_metadata_() override; + bool make_metadata_(std::string& metadata_str) override; bool close_() override; /** @brief Create array writers. */ - [[nodiscard]] bool create_arrays_(); + [[nodiscard]] virtual bool create_arrays_() = 0; /** * @brief Construct OME metadata for this group. @@ -60,13 +56,13 @@ class MultiscaleArray : public ArrayBase [[nodiscard]] virtual nlohmann::json make_multiscales_metadata_() const; /** @brief Create a configuration for a full-resolution Array. */ - std::shared_ptr make_base_array_config_() const; + std::shared_ptr make_base_array_config_() const; /** * @brief Add @p data to downsampler and write downsampled frames to lower- * resolution arrays. * @param data The frame data to write. */ - void write_multiscale_frames_(LockedBuffer& data); + void write_multiscale_frames_(std::vector& data); }; } // namespace zarr \ No newline at end of file diff --git a/src/streaming/posix/platform.cpp b/src/streaming/posix/platform.cpp index cfe9bca5..640279be 100644 --- a/src/streaming/posix/platform.cpp +++ b/src/streaming/posix/platform.cpp @@ -1,6 +1,6 @@ -#include "definitions.hh" #include "macros.hh" +#include #include #include @@ -57,7 +57,7 @@ init_handle(const std::string& filename, void* flags) } bool -seek_and_write(void* handle, size_t offset, ConstByteSpan data) +seek_and_write(void* handle, size_t offset, std::span data) { CHECK(handle); const auto* fd = static_cast(handle); diff --git a/src/streaming/s3.array.cpp b/src/streaming/s3.array.cpp new file mode 100644 index 00000000..525ed3af --- /dev/null +++ b/src/streaming/s3.array.cpp @@ -0,0 +1,384 @@ +#include "macros.hh" +#include "s3.array.hh" +#include "zarr.common.hh" + +#include +#include + +#include // memcpy +#include + +zarr::S3Array::S3Array(std::shared_ptr config, + std::shared_ptr thread_pool, + std::shared_ptr s3_connection_pool) + : Array(config, thread_pool) + , S3Storage(*config->bucket_name, s3_connection_pool) +{ + CHECK(config_->dimensions); + + const auto& dims = config_->dimensions; +} + +bool +zarr::S3Array::write_metadata_() +{ + std::string metadata; + if (!make_metadata_(metadata)) { + LOG_ERROR("Failed to make metadata."); + return false; + } + + if (last_written_metadata_ == metadata) { + return true; // no changes + } + const std::string key = node_path_() + "/zarr.json"; + + bool success; + if ((success = write_string(key, metadata, 0) && finalize_object(key))) { + last_written_metadata_ = metadata; + } + return success; +} + +std::string +zarr::S3Array::index_location_() const +{ + return "end"; +} + +bool +zarr::S3Array::compress_and_flush_data_() +{ + if (!compress_chunks_()) { + LOG_ERROR("Failed to compress chunk data"); + return false; + } + + update_table_entries_(); + + if (!flush_data_()) { + LOG_ERROR("Failed to flush chunk data"); + return false; + } + + if (!is_closing_) { + fill_buffers_(); + } + + return true; +} + +void +zarr::S3Array::finalize_append_shard_() +{ + const bool should_rollover = should_rollover_(); + const bool should_flush_anyway = + is_closing_ && + frames_written_ % config_->dimensions->frames_per_layer() != 0; + + if (should_rollover || should_flush_anyway) { + flush_tables_(); + } + + for (const auto& key : data_paths_) { + EXPECT(finalize_object(key), "Failed to finalize S3 object at ", key); + } + + data_paths_.clear(); +} + +bool +zarr::S3Array::compress_chunks_() +{ + if (!config_->compression_params) { + return true; // nothing to do + } + + std::atomic all_successful = 1; + + const auto& params = *config_->compression_params; + const size_t bytes_per_px = bytes_of_type(config_->dtype); + + const auto& dims = config_->dimensions; + + const uint32_t chunks_in_memory = chunk_buffers_.size(); + const uint32_t chunk_group_offset = current_layer_ * chunks_in_memory; + + std::vector> futures; + futures.reserve(chunks_in_memory); + + for (size_t i = 0; i < chunks_in_memory; ++i) { + auto promise = std::make_shared>(); + futures.emplace_back(promise->get_future()); + + const uint32_t chunk_idx = i + chunk_group_offset; + const uint32_t shard_idx = dims->shard_index_for_chunk(chunk_idx); + const uint32_t internal_idx = dims->shard_internal_index(chunk_idx); + auto* shard_table = shard_tables_.data() + shard_idx; + + auto job = [&chunk_buffer = chunk_buffers_[i], + bytes_per_px, + ¶ms, + shard_table, + shard_idx, + chunk_idx, + internal_idx, + promise, + &all_successful](std::string& err) { + bool success = false; + + try { + std::vector compressed_data(chunk_buffer.size() + + BLOSC_MAX_OVERHEAD); + const auto n_bytes_compressed = + blosc_compress_ctx(params.clevel, + params.shuffle, + bytes_per_px, + chunk_buffer.size(), + chunk_buffer.data(), + compressed_data.data(), + compressed_data.size(), + params.codec_id.c_str(), + 0, + 1); + + if (n_bytes_compressed <= 0) { + err = "blosc_compress_ctx failed with code " + + std::to_string(n_bytes_compressed) + " for chunk " + + std::to_string(chunk_idx) + " (internal index " + + std::to_string(internal_idx) + " of shard " + + std::to_string(shard_idx) + ")"; + success = false; + } else { + compressed_data.resize(n_bytes_compressed); + chunk_buffer.swap(compressed_data); + + // update shard table with size + shard_table->at(2 * internal_idx + 1) = chunk_buffer.size(); + success = true; + } + } catch (const std::exception& exc) { + err = exc.what(); + } + + promise->set_value(); + + all_successful.fetch_and(static_cast(success)); + return success; + }; + + // one thread is reserved for processing the frame queue and runs + // the entire lifetime of the stream + if (thread_pool_->n_threads() == 1 || !thread_pool_->push_job(job)) { + if (std::string err; !job(err)) { + LOG_ERROR(err); + } + } + } + + for (auto& future : futures) { + future.wait(); + } + + return static_cast(all_successful); +} + +void +zarr::S3Array::update_table_entries_() +{ + const uint32_t chunks_in_memory = chunk_buffers_.size(); + const uint32_t chunk_group_offset = current_layer_ * chunks_in_memory; + const auto& dims = config_->dimensions; + + for (auto i = 0; i < chunks_in_memory; ++i) { + const auto& chunk_buffer = chunk_buffers_[i]; + const uint32_t chunk_idx = i + chunk_group_offset; + const uint32_t shard_idx = dims->shard_index_for_chunk(chunk_idx); + const uint32_t internal_idx = dims->shard_internal_index(chunk_idx); + auto& shard_table = shard_tables_[shard_idx]; + + shard_table[2 * internal_idx + 1] = chunk_buffer.size(); + } +} + +zarr::S3Array::ShardLayer +zarr::S3Array::collect_chunks_(uint32_t shard_index) +{ + const auto& dims = config_->dimensions; + CHECK(shard_index < dims->number_of_shards()); + + const auto chunks_per_shard = dims->chunks_per_shard(); + const auto chunks_in_mem = dims->number_of_chunks_in_memory(); + const auto n_layers = dims->chunk_layers_per_shard(); + + const auto chunks_per_layer = chunks_per_shard / n_layers; + const auto layer_offset = current_layer_ * chunks_per_layer; + const auto chunk_offset = current_layer_ * chunks_in_mem; + + auto& shard_table = shard_tables_[shard_index]; + const auto file_offset = shard_file_offsets_[shard_index]; + shard_table[2 * layer_offset] = file_offset; + + uint64_t last_chunk_offset = shard_table[2 * layer_offset]; + uint64_t last_chunk_size = shard_table[2 * layer_offset + 1]; + + for (auto i = 1; i < chunks_per_layer; ++i) { + const auto offset_idx = 2 * (layer_offset + i); + const auto size_idx = offset_idx + 1; + if (shard_table[size_idx] == std::numeric_limits::max()) { + continue; + } + + shard_table[offset_idx] = last_chunk_offset + last_chunk_size; + last_chunk_offset = shard_table[offset_idx]; + last_chunk_size = shard_table[size_idx]; + } + + const auto chunk_indices_this_layer = + dims->chunk_indices_for_shard_layer(shard_index, current_layer_); + + ShardLayer layer{ file_offset, {} }; + layer.chunks.reserve(chunk_indices_this_layer.size()); + + for (const auto& idx : chunk_indices_this_layer) { + layer.chunks.emplace_back(chunk_buffers_[idx - chunk_offset]); + } + + return std::move(layer); +} + +bool +zarr::S3Array::flush_data_() +{ + // construct paths to shard sinks if they don't already exist + if (data_paths_.empty()) { + make_data_paths_(); + } + + const auto& dims = config_->dimensions; + + const auto n_shards = dims->number_of_shards(); + CHECK(data_paths_.size() == n_shards); + + std::atomic all_successful = 1; + + std::vector> futures; + + // wait for the chunks in each shard to finish compressing, then defragment + // and write the shard + for (auto shard_idx = 0; shard_idx < n_shards; ++shard_idx) { + const std::string data_path = data_paths_[shard_idx]; + auto* file_offset = shard_file_offsets_.data() + shard_idx; + + auto promise = std::make_shared>(); + futures.emplace_back(promise->get_future()); + + auto job = + [shard_idx, data_path, file_offset, promise, &all_successful, this]( + std::string& err) { + bool success = true; + + try { + const auto shard_data = collect_chunks_(shard_idx); + if (shard_data.chunks.empty()) { + LOG_ERROR("Failed to collect chunks for shard ", + shard_idx); + return false; + } + if (shard_data.offset != *file_offset) { + LOG_ERROR("Inconsistent file offset for shard ", + shard_idx, + ": expected ", + *file_offset, + ", got ", + shard_data.offset); + return false; + } + + size_t layer_offset = shard_data.offset; + for (auto& chunk : shard_data.chunks) { + if (!write_binary(data_path, chunk, layer_offset)) { + err = "Failed to write chunk " + + std::to_string(shard_idx) + " at offset " + + std::to_string(layer_offset) + " to path " + + data_path; + success = false; + break; + } + layer_offset += chunk.size(); + } + *file_offset = layer_offset; + } catch (const std::exception& exc) { + err = "Failed to flush data: " + std::string(exc.what()); + success = false; + } + + all_successful.fetch_and(success); + promise->set_value(); + + return success; + }; + + // one thread is reserved for processing the frame queue and runs the + // entire lifetime of the stream + if (thread_pool_->n_threads() == 1 || !thread_pool_->push_job(job)) { + std::string err; + if (!job(err)) { + LOG_ERROR(err); + } + } + } + + // wait for all threads to finish + for (auto& future : futures) { + future.wait(); + } + + return static_cast(all_successful); +} + +bool +zarr::S3Array::flush_tables_() +{ + // construct paths to shard sinks if they don't already exist + if (data_paths_.empty()) { + make_data_paths_(); + } + + const auto& dims = config_->dimensions; + const auto n_shards = dims->number_of_shards(); + + for (auto shard_idx = 0; shard_idx < n_shards; ++shard_idx) { + const auto* shard_table = shard_tables_.data() + shard_idx; + auto* file_offset = shard_file_offsets_.data() + shard_idx; + + const size_t table_size = shard_table->size() * sizeof(uint64_t); + std::vector table(table_size + sizeof(uint32_t), 0); + + memcpy(table.data(), shard_table->data(), table_size); + + // compute crc32 checksum of the table + const uint32_t checksum = crc32c::Crc32c(table.data(), table_size); + memcpy(table.data() + table_size, &checksum, sizeof(uint32_t)); + + std::string data_path = data_paths_[shard_idx]; + + if (!write_binary(data_path, table, *file_offset)) { + LOG_ERROR("Failed to write table and checksum to shard ", + shard_idx, + " at path ", + data_path); + return false; + } + } + + // don't reset state if we're closing + if (!is_closing_) { + for (auto& table : shard_tables_) { + std::ranges::fill(table, std::numeric_limits::max()); + } + std::ranges::fill(shard_file_offsets_, 0); + } + + return true; +} diff --git a/src/streaming/s3.array.hh b/src/streaming/s3.array.hh new file mode 100644 index 00000000..92970284 --- /dev/null +++ b/src/streaming/s3.array.hh @@ -0,0 +1,59 @@ +#pragma once + +#include "array.hh" +#include "s3.storage.hh" + +namespace zarr { +class S3Array final + : public Array + , public S3Storage +{ + public: + S3Array(std::shared_ptr config, + std::shared_ptr thread_pool, + std::shared_ptr s3_connection_pool); + + protected: + struct ShardLayer + { + size_t offset; // offset in bytes from start of shard + std::vector> chunks; + }; + + bool write_metadata_() override; + std::string index_location_() const override; + bool compress_and_flush_data_() override; + void finalize_append_shard_() override; + + /** + * @brief Compress all the chunk buffers in place. + * @return True on success, false on failure. + */ + bool compress_chunks_(); + + /** + * @brief Update the shard tables with the sizes of the compressed chunks. + */ + void update_table_entries_(); + + /** + * @brief Collect all the chunks for a given shard index in the current + * layer. + * @param shard_index The index of the shard to collect chunks for. + * @return The collected chunk buffers in a shard layer structure. + */ + ShardLayer collect_chunks_(uint32_t shard_index); + + /** + * @brief Flush the chunk data to S3 or intermediate buffers. + * @return True on success, false on failure. + */ + bool flush_data_(); + + /** + * @brief Flush all shard tables to S3. + * @return True on success, false on failure. + */ + bool flush_tables_(); +}; +} // namespace zarr \ No newline at end of file diff --git a/src/streaming/s3.multiscale.array.cpp b/src/streaming/s3.multiscale.array.cpp new file mode 100644 index 00000000..d561bdc6 --- /dev/null +++ b/src/streaming/s3.multiscale.array.cpp @@ -0,0 +1,63 @@ +#include "macros.hh" +#include "s3.multiscale.array.hh" + +zarr::S3MultiscaleArray::S3MultiscaleArray( + std::shared_ptr config, + std::shared_ptr thread_pool, + std::shared_ptr s3_connection_pool) + : MultiscaleArray(config, thread_pool) + , S3Storage(*config->bucket_name, s3_connection_pool) +{ + // dimensions may be null in the case of intermediate groups, e.g., the + // A in A/1 + if (config_->dimensions) { + CHECK(S3MultiscaleArray::create_arrays_()); + } +} + +bool +zarr::S3MultiscaleArray::write_metadata_() +{ + std::string metadata; + if (!make_metadata_(metadata)) { + LOG_ERROR("Failed to make metadata."); + return false; + } + + if (last_written_metadata_ == metadata) { + return true; // no changes + } + const std::string key = node_path_() + "/zarr.json"; + + bool success; + if ((success = write_string(key, metadata, 0) && finalize_object(key))) { + last_written_metadata_ = metadata; + } + return success; +} + +bool +zarr::S3MultiscaleArray::create_arrays_() +{ + arrays_.clear(); + + try { + if (downsampler_) { + const auto& configs = downsampler_->writer_configurations(); + arrays_.resize(configs.size()); + + for (const auto& [lod, config] : configs) { + arrays_[lod] = std::make_unique( + config, thread_pool_, s3_connection_pool_); + } + } else { + arrays_.push_back(std::make_unique( + make_base_array_config_(), thread_pool_, s3_connection_pool_)); + } + } catch (const std::exception& e) { + LOG_ERROR(e.what()); + return false; + } + + return true; +} diff --git a/src/streaming/s3.multiscale.array.hh b/src/streaming/s3.multiscale.array.hh new file mode 100644 index 00000000..2f9f8b19 --- /dev/null +++ b/src/streaming/s3.multiscale.array.hh @@ -0,0 +1,22 @@ +#pragma once + +#include "multiscale.array.hh" +#include "s3.array.hh" +#include "s3.storage.hh" + +namespace zarr { +class S3MultiscaleArray + : public MultiscaleArray + , public S3Storage +{ + public: + S3MultiscaleArray(std::shared_ptr config, + std::shared_ptr thread_pool, + std::shared_ptr s3_connection_pool); + + protected: + bool write_metadata_() override; + + bool create_arrays_() override; +}; +} // namespace zarr \ No newline at end of file diff --git a/src/streaming/s3.sink.cpp b/src/streaming/s3.object.cpp similarity index 87% rename from src/streaming/s3.sink.cpp rename to src/streaming/s3.object.cpp index adf7a0f9..2be63498 100644 --- a/src/streaming/s3.sink.cpp +++ b/src/streaming/s3.object.cpp @@ -1,5 +1,5 @@ #include "macros.hh" -#include "s3.sink.hh" +#include "s3.object.hh" #include #include @@ -8,9 +8,9 @@ #undef min #endif -zarr::S3Sink::S3Sink(std::string_view bucket_name, - std::string_view object_key, - std::shared_ptr connection_pool) +zarr::S3Object::S3Object(std::string_view bucket_name, + std::string_view object_key, + std::shared_ptr connection_pool) : bucket_name_{ bucket_name } , object_key_{ object_key } , connection_pool_{ connection_pool } @@ -21,39 +21,14 @@ zarr::S3Sink::S3Sink(std::string_view bucket_name, } bool -zarr::S3Sink::flush_() +zarr::S3Object::write(std::span data, size_t offset) { - if (is_multipart_upload_()) { - const auto& parts = multipart_upload_->parts; - if (nbytes_buffered_ > 0 && !flush_part_()) { - LOG_ERROR("Failed to upload part ", - parts.size() + 1, - " of object ", - object_key_); - return false; - } - if (!finalize_multipart_upload_()) { - LOG_ERROR("Failed to finalize multipart upload of object ", - object_key_); - return false; - } - } else if (nbytes_buffered_ > 0) { - if (!put_object_()) { - LOG_ERROR("Failed to upload object: ", object_key_); - return false; - } + if (is_closed_) { + LOG_ERROR("Cannot write to closed stream"); + return false; } - // cleanup - nbytes_buffered_ = 0; - - return true; -} - -bool -zarr::S3Sink::write(size_t offset, ConstByteSpan data) -{ - if (data.data() == nullptr || data.empty()) { + if (data.empty()) { return true; } @@ -90,7 +65,42 @@ zarr::S3Sink::write(size_t offset, ConstByteSpan data) } bool -zarr::S3Sink::put_object_() +zarr::S3Object::close() +{ + if (is_closed_) { + return true; + } + + if (is_multipart_upload_()) { + const auto& parts = multipart_upload_->parts; + if (nbytes_buffered_ > 0 && !flush_part_()) { + LOG_ERROR("Failed to upload part ", + parts.size() + 1, + " of object ", + object_key_); + return false; + } + if (!finalize_multipart_upload_()) { + LOG_ERROR("Failed to finalize multipart upload of object ", + object_key_); + return false; + } + } else if (nbytes_buffered_ > 0) { + if (!put_object_()) { + LOG_ERROR("Failed to upload object: ", object_key_); + return false; + } + } + + // cleanup + nbytes_buffered_ = 0; + is_closed_ = true; + + return true; +} + +bool +zarr::S3Object::put_object_() { if (nbytes_buffered_ == 0) { return false; @@ -121,13 +131,13 @@ zarr::S3Sink::put_object_() } bool -zarr::S3Sink::is_multipart_upload_() const +zarr::S3Object::is_multipart_upload_() const { return multipart_upload_.has_value(); } void -zarr::S3Sink::create_multipart_upload_() +zarr::S3Object::create_multipart_upload_() { multipart_upload_ = MultiPartUpload{}; @@ -139,7 +149,7 @@ zarr::S3Sink::create_multipart_upload_() } bool -zarr::S3Sink::flush_part_() +zarr::S3Object::flush_part_() { if (nbytes_buffered_ == 0) { return false; @@ -188,7 +198,7 @@ zarr::S3Sink::flush_part_() } bool -zarr::S3Sink::finalize_multipart_upload_() +zarr::S3Object::finalize_multipart_upload_() { auto connection = connection_pool_->get_connection(); diff --git a/src/streaming/s3.sink.hh b/src/streaming/s3.object.hh similarity index 61% rename from src/streaming/s3.sink.hh rename to src/streaming/s3.object.hh index 0edeb51e..38b34501 100644 --- a/src/streaming/s3.sink.hh +++ b/src/streaming/s3.object.hh @@ -1,6 +1,5 @@ #pragma once -#include "sink.hh" #include "s3.connection.hh" #include @@ -8,17 +7,26 @@ #include namespace zarr { -class S3Sink : public Sink +class S3Object { public: - S3Sink(std::string_view bucket_name, - std::string_view object_key, - std::shared_ptr connection_pool); + S3Object(std::string_view bucket_name, + std::string_view object_key, + std::shared_ptr connection_pool); - bool write(size_t offset, ConstByteSpan data) override; + /** @brief Write data to the object at the given offset. + * @param data The data to write. + * @param offset The offset to write at. + * @return True if the write was successful, false otherwise. + */ + [[nodiscard]] bool write(std::span data, size_t offset); - protected: - bool flush_() override; + /** + * @brief Close the object, flushing any remaining data. + * @details The object must not be used after calling this function. + * @return True if the object was successfully closed, otherwise false. + */ + [[nodiscard]] bool close(); private: struct MultiPartUpload @@ -27,18 +35,20 @@ class S3Sink : public Sink std::vector parts; }; - static constexpr size_t max_part_size_ = 5 << 20; + static constexpr size_t max_part_size_ = 5ULL << 20; std::string bucket_name_; std::string object_key_; std::shared_ptr connection_pool_; - std::array part_buffer_; + std::array part_buffer_{}; size_t nbytes_buffered_{ 0 }; size_t nbytes_flushed_{ 0 }; std::optional multipart_upload_; + bool is_closed_{ false }; + /** * @brief Upload the object to S3. * @return True if the object was successfully uploaded, otherwise false. diff --git a/src/streaming/s3.storage.cpp b/src/streaming/s3.storage.cpp new file mode 100644 index 00000000..3455cac8 --- /dev/null +++ b/src/streaming/s3.storage.cpp @@ -0,0 +1,77 @@ +#include "macros.hh" +#include "s3.storage.hh" + +#include + +zarr::S3Storage::S3Storage(const std::string& bucket_name, + std::shared_ptr s3_connection_pool) + : bucket_name_(bucket_name) + , s3_connection_pool_(std::move(s3_connection_pool)) +{ + EXPECT(!bucket_name_.empty(), "S3 bucket name is empty"); + EXPECT(s3_connection_pool_, "S3 connection pool is null"); +} + +bool +zarr::S3Storage::finalize_object(const std::string& path) +{ + if (const auto it = s3_objects_.find(path); it != s3_objects_.end()) { + if (const auto& s3_object = it->second; s3_object != nullptr) { + if (!s3_object->close()) { + LOG_ERROR("Failed to finalize S3 object at ", path); + return false; + } + } + s3_objects_.erase(it); + + return true; + } + + return false; +} + +void +zarr::S3Storage::create_s3_object_(const std::string& key) +{ + if (!s3_objects_.contains(key)) { + s3_objects_.emplace( + key, + std::make_unique(bucket_name_, key, s3_connection_pool_)); + } +} + +bool +zarr::S3Storage::write_binary(const std::string& key, + const std::vector& data, + size_t offset) +{ + create_s3_object_(key); + + auto it = s3_objects_.find(key); + EXPECT(it != s3_objects_.end(), "S3 object at ", key, " not found"); + if (auto& s3_object = it->second; s3_object != nullptr) { + return s3_object->write(data, offset); + } + + LOG_ERROR("S3 object at ", key, " is null"); + return false; +} + +bool +zarr::S3Storage::write_string(const std::string& key, + const std::string& data, + size_t offset) +{ + create_s3_object_(key); + + auto it = s3_objects_.find(key); + EXPECT(it != s3_objects_.end(), "S3 object at ", key, " not found"); + if (auto& s3_object = it->second; s3_object != nullptr) { + std::span span{ reinterpret_cast(data.data()), + data.size() }; + return s3_object->write(span, offset); + } + + LOG_ERROR("S3 object at ", key, " is null"); + return false; +} diff --git a/src/streaming/s3.storage.hh b/src/streaming/s3.storage.hh new file mode 100644 index 00000000..c903102e --- /dev/null +++ b/src/streaming/s3.storage.hh @@ -0,0 +1,54 @@ +#pragma once + +#include "s3.object.hh" + +#include + +namespace zarr { +class S3Storage +{ + public: + S3Storage(const std::string& bucket_name, + std::shared_ptr s3_connection_pool); + virtual ~S3Storage() = default; + + /** + * @brief Write binary data to a path at the given offset. + * @param key The path to write to. + * @param data The data to write. + * @param offset The offset to write at. + * @return True if the write was successful, false otherwise. + */ + [[nodiscard]] bool write_binary(const std::string& key, + const std::vector& data, + size_t offset); + + /** + * @brief Write a string to a path at the given offset. + * @param key The path to write to. + * @param data The string to write. + * @param offset The offset to write at. + * @return True if the write was successful, false otherwise. + */ + [[nodiscard]] bool write_string(const std::string& key, + const std::string& data, + size_t offset); + + /** + * @brief Finalize the object at the given path. + * @details This will ensure that any buffered data is flushed and the + * object is properly closed. + * @param path The path of the object to finalize. + * @return True if the object was successfully finalized, otherwise false. + */ + [[nodiscard]] bool finalize_object(const std::string& path); + + protected: + const std::string bucket_name_; + std::shared_ptr s3_connection_pool_; + + void create_s3_object_(const std::string& key); + + std::unordered_map> s3_objects_; +}; +} // namespace zarr \ No newline at end of file diff --git a/src/streaming/sink.cpp b/src/streaming/sink.cpp deleted file mode 100644 index 851e9acd..00000000 --- a/src/streaming/sink.cpp +++ /dev/null @@ -1,346 +0,0 @@ -#include "sink.hh" -#include "file.sink.hh" -#include "s3.sink.hh" -#include "macros.hh" - -#include -#include -#include -#include -#include - -namespace fs = std::filesystem; - -namespace { -bool -bucket_exists(std::string_view bucket_name, - std::shared_ptr connection_pool) -{ - CHECK(!bucket_name.empty()); - EXPECT(connection_pool, "S3 connection pool not provided."); - - auto conn = connection_pool->get_connection(); - bool bucket_exists = conn->bucket_exists(bucket_name); - - connection_pool->return_connection(std::move(conn)); - - return bucket_exists; -} - -bool -make_file_sinks(std::vector& file_paths, - std::shared_ptr thread_pool, - std::shared_ptr file_handle_pool, - std::vector>& sinks) -{ - if (file_paths.empty()) { - return true; - } - - const auto parents = zarr::get_parent_paths(file_paths); - if (!zarr::make_dirs(parents, thread_pool)) { - LOG_ERROR("Failed to make parent directories"); - return false; - } - - std::atomic all_successful = 1; - - const auto n_files = file_paths.size(); - sinks.resize(n_files); - std::fill(sinks.begin(), sinks.end(), nullptr); - std::vector> futures; - - for (auto i = 0; i < n_files; ++i) { - const auto filename = file_paths[i]; - std::unique_ptr* psink = sinks.data() + i; - - auto promise = std::make_shared>(); - futures.emplace_back(promise->get_future()); - - auto job = - [filename, file_handle_pool, psink, promise, &all_successful]( - std::string& err) -> bool { - bool success = false; - - try { - *psink = - std::make_unique(filename, file_handle_pool); - success = true; - } catch (const std::exception& exc) { - err = "Failed to create file '" + filename + "': " + exc.what(); - } - - promise->set_value(); - all_successful.fetch_and(success); - - return success; - }; - - // one thread is reserved for processing the frame queue and runs the - // entire lifetime of the stream - if (thread_pool->n_threads() == 1 || !thread_pool->push_job(job)) { - std::string err; - if (!job(err)) { - LOG_ERROR(err); - } - } - } - - for (auto& future : futures) { - future.wait(); - } - - return (bool)all_successful; -} - -bool -make_s3_sinks(std::string_view bucket_name, - const std::vector& object_keys, - std::shared_ptr connection_pool, - std::vector>& sinks) -{ - if (object_keys.empty()) { - return true; - } - - if (bucket_name.empty()) { - LOG_ERROR("Bucket name not provided."); - return false; - } - if (!connection_pool) { - LOG_ERROR("S3 connection pool not provided."); - return false; - } - - const auto n_objects = object_keys.size(); - sinks.resize(n_objects); - for (auto i = 0; i < n_objects; ++i) { - sinks[i] = std::make_unique( - bucket_name, object_keys[i], connection_pool); - } - - return true; -} -} // namespace - -bool -zarr::finalize_sink(std::unique_ptr&& sink) -{ - if (sink == nullptr) { - LOG_INFO("Sink is null. Nothing to finalize."); - return true; - } - - if (!sink->flush_()) { - return false; - } - - sink.reset(); - return true; -} - -std::vector -zarr::construct_data_paths(std::string_view base_path, - const ArrayDimensions& dimensions, - const DimensionPartsFun& parts_along_dimension) -{ - std::queue paths_queue; - paths_queue.emplace(base_path); - - // create intermediate paths - for (auto i = 1; // skip the last dimension - i < dimensions.ndims() - 1; // skip the x dimension - ++i) { - const auto& dim = dimensions.at(i); - const auto n_parts = parts_along_dimension(dim); - CHECK(n_parts); - - auto n_paths = paths_queue.size(); - for (auto j = 0; j < n_paths; ++j) { - const auto path = paths_queue.front(); - paths_queue.pop(); - - for (auto k = 0; k < n_parts; ++k) { - const auto kstr = std::to_string(k); - paths_queue.push(path + (path.empty() ? kstr : "/" + kstr)); - } - } - } - - // create final paths - std::vector paths_out; - paths_out.reserve(paths_queue.size() * - parts_along_dimension(dimensions.width_dim())); - { - const auto& dim = dimensions.width_dim(); - const auto n_parts = parts_along_dimension(dim); - CHECK(n_parts); - - auto n_paths = paths_queue.size(); - for (auto i = 0; i < n_paths; ++i) { - const auto path = paths_queue.front(); - paths_queue.pop(); - for (auto j = 0; j < n_parts; ++j) - paths_out.push_back(path + "/" + std::to_string(j)); - } - } - - return paths_out; -} - -std::vector -zarr::get_parent_paths(const std::vector& file_paths) -{ - std::unordered_set unique_paths; - for (const auto& file_path : file_paths) { - unique_paths.emplace(fs::path(file_path).parent_path().string()); - } - - return { unique_paths.begin(), unique_paths.end() }; -} - -bool -zarr::make_dirs(const std::vector& dir_paths, - std::shared_ptr thread_pool) -{ - if (dir_paths.empty()) { - return true; - } - EXPECT(thread_pool, "Thread pool not provided."); - - std::atomic all_successful = 1; - const std::unordered_set unique_paths(dir_paths.begin(), dir_paths.end()); - - std::vector> futures; - - for (const auto& path : unique_paths) { - auto promise = std::make_shared>(); - futures.emplace_back(promise->get_future()); - - auto job = [path, promise, &all_successful](std::string& err) { - bool success = true; - try { - if (fs::is_directory(path) || path.empty()) { - promise->set_value(); - return success; - } - - std::error_code ec; - if (!fs::create_directories(path, ec) && - !fs::is_directory(path)) { - err = "Failed to create directory '" + path + - "': " + ec.message(); - success = false; - } - } catch (const std::exception& exc) { - err = - "Failed to create directory '" + path + "': " + exc.what(); - success = false; - } - - promise->set_value(); - all_successful.fetch_and(success); - return success; - }; - - if (thread_pool->n_threads() == 1 || !thread_pool->push_job(job)) { - if (std::string err; !job(err)) { - LOG_ERROR(err); - } - } - } - - // wait for all jobs to finish - for (auto& future : futures) { - future.wait(); - } - - return all_successful; -} - -std::unique_ptr -zarr::make_file_sink(std::string_view file_path, - std::shared_ptr file_handle_pool) -{ - if (file_path.starts_with("file://")) { - file_path = file_path.substr(7); - } - - EXPECT(!file_path.empty(), "File path must not be empty."); - - fs::path path(file_path); - EXPECT(!path.empty(), "Invalid file path: ", file_path); - - fs::path parent_path = path.parent_path(); - - if (!fs::is_directory(parent_path)) { - std::error_code ec; - if (!fs::create_directories(parent_path, ec) && - !fs::is_directory(parent_path)) { - LOG_ERROR( - "Failed to create directory '", parent_path, "': ", ec.message()); - return nullptr; - } - } - - return std::make_unique(file_path, file_handle_pool); -} - -bool -zarr::make_data_file_sinks(std::string_view base_path, - const ArrayDimensions& dimensions, - const DimensionPartsFun& parts_along_dimension, - std::shared_ptr thread_pool, - std::shared_ptr file_handle_pool, - std::vector>& part_sinks) -{ - if (base_path.starts_with("file://")) { - base_path = base_path.substr(7); - } - - EXPECT(!base_path.empty(), "Base path must not be empty."); - - std::vector paths; - try { - paths = - construct_data_paths(base_path, dimensions, parts_along_dimension); - } catch (const std::exception& exc) { - LOG_ERROR("Failed to create dataset paths: ", exc.what()); - return false; - } - - return make_file_sinks(paths, thread_pool, file_handle_pool, part_sinks); -} - -std::unique_ptr -zarr::make_s3_sink(std::string_view bucket_name, - std::string_view object_key, - std::shared_ptr connection_pool) -{ - EXPECT(!object_key.empty(), "Object key must not be empty."); - - // bucket name and connection pool are checked in bucket_exists - if (!bucket_exists(bucket_name, connection_pool)) { - LOG_ERROR("Bucket '", bucket_name, "' does not exist."); - return nullptr; - } - - return std::make_unique(bucket_name, object_key, connection_pool); -} - -bool -zarr::make_data_s3_sinks(std::string_view bucket_name, - std::string_view base_path, - const ArrayDimensions& dimensions, - const DimensionPartsFun& parts_along_dimension, - std::shared_ptr connection_pool, - std::vector>& part_sinks) -{ - EXPECT(!base_path.empty(), "Base path must not be empty."); - EXPECT(!bucket_name.empty(), "Bucket name must not be empty."); - - const auto paths = - construct_data_paths(base_path, dimensions, parts_along_dimension); - - return make_s3_sinks(bucket_name, paths, connection_pool, part_sinks); -} diff --git a/src/streaming/sink.hh b/src/streaming/sink.hh deleted file mode 100644 index 3c514d19..00000000 --- a/src/streaming/sink.hh +++ /dev/null @@ -1,149 +0,0 @@ -#pragma once - -#include "definitions.hh" -#include "s3.connection.hh" -#include "thread.pool.hh" -#include "array.dimensions.hh" - -#include // size_t -#include -#include // std::unique_ptr - -namespace zarr { -class Sink -{ - public: - virtual ~Sink() = default; - - /** - * @brief Write data to the sink. - * @param offset The offset in the sink to write to. - * @param data The buffer to write to the sink. - * @param bytes_of_buf The number of bytes to write from @p buf. - * @return True if the write was successful, false otherwise. - */ - [[nodiscard]] virtual bool write(size_t offset, ConstByteSpan data) = 0; - - protected: - /** - * @brief Flush any buffered data to the sink. - * @note This should ONLY be called when finalizing the sink. - * @return True if the flush was successful, false otherwise. - */ - [[nodiscard]] virtual bool flush_() = 0; - - friend bool finalize_sink(std::unique_ptr&& sink); -}; - -/** - * @brief Finalize and destroy @p sink. - * @note @p sink is no longer accessible after a successful call to this - * function. - * @param[in] sink The Sink to finalize. - * @return True if and only if the Sink was finalized successfully. - */ -bool -finalize_sink(std::unique_ptr&& sink); - -/** - * @brief Construct paths for data sinks, given the dimensions and a function - * to determine the number of parts along a dimension. - * @param base_path The base path for the dataset. - * @param dimensions The dimensions of the dataset. - * @param parts_along_dimension Function to determine the number of parts along - * a dimension. - * @return A vector of paths for the data sinks. - */ -std::vector -construct_data_paths(std::string_view base_path, - const ArrayDimensions& dimensions, - const DimensionPartsFun& parts_along_dimension); - -/** - * @brief Get unique paths to the parent directories of each file in @p - * file_paths. - * @param file_paths Collection of paths to files. - * @return Collection of unique parent directories. - */ -std::vector -get_parent_paths(const std::vector& file_paths); - -/** - * @brief Parallel create directories for a collection of paths. - * @param dir_paths The directories to create. - * @param thread_pool The thread pool to use for parallel creation. - * @return True iff all directories were created successfully. - */ -bool -make_dirs(const std::vector& dir_paths, - std::shared_ptr thread_pool); - -/** - * @brief Create a file sink from a path. - * @param file_path The path to the file. - * @param file_handle_pool Pointer to a pool of file handles. - * @return Pointer to the sink created, or nullptr if the file cannot be - * opened. - * @throws std::runtime_error if the file path is not valid. - */ -std::unique_ptr -make_file_sink(std::string_view file_path, - std::shared_ptr file_handle_pool); - -/** - * @brief Create a collection of file sinks for a Zarr dataset. - * @param[in] base_path The path to the base directory for the dataset. - * @param[in] dimensions The dimensions of the dataset. - * @param[in] parts_along_dimension Function to determine the number of - * parts (i.e., shards or chunks) along a dimension. - * @param[in] thread_pool Pointer to a thread pool object. Used to create files - * in parallel. - * @param file_handle_pool Pointer to a pool of file handles. - * @param[out] part_sinks The sinks created. - * @return True iff all file sinks were created successfully. - * @throws std::runtime_error if @p base_path is not valid, or if the number - * of parts along a dimension is zero. - */ -[[nodiscard]] bool -make_data_file_sinks(std::string_view base_path, - const ArrayDimensions& dimensions, - const DimensionPartsFun& parts_along_dimension, - std::shared_ptr thread_pool, - std::shared_ptr file_handle_pool, - std::vector>& part_sinks); - -/** - * @brief Create a sink from an S3 bucket name and object key. - * @param bucket_name The name of the bucket in which the object is stored. - * @param object_key The key of the object to write to. - * @param connection_pool Pointer to a pool of existing S3 connections. - * @return Pointer to the sink created, or nullptr if the bucket does not - * exist. - * @throws std::runtime_error if the bucket name or object key is not valid, - * or if there is no connection pool. - */ -std::unique_ptr -make_s3_sink(std::string_view bucket_name, - std::string_view object_key, - std::shared_ptr connection_pool); - -/** - * @brief Create a collection of S3 sinks for a Zarr dataset. - * @param[in] bucket_name The name of the bucket in which the dataset is - * stored. - * @param[in] base_path The path to the base directory for the dataset. - * @param[in] dimensions The dimensions of the dataset. - * @param[in] parts_along_dimension Function to determine the number of - * parts (i.e., shards or chunks) along a dimension. - * @param[in] connection_pool Pointer to a pool of existing S3 connections. - * @param[out] part_sinks The sinks created. - * @return True iff all file sinks were created successfully. - */ -[[nodiscard]] bool -make_data_s3_sinks(std::string_view bucket_name, - std::string_view base_path, - const ArrayDimensions& dimensions, - const DimensionPartsFun& parts_along_dimension, - std::shared_ptr connection_pool, - std::vector>& part_sinks); -} // namespace zarr diff --git a/src/streaming/thread.pool.cpp b/src/streaming/thread.pool.cpp index 37bc12ad..d934d961 100644 --- a/src/streaming/thread.pool.cpp +++ b/src/streaming/thread.pool.cpp @@ -34,8 +34,7 @@ bool zarr::ThreadPool::push_job(Task&& job) { std::unique_lock lock(jobs_mutex_); - // only allow pushing jobs from the main thread - if (!accepting_jobs || std::this_thread::get_id() != main_thread_id_) { + if (!accepting_jobs /*|| std::this_thread::get_id() != main_thread_id_*/) { return false; } @@ -105,5 +104,5 @@ zarr::ThreadPool::process_tasks_() uint32_t zarr::ThreadPool::n_threads() const { - return threads_.size(); + return threads_.size() - 1; // exclude frame queue thread } \ No newline at end of file diff --git a/src/streaming/win32/platform.cpp b/src/streaming/win32/platform.cpp index 1d033727..23de41e1 100644 --- a/src/streaming/win32/platform.cpp +++ b/src/streaming/win32/platform.cpp @@ -1,6 +1,6 @@ -#include "definitions.hh" #include "macros.hh" +#include #include #include @@ -77,7 +77,7 @@ init_handle(const std::string& filename, void* flags) } bool -seek_and_write(void* handle, size_t offset, ConstByteSpan data) +seek_and_write(void* handle, size_t offset, std::span data) { CHECK(handle); const auto* fd = static_cast(handle); @@ -121,7 +121,8 @@ bool flush_file(void* handle) { CHECK(handle); - if (const auto* fd = static_cast(handle); *fd != INVALID_HANDLE_VALUE) { + if (const auto* fd = static_cast(handle); + *fd != INVALID_HANDLE_VALUE) { return FlushFileBuffers(*fd); } return true; diff --git a/src/streaming/zarr.common.cpp b/src/streaming/zarr.common.cpp index 52b7bb49..861ef7cc 100644 --- a/src/streaming/zarr.common.cpp +++ b/src/streaming/zarr.common.cpp @@ -17,8 +17,7 @@ zarr::trim(std::string_view s) // trim left std::string trimmed(s); - trimmed.erase(trimmed.begin(), - std::find_if(trimmed.begin(), trimmed.end(), [](char c) { + trimmed.erase(trimmed.begin(), std::ranges::find_if(trimmed, [](char c) { return !std::isspace(c); })); @@ -103,7 +102,7 @@ zarr::shards_along_dimension(const ZarrDimension& dimension) } bool -zarr::compress_in_place(ByteVector& data, +zarr::compress_in_place(std::vector& data, const zarr::BloscCompressionParams& params, size_t type_size) { diff --git a/src/streaming/zarr.common.hh b/src/streaming/zarr.common.hh index 3f7b605e..af8d95c1 100644 --- a/src/streaming/zarr.common.hh +++ b/src/streaming/zarr.common.hh @@ -3,7 +3,6 @@ #include "acquire.zarr.h" #include "thread.pool.hh" #include "array.dimensions.hh" -#include "definitions.hh" #include "blosc.compression.params.hh" namespace zarr { @@ -82,7 +81,7 @@ shards_along_dimension(const ZarrDimension& dimension); * @return true if compression was successful, false otherwise. */ bool -compress_in_place(ByteVector& data, +compress_in_place(std::vector& data, const BloscCompressionParams& params, size_t type_size); diff --git a/src/streaming/zarr.stream.cpp b/src/streaming/zarr.stream.cpp index 32a1c258..04c977ea 100644 --- a/src/streaming/zarr.stream.cpp +++ b/src/streaming/zarr.stream.cpp @@ -1,7 +1,12 @@ #include "acquire.zarr.h" #include "array.base.hh" +#include "fs.array.hh" +#include "fs.multiscale.array.hh" +#include "fs.storage.hh" #include "macros.hh" -#include "sink.hh" +#include "s3.array.hh" +#include "s3.multiscale.array.hh" +#include "s3.storage.hh" #include "zarr.common.hh" #include "zarr.stream.hh" @@ -803,23 +808,6 @@ check_array_structure(std::vector> arrays, return true; } - -std::string -dimension_type_to_string(ZarrDimensionType type) -{ - switch (type) { - case ZarrDimensionType_Time: - return "time"; - case ZarrDimensionType_Channel: - return "channel"; - case ZarrDimensionType_Space: - return "space"; - case ZarrDimensionType_Other: - return "other"; - default: - return "(unknown)"; - } -} } // namespace /* ZarrStream_s implementation */ @@ -878,8 +866,9 @@ ZarrStream::append(const char* key_, const void* data_, size_t nbytes) const size_t bytes_to_copy = std::min(bytes_of_frame - frame_buffer_offset, bytes_remaining); - frame_buffer.assign_at(frame_buffer_offset, - { data + bytes_written, bytes_to_copy }); + memcpy(frame_buffer.data() + frame_buffer_offset, + data + bytes_written, + bytes_to_copy); frame_buffer_offset += bytes_to_copy; bytes_written += bytes_to_copy; @@ -902,12 +891,12 @@ ZarrStream::append(const char* key_, const void* data_, size_t nbytes) frame_buffer_offset = 0; } } else if (bytes_remaining < bytes_of_frame) { // begin partial frame - frame_buffer.assign_at(0, { data, bytes_remaining }); + memcpy(frame_buffer.data(), data, bytes_remaining); frame_buffer_offset = bytes_remaining; bytes_written += bytes_remaining; } else { // at least one full frame - zarr::LockedBuffer frame; - frame.assign({ data, bytes_of_frame }); + std::vector frame(bytes_of_frame); + frame.assign(data, data + bytes_of_frame); std::unique_lock lock(frame_queue_mutex_); while (!frame_queue_->push(frame, key) && process_frames_) { @@ -938,43 +927,33 @@ ZarrStream_s::write_custom_metadata(std::string_view custom_metadata, return ZarrStatusCode_InvalidArgument; } - // check if we have already written custom metadata - if (!custom_metadata_sink_) { - const std::string metadata_key = "acquire.json"; - std::string base_path = store_path_; - if (base_path.starts_with("file://")) { - base_path = base_path.substr(7); - } - const auto prefix = base_path.empty() ? "" : base_path + "/"; - const auto sink_path = prefix + metadata_key; + const std::string prefix = store_path_.empty() ? "" : store_path_ + "/"; + const std::string path = prefix + "acquire.json"; - if (is_s3_acquisition_()) { - custom_metadata_sink_ = zarr::make_s3_sink( - s3_settings_->bucket_name, sink_path, s3_connection_pool_); - } else { - custom_metadata_sink_ = - zarr::make_file_sink(sink_path, file_handle_pool_); - } - } else if (!overwrite) { // custom metadata already written, don't overwrite + // check if we have already written custom metadata + if (custom_metadata_ && + !overwrite) { // custom metadata already written, don't LOG_ERROR("Custom metadata already written, use overwrite flag"); return ZarrStatusCode_WillNotOverwrite; } - if (!custom_metadata_sink_) { - LOG_ERROR("Custom metadata sink not found"); - return ZarrStatusCode_InternalError; - } - const auto metadata_json = nlohmann::json::parse(custom_metadata, nullptr, // callback false, // allow exceptions true // ignore comments ); - const auto metadata_str = metadata_json.dump(4); - std::span data{ reinterpret_cast(metadata_str.data()), - metadata_str.size() }; - if (!custom_metadata_sink_->write(0, data)) { + custom_metadata_ = metadata_json.dump(4); + + bool success; + if (is_s3_acquisition_()) { + success = write_string_to_s3_( + s3_settings_->bucket_name, path, *custom_metadata_); + } else { + success = write_string_to_file_(path, *custom_metadata_); + } + + if (!success) { LOG_ERROR("Error writing custom metadata"); return ZarrStatusCode_IOError; } @@ -1168,8 +1147,23 @@ ZarrStream_s::configure_array_(const ZarrArraySettings* settings, ZarrOutputArray output_node{ .output_key = config->node_key, .frame_buffer_offset = 0 }; try { - output_node.array = zarr::make_array( - config, thread_pool_, file_handle_pool_, s3_connection_pool_); + const bool multiscale = + config->node_key.empty() || config->downsampling_method.has_value(); + const bool s3 = bucket_name.has_value(); + + if (multiscale && s3) { + output_node.array = zarr::make_array( + config, thread_pool_, s3_connection_pool_); + } else if (multiscale) { + output_node.array = zarr::make_array( + config, thread_pool_, file_handle_pool_); + } else if (s3) { + output_node.array = zarr::make_array( + config, thread_pool_, s3_connection_pool_); + } else { + output_node.array = zarr::make_array( + config, thread_pool_, file_handle_pool_); + } } catch (const std::exception& exc) { set_error_(exc.what()); } @@ -1185,7 +1179,8 @@ ZarrStream_s::configure_array_(const ZarrArraySettings* settings, dims->height_dim().array_size_px * zarr::bytes_of_type(settings->data_type); - output_node.frame_buffer.resize_and_fill(frame_size_bytes, 0); + output_node.frame_buffer.resize(frame_size_bytes); + std::ranges::fill(output_node.frame_buffer, 0); output_arrays_.emplace(output_node.output_key, std::move(output_node)); return true; @@ -1304,6 +1299,9 @@ bool ZarrStream_s::commit_settings_(const struct ZarrStreamSettings_s* settings) { store_path_ = zarr::trim(settings->store_path); + if (store_path_.starts_with("file://")) { + store_path_ = store_path_.substr(7); + } std::optional bucket_name; s3_settings_ = make_s3_settings(settings->s3_settings); @@ -1414,11 +1412,6 @@ ZarrStream_s::create_store_(bool overwrite) bool ZarrStream_s::write_intermediate_metadata_() { - std::optional bucket_name; - if (s3_settings_) { - bucket_name = s3_settings_->bucket_name; - } - const nlohmann::json group_metadata = nlohmann::json({ { "zarr_format", 3 }, { "consolidated_metadata", nullptr }, @@ -1462,25 +1455,17 @@ ZarrStream_s::write_intermediate_metadata_() metadata_str = group_metadata.dump(4); } - ConstByteSpan metadata_span( - reinterpret_cast(metadata_str.data()), - metadata_str.size()); - - const std::string sink_path = + const std::string path = store_path_ + "/" + relative_path + "/" + metadata_key; - std::unique_ptr metadata_sink; if (is_s3_acquisition_()) { - metadata_sink = zarr::make_s3_sink( - bucket_name.value(), sink_path, s3_connection_pool_); + if (!write_string_to_s3_( + s3_settings_->bucket_name, path, metadata_str)) { + return false; + } } else { - metadata_sink = zarr::make_file_sink(sink_path, file_handle_pool_); - } - - if (!metadata_sink->write(0, metadata_span) || - !zarr::finalize_sink(std::move(metadata_sink))) { - set_error_("Failed to write intermediate metadata for group '" + - parent_group_key + "'"); - return false; + if (!write_string_to_file_(path, metadata_str)) { + return false; + } } } @@ -1506,7 +1491,7 @@ ZarrStream_s::init_frame_queue_() } // cap the frame buffer at 1 GiB, or 10 frames, whichever is larger - const auto buffer_size_bytes = 1ULL << 30; + constexpr auto buffer_size_bytes = 1ULL << 30; const auto frame_count = std::max(10ULL, buffer_size_bytes / frame_size_bytes); @@ -1547,8 +1532,8 @@ ZarrStream_s::process_frame_queue_() std::string output_key; - zarr::LockedBuffer frame; - while (process_frames_ || !frame_queue_->empty()) { + std::vector frame; + while (process_frames_) { { std::unique_lock lock(frame_queue_mutex_); while (frame_queue_->empty() && process_frames_) { @@ -1563,9 +1548,10 @@ ZarrStream_s::process_frame_queue_() // done if (!process_frames_) { break; - } else { - continue; } + + // spurious wakeup, go back to waiting + continue; } } @@ -1604,11 +1590,36 @@ ZarrStream_s::process_frame_queue_() } } + // finish frame queue if (!frame_queue_->empty()) { - LOG_WARNING("Reached end of frame queue processing with ", - frame_queue_->size(), - " frames remaining on queue"); - frame_queue_->clear(); + const size_t frames_remaining = frame_queue_->size(); + for (size_t i = 0; i < frames_remaining; ++i) { + if (!frame_queue_->pop(frame, output_key)) { + continue; + } + + if (auto it = output_arrays_.find(output_key); + it == output_arrays_.end()) { + // If we have gotten here, something has gone seriously wrong + set_error_("Output node not found for key: '" + output_key + + "'"); + std::unique_lock lock(frame_queue_mutex_); + frame_queue_finished_cv_.notify_all(); + return; + } else { + auto& output_node = it->second; + + if (output_node.array->write_frame(frame) != frame.size()) { + set_error_("Failed to write frame to writer for key: " + + output_key); + std::unique_lock lock(frame_queue_mutex_); + frame_queue_finished_cv_.notify_all(); + return; + } + } + } + + frame_queue_empty_cv_.notify_all(); // queue is now empty } std::unique_lock lock(frame_queue_mutex_); @@ -1648,12 +1659,6 @@ finalize_stream(struct ZarrStream_s* stream) // thread stream->thread_pool_->await_stop(); - if (stream->custom_metadata_sink_ && - !zarr::finalize_sink(std::move(stream->custom_metadata_sink_))) { - LOG_ERROR( - "Error finalizing Zarr stream. Failed to write custom metadata"); - } - for (auto& [key, output] : stream->output_arrays_) { if (!zarr::finalize_array(std::move(output.array))) { LOG_ERROR( @@ -1671,3 +1676,25 @@ finalize_stream(struct ZarrStream_s* stream) return true; } + +bool +ZarrStream_s::write_string_to_file_(const std::string& path, + const std::string& data) const +{ + EXPECT(file_handle_pool_ != nullptr, "File handle pool is not initialized"); + + zarr::FSStorage storage(file_handle_pool_); + return storage.write_string(path, data, 0); +} + +bool +ZarrStream_s::write_string_to_s3_(const std::string& bucket_name, + const std::string& key, + const std::string& data) const +{ + EXPECT(s3_connection_pool_ != nullptr, + "S3 connection pool is not initialized"); + + zarr::S3Storage storage(bucket_name, s3_connection_pool_); + return storage.write_string(key, data, 0) && storage.finalize_object(key); +} diff --git a/src/streaming/zarr.stream.hh b/src/streaming/zarr.stream.hh index 40d2fd0d..66e153c5 100644 --- a/src/streaming/zarr.stream.hh +++ b/src/streaming/zarr.stream.hh @@ -1,33 +1,24 @@ #pragma once -#include "array.hh" +#include "array.base.hh" #include "array.dimensions.hh" -#include "definitions.hh" -#include "downsampler.hh" #include "file.handle.hh" #include "frame.queue.hh" -#include "locked.buffer.hh" -#include "multiscale.array.hh" #include "plate.hh" #include "s3.connection.hh" -#include "sink.hh" #include "thread.pool.hh" -#include - #include -#include // size_t -#include // unique_ptr +#include // unique_ptr #include #include -#include #include #include struct ZarrStream_s { public: - ZarrStream_s(struct ZarrStreamSettings_s* settings); + explicit ZarrStream_s(struct ZarrStreamSettings_s* settings); /** * @brief Append data to the stream with a specific key. @@ -58,7 +49,7 @@ struct ZarrStream_s struct ZarrOutputArray { std::string output_key; - zarr::LockedBuffer frame_buffer; + std::vector frame_buffer; size_t frame_buffer_offset; std::unique_ptr array; }; @@ -87,7 +78,7 @@ struct ZarrStream_s std::shared_ptr s3_connection_pool_; std::shared_ptr file_handle_pool_; - std::unique_ptr custom_metadata_sink_; + std::optional custom_metadata_; bool is_s3_acquisition_() const; @@ -162,6 +153,15 @@ struct ZarrStream_s /** @brief Wait for the frame queue to finish processing. */ void finalize_frame_queue_(); + /** @brief Write a string @p data to a file @p path. */ + bool write_string_to_file_(const std::string& path, + const std::string& data) const; + + /** @brief Write a string @p data to an S3 object @p key on @p bucket. */ + bool write_string_to_s3_(const std::string& bucket_name, + const std::string& key, + const std::string& data) const; + friend bool finalize_stream(struct ZarrStream_s* stream); }; diff --git a/tests/integration/stream-multiple-arrays-to-filesystem.cpp b/tests/integration/stream-multiple-arrays-to-filesystem.cpp index 56cdd967..dcd16d2e 100644 --- a/tests/integration/stream-multiple-arrays-to-filesystem.cpp +++ b/tests/integration/stream-multiple-arrays-to-filesystem.cpp @@ -287,8 +287,8 @@ verify_codecs(const nlohmann::json& metadata, EXPECT(codec_config.contains("index_location"), "Expected key 'index_location' in codec configuration"); auto index_location = codec_config["index_location"].get(); - EXPECT(index_location == "end", - "Expected index_location to be 'end', got '", + EXPECT(index_location == "start", + "Expected index_location to be 'start', got '", index_location, "'"); diff --git a/tests/integration/stream-raw-to-filesystem.cpp b/tests/integration/stream-raw-to-filesystem.cpp index 974cddcd..e6f9fd86 100644 --- a/tests/integration/stream-raw-to-filesystem.cpp +++ b/tests/integration/stream-raw-to-filesystem.cpp @@ -13,44 +13,31 @@ namespace { const std::string test_path = (fs::temp_directory_path() / (TEST ".zarr")).string(); -const unsigned int array_width = 64, array_height = 48, array_planes = 6, - array_channels = 8, array_timepoints = 10; - -const unsigned int chunk_width = 16, chunk_height = 16, chunk_planes = 2, - chunk_channels = 4, chunk_timepoints = 5; - -const unsigned int shard_width = 2, shard_height = 1, shard_planes = 1, - shard_channels = 2, shard_timepoints = 2; -const unsigned int chunks_per_shard = - shard_width * shard_height * shard_planes * shard_channels * shard_timepoints; - -const unsigned int chunks_in_x = +constexpr unsigned int array_width = 2048, array_height = 2048, + array_planes = 1024; +constexpr unsigned int chunk_width = 64, chunk_height = 64, chunk_planes = 64; +constexpr unsigned int shard_width = 16, shard_height = 16, shard_planes = 1; +constexpr unsigned int chunks_per_shard = + shard_width * shard_height * shard_planes; + +constexpr unsigned int chunks_in_x = (array_width + chunk_width - 1) / chunk_width; // 4 chunks -const unsigned int chunks_in_y = +constexpr unsigned int chunks_in_y = (array_height + chunk_height - 1) / chunk_height; // 3 chunks -const unsigned int chunks_in_z = +constexpr unsigned int chunks_in_z = (array_planes + chunk_planes - 1) / chunk_planes; // 3 chunks -const unsigned int chunks_in_c = - (array_channels + chunk_channels - 1) / chunk_channels; // 2 chunks -const unsigned int chunks_in_t = - (array_timepoints + chunk_timepoints - 1) / chunk_timepoints; -const unsigned int shards_in_x = +constexpr unsigned int shards_in_x = (chunks_in_x + shard_width - 1) / shard_width; // 2 shards -const unsigned int shards_in_y = +constexpr unsigned int shards_in_y = (chunks_in_y + shard_height - 1) / shard_height; // 3 shards -const unsigned int shards_in_z = +constexpr unsigned int shards_in_z = (chunks_in_z + shard_planes - 1) / shard_planes; // 3 shards -const unsigned int shards_in_c = - (chunks_in_c + shard_channels - 1) / shard_channels; // 1 shard -const unsigned int shards_in_t = - (chunks_in_t + shard_timepoints - 1) / shard_timepoints; // 1 shard -const size_t nbytes_px = sizeof(uint16_t); -const uint32_t frames_to_acquire = - array_planes * array_channels * array_timepoints; -const size_t bytes_of_frame = array_width * array_height * nbytes_px; -} // namespace/s +constexpr size_t nbytes_px = sizeof(uint16_t); +constexpr uint32_t frames_to_acquire = array_planes; +constexpr size_t bytes_of_frame = array_width * array_height * nbytes_px; +} // namespace ZarrStream* setup() @@ -67,27 +54,9 @@ setup() .array_count = 1, }; - CHECK_OK(ZarrArraySettings_create_dimension_array(settings.arrays, 5)); + CHECK_OK(ZarrArraySettings_create_dimension_array(settings.arrays, 3)); ZarrDimensionProperties* dim = settings.arrays->dimensions; - *dim = DIM("t", - ZarrDimensionType_Time, - array_timepoints, - chunk_timepoints, - shard_timepoints, - nullptr, - 1.0); - - dim = settings.arrays->dimensions + 1; - *dim = DIM("c", - ZarrDimensionType_Channel, - array_channels, - chunk_channels, - shard_channels, - nullptr, - 1.0); - - dim = settings.arrays->dimensions + 2; *dim = DIM("z", ZarrDimensionType_Space, array_planes, @@ -95,8 +64,8 @@ setup() shard_planes, "millimeter", 1.4); + ++dim; - dim = settings.arrays->dimensions + 3; *dim = DIM("y", ZarrDimensionType_Space, array_height, @@ -104,8 +73,8 @@ setup() shard_height, "micrometer", 0.9); + ++dim; - dim = settings.arrays->dimensions + 4; *dim = DIM("x", ZarrDimensionType_Space, array_width, @@ -142,29 +111,12 @@ verify_group_metadata(const nlohmann::json& meta) "'"); const auto axes = multiscales["axes"]; - EXPECT_EQ(size_t, axes.size(), 5); + EXPECT_EQ(size_t, axes.size(), 3); std::string name, type, unit; name = axes[0]["name"]; type = axes[0]["type"]; - EXPECT(name == "t", "Expected name to be 't', but got '", name, "'"); - EXPECT(type == "time", "Expected type to be 'time', but got '", type, "'"); - EXPECT(!axes[0].contains("unit"), - "Expected unit to be missing, got ", - axes[0]["unit"].get()); - - name = axes[1]["name"]; - type = axes[1]["type"]; - EXPECT(name == "c", "Expected name to be 'c', but got '", name, "'"); - EXPECT( - type == "channel", "Expected type to be 'channel', but got '", type, "'"); - EXPECT(!axes[1].contains("unit"), - "Expected unit to be missing, got ", - axes[1]["unit"].get()); - - name = axes[2]["name"]; - type = axes[2]["type"]; - unit = axes[2]["unit"]; + unit = axes[0]["unit"]; EXPECT(name == "z", "Expected name to be 'z', but got '", name, "'"); EXPECT( type == "space", "Expected type to be 'space', but got '", type, "'"); @@ -173,9 +125,9 @@ verify_group_metadata(const nlohmann::json& meta) unit, "'"); - name = axes[3]["name"]; - type = axes[3]["type"]; - unit = axes[3]["unit"]; + name = axes[1]["name"]; + type = axes[1]["type"]; + unit = axes[1]["unit"]; EXPECT(name == "y", "Expected name to be 'y', but got '", name, "'"); EXPECT( type == "space", "Expected type to be 'space', but got '", type, "'"); @@ -184,9 +136,9 @@ verify_group_metadata(const nlohmann::json& meta) unit, "'"); - name = axes[4]["name"]; - type = axes[4]["type"]; - unit = axes[4]["unit"]; + name = axes[2]["name"]; + type = axes[2]["type"]; + unit = axes[2]["unit"]; EXPECT(name == "x", "Expected name to be 'x', but got '", name, "'"); EXPECT( type == "space", "Expected type to be 'space', but got '", type, "'"); @@ -207,32 +159,26 @@ verify_group_metadata(const nlohmann::json& meta) type == "scale", "Expected type to be 'scale', but got '", type, "'"); const auto scale = coordinate_transformations["scale"]; - EXPECT_EQ(size_t, scale.size(), 5); - EXPECT_EQ(int, scale[0].get(), 1.0); - EXPECT_EQ(int, scale[1].get(), 1.0); - EXPECT_EQ(int, scale[2].get(), 1.4); - EXPECT_EQ(int, scale[3].get(), 0.9); - EXPECT_EQ(int, scale[4].get(), 0.9); + EXPECT_EQ(size_t, scale.size(), 3); + EXPECT_EQ(int, scale[0].get(), 1.4); + EXPECT_EQ(int, scale[1].get(), 0.9); + EXPECT_EQ(int, scale[2].get(), 0.9); } void verify_array_metadata(const nlohmann::json& meta) { const auto& shape = meta["shape"]; - EXPECT_EQ(size_t, shape.size(), 5); - EXPECT_EQ(int, shape[0].get(), array_timepoints); - EXPECT_EQ(int, shape[1].get(), array_channels); - EXPECT_EQ(int, shape[2].get(), array_planes); - EXPECT_EQ(int, shape[3].get(), array_height); - EXPECT_EQ(int, shape[4].get(), array_width); + EXPECT_EQ(size_t, shape.size(), 3); + EXPECT_EQ(int, shape[0].get(), array_planes); + EXPECT_EQ(int, shape[1].get(), array_height); + EXPECT_EQ(int, shape[2].get(), array_width); const auto& chunks = meta["chunk_grid"]["configuration"]["chunk_shape"]; - EXPECT_EQ(size_t, chunks.size(), 5); - EXPECT_EQ(int, chunks[0].get(), chunk_timepoints* shard_timepoints); - EXPECT_EQ(int, chunks[1].get(), chunk_channels* shard_channels); - EXPECT_EQ(int, chunks[2].get(), chunk_planes* shard_planes); - EXPECT_EQ(int, chunks[3].get(), chunk_height* shard_height); - EXPECT_EQ(int, chunks[4].get(), chunk_width* shard_width); + EXPECT_EQ(size_t, chunks.size(), 3); + EXPECT_EQ(int, chunks[0].get(), chunk_planes* shard_planes); + EXPECT_EQ(int, chunks[1].get(), chunk_height* shard_height); + EXPECT_EQ(int, chunks[2].get(), chunk_width* shard_width); const auto dtype = meta["data_type"].get(); EXPECT(dtype == "uint16", @@ -245,12 +191,10 @@ verify_array_metadata(const nlohmann::json& meta) const auto& sharding_codec = codecs[0]["configuration"]; const auto& shards = sharding_codec["chunk_shape"]; - EXPECT_EQ(size_t, shards.size(), 5); - EXPECT_EQ(int, shards[0].get(), chunk_timepoints); - EXPECT_EQ(int, shards[1].get(), chunk_channels); - EXPECT_EQ(int, shards[2].get(), chunk_planes); - EXPECT_EQ(int, shards[3].get(), chunk_height); - EXPECT_EQ(int, shards[4].get(), chunk_width); + EXPECT_EQ(size_t, shards.size(), 3); + EXPECT_EQ(int, shards[0].get(), chunk_planes); + EXPECT_EQ(int, shards[1].get(), chunk_height); + EXPECT_EQ(int, shards[2].get(), chunk_width); const auto& internal_codecs = sharding_codec["codecs"]; EXPECT(internal_codecs.size() == 1, @@ -262,85 +206,63 @@ verify_array_metadata(const nlohmann::json& meta) internal_codecs[0]["name"].get()); const auto& dimension_names = meta["dimension_names"]; - EXPECT_EQ(size_t, dimension_names.size(), 5); + EXPECT_EQ(size_t, dimension_names.size(), 3); - EXPECT(dimension_names[0].get() == "t", - "Expected first dimension name to be 't', got ", - dimension_names[0].get()); - EXPECT(dimension_names[1].get() == "c", - "Expected second dimension name to be 'c', got ", - dimension_names[1].get()); - EXPECT(dimension_names[2].get() == "z", + EXPECT(dimension_names[0].get() == "z", "Expected third dimension name to be 'z', got ", - dimension_names[2].get()); - EXPECT(dimension_names[3].get() == "y", + dimension_names[0].get()); + EXPECT(dimension_names[1].get() == "y", "Expected fourth dimension name to be 'y', got ", - dimension_names[3].get()); - EXPECT(dimension_names[4].get() == "x", + dimension_names[1].get()); + EXPECT(dimension_names[2].get() == "x", "Expected fifth dimension name to be 'x', got ", - dimension_names[4].get()); + dimension_names[2].get()); } void verify_file_data() { - const auto chunk_size = chunk_width * chunk_height * chunk_planes * - chunk_channels * chunk_timepoints * nbytes_px; + const auto chunk_size = + chunk_width * chunk_height * chunk_planes * nbytes_px; const auto index_size = chunks_per_shard * sizeof(uint64_t) * // indices are 64 bits 2; // 2 indices per chunk const auto checksum_size = 4; // crc32 checksum is 4 bytes - const auto expected_file_size = shard_width * shard_height * shard_planes * - shard_channels * shard_timepoints * - chunk_size + - index_size + checksum_size; + const auto expected_file_size = + shard_width * shard_height * shard_planes * chunk_size + index_size + + checksum_size; fs::path data_root = fs::path(test_path) / "0"; CHECK(fs::is_directory(data_root)); - for (auto t = 0; t < shards_in_t; ++t) { - const auto t_dir = data_root / "c" / std::to_string(t); - CHECK(fs::is_directory(t_dir)); - - for (auto c = 0; c < shards_in_c; ++c) { - const auto c_dir = t_dir / std::to_string(c); - CHECK(fs::is_directory(c_dir)); - - for (auto z = 0; z < shards_in_z; ++z) { - const auto z_dir = c_dir / std::to_string(z); - CHECK(fs::is_directory(z_dir)); - - for (auto y = 0; y < shards_in_y; ++y) { - const auto y_dir = z_dir / std::to_string(y); - CHECK(fs::is_directory(y_dir)); - - for (auto x = 0; x < shards_in_x; ++x) { - const auto x_file = y_dir / std::to_string(x); - CHECK(fs::is_regular_file(x_file)); - const auto file_size = fs::file_size(x_file); - EXPECT(file_size == expected_file_size, - "Expected file size == ", - expected_file_size, - " for file ", - x_file.string(), - ", got ", - file_size); - } - - CHECK(!fs::is_regular_file(y_dir / - std::to_string(shards_in_x))); - } - - CHECK(!fs::is_directory(z_dir / std::to_string(shards_in_y))); + for (auto z = 0; z < shards_in_z; ++z) { + const auto z_dir = data_root / "c" / std::to_string(z); + CHECK(fs::is_directory(z_dir)); + + for (auto y = 0; y < shards_in_y; ++y) { + const auto y_dir = z_dir / std::to_string(y); + CHECK(fs::is_directory(y_dir)); + + for (auto x = 0; x < shards_in_x; ++x) { + const auto x_file = y_dir / std::to_string(x); + CHECK(fs::is_regular_file(x_file)); + const auto file_size = fs::file_size(x_file); + EXPECT(file_size == expected_file_size, + "Expected file size == ", + expected_file_size, + " for file ", + x_file.string(), + ", got ", + file_size); } - CHECK(!fs::is_directory(c_dir / std::to_string(shards_in_z))); + CHECK(!fs::is_regular_file(y_dir / std::to_string(shards_in_x))); } - CHECK(!fs::is_directory(t_dir / std::to_string(shards_in_c))); + CHECK(!fs::is_directory(z_dir / std::to_string(shards_in_y))); } - CHECK(!fs::is_directory(data_root / "c" / std::to_string(shards_in_t))); + CHECK(!fs::is_directory(data_root / "c" / std::to_string(shards_in_z))); } void @@ -378,18 +300,22 @@ verify() int main() { - Zarr_set_log_level(ZarrLogLevel_Debug); + Zarr_set_log_level(ZarrLogLevel_Info); auto* stream = setup(); - std::vector frame(array_width * array_height, 0); + const std::vector frame(array_width * array_height, 0); int retval = 1; try { size_t bytes_out; for (auto i = 0; i < frames_to_acquire; ++i) { - ZarrStatusCode status = ZarrStream_append( + const auto t1 = std::chrono::high_resolution_clock::now(); + const ZarrStatusCode status = ZarrStream_append( stream, frame.data(), bytes_of_frame, &bytes_out, nullptr); + const auto t2 = std::chrono::high_resolution_clock::now(); + const std::chrono::duration fp_ms = t2 - t1; + LOG_INFO("Appending frame ", i, " took ", fp_ms.count(), " ms"); EXPECT(status == ZarrStatusCode_Success, "Failed to append frame ", i, @@ -398,7 +324,11 @@ main() EXPECT_EQ(size_t, bytes_out, bytes_of_frame); } + const auto t1 = std::chrono::high_resolution_clock::now(); ZarrStream_destroy(stream); + const auto t2 = std::chrono::high_resolution_clock::now(); + const std::chrono::duration fp_ms = t2 - t1; + LOG_INFO("Closing stream took ", fp_ms.count(), " ms"); verify(); diff --git a/tests/unit-tests/CMakeLists.txt b/tests/unit-tests/CMakeLists.txt index 9f84209c..45853fcb 100644 --- a/tests/unit-tests/CMakeLists.txt +++ b/tests/unit-tests/CMakeLists.txt @@ -9,16 +9,12 @@ set(tests array-dimensions-shard-index-for-chunk array-dimensions-shard-internal-index thread-pool-push-to-job-queue - make-dirs - construct-data-paths s3-connection-bucket-exists s3-connection-object-exists-check-false-positives s3-connection-put-object s3-connection-upload-multipart-object - file-sink-write s3-sink-write s3-sink-write-multipart - make-data-sinks array-write-even array-write-ragged-append-dim array-write-ragged-internal-dim diff --git a/tests/unit-tests/array-dimensions-chunk-internal-offset.cpp b/tests/unit-tests/array-dimensions-chunk-internal-offset.cpp index 531389db..0ce05270 100644 --- a/tests/unit-tests/array-dimensions-chunk-internal-offset.cpp +++ b/tests/unit-tests/array-dimensions-chunk-internal-offset.cpp @@ -10,11 +10,11 @@ main() std::vector dims; dims.emplace_back( - "t", ZarrDimensionType_Time, 0, 5, 0); // 5 timepoints / chunk - dims.emplace_back("c", ZarrDimensionType_Channel, 3, 2, 0); // 2 chunks - dims.emplace_back("z", ZarrDimensionType_Space, 5, 2, 0); // 3 chunks - dims.emplace_back("y", ZarrDimensionType_Space, 48, 16, 0); // 3 chunks - dims.emplace_back("x", ZarrDimensionType_Space, 64, 16, 0); // 4 chunks + "t", ZarrDimensionType_Time, 0, 5, 1); // 5 timepoints / chunk + dims.emplace_back("c", ZarrDimensionType_Channel, 3, 2, 1); // 2 chunks + dims.emplace_back("z", ZarrDimensionType_Space, 5, 2, 1); // 3 chunks + dims.emplace_back("y", ZarrDimensionType_Space, 48, 16, 1); // 3 chunks + dims.emplace_back("x", ZarrDimensionType_Space, 64, 16, 1); // 4 chunks ArrayDimensions dimensions(std::move(dims), ZarrDataType_uint16); try { diff --git a/tests/unit-tests/array-dimensions-chunk-lattice-index.cpp b/tests/unit-tests/array-dimensions-chunk-lattice-index.cpp index 53bbcbb0..d8de6859 100644 --- a/tests/unit-tests/array-dimensions-chunk-lattice-index.cpp +++ b/tests/unit-tests/array-dimensions-chunk-lattice-index.cpp @@ -11,11 +11,11 @@ main() try { std::vector dims; dims.emplace_back( - "t", ZarrDimensionType_Time, 0, 5, 0); // 5 timepoints / chunk - dims.emplace_back("c", ZarrDimensionType_Channel, 3, 2, 0); // 2 chunks - dims.emplace_back("z", ZarrDimensionType_Space, 5, 2, 0); // 3 chunks - dims.emplace_back("y", ZarrDimensionType_Space, 48, 16, 0); // 3 chunks - dims.emplace_back("x", ZarrDimensionType_Space, 64, 16, 0); // 4 chunks + "t", ZarrDimensionType_Time, 0, 5, 1); // 5 timepoints / chunk + dims.emplace_back("c", ZarrDimensionType_Channel, 3, 2, 1); // 2 chunks + dims.emplace_back("z", ZarrDimensionType_Space, 5, 2, 1); // 3 chunks + dims.emplace_back("y", ZarrDimensionType_Space, 48, 16, 1); // 3 chunks + dims.emplace_back("x", ZarrDimensionType_Space, 64, 16, 1); // 4 chunks ArrayDimensions dimensions(std::move(dims), ZarrDataType_uint8); EXPECT_EQ(int, dimensions.chunk_lattice_index(0, 2), 0); diff --git a/tests/unit-tests/array-dimensions-tile-group-offset.cpp b/tests/unit-tests/array-dimensions-tile-group-offset.cpp index c62368db..ca0dd2f2 100644 --- a/tests/unit-tests/array-dimensions-tile-group-offset.cpp +++ b/tests/unit-tests/array-dimensions-tile-group-offset.cpp @@ -10,11 +10,11 @@ main() std::vector dims; dims.emplace_back( - "t", ZarrDimensionType_Time, 0, 5, 0); // 5 timepoints / chunk - dims.emplace_back("c", ZarrDimensionType_Channel, 3, 2, 0); // 2 chunks - dims.emplace_back("z", ZarrDimensionType_Space, 5, 2, 0); // 3 chunks - dims.emplace_back("y", ZarrDimensionType_Space, 48, 16, 0); // 3 chunks - dims.emplace_back("x", ZarrDimensionType_Space, 64, 16, 0); // 4 chunks + "t", ZarrDimensionType_Time, 0, 5, 1); // 5 timepoints / chunk + dims.emplace_back("c", ZarrDimensionType_Channel, 3, 2, 1); // 2 chunks + dims.emplace_back("z", ZarrDimensionType_Space, 5, 2, 1); // 3 chunks + dims.emplace_back("y", ZarrDimensionType_Space, 48, 16, 1); // 3 chunks + dims.emplace_back("x", ZarrDimensionType_Space, 64, 16, 1); // 4 chunks ArrayDimensions dimensions(std::move(dims), ZarrDataType_float32); try { diff --git a/tests/unit-tests/array-write-even.cpp b/tests/unit-tests/array-write-even.cpp index f60cc442..45ccefb1 100644 --- a/tests/unit-tests/array-write-even.cpp +++ b/tests/unit-tests/array-write-even.cpp @@ -1,51 +1,54 @@ -#include "array.hh" +#include "fs.array.hh" #include "unit.test.macros.hh" #include "zarr.common.hh" #include #include +#include namespace fs = std::filesystem; namespace { const fs::path base_dir = fs::temp_directory_path() / TEST; -const unsigned int array_width = 64, array_height = 48, array_planes = 6, - array_channels = 8, array_timepoints = 10; -const unsigned int n_frames = array_planes * array_channels * array_timepoints; +constexpr unsigned int array_width = 64, array_height = 48, array_planes = 6, + array_channels = 8, array_timepoints = 10; -const unsigned int chunk_width = 16, chunk_height = 16, chunk_planes = 2, - chunk_channels = 4, chunk_timepoints = 5; +constexpr unsigned int chunk_width = 16, chunk_height = 16, chunk_planes = 2, + chunk_channels = 4, chunk_timepoints = 5; -const unsigned int shard_width = 2, shard_height = 1, shard_planes = 1, - shard_channels = 2, shard_timepoints = 2; -const unsigned int chunks_per_shard = +constexpr unsigned int shard_width = 2, shard_height = 1, shard_planes = 1, + shard_channels = 2, shard_timepoints = 2; +constexpr unsigned int chunks_per_shard = shard_width * shard_height * shard_planes * shard_channels * shard_timepoints; -const unsigned int chunks_in_x = +constexpr unsigned int n_frames = + array_planes * array_channels * chunk_timepoints * shard_timepoints; + +constexpr unsigned int chunks_in_x = (array_width + chunk_width - 1) / chunk_width; // 4 chunks -const unsigned int chunks_in_y = +constexpr unsigned int chunks_in_y = (array_height + chunk_height - 1) / chunk_height; // 3 chunks -const unsigned int chunks_in_z = +constexpr unsigned int chunks_in_z = (array_planes + chunk_planes - 1) / chunk_planes; // 3 chunks -const unsigned int chunks_in_c = +constexpr unsigned int chunks_in_c = (array_channels + chunk_channels - 1) / chunk_channels; // 2 chunks -const unsigned int chunks_in_t = +constexpr unsigned int chunks_in_t = (array_timepoints + chunk_timepoints - 1) / chunk_timepoints; -const unsigned int shards_in_x = +constexpr unsigned int shards_in_x = (chunks_in_x + shard_width - 1) / shard_width; // 2 shards -const unsigned int shards_in_y = +constexpr unsigned int shards_in_y = (chunks_in_y + shard_height - 1) / shard_height; // 3 shards -const unsigned int shards_in_z = +constexpr unsigned int shards_in_z = (chunks_in_z + shard_planes - 1) / shard_planes; // 3 shards -const unsigned int shards_in_c = +constexpr unsigned int shards_in_c = (chunks_in_c + shard_channels - 1) / shard_channels; // 1 shard -const unsigned int shards_in_t = +constexpr unsigned int shards_in_t = (chunks_in_t + shard_timepoints - 1) / shard_timepoints; // 1 shard -const int level_of_detail = 3; +constexpr int level_of_detail = 3; } // namespace void @@ -99,13 +102,12 @@ main() int retval = 1; - const ZarrDataType dtype = ZarrDataType_uint16; + constexpr ZarrDataType dtype = ZarrDataType_uint16; const unsigned int nbytes_px = zarr::bytes_of_type(dtype); - // try { + try { auto thread_pool = std::make_shared( - std::thread::hardware_concurrency(), - [](const std::string& err) { LOG_ERROR("Error: ", err); }); + 1, [](const std::string& err) { LOG_ERROR("Error: ", err); }); std::vector dims; dims.emplace_back("t", @@ -131,25 +133,23 @@ main() dims.emplace_back( "x", ZarrDimensionType_Space, array_width, chunk_width, shard_width); - auto config = std::make_shared( - base_dir.string(), - "", - std::nullopt, - std::nullopt, - std::make_shared(std::move(dims), dtype), - dtype, - std::nullopt, - level_of_detail); - + // write the data { - auto writer = std::make_unique( - config, - thread_pool, - std::make_shared(), - nullptr); + const auto config = std::make_shared( + base_dir.string(), + "", + std::nullopt, + std::nullopt, + std::make_shared(std::move(dims), dtype), + dtype, + std::nullopt, + level_of_detail); + + auto writer = std::make_unique( + config, thread_pool, std::make_shared()); const size_t frame_size = array_width * array_height * nbytes_px; - zarr::LockedBuffer data(std::move(ByteVector(frame_size, 0))); + std::vector data(frame_size, 0); for (auto i = 0; i < n_frames; ++i) { // 2 time points CHECK(writer->write_frame(data)); @@ -211,9 +211,9 @@ main() CHECK(!fs::is_directory(data_root / "c" / std::to_string(shards_in_t))); retval = 0; - // } catch (const std::exception& exc) { - // LOG_ERROR("Exception: ", exc.what()); - // } + } catch (const std::exception& exc) { + LOG_ERROR("Exception: ", exc.what()); + } // cleanup if (fs::exists(base_dir)) { diff --git a/tests/unit-tests/array-write-ragged-append-dim.cpp b/tests/unit-tests/array-write-ragged-append-dim.cpp index f6d1d8f4..f32ce514 100644 --- a/tests/unit-tests/array-write-ragged-append-dim.cpp +++ b/tests/unit-tests/array-write-ragged-append-dim.cpp @@ -1,39 +1,39 @@ -#include "array.hh" +#include "fs.array.hh" #include "unit.test.macros.hh" #include "zarr.common.hh" #include #include +#include namespace fs = std::filesystem; namespace { const fs::path base_dir = fs::temp_directory_path() / TEST; -const unsigned int array_width = 64, array_height = 48, array_planes = 5; -const unsigned int n_frames = array_planes; +constexpr unsigned int array_width = 64, array_height = 48, array_planes = 5; +constexpr unsigned int n_frames = array_planes; -const unsigned int chunk_width = 16, chunk_height = 16, chunk_planes = 2; +constexpr unsigned int chunk_width = 16, chunk_height = 16, chunk_planes = 2; -const unsigned int shard_width = 2, shard_height = 1, shard_planes = 1; -const unsigned int chunks_per_shard = shard_width * shard_height * shard_planes; +constexpr unsigned int shard_width = 2, shard_height = 1, shard_planes = 1; +constexpr unsigned int chunks_per_shard = + shard_width * shard_height * shard_planes; -const unsigned int chunks_in_x = +constexpr unsigned int chunks_in_x = (array_width + chunk_width - 1) / chunk_width; // 4 chunks -const unsigned int chunks_in_y = +constexpr unsigned int chunks_in_y = (array_height + chunk_height - 1) / chunk_height; // 3 chunks -const unsigned int chunks_in_z = +constexpr unsigned int chunks_in_z = (array_planes + chunk_planes - 1) / chunk_planes; // 3 chunks -const unsigned int shards_in_x = +constexpr unsigned int shards_in_x = (chunks_in_x + shard_width - 1) / shard_width; // 2 shards -const unsigned int shards_in_y = +constexpr unsigned int shards_in_y = (chunks_in_y + shard_height - 1) / shard_height; // 3 shards -const unsigned int shards_in_z = +constexpr unsigned int shards_in_z = (chunks_in_z + shard_planes - 1) / shard_planes; // 3 shards - -const int level_of_detail = 4; } // namespace void @@ -80,12 +80,12 @@ main() int retval = 1; - const ZarrDataType dtype = ZarrDataType_int32; + constexpr ZarrDataType dtype = ZarrDataType_int32; const unsigned int nbytes_px = zarr::bytes_of_type(dtype); try { auto thread_pool = std::make_shared( - std::thread::hardware_concurrency(), + 1, [](const std::string& err) { LOG_ERROR("Error: ", err.c_str()); }); std::vector dims; @@ -102,25 +102,21 @@ main() dims.emplace_back( "x", ZarrDimensionType_Space, array_width, chunk_width, shard_width); - auto config = std::make_shared( - base_dir.string(), - "", - std::nullopt, - std::nullopt, - std::make_shared(std::move(dims), dtype), - dtype, - std::nullopt, - 4); - { - auto writer = std::make_unique( - config, - thread_pool, - std::make_shared(), - nullptr); + auto config = std::make_shared( + base_dir.string(), + "", + std::nullopt, + std::nullopt, + std::make_shared(std::move(dims), dtype), + dtype, + std::nullopt, + 4); + auto writer = std::make_unique( + config, thread_pool, std::make_shared()); const size_t frame_size = array_width * array_height * nbytes_px; - zarr::LockedBuffer data(std::move(ByteVector(frame_size, 0))); + std::vector data(frame_size, 0); for (auto i = 0; i < n_frames; ++i) { // 2 time points CHECK(writer->write_frame(data)); @@ -133,10 +129,10 @@ main() const auto chunk_size = chunk_width * chunk_height * chunk_planes * nbytes_px; - const auto index_size = chunks_per_shard * - sizeof(uint64_t) * // indices are 64 bits - 2; // 2 indices per chunk - const auto checksum_size = 4; // CRC32 checksum + constexpr size_t index_size = chunks_per_shard * + sizeof(uint64_t) * // indices are 64 bits + 2; // 2 indices per chunk + constexpr size_t checksum_size = 4; // CRC32 checksum const auto expected_file_size = shard_width * shard_height * shard_planes * chunk_size + index_size + checksum_size; diff --git a/tests/unit-tests/array-write-ragged-internal-dim.cpp b/tests/unit-tests/array-write-ragged-internal-dim.cpp index 9c9d52f7..8266d310 100644 --- a/tests/unit-tests/array-write-ragged-internal-dim.cpp +++ b/tests/unit-tests/array-write-ragged-internal-dim.cpp @@ -1,10 +1,11 @@ -#include "array.hh" +#include "fs.array.hh" #include "unit.test.macros.hh" #include "zarr.common.hh" #include #include +#include namespace fs = std::filesystem; @@ -98,7 +99,7 @@ main() try { auto thread_pool = std::make_shared( std::thread::hardware_concurrency(), - [](const std::string& err) { LOG_ERROR("Error: ", err.c_str()); }); + [](const std::string& err) { LOG_ERROR("Error: ", err); }); std::vector dims; dims.emplace_back("t", @@ -130,14 +131,13 @@ main() 5); { - auto writer = std::make_unique( + auto writer = std::make_unique( config, thread_pool, - std::make_shared(), - nullptr); + std::make_shared()); const size_t frame_size = array_width * array_height * nbytes_px; - zarr::LockedBuffer data(std::move(ByteVector(frame_size, 0))); + std::vector data(frame_size, 0); for (auto i = 0; i < n_frames; ++i) { // 2 time points CHECK(writer->write_frame(data)); diff --git a/tests/unit-tests/construct-data-paths.cpp b/tests/unit-tests/construct-data-paths.cpp deleted file mode 100644 index d340a557..00000000 --- a/tests/unit-tests/construct-data-paths.cpp +++ /dev/null @@ -1,65 +0,0 @@ -#include "unit.test.macros.hh" -#include "sink.hh" -#include "array.dimensions.hh" - -#include -#include -#include - -namespace { -auto -create_parts_fun(size_t parts) -{ - return [parts](const ZarrDimension&) { return parts; }; -} -} // namespace - -int -main() -{ - int retval = 1; - - try { - std::vector dims{ - { "time", ZarrDimensionType_Time, 50, 16, 2 }, - { "height", ZarrDimensionType_Space, 100, 32, 2 }, - { "width", ZarrDimensionType_Space, 100, 32, 2 } - }; - ArrayDimensions dimensions(std::move(dims), ZarrDataType_uint8); - { - const auto parts_fun = create_parts_fun(2); - const auto paths = - zarr::construct_data_paths("", dimensions, parts_fun); - - EXPECT_EQ(int, paths.size(), 4); - EXPECT_STR_EQ(paths[0].c_str(), "0/0"); - EXPECT_STR_EQ(paths[1].c_str(), "0/1"); - EXPECT_STR_EQ(paths[2].c_str(), "1/0"); - EXPECT_STR_EQ(paths[3].c_str(), "1/1"); - } - - { - const auto parts_fun = create_parts_fun(3); - const auto paths = - zarr::construct_data_paths("", dimensions, parts_fun); - - EXPECT_EQ(int, paths.size(), 9); - EXPECT_STR_EQ(paths[0].c_str(), "0/0"); - EXPECT_STR_EQ(paths[1].c_str(), "0/1"); - EXPECT_STR_EQ(paths[2].c_str(), "0/2"); - EXPECT_STR_EQ(paths[3].c_str(), "1/0"); - EXPECT_STR_EQ(paths[4].c_str(), "1/1"); - EXPECT_STR_EQ(paths[5].c_str(), "1/2"); - EXPECT_STR_EQ(paths[6].c_str(), "2/0"); - EXPECT_STR_EQ(paths[7].c_str(), "2/1"); - EXPECT_STR_EQ(paths[8].c_str(), "2/2"); - } - - retval = 0; - } catch (const std::exception& e) { - LOG_ERROR("Test failed: ", e.what()); - throw; - } - - return retval; -} diff --git a/tests/unit-tests/downsampler-odd-z.cpp b/tests/unit-tests/downsampler-odd-z.cpp index 588bed79..7a28cf9c 100644 --- a/tests/unit-tests/downsampler-odd-z.cpp +++ b/tests/unit-tests/downsampler-odd-z.cpp @@ -3,17 +3,17 @@ namespace { template -zarr::LockedBuffer +std::vector create_test_image(size_t width, size_t height, T value = 100) { - ByteVector data(width * height * sizeof(T), 0); + std::vector data(width * height * sizeof(T), 0); auto* typed_data = reinterpret_cast(data.data()); for (size_t i = 0; i < width * height; ++i) { typed_data[i] = value; } - return { std::move(data) }; + return data; } void @@ -25,14 +25,14 @@ check_downsample(zarr::Downsampler& downsampler, uint8_t frame_value) for (auto i = 0; i < 15; ++i) { downsampler.add_frame(first_timepoint); if (i % 2 == 1) { - zarr::LockedBuffer downsampled; + std::vector downsampled; EXPECT(downsampler.take_frame(1, downsampled), "Downsampled frame not found"); ++n_downsampled; - downsampled.with_lock([frame_value](const ByteVector& data) { - for (auto j = 0; j < data.size(); ++j) { - auto value = data[j]; + { + for (auto j = 0; j < downsampled.size(); ++j) { + auto value = downsampled[j]; EXPECT(value == frame_value, "Downsampled value mismatch at timepoint ", j, @@ -41,20 +41,20 @@ check_downsample(zarr::Downsampler& downsampler, uint8_t frame_value) ", got ", value); } - }); + } } } EXPECT( n_downsampled == 7, "Expected 7 downsampled frames, got ", n_downsampled); - zarr::LockedBuffer downsampled; + std::vector downsampled; EXPECT(downsampler.take_frame(1, downsampled), "Downsampled frame not found after all frames added"); - downsampled.with_lock([frame_value](const ByteVector& data) { - for (auto j = 0; j < data.size(); ++j) { - auto value = data[j]; + { + for (auto j = 0; j < downsampled.size(); ++j) { + auto value = downsampled[j]; EXPECT(value == frame_value, "Downsampled value mismatch at timepoint ", j, @@ -63,7 +63,7 @@ check_downsample(zarr::Downsampler& downsampler, uint8_t frame_value) ", got ", value); } - }); + } } } // namespace @@ -80,14 +80,15 @@ main() { "x", ZarrDimensionType_Space, 64, 16, 1 }, }, ZarrDataType_uint8); - auto config = std::make_shared("", - "/0", - std::nullopt, - std::nullopt, - dims, - ZarrDataType_uint8, - ZarrDownsamplingMethod_Mean, - 0); + auto config = + std::make_shared("", + "/0", + std::nullopt, + std::nullopt, + dims, + ZarrDataType_uint8, + ZarrDownsamplingMethod_Mean, + 0); try { zarr::Downsampler downsampler(config, ZarrDownsamplingMethod_Mean); diff --git a/tests/unit-tests/downsampler.cpp b/tests/unit-tests/downsampler.cpp index caf0c3dd..20c47082 100644 --- a/tests/unit-tests/downsampler.cpp +++ b/tests/unit-tests/downsampler.cpp @@ -9,10 +9,10 @@ namespace { // Helper to create simple test images template -zarr::LockedBuffer +std::vector create_test_image(size_t width, size_t height, T value = 100) { - ByteVector data(width * height * sizeof(T), 0); + std::vector data(width * height * sizeof(T), 0); auto* typed_data = reinterpret_cast(data.data()); for (size_t i = 0; i < width * height; ++i) { @@ -55,7 +55,7 @@ test_basic_downsampling() // Add the frame and check that downsampled version is created downsampler.add_frame(image); - zarr::LockedBuffer downsampled; + std::vector downsampled; bool has_frame = downsampler.take_frame(1, downsampled); EXPECT(has_frame, "Downsampled frame not found"); @@ -64,11 +64,9 @@ test_basic_downsampling() // Verify the downsampled values (should still be 100 since all input pixels // were 100) - downsampled.with_lock([](auto& data) { - for (size_t i = 0; i < 5 * 5; ++i) { - EXPECT_EQ(uint8_t, data[i], 100); - } - }); + for (size_t i = 0; i < 5 * 5; ++i) { + EXPECT_EQ(uint8_t, downsampled[i], 100); + } // Check frame is removed from cache after retrieval has_frame = downsampler.take_frame(1, downsampled); @@ -108,7 +106,7 @@ test_3d_downsampling() // Add first frame - should be stored in partial_scaled_frames_ downsampler.add_frame(image1); - zarr::LockedBuffer downsampled; + std::vector downsampled; bool has_frame = downsampler.take_frame(1, downsampled); EXPECT(!has_frame, "Downsampled frame should not be ready yet in 3D mode"); @@ -120,12 +118,10 @@ test_3d_downsampling() EXPECT(has_frame, "Downsampled frame not found after second frame"); // Verify the values (should be average of 100 and 200 = 150) - downsampled.with_lock([](auto& data) { - auto* typed_downsampled = reinterpret_cast(data.data()); - for (size_t i = 0; i < 10 * 10; ++i) { - EXPECT_EQ(uint16_t, typed_downsampled[i], 150); - } - }); + auto* typed_downsampled = reinterpret_cast(downsampled.data()); + for (size_t i = 0; i < 10 * 10; ++i) { + EXPECT_EQ(uint16_t, typed_downsampled[i], 150); + } // second level shouldn't be ready yet has_frame = downsampler.take_frame(2, downsampled); @@ -149,12 +145,10 @@ test_3d_downsampling() EXPECT(has_frame, "Downsampled frame not found after fourth frame"); // Verify the values (should be average of 100, 200, 300, and 400 = 250) - downsampled.with_lock([](auto& data) { - auto* typed_downsampled = reinterpret_cast(data.data()); - for (size_t i = 0; i < 5 * 5; ++i) { - EXPECT_EQ(uint16_t, typed_downsampled[i], 250); - } - }); + typed_downsampled = reinterpret_cast(downsampled.data()); + for (size_t i = 0; i < 5 * 5; ++i) { + EXPECT_EQ(uint16_t, typed_downsampled[i], 250); + } } void @@ -191,7 +185,7 @@ test_data_types() zarr::Downsampler downsampler(config, ZarrDownsamplingMethod_Mean); // Add a frame based on the type - zarr::LockedBuffer image; + std::vector image; size_t pixel_size = 0; switch (type) { @@ -242,7 +236,7 @@ test_data_types() downsampler.add_frame(image); - zarr::LockedBuffer downsampled; + std::vector downsampled; bool has_frame = downsampler.take_frame(1, downsampled); EXPECT(has_frame, "Downsampled frame not found for type " + @@ -439,10 +433,10 @@ test_edge_cases() zarr::Downsampler downsampler(config, ZarrDownsamplingMethod_Mean); // Create a test image (11x11) - zarr::LockedBuffer image(std::move(ByteVector(11 * 11, 100))); + std::vector image(std::move(std::vector(11 * 11, 100))); downsampler.add_frame(image); - zarr::LockedBuffer downsampled; + std::vector downsampled; bool has_frame = downsampler.take_frame(1, downsampled); EXPECT(has_frame, "Downsampled frame not found for odd dimensions"); @@ -472,10 +466,10 @@ test_min_max_downsampling() // Create a test image with a pattern that will show different results for // min/max/mean - zarr::LockedBuffer image( - std::move(ByteVector(10 * 10 * sizeof(uint8_t), 0))); - image.with_lock([](auto& data) { - auto* typed_data = reinterpret_cast(data.data()); + std::vector image( + std::move(std::vector(10 * 10 * sizeof(uint8_t), 0))); + { + auto* typed_data = reinterpret_cast(image.data()); // Create a pattern where each 2x2 block has values [100, 200, 150, 250] for (size_t y = 0; y < 10; y += 2) { @@ -486,23 +480,23 @@ test_min_max_downsampling() typed_data[(y + 1) * 10 + (x + 1)] = 250; // bottom-right } } - }); + } // Test with mean downsampling { zarr::Downsampler downsampler(config, ZarrDownsamplingMethod_Mean); downsampler.add_frame(image); - zarr::LockedBuffer downsampled; + std::vector downsampled; bool has_frame = downsampler.take_frame(1, downsampled); EXPECT(has_frame, "Mean downsampled frame not found"); - downsampled.with_lock([](auto& data) { + { // For mean, we expect (100 + 200 + 150 + 250) / 4 = 175 for (size_t i = 0; i < 5 * 5; ++i) { - EXPECT_EQ(uint8_t, data[i], 175); + EXPECT_EQ(uint8_t, downsampled[i], 175); } - }); + } } // Test with min downsampling @@ -511,16 +505,16 @@ test_min_max_downsampling() zarr::Downsampler downsampler(config, ZarrDownsamplingMethod_Min); downsampler.add_frame(image); - zarr::LockedBuffer downsampled; + std::vector downsampled; bool has_frame = downsampler.take_frame(1, downsampled); EXPECT(has_frame, "Min downsampled frame not found"); - downsampled.with_lock([](auto& data) { + { // For min, we expect min(100, 200, 150, 250) = 100 for (size_t i = 0; i < 5 * 5; ++i) { - EXPECT_EQ(uint8_t, data[i], 100); + EXPECT_EQ(uint8_t, downsampled[i], 100); } - }); + } } // Test with max downsampling @@ -529,16 +523,16 @@ test_min_max_downsampling() zarr::Downsampler downsampler(config, ZarrDownsamplingMethod_Max); downsampler.add_frame(image); - zarr::LockedBuffer downsampled; + std::vector downsampled; bool has_frame = downsampler.take_frame(1, downsampled); EXPECT(has_frame, "Max downsampled frame not found"); - downsampled.with_lock([](auto& data) { + { // For max, we expect max(100, 200, 150, 250) = 250 for (size_t i = 0; i < 5 * 5; ++i) { - EXPECT_EQ(uint8_t, data[i], 250); + EXPECT_EQ(uint8_t, downsampled[i], 250); } - }); + } } } @@ -575,17 +569,18 @@ test_3d_min_max_downsampling() downsampler.add_frame(image1); downsampler.add_frame(image2); - zarr::LockedBuffer downsampled; + std::vector downsampled; bool has_frame = downsampler.take_frame(1, downsampled); EXPECT(has_frame, "Min downsampled frame not found after second frame"); - downsampled.with_lock([](auto& data) { + { // Verify the values (should be min of 100 and 200 = 100) - auto* typed_downsampled = reinterpret_cast(data.data()); + auto* typed_downsampled = + reinterpret_cast(downsampled.data()); for (size_t i = 0; i < 10 * 10; ++i) { EXPECT_EQ(uint16_t, typed_downsampled[i], 100); } - }); + } } // Test with max downsampling @@ -600,17 +595,18 @@ test_3d_min_max_downsampling() downsampler.add_frame(image1); downsampler.add_frame(image2); - zarr::LockedBuffer downsampled; + std::vector downsampled; bool has_frame = downsampler.take_frame(1, downsampled); EXPECT(has_frame, "Max downsampled frame not found after second frame"); - downsampled.with_lock([](auto& data) { + { // Verify the values (should be max of 100 and 200 = 200) - auto* typed_downsampled = reinterpret_cast(data.data()); + auto* typed_downsampled = + reinterpret_cast(downsampled.data()); for (size_t i = 0; i < 10 * 10; ++i) { EXPECT_EQ(uint16_t, typed_downsampled[i], 200); } - }); + } } // Test multi-level downsampling with max @@ -627,17 +623,18 @@ test_3d_min_max_downsampling() downsampler.add_frame(image3); downsampler.add_frame(image4); - zarr::LockedBuffer downsampled; + std::vector downsampled; bool has_frame = downsampler.take_frame(2, downsampled); EXPECT(has_frame, "Level 2 max downsampled frame not found"); - downsampled.with_lock([](auto& data) { + { // Verify the values (should be max of all values = 400) - auto* typed_downsampled = reinterpret_cast(data.data()); + auto* typed_downsampled = + reinterpret_cast(downsampled.data()); for (size_t i = 0; i < 5 * 5; ++i) { EXPECT_EQ(uint16_t, typed_downsampled[i], 400); } - }); + } } } @@ -662,14 +659,14 @@ test_pattern_downsampling() 0); // Create a test image with a gradient pattern - zarr::LockedBuffer image( - std::move(ByteVector(8 * 8 * sizeof(uint16_t), 0))); + std::vector image( + std::move(std::vector(8 * 8 * sizeof(uint16_t), 0))); std::vector expected_mean(4 * 4); std::vector expected_min(4 * 4); std::vector expected_max(4 * 4); - image.with_lock([&](auto& data) { - auto* typed_data = reinterpret_cast(data.data()); + { + auto* typed_data = reinterpret_cast(image.data()); // Values increase from left to right and top to bottom for (size_t y = 0; y < 8; ++y) { @@ -698,23 +695,24 @@ test_pattern_downsampling() std::max(std::max(v1, v2), std::max(v3, v4)); } } - }); + } // Test with mean downsampling { zarr::Downsampler downsampler(config, ZarrDownsamplingMethod_Mean); downsampler.add_frame(image); - zarr::LockedBuffer downsampled; + std::vector downsampled; bool has_frame = downsampler.take_frame(1, downsampled); EXPECT(has_frame, "Mean downsampled frame not found"); - downsampled.with_lock([&expected_mean](auto& data) { - auto* typed_downsampled = reinterpret_cast(data.data()); + { + auto* typed_downsampled = + reinterpret_cast(downsampled.data()); for (size_t i = 0; i < 4 * 4; ++i) { EXPECT_EQ(uint16_t, typed_downsampled[i], expected_mean[i]); } - }); + } } // Test with min downsampling @@ -723,16 +721,17 @@ test_pattern_downsampling() zarr::Downsampler downsampler(config, ZarrDownsamplingMethod_Min); downsampler.add_frame(image); - zarr::LockedBuffer downsampled; + std::vector downsampled; bool has_frame = downsampler.take_frame(1, downsampled); EXPECT(has_frame, "Min downsampled frame not found"); - downsampled.with_lock([&expected_min](auto& data) { - auto* typed_downsampled = reinterpret_cast(data.data()); + { + auto* typed_downsampled = + reinterpret_cast(downsampled.data()); for (size_t i = 0; i < 4 * 4; ++i) { EXPECT_EQ(uint16_t, typed_downsampled[i], expected_min[i]); } - }); + } } // Test with max downsampling @@ -741,16 +740,17 @@ test_pattern_downsampling() zarr::Downsampler downsampler(config, ZarrDownsamplingMethod_Max); downsampler.add_frame(image); - zarr::LockedBuffer downsampled; + std::vector downsampled; bool has_frame = downsampler.take_frame(1, downsampled); EXPECT(has_frame, "Min downsampled frame not found"); - downsampled.with_lock([&expected_max](auto& data) { - auto* typed_downsampled = reinterpret_cast(data.data()); + { + auto* typed_downsampled = + reinterpret_cast(downsampled.data()); for (size_t i = 0; i < 4 * 4; ++i) { EXPECT_EQ(uint16_t, typed_downsampled[i], expected_max[i]); } - }); + } } } } // namespace zarr::test diff --git a/tests/unit-tests/file-sink-write.cpp b/tests/unit-tests/file-sink-write.cpp deleted file mode 100644 index a06d75ca..00000000 --- a/tests/unit-tests/file-sink-write.cpp +++ /dev/null @@ -1,54 +0,0 @@ -#include "file.sink.hh" -#include "unit.test.macros.hh" - -#include -#include -#include - -namespace fs = std::filesystem; - -int -main() -{ - int retval = 0; - fs::path tmp_path = fs::temp_directory_path() / TEST; - - try { - CHECK(!fs::exists(tmp_path)); - { - char str[] = "Hello, Acquire!"; - auto sink = std::make_unique( - tmp_path.string(), std::make_shared()); - - std::span data = { reinterpret_cast(str), - sizeof(str) - 1 }; - CHECK(sink->write(0, data)); - CHECK(zarr::finalize_sink(std::move(sink))); - } - - // The file tmp_path should now contain the string "Hello, world!\n". - CHECK(fs::exists(tmp_path)); - - std::ifstream ifs(tmp_path); - CHECK(ifs.is_open()); - - std::string contents; - while (!ifs.eof()) { - std::getline(ifs, contents); - } - ifs.close(); - - EXPECT_STR_EQ(contents.c_str(), "Hello, Acquire!"); - } catch (const std::exception& e) { - LOG_ERROR("Caught exception: ", e.what()); - retval = 1; - } - - std::error_code ec; - if (!fs::remove(tmp_path, ec)) { - LOG_ERROR("Failed to remove file: ", ec.message()); - retval = 1; - } - - return retval; -} \ No newline at end of file diff --git a/tests/unit-tests/frame-queue.cpp b/tests/unit-tests/frame-queue.cpp index e3d1240c..6a40187f 100644 --- a/tests/unit-tests/frame-queue.cpp +++ b/tests/unit-tests/frame-queue.cpp @@ -16,11 +16,11 @@ test_basic_operations() CHECK(queue.empty()); CHECK(!queue.full()); - ByteVector data(1024); + std::vector data(1024); for (size_t i = 0; i < data.size(); ++i) { data[i] = i % 256; } - zarr::LockedBuffer frame(std::move(data)); + std::vector frame(std::move(data)); // Pushing CHECK(queue.push(frame, "foo")); @@ -28,7 +28,7 @@ test_basic_operations() CHECK(!queue.empty()); // Popping - zarr::LockedBuffer received_frame; + std::vector received_frame; std::string received_key; CHECK(queue.pop(received_frame, received_key)); CHECK(received_frame.size() == 1024); @@ -36,11 +36,11 @@ test_basic_operations() CHECK(queue.empty()); // Verify data - received_frame.with_lock([](auto& data) { - for (size_t i = 0; i < data.size(); ++i) { - CHECK(data[i] == i % 256); + { + for (size_t i = 0; i < received_frame.size(); ++i) { + CHECK(received_frame[i] == i % 256); } - }); + } CHECK(received_key == "foo"); } @@ -52,19 +52,19 @@ test_capacity() // Fill the queue for (size_t i = 0; i < capacity; ++i) { - zarr::LockedBuffer frame(std::move(ByteVector(100, i))); + std::vector frame(std::move(std::vector(100, i))); bool result = queue.push(frame, std::to_string(i)); CHECK(result); } // Queue should be full (next push should fail) - zarr::LockedBuffer extra_frame(std::move(ByteVector(100))); + std::vector extra_frame(std::move(std::vector(100))); bool push_result = queue.push(extra_frame, std::to_string(capacity)); CHECK(!push_result); CHECK(queue.size() == capacity); // Remove one item - zarr::LockedBuffer received_frame; + std::vector received_frame; std::string received_key; bool pop_result = queue.pop(received_frame, received_key); CHECK(pop_result); @@ -72,7 +72,7 @@ test_capacity() CHECK(received_key == "0"); // Should be able to push again - zarr::LockedBuffer new_frame(std::move(ByteVector(100, 99))); + std::vector new_frame(std::move(std::vector(100, 99))); push_result = queue.push(new_frame, std::to_string(capacity)); CHECK(push_result); CHECK(queue.size() == capacity); @@ -91,8 +91,8 @@ test_producer_consumer() // Producer thread std::thread producer([&queue, n_frames, frame_size]() { for (size_t i = 0; i < n_frames; ++i) { - zarr::LockedBuffer frame( - std::move(ByteVector(frame_size, i % 256))); + std::vector frame( + std::move(std::vector(frame_size, i % 256))); // Try until successful while (!queue.push(frame, "spam")) { @@ -106,15 +106,13 @@ test_producer_consumer() size_t frames_received = 0; while (frames_received < n_frames) { - zarr::LockedBuffer frame; + std::vector frame; std::string received_key; if (queue.pop(frame, received_key)) { // Verify frame data (first byte should match frame number % // 256) CHECK(frame.size() > 0); - CHECK(frame.with_lock([&frames_received](auto& data) { - return data[0] == (frames_received % 256); - })); + CHECK(frame[0] == frames_received % 256); CHECK(received_key == "spam"); frames_received++; } else { @@ -142,20 +140,21 @@ test_throughput() // Create large frame for testing std::vector large_frame(frame_size, 42); - zarr::LockedBuffer data(std::move(ByteVector(large_frame))); + std::vector data(std::move(std::vector(large_frame))); auto start_time = std::chrono::high_resolution_clock::now(); // Push and pop in a loop const size_t iterations = 100; - zarr::LockedBuffer received_frame; + std::vector received_frame; std::string received_key; for (size_t i = 0; i < iterations; ++i) { CHECK(queue.push(data, std::to_string(i))); CHECK(queue.pop(received_frame, received_key)); CHECK(received_frame.size() == frame_size); CHECK(received_key == std::to_string(i)); - data.assign(ByteVector(frame_size, 42)); // Reuse the buffer + data.resize(frame_size); + std::ranges::fill(data, 42); // reset data for next push } auto end_time = std::chrono::high_resolution_clock::now(); diff --git a/tests/unit-tests/make-data-sinks.cpp b/tests/unit-tests/make-data-sinks.cpp deleted file mode 100644 index e674ff2c..00000000 --- a/tests/unit-tests/make-data-sinks.cpp +++ /dev/null @@ -1,292 +0,0 @@ -#include "sink.hh" -#include "s3.connection.hh" -#include "zarr.common.hh" -#include "acquire.zarr.h" -#include "unit.test.macros.hh" - -#include -#include - -namespace fs = std::filesystem; - -namespace { -const std::string test_dir = TEST "-data"; - -bool -get_settings(zarr::S3Settings& settings) -{ - char* env = nullptr; - if (!(env = std::getenv("ZARR_S3_ENDPOINT"))) { - LOG_ERROR("ZARR_S3_ENDPOINT not set."); - return false; - } - settings.endpoint = env; - - if (!(env = std::getenv("ZARR_S3_BUCKET_NAME"))) { - LOG_ERROR("ZARR_S3_BUCKET_NAME not set."); - return false; - } - settings.bucket_name = env; - - env = std::getenv("ZARR_S3_REGION"); - if (env) { - settings.region = env; - } - - return true; -} -} // namespace - -void -make_chunk_file_sinks(std::shared_ptr thread_pool, - const ArrayDimensions& dimensions) -{ - // create the sinks, then let them go out of scope to close the handles - { - std::vector> sinks; - CHECK( - zarr::make_data_file_sinks(test_dir, - dimensions, - zarr::chunks_along_dimension, - thread_pool, - std::make_shared(), - sinks)); - - std::vector data(2, 0); - for (auto& sink : sinks) { - CHECK(sink); - // we need to write some data to the sink to ensure it is created - CHECK(sink->write(0, data)); - CHECK(zarr::finalize_sink(std::move(sink))); - } - } - - const auto chunks_in_y = - zarr::chunks_along_dimension(dimensions.height_dim()); - const auto chunks_in_x = - zarr::chunks_along_dimension(dimensions.width_dim()); - - const fs::path base_path(test_dir); - for (auto i = 0; i < chunks_in_y; ++i) { - const fs::path y_dir = base_path / std::to_string(i); - - for (auto j = 0; j < chunks_in_x; ++j) { - const fs::path x_file = y_dir / std::to_string(j); - CHECK(fs::is_regular_file(x_file)); - - // cleanup - fs::remove(x_file); - } - CHECK(!fs::is_regular_file(y_dir / std::to_string(chunks_in_x))); - fs::remove(y_dir); - } - CHECK(!fs::is_directory(base_path / std::to_string(chunks_in_y))); -} - -void -make_chunk_s3_sinks(std::shared_ptr thread_pool, - std::shared_ptr connection_pool, - const std::string& bucket_name, - const ArrayDimensions& dimensions) -{ - // create the sinks, then let them go out of scope to close the handles - { - char data_[] = { 0, 0 }; - std::span data(reinterpret_cast(data_), sizeof(data_)); - std::vector> sinks; - CHECK(make_data_s3_sinks(bucket_name, - test_dir, - dimensions, - zarr::chunks_along_dimension, - connection_pool, - sinks)); - - for (auto& sink : sinks) { - CHECK(sink); - // we need to write some data to the sink to ensure it is created - CHECK(sink->write(0, data)); - CHECK(zarr::finalize_sink(std::move(sink))); - } - } - - const auto chunks_in_y = - zarr::chunks_along_dimension(dimensions.height_dim()); - const auto chunks_in_x = - zarr::chunks_along_dimension(dimensions.width_dim()); - - auto conn = connection_pool->get_connection(); - - const std::string base_path(test_dir); - for (auto i = 0; i < chunks_in_y; ++i) { - const std::string y_dir = base_path + "/" + std::to_string(i); - - for (auto j = 0; j < chunks_in_x; ++j) { - const std::string x_file = y_dir + "/" + std::to_string(j); - CHECK(conn->object_exists(bucket_name, x_file)); - - // cleanup - CHECK(conn->delete_object(bucket_name, x_file)); - } - CHECK(!conn->object_exists(bucket_name, - y_dir + "/" + std::to_string(chunks_in_x))); - CHECK(conn->delete_object(bucket_name, y_dir)); - } - CHECK(!conn->object_exists(bucket_name, - base_path + "/" + std::to_string(chunks_in_y))); - CHECK(conn->delete_object(bucket_name, base_path)); -} - -void -make_shard_file_sinks(std::shared_ptr thread_pool, - const ArrayDimensions& dimensions) -{ - // create the sinks, then let them go out of scope to close the handles - { - std::vector> sinks; - CHECK(make_data_file_sinks(test_dir, - dimensions, - zarr::shards_along_dimension, - thread_pool, - std::make_shared(), - sinks)); - - std::vector data(2, 0); - for (auto& sink : sinks) { - CHECK(sink); - // we need to write some data to the sink to ensure it is created - CHECK(sink->write(0, data)); - CHECK(zarr::finalize_sink(std::move(sink))); - } - } - - const auto shards_in_y = - zarr::shards_along_dimension(dimensions.height_dim()); - const auto shards_in_x = - zarr::shards_along_dimension(dimensions.width_dim()); - - const fs::path base_path(test_dir); - for (auto i = 0; i < shards_in_y; ++i) { - const fs::path y_dir = base_path / std::to_string(i); - - for (auto j = 0; j < shards_in_x; ++j) { - const fs::path x_file = y_dir / std::to_string(j); - CHECK(fs::is_regular_file(x_file)); - - // cleanup - fs::remove(x_file); - } - CHECK(!fs::is_regular_file(y_dir / std::to_string(shards_in_x))); - fs::remove(y_dir); - } - CHECK(!fs::is_directory(base_path / std::to_string(shards_in_y))); -} - -void -make_shard_s3_sinks(std::shared_ptr thread_pool, - std::shared_ptr connection_pool, - const std::string& bucket_name, - const ArrayDimensions& dimensions) -{ - // create the sinks, then let them go out of scope to close the handles - { - char data_[] = { 0, 0 }; - std::span data(reinterpret_cast(data_), sizeof(data_)); - std::vector> sinks; - CHECK(make_data_s3_sinks(bucket_name, - test_dir, - dimensions, - zarr::shards_along_dimension, - connection_pool, - sinks)); - - for (auto& sink : sinks) { - CHECK(sink); - // we need to write some data to the sink to ensure it is created - CHECK(sink->write(0, data)); - CHECK(zarr::finalize_sink(std::move(sink))); - } - } - - const auto shards_in_y = - zarr::shards_along_dimension(dimensions.height_dim()); - const auto shards_in_x = - zarr::shards_along_dimension(dimensions.width_dim()); - - auto conn = connection_pool->get_connection(); - - const std::string base_path(test_dir); - for (auto i = 0; i < shards_in_y; ++i) { - const std::string y_dir = base_path + "/" + std::to_string(i); - - for (auto j = 0; j < shards_in_x; ++j) { - const std::string x_file = y_dir + "/" + std::to_string(j); - CHECK(conn->object_exists(bucket_name, x_file)); - - // cleanup - CHECK(conn->delete_object(bucket_name, x_file)); - } - CHECK(!conn->object_exists(bucket_name, - y_dir + "/" + std::to_string(shards_in_x))); - CHECK(conn->delete_object(bucket_name, y_dir)); - } - CHECK(!conn->object_exists(bucket_name, - base_path + "/" + std::to_string(shards_in_y))); - CHECK(conn->delete_object(bucket_name, base_path)); -} - -int -main() -{ - Logger::set_log_level(LogLevel_Debug); - - std::vector dims; - dims.emplace_back("z", - ZarrDimensionType_Space, - 0, - 3, // 3 planes per chunk - 1); // 1 chunk per shard (3 planes per shard) - dims.emplace_back("y", - ZarrDimensionType_Space, - 4, - 2, // 2 rows per chunk, 2 chunks - 2); // 2 chunks per shard (4 rows per shard, 1 shard) - dims.emplace_back("x", - ZarrDimensionType_Space, - 12, - 3, // 3 columns per chunk, 4 chunks - 2); // 2 chunks per shard (6 columns per shard, 2 shards) - ArrayDimensions dimensions(std::move(dims), ZarrDataType_int8); - - auto thread_pool = std::make_shared( - std::thread::hardware_concurrency(), - [](const std::string& err) { LOG_ERROR("Failed: ", err.c_str()); }); - - try { - make_chunk_file_sinks(thread_pool, dimensions); - make_shard_file_sinks(thread_pool, dimensions); - } catch (const std::exception& e) { - LOG_ERROR("Failed: ", e.what()); - return 1; - } - - zarr::S3Settings settings; - if (!get_settings(settings)) { - LOG_WARNING("Failed to get credentials. Skipping S3 portion of test."); - return 0; - } - - auto connection_pool = - std::make_shared(4, settings); - - try { - make_chunk_s3_sinks( - thread_pool, connection_pool, settings.bucket_name, dimensions); - make_shard_s3_sinks( - thread_pool, connection_pool, settings.bucket_name, dimensions); - } catch (const std::exception& e) { - LOG_ERROR("Failed: ", e.what()); - return 1; - } - - return 0; -} \ No newline at end of file diff --git a/tests/unit-tests/s3-sink-write-multipart.cpp b/tests/unit-tests/s3-sink-write-multipart.cpp index d5da394f..7c0e254b 100644 --- a/tests/unit-tests/s3-sink-write-multipart.cpp +++ b/tests/unit-tests/s3-sink-write-multipart.cpp @@ -1,4 +1,4 @@ -#include "s3.sink.hh" +#include "s3.object.hh" #include "unit.test.macros.hh" #include @@ -57,9 +57,9 @@ main() std::vector data((5 << 20) + 1, 0); { auto sink = - std::make_unique(settings.bucket_name, object_name, pool); - CHECK(sink->write(0, data)); - CHECK(zarr::finalize_sink(std::move(sink))); + std::make_unique(settings.bucket_name, object_name, pool); + CHECK(sink->write(data, 0)); + CHECK(sink->close()); } conn = pool->get_connection(); diff --git a/tests/unit-tests/s3-sink-write.cpp b/tests/unit-tests/s3-sink-write.cpp index acac0cdd..9a7f3b89 100644 --- a/tests/unit-tests/s3-sink-write.cpp +++ b/tests/unit-tests/s3-sink-write.cpp @@ -1,4 +1,4 @@ -#include "s3.sink.hh" +#include "s3.object.hh" #include "unit.test.macros.hh" #include @@ -55,12 +55,12 @@ main() { char str[] = "Hello, Acquire!"; - auto sink = std::make_unique( + auto sink = std::make_unique( settings.bucket_name, object_name, pool); std::span data{ reinterpret_cast(str), sizeof(str) - 1 }; - CHECK(sink->write(0, data)); - CHECK(zarr::finalize_sink(std::move(sink))); + CHECK(sink->write(data, 0)); + CHECK(sink->close()); } conn = pool->get_connection(); diff --git a/tests/unit-tests/zarr-stream-partial-append.cpp b/tests/unit-tests/zarr-stream-partial-append.cpp index b89a84c2..8b233d1b 100644 --- a/tests/unit-tests/zarr-stream-partial-append.cpp +++ b/tests/unit-tests/zarr-stream-partial-append.cpp @@ -48,6 +48,9 @@ verify_file_data(const ZarrStreamSettings& settings) const size_t row_size = settings.arrays->dimensions[2].array_size_px, num_rows = settings.arrays->dimensions[1].array_size_px; + constexpr size_t table_size = 2 * sizeof(uint64_t) + 4; + const size_t chunk_size = row_size * num_rows; + fs::path shard_path = fs::path(settings.store_path) / "0" / "c" / "0" / "0" / "0"; CHECK(fs::is_regular_file(shard_path)); @@ -60,22 +63,31 @@ verify_file_data(const ZarrStreamSettings& settings) // Get file size file.seekg(0, std::ios::end); const auto file_size = file.tellg(); - file.seekg(0, std::ios::beg); + EXPECT(file_size == + static_cast(chunk_size + table_size), + "Unexpected file size: ", + file_size); + file.seekg(table_size, std::ios::beg); // skip table header // Read entire file into buffer - buffer.resize(file_size); - file.read(reinterpret_cast(buffer.data()), file_size); + buffer.resize(chunk_size); + file.read(reinterpret_cast(buffer.data()), chunk_size); CHECK(file.good()); } // Verify each row contains the correct values - constexpr size_t table_size = 2 * sizeof(uint64_t) + 4; - EXPECT_EQ(int, buffer.size(), row_size* num_rows + table_size); + EXPECT_EQ(int, buffer.size(), chunk_size); for (size_t row = 0; row < num_rows; ++row) { // Check each byte in this row for (size_t col = 0; col < row_size; ++col) { const size_t index = row * row_size + col; - EXPECT_EQ(int, buffer[index], row); + EXPECT(buffer[index] == row, + "Unexpected value at row ", + row, + " col ", + col, + ": ", + static_cast(buffer[index])); } } @@ -90,16 +102,20 @@ verify_file_data(const ZarrStreamSettings& settings) // Get file size file.seekg(0, std::ios::end); const auto file_size = file.tellg(); - file.seekg(0, std::ios::beg); + EXPECT(file_size == + static_cast(chunk_size + table_size), + "Unexpected file size: ", + file_size); + file.seekg(table_size, std::ios::beg); // skip table header // Read entire file into buffer - buffer.resize(file_size); - file.read(reinterpret_cast(buffer.data()), file_size); + buffer.resize(chunk_size); + file.read(reinterpret_cast(buffer.data()), chunk_size); CHECK(file.good()); } // Verify each row contains the correct values - EXPECT_EQ(int, buffer.size(), row_size* num_rows + table_size); + EXPECT_EQ(int, buffer.size(), chunk_size); for (size_t row = 0; row < num_rows; ++row) { // Check each byte in this row for (size_t col = 0; col < row_size; ++col) { @@ -124,18 +140,22 @@ verify_file_data(const ZarrStreamSettings& settings) // Get file size file.seekg(0, std::ios::end); const auto file_size = file.tellg(); - file.seekg(0, std::ios::beg); + EXPECT(file_size == + static_cast(chunk_size + table_size), + "Unexpected file size: ", + file_size); + file.seekg(table_size, std::ios::beg); // skip table header // Read entire file into buffer - buffer.resize(file_size); - file.read(reinterpret_cast(buffer.data()), file_size); + buffer.resize(chunk_size); + file.read(reinterpret_cast(buffer.data()), chunk_size); CHECK(file.good()); } // Verify each row contains the correct values - EXPECT_EQ(int, buffer.size(), row_size* num_rows + table_size); + EXPECT_EQ(int, buffer.size(), chunk_size); - for (auto i = 0; i < row_size * num_rows; ++i) { + for (auto i = 0; i < chunk_size; ++i) { EXPECT_EQ(int, buffer[i], px_value++); } @@ -150,18 +170,22 @@ verify_file_data(const ZarrStreamSettings& settings) // Get file size file.seekg(0, std::ios::end); const auto file_size = file.tellg(); - file.seekg(0, std::ios::beg); + EXPECT(file_size == + static_cast(chunk_size + table_size), + "Unexpected file size: ", + file_size); + file.seekg(table_size, std::ios::beg); // skip table header // Read entire file into buffer - buffer.resize(file_size); - file.read(reinterpret_cast(buffer.data()), file_size); + buffer.resize(chunk_size); + file.read(reinterpret_cast(buffer.data()), chunk_size); CHECK(file.good()); } // Verify each row contains the correct values - EXPECT_EQ(int, buffer.size(), row_size* num_rows + table_size); + EXPECT_EQ(int, buffer.size(), chunk_size); - for (auto i = 0; i < row_size * num_rows; ++i) { + for (auto i = 0; i < chunk_size; ++i) { EXPECT_EQ(int, buffer[i], px_value++); } }