From 1a4aebe7007233473f03311720d63ff9d6d3a478 Mon Sep 17 00:00:00 2001 From: Shubham Kaushal Date: Wed, 10 Dec 2025 10:00:24 +0000 Subject: [PATCH 01/12] implement multi stream manager for async downloads --- .../cloud/storage/async/object_descriptor.cc | 13 +- .../storage/async/object_descriptor_test.cc | 1 - .../storage/google_cloud_cpp_storage_grpc.bzl | 2 + .../google_cloud_cpp_storage_grpc.cmake | 3 + .../internal/async/multi_stream_manager.cc | 29 + .../internal/async/multi_stream_manager.h | 163 +++++ .../async/multi_stream_manager_test.cc | 235 +++++++ .../internal/async/object_descriptor_impl.cc | 275 ++++---- .../internal/async/object_descriptor_impl.h | 76 ++- .../async/object_descriptor_impl_test.cc | 620 ++++++++++++------ .../storage_client_grpc_unit_tests.bzl | 1 + 11 files changed, 1068 insertions(+), 350 deletions(-) create mode 100644 google/cloud/storage/internal/async/multi_stream_manager.cc create mode 100644 google/cloud/storage/internal/async/multi_stream_manager.h create mode 100644 google/cloud/storage/internal/async/multi_stream_manager_test.cc diff --git a/google/cloud/storage/async/object_descriptor.cc b/google/cloud/storage/async/object_descriptor.cc index dd197e0109419..f17756fe91bcc 100644 --- a/google/cloud/storage/async/object_descriptor.cc +++ b/google/cloud/storage/async/object_descriptor.cc @@ -27,14 +27,11 @@ absl::optional ObjectDescriptor::metadata() const { std::pair ObjectDescriptor::Read(std::int64_t offset, std::int64_t limit) { - // TODO(15340): This change is causing performance regression. We need to - // revisit it after benchmarking our code. - - // std::int64_t max_range = - // impl_->options().get(); - // if (limit > max_range) { - // impl_->MakeSubsequentStream(); - // } + std::int64_t max_range = + impl_->options().get(); + if (limit > max_range) { + impl_->MakeSubsequentStream(); + } auto reader = impl_->Read({offset, limit}); auto token = storage_internal::MakeAsyncToken(reader.get()); return {AsyncReader(std::move(reader)), std::move(token)}; diff --git a/google/cloud/storage/async/object_descriptor_test.cc b/google/cloud/storage/async/object_descriptor_test.cc index f1541fab05363..0851439821e7c 100644 --- a/google/cloud/storage/async/object_descriptor_test.cc +++ b/google/cloud/storage/async/object_descriptor_test.cc @@ -149,7 +149,6 @@ TEST(ObjectDescriptor, ReadLast) { } TEST(ObjectDescriptor, ReadExceedsMaxRange) { - GTEST_SKIP(); auto mock = std::make_shared(); auto constexpr kMaxRange = 1024; EXPECT_CALL(*mock, options) diff --git a/google/cloud/storage/google_cloud_cpp_storage_grpc.bzl b/google/cloud/storage/google_cloud_cpp_storage_grpc.bzl index 7644ce88dea77..4113b1e4cab04 100644 --- a/google/cloud/storage/google_cloud_cpp_storage_grpc.bzl +++ b/google/cloud/storage/google_cloud_cpp_storage_grpc.bzl @@ -43,6 +43,7 @@ google_cloud_cpp_storage_grpc_hdrs = [ "internal/async/default_options.h", "internal/async/handle_redirect_error.h", "internal/async/insert_object.h", + "internal/async/multi_stream_manager.h", "internal/async/object_descriptor_connection_tracing.h", "internal/async/object_descriptor_impl.h", "internal/async/object_descriptor_reader.h", @@ -120,6 +121,7 @@ google_cloud_cpp_storage_grpc_srcs = [ "internal/async/default_options.cc", "internal/async/handle_redirect_error.cc", "internal/async/insert_object.cc", + "internal/async/multi_stream_manager.cc", "internal/async/object_descriptor_connection_tracing.cc", "internal/async/object_descriptor_impl.cc", "internal/async/object_descriptor_reader.cc", diff --git a/google/cloud/storage/google_cloud_cpp_storage_grpc.cmake b/google/cloud/storage/google_cloud_cpp_storage_grpc.cmake index dfac5b7a8ab75..8349ed1889b8f 100644 --- a/google/cloud/storage/google_cloud_cpp_storage_grpc.cmake +++ b/google/cloud/storage/google_cloud_cpp_storage_grpc.cmake @@ -110,6 +110,8 @@ add_library( internal/async/handle_redirect_error.h internal/async/insert_object.cc internal/async/insert_object.h + internal/async/multi_stream_manager.cc + internal/async/multi_stream_manager.h internal/async/object_descriptor_connection_tracing.cc internal/async/object_descriptor_connection_tracing.h internal/async/object_descriptor_impl.cc @@ -442,6 +444,7 @@ set(storage_client_grpc_unit_tests internal/async/default_options_test.cc internal/async/handle_redirect_error_test.cc internal/async/insert_object_test.cc + internal/async/multi_stream_manager_test.cc internal/async/object_descriptor_connection_tracing_test.cc internal/async/object_descriptor_impl_test.cc internal/async/object_descriptor_reader_test.cc diff --git a/google/cloud/storage/internal/async/multi_stream_manager.cc b/google/cloud/storage/internal/async/multi_stream_manager.cc new file mode 100644 index 0000000000000..5666299eb2697 --- /dev/null +++ b/google/cloud/storage/internal/async/multi_stream_manager.cc @@ -0,0 +1,29 @@ +// Copyright 2024 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// https://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include "google/cloud/storage/internal/async/multi_stream_manager.h" +#include "google/cloud/storage/internal/async/object_descriptor_impl.h" + +namespace google { +namespace cloud { +namespace storage_internal { +GOOGLE_CLOUD_CPP_INLINE_NAMESPACE_BEGIN + +// Explicit instantiation for ObjectDescriptorImpl usage. +template class MultiStreamManager; + +GOOGLE_CLOUD_CPP_INLINE_NAMESPACE_END +} // namespace storage_internal +} // namespace cloud +} // namespace google diff --git a/google/cloud/storage/internal/async/multi_stream_manager.h b/google/cloud/storage/internal/async/multi_stream_manager.h new file mode 100644 index 0000000000000..9f5552470e49f --- /dev/null +++ b/google/cloud/storage/internal/async/multi_stream_manager.h @@ -0,0 +1,163 @@ +// Copyright 2024 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// https://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#ifndef GOOGLE_CLOUD_CPP_GOOGLE_CLOUD_STORAGE_INTERNAL_ASYNC_MULTI_STREAM_MANAGER_H +#define GOOGLE_CLOUD_CPP_GOOGLE_CLOUD_STORAGE_INTERNAL_ASYNC_MULTI_STREAM_MANAGER_H + +#include "google/cloud/status.h" +#include "google/cloud/version.h" +#include +#include +#include +#include +#include +#include + +namespace google { +namespace cloud { +namespace storage_internal { +GOOGLE_CLOUD_CPP_INLINE_NAMESPACE_BEGIN + +class StreamBase { + public: + virtual ~StreamBase() = default; + virtual void Cancel() = 0; +}; + +// Manages a collection of streams. +// +// This class implements the "Subsequent Stream" logic where idle streams +// are moved to the back of the queue for reuse. +// +// THREAD SAFETY: +// This class is NOT thread-safe. The owner (ObjectDescriptorImpl) must +// serialize access, typically by holding `mu_` while calling these methods. +template +class MultiStreamManager { + public: + struct Stream { + std::shared_ptr stream; + std::unordered_map> active_ranges; + }; + + using StreamIterator = typename std::list::iterator; + using StreamFactory = std::function()>; + + // Constructor creates the first stream using the factory immediately. + explicit MultiStreamManager(StreamFactory stream_factory) + : stream_factory_(std::move(stream_factory)) { + streams_.push_back(Stream{stream_factory_(), {}}); + } + + // Constructor accepts an already-created initial stream. + // This is required by ObjectDescriptorImpl which receives an OpenStream. + MultiStreamManager(StreamFactory stream_factory, std::shared_ptr initial_stream) + : stream_factory_(std::move(stream_factory)) { + streams_.push_back(Stream{std::move(initial_stream), {}}); + } + + StreamIterator GetLastStream() { + // SAFETY: The caller must ensure the manager is not empty. + // In ObjectDescriptorImpl, we ensure there is always at least one stream, + // but this assertion protects against future refactoring errors. + assert(!streams_.empty()); + return std::prev(streams_.end()); + } + + StreamIterator GetLeastBusyStream() { + // SAFETY: The caller must ensure the manager is not empty. + // In ObjectDescriptorImpl, we ensure there is always at least one stream, + // but this assertion protects against future refactoring errors. + assert(!streams_.empty()); + auto best_it = streams_.begin(); + // Track min_ranges to avoid calling .size() repeatedly if possible, + // though for std::unordered_map .size() is O(1). + std::size_t min_ranges = best_it->active_ranges.size(); + + // Start checking from the second element + for (auto it = std::next(streams_.begin()); it != streams_.end(); ++it) { + // Strict less-than ensures stability (preferring older streams if tied) + if (it->active_ranges.size() < min_ranges) { + best_it = it; + min_ranges = it->active_ranges.size(); + } + } + return best_it; + } + + StreamIterator AddStream(std::shared_ptr stream) { + streams_.push_back(Stream{std::move(stream), {}}); + return std::prev(streams_.end()); + } + + void CancelAll() { + for (auto& s : streams_) { + if (s.stream) s.stream->Cancel(); + } + } + + void RemoveStreamAndNotifyRanges(StreamIterator it, Status const& status) { + auto ranges = std::move(it->active_ranges); + streams_.erase(it); + for (auto const& kv : ranges) { + kv.second->OnFinish(status); + } + } + + void MoveActiveRanges(StreamIterator from, StreamIterator to) { + to->active_ranges = std::move(from->active_ranges); + } + + void CleanupDoneRanges(StreamIterator it) { + auto& active_ranges = it->active_ranges; + for (auto i = active_ranges.begin(); i != active_ranges.end();) { + if (i->second->IsDone()) { + i = active_ranges.erase(i); + } else { + ++i; + } + } + } + + template + bool ReuseIdleStreamToBack(Pred pred) { + for (auto it = streams_.begin(); it != streams_.end(); ++it) { + if (!pred(*it)) continue; + + // If the idle stream is already at the back, we don't + // need to move it. If it's elsewhere, use splice() to move the node. + // splice() is O(1) and, crucially, does not invalidate iterators + // or copy the Stream object. + if (std::next(it) != streams_.end()) { + streams_.splice(streams_.end(), streams_, it); + } + return true; + } + return false; + } + + bool Empty() const { return streams_.empty(); } + std::size_t Size() const { return streams_.size(); } + + private: + std::list streams_; + StreamFactory stream_factory_; +}; + +GOOGLE_CLOUD_CPP_INLINE_NAMESPACE_END +} // namespace storage_internal +} // namespace cloud +} // namespace google + +#endif // GOOGLE_CLOUD_CPP_GOOGLE_CLOUD_STORAGE_INTERNAL_ASYNC_MULTI_STREAM_MANAGER_H diff --git a/google/cloud/storage/internal/async/multi_stream_manager_test.cc b/google/cloud/storage/internal/async/multi_stream_manager_test.cc new file mode 100644 index 0000000000000..36488361a4998 --- /dev/null +++ b/google/cloud/storage/internal/async/multi_stream_manager_test.cc @@ -0,0 +1,235 @@ +// Copyright 2024 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// https://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include "google/cloud/storage/internal/async/multi_stream_manager.h" +#include +#include +#include + +namespace google { +namespace cloud { +namespace storage_internal { +GOOGLE_CLOUD_CPP_INLINE_NAMESPACE_BEGIN +namespace { + +struct FakeRange { + bool done = false; + int finished = 0; + bool IsDone() const { return done; } + void OnFinish(Status const&) { ++finished; } +}; + +struct FakeStream : public StreamBase { + void Cancel() override { ++cancelled; } + int cancelled = 0; + bool write_pending = false; +}; + +using Manager = MultiStreamManager; + +struct MultiStreamManagerTest : public ::testing::Test { + static Manager MakeManager() { return Manager([] { return std::make_shared(); }); } +}; + +} // namespace + +TEST(MultiStreamManagerTest, ConstructsWithFactoryAndHasOneStream) { + auto mgr = MultiStreamManagerTest::MakeManager(); + EXPECT_FALSE(mgr.Empty()); + EXPECT_EQ(mgr.Size(), 1u); + auto it = mgr.GetLastStream(); + ASSERT_TRUE(it->stream); +} + +TEST(MultiStreamManagerTest, ConstructsWithInitialStream) { + auto initial = std::make_shared(); + Manager mgr([] { return nullptr; }, initial); + EXPECT_EQ(mgr.Size(), 1u); + auto it = mgr.GetLastStream(); + EXPECT_EQ(it->stream, initial); +} + +TEST(MultiStreamManagerTest, AddStreamAppendsAndGetLastReturnsNew) { + auto mgr = MultiStreamManagerTest::MakeManager(); + auto s1 = std::make_shared(); + auto it1 = mgr.AddStream(s1); + EXPECT_EQ(mgr.Size(), 2u); + EXPECT_EQ(it1->stream.get(), s1.get()); + auto it_last = mgr.GetLastStream(); + EXPECT_EQ(it_last->stream.get(), s1.get()); +} + +TEST(MultiStreamManagerTest, GetLeastBusyPrefersFewestActiveRanges) { + auto mgr = MultiStreamManagerTest::MakeManager(); + + // The manager starts with an initial stream (size 0). + // We must make it "busy" so it doesn't win the comparison against our test streams. + auto it_init = mgr.GetLastStream(); + it_init->active_ranges.emplace(999, std::make_shared()); + it_init->active_ranges.emplace(998, std::make_shared()); + + auto s1 = std::make_shared(); + auto s2 = std::make_shared(); + auto it1 = mgr.AddStream(s1); + auto it2 = mgr.AddStream(s2); + + // s1 has 2 ranges. + it1->active_ranges.emplace(1, std::make_shared()); + it1->active_ranges.emplace(2, std::make_shared()); + + // s2 has 1 range. + it2->active_ranges.emplace(3, std::make_shared()); + + auto it_least = mgr.GetLeastBusyStream(); + + // Expect it2 (1 range) over it1 (2 ranges) and it_init (2 ranges). + EXPECT_EQ(it_least, it2); + EXPECT_EQ(it_least->active_ranges.size(), 1u); +} + +TEST(MultiStreamManagerTest, CleanupDoneRangesRemovesFinished) { + auto mgr = MultiStreamManagerTest::MakeManager(); + auto it = mgr.GetLastStream(); + auto r1 = std::make_shared(); r1->done = false; + auto r2 = std::make_shared(); r2->done = true; + auto r3 = std::make_shared(); r3->done = true; + it->active_ranges.emplace(1, r1); + it->active_ranges.emplace(2, r2); + it->active_ranges.emplace(3, r3); + mgr.CleanupDoneRanges(it); + EXPECT_EQ(it->active_ranges.size(), 1u); + EXPECT_TRUE(it->active_ranges.count(1)); +} + +TEST(MultiStreamManagerTest, RemoveStreamAndNotifyRangesCallsOnFinish) { + auto mgr = MultiStreamManagerTest::MakeManager(); + auto it = mgr.GetLastStream(); + auto r1 = std::make_shared(); + auto r2 = std::make_shared(); + it->active_ranges.emplace(11, r1); + it->active_ranges.emplace(22, r2); + mgr.RemoveStreamAndNotifyRanges(it, Status()); // OK status + EXPECT_EQ(mgr.Size(), 0u); + EXPECT_EQ(r1->finished, 1); + EXPECT_EQ(r2->finished, 1); +} + +TEST(MultiStreamManagerTest, CancelAllInvokesCancel) { + auto mgr = MultiStreamManagerTest::MakeManager(); + auto s1 = std::make_shared(); + auto s2 = std::make_shared(); + mgr.AddStream(s1); + mgr.AddStream(s2); + mgr.CancelAll(); + EXPECT_EQ(s1->cancelled, 1); + EXPECT_EQ(s2->cancelled, 1); +} + +TEST(MultiStreamManagerTest, ReuseIdleStreamToBackMovesElement) { + auto mgr = MultiStreamManagerTest::MakeManager(); + // Capture the factory-created stream pointer (initial element) + auto factory_ptr = mgr.GetLastStream()->stream.get(); + auto s1 = std::make_shared(); + mgr.AddStream(s1); + bool moved = mgr.ReuseIdleStreamToBack( + [](Manager::Stream const& s) { + auto fs = s.stream.get(); + return fs != nullptr && s.active_ranges.empty() && !fs->write_pending; + }); + EXPECT_TRUE(moved); + auto it_last = mgr.GetLastStream(); + // After move, the factory stream should be last + EXPECT_EQ(it_last->stream.get(), factory_ptr); + EXPECT_NE(it_last->stream.get(), s1.get()); +} + +TEST(MultiStreamManagerTest, ReuseIdleStreamAlreadyAtBackReturnsTrueWithoutMove) { + auto mgr = MultiStreamManagerTest::MakeManager(); + // The manager starts with one stream. It is the last stream, and it is idle. + auto initial_last = mgr.GetLastStream(); + bool reused = mgr.ReuseIdleStreamToBack( + [](Manager::Stream const& s) { + return s.active_ranges.empty(); + }); + EXPECT_TRUE(reused); + // Pointer should remain the same (it was already at the back) + EXPECT_EQ(mgr.GetLastStream(), initial_last); +} + +TEST(MultiStreamManagerTest, ReuseIdleStreamDoesNotMoveWhenWritePending) { + auto mgr = MultiStreamManagerTest::MakeManager(); + // Mark factory stream as not reusable + mgr.GetLastStream()->stream->write_pending = true; + auto s1 = std::make_shared(); + s1->write_pending = true; // also mark appended stream as not reusable + mgr.AddStream(s1); + bool moved = mgr.ReuseIdleStreamToBack( + [](Manager::Stream const& s) { + auto fs = s.stream.get(); + return fs != nullptr && s.active_ranges.empty() && !fs->write_pending; + }); + EXPECT_FALSE(moved); + auto it_last = mgr.GetLastStream(); + EXPECT_EQ(it_last->stream.get(), s1.get()); +} + +TEST(MultiStreamManagerTest, MoveActiveRangesTransfersAllEntries) { + auto mgr = MultiStreamManagerTest::MakeManager(); + auto s1 = std::make_shared(); + auto s2 = std::make_shared(); + auto it1 = mgr.AddStream(s1); + auto it2 = mgr.AddStream(s2); + it1->active_ranges.emplace(101, std::make_shared()); + it1->active_ranges.emplace(202, std::make_shared()); + ASSERT_EQ(it1->active_ranges.size(), 2u); + ASSERT_TRUE(it2->active_ranges.empty()); + mgr.MoveActiveRanges(it1, it2); + EXPECT_TRUE(it1->active_ranges.empty()); + EXPECT_EQ(it2->active_ranges.size(), 2u); + EXPECT_TRUE(it2->active_ranges.count(101)); + EXPECT_TRUE(it2->active_ranges.count(202)); +} + +TEST(MultiStreamManagerTest, GetLastStreamReflectsRecentAppendAndReuse) { + auto mgr = MultiStreamManagerTest::MakeManager(); + auto s1 = std::make_shared(); + mgr.AddStream(s1); + EXPECT_EQ(mgr.GetLastStream()->stream.get(), s1.get()); + bool moved = mgr.ReuseIdleStreamToBack( + [](Manager::Stream const& s) { + return s.stream != nullptr && s.active_ranges.empty(); + }); + EXPECT_TRUE(moved); + auto it_last = mgr.GetLastStream(); + EXPECT_NE(it_last->stream.get(), s1.get()); +} + +TEST(MultiStreamManagerTest, EmptyAndSizeTransitions) { + auto mgr = MultiStreamManagerTest::MakeManager(); + EXPECT_FALSE(mgr.Empty()); + EXPECT_EQ(mgr.Size(), 1u); + auto it = mgr.GetLastStream(); + mgr.RemoveStreamAndNotifyRanges(it, Status()); + EXPECT_TRUE(mgr.Empty()); + EXPECT_EQ(mgr.Size(), 0u); + auto s = std::make_shared(); + mgr.AddStream(s); + EXPECT_FALSE(mgr.Empty()); + EXPECT_EQ(mgr.Size(), 1u); +} + +GOOGLE_CLOUD_CPP_INLINE_NAMESPACE_END +} // namespace storage_internal +} // namespace cloud +} // namespace google diff --git a/google/cloud/storage/internal/async/object_descriptor_impl.cc b/google/cloud/storage/internal/async/object_descriptor_impl.cc index cebca0a2bbfc5..d7a50a1e4a46e 100644 --- a/google/cloud/storage/internal/async/object_descriptor_impl.cc +++ b/google/cloud/storage/internal/async/object_descriptor_impl.cc @@ -18,6 +18,7 @@ #include "google/cloud/storage/internal/async/object_descriptor_reader_tracing.h" #include "google/cloud/storage/internal/hash_function.h" #include "google/cloud/storage/internal/hash_function_impl.h" +#include "google/cloud/storage/internal/async/multi_stream_manager.h" #include "google/cloud/grpc_error_delegate.h" #include "google/cloud/internal/opentelemetry.h" #include @@ -37,26 +38,35 @@ ObjectDescriptorImpl::ObjectDescriptorImpl( : resume_policy_prototype_(std::move(resume_policy)), make_stream_(std::move(make_stream)), read_object_spec_(std::move(read_object_spec)), - options_(std::move(options)) { - streams_.push_back( - Stream{std::move(stream), {}, resume_policy_prototype_->clone()}); -} + options_(std::move(options)), + // Initialize manager with the first stream. + stream_manager_(std::make_unique( + []() -> std::shared_ptr { return nullptr; }, + std::make_shared(std::move(stream), + resume_policy_prototype_->clone()))) {} ObjectDescriptorImpl::~ObjectDescriptorImpl() { - for (auto const& stream : streams_) { - stream.stream->Cancel(); - } + Cancel(); } void ObjectDescriptorImpl::Start( google::storage::v2::BidiReadObjectResponse first_response) { - OnRead(std::move(first_response)); + std::unique_lock lk(mu_); + auto it = stream_manager_->GetLastStream(); +// FIX: Unlock and start the Read loop FIRST (Senior's logic) + lk.unlock(); + OnRead(it, std::move(first_response)); + + // FIX: Then acquire lock and queue the background stream SECOND + lk.lock(); + AssurePendingStreamQueued(); } void ObjectDescriptorImpl::Cancel() { - for (auto const& stream : streams_) { - stream.stream->Cancel(); - } + std::unique_lock lk(mu_); + cancelled_ = true; + if (stream_manager_) stream_manager_->CancelAll(); + if (pending_stream_.valid()) pending_stream_.cancel(); } absl::optional ObjectDescriptorImpl::metadata() @@ -65,17 +75,51 @@ absl::optional ObjectDescriptorImpl::metadata() return metadata_; } -void ObjectDescriptorImpl::MakeSubsequentStream() { +void ObjectDescriptorImpl::AssurePendingStreamQueued() { + if (pending_stream_.valid()) return; auto request = google::storage::v2::BidiReadObjectRequest{}; - *request.mutable_read_object_spec() = read_object_spec_; - auto stream_result = make_stream_(std::move(request)).get(); + pending_stream_ = make_stream_(std::move(request)); +} +void ObjectDescriptorImpl::MakeSubsequentStream() { std::unique_lock lk(mu_); - streams_.push_back(Stream{ - std::move(stream_result->stream), {}, resume_policy_prototype_->clone()}); + + // 1. REUSE LOGIC (Using Manager) + if (stream_manager_->ReuseIdleStreamToBack( + [](StreamManager::Stream const& s) { + auto const* rs = s.stream.get(); + return rs && s.active_ranges.empty() && !rs->write_pending; + })) { + return; + } + + // 2. PROACTIVE CREATION LOGIC (Senior's Flow) + AssurePendingStreamQueued(); // Ensure a stream is being created + auto stream_future = std::move(pending_stream_); + lk.unlock(); + + // BLOCKING WAIT (Matches senior's logic) + auto stream_result = stream_future.get(); + + if (!stream_result) { + // Stream creation failed. + // The next call to AssurePendingStreamQueued will retry creation. + return; + } + + lk.lock(); + if (cancelled_) return; + + auto rs = std::make_shared(std::move(stream_result->stream), + resume_policy_prototype_->clone()); + auto new_it = stream_manager_->AddStream(std::move(rs)); + + // Now that we consumed pending_stream_, queue the next one immediately. + AssurePendingStreamQueued(); + lk.unlock(); - OnRead(std::move(stream_result->first_response)); + OnRead(new_it, std::move(stream_result->first_response)); } std::unique_ptr @@ -91,13 +135,27 @@ ObjectDescriptorImpl::Read(ReadParams p) { auto range = std::make_shared(p.start, p.length, hash_function); std::unique_lock lk(mu_); + if (stream_manager_->Empty()) { + lk.unlock(); + range->OnFinish(Status(StatusCode::kFailedPrecondition, + "Cannot read object, all streams failed")); + if (!internal::TracingEnabled(options_)) { + return std::unique_ptr( + std::make_unique(std::move(range))); + } + return MakeTracingObjectDescriptorReader(std::move(range)); + } + + auto it = stream_manager_->GetLeastBusyStream(); + auto const id = ++read_id_generator_; - streams_.back().active_ranges.emplace(id, range); - auto& read_range = *next_request_.add_read_ranges(); + it->active_ranges.emplace(id, range); + + auto& read_range = *it->stream->next_request.add_read_ranges(); read_range.set_read_id(id); read_range.set_read_offset(p.start); read_range.set_read_length(p.length); - Flush(std::move(lk)); + Flush(std::move(lk), it); if (!internal::TracingEnabled(options_)) { return std::unique_ptr( @@ -107,149 +165,146 @@ ObjectDescriptorImpl::Read(ReadParams p) { return MakeTracingObjectDescriptorReader(std::move(range)); } -void ObjectDescriptorImpl::Flush(std::unique_lock lk) { - if (streams_.back().write_pending || next_request_.read_ranges().empty()) { +void ObjectDescriptorImpl::Flush(std::unique_lock lk, StreamIterator it) { + if (it->stream->write_pending || it->stream->next_request.read_ranges().empty()) { return; } - streams_.back().write_pending = true; + it->stream->write_pending = true; google::storage::v2::BidiReadObjectRequest request; - request.Swap(&next_request_); - - // Assign CurrentStream to a temporary variable to prevent - // lifetime extension which can cause the lock to be held until the - // end of the block. - auto current_stream = CurrentStream(std::move(lk)); - current_stream->Write(std::move(request)).then([w = WeakFromThis()](auto f) { - if (auto self = w.lock()) self->OnWrite(f.get()); + request.Swap(&it->stream->next_request); + + auto current_stream = it->stream->stream_; + lk.unlock(); + current_stream->Write(std::move(request)).then([w = WeakFromThis(), it](auto f) { + if (auto self = w.lock()) self->OnWrite(it, f.get()); }); } -void ObjectDescriptorImpl::OnWrite(bool ok) { +void ObjectDescriptorImpl::OnWrite(StreamIterator it, bool ok) { std::unique_lock lk(mu_); - if (!ok) return DoFinish(std::move(lk)); - streams_.back().write_pending = false; - Flush(std::move(lk)); + if (!ok) return DoFinish(std::move(lk), it); + it->stream->write_pending = false; + Flush(std::move(lk), it); } -void ObjectDescriptorImpl::DoRead(std::unique_lock lk) { - // Assign CurrentStream to a temporary variable to prevent - // lifetime extension which can cause the lock to be held until the - // end of the block. - auto current_stream = CurrentStream(std::move(lk)); - current_stream->Read().then([w = WeakFromThis()](auto f) { - if (auto self = w.lock()) self->OnRead(f.get()); +void ObjectDescriptorImpl::DoRead(std::unique_lock lk, StreamIterator it) { + if (it->stream->read_pending) return; + it->stream->read_pending = true; + + auto current_stream = it->stream->stream_; + lk.unlock(); + current_stream->Read().then([w = WeakFromThis(), it](auto f) { + if (auto self = w.lock()) self->OnRead(it, f.get()); }); } -void ObjectDescriptorImpl::OnRead( - absl::optional response) { +void ObjectDescriptorImpl::OnRead(StreamIterator it, absl::optional response) { std::unique_lock lk(mu_); - if (!response) return DoFinish(std::move(lk)); + it->stream->read_pending = false; + + if (!response) return DoFinish(std::move(lk), it); + if (response->has_metadata()) { - metadata_ = std::move(*response->mutable_metadata()); + metadata_ = std::move(*response->mutable_metadata()); } if (response->has_read_handle()) { - *read_object_spec_.mutable_read_handle() = - std::move(*response->mutable_read_handle()); + *read_object_spec_.mutable_read_handle() = std::move(*response->mutable_read_handle()); } - auto copy = CopyActiveRanges(lk); - // Release the lock while notifying the ranges. The notifications may trigger - // application code, and that code may callback on this class. + + auto copy = it->active_ranges; lk.unlock(); - for (auto& range_data : *response->mutable_object_data_ranges()) { - auto id = range_data.read_range().read_id(); + + for (auto& rd : *response->mutable_object_data_ranges()) { + auto id = rd.read_range().read_id(); auto const l = copy.find(id); if (l == copy.end()) continue; - // TODO(#15104) - Consider returning if the range is done, and then - // skipping CleanupDoneRanges(). - l->second->OnRead(std::move(range_data)); + l->second->OnRead(std::move(rd)); } + lk.lock(); - CleanupDoneRanges(lk); - DoRead(std::move(lk)); -} - -void ObjectDescriptorImpl::CleanupDoneRanges( - std::unique_lock const&) { - if (streams_.empty()) return; - auto& active_ranges = streams_.back().active_ranges; - for (auto i = active_ranges.begin(); i != active_ranges.end();) { - if (i->second->IsDone()) { - i = active_ranges.erase(i); - } else { - ++i; - } - } + stream_manager_->CleanupDoneRanges(it); + DoRead(std::move(lk), it); } -void ObjectDescriptorImpl::DoFinish(std::unique_lock lk) { - // Assign CurrentStream to a temporary variable to prevent - // lifetime extension which can cause the lock to be held until the - // end of the block. - auto current_stream = CurrentStream(std::move(lk)); - auto pending = current_stream->Finish(); - if (!pending.valid()) return; - pending.then([w = WeakFromThis()](auto f) { - if (auto self = w.lock()) self->OnFinish(f.get()); +void ObjectDescriptorImpl::DoFinish(std::unique_lock lk, StreamIterator it) { + it->stream->read_pending = false; + auto pending = it->stream->stream_->Finish(); + lk.unlock(); + pending.then([w = WeakFromThis(), it](auto f) { + if (auto self = w.lock()) self->OnFinish(it, f.get()); }); } -void ObjectDescriptorImpl::OnFinish(Status const& status) { +void ObjectDescriptorImpl::OnFinish(StreamIterator it, Status const& status) { auto proto_status = ExtractGrpcStatus(status); + if (IsResumable(it, status, proto_status)) return Resume(it, proto_status); - if (IsResumable(status, proto_status)) return Resume(proto_status); std::unique_lock lk(mu_); - auto copy = CopyActiveRanges(std::move(lk)); - for (auto const& kv : copy) { - kv.second->OnFinish(status); - } + stream_manager_->RemoveStreamAndNotifyRanges(it, status); + + // Since a stream died, we might want to ensure a replacement is queued + AssurePendingStreamQueued(); } -void ObjectDescriptorImpl::Resume(google::rpc::Status const& proto_status) { +void ObjectDescriptorImpl::Resume(StreamIterator it, google::rpc::Status const& proto_status) { std::unique_lock lk(mu_); - // This call needs to happen inside the lock, as it may modify - // `read_object_spec_`. ApplyRedirectErrors(read_object_spec_, proto_status); - auto request = google::storage::v2::BidiReadObjectRequest{}; + + google::storage::v2::BidiReadObjectRequest request; *request.mutable_read_object_spec() = read_object_spec_; - for (auto const& kv : streams_.back().active_ranges) { + + for (auto const& kv : it->active_ranges) { auto range = kv.second->RangeForResume(kv.first); if (!range) continue; *request.add_read_ranges() = *std::move(range); } - streams_.back().write_pending = true; lk.unlock(); - make_stream_(std::move(request)).then([w = WeakFromThis()](auto f) { - if (auto self = w.lock()) self->OnResume(f.get()); + + make_stream_(std::move(request)).then([w = WeakFromThis(), it](auto f) { + if (auto self = w.lock()) self->OnResume(it, f.get()); }); } -void ObjectDescriptorImpl::OnResume(StatusOr result) { - if (!result) return OnFinish(std::move(result).status()); +void ObjectDescriptorImpl::OnResume(StreamIterator it, StatusOr result) { + if (!result) return OnFinish(it, std::move(result).status()); + std::unique_lock lk(mu_); - streams_.push_back( - Stream{std::move(result->stream), {}, resume_policy_prototype_->clone()}); - // TODO(#15105) - this should be done without release the lock. - Flush(std::move(lk)); - OnRead(std::move(result->first_response)); + if (cancelled_) return; + + it->stream = std::make_shared(std::move(result->stream), + resume_policy_prototype_->clone()); + it->stream->write_pending = false; + it->stream->read_pending = false; + + Flush(std::move(lk), it); + OnRead(it, std::move(result->first_response)); } -bool ObjectDescriptorImpl::IsResumable( - Status const& status, google::rpc::Status const& proto_status) { +bool ObjectDescriptorImpl::IsResumable(StreamIterator it, Status const& status, google::rpc::Status const& proto_status) { + std::unique_lock lk(mu_); for (auto const& any : proto_status.details()) { auto error = google::storage::v2::BidiReadObjectError{}; if (!any.UnpackTo(&error)) continue; - for (auto const& range : CopyActiveRanges()) { - for (auto const& range_error : error.read_range_errors()) { - if (range.first != range_error.read_id()) continue; - range.second->OnFinish(MakeStatusFromRpcError(range_error.status())); + + std::vector> notify; + for (auto const& re : error.read_range_errors()) { + if (it->active_ranges.count(re.read_id())) { + notify.emplace_back(re.read_id(), MakeStatusFromRpcError(re.status())); } } - CleanupDoneRanges(std::unique_lock(mu_)); + if (notify.empty()) continue; + + auto copy = it->active_ranges; + lk.unlock(); + for (auto const& p : notify) { + auto l = copy.find(p.first); + if (l != copy.end()) l->second->OnFinish(p.second); + } + lk.lock(); + stream_manager_->CleanupDoneRanges(it); return true; } - std::unique_lock lk(mu_); - return streams_.back().resume_policy->OnFinish(status) == + return it->stream->resume_policy_->OnFinish(status) == storage_experimental::ResumePolicy::kContinue; } diff --git a/google/cloud/storage/internal/async/object_descriptor_impl.h b/google/cloud/storage/internal/async/object_descriptor_impl.h index f4c467de5c42e..5139e224af9ea 100644 --- a/google/cloud/storage/internal/async/object_descriptor_impl.h +++ b/google/cloud/storage/internal/async/object_descriptor_impl.h @@ -20,6 +20,7 @@ #include "google/cloud/storage/internal/async/object_descriptor_reader.h" #include "google/cloud/storage/internal/async/open_stream.h" #include "google/cloud/storage/internal/async/read_range.h" +#include "google/cloud/storage/internal/async/multi_stream_manager.h" #include "google/cloud/storage/options.h" #include "google/cloud/status.h" #include "google/cloud/version.h" @@ -35,16 +36,27 @@ namespace cloud { namespace storage_internal { GOOGLE_CLOUD_CPP_INLINE_NAMESPACE_BEGIN +class ReadStream : public storage_internal::StreamBase { + public: + ReadStream(std::shared_ptr stream, + std::unique_ptr resume_policy) + : stream_(std::move(stream)), resume_policy_(std::move(resume_policy)) {} + + void Cancel() override { if (stream_) stream_->Cancel(); } + + std::shared_ptr stream_; + std::unique_ptr resume_policy_; + google::storage::v2::BidiReadObjectRequest next_request; + bool write_pending = false; + bool read_pending = false; +}; + class ObjectDescriptorImpl : public storage_experimental::ObjectDescriptorConnection, public std::enable_shared_from_this { private: - struct Stream { - std::shared_ptr stream; - std::unordered_map> active_ranges; - std::unique_ptr resume_policy; - bool write_pending = false; - }; + using StreamManager = MultiStreamManager; + using StreamIterator = StreamManager::StreamIterator; public: ObjectDescriptorImpl( @@ -54,19 +66,13 @@ class ObjectDescriptorImpl std::shared_ptr stream, Options options = {}); ~ObjectDescriptorImpl() override; - // Start the read loop. void Start(google::storage::v2::BidiReadObjectResponse first_response); - - // Cancel the underlying RPC and stop the resume loop. void Cancel(); Options options() const override { return options_; } - // Return the object metadata. This is only available after the first `Read()` - // returns. absl::optional metadata() const override; - // Start a new ranged read. std::unique_ptr Read( ReadParams p) override; @@ -77,32 +83,18 @@ class ObjectDescriptorImpl return shared_from_this(); } - // This may seem expensive, but it is less bug-prone than iterating over - // the map with the lock held. - auto CopyActiveRanges(std::unique_lock const&) const { - return streams_.back().active_ranges; - } + // Restored: Logic to ensure a background stream is always connecting + void AssurePendingStreamQueued(); - auto CopyActiveRanges() const { - return CopyActiveRanges(std::unique_lock(mu_)); - } - - auto CurrentStream(std::unique_lock) const { - return streams_.back().stream; - } - - void Flush(std::unique_lock lk); - void OnWrite(bool ok); - void DoRead(std::unique_lock); - void OnRead( - absl::optional response); - void CleanupDoneRanges(std::unique_lock const&); - void DoFinish(std::unique_lock); - void OnFinish(Status const& status); - void Resume(google::rpc::Status const& proto_status); - void OnResume(StatusOr result); - bool IsResumable(Status const& status, - google::rpc::Status const& proto_status); + void Flush(std::unique_lock lk, StreamIterator it); + void OnWrite(StreamIterator it, bool ok); + void DoRead(std::unique_lock lk, StreamIterator it); + void OnRead(StreamIterator it, absl::optional response); + void DoFinish(std::unique_lock lk, StreamIterator it); + void OnFinish(StreamIterator it, Status const& status); + void Resume(StreamIterator it, google::rpc::Status const& proto_status); + void OnResume(StreamIterator it, StatusOr result); + bool IsResumable(StreamIterator it, Status const& status, google::rpc::Status const& proto_status); std::unique_ptr resume_policy_prototype_; OpenStreamFactory make_stream_; @@ -111,10 +103,16 @@ class ObjectDescriptorImpl google::storage::v2::BidiReadObjectSpec read_object_spec_; absl::optional metadata_; std::int64_t read_id_generator_ = 0; - google::storage::v2::BidiReadObjectRequest next_request_; Options options_; - std::vector streams_; + std::unique_ptr stream_manager_; + + // Restored: The future for the proactive background stream + google::cloud::future< + google::cloud::StatusOr> + pending_stream_; + + bool cancelled_ = false; }; GOOGLE_CLOUD_CPP_INLINE_NAMESPACE_END diff --git a/google/cloud/storage/internal/async/object_descriptor_impl_test.cc b/google/cloud/storage/internal/async/object_descriptor_impl_test.cc index 595dd34231b6d..58071abdccd93 100644 --- a/google/cloud/storage/internal/async/object_descriptor_impl_test.cc +++ b/google/cloud/storage/internal/async/object_descriptor_impl_test.cc @@ -27,6 +27,7 @@ #include #include #include +#include // Make sure to add this include for std::thread namespace google { namespace cloud { @@ -46,6 +47,10 @@ using ::testing::NotNull; using ::testing::Optional; using ::testing::ResultOf; using ::testing::VariantWith; +using ::testing::AtMost; +using ::testing::Return; +using ::testing::UnorderedElementsAre; +using ::testing::_; using Request = google::storage::v2::BidiReadObjectRequest; using Response = google::storage::v2::BidiReadObjectResponse; @@ -88,12 +93,12 @@ TEST(ObjectDescriptorImpl, LifecycleNoRead) { return sequencer.PushBack("Finish").then( [](auto) { return PermanentError(); }); }); - EXPECT_CALL(*stream, Cancel).WillOnce([&sequencer]() { - sequencer.PushBack("Cancel"); - }); + EXPECT_CALL(*stream, Cancel).Times(AtMost(1)); MockFactory factory; - EXPECT_CALL(factory, Call).Times(0); + EXPECT_CALL(factory, Call).WillOnce([](Request const&) { + return make_ready_future(StatusOr(PermanentError())); + }); auto tested = std::make_shared( NoResume(), factory.AsStdFunction(), google::storage::v2::BidiReadObjectSpec{}, @@ -117,9 +122,56 @@ TEST(ObjectDescriptorImpl, LifecycleNoRead) { next.first.set_value(true); tested.reset(); - next = sequencer.PopFrontWithName(); - EXPECT_EQ(next.second, "Cancel"); - next.first.set_value(true); +} + +/// @test Verify that Cancel() is called if OnFinish() is delayed. +TEST(ObjectDescriptorImpl, LifecycleCancelRacesWithFinish) { + AsyncSequencer sequencer; + auto stream = std::make_unique(); + EXPECT_CALL(*stream, Read).WillOnce([&sequencer]() { + return sequencer.PushBack("Read[1]").then( + [](auto) { return absl::optional{}; }); + }); + EXPECT_CALL(*stream, Finish).WillOnce([&sequencer]() { + return sequencer.PushBack("Finish").then( + [](auto) { return PermanentError(); }); + }); + // This time we expect Cancel() because we will delay OnFinish(). + EXPECT_CALL(*stream, Cancel).Times(1); + + MockFactory factory; + EXPECT_CALL(factory, Call).WillOnce([](Request const&) { + return make_ready_future(StatusOr(PermanentError())); + }); + auto tested = std::make_shared( + NoResume(), factory.AsStdFunction(), + google::storage::v2::BidiReadObjectSpec{}, + std::make_shared(std::move(stream))); + auto response = Response{}; + EXPECT_TRUE( + TextFormat::ParseFromString(kMetadataText, response.mutable_metadata())); + tested->Start(std::move(response)); + EXPECT_TRUE(tested->metadata().has_value()); + + auto expected_metadata = google::storage::v2::Object{}; + EXPECT_TRUE(TextFormat::ParseFromString(kMetadataText, &expected_metadata)); + EXPECT_THAT(tested->metadata(), Optional(IsProtoEqual(expected_metadata))); + + auto read1 = sequencer.PopFrontWithName(); + EXPECT_EQ(read1.second, "Read[1]"); + read1.first.set_value(true); + + // This pops the future that OnFinish() waits for. + auto finish = sequencer.PopFrontWithName(); + EXPECT_EQ(finish.second, "Finish"); + + // Reset the descriptor *before* OnFinish() gets to run. This invokes the + // destructor, which calls Cancel() on any streams that have not yet been + // removed by OnFinish(). + tested.reset(); + + // Now allow OnFinish() to run. + finish.first.set_value(true); } /// @test Read a single stream and then close. @@ -174,10 +226,12 @@ TEST(ObjectDescriptorImpl, ReadSingleRange) { return sequencer.PushBack("Finish").then( [](auto) { return PermanentError(); }); }); - EXPECT_CALL(*stream, Cancel).Times(1); + EXPECT_CALL(*stream, Cancel).Times(AtMost(1)); MockFactory factory; - EXPECT_CALL(factory, Call).Times(0); + EXPECT_CALL(factory, Call).WillOnce([](Request const&) { + return make_ready_future(StatusOr(PermanentError())); + }); auto tested = std::make_shared( NoResume(), factory.AsStdFunction(), google::storage::v2::BidiReadObjectSpec{}, @@ -225,6 +279,7 @@ TEST(ObjectDescriptorImpl, ReadSingleRange) { next.first.set_value(true); } + /// @test Reading multiple ranges creates a single request. TEST(ObjectDescriptorImpl, ReadMultipleRanges) { auto constexpr kLength = 100; @@ -292,10 +347,12 @@ TEST(ObjectDescriptorImpl, ReadMultipleRanges) { return sequencer.PushBack("Finish").then( [](auto) { return PermanentError(); }); }); - EXPECT_CALL(*stream, Cancel).Times(1); + EXPECT_CALL(*stream, Cancel).Times(AtMost(1)); MockFactory factory; - EXPECT_CALL(factory, Call).Times(0); + EXPECT_CALL(factory, Call).WillOnce([](Request const&) { + return make_ready_future(StatusOr(PermanentError())); + }); auto tested = std::make_shared( NoResume(), factory.AsStdFunction(), google::storage::v2::BidiReadObjectSpec{}, @@ -440,10 +497,12 @@ TEST(ObjectDescriptorImpl, ReadSingleRangeManyMessages) { return sequencer.PushBack("Finish").then( [](auto) { return PermanentError(); }); }); - EXPECT_CALL(*stream, Cancel).Times(1); + EXPECT_CALL(*stream, Cancel).Times(AtMost(1)); MockFactory factory; - EXPECT_CALL(factory, Call).Times(0); + EXPECT_CALL(factory, Call).WillOnce([](Request const&) { + return make_ready_future(StatusOr(PermanentError())); + }); auto tested = std::make_shared( NoResume(), factory.AsStdFunction(), google::storage::v2::BidiReadObjectSpec{}, @@ -554,10 +613,12 @@ TEST(ObjectDescriptorImpl, AllRangesFailOnUnrecoverableError) { return sequencer.PushBack("Finish").then( [](auto) { return PermanentError(); }); }); - EXPECT_CALL(*stream, Cancel).Times(1); + EXPECT_CALL(*stream, Cancel).Times(AtMost(1)); MockFactory factory; - EXPECT_CALL(factory, Call).Times(0); + EXPECT_CALL(factory, Call).WillOnce([](Request const&) { + return make_ready_future(StatusOr(PermanentError())); + }); auto tested = std::make_shared( NoResume(), factory.AsStdFunction(), google::storage::v2::BidiReadObjectSpec{}, @@ -637,7 +698,7 @@ auto InitialStream(AsyncSequencer& sequencer) { )pb"; auto stream = std::make_unique(); - EXPECT_CALL(*stream, Cancel).Times(1); // Always called by OpenStream + EXPECT_CALL(*stream, Cancel).Times(AtMost(1)); // Always called by OpenStream EXPECT_CALL(*stream, Write) .WillOnce([=, &sequencer](Request const& request, grpc::WriteOptions) { auto expected = Request{}; @@ -716,16 +777,24 @@ TEST(ObjectDescriptorImpl, ResumeRangesOnRecoverableError) { )pb"; AsyncSequencer sequencer; + Request expected_resume_request; + ASSERT_TRUE( + TextFormat::ParseFromString(kResumeRequest, &expected_resume_request)); MockFactory factory; - EXPECT_CALL(factory, Call).WillOnce([=, &sequencer](Request const& request) { - auto expected = Request{}; - EXPECT_TRUE(TextFormat::ParseFromString(kResumeRequest, &expected)); - EXPECT_THAT(request, IsProtoEqualModuloRepeatedFieldOrdering(expected)); - // Resume with an unrecoverable failure to simplify the test. - return sequencer.PushBack("Factory").then( - [&](auto) { return StatusOr(PermanentError()); }); - }); + EXPECT_CALL(factory, Call) + .WillOnce([](Request const& request) { + EXPECT_TRUE(request.read_ranges().empty()); + return make_ready_future( + StatusOr(TransientError())); + }) + .WillOnce([&](Request const& request) { + EXPECT_THAT(request, IsProtoEqualModuloRepeatedFieldOrdering( + expected_resume_request)); + return sequencer.PushBack("Factory").then([](auto) { + return StatusOr(PermanentError()); + }); + }); auto spec = google::storage::v2::BidiReadObjectSpec{}; ASSERT_TRUE(TextFormat::ParseFromString(kReadSpecText, &spec)); @@ -852,7 +921,7 @@ TEST(ObjectDescriptorImpl, PendingFinish) { )pb"; auto stream = std::make_unique(); - EXPECT_CALL(*stream, Cancel).Times(1); // Always called by OpenStream + EXPECT_CALL(*stream, Cancel).Times(AtMost(1)); EXPECT_CALL(*stream, Write) .WillOnce([=, &sequencer](Request const& request, grpc::WriteOptions) { auto expected = Request{}; @@ -886,7 +955,9 @@ TEST(ObjectDescriptorImpl, PendingFinish) { )pb"; MockFactory factory; - EXPECT_CALL(factory, Call).Times(0); + EXPECT_CALL(factory, Call).WillOnce([](Request const&) { + return make_ready_future(StatusOr(PermanentError())); + }); auto spec = google::storage::v2::BidiReadObjectSpec{}; ASSERT_TRUE(TextFormat::ParseFromString(kReadSpecText, &spec)); @@ -939,7 +1010,7 @@ TEST(ObjectDescriptorImpl, ResumeUsesRouting) { )pb"; auto stream = std::make_unique(); - EXPECT_CALL(*stream, Cancel).Times(1); // Always called by OpenStream + EXPECT_CALL(*stream, Cancel).Times(AtMost(1)); EXPECT_CALL(*stream, Write) .WillOnce([=, &sequencer](Request const& request, grpc::WriteOptions) { auto expected = Request{}; @@ -986,15 +1057,25 @@ TEST(ObjectDescriptorImpl, ResumeUsesRouting) { read_ranges { read_id: 1 read_offset: 20000 read_length: 100 } )pb"; + Request expected_resume_request; + ASSERT_TRUE( + TextFormat::ParseFromString(kResumeRequest, &expected_resume_request)); + MockFactory factory; - EXPECT_CALL(factory, Call).WillOnce([=, &sequencer](Request const& request) { - auto expected = Request{}; - EXPECT_TRUE(TextFormat::ParseFromString(kResumeRequest, &expected)); - EXPECT_THAT(request, IsProtoEqualModuloRepeatedFieldOrdering(expected)); - // Resume with an unrecoverable failure to simplify the test. - return sequencer.PushBack("Factory").then( - [&](auto) { return StatusOr(PermanentError()); }); - }); + EXPECT_CALL(factory, Call) + .WillOnce([](Request const& request) { + EXPECT_TRUE(request.read_ranges().empty()); + return make_ready_future( + StatusOr(TransientError())); + }) + .WillOnce([&](Request const& request) { + EXPECT_THAT(request, IsProtoEqualModuloRepeatedFieldOrdering( + expected_resume_request)); + return sequencer.PushBack("Factory").then([](auto) { + return StatusOr(PermanentError()); + }); + }); + auto spec = google::storage::v2::BidiReadObjectSpec{}; ASSERT_TRUE(TextFormat::ParseFromString(kReadSpecText, &spec)); @@ -1117,17 +1198,25 @@ TEST(ObjectDescriptorImpl, RecoverFromPartialFailure) { return PartialFailure(2); }); }); - EXPECT_CALL(*stream, Cancel).Times(1); + EXPECT_CALL(*stream, Cancel).Times(AtMost(1)); + Request expected_resume_request; + ASSERT_TRUE( + TextFormat::ParseFromString(kResumeRequest, &expected_resume_request)); MockFactory factory; - EXPECT_CALL(factory, Call).WillOnce([=, &sequencer](Request const& request) { - auto expected = Request{}; - EXPECT_TRUE(TextFormat::ParseFromString(kResumeRequest, &expected)); - EXPECT_THAT(request, IsProtoEqualModuloRepeatedFieldOrdering(expected)); - // Resume with an unrecoverable failure to simplify the test. - return sequencer.PushBack("Factory").then( - [&](auto) { return StatusOr(PermanentError()); }); - }); + EXPECT_CALL(factory, Call) + .WillOnce([](Request const& request) { + EXPECT_TRUE(request.read_ranges().empty()); + return make_ready_future( + StatusOr(TransientError())); + }) + .WillOnce([&](Request const& request) { + EXPECT_THAT(request, IsProtoEqualModuloRepeatedFieldOrdering( + expected_resume_request)); + return sequencer.PushBack("Factory").then([](auto) { + return StatusOr(PermanentError()); + }); + }); auto spec = google::storage::v2::BidiReadObjectSpec{}; EXPECT_TRUE(TextFormat::ParseFromString(kReadSpecText, &spec)); @@ -1190,188 +1279,335 @@ TEST(ObjectDescriptorImpl, RecoverFromPartialFailure) { EXPECT_THAT(s3r1.get(), VariantWith(PermanentError())); } -/// @test Verify that we can create a subsequent stream and read from it. -TEST(ObjectDescriptorImpl, ReadWithSubsequentStream) { - // Setup - auto constexpr kResponse0 = R"pb( - metadata { - bucket: "projects/_/buckets/test-bucket" - name: "test-object" - generation: 42 - } - read_handle { handle: "handle-12345" } - )pb"; - auto constexpr kRequest1 = R"pb( - read_ranges { read_id: 1 read_offset: 100 read_length: 100 } - )pb"; - auto constexpr kResponse1 = R"pb( - object_data_ranges { - range_end: true - read_range { read_id: 1 read_offset: 100 } - checksummed_data { content: "payload-for-stream-1" } - } - )pb"; - auto constexpr kRequest2 = R"pb( - read_ranges { read_id: 2 read_offset: 200 read_length: 200 } - )pb"; - auto constexpr kResponse2 = R"pb( - object_data_ranges { - range_end: true - read_range { read_id: 2 read_offset: 200 } - checksummed_data { content: "payload-for-stream-2" } - } - )pb"; +/// @test Verify that a background stream is created proactively. +TEST(ObjectDescriptorImpl, ProactiveStreamCreation) { + AsyncSequencer sequencer; + auto stream = std::make_unique(); + EXPECT_CALL(*stream, Read).WillOnce([&] { + return sequencer.PushBack("Read").then( + [](auto) { return absl::optional(); }); + }); + EXPECT_CALL(*stream, Finish).WillOnce(Return(make_ready_future(Status{}))); + // The stream may be cancelled by the destructor if OnFinish has not run yet. + EXPECT_CALL(*stream, Cancel).Times(AtMost(1)); + MockFactory factory; + // This is the proactive stream creation. The implementation is designed to + // always keep a pending stream in flight, so it may call the factory more + // than once if a pending stream is consumed or fails. We use WillRepeatedly + // to make the test robust to this behavior. + EXPECT_CALL(factory, Call).WillRepeatedly([&](Request const& request) { + EXPECT_TRUE(request.read_ranges().empty()); + return sequencer.PushBack("Factory").then( + [](auto) { return StatusOr(PermanentError()); }); + }); + + auto tested = std::make_shared( + NoResume(), factory.AsStdFunction(), + google::storage::v2::BidiReadObjectSpec{}, + std::make_shared(std::move(stream))); + + tested->Start(Response{}); + + // In the implementation, the Read() is started before the factory call. + auto read_called = sequencer.PopFrontWithName(); + EXPECT_EQ(read_called.second, "Read"); + + // The factory call for the proactive stream should be triggered by Start(). + auto factory_called = sequencer.PopFrontWithName(); + EXPECT_EQ(factory_called.second, "Factory"); + + // Allow the events to complete. + read_called.first.set_value(true); + factory_called.first.set_value(true); +} + + +/// @test Verify a new stream is used if all existing streams are busy. +TEST(ObjectDescriptorImpl, MakeSubsequentStreamCreatesNewWhenAllBusy) { AsyncSequencer sequencer; - // First stream setup + // Setup for the first stream, which will remain busy. auto stream1 = std::make_unique(); - EXPECT_CALL(*stream1, Write) - .WillOnce([&](Request const& request, grpc::WriteOptions) { - auto expected = Request{}; - EXPECT_TRUE(TextFormat::ParseFromString(kRequest1, &expected)); - EXPECT_THAT(request, IsProtoEqual(expected)); - return sequencer.PushBack("Write[1]").then([](auto f) { - return f.get(); - }); - }); - EXPECT_CALL(*stream1, Read) - .WillOnce([&]() { - return sequencer.PushBack("Read[1]").then([&](auto) { - auto response = Response{}; - EXPECT_TRUE(TextFormat::ParseFromString(kResponse1, &response)); - return absl::make_optional(response); - }); - }) - .WillOnce([&]() { - return sequencer.PushBack("Read[1.eos]").then([&](auto) { - return absl::optional{}; - }); - }); - EXPECT_CALL(*stream1, Finish).WillOnce([&]() { - return sequencer.PushBack("Finish[1]").then([](auto) { - return PermanentError(); - }); + EXPECT_CALL(*stream1, Write(_, _)).WillOnce([&](Request const&, auto) { + return sequencer.PushBack("Write[1]").then([](auto f) { return f.get(); }); + }); + // No Read() expectations, to keep it busy. + EXPECT_CALL(*stream1, Read).WillRepeatedly([&] { + return sequencer.PushBack("Read[1]").then( + [](auto) { return absl::optional{}; }); }); - EXPECT_CALL(*stream1, Cancel).Times(1); + EXPECT_CALL(*stream1, Finish).Times(AtMost(1)); + EXPECT_CALL(*stream1, Cancel).Times(AtMost(1)); - // Second stream setup + // Setup for the second stream, which will be created proactively. auto stream2 = std::make_unique(); - EXPECT_CALL(*stream2, Write) - .WillOnce([&](Request const& request, grpc::WriteOptions) { - auto expected = Request{}; - EXPECT_TRUE(TextFormat::ParseFromString(kRequest2, &expected)); - EXPECT_THAT(request, IsProtoEqual(expected)); - return sequencer.PushBack("Write[2]").then([](auto f) { - return f.get(); - }); - }); - EXPECT_CALL(*stream2, Read) - .WillOnce([&]() { - return sequencer.PushBack("Read[2]").then([&](auto) { - auto response = Response{}; - EXPECT_TRUE(TextFormat::ParseFromString(kResponse2, &response)); - return absl::make_optional(response); - }); - }) - .WillOnce([&]() { - return sequencer.PushBack("Read[2.eos]").then([](auto) { - return absl::optional{}; - }); - }); - EXPECT_CALL(*stream2, Finish).WillOnce([&]() { - return sequencer.PushBack("Finish[2]").then([](auto) { return Status{}; }); + EXPECT_CALL(*stream2, Write(_, _)).WillOnce([&](Request const&, auto) { + return sequencer.PushBack("Write[2]").then([](auto f) { return f.get(); }); }); - EXPECT_CALL(*stream2, Cancel).Times(1); + EXPECT_CALL(*stream2, Read).WillRepeatedly([&] { + return sequencer.PushBack("Read[2]").then( + [](auto) { return absl::optional{}; }); + }); + EXPECT_CALL(*stream2, Finish).Times(AtMost(1)); + EXPECT_CALL(*stream2, Cancel).Times(AtMost(1)); - // Mock factory for subsequent streams MockFactory factory; - EXPECT_CALL(factory, Call).WillOnce([&](Request const& request) { - EXPECT_TRUE(request.read_object_spec().has_read_handle()); - EXPECT_EQ(request.read_object_spec().read_handle().handle(), - "handle-12345"); - auto stream_result = OpenStreamResult{ - std::make_shared(std::move(stream2)), Response{}}; - return make_ready_future(make_status_or(std::move(stream_result))); + // First call is proactive for stream2. It may be called more than once. + EXPECT_CALL(factory, Call).WillRepeatedly([&](Request const&) { + return make_ready_future(StatusOr( + OpenStreamResult{std::make_shared(std::move(stream2)), + Response{}})); }); - // Create the ObjectDescriptorImpl auto tested = std::make_shared( NoResume(), factory.AsStdFunction(), google::storage::v2::BidiReadObjectSpec{}, std::make_shared(std::move(stream1))); + tested->Start(Response{}); + + // Start a read on stream1 to make it busy. + auto reader1 = tested->Read({0, 100}); + // Expect Write[1] + sequencer.PopFrontWithName().first.set_value(true); + // Do not complete the read, so stream1 remains active. - auto response0 = Response{}; - EXPECT_TRUE(TextFormat::ParseFromString(kResponse0, &response0)); - tested->Start(std::move(response0)); + // Call MakeSubsequentStream. Since stream1 is busy, this should activate the + // pending stream (stream2). + tested->MakeSubsequentStream(); + + // A new read should now be routed to stream2. + auto reader2 = tested->Read({100, 200}); + // Expect Write[2] + sequencer.PopFrontWithName().first.set_value(true); +} + +/// @test Verify reusing an idle stream that is already last is a no-op. +TEST(ObjectDescriptorImpl, MakeSubsequentStreamReusesIdleStreamAlreadyLast) { + AsyncSequencer sequencer; + + // Setup for the first (and only) stream. + auto stream1 = std::make_unique(); + EXPECT_CALL(*stream1, Write(_, _)).WillOnce([&](Request const&, auto) { + return sequencer.PushBack("Write[1]").then([](auto f) { return f.get(); }); + }); + + EXPECT_CALL(*stream1, Read) + .WillOnce([&] { // From Start() + return sequencer.PushBack("Read[1.1]").then( + [](auto) { return absl::make_optional(Response{}); }); + }) + .WillOnce([&] { // From OnRead() loop + return sequencer.PushBack("Read[1.2]").then( + [](auto) { return absl::make_optional(Response{}); }); + }) + .WillRepeatedly([&] { // Subsequent loop calls + return promise>().get_future(); + }); + + // Finish() will be called by the OpenStream destructor. + EXPECT_CALL(*stream1, Finish) + .WillOnce(Return(make_ready_future(Status{}))); + EXPECT_CALL(*stream1, Cancel).Times(AtMost(1)); + + // *** THIS IS THE FIX: *** + // We no longer need a 'stream2' mock. The factory will be + // called, but the resulting promise/future will just be + // abandoned when the test ends, which is correct. + MockFactory factory; + promise> p_factory; + EXPECT_CALL(factory, Call).WillOnce(Return(p_factory.get_future())); + + auto tested = std::make_shared( + NoResume(), factory.AsStdFunction(), + google::storage::v2::BidiReadObjectSpec{}, + std::make_shared(std::move(stream1))); + tested->Start(Response{}); + // 1. Pop the first Read() from Start() auto read1 = sequencer.PopFrontWithName(); - EXPECT_EQ(read1.second, "Read[1]"); - // Start a read on the first stream - auto reader1 = tested->Read({100, 100}); - auto future1 = reader1->Read(); - // The implementation starts a read loop eagerly after Start(), and then - // the call to tested->Read() schedules a write. - auto write1 = sequencer.PopFrontWithName(); - EXPECT_EQ(write1.second, "Write[1]"); - write1.first.set_value(true); + EXPECT_EQ(read1.second, "Read[1.1]"); - // Now we can satisfy the read. This will deliver the data to the reader. + // 2. Complete it. This schedules OnRead -> DoRead -> Read[1.2] read1.first.set_value(true); - EXPECT_THAT(future1.get(), - VariantWith(ResultOf( - "contents are", - [](storage_experimental::ReadPayload const& p) { - return p.contents(); - }, - ElementsAre(absl::string_view{"payload-for-stream-1"})))); + // 3. Wait for OnRead to run by popping Read[1.2] + auto read2 = sequencer.PopFrontWithName(); + EXPECT_EQ(read2.second, "Read[1.2]"); - EXPECT_THAT(reader1->Read().get(), VariantWith(IsOk())); + // 4. Now stream1 is IDLE. + // (active_ranges = empty, write_pending = false) - auto next = sequencer.PopFrontWithName(); - EXPECT_EQ(next.second, "Read[1.eos]"); - next.first.set_value(true); - - // Create and switch to a new stream. This happens before the first - // stream is finished. + // 5. Call MakeSubsequentStream. It should find stream1 and return. + // It will *not* consume p_factory. tested->MakeSubsequentStream(); - // The events are interleaved. Based on the log, Finish[1] comes first. - auto finish1 = sequencer.PopFrontWithName(); - EXPECT_EQ(finish1.second, "Finish[1]"); + // 6. Start a new read. It should be routed to stream1. + auto reader = tested->Read({0, 100}); + auto write1 = sequencer.PopFrontWithName(); + EXPECT_EQ(write1.second, "Write[1]"); - auto read2 = sequencer.PopFrontWithName(); - EXPECT_EQ(read2.second, "Read[2]"); - finish1.first.set_value(true); + // 7. Clean up. + write1.first.set_value(true); + + // 8. Satisfy Read[1.2] to let the read loop continue. + read2.first.set_value(true); + + // 9. *** FIX: *** + // Remove p_factory.set_value() and tested.reset(). + // The test will now end, destructors will run, and + // p_factory will be broken safely. +} - // Start a read on the second stream - auto reader2 = tested->Read({200, 200}); - auto future2 = reader2->Read(); +/// @test Verify an idle stream at the front is moved to the back and reused. +TEST(ObjectDescriptorImpl, MakeSubsequentStreamReusesAndMovesIdleStream) { + AsyncSequencer sequencer; - auto write2 = sequencer.PopFrontWithName(); - EXPECT_EQ(write2.second, "Write[2]"); - write2.first.set_value(true); + // Setup for stream1 + auto stream1 = std::make_unique(); + EXPECT_CALL(*stream1, Write(_, _)) + .WillOnce([&](Request const&, auto) { // For reader1 + return sequencer.PushBack("Write[1.1]").then([](auto f) { + return f.get(); + }); + }) + .WillOnce([&](Request const&, auto) { // For reader3 + return sequencer.PushBack("Write[1.2]").then([](auto f) { + return f.get(); + }); + }); - read2.first.set_value(true); + EXPECT_CALL(*stream1, Read) + .WillOnce([&] { // From Start() + return sequencer.PushBack("Read[1.1]").then( + [](auto) { return absl::make_optional(Response{}); }); + }) + .WillOnce([&] { // From OnRead() loop + return sequencer.PushBack("Read[1.2]").then([](auto) { + auto constexpr kResponse = R"pb( + object_data_ranges { + range_end: true + read_range { read_id: 1 read_offset: 0 } + } + )pb"; + auto response = Response{}; + EXPECT_TRUE(TextFormat::ParseFromString(kResponse, &response)); + return absl::make_optional(std::move(response)); + }); + }) + .WillOnce([&] { // From OnRead() loop after reader1 done + return sequencer.PushBack("Read[1.3]").then( + [](auto) { return absl::make_optional(Response{}); }); + }) + .WillRepeatedly([&] { + return promise>().get_future(); + }); + EXPECT_CALL(*stream1, Finish) + .WillOnce(Return(make_ready_future(Status{}))); + EXPECT_CALL(*stream1, Cancel).Times(AtMost(1)); - EXPECT_THAT(future2.get(), - VariantWith(ResultOf( - "contents are", - [](storage_experimental::ReadPayload const& p) { - return p.contents(); - }, - ElementsAre(absl::string_view{"payload-for-stream-2"})))); + // Setup for stream2 + auto stream2 = std::make_unique(); + EXPECT_CALL(*stream2, Write(_, _)).WillOnce([&](Request const&, auto) { + return sequencer.PushBack("Write[2.1]").then([](auto f) { + return f.get(); + }); + }); + EXPECT_CALL(*stream2, Read) + .WillOnce([&] { + return sequencer.PushBack("Read[2.1]").then( + [](auto) { return absl::make_optional(Response{}); }); + }) + .WillRepeatedly([&] { + return promise>().get_future(); + }); + EXPECT_CALL(*stream2, Finish) + .WillOnce(Return(make_ready_future(Status{}))); + EXPECT_CALL(*stream2, Cancel).Times(AtMost(1)); - EXPECT_THAT(reader2->Read().get(), VariantWith(IsOk())); + // Setup for stream3 (The pending one that never gets used) + auto stream3 = std::make_unique(); + // FIX: stream3 sits in pending_stream_ and is destroyed when the test ends. + // It is never "started", so Finish() is never called. + EXPECT_CALL(*stream3, Finish).Times(AtMost(1)); + EXPECT_CALL(*stream3, Cancel).Times(AtMost(1)); - auto read2_eos = sequencer.PopFrontWithName(); - EXPECT_EQ(read2_eos.second, "Read[2.eos]"); - read2_eos.first.set_value(true); + MockFactory factory; + // Expect TWO factory calls. + EXPECT_CALL(factory, Call) + .WillOnce([&] { // For stream2 (Triggered by Start) + return make_ready_future(StatusOr( + OpenStreamResult{std::make_shared(std::move(stream2)), + Response{}})); + }) + .WillOnce([&] { // For stream3 (Triggered by MakeSubsequentStream #1) + return make_ready_future(StatusOr( + OpenStreamResult{std::make_shared(std::move(stream3)), + Response{}})); + }); + + auto tested = std::make_shared( + NoResume(), factory.AsStdFunction(), + google::storage::v2::BidiReadObjectSpec{}, + std::make_shared(std::move(stream1))); + + tested->Start(Response{}); + + // 1. Pop Read[1.1] from Start() + auto read1_1 = sequencer.PopFrontWithName(); + EXPECT_EQ(read1_1.second, "Read[1.1]"); + + // 2. Make stream1 busy + auto reader1 = tested->Read({0, 100}); + auto write1_1 = sequencer.PopFrontWithName(); + EXPECT_EQ(write1_1.second, "Write[1.1]"); + write1_1.first.set_value(true); // Complete write immediately + + // 3. Create stream2. This call is non-blocking. + // This will consume the pending stream (stream2) and queue stream3. + tested->MakeSubsequentStream(); + + auto read2_1 = sequencer.PopFrontWithName(); + EXPECT_EQ(read2_1.second, "Read[2.1]"); + + // 4. Make stream2 busy + auto reader2 = tested->Read({100, 200}); + auto write2_1 = sequencer.PopFrontWithName(); + EXPECT_EQ(write2_1.second, "Write[2.1]"); + write2_1.first.set_value(true); // Complete write immediately + // At this point: streams_ = {stream1 (busy), stream2 (busy)} + + // 5. Now, make stream1 IDLE. + auto r1f1 = reader1->Read(); + read1_1.first.set_value(true); // Complete Read[1.1] (empty) + auto read1_2 = sequencer.PopFrontWithName(); // Pop Read[1.2] (with data) + EXPECT_EQ(read1_2.second, "Read[1.2]"); + read1_2.first.set_value(true); // Complete Read[1.2] + ASSERT_THAT(r1f1.get(), VariantWith(_)); + auto r1f2 = reader1->Read(); + ASSERT_THAT(r1f2.get(), VariantWith(IsOk())); + + // 6. Wait for the cleanup and next read loop + auto read1_3 = sequencer.PopFrontWithName(); + EXPECT_EQ(read1_3.second, "Read[1.3]"); + // Now: stream1 is IDLE. streams_ = {stream1 (idle), stream2 (busy)} + + // 7. Call MakeSubsequentStream. It finds stream1, moves it, and returns. + // It does NOT consume the pending stream (stream3). + tested->MakeSubsequentStream(); - auto finish2 = sequencer.PopFrontWithName(); - EXPECT_EQ(finish2.second, "Finish[2]"); - finish2.first.set_value(true); + // 8. Start a new read. It should be routed to stream1 (now at the back). + auto reader3 = tested->Read({200, 300}); + auto write1_2 = sequencer.PopFrontWithName(); + EXPECT_EQ(write1_2.second, "Write[1.2]"); + + // 9. Clean up + write1_2.first.set_value(true); + read1_3.first.set_value(true); + read2_1.first.set_value(true); + + // Explicitly reset to trigger destructors while expectations are active + tested.reset(); } } // namespace diff --git a/google/cloud/storage/storage_client_grpc_unit_tests.bzl b/google/cloud/storage/storage_client_grpc_unit_tests.bzl index 9387e768810aa..d009c79025598 100644 --- a/google/cloud/storage/storage_client_grpc_unit_tests.bzl +++ b/google/cloud/storage/storage_client_grpc_unit_tests.bzl @@ -41,6 +41,7 @@ storage_client_grpc_unit_tests = [ "internal/async/default_options_test.cc", "internal/async/handle_redirect_error_test.cc", "internal/async/insert_object_test.cc", + "internal/async/multi_stream_manager_test.cc", "internal/async/object_descriptor_connection_tracing_test.cc", "internal/async/object_descriptor_impl_test.cc", "internal/async/object_descriptor_reader_test.cc", From 017da801c0287cb4f9c0e63805399e78242ace4c Mon Sep 17 00:00:00 2001 From: Shubham Kaushal Date: Thu, 11 Dec 2025 09:11:27 +0000 Subject: [PATCH 02/12] file formatting --- .../cloud/storage/async/object_descriptor.cc | 2 +- .../async/connection_impl_open_test.cc | 16 ++ .../internal/async/multi_stream_manager.cc | 2 +- .../internal/async/multi_stream_manager.h | 7 +- .../async/multi_stream_manager_test.cc | 62 +++--- .../internal/async/object_descriptor_impl.cc | 142 ++++++------ .../internal/async/object_descriptor_impl.h | 27 ++- .../async/object_descriptor_impl_test.cc | 203 +++++++----------- .../tests/async_client_integration_test.cc | 2 +- 9 files changed, 233 insertions(+), 230 deletions(-) diff --git a/google/cloud/storage/async/object_descriptor.cc b/google/cloud/storage/async/object_descriptor.cc index f17756fe91bcc..90877d414f9f6 100644 --- a/google/cloud/storage/async/object_descriptor.cc +++ b/google/cloud/storage/async/object_descriptor.cc @@ -28,7 +28,7 @@ absl::optional ObjectDescriptor::metadata() const { std::pair ObjectDescriptor::Read(std::int64_t offset, std::int64_t limit) { std::int64_t max_range = - impl_->options().get(); + impl_->options().get(); if (limit > max_range) { impl_->MakeSubsequentStream(); } diff --git a/google/cloud/storage/internal/async/connection_impl_open_test.cc b/google/cloud/storage/internal/async/connection_impl_open_test.cc index 6d72d6f40a6ea..a8396d612bcd6 100644 --- a/google/cloud/storage/internal/async/connection_impl_open_test.cc +++ b/google/cloud/storage/internal/async/connection_impl_open_test.cc @@ -44,8 +44,11 @@ using ::google::cloud::testing_util::IsProtoEqual; using ::google::cloud::testing_util::MockCompletionQueueImpl; using ::google::cloud::testing_util::StatusIs; using ::google::protobuf::TextFormat; +using ::testing::InvokeWithoutArgs; +using ::testing::NiceMock; using ::testing::NotNull; using ::testing::Optional; +using ::testing::Return; using BidiReadStream = google::cloud::AsyncStreamingReadWriteRpc< google::storage::v2::BidiReadObjectRequest, @@ -183,6 +186,19 @@ TEST(AsyncConnectionImplTest, OpenSimple) { [](auto) { return Status{}; }); }); + return std::unique_ptr(std::move(stream)); + }) + .WillRepeatedly([](CompletionQueue const&, + std::shared_ptr const&, + google::cloud::internal::ImmutableOptions const&) { + auto stream = std::make_unique>(); + ON_CALL(*stream, Start).WillByDefault(InvokeWithoutArgs([] { + return make_ready_future(false); + })); + ON_CALL(*stream, Finish).WillByDefault(InvokeWithoutArgs([] { + return make_ready_future(Status{}); + })); + ON_CALL(*stream, Cancel).WillByDefault([] {}); return std::unique_ptr(std::move(stream)); }); diff --git a/google/cloud/storage/internal/async/multi_stream_manager.cc b/google/cloud/storage/internal/async/multi_stream_manager.cc index 5666299eb2697..2425c990a7712 100644 --- a/google/cloud/storage/internal/async/multi_stream_manager.cc +++ b/google/cloud/storage/internal/async/multi_stream_manager.cc @@ -1,4 +1,4 @@ -// Copyright 2024 Google LLC +// Copyright 2025 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/google/cloud/storage/internal/async/multi_stream_manager.h b/google/cloud/storage/internal/async/multi_stream_manager.h index 9f5552470e49f..f3a781e8852fb 100644 --- a/google/cloud/storage/internal/async/multi_stream_manager.h +++ b/google/cloud/storage/internal/async/multi_stream_manager.h @@ -1,4 +1,4 @@ -// Copyright 2024 Google LLC +// Copyright 2025 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -62,7 +62,8 @@ class MultiStreamManager { // Constructor accepts an already-created initial stream. // This is required by ObjectDescriptorImpl which receives an OpenStream. - MultiStreamManager(StreamFactory stream_factory, std::shared_ptr initial_stream) + MultiStreamManager(StreamFactory stream_factory, + std::shared_ptr initial_stream) : stream_factory_(std::move(stream_factory)) { streams_.push_back(Stream{std::move(initial_stream), {}}); } @@ -72,7 +73,7 @@ class MultiStreamManager { // In ObjectDescriptorImpl, we ensure there is always at least one stream, // but this assertion protects against future refactoring errors. assert(!streams_.empty()); - return std::prev(streams_.end()); + return std::prev(streams_.end()); } StreamIterator GetLeastBusyStream() { diff --git a/google/cloud/storage/internal/async/multi_stream_manager_test.cc b/google/cloud/storage/internal/async/multi_stream_manager_test.cc index 36488361a4998..a4a6bde72e5dc 100644 --- a/google/cloud/storage/internal/async/multi_stream_manager_test.cc +++ b/google/cloud/storage/internal/async/multi_stream_manager_test.cc @@ -1,4 +1,4 @@ -// Copyright 2024 Google LLC +// Copyright 2025 Google LLC // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -39,7 +39,9 @@ struct FakeStream : public StreamBase { using Manager = MultiStreamManager; struct MultiStreamManagerTest : public ::testing::Test { - static Manager MakeManager() { return Manager([] { return std::make_shared(); }); } + static Manager MakeManager() { + return Manager([] { return std::make_shared(); }); + } }; } // namespace @@ -72,9 +74,10 @@ TEST(MultiStreamManagerTest, AddStreamAppendsAndGetLastReturnsNew) { TEST(MultiStreamManagerTest, GetLeastBusyPrefersFewestActiveRanges) { auto mgr = MultiStreamManagerTest::MakeManager(); - - // The manager starts with an initial stream (size 0). - // We must make it "busy" so it doesn't win the comparison against our test streams. + + // The manager starts with an initial stream (size 0). + // We must make it "busy" so it doesn't win the comparison against our test + // streams. auto it_init = mgr.GetLastStream(); it_init->active_ranges.emplace(999, std::make_shared()); it_init->active_ranges.emplace(998, std::make_shared()); @@ -87,23 +90,26 @@ TEST(MultiStreamManagerTest, GetLeastBusyPrefersFewestActiveRanges) { // s1 has 2 ranges. it1->active_ranges.emplace(1, std::make_shared()); it1->active_ranges.emplace(2, std::make_shared()); - + // s2 has 1 range. it2->active_ranges.emplace(3, std::make_shared()); - + auto it_least = mgr.GetLeastBusyStream(); - + // Expect it2 (1 range) over it1 (2 ranges) and it_init (2 ranges). - EXPECT_EQ(it_least, it2); + EXPECT_EQ(it_least, it2); EXPECT_EQ(it_least->active_ranges.size(), 1u); } TEST(MultiStreamManagerTest, CleanupDoneRangesRemovesFinished) { auto mgr = MultiStreamManagerTest::MakeManager(); auto it = mgr.GetLastStream(); - auto r1 = std::make_shared(); r1->done = false; - auto r2 = std::make_shared(); r2->done = true; - auto r3 = std::make_shared(); r3->done = true; + auto r1 = std::make_shared(); + r1->done = false; + auto r2 = std::make_shared(); + r2->done = true; + auto r3 = std::make_shared(); + r3->done = true; it->active_ranges.emplace(1, r1); it->active_ranges.emplace(2, r2); it->active_ranges.emplace(3, r3); @@ -142,11 +148,10 @@ TEST(MultiStreamManagerTest, ReuseIdleStreamToBackMovesElement) { auto factory_ptr = mgr.GetLastStream()->stream.get(); auto s1 = std::make_shared(); mgr.AddStream(s1); - bool moved = mgr.ReuseIdleStreamToBack( - [](Manager::Stream const& s) { - auto fs = s.stream.get(); - return fs != nullptr && s.active_ranges.empty() && !fs->write_pending; - }); + bool moved = mgr.ReuseIdleStreamToBack([](Manager::Stream const& s) { + auto fs = s.stream.get(); + return fs != nullptr && s.active_ranges.empty() && !fs->write_pending; + }); EXPECT_TRUE(moved); auto it_last = mgr.GetLastStream(); // After move, the factory stream should be last @@ -154,14 +159,13 @@ TEST(MultiStreamManagerTest, ReuseIdleStreamToBackMovesElement) { EXPECT_NE(it_last->stream.get(), s1.get()); } -TEST(MultiStreamManagerTest, ReuseIdleStreamAlreadyAtBackReturnsTrueWithoutMove) { +TEST(MultiStreamManagerTest, + ReuseIdleStreamAlreadyAtBackReturnsTrueWithoutMove) { auto mgr = MultiStreamManagerTest::MakeManager(); // The manager starts with one stream. It is the last stream, and it is idle. auto initial_last = mgr.GetLastStream(); bool reused = mgr.ReuseIdleStreamToBack( - [](Manager::Stream const& s) { - return s.active_ranges.empty(); - }); + [](Manager::Stream const& s) { return s.active_ranges.empty(); }); EXPECT_TRUE(reused); // Pointer should remain the same (it was already at the back) EXPECT_EQ(mgr.GetLastStream(), initial_last); @@ -174,11 +178,10 @@ TEST(MultiStreamManagerTest, ReuseIdleStreamDoesNotMoveWhenWritePending) { auto s1 = std::make_shared(); s1->write_pending = true; // also mark appended stream as not reusable mgr.AddStream(s1); - bool moved = mgr.ReuseIdleStreamToBack( - [](Manager::Stream const& s) { - auto fs = s.stream.get(); - return fs != nullptr && s.active_ranges.empty() && !fs->write_pending; - }); + bool moved = mgr.ReuseIdleStreamToBack([](Manager::Stream const& s) { + auto fs = s.stream.get(); + return fs != nullptr && s.active_ranges.empty() && !fs->write_pending; + }); EXPECT_FALSE(moved); auto it_last = mgr.GetLastStream(); EXPECT_EQ(it_last->stream.get(), s1.get()); @@ -206,10 +209,9 @@ TEST(MultiStreamManagerTest, GetLastStreamReflectsRecentAppendAndReuse) { auto s1 = std::make_shared(); mgr.AddStream(s1); EXPECT_EQ(mgr.GetLastStream()->stream.get(), s1.get()); - bool moved = mgr.ReuseIdleStreamToBack( - [](Manager::Stream const& s) { - return s.stream != nullptr && s.active_ranges.empty(); - }); + bool moved = mgr.ReuseIdleStreamToBack([](Manager::Stream const& s) { + return s.stream != nullptr && s.active_ranges.empty(); + }); EXPECT_TRUE(moved); auto it_last = mgr.GetLastStream(); EXPECT_NE(it_last->stream.get(), s1.get()); diff --git a/google/cloud/storage/internal/async/object_descriptor_impl.cc b/google/cloud/storage/internal/async/object_descriptor_impl.cc index d7a50a1e4a46e..c0b96b749b536 100644 --- a/google/cloud/storage/internal/async/object_descriptor_impl.cc +++ b/google/cloud/storage/internal/async/object_descriptor_impl.cc @@ -15,10 +15,10 @@ #include "google/cloud/storage/internal/async/object_descriptor_impl.h" #include "google/cloud/storage/async/options.h" #include "google/cloud/storage/internal/async/handle_redirect_error.h" +#include "google/cloud/storage/internal/async/multi_stream_manager.h" #include "google/cloud/storage/internal/async/object_descriptor_reader_tracing.h" #include "google/cloud/storage/internal/hash_function.h" #include "google/cloud/storage/internal/hash_function_impl.h" -#include "google/cloud/storage/internal/async/multi_stream_manager.h" #include "google/cloud/grpc_error_delegate.h" #include "google/cloud/internal/opentelemetry.h" #include @@ -38,26 +38,23 @@ ObjectDescriptorImpl::ObjectDescriptorImpl( : resume_policy_prototype_(std::move(resume_policy)), make_stream_(std::move(make_stream)), read_object_spec_(std::move(read_object_spec)), - options_(std::move(options)), - // Initialize manager with the first stream. - stream_manager_(std::make_unique( - []() -> std::shared_ptr { return nullptr; }, - std::make_shared(std::move(stream), - resume_policy_prototype_->clone()))) {} - -ObjectDescriptorImpl::~ObjectDescriptorImpl() { - Cancel(); + options_(std::move(options)) { + stream_manager_ = std::make_unique( + []() -> std::shared_ptr { return nullptr; }, + std::make_shared(std::move(stream), + resume_policy_prototype_->clone())); } +ObjectDescriptorImpl::~ObjectDescriptorImpl() { Cancel(); } + void ObjectDescriptorImpl::Start( google::storage::v2::BidiReadObjectResponse first_response) { std::unique_lock lk(mu_); auto it = stream_manager_->GetLastStream(); -// FIX: Unlock and start the Read loop FIRST (Senior's logic) + // Unlock and start the Read loop first. lk.unlock(); OnRead(it, std::move(first_response)); - - // FIX: Then acquire lock and queue the background stream SECOND + // Acquire lock and queue the background stream. lk.lock(); AssurePendingStreamQueued(); } @@ -78,14 +75,14 @@ absl::optional ObjectDescriptorImpl::metadata() void ObjectDescriptorImpl::AssurePendingStreamQueued() { if (pending_stream_.valid()) return; auto request = google::storage::v2::BidiReadObjectRequest{}; + *request.mutable_read_object_spec() = read_object_spec_; pending_stream_ = make_stream_(std::move(request)); } void ObjectDescriptorImpl::MakeSubsequentStream() { std::unique_lock lk(mu_); - - // 1. REUSE LOGIC (Using Manager) + // Reuse an idle stream if possible. if (stream_manager_->ReuseIdleStreamToBack( [](StreamManager::Stream const& s) { auto const* rs = s.stream.get(); @@ -93,31 +90,27 @@ void ObjectDescriptorImpl::MakeSubsequentStream() { })) { return; } - - // 2. PROACTIVE CREATION LOGIC (Senior's Flow) - AssurePendingStreamQueued(); // Ensure a stream is being created + // Proactively create a new stream if needed. + AssurePendingStreamQueued(); auto stream_future = std::move(pending_stream_); lk.unlock(); - // BLOCKING WAIT (Matches senior's logic) + // Wait for the stream to be created. auto stream_result = stream_future.get(); - if (!stream_result) { - // Stream creation failed. + // Stream creation failed. // The next call to AssurePendingStreamQueued will retry creation. return; } lk.lock(); if (cancelled_) return; - - auto rs = std::make_shared(std::move(stream_result->stream), - resume_policy_prototype_->clone()); - auto new_it = stream_manager_->AddStream(std::move(rs)); - + auto read_stream = std::make_shared( + std::move(stream_result->stream), resume_policy_prototype_->clone()); + auto new_it = stream_manager_->AddStream(std::move(read_stream)); // Now that we consumed pending_stream_, queue the next one immediately. AssurePendingStreamQueued(); - + lk.unlock(); OnRead(new_it, std::move(stream_result->first_response)); } @@ -147,10 +140,8 @@ ObjectDescriptorImpl::Read(ReadParams p) { } auto it = stream_manager_->GetLeastBusyStream(); - auto const id = ++read_id_generator_; it->active_ranges.emplace(id, range); - auto& read_range = *it->stream->next_request.add_read_ranges(); read_range.set_read_id(id); read_range.set_read_offset(p.start); @@ -165,19 +156,25 @@ ObjectDescriptorImpl::Read(ReadParams p) { return MakeTracingObjectDescriptorReader(std::move(range)); } -void ObjectDescriptorImpl::Flush(std::unique_lock lk, StreamIterator it) { - if (it->stream->write_pending || it->stream->next_request.read_ranges().empty()) { +void ObjectDescriptorImpl::Flush(std::unique_lock lk, + StreamIterator it) { + if (it->stream->write_pending || + it->stream->next_request.read_ranges().empty()) { return; } it->stream->write_pending = true; google::storage::v2::BidiReadObjectRequest request; request.Swap(&it->stream->next_request); + // Assign CurrentStream to a temporary variable to prevent + // lifetime extension which can cause the lock to be held until the + // end of the block. auto current_stream = it->stream->stream_; lk.unlock(); - current_stream->Write(std::move(request)).then([w = WeakFromThis(), it](auto f) { - if (auto self = w.lock()) self->OnWrite(it, f.get()); - }); + current_stream->Write(std::move(request)) + .then([w = WeakFromThis(), it](auto f) { + if (auto self = w.lock()) self->OnWrite(it, f.get()); + }); } void ObjectDescriptorImpl::OnWrite(StreamIterator it, bool ok) { @@ -187,10 +184,14 @@ void ObjectDescriptorImpl::OnWrite(StreamIterator it, bool ok) { Flush(std::move(lk), it); } -void ObjectDescriptorImpl::DoRead(std::unique_lock lk, StreamIterator it) { +void ObjectDescriptorImpl::DoRead(std::unique_lock lk, + StreamIterator it) { if (it->stream->read_pending) return; it->stream->read_pending = true; - + + // Assign CurrentStream to a temporary variable to prevent + // lifetime extension which can cause the lock to be held until the + // end of the block. auto current_stream = it->stream->stream_; lk.unlock(); current_stream->Read().then([w = WeakFromThis(), it](auto f) { @@ -198,38 +199,47 @@ void ObjectDescriptorImpl::DoRead(std::unique_lock lk, StreamIterato }); } -void ObjectDescriptorImpl::OnRead(StreamIterator it, absl::optional response) { +void ObjectDescriptorImpl::OnRead( + StreamIterator it, + absl::optional response) { std::unique_lock lk(mu_); it->stream->read_pending = false; - + if (!response) return DoFinish(std::move(lk), it); - if (response->has_metadata()) { - metadata_ = std::move(*response->mutable_metadata()); + metadata_ = std::move(*response->mutable_metadata()); } if (response->has_read_handle()) { - *read_object_spec_.mutable_read_handle() = std::move(*response->mutable_read_handle()); + *read_object_spec_.mutable_read_handle() = + std::move(*response->mutable_read_handle()); } - auto copy = it->active_ranges; + // Release the lock while notifying the ranges. The notifications may trigger + // application code, and that code may callback on this class. lk.unlock(); - - for (auto& rd : *response->mutable_object_data_ranges()) { - auto id = rd.read_range().read_id(); + for (auto& range_data : *response->mutable_object_data_ranges()) { + auto id = range_data.read_range().read_id(); auto const l = copy.find(id); if (l == copy.end()) continue; - l->second->OnRead(std::move(rd)); + // TODO(#15104) - Consider returning if the range is done, and then + // skipping CleanupDoneRanges(). + l->second->OnRead(std::move(range_data)); } - lk.lock(); stream_manager_->CleanupDoneRanges(it); DoRead(std::move(lk), it); } -void ObjectDescriptorImpl::DoFinish(std::unique_lock lk, StreamIterator it) { +void ObjectDescriptorImpl::DoFinish(std::unique_lock lk, + StreamIterator it) { it->stream->read_pending = false; - auto pending = it->stream->stream_->Finish(); + // Assign CurrentStream to a temporary variable to prevent + // lifetime extension which can cause the lock to be held until the + // end of the block. + auto current_stream = it->stream->stream_; lk.unlock(); + auto pending = current_stream->Finish(); + if (!pending.valid()) return; pending.then([w = WeakFromThis(), it](auto f) { if (auto self = w.lock()) self->OnFinish(it, f.get()); }); @@ -237,55 +247,57 @@ void ObjectDescriptorImpl::DoFinish(std::unique_lock lk, StreamItera void ObjectDescriptorImpl::OnFinish(StreamIterator it, Status const& status) { auto proto_status = ExtractGrpcStatus(status); - if (IsResumable(it, status, proto_status)) return Resume(it, proto_status); + if (IsResumable(it, status, proto_status)) return Resume(it, proto_status); std::unique_lock lk(mu_); stream_manager_->RemoveStreamAndNotifyRanges(it, status); - - // Since a stream died, we might want to ensure a replacement is queued + // Since a stream died, we might want to ensure a replacement is queued. AssurePendingStreamQueued(); } -void ObjectDescriptorImpl::Resume(StreamIterator it, google::rpc::Status const& proto_status) { +void ObjectDescriptorImpl::Resume(StreamIterator it, + google::rpc::Status const& proto_status) { std::unique_lock lk(mu_); + // This call needs to happen inside the lock, as it may modify + // `read_object_spec_`. ApplyRedirectErrors(read_object_spec_, proto_status); - - google::storage::v2::BidiReadObjectRequest request; + auto request = google::storage::v2::BidiReadObjectRequest{}; *request.mutable_read_object_spec() = read_object_spec_; - for (auto const& kv : it->active_ranges) { auto range = kv.second->RangeForResume(kv.first); if (!range) continue; *request.add_read_ranges() = *std::move(range); } lk.unlock(); - make_stream_(std::move(request)).then([w = WeakFromThis(), it](auto f) { if (auto self = w.lock()) self->OnResume(it, f.get()); }); } -void ObjectDescriptorImpl::OnResume(StreamIterator it, StatusOr result) { +void ObjectDescriptorImpl::OnResume(StreamIterator it, + StatusOr result) { if (!result) return OnFinish(it, std::move(result).status()); - std::unique_lock lk(mu_); if (cancelled_) return; - - it->stream = std::make_shared(std::move(result->stream), + + it->stream = std::make_shared(std::move(result->stream), resume_policy_prototype_->clone()); it->stream->write_pending = false; it->stream->read_pending = false; - + + // TODO(#15105) - this should be done without release the lock. Flush(std::move(lk), it); OnRead(it, std::move(result->first_response)); } -bool ObjectDescriptorImpl::IsResumable(StreamIterator it, Status const& status, google::rpc::Status const& proto_status) { +bool ObjectDescriptorImpl::IsResumable( + StreamIterator it, Status const& status, + google::rpc::Status const& proto_status) { std::unique_lock lk(mu_); for (auto const& any : proto_status.details()) { auto error = google::storage::v2::BidiReadObjectError{}; if (!any.UnpackTo(&error)) continue; - + std::vector> notify; for (auto const& re : error.read_range_errors()) { if (it->active_ranges.count(re.read_id())) { @@ -293,7 +305,7 @@ bool ObjectDescriptorImpl::IsResumable(StreamIterator it, Status const& status, } } if (notify.empty()) continue; - + auto copy = it->active_ranges; lk.unlock(); for (auto const& p : notify) { diff --git a/google/cloud/storage/internal/async/object_descriptor_impl.h b/google/cloud/storage/internal/async/object_descriptor_impl.h index 5139e224af9ea..8c0f27b4404bf 100644 --- a/google/cloud/storage/internal/async/object_descriptor_impl.h +++ b/google/cloud/storage/internal/async/object_descriptor_impl.h @@ -17,10 +17,10 @@ #include "google/cloud/storage/async/object_descriptor_connection.h" #include "google/cloud/storage/async/resume_policy.h" +#include "google/cloud/storage/internal/async/multi_stream_manager.h" #include "google/cloud/storage/internal/async/object_descriptor_reader.h" #include "google/cloud/storage/internal/async/open_stream.h" #include "google/cloud/storage/internal/async/read_range.h" -#include "google/cloud/storage/internal/async/multi_stream_manager.h" #include "google/cloud/storage/options.h" #include "google/cloud/status.h" #include "google/cloud/version.h" @@ -41,8 +41,10 @@ class ReadStream : public storage_internal::StreamBase { ReadStream(std::shared_ptr stream, std::unique_ptr resume_policy) : stream_(std::move(stream)), resume_policy_(std::move(resume_policy)) {} - - void Cancel() override { if (stream_) stream_->Cancel(); } + + void Cancel() override { + if (stream_) stream_->Cancel(); + } std::shared_ptr stream_; std::unique_ptr resume_policy_; @@ -66,13 +68,19 @@ class ObjectDescriptorImpl std::shared_ptr stream, Options options = {}); ~ObjectDescriptorImpl() override; + // Start the read loop. void Start(google::storage::v2::BidiReadObjectResponse first_response); + + // Cancel the underlying RPC and stop the resume loop. void Cancel(); Options options() const override { return options_; } + // Return the object metadata. This is only available after the first `Read()` + // returns. absl::optional metadata() const override; + // Start a new ranged read. std::unique_ptr Read( ReadParams p) override; @@ -83,18 +91,21 @@ class ObjectDescriptorImpl return shared_from_this(); } - // Restored: Logic to ensure a background stream is always connecting + // Logic to ensure a background stream is always connecting. void AssurePendingStreamQueued(); void Flush(std::unique_lock lk, StreamIterator it); void OnWrite(StreamIterator it, bool ok); void DoRead(std::unique_lock lk, StreamIterator it); - void OnRead(StreamIterator it, absl::optional response); + void OnRead( + StreamIterator it, + absl::optional response); void DoFinish(std::unique_lock lk, StreamIterator it); void OnFinish(StreamIterator it, Status const& status); void Resume(StreamIterator it, google::rpc::Status const& proto_status); void OnResume(StreamIterator it, StatusOr result); - bool IsResumable(StreamIterator it, Status const& status, google::rpc::Status const& proto_status); + bool IsResumable(StreamIterator it, Status const& status, + google::rpc::Status const& proto_status); std::unique_ptr resume_policy_prototype_; OpenStreamFactory make_stream_; @@ -106,12 +117,10 @@ class ObjectDescriptorImpl Options options_; std::unique_ptr stream_manager_; - - // Restored: The future for the proactive background stream + // The future for the proactive background stream. google::cloud::future< google::cloud::StatusOr> pending_stream_; - bool cancelled_ = false; }; diff --git a/google/cloud/storage/internal/async/object_descriptor_impl_test.cc b/google/cloud/storage/internal/async/object_descriptor_impl_test.cc index 58071abdccd93..fd354b3fbdb87 100644 --- a/google/cloud/storage/internal/async/object_descriptor_impl_test.cc +++ b/google/cloud/storage/internal/async/object_descriptor_impl_test.cc @@ -27,7 +27,7 @@ #include #include #include -#include // Make sure to add this include for std::thread +#include namespace google { namespace cloud { @@ -42,15 +42,15 @@ using ::google::cloud::testing_util::IsOk; using ::google::cloud::testing_util::IsProtoEqual; using ::google::cloud::testing_util::StatusIs; using ::google::protobuf::TextFormat; +using ::testing::_; +using ::testing::AtMost; using ::testing::ElementsAre; using ::testing::NotNull; using ::testing::Optional; using ::testing::ResultOf; -using ::testing::VariantWith; -using ::testing::AtMost; using ::testing::Return; using ::testing::UnorderedElementsAre; -using ::testing::_; +using ::testing::VariantWith; using Request = google::storage::v2::BidiReadObjectRequest; using Response = google::storage::v2::BidiReadObjectResponse; @@ -279,7 +279,6 @@ TEST(ObjectDescriptorImpl, ReadSingleRange) { next.first.set_value(true); } - /// @test Reading multiple ranges creates a single request. TEST(ObjectDescriptorImpl, ReadMultipleRanges) { auto constexpr kLength = 100; @@ -785,15 +784,14 @@ TEST(ObjectDescriptorImpl, ResumeRangesOnRecoverableError) { EXPECT_CALL(factory, Call) .WillOnce([](Request const& request) { EXPECT_TRUE(request.read_ranges().empty()); - return make_ready_future( - StatusOr(TransientError())); + return make_ready_future(StatusOr(TransientError())); }) .WillOnce([&](Request const& request) { EXPECT_THAT(request, IsProtoEqualModuloRepeatedFieldOrdering( expected_resume_request)); - return sequencer.PushBack("Factory").then([](auto) { - return StatusOr(PermanentError()); - }); + // Resume with an unrecoverable failure to simplify the test. + return sequencer.PushBack("Factory").then( + [](auto) { return StatusOr(PermanentError()); }); }); auto spec = google::storage::v2::BidiReadObjectSpec{}; @@ -921,7 +919,8 @@ TEST(ObjectDescriptorImpl, PendingFinish) { )pb"; auto stream = std::make_unique(); - EXPECT_CALL(*stream, Cancel).Times(AtMost(1)); + EXPECT_CALL(*stream, Cancel) + .Times(AtMost(1)); // Always called by OpenStream EXPECT_CALL(*stream, Write) .WillOnce([=, &sequencer](Request const& request, grpc::WriteOptions) { auto expected = Request{}; @@ -1065,18 +1064,16 @@ TEST(ObjectDescriptorImpl, ResumeUsesRouting) { EXPECT_CALL(factory, Call) .WillOnce([](Request const& request) { EXPECT_TRUE(request.read_ranges().empty()); - return make_ready_future( - StatusOr(TransientError())); + return make_ready_future(StatusOr(TransientError())); }) .WillOnce([&](Request const& request) { EXPECT_THAT(request, IsProtoEqualModuloRepeatedFieldOrdering( expected_resume_request)); - return sequencer.PushBack("Factory").then([](auto) { - return StatusOr(PermanentError()); - }); + // Resume with an unrecoverable failure to simplify the test. + return sequencer.PushBack("Factory").then( + [](auto) { return StatusOr(PermanentError()); }); }); - auto spec = google::storage::v2::BidiReadObjectSpec{}; ASSERT_TRUE(TextFormat::ParseFromString(kReadSpecText, &spec)); auto tested = std::make_shared( @@ -1200,22 +1197,21 @@ TEST(ObjectDescriptorImpl, RecoverFromPartialFailure) { }); EXPECT_CALL(*stream, Cancel).Times(AtMost(1)); - Request expected_resume_request; + Request expected_resume_request; ASSERT_TRUE( TextFormat::ParseFromString(kResumeRequest, &expected_resume_request)); MockFactory factory; EXPECT_CALL(factory, Call) .WillOnce([](Request const& request) { EXPECT_TRUE(request.read_ranges().empty()); - return make_ready_future( - StatusOr(TransientError())); + return make_ready_future(StatusOr(TransientError())); }) .WillOnce([&](Request const& request) { EXPECT_THAT(request, IsProtoEqualModuloRepeatedFieldOrdering( expected_resume_request)); - return sequencer.PushBack("Factory").then([](auto) { - return StatusOr(PermanentError()); - }); + // Resume with an unrecoverable failure to simplify the test. + return sequencer.PushBack("Factory").then( + [](auto) { return StatusOr(PermanentError()); }); }); auto spec = google::storage::v2::BidiReadObjectSpec{}; @@ -1288,8 +1284,7 @@ TEST(ObjectDescriptorImpl, ProactiveStreamCreation) { [](auto) { return absl::optional(); }); }); EXPECT_CALL(*stream, Finish).WillOnce(Return(make_ready_future(Status{}))); - // The stream may be cancelled by the destructor if OnFinish has not run yet. - EXPECT_CALL(*stream, Cancel).Times(AtMost(1)); + EXPECT_CALL(*stream, Cancel).Times(AtMost(1)); // Always called by OpenStream MockFactory factory; // This is the proactive stream creation. The implementation is designed to @@ -1313,7 +1308,6 @@ TEST(ObjectDescriptorImpl, ProactiveStreamCreation) { auto read_called = sequencer.PopFrontWithName(); EXPECT_EQ(read_called.second, "Read"); - // The factory call for the proactive stream should be triggered by Start(). auto factory_called = sequencer.PopFrontWithName(); EXPECT_EQ(factory_called.second, "Factory"); @@ -1322,7 +1316,6 @@ TEST(ObjectDescriptorImpl, ProactiveStreamCreation) { factory_called.first.set_value(true); } - /// @test Verify a new stream is used if all existing streams are busy. TEST(ObjectDescriptorImpl, MakeSubsequentStreamCreatesNewWhenAllBusy) { AsyncSequencer sequencer; @@ -1355,9 +1348,8 @@ TEST(ObjectDescriptorImpl, MakeSubsequentStreamCreatesNewWhenAllBusy) { MockFactory factory; // First call is proactive for stream2. It may be called more than once. EXPECT_CALL(factory, Call).WillRepeatedly([&](Request const&) { - return make_ready_future(StatusOr( - OpenStreamResult{std::make_shared(std::move(stream2)), - Response{}})); + return make_ready_future(StatusOr(OpenStreamResult{ + std::make_shared(std::move(stream2)), Response{}})); }); auto tested = std::make_shared( @@ -1368,9 +1360,7 @@ TEST(ObjectDescriptorImpl, MakeSubsequentStreamCreatesNewWhenAllBusy) { // Start a read on stream1 to make it busy. auto reader1 = tested->Read({0, 100}); - // Expect Write[1] sequencer.PopFrontWithName().first.set_value(true); - // Do not complete the read, so stream1 remains active. // Call MakeSubsequentStream. Since stream1 is busy, this should activate the // pending stream (stream2). @@ -1378,7 +1368,6 @@ TEST(ObjectDescriptorImpl, MakeSubsequentStreamCreatesNewWhenAllBusy) { // A new read should now be routed to stream2. auto reader2 = tested->Read({100, 200}); - // Expect Write[2] sequencer.PopFrontWithName().first.set_value(true); } @@ -1386,31 +1375,32 @@ TEST(ObjectDescriptorImpl, MakeSubsequentStreamCreatesNewWhenAllBusy) { TEST(ObjectDescriptorImpl, MakeSubsequentStreamReusesIdleStreamAlreadyLast) { AsyncSequencer sequencer; - // Setup for the first (and only) stream. + // Setup for the first and only stream. auto stream1 = std::make_unique(); EXPECT_CALL(*stream1, Write(_, _)).WillOnce([&](Request const&, auto) { return sequencer.PushBack("Write[1]").then([](auto f) { return f.get(); }); }); EXPECT_CALL(*stream1, Read) - .WillOnce([&] { // From Start() - return sequencer.PushBack("Read[1.1]").then( - [](auto) { return absl::make_optional(Response{}); }); + .WillOnce([&] { // From Start() + return sequencer.PushBack("Read[1.1]").then([](auto) { + return absl::make_optional(Response{}); + }); }) - .WillOnce([&] { // From OnRead() loop - return sequencer.PushBack("Read[1.2]").then( - [](auto) { return absl::make_optional(Response{}); }); + .WillOnce([&] { // From OnRead() loop + return sequencer.PushBack("Read[1.2]").then([](auto) { + return absl::make_optional(Response{}); + }); }) - .WillRepeatedly([&] { // Subsequent loop calls + .WillRepeatedly([&] { // Subsequent loop calls return promise>().get_future(); }); // Finish() will be called by the OpenStream destructor. - EXPECT_CALL(*stream1, Finish) - .WillOnce(Return(make_ready_future(Status{}))); - EXPECT_CALL(*stream1, Cancel).Times(AtMost(1)); + EXPECT_CALL(*stream1, Finish).WillOnce(Return(make_ready_future(Status{}))); + EXPECT_CALL(*stream1, Cancel) + .Times(AtMost(1)); // Always called by OpenStream - // *** THIS IS THE FIX: *** // We no longer need a 'stream2' mock. The factory will be // called, but the resulting promise/future will just be // abandoned when the test ends, which is correct. @@ -1424,65 +1414,51 @@ TEST(ObjectDescriptorImpl, MakeSubsequentStreamReusesIdleStreamAlreadyLast) { std::make_shared(std::move(stream1))); tested->Start(Response{}); - // 1. Pop the first Read() from Start() auto read1 = sequencer.PopFrontWithName(); EXPECT_EQ(read1.second, "Read[1.1]"); - // 2. Complete it. This schedules OnRead -> DoRead -> Read[1.2] read1.first.set_value(true); - // 3. Wait for OnRead to run by popping Read[1.2] auto read2 = sequencer.PopFrontWithName(); EXPECT_EQ(read2.second, "Read[1.2]"); - // 4. Now stream1 is IDLE. - // (active_ranges = empty, write_pending = false) - - // 5. Call MakeSubsequentStream. It should find stream1 and return. - // It will *not* consume p_factory. + // Call MakeSubsequentStream. It should find stream1 and return. tested->MakeSubsequentStream(); - // 6. Start a new read. It should be routed to stream1. + // Start a new read. It should be routed to stream1. auto reader = tested->Read({0, 100}); auto write1 = sequencer.PopFrontWithName(); EXPECT_EQ(write1.second, "Write[1]"); - // 7. Clean up. write1.first.set_value(true); - - // 8. Satisfy Read[1.2] to let the read loop continue. - read2.first.set_value(true); - - // 9. *** FIX: *** - // Remove p_factory.set_value() and tested.reset(). - // The test will now end, destructors will run, and - // p_factory will be broken safely. + read2.first.set_value(true); } /// @test Verify an idle stream at the front is moved to the back and reused. TEST(ObjectDescriptorImpl, MakeSubsequentStreamReusesAndMovesIdleStream) { AsyncSequencer sequencer; - // Setup for stream1 + // First stream setup auto stream1 = std::make_unique(); EXPECT_CALL(*stream1, Write(_, _)) - .WillOnce([&](Request const&, auto) { // For reader1 + .WillOnce([&](Request const&, auto) { // For reader1 return sequencer.PushBack("Write[1.1]").then([](auto f) { return f.get(); }); }) - .WillOnce([&](Request const&, auto) { // For reader3 + .WillOnce([&](Request const&, auto) { // For reader3 return sequencer.PushBack("Write[1.2]").then([](auto f) { return f.get(); }); }); EXPECT_CALL(*stream1, Read) - .WillOnce([&] { // From Start() - return sequencer.PushBack("Read[1.1]").then( - [](auto) { return absl::make_optional(Response{}); }); + .WillOnce([&] { // From Start() + return sequencer.PushBack("Read[1.1]").then([](auto) { + return absl::make_optional(Response{}); + }); }) - .WillOnce([&] { // From OnRead() loop + .WillOnce([&] { // From OnRead() loop return sequencer.PushBack("Read[1.2]").then([](auto) { auto constexpr kResponse = R"pb( object_data_ranges { @@ -1495,18 +1471,17 @@ TEST(ObjectDescriptorImpl, MakeSubsequentStreamReusesAndMovesIdleStream) { return absl::make_optional(std::move(response)); }); }) - .WillOnce([&] { // From OnRead() loop after reader1 done - return sequencer.PushBack("Read[1.3]").then( - [](auto) { return absl::make_optional(Response{}); }); + .WillOnce([&] { // From OnRead() loop after reader1 done + return sequencer.PushBack("Read[1.3]").then([](auto) { + return absl::make_optional(Response{}); + }); }) - .WillRepeatedly([&] { - return promise>().get_future(); - }); - EXPECT_CALL(*stream1, Finish) - .WillOnce(Return(make_ready_future(Status{}))); + .WillRepeatedly( + [&] { return promise>().get_future(); }); + EXPECT_CALL(*stream1, Finish).WillOnce(Return(make_ready_future(Status{}))); EXPECT_CALL(*stream1, Cancel).Times(AtMost(1)); - // Setup for stream2 + // Second stream setup auto stream2 = std::make_unique(); EXPECT_CALL(*stream2, Write(_, _)).WillOnce([&](Request const&, auto) { return sequencer.PushBack("Write[2.1]").then([](auto f) { @@ -1515,99 +1490,87 @@ TEST(ObjectDescriptorImpl, MakeSubsequentStreamReusesAndMovesIdleStream) { }); EXPECT_CALL(*stream2, Read) .WillOnce([&] { - return sequencer.PushBack("Read[2.1]").then( - [](auto) { return absl::make_optional(Response{}); }); + return sequencer.PushBack("Read[2.1]").then([](auto) { + return absl::make_optional(Response{}); + }); }) - .WillRepeatedly([&] { - return promise>().get_future(); - }); - EXPECT_CALL(*stream2, Finish) - .WillOnce(Return(make_ready_future(Status{}))); + .WillRepeatedly( + [&] { return promise>().get_future(); }); + EXPECT_CALL(*stream2, Finish).WillOnce(Return(make_ready_future(Status{}))); EXPECT_CALL(*stream2, Cancel).Times(AtMost(1)); - // Setup for stream3 (The pending one that never gets used) + // Third stream setup auto stream3 = std::make_unique(); - // FIX: stream3 sits in pending_stream_ and is destroyed when the test ends. + // stream3 sits in pending_stream_ and is destroyed when the test ends. // It is never "started", so Finish() is never called. EXPECT_CALL(*stream3, Finish).Times(AtMost(1)); EXPECT_CALL(*stream3, Cancel).Times(AtMost(1)); + // Mock factory for subsequent streams MockFactory factory; - // Expect TWO factory calls. EXPECT_CALL(factory, Call) - .WillOnce([&] { // For stream2 (Triggered by Start) - return make_ready_future(StatusOr( - OpenStreamResult{std::make_shared(std::move(stream2)), - Response{}})); + .WillOnce([&] { // For stream2 (Triggered by Start) + return make_ready_future(StatusOr(OpenStreamResult{ + std::make_shared(std::move(stream2)), Response{}})); }) - .WillOnce([&] { // For stream3 (Triggered by MakeSubsequentStream #1) - return make_ready_future(StatusOr( - OpenStreamResult{std::make_shared(std::move(stream3)), - Response{}})); + .WillOnce([&] { // For stream3 (Triggered by MakeSubsequentStream) + return make_ready_future(StatusOr(OpenStreamResult{ + std::make_shared(std::move(stream3)), Response{}})); }); auto tested = std::make_shared( NoResume(), factory.AsStdFunction(), google::storage::v2::BidiReadObjectSpec{}, std::make_shared(std::move(stream1))); - + tested->Start(Response{}); - // 1. Pop Read[1.1] from Start() auto read1_1 = sequencer.PopFrontWithName(); EXPECT_EQ(read1_1.second, "Read[1.1]"); - // 2. Make stream1 busy + // Make stream1 busy auto reader1 = tested->Read({0, 100}); auto write1_1 = sequencer.PopFrontWithName(); EXPECT_EQ(write1_1.second, "Write[1.1]"); - write1_1.first.set_value(true); // Complete write immediately + write1_1.first.set_value(true); // Complete write immediately - // 3. Create stream2. This call is non-blocking. - // This will consume the pending stream (stream2) and queue stream3. + // Create and switch to a new stream. This happens before the first + // stream is finished. tested->MakeSubsequentStream(); - + auto read2_1 = sequencer.PopFrontWithName(); EXPECT_EQ(read2_1.second, "Read[2.1]"); - // 4. Make stream2 busy + // Make stream2 busy auto reader2 = tested->Read({100, 200}); auto write2_1 = sequencer.PopFrontWithName(); EXPECT_EQ(write2_1.second, "Write[2.1]"); - write2_1.first.set_value(true); // Complete write immediately - // At this point: streams_ = {stream1 (busy), stream2 (busy)} + write2_1.first.set_value(true); // Complete write immediately - // 5. Now, make stream1 IDLE. + // Make stream1 IDLE. auto r1f1 = reader1->Read(); - read1_1.first.set_value(true); // Complete Read[1.1] (empty) - auto read1_2 = sequencer.PopFrontWithName(); // Pop Read[1.2] (with data) + read1_1.first.set_value(true); + auto read1_2 = sequencer.PopFrontWithName(); EXPECT_EQ(read1_2.second, "Read[1.2]"); - read1_2.first.set_value(true); // Complete Read[1.2] + read1_2.first.set_value(true); ASSERT_THAT(r1f1.get(), VariantWith(_)); auto r1f2 = reader1->Read(); ASSERT_THAT(r1f2.get(), VariantWith(IsOk())); - // 6. Wait for the cleanup and next read loop auto read1_3 = sequencer.PopFrontWithName(); EXPECT_EQ(read1_3.second, "Read[1.3]"); - // Now: stream1 is IDLE. streams_ = {stream1 (idle), stream2 (busy)} - // 7. Call MakeSubsequentStream. It finds stream1, moves it, and returns. - // It does NOT consume the pending stream (stream3). + // Call MakeSubsequentStream. It finds stream1, moves it, and returns. tested->MakeSubsequentStream(); - // 8. Start a new read. It should be routed to stream1 (now at the back). auto reader3 = tested->Read({200, 300}); auto write1_2 = sequencer.PopFrontWithName(); EXPECT_EQ(write1_2.second, "Write[1.2]"); - // 9. Clean up write1_2.first.set_value(true); read1_3.first.set_value(true); read2_1.first.set_value(true); - - // Explicitly reset to trigger destructors while expectations are active - tested.reset(); + tested.reset(); } } // namespace diff --git a/google/cloud/storage/tests/async_client_integration_test.cc b/google/cloud/storage/tests/async_client_integration_test.cc index fed96694e768d..7424a2de485d4 100644 --- a/google/cloud/storage/tests/async_client_integration_test.cc +++ b/google/cloud/storage/tests/async_client_integration_test.cc @@ -1005,7 +1005,7 @@ TEST_F(AsyncClientIntegrationTest, Open) { } TEST_F(AsyncClientIntegrationTest, OpenExceedMaximumRange) { - GTEST_SKIP(); + if (!UsingEmulator()) GTEST_SKIP(); auto async = AsyncClient( TestOptions().set(1024)); auto client = MakeIntegrationTestClient(true, TestOptions()); From 4fdf57ee7a124adbeefae727bc090326275aa87d Mon Sep 17 00:00:00 2001 From: Shubham Kaushal Date: Fri, 12 Dec 2025 05:42:11 +0000 Subject: [PATCH 03/12] fixing clang-tidy --- .../async/connection_impl_open_test.cc | 1 - .../async/multi_stream_manager_test.cc | 28 +++++++++---------- .../internal/async/object_descriptor_impl.cc | 5 ++-- .../async/object_descriptor_impl_test.cc | 1 - 4 files changed, 17 insertions(+), 18 deletions(-) diff --git a/google/cloud/storage/internal/async/connection_impl_open_test.cc b/google/cloud/storage/internal/async/connection_impl_open_test.cc index a8396d612bcd6..e7849291cf5e8 100644 --- a/google/cloud/storage/internal/async/connection_impl_open_test.cc +++ b/google/cloud/storage/internal/async/connection_impl_open_test.cc @@ -48,7 +48,6 @@ using ::testing::InvokeWithoutArgs; using ::testing::NiceMock; using ::testing::NotNull; using ::testing::Optional; -using ::testing::Return; using BidiReadStream = google::cloud::AsyncStreamingReadWriteRpc< google::storage::v2::BidiReadObjectRequest, diff --git a/google/cloud/storage/internal/async/multi_stream_manager_test.cc b/google/cloud/storage/internal/async/multi_stream_manager_test.cc index a4a6bde72e5dc..95a4853c3298e 100644 --- a/google/cloud/storage/internal/async/multi_stream_manager_test.cc +++ b/google/cloud/storage/internal/async/multi_stream_manager_test.cc @@ -49,7 +49,7 @@ struct MultiStreamManagerTest : public ::testing::Test { TEST(MultiStreamManagerTest, ConstructsWithFactoryAndHasOneStream) { auto mgr = MultiStreamManagerTest::MakeManager(); EXPECT_FALSE(mgr.Empty()); - EXPECT_EQ(mgr.Size(), 1u); + EXPECT_EQ(mgr.Size(), 1U); auto it = mgr.GetLastStream(); ASSERT_TRUE(it->stream); } @@ -57,7 +57,7 @@ TEST(MultiStreamManagerTest, ConstructsWithFactoryAndHasOneStream) { TEST(MultiStreamManagerTest, ConstructsWithInitialStream) { auto initial = std::make_shared(); Manager mgr([] { return nullptr; }, initial); - EXPECT_EQ(mgr.Size(), 1u); + EXPECT_EQ(mgr.Size(), 1U); auto it = mgr.GetLastStream(); EXPECT_EQ(it->stream, initial); } @@ -66,7 +66,7 @@ TEST(MultiStreamManagerTest, AddStreamAppendsAndGetLastReturnsNew) { auto mgr = MultiStreamManagerTest::MakeManager(); auto s1 = std::make_shared(); auto it1 = mgr.AddStream(s1); - EXPECT_EQ(mgr.Size(), 2u); + EXPECT_EQ(mgr.Size(), 2U); EXPECT_EQ(it1->stream.get(), s1.get()); auto it_last = mgr.GetLastStream(); EXPECT_EQ(it_last->stream.get(), s1.get()); @@ -98,7 +98,7 @@ TEST(MultiStreamManagerTest, GetLeastBusyPrefersFewestActiveRanges) { // Expect it2 (1 range) over it1 (2 ranges) and it_init (2 ranges). EXPECT_EQ(it_least, it2); - EXPECT_EQ(it_least->active_ranges.size(), 1u); + EXPECT_EQ(it_least->active_ranges.size(), 1U); } TEST(MultiStreamManagerTest, CleanupDoneRangesRemovesFinished) { @@ -114,7 +114,7 @@ TEST(MultiStreamManagerTest, CleanupDoneRangesRemovesFinished) { it->active_ranges.emplace(2, r2); it->active_ranges.emplace(3, r3); mgr.CleanupDoneRanges(it); - EXPECT_EQ(it->active_ranges.size(), 1u); + EXPECT_EQ(it->active_ranges.size(), 1U); EXPECT_TRUE(it->active_ranges.count(1)); } @@ -126,7 +126,7 @@ TEST(MultiStreamManagerTest, RemoveStreamAndNotifyRangesCallsOnFinish) { it->active_ranges.emplace(11, r1); it->active_ranges.emplace(22, r2); mgr.RemoveStreamAndNotifyRanges(it, Status()); // OK status - EXPECT_EQ(mgr.Size(), 0u); + EXPECT_EQ(mgr.Size(), 0U); EXPECT_EQ(r1->finished, 1); EXPECT_EQ(r2->finished, 1); } @@ -145,11 +145,11 @@ TEST(MultiStreamManagerTest, CancelAllInvokesCancel) { TEST(MultiStreamManagerTest, ReuseIdleStreamToBackMovesElement) { auto mgr = MultiStreamManagerTest::MakeManager(); // Capture the factory-created stream pointer (initial element) - auto factory_ptr = mgr.GetLastStream()->stream.get(); + auto* factory_ptr = mgr.GetLastStream()->stream.get(); auto s1 = std::make_shared(); mgr.AddStream(s1); bool moved = mgr.ReuseIdleStreamToBack([](Manager::Stream const& s) { - auto fs = s.stream.get(); + auto* fs = s.stream.get(); return fs != nullptr && s.active_ranges.empty() && !fs->write_pending; }); EXPECT_TRUE(moved); @@ -179,7 +179,7 @@ TEST(MultiStreamManagerTest, ReuseIdleStreamDoesNotMoveWhenWritePending) { s1->write_pending = true; // also mark appended stream as not reusable mgr.AddStream(s1); bool moved = mgr.ReuseIdleStreamToBack([](Manager::Stream const& s) { - auto fs = s.stream.get(); + auto* fs = s.stream.get(); return fs != nullptr && s.active_ranges.empty() && !fs->write_pending; }); EXPECT_FALSE(moved); @@ -195,11 +195,11 @@ TEST(MultiStreamManagerTest, MoveActiveRangesTransfersAllEntries) { auto it2 = mgr.AddStream(s2); it1->active_ranges.emplace(101, std::make_shared()); it1->active_ranges.emplace(202, std::make_shared()); - ASSERT_EQ(it1->active_ranges.size(), 2u); + ASSERT_EQ(it1->active_ranges.size(), 2U); ASSERT_TRUE(it2->active_ranges.empty()); mgr.MoveActiveRanges(it1, it2); EXPECT_TRUE(it1->active_ranges.empty()); - EXPECT_EQ(it2->active_ranges.size(), 2u); + EXPECT_EQ(it2->active_ranges.size(), 2U); EXPECT_TRUE(it2->active_ranges.count(101)); EXPECT_TRUE(it2->active_ranges.count(202)); } @@ -220,15 +220,15 @@ TEST(MultiStreamManagerTest, GetLastStreamReflectsRecentAppendAndReuse) { TEST(MultiStreamManagerTest, EmptyAndSizeTransitions) { auto mgr = MultiStreamManagerTest::MakeManager(); EXPECT_FALSE(mgr.Empty()); - EXPECT_EQ(mgr.Size(), 1u); + EXPECT_EQ(mgr.Size(), 1U); auto it = mgr.GetLastStream(); mgr.RemoveStreamAndNotifyRanges(it, Status()); EXPECT_TRUE(mgr.Empty()); - EXPECT_EQ(mgr.Size(), 0u); + EXPECT_EQ(mgr.Size(), 0U); auto s = std::make_shared(); mgr.AddStream(s); EXPECT_FALSE(mgr.Empty()); - EXPECT_EQ(mgr.Size(), 1u); + EXPECT_EQ(mgr.Size(), 1U); } GOOGLE_CLOUD_CPP_INLINE_NAMESPACE_END diff --git a/google/cloud/storage/internal/async/object_descriptor_impl.cc b/google/cloud/storage/internal/async/object_descriptor_impl.cc index c0b96b749b536..81aae76ba8a5a 100644 --- a/google/cloud/storage/internal/async/object_descriptor_impl.cc +++ b/google/cloud/storage/internal/async/object_descriptor_impl.cc @@ -86,7 +86,8 @@ void ObjectDescriptorImpl::MakeSubsequentStream() { if (stream_manager_->ReuseIdleStreamToBack( [](StreamManager::Stream const& s) { auto const* rs = s.stream.get(); - return rs && s.active_ranges.empty() && !rs->write_pending; + return rs != nullptr && s.active_ranges.empty() && + !rs->write_pending; })) { return; } @@ -300,7 +301,7 @@ bool ObjectDescriptorImpl::IsResumable( std::vector> notify; for (auto const& re : error.read_range_errors()) { - if (it->active_ranges.count(re.read_id())) { + if (it->active_ranges.count(re.read_id()) != 0) { notify.emplace_back(re.read_id(), MakeStatusFromRpcError(re.status())); } } diff --git a/google/cloud/storage/internal/async/object_descriptor_impl_test.cc b/google/cloud/storage/internal/async/object_descriptor_impl_test.cc index fd354b3fbdb87..3f1267c958c6a 100644 --- a/google/cloud/storage/internal/async/object_descriptor_impl_test.cc +++ b/google/cloud/storage/internal/async/object_descriptor_impl_test.cc @@ -49,7 +49,6 @@ using ::testing::NotNull; using ::testing::Optional; using ::testing::ResultOf; using ::testing::Return; -using ::testing::UnorderedElementsAre; using ::testing::VariantWith; using Request = google::storage::v2::BidiReadObjectRequest; From e5a139aed6ee0b7d66365e671d5eee17505a8b8f Mon Sep 17 00:00:00 2001 From: Shubham Kaushal Date: Fri, 12 Dec 2025 09:19:06 +0000 Subject: [PATCH 04/12] add tests --- .../internal/async/object_descriptor_impl.cc | 43 +++-- .../async/object_descriptor_impl_test.cc | 172 ++++++++++++++++++ 2 files changed, 198 insertions(+), 17 deletions(-) diff --git a/google/cloud/storage/internal/async/object_descriptor_impl.cc b/google/cloud/storage/internal/async/object_descriptor_impl.cc index 81aae76ba8a5a..0136c85993a3c 100644 --- a/google/cloud/storage/internal/async/object_descriptor_impl.cc +++ b/google/cloud/storage/internal/async/object_descriptor_impl.cc @@ -40,7 +40,7 @@ ObjectDescriptorImpl::ObjectDescriptorImpl( read_object_spec_(std::move(read_object_spec)), options_(std::move(options)) { stream_manager_ = std::make_unique( - []() -> std::shared_ptr { return nullptr; }, + []() -> std::shared_ptr { return nullptr; }, // NOLINT std::make_shared(std::move(stream), resume_policy_prototype_->clone())); } @@ -96,24 +96,33 @@ void ObjectDescriptorImpl::MakeSubsequentStream() { auto stream_future = std::move(pending_stream_); lk.unlock(); - // Wait for the stream to be created. - auto stream_result = stream_future.get(); - if (!stream_result) { - // Stream creation failed. - // The next call to AssurePendingStreamQueued will retry creation. - return; - } + // Use .then() to retrieves the result without blocking. + stream_future.then([w = WeakFromThis()](auto f) { + auto self = w.lock(); + if (!self) return; - lk.lock(); - if (cancelled_) return; - auto read_stream = std::make_shared( - std::move(stream_result->stream), resume_policy_prototype_->clone()); - auto new_it = stream_manager_->AddStream(std::move(read_stream)); - // Now that we consumed pending_stream_, queue the next one immediately. - AssurePendingStreamQueued(); + auto stream_result = f.get(); + if (!stream_result) { + // Stream creation failed. + // The next call to AssurePendingStreamQueued will retry creation. + return; + } - lk.unlock(); - OnRead(new_it, std::move(stream_result->first_response)); + std::unique_lock lk(self->mu_); + if (self->cancelled_) return; + + auto read_stream = + std::make_shared(std::move(stream_result->stream), + self->resume_policy_prototype_->clone()); + + auto new_it = self->stream_manager_->AddStream(std::move(read_stream)); + + // Now that we consumed pending_stream_, queue the next one immediately. + self->AssurePendingStreamQueued(); + + lk.unlock(); + self->OnRead(new_it, std::move(stream_result->first_response)); + }); } std::unique_ptr diff --git a/google/cloud/storage/internal/async/object_descriptor_impl_test.cc b/google/cloud/storage/internal/async/object_descriptor_impl_test.cc index 3f1267c958c6a..6132eb16b6076 100644 --- a/google/cloud/storage/internal/async/object_descriptor_impl_test.cc +++ b/google/cloud/storage/internal/async/object_descriptor_impl_test.cc @@ -1572,6 +1572,178 @@ TEST(ObjectDescriptorImpl, MakeSubsequentStreamReusesAndMovesIdleStream) { tested.reset(); } +/// @test Verify that a successful resume executes the OnResume logic correctly. +TEST(ObjectDescriptorImpl, OnResumeSuccessful) { + AsyncSequencer sequencer; + + auto expect_startup_events = [&](AsyncSequencer& seq) { + auto e1 = seq.PopFrontWithName(); + auto e2 = seq.PopFrontWithName(); + std::set names = {e1.second, e2.second}; + if (names.count("Read[1]") && names.count("ProactiveFactory")) { + e1.first.set_value(true); // Allow read to proceed + e2.first.set_value(true); // Allow factory to proceed + } else { + ADD_FAILURE() << "Got unexpected events: " << e1.second << ", " + << e2.second; + } + }; + + auto stream1 = std::make_unique(); + EXPECT_CALL(*stream1, Write).WillOnce([&](auto, auto) { + return sequencer.PushBack("Write[1]").then([](auto f) { return f.get(); }); + }); + + // To keep Stream 1 alive during startup, the first Read returns a valid + // (empty) response. Subsequent reads return nullopt to trigger the + // Finish/Resume logic. + EXPECT_CALL(*stream1, Read) + .WillOnce([&] { + return sequencer.PushBack("Read[1]").then( + [](auto) { return absl::make_optional(Response{}); }); + }) + .WillRepeatedly([&] { + return sequencer.PushBack("Read[Loop]").then([](auto) { + return absl::optional{}; + }); + }); + + EXPECT_CALL(*stream1, Finish).WillOnce([&] { + return sequencer.PushBack("Finish[1]").then([](auto) { + return TransientError(); + }); + }); + EXPECT_CALL(*stream1, Cancel).Times(AtMost(1)); + + auto stream2 = std::make_unique(); + // The resumed stream to starts reading immediately. + EXPECT_CALL(*stream2, Read).WillRepeatedly([&] { + return sequencer.PushBack("Read[2]").then( + [](auto) { return absl::make_optional(Response{}); }); + }); + EXPECT_CALL(*stream2, Finish).WillOnce(Return(make_ready_future(Status{}))); + EXPECT_CALL(*stream2, Cancel).Times(AtMost(1)); + + MockFactory factory; + EXPECT_CALL(factory, Call) + .WillOnce([&](auto) { + return sequencer.PushBack("ProactiveFactory").then([](auto) { + return StatusOr(TransientError()); + }); + }) + .WillOnce([&](auto) { + return sequencer.PushBack("ResumeFactory").then([&](auto) { + return StatusOr(OpenStreamResult{ + std::make_shared(std::move(stream2)), Response{}}); + }); + }); + + auto tested = std::make_shared( + storage_experimental::LimitedErrorCountResumePolicy(1)(), + factory.AsStdFunction(), google::storage::v2::BidiReadObjectSpec{}, + std::make_shared(std::move(stream1))); + + tested->Start(Response{}); + expect_startup_events(sequencer); + + // Register the read range. + auto reader = tested->Read({0, 100}); + + auto next_event = sequencer.PopFrontWithName(); + promise fail_stream_promise; + + if (next_event.second == "Read[Loop]") { + fail_stream_promise = std::move(next_event.first); + // Now expect Write[1] + auto w1 = sequencer.PopFrontWithName(); + EXPECT_EQ(w1.second, "Write[1]"); + w1.first.set_value(true); + } else { + // It was Write[1] immediately + EXPECT_EQ(next_event.second, "Write[1]"); + next_event.first.set_value(true); + + // Now wait for Read[Loop] + auto read_loop = sequencer.PopFrontWithName(); + EXPECT_EQ(read_loop.second, "Read[Loop]"); + fail_stream_promise = std::move(read_loop.first); + } + + // Trigger Failure on Stream 1. + fail_stream_promise.set_value(true); + + auto f1 = sequencer.PopFrontWithName(); + EXPECT_EQ(f1.second, "Finish[1]"); + f1.first.set_value(true); + + auto resume = sequencer.PopFrontWithName(); + EXPECT_EQ(resume.second, "ResumeFactory"); + resume.first.set_value(true); + + // The OnResume block calls OnRead, which triggers Read() on Stream 2. + auto r2 = sequencer.PopFrontWithName(); + EXPECT_EQ(r2.second, "Read[2]"); + r2.first.set_value(true); + + tested.reset(); +} + +/// @test Verify Read() behavior when all streams have failed permanently. +TEST(ObjectDescriptorImpl, ReadFailsWhenAllStreamsAreDead) { + AsyncSequencer sequencer; + auto stream = std::make_unique(); + + // Initial Read returns empty (EOF) to trigger Finish + EXPECT_CALL(*stream, Read).WillOnce([&] { + return sequencer.PushBack("Read[1]").then( + [](auto) { return absl::optional{}; }); + }); + + EXPECT_CALL(*stream, Finish).WillOnce([&] { + return sequencer.PushBack("Finish").then( + [](auto) { return PermanentError(); }); + }); + EXPECT_CALL(*stream, Cancel).Times(AtMost(1)); + + MockFactory factory; + EXPECT_CALL(factory, Call).WillOnce([&](auto) { + return sequencer.PushBack("ProactiveFactory").then([](auto) { + return StatusOr(PermanentError()); + }); + }); + + auto tested = std::make_shared( + NoResume(), factory.AsStdFunction(), + google::storage::v2::BidiReadObjectSpec{}, + std::make_shared(std::move(stream))); + + tested->Start(Response{}); + + auto e1 = sequencer.PopFrontWithName(); + auto e2 = sequencer.PopFrontWithName(); + + std::set names = {e1.second, e2.second}; + ASSERT_EQ(names.count("Read[1]"), 1); + ASSERT_EQ(names.count("ProactiveFactory"), 1); + + e1.first.set_value(true); + e2.first.set_value(true); + + auto finish = sequencer.PopFrontWithName(); + EXPECT_EQ(finish.second, "Finish"); + finish.first.set_value(true); + + // At this point, the only active stream failed permanently and was removed. + // The proactive stream creation also failed. The manager is now EMPTY. + + auto reader = tested->Read({0, 100}); + + // The Read() should immediately fail with FailedPrecondition. + auto result = reader->Read().get(); + EXPECT_THAT(result, + VariantWith(StatusIs(StatusCode::kFailedPrecondition))); +} + } // namespace GOOGLE_CLOUD_CPP_INLINE_NAMESPACE_END } // namespace storage_internal From 0b4b95f4b29fe8ac6161fe5f67db110cc2286d56 Mon Sep 17 00:00:00 2001 From: Shubham Kaushal Date: Sun, 14 Dec 2025 07:06:39 +0000 Subject: [PATCH 05/12] fixing tests --- .../storage/internal/async/connection_impl_open_test.cc | 6 ++++++ .../cloud/storage/internal/async/object_descriptor_impl.cc | 1 + .../storage/internal/async/object_descriptor_impl_test.cc | 2 +- 3 files changed, 8 insertions(+), 1 deletion(-) diff --git a/google/cloud/storage/internal/async/connection_impl_open_test.cc b/google/cloud/storage/internal/async/connection_impl_open_test.cc index e7849291cf5e8..c1902d8ba92e7 100644 --- a/google/cloud/storage/internal/async/connection_impl_open_test.cc +++ b/google/cloud/storage/internal/async/connection_impl_open_test.cc @@ -202,6 +202,12 @@ TEST(AsyncConnectionImplTest, OpenSimple) { }); auto mock_cq = std::make_shared(); + EXPECT_CALL(*mock_cq, MakeRelativeTimer) + .WillRepeatedly([](std::chrono::nanoseconds) { + return make_ready_future( + StatusOr( + std::chrono::system_clock::now())); + }); auto connection = std::make_shared( CompletionQueue(mock_cq), std::shared_ptr(), mock, TestOptions()); diff --git a/google/cloud/storage/internal/async/object_descriptor_impl.cc b/google/cloud/storage/internal/async/object_descriptor_impl.cc index 0136c85993a3c..14b5839598149 100644 --- a/google/cloud/storage/internal/async/object_descriptor_impl.cc +++ b/google/cloud/storage/internal/async/object_descriptor_impl.cc @@ -93,6 +93,7 @@ void ObjectDescriptorImpl::MakeSubsequentStream() { } // Proactively create a new stream if needed. AssurePendingStreamQueued(); + if (!pending_stream_.valid()) return; auto stream_future = std::move(pending_stream_); lk.unlock(); diff --git a/google/cloud/storage/internal/async/object_descriptor_impl_test.cc b/google/cloud/storage/internal/async/object_descriptor_impl_test.cc index 6132eb16b6076..0309ccc6c947b 100644 --- a/google/cloud/storage/internal/async/object_descriptor_impl_test.cc +++ b/google/cloud/storage/internal/async/object_descriptor_impl_test.cc @@ -1580,7 +1580,7 @@ TEST(ObjectDescriptorImpl, OnResumeSuccessful) { auto e1 = seq.PopFrontWithName(); auto e2 = seq.PopFrontWithName(); std::set names = {e1.second, e2.second}; - if (names.count("Read[1]") && names.count("ProactiveFactory")) { + if (names.count("Read[1]") != 0 && names.count("ProactiveFactory") != 0) { e1.first.set_value(true); // Allow read to proceed e2.first.set_value(true); // Allow factory to proceed } else { From ee681cdb433bb7f58ca43ff1747fa05a2d967f9f Mon Sep 17 00:00:00 2001 From: Shubham Kaushal Date: Sun, 14 Dec 2025 16:35:56 +0000 Subject: [PATCH 06/12] fixing tests --- .../async/object_descriptor_impl_test.cc | 217 ++++++++++-------- 1 file changed, 123 insertions(+), 94 deletions(-) diff --git a/google/cloud/storage/internal/async/object_descriptor_impl_test.cc b/google/cloud/storage/internal/async/object_descriptor_impl_test.cc index 0309ccc6c947b..49eca7630a9ae 100644 --- a/google/cloud/storage/internal/async/object_descriptor_impl_test.cc +++ b/google/cloud/storage/internal/async/object_descriptor_impl_test.cc @@ -1321,35 +1321,42 @@ TEST(ObjectDescriptorImpl, MakeSubsequentStreamCreatesNewWhenAllBusy) { // Setup for the first stream, which will remain busy. auto stream1 = std::make_unique(); - EXPECT_CALL(*stream1, Write(_, _)).WillOnce([&](Request const&, auto) { - return sequencer.PushBack("Write[1]").then([](auto f) { return f.get(); }); + EXPECT_CALL(*stream1, Write(_, _)) + .Times(::testing::AtMost(1)) + .WillRepeatedly( + [](Request const&, auto) { return make_ready_future(true); }); + // Keep stream1 busy but return ready futures. + EXPECT_CALL(*stream1, Read).WillRepeatedly([] { + return make_ready_future(absl::optional{}); }); - // No Read() expectations, to keep it busy. - EXPECT_CALL(*stream1, Read).WillRepeatedly([&] { - return sequencer.PushBack("Read[1]").then( - [](auto) { return absl::optional{}; }); + EXPECT_CALL(*stream1, Finish).WillOnce([] { + return make_ready_future(Status{}); }); - EXPECT_CALL(*stream1, Finish).Times(AtMost(1)); EXPECT_CALL(*stream1, Cancel).Times(AtMost(1)); // Setup for the second stream, which will be created proactively. auto stream2 = std::make_unique(); - EXPECT_CALL(*stream2, Write(_, _)).WillOnce([&](Request const&, auto) { - return sequencer.PushBack("Write[2]").then([](auto f) { return f.get(); }); + EXPECT_CALL(*stream2, Write(_, _)) + .Times(::testing::AtMost(1)) + .WillRepeatedly([](...) { return make_ready_future(true); }); + EXPECT_CALL(*stream2, Read).WillRepeatedly([] { + return make_ready_future(absl::optional{}); }); - EXPECT_CALL(*stream2, Read).WillRepeatedly([&] { - return sequencer.PushBack("Read[2]").then( - [](auto) { return absl::optional{}; }); + EXPECT_CALL(*stream2, Finish).WillRepeatedly([] { + return make_ready_future(Status{}); }); - EXPECT_CALL(*stream2, Finish).Times(AtMost(1)); EXPECT_CALL(*stream2, Cancel).Times(AtMost(1)); - MockFactory factory; // First call is proactive for stream2. It may be called more than once. - EXPECT_CALL(factory, Call).WillRepeatedly([&](Request const&) { - return make_ready_future(StatusOr(OpenStreamResult{ - std::make_shared(std::move(stream2)), Response{}})); - }); + MockFactory factory; + EXPECT_CALL(factory, Call) + .WillOnce([&](Request const&) { + return make_ready_future(StatusOr(OpenStreamResult{ + std::make_shared(std::move(stream2)), Response{}})); + }) + .WillRepeatedly([](Request const&) { + return make_ready_future(StatusOr(PermanentError())); + }); auto tested = std::make_shared( NoResume(), factory.AsStdFunction(), @@ -1359,7 +1366,6 @@ TEST(ObjectDescriptorImpl, MakeSubsequentStreamCreatesNewWhenAllBusy) { // Start a read on stream1 to make it busy. auto reader1 = tested->Read({0, 100}); - sequencer.PopFrontWithName().first.set_value(true); // Call MakeSubsequentStream. Since stream1 is busy, this should activate the // pending stream (stream2). @@ -1367,7 +1373,10 @@ TEST(ObjectDescriptorImpl, MakeSubsequentStreamCreatesNewWhenAllBusy) { // A new read should now be routed to stream2. auto reader2 = tested->Read({100, 200}); - sequencer.PopFrontWithName().first.set_value(true); + + // stream2 returns ready futures; no sequencer pop needed here. + tested->Cancel(); + tested.reset(); } /// @test Verify reusing an idle stream that is already last is a no-op. @@ -1376,14 +1385,20 @@ TEST(ObjectDescriptorImpl, MakeSubsequentStreamReusesIdleStreamAlreadyLast) { // Setup for the first and only stream. auto stream1 = std::make_unique(); - EXPECT_CALL(*stream1, Write(_, _)).WillOnce([&](Request const&, auto) { - return sequencer.PushBack("Write[1]").then([](auto f) { return f.get(); }); - }); + EXPECT_CALL(*stream1, Write(_, _)) + .Times(AtMost(1)) + .WillRepeatedly( + [](auto const&, auto) { return make_ready_future(true); }); EXPECT_CALL(*stream1, Read) .WillOnce([&] { // From Start() return sequencer.PushBack("Read[1.1]").then([](auto) { - return absl::make_optional(Response{}); + auto resp = Response{}; + auto* r = resp.add_object_data_ranges(); + r->set_range_end(true); + r->mutable_read_range()->set_read_id(0); + r->mutable_read_range()->set_read_offset(0); + return absl::make_optional(std::move(resp)); }); }) .WillOnce([&] { // From OnRead() loop @@ -1391,46 +1406,55 @@ TEST(ObjectDescriptorImpl, MakeSubsequentStreamReusesIdleStreamAlreadyLast) { return absl::make_optional(Response{}); }); }) - .WillRepeatedly([&] { // Subsequent loop calls - return promise>().get_future(); - }); + .WillOnce([] { // Complete read_id=1 + auto resp = Response{}; + auto* r = resp.add_object_data_ranges(); + r->set_range_end(true); + r->mutable_read_range()->set_read_id(1); + r->mutable_read_range()->set_read_offset(0); + return make_ready_future(absl::make_optional(std::move(resp))); + }) + .WillRepeatedly( + [] { return make_ready_future(absl::optional{}); }); // Finish() will be called by the OpenStream destructor. EXPECT_CALL(*stream1, Finish).WillOnce(Return(make_ready_future(Status{}))); EXPECT_CALL(*stream1, Cancel) .Times(AtMost(1)); // Always called by OpenStream - // We no longer need a 'stream2' mock. The factory will be - // called, but the resulting promise/future will just be - // abandoned when the test ends, which is correct. MockFactory factory; - promise> p_factory; - EXPECT_CALL(factory, Call).WillOnce(Return(p_factory.get_future())); + EXPECT_CALL(factory, Call) + .WillOnce([](Request const&) { + return make_ready_future(StatusOr(PermanentError())); + }) + .WillRepeatedly([](Request const&) { + return make_ready_future(StatusOr(PermanentError())); + }); auto tested = std::make_shared( NoResume(), factory.AsStdFunction(), google::storage::v2::BidiReadObjectSpec{}, std::make_shared(std::move(stream1))); - tested->Start(Response{}); + auto start_response = Response{}; + EXPECT_TRUE(TextFormat::ParseFromString( + R"pb(object_data_ranges { read_range { read_id: 0 read_offset: 0 } })pb", + &start_response)); + tested->Start(std::move(start_response)); auto read1 = sequencer.PopFrontWithName(); EXPECT_EQ(read1.second, "Read[1.1]"); - read1.first.set_value(true); auto read2 = sequencer.PopFrontWithName(); EXPECT_EQ(read2.second, "Read[1.2]"); + read2.first.set_value(true); // Call MakeSubsequentStream. It should find stream1 and return. tested->MakeSubsequentStream(); - // Start a new read. It should be routed to stream1. - auto reader = tested->Read({0, 100}); - auto write1 = sequencer.PopFrontWithName(); - EXPECT_EQ(write1.second, "Write[1]"); - - write1.first.set_value(true); - read2.first.set_value(true); + // Ensure background activity stops cleanly. + tested->Cancel(); + tested.reset(); } /// @test Verify an idle stream at the front is moved to the back and reused. @@ -1439,17 +1463,9 @@ TEST(ObjectDescriptorImpl, MakeSubsequentStreamReusesAndMovesIdleStream) { // First stream setup auto stream1 = std::make_unique(); - EXPECT_CALL(*stream1, Write(_, _)) - .WillOnce([&](Request const&, auto) { // For reader1 - return sequencer.PushBack("Write[1.1]").then([](auto f) { - return f.get(); - }); - }) - .WillOnce([&](Request const&, auto) { // For reader3 - return sequencer.PushBack("Write[1.2]").then([](auto f) { - return f.get(); - }); - }); + EXPECT_CALL(*stream1, Write(_, _)).WillRepeatedly([](auto const&, auto) { + return make_ready_future(true); + }); EXPECT_CALL(*stream1, Read) .WillOnce([&] { // From Start() @@ -1457,64 +1473,86 @@ TEST(ObjectDescriptorImpl, MakeSubsequentStreamReusesAndMovesIdleStream) { return absl::make_optional(Response{}); }); }) - .WillOnce([&] { // From OnRead() loop - return sequencer.PushBack("Read[1.2]").then([](auto) { - auto constexpr kResponse = R"pb( - object_data_ranges { - range_end: true - read_range { read_id: 1 read_offset: 0 } - } - )pb"; - auto response = Response{}; - EXPECT_TRUE(TextFormat::ParseFromString(kResponse, &response)); - return absl::make_optional(std::move(response)); - }); + .WillOnce([] { // From OnRead() loop + auto response = Response{}; + auto* r = response.add_object_data_ranges(); + r->set_range_end(true); + r->mutable_read_range()->set_read_id(1); + r->mutable_read_range()->set_read_offset(0); + return make_ready_future(absl::make_optional(std::move(response))); }) - .WillOnce([&] { // From OnRead() loop after reader1 done - return sequencer.PushBack("Read[1.3]").then([](auto) { - return absl::make_optional(Response{}); - }); + .WillOnce([] { // From OnRead() loop after reader1 done + return make_ready_future(absl::make_optional(Response{})); }) .WillRepeatedly( - [&] { return promise>().get_future(); }); + [] { return make_ready_future(absl::optional{}); }); EXPECT_CALL(*stream1, Finish).WillOnce(Return(make_ready_future(Status{}))); EXPECT_CALL(*stream1, Cancel).Times(AtMost(1)); // Second stream setup auto stream2 = std::make_unique(); - EXPECT_CALL(*stream2, Write(_, _)).WillOnce([&](Request const&, auto) { - return sequencer.PushBack("Write[2.1]").then([](auto f) { - return f.get(); - }); - }); + EXPECT_CALL(*stream2, Write(_, _)) + .WillOnce([&](Request const&, auto) { + return sequencer.PushBack("Write[2.1]").then([](auto f) { + return f.get(); + }); + }) + .WillRepeatedly( + [](auto const&, auto) { return make_ready_future(true); }); EXPECT_CALL(*stream2, Read) .WillOnce([&] { return sequencer.PushBack("Read[2.1]").then([](auto) { - return absl::make_optional(Response{}); + auto resp = Response{}; + auto* r = resp.add_object_data_ranges(); + r->set_range_end(true); + r->mutable_read_range()->set_read_id(2); + r->mutable_read_range()->set_read_offset(100); + return absl::make_optional(std::move(resp)); }); }) .WillRepeatedly( - [&] { return promise>().get_future(); }); + [] { return make_ready_future(absl::optional{}); }); EXPECT_CALL(*stream2, Finish).WillOnce(Return(make_ready_future(Status{}))); EXPECT_CALL(*stream2, Cancel).Times(AtMost(1)); // Third stream setup auto stream3 = std::make_unique(); + // stream3 sits in pending_stream_ and may become active; provide ready + // defaults. + EXPECT_CALL(*stream3, Write(_, _)).WillRepeatedly([](auto const&, auto) { + return make_ready_future(true); + }); + EXPECT_CALL(*stream3, Read).WillRepeatedly([] { + return make_ready_future(absl::optional{}); + }); // stream3 sits in pending_stream_ and is destroyed when the test ends. // It is never "started", so Finish() is never called. - EXPECT_CALL(*stream3, Finish).Times(AtMost(1)); + EXPECT_CALL(*stream3, Finish).Times(AtMost(1)).WillRepeatedly([] { + return make_ready_future(Status{}); + }); EXPECT_CALL(*stream3, Cancel).Times(AtMost(1)); // Mock factory for subsequent streams MockFactory factory; EXPECT_CALL(factory, Call) .WillOnce([&] { // For stream2 (Triggered by Start) - return make_ready_future(StatusOr(OpenStreamResult{ - std::make_shared(std::move(stream2)), Response{}})); + auto resp = Response{}; + EXPECT_TRUE( + TextFormat::ParseFromString( + R"pb(object_data_ranges { + read_range { read_id: 0 read_offset: 0 } + })pb", + &resp)); + return make_ready_future(StatusOr( + OpenStreamResult{std::make_shared(std::move(stream2)), + std::move(resp)})); }) .WillOnce([&] { // For stream3 (Triggered by MakeSubsequentStream) return make_ready_future(StatusOr(OpenStreamResult{ std::make_shared(std::move(stream3)), Response{}})); + }) + .WillRepeatedly([&](Request const&) { + return make_ready_future(StatusOr(PermanentError())); }); auto tested = std::make_shared( @@ -1522,16 +1560,17 @@ TEST(ObjectDescriptorImpl, MakeSubsequentStreamReusesAndMovesIdleStream) { google::storage::v2::BidiReadObjectSpec{}, std::make_shared(std::move(stream1))); - tested->Start(Response{}); + auto start_response = Response{}; + ASSERT_TRUE(TextFormat::ParseFromString( + R"pb(object_data_ranges { read_range { read_id: 0 read_offset: 0 } })pb", + &start_response)); + tested->Start(std::move(start_response)); auto read1_1 = sequencer.PopFrontWithName(); EXPECT_EQ(read1_1.second, "Read[1.1]"); // Make stream1 busy auto reader1 = tested->Read({0, 100}); - auto write1_1 = sequencer.PopFrontWithName(); - EXPECT_EQ(write1_1.second, "Write[1.1]"); - write1_1.first.set_value(true); // Complete write immediately // Create and switch to a new stream. This happens before the first // stream is finished. @@ -1549,25 +1588,15 @@ TEST(ObjectDescriptorImpl, MakeSubsequentStreamReusesAndMovesIdleStream) { // Make stream1 IDLE. auto r1f1 = reader1->Read(); read1_1.first.set_value(true); - auto read1_2 = sequencer.PopFrontWithName(); - EXPECT_EQ(read1_2.second, "Read[1.2]"); - read1_2.first.set_value(true); + // read1_2/read1_3 completed via ready futures; no sequencer pops needed. ASSERT_THAT(r1f1.get(), VariantWith(_)); auto r1f2 = reader1->Read(); ASSERT_THAT(r1f2.get(), VariantWith(IsOk())); - auto read1_3 = sequencer.PopFrontWithName(); - EXPECT_EQ(read1_3.second, "Read[1.3]"); - // Call MakeSubsequentStream. It finds stream1, moves it, and returns. tested->MakeSubsequentStream(); auto reader3 = tested->Read({200, 300}); - auto write1_2 = sequencer.PopFrontWithName(); - EXPECT_EQ(write1_2.second, "Write[1.2]"); - - write1_2.first.set_value(true); - read1_3.first.set_value(true); read2_1.first.set_value(true); tested.reset(); } From e73e1bcda673709b8f27383657ca657a0386ee73 Mon Sep 17 00:00:00 2001 From: Shubham Kaushal Date: Thu, 18 Dec 2025 07:17:22 +0000 Subject: [PATCH 07/12] resolving comments --- .../internal/async/multi_stream_manager.h | 28 +++++++++++++++---- .../internal/async/object_descriptor_impl.cc | 27 +++++++++--------- .../internal/async/object_descriptor_impl.h | 6 ++-- 3 files changed, 38 insertions(+), 23 deletions(-) diff --git a/google/cloud/storage/internal/async/multi_stream_manager.h b/google/cloud/storage/internal/async/multi_stream_manager.h index f3a781e8852fb..9c55367f6c343 100644 --- a/google/cloud/storage/internal/async/multi_stream_manager.h +++ b/google/cloud/storage/internal/async/multi_stream_manager.h @@ -29,6 +29,10 @@ namespace cloud { namespace storage_internal { GOOGLE_CLOUD_CPP_INLINE_NAMESPACE_BEGIN +// Defines the interface contract that any stream type (e.g., ReadStream, +// WriteStream) managed by MultiStreamManager must implement. This explicit base +// class ensures we have a clear, enforceable interface for operations like +// CancelAll(). class StreamBase { public: virtual ~StreamBase() = default; @@ -41,8 +45,20 @@ class StreamBase { // are moved to the back of the queue for reuse. // // THREAD SAFETY: -// This class is NOT thread-safe. The owner (ObjectDescriptorImpl) must -// serialize access, typically by holding `mu_` while calling these methods. +// This class is NOT thread-safe. The owner (e.g. ObjectDescriptorImpl +// or AsyncWriterImpl etc) must serialize access, typically by holding +// an external mutex while calling these methods. +// +// EXAMPLE USAGE: +// class MyOwner { +// std::mutex mu_; +// MultiStreamManager manager_; +// +// void StartRead() { +// std::unique_lock lk(mu_); +// auto it = manager_.GetLeastBusyStream(); +// } +// }; template class MultiStreamManager { public: @@ -81,20 +97,20 @@ class MultiStreamManager { // In ObjectDescriptorImpl, we ensure there is always at least one stream, // but this assertion protects against future refactoring errors. assert(!streams_.empty()); - auto best_it = streams_.begin(); + auto least_busy_it = streams_.begin(); // Track min_ranges to avoid calling .size() repeatedly if possible, // though for std::unordered_map .size() is O(1). - std::size_t min_ranges = best_it->active_ranges.size(); + std::size_t min_ranges = least_busy_it->active_ranges.size(); // Start checking from the second element for (auto it = std::next(streams_.begin()); it != streams_.end(); ++it) { // Strict less-than ensures stability (preferring older streams if tied) if (it->active_ranges.size() < min_ranges) { - best_it = it; + least_busy_it = it; min_ranges = it->active_ranges.size(); } } - return best_it; + return least_busy_it; } StreamIterator AddStream(std::shared_ptr stream) { diff --git a/google/cloud/storage/internal/async/object_descriptor_impl.cc b/google/cloud/storage/internal/async/object_descriptor_impl.cc index 14b5839598149..94089b8059f3f 100644 --- a/google/cloud/storage/internal/async/object_descriptor_impl.cc +++ b/google/cloud/storage/internal/async/object_descriptor_impl.cc @@ -51,7 +51,6 @@ void ObjectDescriptorImpl::Start( google::storage::v2::BidiReadObjectResponse first_response) { std::unique_lock lk(mu_); auto it = stream_manager_->GetLastStream(); - // Unlock and start the Read loop first. lk.unlock(); OnRead(it, std::move(first_response)); // Acquire lock and queue the background stream. @@ -87,7 +86,7 @@ void ObjectDescriptorImpl::MakeSubsequentStream() { [](StreamManager::Stream const& s) { auto const* rs = s.stream.get(); return rs != nullptr && s.active_ranges.empty() && - !rs->write_pending; + !rs->write_pending_; })) { return; } @@ -153,7 +152,7 @@ ObjectDescriptorImpl::Read(ReadParams p) { auto it = stream_manager_->GetLeastBusyStream(); auto const id = ++read_id_generator_; it->active_ranges.emplace(id, range); - auto& read_range = *it->stream->next_request.add_read_ranges(); + auto& read_range = *it->stream->next_request_.add_read_ranges(); read_range.set_read_id(id); read_range.set_read_offset(p.start); read_range.set_read_length(p.length); @@ -169,13 +168,13 @@ ObjectDescriptorImpl::Read(ReadParams p) { void ObjectDescriptorImpl::Flush(std::unique_lock lk, StreamIterator it) { - if (it->stream->write_pending || - it->stream->next_request.read_ranges().empty()) { + if (it->stream->write_pending_ || + it->stream->next_request_.read_ranges().empty()) { return; } - it->stream->write_pending = true; + it->stream->write_pending_ = true; google::storage::v2::BidiReadObjectRequest request; - request.Swap(&it->stream->next_request); + request.Swap(&it->stream->next_request_); // Assign CurrentStream to a temporary variable to prevent // lifetime extension which can cause the lock to be held until the @@ -191,14 +190,14 @@ void ObjectDescriptorImpl::Flush(std::unique_lock lk, void ObjectDescriptorImpl::OnWrite(StreamIterator it, bool ok) { std::unique_lock lk(mu_); if (!ok) return DoFinish(std::move(lk), it); - it->stream->write_pending = false; + it->stream->write_pending_ = false; Flush(std::move(lk), it); } void ObjectDescriptorImpl::DoRead(std::unique_lock lk, StreamIterator it) { - if (it->stream->read_pending) return; - it->stream->read_pending = true; + if (it->stream->read_pending_) return; + it->stream->read_pending_ = true; // Assign CurrentStream to a temporary variable to prevent // lifetime extension which can cause the lock to be held until the @@ -214,7 +213,7 @@ void ObjectDescriptorImpl::OnRead( StreamIterator it, absl::optional response) { std::unique_lock lk(mu_); - it->stream->read_pending = false; + it->stream->read_pending_ = false; if (!response) return DoFinish(std::move(lk), it); if (response->has_metadata()) { @@ -243,7 +242,7 @@ void ObjectDescriptorImpl::OnRead( void ObjectDescriptorImpl::DoFinish(std::unique_lock lk, StreamIterator it) { - it->stream->read_pending = false; + it->stream->read_pending_ = false; // Assign CurrentStream to a temporary variable to prevent // lifetime extension which can cause the lock to be held until the // end of the block. @@ -293,8 +292,8 @@ void ObjectDescriptorImpl::OnResume(StreamIterator it, it->stream = std::make_shared(std::move(result->stream), resume_policy_prototype_->clone()); - it->stream->write_pending = false; - it->stream->read_pending = false; + it->stream->write_pending_ = false; + it->stream->read_pending_ = false; // TODO(#15105) - this should be done without release the lock. Flush(std::move(lk), it); diff --git a/google/cloud/storage/internal/async/object_descriptor_impl.h b/google/cloud/storage/internal/async/object_descriptor_impl.h index 8c0f27b4404bf..36d26f485e977 100644 --- a/google/cloud/storage/internal/async/object_descriptor_impl.h +++ b/google/cloud/storage/internal/async/object_descriptor_impl.h @@ -48,9 +48,9 @@ class ReadStream : public storage_internal::StreamBase { std::shared_ptr stream_; std::unique_ptr resume_policy_; - google::storage::v2::BidiReadObjectRequest next_request; - bool write_pending = false; - bool read_pending = false; + google::storage::v2::BidiReadObjectRequest next_request_; + bool write_pending_ = false; + bool read_pending_ = false; }; class ObjectDescriptorImpl From bf8d810b9893b1e2b909da96d614c0e34e32bb2e Mon Sep 17 00:00:00 2001 From: Shubham Kaushal Date: Fri, 19 Dec 2025 07:24:10 +0000 Subject: [PATCH 08/12] resolving comments --- .../internal/async/multi_stream_manager.h | 12 ++--- .../internal/async/object_descriptor_impl.cc | 45 ++++++++++++------- .../internal/async/object_descriptor_impl.h | 25 +++++------ 3 files changed, 43 insertions(+), 39 deletions(-) diff --git a/google/cloud/storage/internal/async/multi_stream_manager.h b/google/cloud/storage/internal/async/multi_stream_manager.h index 9c55367f6c343..52aec213aa520 100644 --- a/google/cloud/storage/internal/async/multi_stream_manager.h +++ b/google/cloud/storage/internal/async/multi_stream_manager.h @@ -17,7 +17,6 @@ #include "google/cloud/status.h" #include "google/cloud/version.h" -#include #include #include #include @@ -85,18 +84,12 @@ class MultiStreamManager { } StreamIterator GetLastStream() { - // SAFETY: The caller must ensure the manager is not empty. - // In ObjectDescriptorImpl, we ensure there is always at least one stream, - // but this assertion protects against future refactoring errors. - assert(!streams_.empty()); + if (streams_.empty()) return streams_.end(); return std::prev(streams_.end()); } StreamIterator GetLeastBusyStream() { - // SAFETY: The caller must ensure the manager is not empty. - // In ObjectDescriptorImpl, we ensure there is always at least one stream, - // but this assertion protects against future refactoring errors. - assert(!streams_.empty()); + if (streams_.empty()) return streams_.end(); auto least_busy_it = streams_.begin(); // Track min_ranges to avoid calling .size() repeatedly if possible, // though for std::unordered_map .size() is O(1). @@ -165,6 +158,7 @@ class MultiStreamManager { } bool Empty() const { return streams_.empty(); } + StreamIterator End() { return streams_.end(); } std::size_t Size() const { return streams_.size(); } private: diff --git a/google/cloud/storage/internal/async/object_descriptor_impl.cc b/google/cloud/storage/internal/async/object_descriptor_impl.cc index 94089b8059f3f..4f8dcf9521e62 100644 --- a/google/cloud/storage/internal/async/object_descriptor_impl.cc +++ b/google/cloud/storage/internal/async/object_descriptor_impl.cc @@ -51,6 +51,7 @@ void ObjectDescriptorImpl::Start( google::storage::v2::BidiReadObjectResponse first_response) { std::unique_lock lk(mu_); auto it = stream_manager_->GetLastStream(); + if (it == stream_manager_->End()) return; lk.unlock(); OnRead(it, std::move(first_response)); // Acquire lock and queue the background stream. @@ -86,7 +87,7 @@ void ObjectDescriptorImpl::MakeSubsequentStream() { [](StreamManager::Stream const& s) { auto const* rs = s.stream.get(); return rs != nullptr && s.active_ranges.empty() && - !rs->write_pending_; + !rs->write_pending; })) { return; } @@ -150,9 +151,19 @@ ObjectDescriptorImpl::Read(ReadParams p) { } auto it = stream_manager_->GetLeastBusyStream(); + if (it == stream_manager_->End()) { + lk.unlock(); + range->OnFinish(Status(StatusCode::kFailedPrecondition, + "Cannot read object, all streams failed")); + if (!internal::TracingEnabled(options_)) { + return std::unique_ptr( + std::make_unique(std::move(range))); + } + return MakeTracingObjectDescriptorReader(std::move(range)); + } auto const id = ++read_id_generator_; it->active_ranges.emplace(id, range); - auto& read_range = *it->stream->next_request_.add_read_ranges(); + auto& read_range = *it->stream->next_request.add_read_ranges(); read_range.set_read_id(id); read_range.set_read_offset(p.start); read_range.set_read_length(p.length); @@ -168,18 +179,18 @@ ObjectDescriptorImpl::Read(ReadParams p) { void ObjectDescriptorImpl::Flush(std::unique_lock lk, StreamIterator it) { - if (it->stream->write_pending_ || - it->stream->next_request_.read_ranges().empty()) { + if (it->stream->write_pending || + it->stream->next_request.read_ranges().empty()) { return; } - it->stream->write_pending_ = true; + it->stream->write_pending = true; google::storage::v2::BidiReadObjectRequest request; - request.Swap(&it->stream->next_request_); + request.Swap(&it->stream->next_request); // Assign CurrentStream to a temporary variable to prevent // lifetime extension which can cause the lock to be held until the // end of the block. - auto current_stream = it->stream->stream_; + auto current_stream = it->stream->stream; lk.unlock(); current_stream->Write(std::move(request)) .then([w = WeakFromThis(), it](auto f) { @@ -190,19 +201,19 @@ void ObjectDescriptorImpl::Flush(std::unique_lock lk, void ObjectDescriptorImpl::OnWrite(StreamIterator it, bool ok) { std::unique_lock lk(mu_); if (!ok) return DoFinish(std::move(lk), it); - it->stream->write_pending_ = false; + it->stream->write_pending = false; Flush(std::move(lk), it); } void ObjectDescriptorImpl::DoRead(std::unique_lock lk, StreamIterator it) { - if (it->stream->read_pending_) return; - it->stream->read_pending_ = true; + if (it->stream->read_pending) return; + it->stream->read_pending = true; // Assign CurrentStream to a temporary variable to prevent // lifetime extension which can cause the lock to be held until the // end of the block. - auto current_stream = it->stream->stream_; + auto current_stream = it->stream->stream; lk.unlock(); current_stream->Read().then([w = WeakFromThis(), it](auto f) { if (auto self = w.lock()) self->OnRead(it, f.get()); @@ -213,7 +224,7 @@ void ObjectDescriptorImpl::OnRead( StreamIterator it, absl::optional response) { std::unique_lock lk(mu_); - it->stream->read_pending_ = false; + it->stream->read_pending = false; if (!response) return DoFinish(std::move(lk), it); if (response->has_metadata()) { @@ -242,11 +253,11 @@ void ObjectDescriptorImpl::OnRead( void ObjectDescriptorImpl::DoFinish(std::unique_lock lk, StreamIterator it) { - it->stream->read_pending_ = false; + it->stream->read_pending = false; // Assign CurrentStream to a temporary variable to prevent // lifetime extension which can cause the lock to be held until the // end of the block. - auto current_stream = it->stream->stream_; + auto current_stream = it->stream->stream; lk.unlock(); auto pending = current_stream->Finish(); if (!pending.valid()) return; @@ -292,8 +303,8 @@ void ObjectDescriptorImpl::OnResume(StreamIterator it, it->stream = std::make_shared(std::move(result->stream), resume_policy_prototype_->clone()); - it->stream->write_pending_ = false; - it->stream->read_pending_ = false; + it->stream->write_pending = false; + it->stream->read_pending = false; // TODO(#15105) - this should be done without release the lock. Flush(std::move(lk), it); @@ -326,7 +337,7 @@ bool ObjectDescriptorImpl::IsResumable( stream_manager_->CleanupDoneRanges(it); return true; } - return it->stream->resume_policy_->OnFinish(status) == + return it->stream->resume_policy->OnFinish(status) == storage_experimental::ResumePolicy::kContinue; } diff --git a/google/cloud/storage/internal/async/object_descriptor_impl.h b/google/cloud/storage/internal/async/object_descriptor_impl.h index 36d26f485e977..1a521c934946e 100644 --- a/google/cloud/storage/internal/async/object_descriptor_impl.h +++ b/google/cloud/storage/internal/async/object_descriptor_impl.h @@ -36,30 +36,25 @@ namespace cloud { namespace storage_internal { GOOGLE_CLOUD_CPP_INLINE_NAMESPACE_BEGIN -class ReadStream : public storage_internal::StreamBase { - public: +struct ReadStream : public storage_internal::StreamBase { ReadStream(std::shared_ptr stream, std::unique_ptr resume_policy) - : stream_(std::move(stream)), resume_policy_(std::move(resume_policy)) {} + : stream(std::move(stream)), resume_policy(std::move(resume_policy)) {} void Cancel() override { - if (stream_) stream_->Cancel(); + if (stream) stream->Cancel(); } - std::shared_ptr stream_; - std::unique_ptr resume_policy_; - google::storage::v2::BidiReadObjectRequest next_request_; - bool write_pending_ = false; - bool read_pending_ = false; + std::shared_ptr stream; + std::unique_ptr resume_policy; + google::storage::v2::BidiReadObjectRequest next_request; + bool write_pending = false; + bool read_pending = false; }; class ObjectDescriptorImpl : public storage_experimental::ObjectDescriptorConnection, public std::enable_shared_from_this { - private: - using StreamManager = MultiStreamManager; - using StreamIterator = StreamManager::StreamIterator; - public: ObjectDescriptorImpl( std::unique_ptr resume_policy, @@ -87,6 +82,10 @@ class ObjectDescriptorImpl void MakeSubsequentStream() override; private: + using StreamManager = MultiStreamManager; + using StreamIterator = + MultiStreamManager::StreamIterator; + std::weak_ptr WeakFromThis() { return shared_from_this(); } From 141060a12dec2f8872c6ed928945c76b23ae74e1 Mon Sep 17 00:00:00 2001 From: Shubham Kaushal Date: Fri, 19 Dec 2025 12:38:10 +0000 Subject: [PATCH 09/12] resolving comments --- .../internal/async/multi_stream_manager.h | 29 ++++----- .../async/multi_stream_manager_test.cc | 63 ++++++++++--------- .../internal/async/object_descriptor_impl.cc | 14 +---- 3 files changed, 49 insertions(+), 57 deletions(-) diff --git a/google/cloud/storage/internal/async/multi_stream_manager.h b/google/cloud/storage/internal/async/multi_stream_manager.h index 52aec213aa520..c8b213611963d 100644 --- a/google/cloud/storage/internal/async/multi_stream_manager.h +++ b/google/cloud/storage/internal/async/multi_stream_manager.h @@ -41,7 +41,7 @@ class StreamBase { // Manages a collection of streams. // // This class implements the "Subsequent Stream" logic where idle streams -// are moved to the back of the queue for reuse. +// are moved to the front of the queue for reuse. // // THREAD SAFETY: // This class is NOT thread-safe. The owner (e.g. ObjectDescriptorImpl @@ -83,9 +83,9 @@ class MultiStreamManager { streams_.push_back(Stream{std::move(initial_stream), {}}); } - StreamIterator GetLastStream() { + StreamIterator GetFirstStream() { if (streams_.empty()) return streams_.end(); - return std::prev(streams_.end()); + return streams_.begin(); } StreamIterator GetLeastBusyStream() { @@ -94,21 +94,24 @@ class MultiStreamManager { // Track min_ranges to avoid calling .size() repeatedly if possible, // though for std::unordered_map .size() is O(1). std::size_t min_ranges = least_busy_it->active_ranges.size(); + if (min_ranges == 0) return least_busy_it; // Start checking from the second element for (auto it = std::next(streams_.begin()); it != streams_.end(); ++it) { // Strict less-than ensures stability (preferring older streams if tied) - if (it->active_ranges.size() < min_ranges) { + auto size = it->active_ranges.size(); + if (size < min_ranges) { least_busy_it = it; - min_ranges = it->active_ranges.size(); + min_ranges = size; + if (min_ranges == 0) return least_busy_it; } } return least_busy_it; } StreamIterator AddStream(std::shared_ptr stream) { - streams_.push_back(Stream{std::move(stream), {}}); - return std::prev(streams_.end()); + streams_.push_front(Stream{std::move(stream), {}}); + return streams_.begin(); } void CancelAll() { @@ -141,16 +144,14 @@ class MultiStreamManager { } template - bool ReuseIdleStreamToBack(Pred pred) { + bool ReuseIdleStreamToFront(Pred pred) { for (auto it = streams_.begin(); it != streams_.end(); ++it) { if (!pred(*it)) continue; - // If the idle stream is already at the back, we don't - // need to move it. If it's elsewhere, use splice() to move the node. - // splice() is O(1) and, crucially, does not invalidate iterators - // or copy the Stream object. - if (std::next(it) != streams_.end()) { - streams_.splice(streams_.end(), streams_, it); + // If the idle stream is already at the front, we don't + // need to move it. Otherwise splice to the front in O(1). + if (it != streams_.begin()) { + streams_.splice(streams_.begin(), streams_, it); } return true; } diff --git a/google/cloud/storage/internal/async/multi_stream_manager_test.cc b/google/cloud/storage/internal/async/multi_stream_manager_test.cc index 95a4853c3298e..433864d058228 100644 --- a/google/cloud/storage/internal/async/multi_stream_manager_test.cc +++ b/google/cloud/storage/internal/async/multi_stream_manager_test.cc @@ -50,7 +50,7 @@ TEST(MultiStreamManagerTest, ConstructsWithFactoryAndHasOneStream) { auto mgr = MultiStreamManagerTest::MakeManager(); EXPECT_FALSE(mgr.Empty()); EXPECT_EQ(mgr.Size(), 1U); - auto it = mgr.GetLastStream(); + auto it = mgr.GetFirstStream(); ASSERT_TRUE(it->stream); } @@ -58,18 +58,18 @@ TEST(MultiStreamManagerTest, ConstructsWithInitialStream) { auto initial = std::make_shared(); Manager mgr([] { return nullptr; }, initial); EXPECT_EQ(mgr.Size(), 1U); - auto it = mgr.GetLastStream(); + auto it = mgr.GetFirstStream(); EXPECT_EQ(it->stream, initial); } -TEST(MultiStreamManagerTest, AddStreamAppendsAndGetLastReturnsNew) { +TEST(MultiStreamManagerTest, AddStreamAppendsAndGetFirstReturnsNew) { auto mgr = MultiStreamManagerTest::MakeManager(); auto s1 = std::make_shared(); auto it1 = mgr.AddStream(s1); EXPECT_EQ(mgr.Size(), 2U); EXPECT_EQ(it1->stream.get(), s1.get()); - auto it_last = mgr.GetLastStream(); - EXPECT_EQ(it_last->stream.get(), s1.get()); + auto it_first = mgr.GetFirstStream(); + EXPECT_EQ(it_first->stream.get(), s1.get()); } TEST(MultiStreamManagerTest, GetLeastBusyPrefersFewestActiveRanges) { @@ -78,7 +78,7 @@ TEST(MultiStreamManagerTest, GetLeastBusyPrefersFewestActiveRanges) { // The manager starts with an initial stream (size 0). // We must make it "busy" so it doesn't win the comparison against our test // streams. - auto it_init = mgr.GetLastStream(); + auto it_init = mgr.GetFirstStream(); it_init->active_ranges.emplace(999, std::make_shared()); it_init->active_ranges.emplace(998, std::make_shared()); @@ -103,7 +103,7 @@ TEST(MultiStreamManagerTest, GetLeastBusyPrefersFewestActiveRanges) { TEST(MultiStreamManagerTest, CleanupDoneRangesRemovesFinished) { auto mgr = MultiStreamManagerTest::MakeManager(); - auto it = mgr.GetLastStream(); + auto it = mgr.GetFirstStream(); auto r1 = std::make_shared(); r1->done = false; auto r2 = std::make_shared(); @@ -120,7 +120,7 @@ TEST(MultiStreamManagerTest, CleanupDoneRangesRemovesFinished) { TEST(MultiStreamManagerTest, RemoveStreamAndNotifyRangesCallsOnFinish) { auto mgr = MultiStreamManagerTest::MakeManager(); - auto it = mgr.GetLastStream(); + auto it = mgr.GetFirstStream(); auto r1 = std::make_shared(); auto r2 = std::make_shared(); it->active_ranges.emplace(11, r1); @@ -142,49 +142,50 @@ TEST(MultiStreamManagerTest, CancelAllInvokesCancel) { EXPECT_EQ(s2->cancelled, 1); } -TEST(MultiStreamManagerTest, ReuseIdleStreamToBackMovesElement) { +TEST(MultiStreamManagerTest, ReuseIdleStreamToFrontMovesElement) { auto mgr = MultiStreamManagerTest::MakeManager(); // Capture the factory-created stream pointer (initial element) - auto* factory_ptr = mgr.GetLastStream()->stream.get(); + auto* factory_ptr = mgr.GetFirstStream()->stream.get(); auto s1 = std::make_shared(); mgr.AddStream(s1); - bool moved = mgr.ReuseIdleStreamToBack([](Manager::Stream const& s) { + bool moved = mgr.ReuseIdleStreamToFront([](Manager::Stream const& s) { auto* fs = s.stream.get(); return fs != nullptr && s.active_ranges.empty() && !fs->write_pending; }); EXPECT_TRUE(moved); - auto it_last = mgr.GetLastStream(); - // After move, the factory stream should be last - EXPECT_EQ(it_last->stream.get(), factory_ptr); - EXPECT_NE(it_last->stream.get(), s1.get()); + auto it_first = mgr.GetFirstStream(); + // After move, the s1 stream should be first + EXPECT_EQ(it_first->stream.get(), s1.get()); + EXPECT_NE(it_first->stream.get(), factory_ptr); } TEST(MultiStreamManagerTest, - ReuseIdleStreamAlreadyAtBackReturnsTrueWithoutMove) { + ReuseIdleStreamAlreadyAtFrontReturnsTrueWithoutMove) { auto mgr = MultiStreamManagerTest::MakeManager(); - // The manager starts with one stream. It is the last stream, and it is idle. - auto initial_last = mgr.GetLastStream(); - bool reused = mgr.ReuseIdleStreamToBack( + // The manager starts with one stream. It is the first stream, and it is idle. + auto initial_first = mgr.GetFirstStream(); + bool reused = mgr.ReuseIdleStreamToFront( [](Manager::Stream const& s) { return s.active_ranges.empty(); }); EXPECT_TRUE(reused); - // Pointer should remain the same (it was already at the back) - EXPECT_EQ(mgr.GetLastStream(), initial_last); + // Pointer should remain the same (it was already at the front) + EXPECT_EQ(mgr.GetFirstStream(), initial_first); } TEST(MultiStreamManagerTest, ReuseIdleStreamDoesNotMoveWhenWritePending) { auto mgr = MultiStreamManagerTest::MakeManager(); + auto factory_ptr = mgr.GetFirstStream()->stream.get(); // Mark factory stream as not reusable - mgr.GetLastStream()->stream->write_pending = true; + mgr.GetFirstStream()->stream->write_pending = true; auto s1 = std::make_shared(); s1->write_pending = true; // also mark appended stream as not reusable mgr.AddStream(s1); - bool moved = mgr.ReuseIdleStreamToBack([](Manager::Stream const& s) { + bool moved = mgr.ReuseIdleStreamToFront([](Manager::Stream const& s) { auto* fs = s.stream.get(); return fs != nullptr && s.active_ranges.empty() && !fs->write_pending; }); EXPECT_FALSE(moved); - auto it_last = mgr.GetLastStream(); - EXPECT_EQ(it_last->stream.get(), s1.get()); + auto it_first = mgr.GetFirstStream(); + EXPECT_EQ(it_first->stream.get(), factory_ptr); } TEST(MultiStreamManagerTest, MoveActiveRangesTransfersAllEntries) { @@ -204,24 +205,24 @@ TEST(MultiStreamManagerTest, MoveActiveRangesTransfersAllEntries) { EXPECT_TRUE(it2->active_ranges.count(202)); } -TEST(MultiStreamManagerTest, GetLastStreamReflectsRecentAppendAndReuse) { +TEST(MultiStreamManagerTest, GetFirstStreamReflectsFrontReuse) { auto mgr = MultiStreamManagerTest::MakeManager(); auto s1 = std::make_shared(); mgr.AddStream(s1); - EXPECT_EQ(mgr.GetLastStream()->stream.get(), s1.get()); - bool moved = mgr.ReuseIdleStreamToBack([](Manager::Stream const& s) { + EXPECT_EQ(mgr.GetFirstStream()->stream.get(), s1.get()); + bool moved = mgr.ReuseIdleStreamToFront([](Manager::Stream const& s) { return s.stream != nullptr && s.active_ranges.empty(); }); EXPECT_TRUE(moved); - auto it_last = mgr.GetLastStream(); - EXPECT_NE(it_last->stream.get(), s1.get()); + auto it_first = mgr.GetFirstStream(); + EXPECT_EQ(it_first->stream.get(), s1.get()); } TEST(MultiStreamManagerTest, EmptyAndSizeTransitions) { auto mgr = MultiStreamManagerTest::MakeManager(); EXPECT_FALSE(mgr.Empty()); EXPECT_EQ(mgr.Size(), 1U); - auto it = mgr.GetLastStream(); + auto it = mgr.GetFirstStream(); mgr.RemoveStreamAndNotifyRanges(it, Status()); EXPECT_TRUE(mgr.Empty()); EXPECT_EQ(mgr.Size(), 0U); diff --git a/google/cloud/storage/internal/async/object_descriptor_impl.cc b/google/cloud/storage/internal/async/object_descriptor_impl.cc index 4f8dcf9521e62..6d2bfd0c986c0 100644 --- a/google/cloud/storage/internal/async/object_descriptor_impl.cc +++ b/google/cloud/storage/internal/async/object_descriptor_impl.cc @@ -50,7 +50,7 @@ ObjectDescriptorImpl::~ObjectDescriptorImpl() { Cancel(); } void ObjectDescriptorImpl::Start( google::storage::v2::BidiReadObjectResponse first_response) { std::unique_lock lk(mu_); - auto it = stream_manager_->GetLastStream(); + auto it = stream_manager_->GetFirstStream(); if (it == stream_manager_->End()) return; lk.unlock(); OnRead(it, std::move(first_response)); @@ -83,7 +83,7 @@ void ObjectDescriptorImpl::AssurePendingStreamQueued() { void ObjectDescriptorImpl::MakeSubsequentStream() { std::unique_lock lk(mu_); // Reuse an idle stream if possible. - if (stream_manager_->ReuseIdleStreamToBack( + if (stream_manager_->ReuseIdleStreamToFront( [](StreamManager::Stream const& s) { auto const* rs = s.stream.get(); return rs != nullptr && s.active_ranges.empty() && @@ -151,16 +151,6 @@ ObjectDescriptorImpl::Read(ReadParams p) { } auto it = stream_manager_->GetLeastBusyStream(); - if (it == stream_manager_->End()) { - lk.unlock(); - range->OnFinish(Status(StatusCode::kFailedPrecondition, - "Cannot read object, all streams failed")); - if (!internal::TracingEnabled(options_)) { - return std::unique_ptr( - std::make_unique(std::move(range))); - } - return MakeTracingObjectDescriptorReader(std::move(range)); - } auto const id = ++read_id_generator_; it->active_ranges.emplace(id, range); auto& read_range = *it->stream->next_request.add_read_ranges(); From f7fa96cb6409516e2f2fe06db0ff8936140e0f56 Mon Sep 17 00:00:00 2001 From: Shubham Kaushal Date: Fri, 19 Dec 2025 14:47:52 +0000 Subject: [PATCH 10/12] minor fix --- .../cloud/storage/internal/async/multi_stream_manager_test.cc | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/google/cloud/storage/internal/async/multi_stream_manager_test.cc b/google/cloud/storage/internal/async/multi_stream_manager_test.cc index 433864d058228..0cf948450e7d6 100644 --- a/google/cloud/storage/internal/async/multi_stream_manager_test.cc +++ b/google/cloud/storage/internal/async/multi_stream_manager_test.cc @@ -185,7 +185,8 @@ TEST(MultiStreamManagerTest, ReuseIdleStreamDoesNotMoveWhenWritePending) { }); EXPECT_FALSE(moved); auto it_first = mgr.GetFirstStream(); - EXPECT_EQ(it_first->stream.get(), factory_ptr); + EXPECT_EQ(it_first->stream.get(), s1.get()); + EXPECT_NE(it_first->stream.get(), factory_ptr); } TEST(MultiStreamManagerTest, MoveActiveRangesTransfersAllEntries) { From 053463a69ba2deb7ebf66036cdfdb24e7187a95d Mon Sep 17 00:00:00 2001 From: Shubham Kaushal Date: Fri, 19 Dec 2025 15:10:19 +0000 Subject: [PATCH 11/12] minor fix --- .../cloud/storage/internal/async/multi_stream_manager_test.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/google/cloud/storage/internal/async/multi_stream_manager_test.cc b/google/cloud/storage/internal/async/multi_stream_manager_test.cc index 0cf948450e7d6..482390f97fd0e 100644 --- a/google/cloud/storage/internal/async/multi_stream_manager_test.cc +++ b/google/cloud/storage/internal/async/multi_stream_manager_test.cc @@ -173,7 +173,7 @@ TEST(MultiStreamManagerTest, TEST(MultiStreamManagerTest, ReuseIdleStreamDoesNotMoveWhenWritePending) { auto mgr = MultiStreamManagerTest::MakeManager(); - auto factory_ptr = mgr.GetFirstStream()->stream.get(); + auto* factory_ptr = mgr.GetFirstStream()->stream.get(); // Mark factory stream as not reusable mgr.GetFirstStream()->stream->write_pending = true; auto s1 = std::make_shared(); From 91abbce4a781b96f196bfb74484979c489a7e4de Mon Sep 17 00:00:00 2001 From: Shubham Kaushal Date: Mon, 22 Dec 2025 07:16:38 +0000 Subject: [PATCH 12/12] resolving comments --- .../storage/internal/async/object_descriptor_impl.cc | 11 ++++++----- .../storage/internal/async/object_descriptor_impl.h | 5 +++-- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/google/cloud/storage/internal/async/object_descriptor_impl.cc b/google/cloud/storage/internal/async/object_descriptor_impl.cc index 6d2bfd0c986c0..f3651cc390f7c 100644 --- a/google/cloud/storage/internal/async/object_descriptor_impl.cc +++ b/google/cloud/storage/internal/async/object_descriptor_impl.cc @@ -56,7 +56,7 @@ void ObjectDescriptorImpl::Start( OnRead(it, std::move(first_response)); // Acquire lock and queue the background stream. lk.lock(); - AssurePendingStreamQueued(); + AssurePendingStreamQueued(lk); } void ObjectDescriptorImpl::Cancel() { @@ -72,7 +72,8 @@ absl::optional ObjectDescriptorImpl::metadata() return metadata_; } -void ObjectDescriptorImpl::AssurePendingStreamQueued() { +void ObjectDescriptorImpl::AssurePendingStreamQueued( + std::unique_lock const&) { if (pending_stream_.valid()) return; auto request = google::storage::v2::BidiReadObjectRequest{}; @@ -92,7 +93,7 @@ void ObjectDescriptorImpl::MakeSubsequentStream() { return; } // Proactively create a new stream if needed. - AssurePendingStreamQueued(); + AssurePendingStreamQueued(lk); if (!pending_stream_.valid()) return; auto stream_future = std::move(pending_stream_); lk.unlock(); @@ -119,7 +120,7 @@ void ObjectDescriptorImpl::MakeSubsequentStream() { auto new_it = self->stream_manager_->AddStream(std::move(read_stream)); // Now that we consumed pending_stream_, queue the next one immediately. - self->AssurePendingStreamQueued(); + self->AssurePendingStreamQueued(lk); lk.unlock(); self->OnRead(new_it, std::move(stream_result->first_response)); @@ -263,7 +264,7 @@ void ObjectDescriptorImpl::OnFinish(StreamIterator it, Status const& status) { std::unique_lock lk(mu_); stream_manager_->RemoveStreamAndNotifyRanges(it, status); // Since a stream died, we might want to ensure a replacement is queued. - AssurePendingStreamQueued(); + AssurePendingStreamQueued(lk); } void ObjectDescriptorImpl::Resume(StreamIterator it, diff --git a/google/cloud/storage/internal/async/object_descriptor_impl.h b/google/cloud/storage/internal/async/object_descriptor_impl.h index 1a521c934946e..ff91e4c90f048 100644 --- a/google/cloud/storage/internal/async/object_descriptor_impl.h +++ b/google/cloud/storage/internal/async/object_descriptor_impl.h @@ -90,8 +90,9 @@ class ObjectDescriptorImpl return shared_from_this(); } - // Logic to ensure a background stream is always connecting. - void AssurePendingStreamQueued(); + // Logic to ensure a background stream is always connecting which must be + // invoked while holding `mu_`. + void AssurePendingStreamQueued(std::unique_lock const&); void Flush(std::unique_lock lk, StreamIterator it); void OnWrite(StreamIterator it, bool ok);