diff --git a/src/groups/mqb/mqba/mqba_application.cpp b/src/groups/mqb/mqba/mqba_application.cpp index 55ff3e51b..055c33d2e 100644 --- a/src/groups/mqb/mqba/mqba_application.cpp +++ b/src/groups/mqb/mqba/mqba_application.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include #include #include @@ -159,6 +160,8 @@ Application::Application(bdlmt::EventScheduler* scheduler, bdlf::PlaceHolders::_2), // allocator k_BLOB_POOL_GROWTH_STRATEGY, d_allocators.get("BlobSpPool")) +, d_pushElementsPool(sizeof(mqbblp::PushStream::Element), + d_allocators.get("PushElementsPool")) , d_allocatorsStatContext_p(allocatorsStatContext) , d_pluginManager_mp() , d_statController_mp() @@ -269,6 +272,11 @@ int Application::start(bsl::ostream& errorDescription) } } + mqbi::ClusterResources resources(d_scheduler_p, + &d_bufferFactory, + &d_blobSpPool, + &d_pushElementsPool); + // Start the StatController d_statController_mp.load( new (*d_allocator_p) mqbstat::StatController( @@ -355,12 +363,10 @@ int Application::start(bsl::ostream& errorDescription) // Start the ClusterCatalog d_clusterCatalog_mp.load(new (*d_allocator_p) mqbblp::ClusterCatalog( - d_scheduler_p, d_dispatcher_mp.get(), d_transportManager_mp.get(), statContextsMap, - &d_bufferFactory, - &d_blobSpPool, + resources, d_allocators.get("ClusterCatalog")), d_allocator_p); diff --git a/src/groups/mqb/mqba/mqba_application.h b/src/groups/mqb/mqba/mqba_application.h index 850af282b..f7707aa85 100644 --- a/src/groups/mqb/mqba/mqba_application.h +++ b/src/groups/mqb/mqba/mqba_application.h @@ -136,6 +136,8 @@ class Application { BlobSpPool d_blobSpPool; + bdlma::ConcurrentPool d_pushElementsPool; + mwcst::StatContext* d_allocatorsStatContext_p; // Stat context of the counting allocators, // if used diff --git a/src/groups/mqb/mqbblp/mqbblp_cluster.cpp b/src/groups/mqb/mqbblp/mqbblp_cluster.cpp index 4941b727a..4b627549f 100644 --- a/src/groups/mqb/mqbblp/mqbblp_cluster.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_cluster.cpp @@ -2537,12 +2537,10 @@ Cluster::Cluster(const bslstl::StringRef& name, bslma::ManagedPtr netCluster, const StatContextsMap& statContexts, mqbi::DomainFactory* domainFactory, - bdlmt::EventScheduler* scheduler, mqbi::Dispatcher* dispatcher, - BlobSpPool* blobSpPool, - bdlbb::BlobBufferFactory* bufferFactory, mqbnet::TransportManager* transportManager, StopRequestManagerType* stopRequestsManager, + const mqbi::ClusterResources& resources, bslma::Allocator* allocator, const mqbnet::Session::AdminCommandEnqueueCb& adminCb) : d_allocator_p(allocator) @@ -2550,9 +2548,7 @@ Cluster::Cluster(const bslstl::StringRef& name, , d_isStarted(false) , d_isStopping(false) , d_clusterData(name, - scheduler, - bufferFactory, - blobSpPool, + resources, clusterConfig, mqbcfg::ClusterProxyDefinition(allocator), netCluster, @@ -2588,7 +2584,8 @@ Cluster::Cluster(const bslstl::StringRef& name, BSLS_ASSERT(d_allocator_p); mqbnet::Cluster* netCluster_p = d_clusterData.membership().netCluster(); BSLS_ASSERT(netCluster_p && "NetCluster not set !"); - BSLS_ASSERT(scheduler->clockType() == bsls::SystemClockType::e_MONOTONIC); + BSLS_ASSERT(resources.scheduler()->clockType() == + bsls::SystemClockType::e_MONOTONIC); BSLS_ASSERT_SAFE(d_clusterData.membership().selfNode() && "SelfNode not found in cluster!"); diff --git a/src/groups/mqb/mqbblp/mqbblp_cluster.h b/src/groups/mqb/mqbblp/mqbblp_cluster.h index 5c51bffed..1609461d1 100644 --- a/src/groups/mqb/mqbblp/mqbblp_cluster.h +++ b/src/groups/mqb/mqbblp/mqbblp_cluster.h @@ -467,17 +467,17 @@ class Cluster : public mqbi::Cluster, const bmqp::QueueId& queueId, mqbc::ClusterNodeSession* ns, bmqp::EventType::Enum eventType); - // Validate a message of the specified 'eventType' using the specified - // 'queueId' and 'ns'. Return one of `ValidationResult` values. Populate - // the specified 'queueHandle' if the queue is found. + // Validate a message of the specified `eventType` using the specified + // `queueId` and `ns`. Return one of `ValidationResult` values. Populate + // the specified `queueHandle` if the queue is found. bool validateRelayMessage(mqbc::ClusterNodeSession** ns, bsl::ostream* errorStream, const int pid); - // Validate a relay message using the specified 'pid'. Return true if the - // message is valid and false otherwise. Populate the specified 'ns' if the + // Validate a relay message using the specified `pid`. Return true if the + // message is valid and false otherwise. Populate the specified `ns` if the // message is valid or load a descriptive error message into the - // 'errorStream' if the message is invalid. + // `errorStream` if the message is invalid. /// Executes in any thread. void @@ -541,19 +541,17 @@ class Cluster : public mqbi::Cluster, /// Create a new object representing a cluster having the specified /// `name`, `clusterConfig` and `statContexts`, associated to the /// specified `netCluster` and using the specified `domainFactory`, - /// `scheduler`, `dispatcher`, `blobSpPool` and `bufferFactory`. Use + /// `scheduler`, `dispatcher`, `transportManager`, and `resources`. Use /// the specified `allocator` for any memory allocation. Cluster(const bslstl::StringRef& name, const mqbcfg::ClusterDefinition& clusterConfig, bslma::ManagedPtr netCluster, const StatContextsMap& statContexts, mqbi::DomainFactory* domainFactory, - bdlmt::EventScheduler* scheduler, mqbi::Dispatcher* dispatcher, - BlobSpPool* blobSpPool, - bdlbb::BlobBufferFactory* bufferFactory, mqbnet::TransportManager* transportManager, StopRequestManagerType* stopRequestsManager, + const mqbi::ClusterResources& resources, bslma::Allocator* allocator, const mqbnet::Session::AdminCommandEnqueueCb& adminCb); diff --git a/src/groups/mqb/mqbblp/mqbblp_clustercatalog.cpp b/src/groups/mqb/mqbblp/mqbblp_clustercatalog.cpp index e693a53b3..a64a2ce98 100644 --- a/src/groups/mqb/mqbblp/mqbblp_clustercatalog.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_clustercatalog.cpp @@ -182,12 +182,10 @@ int ClusterCatalog::createCluster(bsl::ostream& errorDescription, netCluster, d_statContexts, d_domainFactory_p, - d_scheduler_p, d_dispatcher_p, - d_blobSpPool_p, - d_bufferFactory_p, d_transportManager_p, &d_stopRequestsManager, + d_resources, clusterAllocator, d_adminCb); @@ -226,12 +224,10 @@ int ClusterCatalog::createCluster(bsl::ostream& errorDescription, clusterProxyDefinition, netCluster, d_statContexts, - d_scheduler_p, - d_bufferFactory_p, - d_blobSpPool_p, d_dispatcher_p, d_transportManager_p, &d_stopRequestsManager, + d_resources, clusterAllocator); info.d_cluster_sp.reset(cluster, clusterAllocator); @@ -362,20 +358,15 @@ int ClusterCatalog::initiateReversedClusterConnectionsImp( return rc; } -ClusterCatalog::ClusterCatalog(bdlmt::EventScheduler* scheduler, - mqbi::Dispatcher* dispatcher, - mqbnet::TransportManager* transportManager, - const StatContextsMap& statContexts, - bdlbb::BlobBufferFactory* bufferFactory, - BlobSpPool* blobSpPool, - bslma::Allocator* allocator) +ClusterCatalog::ClusterCatalog(mqbi::Dispatcher* dispatcher, + mqbnet::TransportManager* transportManager, + const StatContextsMap& statContexts, + const mqbi::ClusterResources& resources, + bslma::Allocator* allocator) : d_allocator_p(allocator) , d_allocators(d_allocator_p) , d_isStarted(false) -, d_scheduler_p(scheduler) , d_dispatcher_p(dispatcher) -, d_bufferFactory_p(bufferFactory) -, d_blobSpPool_p(blobSpPool) , d_transportManager_p(transportManager) , d_domainFactory_p(0) , d_clustersDefinition(d_allocator_p) @@ -386,14 +377,15 @@ ClusterCatalog::ClusterCatalog(bdlmt::EventScheduler* scheduler, , d_clusters(d_allocator_p) , d_statContexts(statContexts) , d_requestManager(bmqp::EventType::e_CONTROL, - d_bufferFactory_p, - d_scheduler_p, + resources.bufferFactory(), + resources.scheduler(), false, // lateResponseMode d_allocator_p) , d_stopRequestsManager(&d_requestManager, d_allocator_p) +, d_resources(resources) { // PRECONDITIONS - BSLS_ASSERT_SAFE(scheduler->clockType() == + BSLS_ASSERT_SAFE(d_resources.scheduler()->clockType() == bsls::SystemClockType::e_MONOTONIC); } diff --git a/src/groups/mqb/mqbblp/mqbblp_clustercatalog.h b/src/groups/mqb/mqbblp/mqbblp_clustercatalog.h index 38d65e609..90665fd88 100644 --- a/src/groups/mqb/mqbblp/mqbblp_clustercatalog.h +++ b/src/groups/mqb/mqbblp/mqbblp_clustercatalog.h @@ -140,16 +140,6 @@ class ClusterCatalog { // FRIENDS friend class ClusterCatalogIterator; - public: - // TYPES - - /// Pool of shared pointers to Blobs - typedef bdlcc::SharedObjectPool< - bdlbb::Blob, - bdlcc::ObjectPoolFunctors::DefaultCreator, - bdlcc::ObjectPoolFunctors::RemoveAll > - BlobSpPool; - public: // TYPES @@ -225,19 +215,9 @@ class ClusterCatalog { bool d_isStarted; // True if this component is started - bdlmt::EventScheduler* d_scheduler_p; - // EventScheduler to use - mqbi::Dispatcher* d_dispatcher_p; // Dispatcher to use - bdlbb::BlobBufferFactory* d_bufferFactory_p; - // Blob buffer factory to use - - BlobSpPool* d_blobSpPool_p; - // Pool of shared pointers to blob to - // use. - mqbnet::TransportManager* d_transportManager_p; // TransportManager for creating // mqbnet::Cluster @@ -290,6 +270,8 @@ class ClusterCatalog { StatContextsMap d_statContexts; // Map of stat contexts + const mqbi::ClusterResources d_resources; + mqbnet::Session::AdminCommandEnqueueCb d_adminCb; // Callback function to enqueue admin commands @@ -352,16 +334,14 @@ class ClusterCatalog { // CREATORS - /// Create a new object using the specified `scheduler`, `dispatcher`, - /// `transportManager`, `bufferFactory`, `blobSpPool`, and the specified - /// `allocator`. - ClusterCatalog(bdlmt::EventScheduler* scheduler, - mqbi::Dispatcher* dispatcher, - mqbnet::TransportManager* transportManager, - const StatContextsMap& statContexts, - bdlbb::BlobBufferFactory* bufferFactory, - BlobSpPool* blobSpPool, - bslma::Allocator* allocator); + /// Create a new object using the specified 'dispatcher', + /// 'transportManager', 'statContexts', 'resources', and the specified + /// 'allocator'. + ClusterCatalog(mqbi::Dispatcher* dispatcher, + mqbnet::TransportManager* transportManager, + const StatContextsMap& statContexts, + const mqbi::ClusterResources& resources, + bslma::Allocator* allocator); /// Destructor. ~ClusterCatalog(); diff --git a/src/groups/mqb/mqbblp/mqbblp_clusterproxy.cpp b/src/groups/mqb/mqbblp/mqbblp_clusterproxy.cpp index bbe62bc74..066eedfc4 100644 --- a/src/groups/mqb/mqbblp/mqbblp_clusterproxy.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_clusterproxy.cpp @@ -1047,20 +1047,16 @@ ClusterProxy::ClusterProxy( const mqbcfg::ClusterProxyDefinition& clusterProxyConfig, bslma::ManagedPtr netCluster, const StatContextsMap& statContexts, - bdlmt::EventScheduler* scheduler, - bdlbb::BlobBufferFactory* bufferFactory, - BlobSpPool* blobSpPool, mqbi::Dispatcher* dispatcher, mqbnet::TransportManager* transportManager, StopRequestManagerType* stopRequestsManager, + const mqbi::ClusterResources& resources, bslma::Allocator* allocator) : d_allocator_p(allocator) , d_isStarted(false) , d_isStopping(false) , d_clusterData(name, - scheduler, - bufferFactory, - blobSpPool, + resources, mqbcfg::ClusterDefinition(allocator), clusterProxyConfig, netCluster, @@ -1088,7 +1084,7 @@ ClusterProxy::ClusterProxy( // PRECONDITIONS mqbnet::Cluster* netCluster_p = d_clusterData.membership().netCluster(); BSLS_ASSERT_SAFE(netCluster_p && "NetCluster not set !"); - BSLS_ASSERT_SAFE(scheduler->clockType() == + BSLS_ASSERT_SAFE(resources.scheduler()->clockType() == bsls::SystemClockType::e_MONOTONIC); d_clusterData.clusterConfig().queueOperations() = diff --git a/src/groups/mqb/mqbblp/mqbblp_clusterproxy.h b/src/groups/mqb/mqbblp/mqbblp_clusterproxy.h index 2d81a926e..89cf41f20 100644 --- a/src/groups/mqb/mqbblp/mqbblp_clusterproxy.h +++ b/src/groups/mqb/mqbblp/mqbblp_clusterproxy.h @@ -291,11 +291,8 @@ class ClusterProxy : public mqbc::ClusterStateObserver, mqbnet::ClusterNode* node, BSLS_ANNOTATION_UNUSED const bmqp_ctrlmsg::ClientIdentity& identity); - // Executed by the dispatcher thread when there is a change in the - // specified 'node' connectivity. If not empty, the specified - // 'session' points to newly connected 'Session'. Empty 'session' - // indicates loss of connectivity. - + /// Executed by the dispatcher thread when the specified `node` becomes + /// unavailable. void onNodeDownDispatched(mqbnet::ClusterNode* node); /// Callback method when the `activeNodeLookupEvent` has expired. @@ -429,20 +426,18 @@ class ClusterProxy : public mqbc::ClusterStateObserver, // CREATORS /// Create a new object representing a cluster having the specified - /// `name`, `clusterProxyConfig` and `statContexts`, associated to the - /// specified `netCluster` and using the specified `scheduler`, - /// `bufferFactory`, `blobSpPool` and `dispatcher`. Use the specified - /// `allocator` for any memory allocation. + /// `name`, `clusterConfig` and `statContexts`, associated to the + /// specified `netCluster` and using the specified `domainFactory`, + /// `scheduler`, `dispatcher`, `transportManager`, and `resources`. Use + /// the specified `allocator` for any memory allocation. ClusterProxy(const bslstl::StringRef& name, const mqbcfg::ClusterProxyDefinition& clusterProxyConfig, bslma::ManagedPtr netCluster, const StatContextsMap& statContexts, - bdlmt::EventScheduler* scheduler, - bdlbb::BlobBufferFactory* bufferFactory, - BlobSpPool* blobSpPool, mqbi::Dispatcher* dispatcher, mqbnet::TransportManager* transportManager, StopRequestManagerType* stopRequestsManager, + const mqbi::ClusterResources& resources, bslma::Allocator* allocator); /// Destructor diff --git a/src/groups/mqb/mqbblp/mqbblp_clusterqueuehelper.cpp b/src/groups/mqb/mqbblp/mqbblp_clusterqueuehelper.cpp index f8fa9467d..761f9d37d 100644 --- a/src/groups/mqb/mqbblp/mqbblp_clusterqueuehelper.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_clusterqueuehelper.cpp @@ -2114,8 +2114,7 @@ bsl::shared_ptr ClusterQueueHelper::createQueueFactory( context.d_queueContext_p->partitionId(), context.d_domain_p, d_storageManager_p, - &d_clusterData_p->bufferFactory(), - &d_clusterData_p->scheduler(), + d_clusterData_p->resources(), &d_clusterData_p->miscWorkThreadPool(), openQueueResponse.routingConfiguration(), d_allocator_p), @@ -4428,15 +4427,8 @@ void ClusterQueueHelper::onQueueUpdated(const bmqt::Uri& uri, for (AppIdInfosCIter cit = addedAppIds.cbegin(); cit != addedAppIds.cend(); ++cit) { - if (d_clusterState_p->isSelfPrimary(partitionId) && queue) { - d_cluster_p->dispatcher()->execute( - bdlf::BindUtil::bind(afterAppIdRegisteredDispatched, - queue, - *cit), - queue); - } - else { - // Note: In non-CSL mode, the queue creation callback is instead + if (!d_clusterState_p->isSelfPrimary(partitionId) || queue == 0) { + // Note: In non-CSL mode, the queue creation callback is // invoked at replica nodes when they receive a queue creation // record from the primary in the partition stream. mqbi::Storage::AppIdKeyPair appIdKeyPair(cit->first, cit->second); @@ -4450,20 +4442,20 @@ void ClusterQueueHelper::onQueueUpdated(const bmqt::Uri& uri, .at(uri.qualifiedDomain()) ->domain()); } - } - - for (AppIdInfosCIter cit = removedAppIds.cbegin(); - cit != removedAppIds.cend(); - ++cit) { - if (d_clusterState_p->isSelfPrimary(partitionId) && queue) { + if (queue) { d_cluster_p->dispatcher()->execute( - bdlf::BindUtil::bind(afterAppIdUnregisteredDispatched, + bdlf::BindUtil::bind(afterAppIdRegisteredDispatched, queue, *cit), queue); } - else { - // Note: In non-CSL mode, the queue deletion callback is instead + } + + for (AppIdInfosCIter cit = removedAppIds.cbegin(); + cit != removedAppIds.cend(); + ++cit) { + if (!d_clusterState_p->isSelfPrimary(partitionId) || queue == 0) { + // Note: In non-CSL mode, the queue deletion callback is // invoked at replica nodes when they receive a queue deletion // record from the primary in the partition stream. d_storageManager_p->unregisterQueueReplica(partitionId, @@ -4471,6 +4463,13 @@ void ClusterQueueHelper::onQueueUpdated(const bmqt::Uri& uri, qiter->second->key(), cit->second); } + if (queue) { + d_cluster_p->dispatcher()->execute( + bdlf::BindUtil::bind(afterAppIdUnregisteredDispatched, + queue, + *cit), + queue); + } } mwcu::Printer printer1(&addedAppIds); diff --git a/src/groups/mqb/mqbblp/mqbblp_pushstream.cpp b/src/groups/mqb/mqbblp/mqbblp_pushstream.cpp new file mode 100644 index 000000000..bf63aabc8 --- /dev/null +++ b/src/groups/mqb/mqbblp/mqbblp_pushstream.cpp @@ -0,0 +1,323 @@ +// Copyright 2024 Bloomberg Finance L.P. +// SPDX-License-Identifier: Apache-2.0 +// +// 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 +// +// http://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. + +// mqbblp_pushstream.cpp -*-C++-*- + +#include + +#include + +namespace BloombergLP { +namespace mqbblp { + +namespace { + +void noOpDeleter(bdlma::ConcurrentPool*) +{ + // NOTHING +} + +} // close unnamed namespace + +// ---------------- +// class PushStream +// ---------------- + +PushStream::PushStream( + const bsl::optional& pushElementsPool, + bslma::Allocator* allocator) +: d_stream(allocator) +, d_apps(allocator) +, d_pushElementsPool_sp(pushElementsPool.value_or(bsl::nullptr_t()), + noOpDeleter, + allocator) +{ + allocator = bslma::Default::allocator(allocator); + + if (!d_pushElementsPool_sp) { + d_pushElementsPool_sp.load( + new (*allocator) bdlma::ConcurrentPool(sizeof(Element), allocator), + allocator); + } + BSLS_ASSERT_SAFE(d_pushElementsPool_sp); + BSLS_ASSERT_SAFE(d_pushElementsPool_sp->blockSize() == sizeof(Element)); +} + +// ---------------------------- +// class VirtualStorageIterator +// ---------------------------- + +// PRIVATE MANIPULATORS +void PushStreamIterator::clear() +{ + // Clear previous state, if any. This is required so that new state can be + // loaded in 'appData', 'options' or 'attributes' routines. + d_appData_sp.reset(); + d_options_sp.reset(); + d_attributes.reset(); +} + +// PRIVATE ACCESSORS +bool PushStreamIterator::loadMessageAndAttributes() const +{ + BSLS_ASSERT_SAFE(!atEnd()); + + if (!d_appData_sp) { + mqbi::StorageResult::Enum rc = d_storage_p->get(&d_appData_sp, + &d_options_sp, + &d_attributes, + d_iterator->first); + BSLS_ASSERT_SAFE(mqbi::StorageResult::e_SUCCESS == rc); + static_cast(rc); // suppress compiler warning + return true; // RETURN + } + return false; +} + +// CREATORS +PushStreamIterator::PushStreamIterator( + mqbi::Storage* storage, + PushStream* owner, + const PushStream::iterator& initialPosition) +: d_storage_p(storage) +, d_iterator(initialPosition) +, d_attributes() +, d_appData_sp() +, d_options_sp() +, d_owner_p(owner) +, d_currentElement(0) +, d_currentOrdinal(mqbi::Storage::k_INVALID_ORDINAL) +{ + BSLS_ASSERT_SAFE(d_storage_p); + BSLS_ASSERT_SAFE(d_owner_p); +} + +PushStreamIterator::~PushStreamIterator() +{ + // NOTHING +} + +unsigned int PushStreamIterator::numApps() const +{ + BSLS_ASSERT_SAFE(!atEnd()); + return d_iterator->second.numElements(); +} + +void PushStreamIterator::removeCurrentElement() +{ + BSLS_ASSERT_SAFE(!atEnd()); + BSLS_ASSERT_SAFE(d_currentElement); + + PushStream::Element* del = d_currentElement; + + // still keep the same ordinal numbering + d_currentElement = d_currentElement->next(); + ++d_currentOrdinal; + + d_owner_p->remove(del); + d_owner_p->destroy(del, true); + // doKeepGuid because of the d_iterator + + if (d_iterator->second.numElements() == 0) { + BSLS_ASSERT_SAFE(d_currentElement == 0); + } +} + +// MANIPULATORS +bool PushStreamIterator::advance() +{ + // PRECONDITIONS + BSLS_ASSERT_SAFE(!atEnd()); + + clear(); + + if (d_iterator->second.numElements() == 0) { + d_iterator = d_owner_p->d_stream.erase(d_iterator); + } + else { + ++d_iterator; + } + + d_currentOrdinal = mqbi::Storage::k_INVALID_ORDINAL; + + return !atEnd(); +} + +void PushStreamIterator::reset(const bmqt::MessageGUID& where) +{ + clear(); + + if (where.isUnset()) { + // Reset iterator to beginning + d_iterator = d_owner_p->d_stream.begin(); + } + else { + d_iterator = d_owner_p->d_stream.find(where); + } + + d_currentOrdinal = mqbi::Storage::k_INVALID_ORDINAL; +} + +// ACCESSORS +const bmqt::MessageGUID& PushStreamIterator::guid() const +{ + // PRECONDITIONS + BSLS_ASSERT_SAFE(!atEnd()); + + return d_iterator->first; +} + +PushStream::Element* PushStreamIterator::element(unsigned int appOrdinal) const +{ + // PRECONDITIONS + BSLS_ASSERT_SAFE(!atEnd()); + BSLS_ASSERT_SAFE(appOrdinal < mqbi::Storage::k_INVALID_ORDINAL); + + if (d_currentOrdinal > appOrdinal) { + d_currentOrdinal = 0; + d_currentElement = d_iterator->second.front(); + } + + BSLS_ASSERT_SAFE(d_currentElement); + + while (appOrdinal > d_currentOrdinal) { + ++d_currentOrdinal; + d_currentElement = d_currentElement->next(); + + BSLS_ASSERT_SAFE(d_currentElement); + } + + return d_currentElement; +} + +const mqbi::AppMessage& +PushStreamIterator::appMessageView(unsigned int appOrdinal) const +{ + return *element(appOrdinal)->appView(); +} + +mqbi::AppMessage& PushStreamIterator::appMessageState(unsigned int appOrdinal) +{ + return *element(appOrdinal)->appState(); +} + +const bsl::shared_ptr& PushStreamIterator::appData() const +{ + loadMessageAndAttributes(); + return d_appData_sp; +} + +const bsl::shared_ptr& PushStreamIterator::options() const +{ + loadMessageAndAttributes(); + return d_options_sp; +} + +const mqbi::StorageMessageAttributes& PushStreamIterator::attributes() const +{ + loadMessageAndAttributes(); + return d_attributes; +} + +bool PushStreamIterator::atEnd() const +{ + return (d_iterator == d_owner_p->d_stream.end()); +} + +bool PushStreamIterator::hasReceipt() const +{ + return !atEnd(); +} + +// CREATORS +VirtualPushStreamIterator::VirtualPushStreamIterator( + unsigned int upstreamSubQueueId, + mqbi::Storage* storage, + PushStream* owner, + const PushStream::iterator& initialPosition) +: PushStreamIterator(storage, owner, initialPosition) +{ + d_itApp = owner->d_apps.find(upstreamSubQueueId); + + BSLS_ASSERT_SAFE(d_itApp != owner->d_apps.end()); + + d_currentElement = d_itApp->second.d_elements.front(); + + BSLS_ASSERT_SAFE(d_currentElement->app().d_app == d_itApp->second.d_app); +} + +VirtualPushStreamIterator::~VirtualPushStreamIterator() +{ + // NOTHING +} + +unsigned int VirtualPushStreamIterator::numApps() const +{ + BSLS_ASSERT_SAFE(!atEnd()); + + return 1; +} + +void VirtualPushStreamIterator::removeCurrentElement() +{ + advance(); +} + +// MANIPULATORS +bool VirtualPushStreamIterator::advance() +{ + // PRECONDITIONS + BSLS_ASSERT_SAFE(!atEnd()); + + clear(); + + PushStream::Element* del = d_currentElement; + + d_currentElement = d_currentElement->nextInApp(); + + d_owner_p->remove(del); + d_owner_p->destroy(del, false); + // do not keep Guid + + if (atEnd()) { + return false; + } + + BSLS_ASSERT_SAFE(d_itApp->second.d_elements.numElements()); + BSLS_ASSERT_SAFE(d_currentElement); + + return true; +} + +bool VirtualPushStreamIterator::atEnd() const +{ + return (d_currentElement == 0); +} + +PushStream::Element* +VirtualPushStreamIterator::element(unsigned int appOrdinal) const +{ + // PRECONDITIONS + BSLS_ASSERT_SAFE(!atEnd()); + + // Ignore ordinal when the app is fixed; + // 'd_currentElement' does not depend on 'appOrdinal' + (void)appOrdinal; + + return d_currentElement; +} + +} // close package namespace +} // close enterprise namespace diff --git a/src/groups/mqb/mqbblp/mqbblp_pushstream.h b/src/groups/mqb/mqbblp/mqbblp_pushstream.h new file mode 100644 index 000000000..87bb9378f --- /dev/null +++ b/src/groups/mqb/mqbblp/mqbblp_pushstream.h @@ -0,0 +1,707 @@ +// Copyright 2024 Bloomberg Finance L.P. +// SPDX-License-Identifier: Apache-2.0 +// +// 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 +// +// http://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. + +// mqbblp_pushstream.h -*-C++-*- +#ifndef INCLUDED_MQBBLP_PUSHSTREAM +#define INCLUDED_MQBBLP_PUSHSTREAM + +//@PURPOSE: Provide a Storage for one-time PUSH delivery +// +//@CLASSES: +// mqbblp::PushStream: the ordered sequence of GUID for one time delivery +// +//@DESCRIPTION: An additional Storage layer for one-time PUSH delivery at +// Replica/Proxy. +// When PUSH message is a result of round-robin, the number of App ids in the +// message may not be equal to the number of Apps known to the +// RelayQueueEngine. Moreover, the order of data replication in Replica may +// not be equal to the order of PUSH messages. The main storage DataStream +// cannot feed the deliver logic, we need an additional layer. +// This layer supports the 'mqbi::StorageIterator' interface because this is +// how the delivery logic accesses data in all cases including Primary where +// the main DataStream storage is used. And including the future Reliable +// Broadcast mode. +// An efficient iteration requires records of variable size per each GUID. +// On the other side, there is only sequential access - either for each GUID or +// for each App. An 'Element' holding PUSH context for a GUID and an App is in +// two lists - for the GUID for the App. Removal can be random for the GUID +// list and always sequential for the App list. +// The storage supports 'mqbi::StorageIterator' interface but the meaning of +// 'appOrdinal' in the 'appMessageView' is different; the access is always +// sequential with monotonically increasing 'appOrdinal' and the 'appOrdinal' +// can be different for the same App depending on the GUID. +// Upon GUIDs iteration followed by he GUID list iteration, if the App succeeds +// in delivering the PUSH, the engine removes the 'Element' from both lists. +// If the App is at capacity, the 'Element' stays, the iterations continue. +// Upon 'onHandleUsable', the App need to catchup by iterating the App list. + +// MQB + +#include + +// BMQ +#include + +// MWC +#include + +// BDE +#include +#include +#include +#include +#include + +namespace BloombergLP { + +namespace mqbblp { + +// FORWARD DECLARATION +struct RelayQueueEngine_AppState; + +struct PushStream { + // forward declaration + struct Element; + + enum ElementList { e_GUID = 0, e_APP = 1, e_TOTAL = 2 }; + + struct ElementBase { + Element* d_next_p; + Element* d_previous_p; + + ElementBase(); + }; + + struct Elements { + // A list of Elements to associate Elements with 1) GUID, 2) App. + // In the case of GUID, the list is doubly-linked for random removal + // In the case of App, the list is singly-linked; the removal is always + // sequential. + + private: + Element* d_first_p; + Element* d_last_p; + unsigned int d_numElements; + + private: + void onRemove(); + void onAdd(Element* element); + + public: + Elements(); + + /// Add the specified `element` to doubly-linked list for GUID + void add(Element* element, ElementList where); + + /// Remove the specified `element` from doubly-linked list for GUID + void remove(Element* element, ElementList where); + + /// Return the first Element in the list + Element* front() const; + unsigned int numElements() const; + }; + + struct App { + Elements d_elements; + bsl::shared_ptr d_app; + + App(const bsl::shared_ptr& app); + void add(Element* element); + void remove(Element* element); + }; + + typedef mwcc::OrderedHashMap > + Stream; + typedef Stream::iterator iterator; + typedef bsl::unordered_map Apps; + + struct Element { + friend struct Elements; + + private: + ElementBase d_base[e_TOTAL]; + mqbi::AppMessage d_app; + const iterator d_iteratorGuid; + const Apps::iterator d_iteratorApp; + + public: + Element(const bmqp::SubQueueInfo& subscription, + const iterator& iterator, + const Apps::iterator& iteratorApp); + + /// Return a modifiable reference to the App state associated with this + /// Element. + mqbi::AppMessage* appState(); + + /// Return a non-modifiable reference to the App state associated with + /// this Element. + const mqbi::AppMessage* appView() const; + + /// Return the GUID associated with this Element. + Elements& guid() const; + App& app() const; + + void eraseGuid(Stream& stream); + void eraseApp(Apps& apps); + + /// Return true if this Element is associated with the specified + /// `iterator` position in the PushStream. + bool isInStream(const PushStream::Stream::iterator& iterator) const; + + /// Return pointer to the next Element associated with the same GUID + /// or `0` if this is the last Element. + Element* next() const; + + /// Return pointer to the next Element associated with the same App + /// or `0` if this is the last Element. + Element* nextInApp() const; + }; + + Stream d_stream; + + Apps d_apps; + + bsl::shared_ptr d_pushElementsPool_sp; + + PushStream(const bsl::optional& pushElementsPool, + bslma::Allocator* allocator); + + /// Introduce the specified `guid` to the Push Stream if it is not present. + /// Return an iterator pointing to the `guid`. + iterator findOrAppendMessage(const bmqt::MessageGUID& guid); + + /// Add the specified `element` to both GUID and App corresponding to the + /// `element` (and specified when constructing the `element`). + void add(Element* element); + + /// Remove the specified `element` from both GUID and App corresponding to + /// the `element` (and specified when constructing the `element`). + /// Return the number of remaining Elements in the corresponding GUID. + unsigned int remove(Element* element); + + /// Remove all PushStream Elements corresponding to the specified + /// `upstreamSubQueueId`. Erase each corresponding GUIDs from the + /// PushStream with no remaining Elements. Erase the corresponding App. + /// Return the number of removed elements. + unsigned int removeApp(unsigned int upstreamSubQueueId); + + /// Remove all PushStream Elements corresponding to the specified + /// `itApp`. Erase each corresponding GUIDs from the PushStream with no + /// remaining Elements. Erase the corresponding App. + /// Return the number of removed elements. + unsigned int removeApp(Apps::iterator itApp); + + /// Remove all Elements, Apps, and GUIDs. + unsigned int removeAll(); + + /// Create new Element associated with the specified `info`, + // `upstreamSubQueueId`, and `iterator`. + Element* create(const bmqp::SubQueueInfo& info, + const iterator& iterator, + const Apps::iterator& iteratorApp); + + /// Destroy the specified `element` + void destroy(Element* element, bool doKeepGuid); +}; + +// ======================== +// class PushStreamIterator +// ======================== + +class PushStreamIterator : public mqbi::StorageIterator { + // A mechanism to iterate the PushStream; see above. To be used by the + // QueueEngine routing in the same way as another `mqbi::StorageIterator` + // implementation(s). + + private: + // DATA + mqbi::Storage* d_storage_p; + + PushStream::iterator d_iterator; + + mutable mqbi::StorageMessageAttributes d_attributes; + + mutable bsl::shared_ptr d_appData_sp; + // If this variable is empty, it is + // assumed that attributes, message, + // and options have not been loaded in + // this iteration (see also + // `loadMessageAndAttributes` impl). + + mutable bsl::shared_ptr d_options_sp; + + protected: + PushStream* d_owner_p; + + /// Current (`mqbi::AppMessage`, `upstreamSubQueueId`) pair. + mutable PushStream::Element* d_currentElement; + + /// Current ordinal corresponding to the `d_currentElement`. + mutable unsigned int d_currentOrdinal; + + private: + // NOT IMPLEMENTED + PushStreamIterator(const StorageIterator&); // = delete + PushStreamIterator& operator=(const PushStreamIterator&); // = delete + + protected: + // PRIVATE MANIPULATORS + + /// Clear previous state, if any. This is required so that new state + /// can be loaded in `appData`, `options` or `attributes` routines. + void clear(); + + // PRIVATE ACCESSORS + + /// Load the internal state of this iterator instance with the + /// attributes and blob pointed to by the MessageGUID to which this + /// iterator is currently pointing. Behavior is undefined if `atEnd()` + /// returns true or if underlying storage does not contain the + /// MessageGUID being pointed to by this iterator. Return `false` if + /// data are already loaded; return `true` otherwise. + bool loadMessageAndAttributes() const; + + public: + // CREATORS + + /// Create a new VirtualStorageIterator from the specified `storage` and + /// pointing at the specified `initialPosition`. + PushStreamIterator(mqbi::Storage* storage, + PushStream* owner, + const PushStream::iterator& initialPosition); + + /// Destructor + virtual ~PushStreamIterator() BSLS_KEYWORD_OVERRIDE; + + /// Remove the current element (`mqbi::AppMessage`, `upstreamSubQueueId` + /// pair) from the current PUSH GUID. + /// The behavior is undefined unless `atEnd` returns `false`. + void removeCurrentElement(); + + /// Return the number of elements (`mqbi::AppMessage`, `upstreamSubQueueId` + /// pairs) for the current PUSH GUID. + /// The behavior is undefined unless `atEnd` returns `false`. + unsigned int numApps() const; + + /// Return the current element (`mqbi::AppMessage`, `upstreamSubQueueId` + /// pair). + /// The behavior is undefined unless `atEnd` returns `false`. + virtual PushStream::Element* element(unsigned int appOrdinal) const; + + // MANIPULATORS + bool advance() BSLS_KEYWORD_OVERRIDE; + + /// If the specified `atEnd` is `true`, reset the iterator to point to the + /// to the end of the underlying storage. Otherwise, reset the iterator to + /// point first item, if any, in the underlying storage. + void reset(const bmqt::MessageGUID& where = bmqt::MessageGUID()) + BSLS_KEYWORD_OVERRIDE; + + // ACCESSORS + + /// Return a reference offering non-modifiable access to the guid + /// associated to the item currently pointed at by this iterator. The + /// behavior is undefined unless `atEnd` returns `false`. + const bmqt::MessageGUID& guid() const BSLS_KEYWORD_OVERRIDE; + + /// Return a reference offering non-modifiable access to the App state + /// associated to the item currently pointed at by this iterator. The + /// behavior is undefined unless `atEnd` returns `false`. + const mqbi::AppMessage& + appMessageView(unsigned int appOrdinal) const BSLS_KEYWORD_OVERRIDE; + + /// Return a reference offering modifiable access to the App state + /// associated to the item currently pointed at by this iterator. The + /// behavior is undefined unless `atEnd` returns `false`. + mqbi::AppMessage& + appMessageState(unsigned int appOrdinal) BSLS_KEYWORD_OVERRIDE; + + /// Return a reference offering non-modifiable access to the application + /// data associated with the item currently pointed at by this iterator. + /// The behavior is undefined unless `atEnd` returns `false`. + const bsl::shared_ptr& appData() const BSLS_KEYWORD_OVERRIDE; + + /// Return a reference offering non-modifiable access to the options + /// associated with the item currently pointed at by this iterator. The + /// behavior is undefined unless `atEnd` returns `false`. + const bsl::shared_ptr& options() const BSLS_KEYWORD_OVERRIDE; + + /// Return a reference offering non-modifiable access to the attributes + /// associated with the message currently pointed at by this iterator. + /// The behavior is undefined unless `atEnd` returns `false`. + const mqbi::StorageMessageAttributes& + attributes() const BSLS_KEYWORD_OVERRIDE; + + /// Return `true` if this iterator is currently at the end of the items' + /// collection, and hence doesn't reference a valid item. + bool atEnd() const BSLS_KEYWORD_OVERRIDE; + + /// Return `true` if this iterator is currently not at the end of the + /// `items` collection and the message currently pointed at by this + /// iterator has received replication factor Receipts. + bool hasReceipt() const BSLS_KEYWORD_OVERRIDE; +}; + +// ============================ +// class VirtualStorageIterator +// ============================ + +class VirtualPushStreamIterator : public PushStreamIterator { + // A mechanism to iterate Elements related to one App only. + + private: + // DATA + + PushStream::Apps::iterator d_itApp; + // An iterator to the App being iterated + + private: + // NOT IMPLEMENTED + VirtualPushStreamIterator(const VirtualPushStreamIterator&); // = delete + VirtualPushStreamIterator& + operator=(const VirtualPushStreamIterator&); // = delete + + public: + // CREATORS + + /// Create a new VirtualStorageIterator from the specified `storage` and + /// pointing at the specified `initialPosition`. + VirtualPushStreamIterator(unsigned int upstreamSubQueueId, + mqbi::Storage* storage, + PushStream* owner, + const PushStream::iterator& initialPosition); + + /// Destructor + virtual ~VirtualPushStreamIterator() BSLS_KEYWORD_OVERRIDE; + + /// Remove the current element (`mqbi::AppMessage`, `upstreamSubQueueId` + /// pair) from the current PUSH GUID. + /// The behavior is undefined unless `atEnd` returns `false`. + void removeCurrentElement(); + + /// Return the number of elements (`mqbi::AppMessage`, `upstreamSubQueueId` + /// pairs) for the current PUSH GUID. + /// The behavior is undefined unless `atEnd` returns `false`. + unsigned int numApps() const; + + /// Return the current element (`mqbi::AppMessage`, `upstreamSubQueueId` + /// pair). + /// The behavior is undefined unless `atEnd` returns `false`. + PushStream::Element* + element(unsigned int appOrdinal) const BSLS_KEYWORD_OVERRIDE; + + // MANIPULATORS + bool advance() BSLS_KEYWORD_OVERRIDE; + + /// Return `true` if this iterator is currently at the end of the items' + /// collection, and hence doesn't reference a valid item. + bool atEnd() const BSLS_KEYWORD_OVERRIDE; +}; + +// FREE OPERATORS +bool operator==(const VirtualPushStreamIterator& lhs, + const VirtualPushStreamIterator& rhs); + +// -------------------------- +// struct PushStream::Element +// -------------------------- +inline PushStream::ElementBase::ElementBase() +: d_next_p(0) +, d_previous_p(0) +{ + // NOTHING +} + +inline PushStream::Element::Element(const bmqp::SubQueueInfo& subscription, + const iterator& iterator, + const Apps::iterator& iteratorApp) +: d_app(subscription.rdaInfo()) +, d_iteratorGuid(iterator) +, d_iteratorApp(iteratorApp) +{ + d_app.d_subscriptionId = subscription.id(); +} + +inline void PushStream::Element::eraseGuid(PushStream::Stream& stream) +{ + stream.erase(d_iteratorGuid); +} + +inline void PushStream::Element::eraseApp(PushStream::Apps& apps) +{ + apps.erase(d_iteratorApp); +} + +inline mqbi::AppMessage* PushStream::Element::appState() +{ + return &d_app; +} + +inline const mqbi::AppMessage* PushStream::Element::appView() const +{ + return &d_app; +} + +inline PushStream::Elements& PushStream::Element::guid() const +{ + return d_iteratorGuid->second; +} + +inline PushStream::App& PushStream::Element::app() const +{ + return d_iteratorApp->second; +} + +inline bool PushStream::Element::isInStream( + const PushStream::Stream::iterator& iterator) const +{ + return d_iteratorGuid != iterator; +} + +inline PushStream::Element* PushStream::Element::next() const +{ + return d_base[e_GUID].d_next_p; +} + +inline PushStream::Element* PushStream::Element::nextInApp() const +{ + return d_base[e_APP].d_next_p; +} + +// --------------------------- +// struct PushStream::Elements +// --------------------------- + +inline PushStream::Elements::Elements() +: d_first_p(0) +, d_last_p(0) +, d_numElements(0) +{ + // NOTHING +} + +inline void PushStream::Elements::onAdd(Element* element) +{ + if (++d_numElements == 1) { + BSLS_ASSERT_SAFE(d_first_p == 0); + BSLS_ASSERT_SAFE(d_last_p == 0); + + d_first_p = element; + d_last_p = element; + } + else { + BSLS_ASSERT_SAFE(d_first_p); + BSLS_ASSERT_SAFE(d_last_p); + + d_last_p = element; + } +} + +inline void PushStream::Elements::onRemove() +{ + BSLS_ASSERT_SAFE(d_numElements); + + if (--d_numElements == 0) { + BSLS_ASSERT_SAFE(d_first_p == 0); + BSLS_ASSERT_SAFE(d_last_p == 0); + } + else { + BSLS_ASSERT_SAFE(d_first_p); + BSLS_ASSERT_SAFE(d_last_p); + } +} + +inline void PushStream::Elements::remove(Element* element, ElementList where) +{ + BSLS_ASSERT_SAFE(element); + + if (d_first_p == element) { + BSLS_ASSERT_SAFE(element->d_base[where].d_previous_p == 0); + + d_first_p = element->d_base[where].d_next_p; + } + else { + BSLS_ASSERT_SAFE(element->d_base[where].d_previous_p); + + element->d_base[where].d_previous_p->d_base[where].d_next_p = + element->d_base[where].d_next_p; + } + + if (d_last_p == element) { + BSLS_ASSERT_SAFE(element->d_base[where].d_next_p == 0); + + d_last_p = element->d_base[where].d_previous_p; + } + else { + BSLS_ASSERT_SAFE(element->d_base[where].d_next_p); + + element->d_base[where].d_next_p->d_base[where].d_previous_p = + element->d_base[where].d_previous_p; + } + + onRemove(); + + element->d_base[where].d_previous_p = element->d_base[where].d_next_p = 0; +} + +inline void PushStream::Elements::add(Element* element, ElementList where) +{ + BSLS_ASSERT_SAFE(element->d_base[where].d_previous_p == 0); + BSLS_ASSERT_SAFE(element->d_base[where].d_next_p == 0); + + element->d_base[where].d_previous_p = d_last_p; + + if (d_last_p) { + BSLS_ASSERT_SAFE(d_last_p->d_base[where].d_next_p == 0); + + d_last_p->d_base[where].d_next_p = element; + } + + onAdd(element); +} + +inline PushStream::Element* PushStream::Elements::front() const +{ + return d_first_p; +} + +inline unsigned int PushStream::Elements::numElements() const +{ + return d_numElements; +} + +inline PushStream::App::App( + const bsl::shared_ptr& app) +: d_elements() +, d_app(app) +{ +} + +inline void PushStream::App::add(Element* element) +{ + d_elements.add(element, e_APP); +} +inline void PushStream::App::remove(Element* element) +{ + d_elements.remove(element, e_APP); +} + +// ------------------ +// struct PushStream +// ----------------- + +inline PushStream::Element* +PushStream::create(const bmqp::SubQueueInfo& subscription, + const iterator& it, + const Apps::iterator& iteratorApp) +{ + BSLS_ASSERT_SAFE(it != d_stream.end()); + + Element* element = new (d_pushElementsPool_sp->allocate()) + Element(subscription, it, iteratorApp); + return element; +} + +inline void PushStream::destroy(Element* element, bool doKeepGuid) +{ + if (element->app().d_elements.numElements() == 0) { + element->eraseApp(d_apps); + } + + if (!doKeepGuid && element->guid().numElements() == 0) { + element->eraseGuid(d_stream); + } + + d_pushElementsPool_sp->deallocate(element); +} + +inline PushStream::iterator +PushStream::findOrAppendMessage(const bmqt::MessageGUID& guid) +{ + return d_stream.insert(bsl::make_pair(guid, Elements())).first; +} + +inline void PushStream::add(Element* element) +{ + // Add to the GUID + BSLS_ASSERT_SAFE(element); + BSLS_ASSERT_SAFE(element->isInStream(d_stream.end())); + + element->guid().add(element, e_GUID); + + // Add to the App + element->app().add(element); +} + +inline unsigned int PushStream::remove(Element* element) +{ + BSLS_ASSERT_SAFE(element); + BSLS_ASSERT_SAFE(element->isInStream(d_stream.end())); + + // remove from the App + element->app().remove(element); + + // remove from the guid + element->guid().remove(element, e_GUID); + + return element->guid().numElements(); +} + +inline unsigned int PushStream::removeApp(unsigned int upstreamSubQueueId) +{ + // remove from the App + Apps::iterator itApp = d_apps.find(upstreamSubQueueId); + + unsigned int numMessages = 0; + if (itApp != d_apps.end()) { + numMessages = removeApp(itApp); + } + + return numMessages; +} + +inline unsigned int PushStream::removeApp(Apps::iterator itApp) +{ + unsigned int numElements = itApp->second.d_elements.numElements(); + for (unsigned int count = 0; count < numElements; ++count) { + Element* element = itApp->second.d_elements.front(); + + remove(element); + + destroy(element, false); + // do not keep Guid + } + + return numElements; +} + +inline unsigned int PushStream::removeAll() +{ + unsigned int numMessages = 0; + + while (!d_apps.empty()) { + numMessages += removeApp(d_apps.begin()); + } + + return numMessages; +} + +} // close package namespace + +} // close enterprise namespace + +#endif diff --git a/src/groups/mqb/mqbblp/mqbblp_pushstream.t.cpp b/src/groups/mqb/mqbblp/mqbblp_pushstream.t.cpp new file mode 100644 index 000000000..19f09b24f --- /dev/null +++ b/src/groups/mqb/mqbblp/mqbblp_pushstream.t.cpp @@ -0,0 +1,214 @@ +// Copyright 2024 Bloomberg Finance L.P. +// SPDX-License-Identifier: Apache-2.0 +// +// 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 +// +// http://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. + +// mqbblp_pushstream.t.cpp -*-C++-*- +#include + +// BMQ +#include + +#include +#include + +#include + +// TEST DRIVER +#include + +// CONVENIENCE +using namespace BloombergLP; +using namespace bsl; + +// ============================================================================ +// TESTS +// ---------------------------------------------------------------------------- + +static void test1_basic() +{ + mwctst::TestHelper::printTestName("PushStream basic test"); + + bdlma::ConcurrentPool pushElementsPool(sizeof(mqbblp::PushStream::Element), + s_allocator_p); + + mqbblp::PushStream ps(&pushElementsPool, s_allocator_p); + unsigned int subQueueId = 0; + bsl::shared_ptr app; // unused + bmqp::SubQueueInfo subscription; + + mqbblp::PushStream::iterator itGuid = ps.findOrAppendMessage( + bmqp::MessageGUIDGenerator::testGUID()); + + mqbblp::PushStream::Apps::iterator itApp = + ps.d_apps.emplace(subQueueId, app).first; + + mqbblp::PushStream::Element* element = ps.create(subscription, + itGuid, + itApp); + + ps.add(element); + ps.remove(element); + ps.destroy(element, false); +} + +static void test2_iterations() +{ + mwctst::TestHelper::printTestName("PushStream basic test"); + + // Imitate {m1, a1}, {m2, a2}, {m1, a2}, {m2, a1} + + mqbblp::PushStream ps(bsl::nullptr_t(), s_allocator_p); + unsigned int subQueueId1 = 1; + unsigned int subQueueId2 = 2; + + bsl::shared_ptr unused; + + bmqp::SubQueueInfo subscription1(1); + bmqp::SubQueueInfo subscription2(2); + + mqbblp::PushStream::iterator itGuid1 = ps.findOrAppendMessage( + bmqp::MessageGUIDGenerator::testGUID()); + + mqbblp::PushStream::Apps::iterator itApp1 = + ps.d_apps.emplace(subQueueId1, unused).first; + + mqbblp::PushStream::Element* element1 = ps.create(subscription1, + itGuid1, + itApp1); + + ps.add(element1); + + mqbblp::PushStream::iterator itGuid2 = ps.findOrAppendMessage( + bmqp::MessageGUIDGenerator::testGUID()); + + mqbblp::PushStream::Apps::iterator itApp2 = + ps.d_apps.emplace(subQueueId2, unused).first; + + mqbblp::PushStream::Element* element2 = ps.create(subscription2, + itGuid2, + itApp2); + + ps.add(element2); + + mqbblp::PushStream::Element* element3 = ps.create(subscription2, + itGuid1, + itApp2); + + ps.add(element3); + + mqbblp::PushStream::Element* element4 = ps.create(subscription1, + itGuid2, + itApp1); + + ps.add(element4); + + mqbu::CapacityMeter dummyCapacityMeter("dummy", s_allocator_p); + bmqt::Uri dummyUri("dummy", s_allocator_p); + mqbconfm::Domain dummyDomain(s_allocator_p); + + mqbs::InMemoryStorage dummyStorage(dummyUri, + mqbu::StorageKey::k_NULL_KEY, + mqbs::DataStore::k_INVALID_PARTITION_ID, + dummyDomain, + &dummyCapacityMeter, + s_allocator_p); + + mqbconfm::Storage config; + mqbconfm::Limits limits; + + config.makeInMemory(); + + limits.messages() = bsl::numeric_limits::max(); + limits.bytes() = bsl::numeric_limits::max(); + + mwcu::MemOutStream errorDescription(s_allocator_p); + dummyStorage.configure(errorDescription, + config, + limits, + bsl::numeric_limits::max(), + 0); + + { + mqbblp::PushStreamIterator pit(&dummyStorage, + &ps, + ps.d_stream.begin()); + + ASSERT(!pit.atEnd()); + ASSERT_EQ(pit.numApps(), 2); + + ASSERT_EQ(element1, pit.element(0)); + ASSERT_EQ(element3, pit.element(1)); + + ASSERT(pit.advance()); + ASSERT_EQ(pit.numApps(), 2); + + ASSERT_EQ(element2, pit.element(0)); + ASSERT_EQ(element4, pit.element(1)); + + ASSERT(!pit.advance()); + } + + { + mqbblp::VirtualPushStreamIterator vit(subQueueId1, + &dummyStorage, + &ps, + ps.d_stream.begin()); + + ASSERT(!vit.atEnd()); + ASSERT_EQ(vit.numApps(), 1); + + ASSERT_EQ(element1, vit.element(0)); + + vit.advance(); + + ASSERT(!vit.atEnd()); + ASSERT_EQ(vit.numApps(), 1); + + ASSERT_EQ(element4, vit.element(0)); + + vit.advance(); + + ASSERT(vit.atEnd()); + } + + ps.remove(element2); + ps.destroy(element2, false); + ps.remove(element3); + ps.destroy(element3, false); +} + +// ============================================================================ +// MAIN PROGRAM +// ---------------------------------------------------------------------------- + +int main(int argc, char* argv[]) +{ + TEST_PROLOG(mwctst::TestHelper::e_DEFAULT); + + bmqt::UriParser::initialize(s_allocator_p); + + switch (_testCase) { + case 0: + case 1: test1_basic(); break; + case 2: test2_iterations(); break; + default: { + cerr << "WARNING: CASE '" << _testCase << "' NOT FOUND." << endl; + s_testStatus = -1; + } break; + } + + bmqt::UriParser::shutdown(); + + TEST_EPILOG(mwctst::TestHelper::e_CHECK_DEF_GBL_ALLOC); +} diff --git a/src/groups/mqb/mqbblp/mqbblp_queue.cpp b/src/groups/mqb/mqbblp/mqbblp_queue.cpp index 1834eeb35..b544b9a46 100644 --- a/src/groups/mqb/mqbblp/mqbblp_queue.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_queue.cpp @@ -454,14 +454,13 @@ Queue::Queue(const bmqt::Uri& uri, int partitionId, mqbi::Domain* domain, mqbi::StorageManager* storageManager, - bdlbb::BlobBufferFactory* blobBufferFactory, - bdlmt::EventScheduler* scheduler, + const mqbi::ClusterResources& resources, bdlmt::FixedThreadPool* threadPool, const bmqp_ctrlmsg::RoutingConfiguration& routingCfg, bslma::Allocator* allocator) : d_allocator_p(allocator) , d_schemaLearner(allocator) -, d_state(this, uri, id, key, partitionId, domain, allocator) +, d_state(this, uri, id, key, partitionId, domain, resources, allocator) , d_localQueue_mp(0) , d_remoteQueue_mp(0) { @@ -486,8 +485,6 @@ Queue::Queue(const bmqt::Uri& uri, d_state.setStorageManager(storageManager) .setAppKeyGenerator(storageManager) - .setBlobBufferFactory(blobBufferFactory) - .setEventScheduler(scheduler) .setMiscWorkThreadPool(threadPool) .setRoutingConfig(routingCfg) .setMessageThrottleConfig(messageThrottleConfig); diff --git a/src/groups/mqb/mqbblp/mqbblp_queue.h b/src/groups/mqb/mqbblp/mqbblp_queue.h index 20abac452..12a053a80 100644 --- a/src/groups/mqb/mqbblp/mqbblp_queue.h +++ b/src/groups/mqb/mqbblp/mqbblp_queue.h @@ -166,8 +166,7 @@ class Queue BSLS_CPP11_FINAL : public mqbi::Queue { int partitionId, mqbi::Domain* domain, mqbi::StorageManager* storageManager, - bdlbb::BlobBufferFactory* blobBufferFactory, - bdlmt::EventScheduler* scheduler, + const mqbi::ClusterResources& resources, bdlmt::FixedThreadPool* threadPool, const bmqp_ctrlmsg::RoutingConfiguration& routingCfg, bslma::Allocator* allocator); diff --git a/src/groups/mqb/mqbblp/mqbblp_queueconsumptionmonitor.t.cpp b/src/groups/mqb/mqbblp/mqbblp_queueconsumptionmonitor.t.cpp index c9061a16f..6ecaed15e 100644 --- a/src/groups/mqb/mqbblp/mqbblp_queueconsumptionmonitor.t.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_queueconsumptionmonitor.t.cpp @@ -151,6 +151,7 @@ Test::Test() d_storageKey, d_partitionId, &d_domain, + d_cluster._resources(), s_allocator_p) , d_monitor(&d_queueState, s_allocator_p) , d_storage(d_queue.uri(), @@ -158,7 +159,6 @@ Test::Test() mqbs::DataStore::k_INVALID_PARTITION_ID, getDomainConfig(), d_domain.capacityMeter(), - bmqp::RdaInfo(), s_allocator_p) , d_advance(s_allocator_p) , d_clientId(0) @@ -236,11 +236,7 @@ void Test::putMessage() 0, bslma::ManagedPtrUtil::noOpDeleter); - ASSERT_EQ(d_storage.put(&messageAttributes, - messageGUID, - appData, - options, - mqbi::Storage::StorageKeys()), + ASSERT_EQ(d_storage.put(&messageAttributes, messageGUID, appData, options), mqbi::StorageResult::e_SUCCESS); } diff --git a/src/groups/mqb/mqbblp/mqbblp_queueenginetester.cpp b/src/groups/mqb/mqbblp/mqbblp_queueenginetester.cpp index 7011205f6..afbff91de 100644 --- a/src/groups/mqb/mqbblp/mqbblp_queueenginetester.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_queueenginetester.cpp @@ -381,28 +381,16 @@ QueueEngineTester::QueueEngineTester(const mqbconfm::Domain& domainConfig, , d_subQueueIdCounter(bmqimp::QueueManager::k_INITIAL_SUBQUEUE_ID) , d_deletedHandles(allocator) , d_messageCount(0) -, d_scheduler(bsls::SystemClockType::e_MONOTONIC, allocator) , d_allocator_p(allocator) { oneTimeInit(); - const int maxDeliveryAttempts = domainConfig.maxDeliveryAttempts(); + mqbconfm::Domain config = domainConfig; - // A value of zero represents unlimited delivery attempts. - if (maxDeliveryAttempts == 0 || - maxDeliveryAttempts > bmqp::RdaInfo::k_MAX_COUNTER_VALUE) { - d_rdaInfo = bmqp::RdaInfo(); // Default constructed is unlimited - } - else { - d_rdaInfo = bmqp::RdaInfo().setCounter(maxDeliveryAttempts); - } + config.deduplicationTimeMs() = 0; // No history + config.messageTtl() = k_MAX_MESSAGES; - init(domainConfig); - // Scheduler - if (startScheduler) { - int rc = d_scheduler.start(); - BSLS_ASSERT_OPT(rc == 0); - } + init(config, startScheduler); } QueueEngineTester::~QueueEngineTester() @@ -413,7 +401,7 @@ QueueEngineTester::~QueueEngineTester() d_deletedHandles.clear(); d_mockDomain_mp->unregisterQueue(d_mockQueue_sp.get()); - d_scheduler.stop(); + d_mockCluster_mp->stop(); oneTimeShutdown(); } @@ -428,7 +416,8 @@ void QueueEngineTester::oneTimeInit() } } -void QueueEngineTester::init(const mqbconfm::Domain& domainConfig) +void QueueEngineTester::init(const mqbconfm::Domain& domainConfig, + bool startScheduler) { mwcu::MemOutStream errorDescription(d_allocator_p); int rc = 0; @@ -452,6 +441,11 @@ void QueueEngineTester::init(const mqbconfm::Domain& domainConfig) BSLS_ASSERT_OPT(d_mockCluster_mp->isClusterMember()); + if (startScheduler) { + rc = d_mockCluster_mp->start(errorDescription); + BSLS_ASSERT_OPT(rc == 0); + } + // Domain d_mockDomain_mp.load(new (*d_allocator_p) mqbmock::Domain(d_mockCluster_mp.get(), @@ -502,11 +496,11 @@ void QueueEngineTester::init(const mqbconfm::Domain& domainConfig) k_NULL_QUEUE_KEY, k_PARTITION_ID, d_mockDomain_mp.get(), + d_mockCluster_mp->_resources(), d_allocator_p), d_allocator_p); d_queueState_mp->setAppKeyGenerator(&d_appKeyGenerator); - d_queueState_mp->setEventScheduler(&d_scheduler); bmqp_ctrlmsg::RoutingConfiguration routingConfig; @@ -532,17 +526,13 @@ void QueueEngineTester::init(const mqbconfm::Domain& domainConfig) d_queueState_mp->setRoutingConfig(routingConfig); // Create Storage - mqbconfm::Domain domainCfg; - domainCfg.deduplicationTimeMs() = 0; // No history is maintained at proxy - domainCfg.messageTtl() = k_MAX_MESSAGES; mqbi::Storage* storage_p = new (*d_allocator_p) mqbs::InMemoryStorage(d_mockQueue_sp->uri(), k_NULL_QUEUE_KEY, - mqbs::DataStore::k_INVALID_PARTITION_ID, - domainCfg, + k_PARTITION_ID, + domainConfig, d_mockDomain_mp->capacityMeter(), - d_rdaInfo, d_allocator_p); mqbconfm::Storage config; @@ -555,8 +545,8 @@ void QueueEngineTester::init(const mqbconfm::Domain& domainConfig) rc = storage_p->configure(errorDescription, config, limits, - domainCfg.messageTtl(), - domainCfg.maxDeliveryAttempts()); + domainConfig.messageTtl(), + domainConfig.maxDeliveryAttempts()); BSLS_ASSERT_OPT(rc == 0 && "storage configure fail"); // Add virtual storages @@ -580,8 +570,16 @@ void QueueEngineTester::init(const mqbconfm::Domain& domainConfig) BSLS_ASSERT_SAFE(rc == 0); } } + else { + rc = storage_p->addVirtualStorage( + errorDescription, + bmqp::ProtocolUtil::k_DEFAULT_APP_ID, + mqbi::QueueEngine::k_DEFAULT_APP_KEY); + BSLS_ASSERT_SAFE(rc == 0); + } d_mockQueue_sp->_setStorage(storage_p); + storage_p->setQueue(d_mockQueue_sp.get()); bslma::ManagedPtr storage_mp(storage_p, d_allocator_p); d_queueState_mp->setStorage(storage_mp); @@ -598,24 +596,6 @@ void QueueEngineTester::init(const mqbconfm::Domain& domainConfig) const_cast(d_queueState_mp->handleCatalog()) .setHandleFactory(handleFactory_mp); - - d_subStreamMessages_mp.load(new (*d_allocator_p) - mqbs::VirtualStorageCatalog(storage_p, - d_allocator_p), - d_allocator_p); - - if (!isFanoutMode) { - rc = d_subStreamMessages_mp->addVirtualStorage( - errorDescription, - bmqp::ProtocolUtil::k_DEFAULT_APP_ID, - mqbi::QueueEngine::k_DEFAULT_APP_KEY); - BSLS_ASSERT_SAFE(rc == 0); - rc = storage_p->addVirtualStorage( - errorDescription, - bmqp::ProtocolUtil::k_DEFAULT_APP_ID, - mqbi::QueueEngine::k_DEFAULT_APP_KEY); - BSLS_ASSERT_SAFE(rc == 0); - } } void QueueEngineTester::oneTimeShutdown() @@ -893,6 +873,8 @@ int QueueEngineTester::configureHandle(const bsl::string& clientText) streamParams.appId() = appId; // Temporary + // Assume, RelayQueueEngine will use upstreramSubQueueIds as the + // subscriptionIds. This needs to be in accord with the 'post' logic. BSLS_ASSERT_SAFE(streamParams.subscriptions().size() == 1); streamParams.subscriptions()[0].sId() = subIdCiter->second; @@ -926,7 +908,8 @@ QueueEngineTester::client(const bslstl::StringRef& clientKey) return d_handles.find(clientKey)->second; } -void QueueEngineTester::post(const bslstl::StringRef& messages) +void QueueEngineTester::post(const bslstl::StringRef& messages, + RelayQueueEngine* downstream) { // PRECONDITIONS BSLS_ASSERT_OPT(d_queueEngine_mp && @@ -935,6 +918,24 @@ void QueueEngineTester::post(const bslstl::StringRef& messages) bsl::vector msgs(d_allocator_p); parseMessages(&msgs, messages); + bmqp::Protocol::SubQueueInfosArray subscriptions(d_allocator_p); + + if (d_subIds.empty()) { + subscriptions.push_back( + bmqp::SubQueueInfo(bmqp::Protocol::k_DEFAULT_SUBSCRIPTION_ID)); + } + else { + // Assume, RelayQueueEngine will use upstreamSubQueueIds as the + // subscriptionIds. + // This needs to be in accord with the 'configureHandle' logic. + + for (SubIdsMap::const_iterator cit = d_subIds.cbegin(); + cit != d_subIds.cend(); + ++cit) { + subscriptions.push_back(bmqp::SubQueueInfo(cit->second)); + } + } + for (unsigned int i = 0; i < msgs.size(); ++i) { // Put in storage bmqt::MessageGUID msgGUID; @@ -954,19 +955,21 @@ void QueueEngineTester::post(const bslstl::StringRef& messages) msgs[i].data(), msgs[i].length()); + // Consider this non-Proxy. Imitate replication or Primary PUT + // ('d_mockCluster_mp->_setIsClusterMember(true)') + int rc = d_queueState_mp->storage()->put(&msgAttributes, msgGUID, appData, options); BSLS_ASSERT_OPT((rc == 0) && "Storage put failure"); - if (d_subStreamMessages_mp) { - d_subStreamMessages_mp->put( - msgGUID, - appData->length(), - d_rdaInfo, - bmqp::Protocol::k_DEFAULT_SUBSCRIPTION_ID, - mqbu::StorageKey::k_NULL_KEY); + if (downstream) { + downstream->push(&msgAttributes, + msgGUID, + bsl::shared_ptr(), + subscriptions, + false); } // Insert into messages maps @@ -1190,13 +1193,6 @@ void QueueEngineTester::purgeQueue(const bslstl::StringRef& appId) } } - // Remove all messages from the virtual storages, if applicable - if (d_subStreamMessages_mp && !isFanout) { - rc = d_subStreamMessages_mp->removeAll(appKey); - BSLS_ASSERT_OPT((rc == mqbi::StorageResult::e_SUCCESS) && - "'msgGUID' was not found in virtual storage"); - } - // Remove all messages from physical/in-memory storage rc = d_queueState_mp->storage()->removeAll(appKey); BSLS_ASSERT_OPT((rc == mqbi::StorageResult::e_SUCCESS) && diff --git a/src/groups/mqb/mqbblp/mqbblp_queueenginetester.h b/src/groups/mqb/mqbblp/mqbblp_queueenginetester.h index 1ad211b90..0f2296c92 100644 --- a/src/groups/mqb/mqbblp/mqbblp_queueenginetester.h +++ b/src/groups/mqb/mqbblp/mqbblp_queueenginetester.h @@ -201,12 +201,6 @@ class QueueEngineTester { bslma::ManagedPtr d_queueEngine_mp; // Queue Engine being tested - bslma::ManagedPtr d_subStreamMessages_mp; - - bmqp::RdaInfo d_rdaInfo; - // rdaInfo valuef for all posted - // messages - HandleMap d_handles; // Map of all created handles. Note // that a handle in this map is owned @@ -248,8 +242,6 @@ class QueueEngineTester { size_t d_messageCount; - bdlmt::EventScheduler d_scheduler; - bslma::Allocator* d_allocator_p; // Allocator to use @@ -268,7 +260,7 @@ class QueueEngineTester { /// Reset and recreate all objects using the currently set options and /// the specific `domainConfig`. - void init(const mqbconfm::Domain& domainConfig); + void init(const mqbconfm::Domain& domainConfig, bool startScheduler); /// Pendant operation of the `oneTimeInit` one. void oneTimeShutdown(); @@ -325,7 +317,7 @@ class QueueEngineTester { /// Finally note that the behavior of all other methods in this class is /// undefined unless this method was called exactly once. template - mqbi::QueueEngine* createQueueEngine(); + T* createQueueEngine(); /// Obtain and return a handle for the client with handle parameters in /// the specified `clientText` per the following format: @@ -363,12 +355,15 @@ class QueueEngineTester { /// with the queue engine under test. The format of `messages` must be /// as follows: /// `,,...,` - /// + /// If the optionally specified 'downstream' is not zero, consider it as + /// representing 'RelayQueueEngine' involved in the data delivery and + /// invoke its 'push' method for each message. /// The order of posting each message is from left to right per the /// format above. The behavior is undefined unless `messages` is /// formatted as above and each message is unique (across the lifetime /// of this object), or if `createQueueEngine()` was not called. - void post(const bslstl::StringRef& messages); + void post(const bslstl::StringRef& messages, + RelayQueueEngine* downstream = 0); /// Invoke the Queue Engine's `afterNewMessage()` method for the /// specified `numMessages` newly posted messages if `numMessages > 0`, @@ -496,6 +491,7 @@ class QueueEngineTesterGuard { private: // DATA QueueEngineTester* d_tester_p; + QUEUE_ENGINE_TYPE* d_engine_p; // an alias private: // NOT IMPLEMENTED @@ -519,7 +515,7 @@ class QueueEngineTesterGuard { public: // MANIPULATORS - /// Return the address of the modifiable QueueEngineTester object under + /// Return the address of the modifiable QueueEngine object under /// management by this proctor, and release the tester from further /// management by this proctor. If no tester is currently being /// managed, return 0 with no other effect. Note that this operation @@ -529,10 +525,10 @@ class QueueEngineTesterGuard { // ACCESSORS - /// Return the address of the modifiable QueueEngineTester object under - /// management by this proctor, or 0 if no QueueEngineTester is - /// currently being managed. - QueueEngineTester* ptr() const; + /// Return the address of the modifiable QueueEngine object under + /// management by this proctor, or 0 if no QueueEngine is currently being + /// managed. + QUEUE_ENGINE_TYPE* engine() const; }; // ========================== @@ -588,23 +584,20 @@ struct TestClock { class TimeControlledQueueEngineTester : public mqbblp::QueueEngineTester { private: - bdlmt::EventSchedulerTestTimeSource d_timeSource; - TestClock d_testClock; + TestClock d_testClock; public: // CREATORS TimeControlledQueueEngineTester(const mqbconfm::Domain& domainConfig, bslma::Allocator* allocator) - : mqbblp::QueueEngineTester(domainConfig, false, allocator) - , d_timeSource(&d_scheduler) - , d_testClock(d_timeSource) + : mqbblp::QueueEngineTester(domainConfig, true, allocator) + , d_testClock(d_mockCluster_mp->_timeSource()) { mwcsys::Time::shutdown(); mwcsys::Time::initialize( bdlf::BindUtil::bind(&TestClock::realtimeClock, &d_testClock), bdlf::BindUtil::bind(&TestClock::monotonicClock, &d_testClock), bdlf::BindUtil::bind(&TestClock::highResTimer, &d_testClock)); - d_scheduler.start(); } // MANIPULATORS @@ -642,37 +635,20 @@ QueueEngineTester::createQueueEngineHelper(mqbi::QueueEngine* engine) // MANIPULATORS template -inline mqbi::QueueEngine* QueueEngineTester::createQueueEngine() +inline T* QueueEngineTester::createQueueEngine() { // PRECONDITIONS BSLS_ASSERT_OPT(!d_queueEngine_mp && "'createQueueEngine()' was called"); - + T* result = new (*d_allocator_p) + T(d_queueState_mp.get(), + d_queueState_mp->queue()->domain()->config(), + d_allocator_p); // Create and configure Queue Engine - d_queueEngine_mp.load(new (*d_allocator_p) - T(d_queueState_mp.get(), - d_queueState_mp->queue()->domain()->config(), - d_allocator_p), - d_allocator_p); + d_queueEngine_mp.load(result, d_allocator_p); - return createQueueEngineHelper(d_queueEngine_mp.get()); -} + createQueueEngineHelper(d_queueEngine_mp.get()); -template <> -inline mqbi::QueueEngine* -QueueEngineTester::createQueueEngine() -{ - // PRECONDITIONS - BSLS_ASSERT_OPT(!d_queueEngine_mp && "'createQueueEngine()' was called"); - - // Create and configure Queue Engine - d_queueEngine_mp.load(new (*d_allocator_p) - RelayQueueEngine(d_queueState_mp.get(), - d_subStreamMessages_mp.get(), - mqbconfm::Domain(), - d_allocator_p), - d_allocator_p); - - return createQueueEngineHelper(d_queueEngine_mp.get()); + return result; } inline mqbmock::AppKeyGenerator& QueueEngineTester::appKeyGenerator() @@ -682,14 +658,7 @@ inline mqbmock::AppKeyGenerator& QueueEngineTester::appKeyGenerator() inline void QueueEngineTester::synchronizeScheduler() { - bslmt::Semaphore semaphore; - - d_scheduler.scheduleEvent( - mwcsys::Time::nowMonotonicClock(), - bdlf::MemFnUtil::memFn( - static_cast(&bslmt::Semaphore::post), - &semaphore)); - semaphore.wait(); + d_mockCluster_mp->waitForScheduler(); } // ---------------------- // QueueEngineTesterGuard @@ -700,9 +669,10 @@ template inline QueueEngineTesterGuard::QueueEngineTesterGuard( QueueEngineTester* queueEngineTester) : d_tester_p(queueEngineTester) +, d_engine_p(0) { if (d_tester_p) { - d_tester_p->createQueueEngine(); + d_engine_p = d_tester_p->createQueueEngine(); } } @@ -712,6 +682,7 @@ inline QueueEngineTesterGuard::~QueueEngineTesterGuard() if (d_tester_p) { d_tester_p->dropHandles(); } + d_engine_p = 0; } // MANIPULATORS @@ -720,15 +691,16 @@ inline QueueEngineTester* QueueEngineTesterGuard::release() { QueueEngineTester* tester = d_tester_p; d_tester_p = 0; + d_engine_p = 0; return tester; } // ACCESSORS template -inline QueueEngineTester* -QueueEngineTesterGuard::ptr() const +inline QUEUE_ENGINE_TYPE* +QueueEngineTesterGuard::engine() const { - return d_tester_p; + return d_engine_p; } // ------------------------------- @@ -738,7 +710,8 @@ QueueEngineTesterGuard::ptr() const inline void TimeControlledQueueEngineTester::advanceTime(const bsls::TimeInterval& step) { - d_timeSource.advanceTime(step); + d_mockCluster_mp->advanceTime(step); + synchronizeScheduler(); } diff --git a/src/groups/mqb/mqbblp/mqbblp_queueengineutil.cpp b/src/groups/mqb/mqbblp/mqbblp_queueengineutil.cpp index 61617db15..41465337b 100644 --- a/src/groups/mqb/mqbblp/mqbblp_queueengineutil.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_queueengineutil.cpp @@ -48,6 +48,7 @@ // BDE #include +#include #include #include #include @@ -70,6 +71,17 @@ const bsls::Types::Int64 k_MAX_SECONDS = const int k_MAX_NANOSECONDS = 999999999; +const int k_MAX_INSTANT_MESSAGES = 10; +// Maximum messages logged with throttling in a short period of time. + +const bsls::Types::Int64 k_NS_PER_MESSAGE = + bdlt::TimeUnitRatio::k_NANOSECONDS_PER_MINUTE / k_MAX_INSTANT_MESSAGES; +// Time interval between messages logged with throttling. + +#define BMQ_LOGTHROTTLE_INFO() \ + BALL_LOGTHROTTLE_INFO(k_MAX_INSTANT_MESSAGES, k_NS_PER_MESSAGE) \ + << "[THROTTLED] " + /// Dummy method enqueued to the associated client's dispatcher thread when /// the specified `handle` was dropped and deleted without providing a /// `releasedCb`, in order to delay its destruction until after the client's @@ -622,139 +634,130 @@ QueueEngineUtil_AppsDeliveryContext::QueueEngineUtil_AppsDeliveryContext( mqbi::Queue* queue, bslma::Allocator* allocator) : d_consumers(allocator) -, d_doRepeat(true) +, d_isReady(false) , d_currentMessage(0) , d_queue_p(queue) -, d_activeAppIds(allocator) +, d_timeDelta() { - // NOTHING + BSLS_ASSERT_SAFE(queue); } -void QueueEngineUtil_AppsDeliveryContext::reset() +void QueueEngineUtil_AppsDeliveryContext::start() +{ + d_isReady = true; +} + +bool QueueEngineUtil_AppsDeliveryContext::reset( + mqbi::StorageIterator* currentMessage) { - d_doRepeat = false; - d_currentMessage = 0; d_consumers.clear(); - d_activeAppIds.clear(); + d_timeDelta.reset(); + + if (!d_isReady) { + return false; // RETURN + } + + d_currentMessage = currentMessage; + d_isReady = false; + + return d_currentMessage ? d_currentMessage->hasReceipt() : false; } bool QueueEngineUtil_AppsDeliveryContext::processApp( - QueueEngineUtil_AppState& app) + QueueEngineUtil_AppState& app, + unsigned int ordinal) { - // For each App: - // 1. Pick next message to send (next message of first 'not at the end' - // subStream). - // 2. Accumulate subQueueIds of all subStreams if their next message is - // the same. - // 3. Deliver the message. - // 4. Advance all affected subStreams. - // 5. If there was a different message somewhere or if an advance - // indicated there are other messages, repeat. - - BSLS_ASSERT_SAFE(app.d_storageIter_mp); - - if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( - !app.d_storageIter_mp->hasReceipt())) { - // An unregistered App Id has void iterator that always points - // at the end. - // Or, still waiting for Receipt - BSLS_PERFORMANCEHINT_UNLIKELY_HINT; + BSLS_ASSERT_SAFE(d_currentMessage->hasReceipt()); + + if (d_queue_p->isDeliverAll()) { + // collect all handles + app.routing()->iterateConsumers( + bdlf::BindUtil::bind( + &QueueEngineUtil_AppsDeliveryContext::visitBroadcast, + this, + bdlf::PlaceHolders::_1), + d_currentMessage); + + d_isReady = true; + + // Broadcast does not need stats nor any special per-message treatment. return false; // RETURN } - if (d_currentMessage) { - if (d_currentMessage->guid() != app.d_storageIter_mp->guid()) { - // This app needs to deliver another message. It will be picked up - // in next iteration(s). - d_doRepeat = true; - return false; // RETURN + + if (!app.isReadyForDelivery()) { + if (app.resumePoint().isUnset() && app.isAuthorized()) { + // The 'app' needs to resume (in 'deliverMessages'). + // The queue iterator can advance leaving the 'app' behind. + app.setResumePoint(d_currentMessage->guid()); } + // else the existing resumePoint is earlier (if authorized) + return false; // RETURN } - // This app has a message to deliver that is either the first - // message in the current iteration or an identical message. - - const bool isBroadcast = d_queue_p->isDeliverAll(); - bool isSelected = false; - if (!isBroadcast) { - // non-broadcast - Routers::Result result = app.selectConsumer( - bdlf::BindUtil::bind(&QueueEngineUtil_AppsDeliveryContext::visit, - this, - bdlf::PlaceHolders::_1, - app.d_storageIter_mp.get()), - app.d_storageIter_mp.get()); + const mqbi::AppMessage& appView = d_currentMessage->appMessageView( + ordinal); - if (result == Routers::e_SUCCESS) { - isSelected = true; - } - else if (result == Routers::e_NO_CAPACITY_ALL) { - // Do not want to advance the iterator - return false; // RETURN - } - else { - // NOTE: Above, we do not call 'StorageIterator::appData' - // associated with the message. This is being done so that we - // don't end up aliasing the corresponding message and options area - // to the mapped file, which will otherwise increment the aliased - // counter of the mapped file, which can delay the unmapping of - // files in case this message has a huge TTL and there are no - // consumers for this message. Also see internal-ticket D164392124 - // for additional context. - - // This app does not have capacity to deliver - app.d_putAsideList.add(app.d_storageIter_mp->guid()); - } - } - else { - isSelected = true; + if (!appView.isNew()) { + d_isReady = true; + return true; // RETURN } - // This app has a message to deliver. It is not at the end, has - // capacity (if it is a non-broadcast). - if (0 == d_currentMessage && isSelected) { - // This is the first message to deliver in the current Apps iteration - d_currentMessage = app.d_storageIter_mp.get(); - // Do not advance current; will need to call messageIterator->appData() - } - else if (app.d_storageIter_mp->advance()) { - // Advance the storageIter of this subStream. Ideally this should be - // done *after* 'QueueHandle::deliverMessage' has been invoked, but - // there is no failure path b/w here and that code (apart from the fact - // that 'getMessageDetails' could fail, at which point, it is probably - // a good idea to advance the iterator since that message cannot be - // recovered from the storage). + // NOTE: We avoid calling 'StorageIterator::appData'associated with the + // message unless necessary. This is being done so that we don't end + // up aliasing the corresponding message and options area to the mapped + // file, which will otherwise increment the aliased counter of the + // mapped file, which can delay the unmapping of files in case this + // message has a huge TTL and there are no consumers for this message. - // There is at least one more message to deliver - d_doRepeat = true; + Routers::Result result = app.selectConsumer( + bdlf::BindUtil::bind(&QueueEngineUtil_AppsDeliveryContext::visit, + this, + bdlf::PlaceHolders::_1, + appView), + d_currentMessage, + ordinal); + + if (result == Routers::e_SUCCESS) { + // RootQueueEngine makes stat reports } + else if (result == Routers::e_NO_CAPACITY_ALL) { + // All subscriptions of thes App are at capacity + // Do not grow the 'd_putAsideList' + // Instead, wait for 'onHandleUsable' event and then catch up + // from this resume point. - if (isBroadcast) { - // collect all handles - app.d_routing_sp->iterateConsumers( - bdlf::BindUtil::bind( - &QueueEngineUtil_AppsDeliveryContext::visitBroadcast, - this, - bdlf::PlaceHolders::_1), - app.d_storageIter_mp.get()); + app.setResumePoint(d_currentMessage->guid()); + + // Early return. + // If all Apps return 'e_NO_CAPACITY_ALL', stop the iteration + // (d_isReady == false). + + return false; // RETURN } else { - // Store appId of active consumer for domain stats (reporting queue - // time metric) - d_activeAppIds.push_back(app.d_appId); + BSLS_ASSERT_SAFE(result == Routers::e_NO_SUBSCRIPTION || + result == Routers::e_NO_CAPACITY); + + // This app does not have capacity to deliver. Still, move on and + // consider (evaluate) subsequent messages for the 'app'. + app.putAside(d_currentMessage->guid()); } - return isSelected; + // Still making progress (result != Routers::e_NO_CAPACITY_ALL) + d_isReady = true; + + return (result == Routers::e_SUCCESS); } bool QueueEngineUtil_AppsDeliveryContext::visit( const Routers::Subscription* subscription, - const mqbi::StorageIterator* message) + const mqbi::AppMessage& appView) { BSLS_ASSERT_SAFE(subscription); d_consumers[subscription->handle()].push_back( bmqp::SubQueueInfo(subscription->d_downstreamSubscriptionId, - message->rdaInfo())); + appView.d_rdaInfo)); return true; } @@ -772,7 +775,9 @@ bool QueueEngineUtil_AppsDeliveryContext::visitBroadcast( void QueueEngineUtil_AppsDeliveryContext::deliverMessage() { - if (d_currentMessage) { + BSLS_ASSERT_SAFE(d_currentMessage); + + if (!d_consumers.empty()) { const mqbi::StorageMessageAttributes& attributes = d_currentMessage->attributes(); for (Consumers::const_iterator it = d_consumers.begin(); @@ -796,33 +801,26 @@ void QueueEngineUtil_AppsDeliveryContext::deliverMessage() false); } } + } - if (bmqp::QueueId::k_PRIMARY_QUEUE_ID == d_queue_p->id()) { - const bsls::Types::Int64 timeDelta = getMessageQueueTime( - attributes); - - // First report 'queue time' metric for the entire queue - d_queue_p->stats()->onEvent( - mqbstat::QueueStatsDomain::EventType::e_QUEUE_TIME, - timeDelta); - - // Then report 'queue time' metric for all active appIds - bsl::vector::const_iterator it = - d_activeAppIds.begin(); - for (; it != d_activeAppIds.cend(); ++it) { - d_queue_p->stats()->onEvent( - mqbstat::QueueStatsDomain::EventType::e_QUEUE_TIME, - timeDelta, - *it // appId - ); - } - } + if (d_isReady) { + d_currentMessage->advance(); + } - if (d_currentMessage->advance()) { - // There is at least one more message to deliver - d_doRepeat = true; - } + d_currentMessage = 0; +} + +bool QueueEngineUtil_AppsDeliveryContext::isEmpty() const +{ + return d_consumers.empty(); +} + +bsls::Types::Int64 QueueEngineUtil_AppsDeliveryContext::timeDelta() +{ + if (!d_timeDelta.has_value()) { + d_timeDelta = getMessageQueueTime(d_currentMessage->attributes()); } + return d_timeDelta.value(); } // ------------------------- @@ -831,28 +829,26 @@ void QueueEngineUtil_AppsDeliveryContext::deliverMessage() // CREATORS QueueEngineUtil_AppState::QueueEngineUtil_AppState( - bslma::ManagedPtr iterator, - mqbi::Queue* queue, - bdlmt::EventScheduler* scheduler, - bool isAuthorized, - Routers::QueueRoutingContext& queueContext, - unsigned int upstreamSubQueueId, - const bsl::string& appId, - const mqbu::StorageKey& appKey, - bslma::Allocator* allocator) -: d_storageIter_mp(iterator) -, d_routing_sp(new (*allocator) Routers::AppContext(queueContext, allocator), + mqbi::Queue* queue, + bdlmt::EventScheduler* scheduler, + Routers::QueueRoutingContext& queueContext, + unsigned int upstreamSubQueueId, + const bsl::string& appId, + const mqbu::StorageKey& appKey, + bslma::Allocator* allocator) +: d_routing_sp(new(*allocator) Routers::AppContext(queueContext, allocator), allocator) , d_redeliveryList(allocator) , d_putAsideList(allocator) , d_priorityCount(0) , d_queue_p(queue) -, d_isAuthorized(isAuthorized) +, d_isAuthorized(false) , d_scheduler_p(scheduler) , d_appKey(appKey) , d_appId(appId) , d_upstreamSubQueueId(upstreamSubQueueId) , d_isScheduled(false) +, d_appOrdinal(mqbi::Storage::k_INVALID_ORDINAL) { // Above, we retrieve domain config from 'queue' only if self node is a // cluster member, and pass a dummy config if self is proxy, because proxy @@ -882,75 +878,83 @@ QueueEngineUtil_AppState::~QueueEngineUtil_AppState() } size_t -QueueEngineUtil_AppState::deliverMessages(bsls::TimeInterval* delay, - const mqbu::StorageKey& appKey, - mqbi::Storage& storage, - const bsl::string& appId) +QueueEngineUtil_AppState::deliverMessages(bsls::TimeInterval* delay, + mqbi::StorageIterator* reader, + mqbi::StorageIterator* start, + const mqbi::StorageIterator* end) { // executed by the *QUEUE DISPATCHER* thread + // PRECONDITIONS + BSLS_ASSERT_SAFE(delay); + BSLS_ASSERT_SAFE(reader); + BSLS_ASSERT_SAFE(start); + BSLS_ASSERT_SAFE(end); + + // deliver everything up to the 'end' + if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY(!hasConsumers())) { BSLS_PERFORMANCEHINT_UNLIKELY_HINT; return 0; // RETURN } - size_t numMessages = processDeliveryLists(delay, appKey, storage, appId); + size_t numMessages = processDeliveryLists(delay, reader); - if (BSLS_PERFORMANCEHINT_PREDICT_LIKELY(redeliveryListSize())) { + if (BSLS_PERFORMANCEHINT_PREDICT_LIKELY(d_redeliveryList.size())) { // We only attempt to deliver new messages if we successfully // redelivered all messages in the redelivery list. return numMessages; // RETURN } - BSLS_ASSERT_SAFE(d_storageIter_mp); - // Deliver messages until either: // 1. End of storage; or // 2. subStream's capacity is saturated - mqbi::StorageIterator* storageIter_p = d_storageIter_mp.get(); + // 3. 'storageIter == end' + // + // 'end' is never CONFIRMed, so the 'VirtualStorageIterator' cannot skip it + + d_resumePoint = bmqt::MessageGUID(); + while (BSLS_PERFORMANCEHINT_PREDICT_LIKELY(start->hasReceipt())) { + if (!end->atEnd()) { + if (start->guid() == end->guid()) { + // Deliver the rest by 'QueueEngineUtil_AppsDeliveryContext' + break; + } + } - while (BSLS_PERFORMANCEHINT_PREDICT_LIKELY(storageIter_p->hasReceipt())) { Routers::Result result = Routers::e_SUCCESS; if (QueueEngineUtil::isBroadcastMode(d_queue_p)) { - broadcastOneMessage(storageIter_p); + // No checking the state for broadcast + broadcastOneMessage(start); } else { - result = tryDeliverOneMessage(delay, storageIter_p, false); + result = tryDeliverOneMessage(delay, start, false); + + if (result == Routers::e_SUCCESS) { + reportStats(start); - if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( - result == Routers::e_NO_CAPACITY || - result == Routers::e_NO_SUBSCRIPTION)) { + ++numMessages; + } + else if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( + result == Routers::e_NO_CAPACITY || + result == Routers::e_NO_SUBSCRIPTION)) { BSLS_PERFORMANCEHINT_UNLIKELY_HINT; - d_putAsideList.add(storageIter_p->guid()); + d_putAsideList.add(start->guid()); // Do not block other Subscriptions. Continue. } else if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( - result != Routers::e_SUCCESS)) { + result == Routers::e_NO_CAPACITY_ALL)) { + d_resumePoint = start->guid(); break; } - } - - if (result == Routers::e_SUCCESS) { - if (bmqp::QueueId::k_PRIMARY_QUEUE_ID == d_queue_p->id()) { - const bsls::Types::Int64 timeDelta = getMessageQueueTime( - storageIter_p->attributes()); - - // First report 'queue time' metric for the entire queue - d_queue_p->stats()->onEvent( - mqbstat::QueueStatsDomain::EventType::e_QUEUE_TIME, - timeDelta); - - // Then report 'queue time' metric for appId - d_queue_p->stats()->onEvent( - mqbstat::QueueStatsDomain::EventType::e_QUEUE_TIME, - timeDelta, - appId); + else { + BSLS_ASSERT_SAFE(result == Routers::e_INVALID); + // The {GUID, App} is not valid anymore } - ++numMessages; } - storageIter_p->advance(); + start->advance(); } return numMessages; } @@ -960,6 +964,13 @@ Routers::Result QueueEngineUtil_AppState::tryDeliverOneMessage( const mqbi::StorageIterator* message, bool isOutOfOrder) { + BSLS_ASSERT_SAFE(message); + + const mqbi::AppMessage& appView = message->appMessageView(ordinal()); + if (!appView.isPending()) { + return Routers::e_INVALID; // RETURN + } + // In order to try and deliver a message, we need to: // 1. Determine if a message has a delay based on its rdaInfo. // 2. Use the router to get candidate handle for the message. @@ -985,15 +996,18 @@ Routers::Result QueueEngineUtil_AppState::tryDeliverOneMessage( bsls::TimeInterval messageDelay; bsls::TimeInterval now = mwcsys::Time::nowMonotonicClock(); Visitor visitor; - Routers::Result result = Routers::e_SUCCESS; - if (!QueueEngineUtil::loadMessageDelay(message->rdaInfo(), + Routers::Result result = Routers::e_SUCCESS; + + if (!QueueEngineUtil::loadMessageDelay(appView.d_rdaInfo, d_queue_p->messageThrottleConfig(), &messageDelay)) { result = selectConsumer(bdlf::BindUtil::bind(&Visitor::oneConsumer, &visitor, bdlf::PlaceHolders::_1), - message); + message, + ordinal()); + // RelayQueueEngine_VirtualPushStorageIterator ignores ordinal } else { // Iterate all highest priority consumers and find the lowest delay @@ -1024,7 +1038,7 @@ Routers::Result QueueEngineUtil_AppState::tryDeliverOneMessage( const bmqp::Protocol::SubQueueInfosArray subQueueInfos( 1, bmqp::SubQueueInfo(visitor.d_downstreamSubscriptionId, - message->rdaInfo())); + message->appMessageView(ordinal()).d_rdaInfo)); visitor.d_handle->deliverMessage(message->appData(), message->guid(), message->attributes(), @@ -1038,24 +1052,6 @@ Routers::Result QueueEngineUtil_AppState::tryDeliverOneMessage( return result; } -void QueueEngineUtil_AppState::beforeMessageRemoved( - const bmqt::MessageGUID& msgGUID, - bool isExpired) -{ - // PRECONDITIONS - BSLS_ASSERT_SAFE(d_storageIter_mp); - - if (!d_storageIter_mp->atEnd() && (d_storageIter_mp->guid() == msgGUID)) { - d_storageIter_mp->advance(); - } - - d_redeliveryList.erase(msgGUID); - - if (isExpired) { - d_putAsideList.erase(msgGUID); - } -} - void QueueEngineUtil_AppState::broadcastOneMessage( const mqbi::StorageIterator* storageIter) { @@ -1087,68 +1083,64 @@ bool QueueEngineUtil_AppState::visitBroadcast( } size_t -QueueEngineUtil_AppState::processDeliveryLists(bsls::TimeInterval* delay, - const mqbu::StorageKey& appKey, - mqbi::Storage& storage, - const bsl::string& appId) +QueueEngineUtil_AppState::processDeliveryLists(bsls::TimeInterval* delay, + mqbi::StorageIterator* reader) { - size_t numMessages = - processDeliveryList(delay, appKey, storage, appId, d_redeliveryList); + BSLS_ASSERT_SAFE(delay); + + size_t numMessages = processDeliveryList(delay, reader, d_redeliveryList); if (*delay == bsls::TimeInterval()) { // The only excuse for stopping the iteration is poisonous message - numMessages = - processDeliveryList(delay, appKey, storage, appId, d_putAsideList); + numMessages += processDeliveryList(delay, reader, d_putAsideList); } return numMessages; } size_t -QueueEngineUtil_AppState::processDeliveryList(bsls::TimeInterval* delay, - const mqbu::StorageKey& appKey, - mqbi::Storage& storage, - const bsl::string& appId, - RedeliveryList& list) +QueueEngineUtil_AppState::processDeliveryList(bsls::TimeInterval* delay, + mqbi::StorageIterator* reader, + RedeliveryList& list) { if (BSLS_PERFORMANCEHINT_PREDICT_LIKELY(list.empty())) { return 0; // RETURN } - // For each message in the pending redelivery list + // For each reader in the pending redelivery list RedeliveryList::iterator it = list.begin(); bmqt::MessageGUID firstGuid = *it; size_t numMessages = 0; while (!list.isEnd(it)) { - // Retrieve message from storage - bslma::ManagedPtr message; - - mqbi::StorageResult::Enum rc = storage.getIterator(&message, - appKey, - *it); - if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( - rc != mqbi::StorageResult::e_SUCCESS)) { - BSLS_PERFORMANCEHINT_UNLIKELY_HINT; + Routers::Result result = Routers::e_INVALID; - BALL_LOG_WARN - << "#STORAGE_UNKNOWN_MESSAGE " - << "Error '" << rc << "' while retrieving msg from queue: '" - << d_queue_p->description() - << "', while attempting to re-distribute unconfirmed message " - << "with GUID: '" << *it << "'"; + // Retrieve message from the storage + reader->reset(*it); - // Couldn't send it, but will never be able to do so; just consider - // it as sent. - it = list.erase(it); - continue; // CONTINUE + if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY(reader->atEnd())) { + BSLS_PERFORMANCEHINT_UNLIKELY_HINT; + + // The message got gc'ed or purged + BMQ_LOGTHROTTLE_INFO() + << "#STORAGE_UNKNOWN_MESSAGE " << "Queue: '" + << d_queue_p->description() << "', app: '" << appId() + << "' could not redeliver GUID: '" << *it + << "' (not in the storage)"; + } + else if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( + !reader->appMessageView(ordinal()).isPending())) { + BMQ_LOGTHROTTLE_INFO() + << "#STORAGE_UNKNOWN_MESSAGE " << "Queue: '" + << d_queue_p->description() << "', app: '" << appId() + << "' could not redeliver GUID: '" << *it << "' (wrong state " + << reader->appMessageView(ordinal()).d_state << ")"; + } + else { + result = tryDeliverOneMessage(delay, reader, true); } // TEMPORARILY handling unknown 'Group's in RelayQE by reevaluating. // Instead, should communicate them upstream either in CloseQueue or in // Rejects. - Routers::Result result = tryDeliverOneMessage(delay, - message.get(), - true); - if (result == Routers::e_NO_CAPACITY_ALL) { break; // BREAK } @@ -1160,21 +1152,13 @@ QueueEngineUtil_AppState::processDeliveryList(bsls::TimeInterval* delay, it = list.erase(it); ++numMessages; - if (bmqp::QueueId::k_PRIMARY_QUEUE_ID == d_queue_p->id()) { - const bsls::Types::Int64 timeDelta = getMessageQueueTime( - message->attributes()); - - // First report 'queue time' metric for the entire queue - d_queue_p->stats()->onEvent( - mqbstat::QueueStatsDomain::EventType::e_QUEUE_TIME, - timeDelta); - - // Then report 'queue time' metric for appId - d_queue_p->stats()->onEvent( - mqbstat::QueueStatsDomain::EventType::e_QUEUE_TIME, - timeDelta, - appId); - } + reportStats(reader); + } + else if (result == Routers::e_INVALID) { + // Couldn't send it, but will never be able to do so; just consider + // it as sent. + // Remove from the redeliveryList + it = list.erase(it); } else { if (result == Routers::e_NO_SUBSCRIPTION) { @@ -1187,7 +1171,7 @@ QueueEngineUtil_AppState::processDeliveryList(bsls::TimeInterval* delay, if (numMessages) { BALL_LOG_INFO << "Queue '" << d_queue_p->description() - << "', appId = '" << appId << " (re)delivered " + << "', appId = '" << appId() << "' (re)delivered " << numMessages << " messages starting from " << firstGuid << "."; } @@ -1325,22 +1309,37 @@ bool QueueEngineUtil_AppState::transferUnconfirmedMessages( // We lost a reader, try to redeliver any potential messages // that need redelivery. - BALL_LOG_INFO << "Lost a reader for queue '" << d_queue_p->description() - << "', redelivering " << numMsgs << " message(s) to " - << consumers().size() << " remaining readers."; + if (numMsgs) { + BALL_LOG_INFO << "Lost a reader for queue '" + << d_queue_p->description() << "' " << appId() + << ", redelivering " << numMsgs << " message(s) to " + << consumers().size() + << " remaining readers starting from " + << *d_redeliveryList.begin(); + } + else { + BALL_LOG_INFO << "Lost a reader for queue '" + << d_queue_p->description() << "' " << appId() + << ", nothing to redeliver."; + } return hasConsumers(); } Routers::Result QueueEngineUtil_AppState::selectConsumer( const Routers::Visitor& visitor, - const mqbi::StorageIterator* currentMessage) + const mqbi::StorageIterator* currentMessage, + unsigned int ordinal) { + unsigned int sId = + currentMessage->appMessageView(ordinal).d_subscriptionId; + Routers::Result result = d_routing_sp->selectConsumer(visitor, - currentMessage); + currentMessage, + sId); if (result == Routers::e_NO_CAPACITY_ALL) { if (d_throttledEarlyExits.requestPermission()) { BALL_LOG_INFO << "[THROTTLED] Queue '" << d_queue_p->description() - << "', appId = '" << d_appId + << "', appId = '" << appId() << "' does not have any subscription " "capacity; early exits delivery at " << currentMessage->guid(); @@ -1374,31 +1373,80 @@ bool QueueEngineUtil_AppState::evaluateAutoSubcription() return d_autoSubscription.evaluate(); } -bslma::ManagedPtr QueueEngineUtil_AppState::head() const +void QueueEngineUtil_AppState::authorize(const mqbu::StorageKey& appKey, + unsigned int appOrdinal) { - bslma::ManagedPtr out; + BSLS_ASSERT_SAFE(d_queue_p->storage()); - if (!d_putAsideList.empty()) { - d_queue_p->storage()->getIterator(&out, - d_appKey, - d_putAsideList.first()); - } - else if (!d_storageIter_mp->atEnd()) { - d_queue_p->storage()->getIterator(&out, - d_appKey, - d_storageIter_mp->guid()); + d_appKey = appKey; + d_appOrdinal = appOrdinal; + d_isAuthorized = true; +} + +bool QueueEngineUtil_AppState::authorize() +{ + BSLS_ASSERT_SAFE(d_queue_p->storage()); + + unsigned int ordinal = 0; + mqbu::StorageKey appKey; + + const bool hasVirtualStorage = + d_queue_p->storage()->hasVirtualStorage(appId(), &appKey, &ordinal); + if (hasVirtualStorage) { + d_appKey = appKey; + d_appOrdinal = ordinal; + d_isAuthorized = true; + + return true; // RETURN } + return false; +} + +void QueueEngineUtil_AppState::unauthorize() +{ + BSLS_ASSERT_SAFE(d_queue_p->storage()); + + // Keep the d_appKey + d_appOrdinal = mqbi::Storage::k_INVALID_ORDINAL; + d_isAuthorized = false; + + clear(); +} - return out; +void QueueEngineUtil_AppState::clear() +{ + d_resumePoint = bmqt::MessageGUID(); + d_redeliveryList.clear(); + d_putAsideList.clear(); } void QueueEngineUtil_AppState::loadInternals(mqbcmd::AppState* out) const { - out->appId() = d_appId; + out->appId() = appId(); out->numConsumers() = d_routing_sp->d_consumers.size(); out->redeliveryListLength() = d_redeliveryList.size(); d_routing_sp->loadInternals(&out->roundRobinRouter()); } +void QueueEngineUtil_AppState::reportStats( + const mqbi::StorageIterator* message) const +{ + if (bmqp::QueueId::k_PRIMARY_QUEUE_ID == d_queue_p->id()) { + const bsls::Types::Int64 timeDelta = getMessageQueueTime( + message->attributes()); + + // First report 'queue time' metric for the entire queue + d_queue_p->stats()->onEvent( + mqbstat::QueueStatsDomain::EventType::e_QUEUE_TIME, + timeDelta); + + // Then report 'queue time' metric for appId + d_queue_p->stats()->onEvent( + mqbstat::QueueStatsDomain::EventType::e_QUEUE_TIME, + timeDelta, + appId()); + } +} + } // close namespace mqbblp } // close namespace BloombergLP diff --git a/src/groups/mqb/mqbblp/mqbblp_queueengineutil.h b/src/groups/mqb/mqbblp/mqbblp_queueengineutil.h index 2912b1494..377688654 100644 --- a/src/groups/mqb/mqbblp/mqbblp_queueengineutil.h +++ b/src/groups/mqb/mqbblp/mqbblp_queueengineutil.h @@ -331,17 +331,14 @@ class RedeliveryList { /// Mechanism managing state of a group of consumers supporting priorities. struct QueueEngineUtil_AppState { + public: // PUBLIC TYPES /// Set of alive consumers typedef Routers::Consumers Consumers; - // PUBLIC DATA - bslma::ManagedPtr d_storageIter_mp; - // Storage iterator to the logical - // stream of messages for this fanout - // consumer - + private: + // PRIVATE DATA bsl::shared_ptr d_routing_sp; // Set of alive consumers and their // states @@ -367,7 +364,7 @@ struct QueueEngineUtil_AppState { bdlmt::EventSchedulerEventHandle d_throttleEventHandle; // EventHandle for poison pill message // throttling. - const mqbu::StorageKey d_appKey; + mqbu::StorageKey d_appKey; const bsl::string d_appId; unsigned int d_upstreamSubQueueId; @@ -382,6 +379,12 @@ struct QueueEngineUtil_AppState { Routers::Expression d_autoSubscription; // Evaluator of the auto subscription + unsigned int d_appOrdinal; + + bmqt::MessageGUID d_resumePoint; + // When at capacity, resume point. + + public: // TRAITS BSLMF_NESTED_TRAIT_DECLARATION(QueueEngineUtil_AppState, bslma::UsesBslmaAllocator) @@ -389,10 +392,8 @@ struct QueueEngineUtil_AppState { BALL_LOG_SET_CLASS_CATEGORY("MQBBLP.QUEUEENGINEUTIL_APPSTATE"); // CREATORS - QueueEngineUtil_AppState(bslma::ManagedPtr iterator, - mqbi::Queue* queue, + QueueEngineUtil_AppState(mqbi::Queue* queue, bdlmt::EventScheduler* scheduler, - bool isAuthorized, Routers::QueueRoutingContext& queueContext, unsigned int upstreamSubQueueId, const bsl::string& appId, @@ -403,32 +404,33 @@ struct QueueEngineUtil_AppState { // MANIPULATORS - /// Called by Queue Engine before a message with the specified `msgGUID` - /// is removed. If the specified `isExpired` is `true`, the message is - /// TTL expired, otherwise it was confirmed or rejected. - void beforeMessageRemoved(const bmqt::MessageGUID& msgGUID, - bool isExpired); - /// Reset the internal state to have no consumers. void undoRouting(); - /// Deliver all messages in the storage to the consumer represented by - /// this instance. Load the message delay into the specified `delay`. + /// Attempt to deliver all pending data up to the specified `end` (the + /// iterator). First, attempt to drain the Redelivery and PutAside Lists. + /// While doing so, load the message delay into the specified `delay` and + /// throttle the redelivery as in the case of a Poisonous message. + /// If the Redelivery List is empty, attempt to deliver data starting from + /// `start` (the resumePoint or the beginning of the stream) to the `end`. + /// This is what any QueueEngine calls whenever there is a chance for the + /// App to make progress: any configuration change, capacity availability. /// Note that depending upon queue's mode, messages are delivered either /// to all consumers (broadcast mode), or in a round-robin manner (every /// other mode). - size_t deliverMessages(bsls::TimeInterval* delay, - const mqbu::StorageKey& appKey, - mqbi::Storage& storage, - const bsl::string& appId); + /// Use the specified `reader` to read data for delivery. + size_t deliverMessages(bsls::TimeInterval* delay, + mqbi::StorageIterator* reader, + mqbi::StorageIterator* start, + const mqbi::StorageIterator* end); - /// Try to deliver to the next available consumer the specified 'message'. + /// Try to deliver to the next available consumer the specified `message`. /// If poisonous message handling requires a delay in the delivery, iterate /// all highest priority consumers, load the lowest delay into the - /// specified 'delay' and return 'e_DELAY. If no delay is required, try to - /// send the 'message' to a highest priority consumer with matching - /// subscription. Return corresponding result: 'e_SUCCESS', - /// 'e_NO_SUBSCRIPTION', 'e_NO_CAPACITY'. or 'e_NO_CAPACITY_ALL'. + /// specified `delay` and return `e_DELAY`. If no delay is required, try + /// to send the `message` to a highest priority consumer with matching + /// subscription. Return corresponding result: `e_SUCCESS`, + /// `e_NO_SUBSCRIPTION`, `e_NO_CAPACITY`. or `e_NO_CAPACITY_ALL`. Routers::Result tryDeliverOneMessage(bsls::TimeInterval* delay, const mqbi::StorageIterator* message, bool isOutOfOrder); @@ -440,20 +442,16 @@ struct QueueEngineUtil_AppState { bool visitBroadcast(const mqbi::StorageIterator* message, const Routers::Subscription* subscription); - size_t processDeliveryLists(bsls::TimeInterval* delay, - const mqbu::StorageKey& appKey, - mqbi::Storage& storage, - const bsl::string& appId); + size_t processDeliveryLists(bsls::TimeInterval* delay, + mqbi::StorageIterator* reader); /// Process delivery of messages in the redelivery list. The specified /// `getMessageCb` provides message details for redelivery. Load the /// lowest handle delay into the specified `delay`. Return number of /// re-delivered messages. - size_t processDeliveryList(bsls::TimeInterval* delay, - const mqbu::StorageKey& appKey, - mqbi::Storage& storage, - const bsl::string& appId, - RedeliveryList& list); + size_t processDeliveryList(bsls::TimeInterval* delay, + mqbi::StorageIterator* reader, + RedeliveryList& list); /// Load into the specified `out` object' internal information about /// this consumers group and associated queue handles. @@ -501,9 +499,9 @@ struct QueueEngineUtil_AppState { void invalidate(mqbi::QueueHandle* handle); - Routers::Result - selectConsumer(const Routers::Visitor& visitor, - const mqbi::StorageIterator* currentMessage); + Routers::Result selectConsumer(const Routers::Visitor& visitor, + const mqbi::StorageIterator* currentMessage, + unsigned int ordinal); // Set the auto subscription int setSubscription(const mqbconfm::Expression& value); @@ -511,8 +509,46 @@ struct QueueEngineUtil_AppState { // Evaluate the auto subscription bool evaluateAutoSubcription(); + /// Change the state to authorized, thus enabling delivery + void authorize(const mqbu::StorageKey& appKey, unsigned int appOrdinal); + + /// Change the state to authorized, thus enabling delivery + bool authorize(); + + /// Change the state to authorized, thus disabling delivery. Clear all + /// pending data. + void unauthorize(); + + /// Save the specified `guid` in the PutAside list of messages for which + /// there is no matching subscription. The delivery of those messages will + // be attempted upon configuration change. + void putAside(const bmqt::MessageGUID& guid); + + /// Save the specified `guid` in the Redelivery list of messages. These + /// messages get delivered as Out-Of-Order. + void putForRedelivery(const bmqt::MessageGUID& guid); + + /// Save the current position in the data stream to resume the delivery (by + /// `deliverMessages`). + void setResumePoint(const bmqt::MessageGUID& guid); + + /// Return a reference offering modifiable access to the current routing + /// state controlling evaluation of subscriptions. + bsl::shared_ptr& routing(); + + /// Clear all pending data as in the case of either un-authorization, + /// purge operation, or Replica / Proxy losing last consumer. + void clear(); + // ACCESSORS - size_t redeliveryListSize() const; + + /// Return `true` if this App is not behind: authorized, empty Redelivery + /// List and no resume point. + bool isReadyForDelivery() const; + + /// Return `true` if this App does not have any never delivered data before + /// the queue iterator: empty PutAside List and no resume point. + bool isAtEndOfStorage() const; Routers::Consumer* findQueueHandleContext(mqbi::QueueHandle* handle); @@ -520,9 +556,31 @@ struct QueueEngineUtil_AppState { bool hasConsumers() const; - /// Returns storage iterator to the 1st un-delivered message including - /// `put-aside` messages (those without matching Subscriptions). - bslma::ManagedPtr head() const; + /// Return storage ordinal to access App state for each message. + unsigned int ordinal() const; + + /// Return a reference offering non-modifiable access to the PutAside list + /// of messages for which there is no matching subscription. + const RedeliveryList& putAsideList() const; + + /// Return current storage key. + const mqbu::StorageKey& appKey() const; + + /// Return the Id. + const bsl::string& appId() const; + + /// Return `true` if this App is authorized. + bool isAuthorized() const; + + /// Return the current resume point (empty when none). + const bmqt::MessageGUID& resumePoint() const; + + /// Return a reference offering non-modifiable access to the current + /// routing state controlling evaluation of subscriptions. + const bsl::shared_ptr& routing() const; + + /// Report queue stats upon delivery of the specified `message`. + void reportStats(const mqbi::StorageIterator* message) const; }; // ========================================== @@ -535,27 +593,57 @@ struct QueueEngineUtil_AppsDeliveryContext { bmqp::Protocol::SubQueueInfosArray> Consumers; - Consumers d_consumers; - bool d_doRepeat; - mqbi::StorageIterator* d_currentMessage; - mqbi::Queue* d_queue_p; - bsl::vector d_activeAppIds; + private: + Consumers d_consumers; + bool d_isReady; + mqbi::StorageIterator* d_currentMessage; + mqbi::Queue* d_queue_p; + bsl::optional d_timeDelta; + // Avoid reading the attributes if not necessary. Get timeDelta on demand. + // See comment in `QueueEngineUtil_AppsDeliveryContext::processApp`. + public: QueueEngineUtil_AppsDeliveryContext(mqbi::Queue* queue, bslma::Allocator* allocator); - /// Prepare the context to pick up and deliver next message. - void reset(); - - /// Return true if the specified `app` has next available handle to - /// deliver the current message and prepare for `deliverMessage` call. - bool processApp(QueueEngineUtil_AppState& app); + /// Start delivery cycle(s). + void start(); + + /// Prepare the context to process next message. + /// Return `true` if the delivery can continue iterating dataStream + /// The `false` return value indicates either the end of the dataStream or + /// the the `e_NO_CAPACITY_ALL` case. + bool reset(mqbi::StorageIterator* currentMessage); + + /// Return `true` if the specified `app` is not a broadcast app and has an + /// available handle to deliver the current message with the specified + /// `ordinal`. + /// `false` return value indicates that the `app` is either a broadcast, + /// or unauthorized, or does not have matching subscription, or does not + /// have the capacity for any existing subscription or did not drain its + /// redelivery list, or is already behind. In any case, an authorized app + /// sets its resume point, so the queue iterator can continue to advance + /// unless in the `e_NO_CAPACITY_ALL` case when all apps have no capacity + /// for any existing subscription. + /// The `ordinal` controls how we read the state from the data stream. In + /// the case of `RootQueueEngine` (Primary), the data stream is the storage + /// and the `ordinal` is the App ordinal in the stream. + /// In the case of `RelayQueueEngine`, the stream is `PushStream` and the + /// `ordinal` is the offset in the received PUSH message. + bool processApp(QueueEngineUtil_AppState& app, unsigned int ordina); + + /// Collect and prepare data for the subsequent `deliverMessage` call. bool visit(const Routers::Subscription* subscription, - const mqbi::StorageIterator* message); + const mqbi::AppMessage& appView); bool visitBroadcast(const Routers::Subscription* subscription); /// Deliver message to the previously processed handles. void deliverMessage(); + + /// Return `true` if there is at least one delivery target selected. + bool isEmpty() const; + + bsls::Types::Int64 timeDelta(); }; // ============================================================================ @@ -690,9 +778,10 @@ inline void RedeliveryList::trim(iterator* cit) const // struct QueueEngineUtil_AppState // ------------------------------- -inline size_t QueueEngineUtil_AppState::redeliveryListSize() const +inline void +QueueEngineUtil_AppState::putForRedelivery(const bmqt::MessageGUID& guid) { - return d_redeliveryList.size(); + d_redeliveryList.add(guid); } inline Routers::Consumer* @@ -718,6 +807,43 @@ QueueEngineUtil_AppState::find(mqbi::QueueHandle* handle) return d_routing_sp->d_consumers.find(handle); } +inline void QueueEngineUtil_AppState::setUpstreamSubQueueId(unsigned int value) +{ + d_upstreamSubQueueId = value; +} + +inline void QueueEngineUtil_AppState::invalidate(mqbi::QueueHandle* handle) +{ + Routers::Consumers::SharedItem itConsumer = find(handle); + + if (itConsumer) { + itConsumer->invalidate(); + + d_priorityCount = d_routing_sp->finalize(); + + d_routing_sp->registerSubscriptions(); + } +} + +inline void QueueEngineUtil_AppState::putAside(const bmqt::MessageGUID& guid) +{ + d_putAsideList.add(guid); +} + +inline void +QueueEngineUtil_AppState::setResumePoint(const bmqt::MessageGUID& guid) +{ + BSLS_ASSERT_SAFE(d_resumePoint.isUnset()); + d_resumePoint = guid; +} + +inline bsl::shared_ptr& +QueueEngineUtil_AppState::routing() +{ + return d_routing_sp; +} + +// ACCESSORS inline bool QueueEngineUtil_AppState::hasConsumers() const { return d_routing_sp ? !d_routing_sp->d_consumers.empty() : false; @@ -728,22 +854,51 @@ inline unsigned int QueueEngineUtil_AppState::upstreamSubQueueId() const return d_upstreamSubQueueId; } -inline void QueueEngineUtil_AppState::setUpstreamSubQueueId(unsigned int value) +inline bool QueueEngineUtil_AppState::isReadyForDelivery() const { - d_upstreamSubQueueId = value; + return d_redeliveryList.size() == 0 && d_isAuthorized && + d_resumePoint.isUnset(); } -inline void QueueEngineUtil_AppState::invalidate(mqbi::QueueHandle* handle) +inline bool QueueEngineUtil_AppState::isAtEndOfStorage() const { - Routers::Consumers::SharedItem itConsumer = find(handle); + return d_putAsideList.size() == 0 && d_resumePoint.isUnset(); +} - if (itConsumer) { - itConsumer->invalidate(); +inline unsigned int QueueEngineUtil_AppState::ordinal() const +{ + return d_appOrdinal; +} - d_priorityCount = d_routing_sp->finalize(); +inline const RedeliveryList& QueueEngineUtil_AppState::putAsideList() const +{ + return d_putAsideList; +} - d_routing_sp->registerSubscriptions(); - } +inline const mqbu::StorageKey& QueueEngineUtil_AppState::appKey() const +{ + return d_appKey; +} + +inline const bsl::string& QueueEngineUtil_AppState::appId() const +{ + return d_appId; +} + +inline bool QueueEngineUtil_AppState::isAuthorized() const +{ + return d_isAuthorized; +} + +inline const bmqt::MessageGUID& QueueEngineUtil_AppState::resumePoint() const +{ + return d_resumePoint; +} + +inline const bsl::shared_ptr& +QueueEngineUtil_AppState::routing() const +{ + return d_routing_sp; } // ------------------------------------------ diff --git a/src/groups/mqb/mqbblp/mqbblp_queuestate.cpp b/src/groups/mqb/mqbblp/mqbblp_queuestate.cpp index 3e0d9da45..864c090fb 100644 --- a/src/groups/mqb/mqbblp/mqbblp_queuestate.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_queuestate.cpp @@ -45,13 +45,14 @@ namespace mqbblp { // class QueueState // ---------------- -QueueState::QueueState(mqbi::Queue* queue, - const bmqt::Uri& uri, - unsigned int id, - const mqbu::StorageKey& key, - int partitionId, - mqbi::Domain* domain, - bslma::Allocator* allocator) +QueueState::QueueState(mqbi::Queue* queue, + const bmqt::Uri& uri, + unsigned int id, + const mqbu::StorageKey& key, + int partitionId, + mqbi::Domain* domain, + const mqbi::ClusterResources resources, + bslma::Allocator* allocator) : d_queue_p(queue) , d_uri(uri, allocator) , d_description(allocator) @@ -63,8 +64,7 @@ QueueState::QueueState(mqbi::Queue* queue, , d_partitionId(partitionId) , d_domain_p(domain) , d_storageManager_p(0) -, d_blobBufferFactory_p(0) -, d_scheduler_p(0) +, d_resources(resources) , d_miscWorkThreadPool_p(0) , d_storage_mp(0) , d_stats(allocator) diff --git a/src/groups/mqb/mqbblp/mqbblp_queuestate.h b/src/groups/mqb/mqbblp/mqbblp_queuestate.h index 2366c66ff..131ae6927 100644 --- a/src/groups/mqb/mqbblp/mqbblp_queuestate.h +++ b/src/groups/mqb/mqbblp/mqbblp_queuestate.h @@ -151,11 +151,7 @@ class QueueState { mqbi::AppKeyGenerator* d_appKeyGenerator_p; // App key generator to use. - bdlbb::BlobBufferFactory* d_blobBufferFactory_p; - // BlobBufferFactory to use. - - bdlmt::EventScheduler* d_scheduler_p; - // EventScheduler to use. + const mqbi::ClusterResources d_resources; bdlmt::FixedThreadPool* d_miscWorkThreadPool_p; // Thread pool used for any standalone @@ -201,25 +197,22 @@ class QueueState { // CREATORS - /// Create a new `QueueState` associated to the specified `queue` and - /// having the specified `uri`, `id`, `key`, `partitionId` and `domain`. - /// Use the specified `allocator` for any memory allocations. Use the - /// specified 'unconfirmedCounter' to aggregate the counting of - /// unconfirmed by each queue handle. - QueueState(mqbi::Queue* queue, - const bmqt::Uri& uri, - unsigned int id, - const mqbu::StorageKey& key, - int partitionId, - mqbi::Domain* domain, - bslma::Allocator* allocator); + /// Create a new 'QueueState' associated to the specified 'queue' and + /// having the specified 'uri', 'id', 'key', 'partitionId', 'domain', and + /// 'resources'. Use the specified 'allocator' for any memory allocations. + QueueState(mqbi::Queue* queue, + const bmqt::Uri& uri, + unsigned int id, + const mqbu::StorageKey& key, + int partitionId, + mqbi::Domain* domain, + const mqbi::ClusterResources resources, + bslma::Allocator* allocator); /// Destructor ~QueueState(); // MANIPULATORS - QueueState& setBlobBufferFactory(bdlbb::BlobBufferFactory* value); - QueueState& setEventScheduler(bdlmt::EventScheduler* scheduler); QueueState& setMiscWorkThreadPool(bdlmt::FixedThreadPool* threadPool); QueueState& setDescription(const bslstl::StringRef& value); QueueState& setDomain(mqbi::Domain* value); @@ -296,16 +289,18 @@ class QueueState { bmqp_ctrlmsg::StreamParameters* value, unsigned int subQueueId = bmqp::QueueId::k_DEFAULT_SUBQUEUE_ID) const; - bdlbb::BlobBufferFactory* blobBufferFactory() const; - bdlmt::EventScheduler* scheduler() const; - bdlmt::FixedThreadPool* miscWorkThreadPool() const; - const bsl::string& description() const; - const mqbi::DispatcherClientData& dispatcherClientData() const; - mqbi::Domain* domain() const; - unsigned int id() const; - const mqbu::StorageKey& key() const; - const QueueHandleCatalog& handleCatalog() const; - const bmqp_ctrlmsg::QueueHandleParameters& handleParameters() const; + bdlbb::BlobBufferFactory* blobBufferFactory() const; + bdlmt::EventScheduler* scheduler() const; + mqbi::ClusterResources::BlobSpPool* blobSpPool() const; + const bsl::optional& pushElementsPool() const; + bdlmt::FixedThreadPool* miscWorkThreadPool() const; + const bsl::string& description() const; + const mqbi::DispatcherClientData& dispatcherClientData() const; + mqbi::Domain* domain() const; + unsigned int id() const; + const mqbu::StorageKey& key() const; + const QueueHandleCatalog& handleCatalog() const; + const bmqp_ctrlmsg::QueueHandleParameters& handleParameters() const; int partitionId() const; mqbi::Queue* queue() const; @@ -359,19 +354,6 @@ class QueueState { // ---------------- // MANIPULATORS -inline QueueState& -QueueState::setBlobBufferFactory(bdlbb::BlobBufferFactory* value) -{ - d_blobBufferFactory_p = value; - return *this; -} - -inline QueueState& -QueueState::setEventScheduler(bdlmt::EventScheduler* scheduler) -{ - d_scheduler_p = scheduler; - return *this; -} inline QueueState& QueueState::setMiscWorkThreadPool(bdlmt::FixedThreadPool* threadPool) @@ -529,12 +511,23 @@ inline Routers::QueueRoutingContext& QueueState::routingContext() // ACCESSORS inline bdlbb::BlobBufferFactory* QueueState::blobBufferFactory() const { - return d_blobBufferFactory_p; + return d_resources.bufferFactory(); } inline bdlmt::EventScheduler* QueueState::scheduler() const { - return d_scheduler_p; + return d_resources.scheduler(); +} + +inline mqbi::ClusterResources::BlobSpPool* QueueState::blobSpPool() const +{ + return d_resources.blobSpPool(); +} + +inline const bsl::optional& +QueueState::pushElementsPool() const +{ + return d_resources.pushElementsPool(); } inline bdlmt::FixedThreadPool* QueueState::miscWorkThreadPool() const diff --git a/src/groups/mqb/mqbblp/mqbblp_relayqueueengine.cpp b/src/groups/mqb/mqbblp/mqbblp_relayqueueengine.cpp index ff10791f1..8f3dd58d1 100644 --- a/src/groups/mqb/mqbblp/mqbblp_relayqueueengine.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_relayqueueengine.cpp @@ -78,6 +78,10 @@ const bsls::Types::Int64 k_NS_PER_MESSAGE = BALL_LOGTHROTTLE_INFO(k_MAX_INSTANT_MESSAGES, k_NS_PER_MESSAGE) \ << "[THROTTLED] " +#define BMQ_LOGTHROTTLE_ERROR() \ + BALL_LOGTHROTTLE_ERROR(k_MAX_INSTANT_MESSAGES, k_NS_PER_MESSAGE) \ + << "[THROTTLED] " + // ==================== // class LimitedPrinter // ==================== @@ -161,6 +165,7 @@ class RelayQueueEngine::AutoPurger { d_relayQueueEngine.afterQueuePurged( bmqp::ProtocolUtil::k_NULL_APP_ID, mqbu::StorageKey::k_NULL_KEY); + d_relayQueueEngine.d_storageIter_mp->reset(); } } }; @@ -323,14 +328,12 @@ void RelayQueueEngine::onHandleConfiguredDispatched( // Rebuild consumers for routing const mqbi::QueueCounts& counts = it->second.d_counts; - + App_State* app = 0; if (counts.d_readCount > 0) { - AppsMap::const_iterator itApp = d_apps.find(upstreamSubQueueId); - App_State& app = *itApp->second; + app = findApp(upstreamSubQueueId); + BSLS_ASSERT_SAFE(app); - BSLS_ASSERT_SAFE(itApp != d_apps.end()); - - applyConfiguration(app, *context); + applyConfiguration(*app, *context); } BALL_LOGTHROTTLE_INFO_BLOCK(k_MAX_INSTANT_MESSAGES, k_NS_PER_MESSAGE) @@ -347,11 +350,9 @@ void RelayQueueEngine::onHandleConfiguredDispatched( // Invoke callback sending ConfigureQueue response before PUSHing. context->invokeCallback(); - // 'downstreamSubQueueId' can represent a producer, or a consumer with no - // capacity or with some capacity. We want to invoke 'onHandleUsable' only - // in the later case. - - deliverMessages(); + if (app) { + processAppRedelivery(upstreamSubQueueId, app); + } } void RelayQueueEngine::onHandleReleased( @@ -459,7 +460,7 @@ void RelayQueueEngine::onHandleReleasedDispatched( BSLS_ASSERT_SAFE(appStateIt != d_apps.end()); BSLS_ASSERT_SAFE(appStateIt->second); - App_State& app = *appStateIt->second; + App_State* app = appStateIt->second.get(); if (streamResult.hasNoHandleStreamConsumers()) { // There are no more consumers for this subStream. @@ -503,7 +504,7 @@ void RelayQueueEngine::onHandleReleasedDispatched( handle->setStreamParameters(nullStreamParams); } - app.invalidate(handle); + app->invalidate(handle); // This is in continuation of the special-case handling above. If the // client is attempting to release the consumer portion of an *active* @@ -513,8 +514,8 @@ void RelayQueueEngine::onHandleReleasedDispatched( // the effects of a configureQueue with null streamParameters. Note // that this may affect the 'd_queueState_p->streamParameters()'. - if (app.transferUnconfirmedMessages(handle, info)) { - processAppRedelivery(app, info.appId()); + if (app->transferUnconfirmedMessages(handle, info)) { + processAppRedelivery(upstreamSubQueueId, app); } else { // We lost the last reader. @@ -524,30 +525,30 @@ void RelayQueueEngine::onHandleReleasedDispatched( // readers), because those messages may be re-routed by the primary // to another client. Also get rid of any pending and // to-be-redelivered messages. - mqbu::StorageKey appKey = mqbu::StorageKey(upstreamSubQueueId); - afterQueuePurged(info.appId(), appKey); + beforeOneAppRemoved(upstreamSubQueueId); + d_pushStream.removeApp(upstreamSubQueueId); + app->clear(); } if (streamResult.hasNoHandleStreamProducers()) { // Remove the producer handle from cached producers/consumers - app.d_cache.erase(handle); + app->d_cache.erase(handle); } } if (streamResult.isQueueStreamEmpty()) { + unsigned int numMessages = d_pushStream.removeApp(upstreamSubQueueId); + BALL_LOG_INFO << "For queue [" << d_queueState_p->uri() << "], removing App for appId: [" << info.appId() << "] and virtual storage associated with" - << " upstreamSubQueueId: [" << upstreamSubQueueId << "]"; + << " upstreamSubQueueId: [" << upstreamSubQueueId << "]" + << " with " << numMessages << " messages"; - BSLS_ASSERT_SAFE(app.d_cache.empty()); - const bool removed = d_subStreamMessages_p->removeVirtualStorage( - mqbu::StorageKey(upstreamSubQueueId)); - - BSLS_ASSERT_SAFE(removed); - (void)removed; // Compiler happiness + BSLS_ASSERT_SAFE(app->d_cache.empty()); // Remove and delete empty App_State d_apps.erase(appStateIt); + d_appIds.erase(app->appId()); d_queueState_p->removeUpstreamParameters(upstreamSubQueueId); d_queueState_p->abandon(upstreamSubQueueId); } @@ -568,12 +569,7 @@ void RelayQueueEngine::deliverMessages() BSLS_ASSERT_SAFE(d_queueState_p->queue()->dispatcher()->inDispatcherThread( d_queueState_p->queue())); - // Auto-purge on exit. - // TBD: We need to revisit this. We would prefer to have - // 'deliverMessages()' called once for a bunch of messages, since - // what we have now is inefficient but it's less of an overhead here - // as compared to the primary, so this is why we haven't implemented - // this optimization yet. + // Auto-purge broadcast storage on exit. AutoPurger onExit(*this); // The guard MUST be created before the no consumers check. This is @@ -583,68 +579,100 @@ void RelayQueueEngine::deliverMessages() // Deliver messages until either: // 1. End of storage; or - // 2. subStream's capacity is saturated + // 2. All subStreams return 'e_NO_CAPACITY_ALL' - do { - d_appsDeliveryContext.reset(); + d_appsDeliveryContext.start(); - for (AppsMap::iterator it = d_apps.begin(); it != d_apps.end(); ++it) { - // First try to send any message in the redelivery list ... - bsls::TimeInterval delay; - const AppStateSp& appSp = it->second; - const mqbu::StorageKey key(appSp->upstreamSubQueueId()); - mqbi::Storage* storage = d_subStreamMessages_p->virtualStorage( - key); - const bsl::string& appId = appSp->d_appId; - BSLS_ASSERT_SAFE(storage); - - appSp->processDeliveryLists(&delay, key, *storage, appId); - - if (delay != bsls::TimeInterval()) { - appSp->scheduleThrottle( - mwcsys::Time::nowMonotonicClock() + delay, - bdlf::BindUtil::bind( - &RelayQueueEngine::processAppRedelivery, - this, - bsl::ref(*appSp), - appSp->d_appId)); + while (d_appsDeliveryContext.reset(d_storageIter_mp.get())) { + // Assume, all Apps need to deliver (some may be at capacity) + unsigned int numApps = d_storageIter_mp->numApps(); + + for (unsigned int i = 0; i < numApps; ++i) { + const PushStream::Element* element = d_storageIter_mp->element(i); + + App_State* app = element->app().d_app.get(); + + BSLS_ASSERT_SAFE(app); + if (!app->isAuthorized()) { + // This App got the PUSH (recorded in the PushStream) + BMQ_LOGTHROTTLE_ERROR() + << "#NOT AUTHORIZED " + << "Remote queue: " << d_queueState_p->uri() + << " (id: " << d_queueState_p->id() + << ") discarding a PUSH message for guid " + << d_storageIter_mp->guid() << ", with NOT AUTHORIZED App " + << app->appId(); + + d_storageIter_mp->removeCurrentElement(); } - else if (appSp->redeliveryListSize() == 0) { - d_appsDeliveryContext.processApp(*appSp); + + if (d_appsDeliveryContext.processApp(*app, i)) { + // The current element has made it either to delivery or + // putAside or resumerPoint and it can be removed + d_storageIter_mp->removeCurrentElement(); } + // Else, the current element has made it to resumerPoint and + // it cannot be removed } d_appsDeliveryContext.deliverMessage(); - } while (d_appsDeliveryContext.d_doRepeat); + } } -void RelayQueueEngine::processAppRedelivery(App_State& state, - const bsl::string& appId) +void RelayQueueEngine::processAppRedelivery(unsigned int upstreamSubQueueId, + App_State* app) { // executed by the *QUEUE DISPATCHER* thread + // PRECONDITIONS BSLS_ASSERT_SAFE(d_queueState_p->queue()->dispatcher()->inDispatcherThread( d_queueState_p->queue())); - BSLS_ASSERT_SAFE(d_subStreamMessages_p); + // Position to the last 'Routers::e_NO_CAPACITY_ALL' point + bslma::ManagedPtr storageIter_mp; + PushStreamIterator* start = 0; + + if (app->resumePoint().isUnset()) { + start = d_storageIter_mp.get(); + } + else { + PushStream::iterator it = d_pushStream.d_stream.find( + app->resumePoint()); + + if (it == d_pushStream.d_stream.end()) { + // The message is gone because of purge + // Start at the beginning + it = d_pushStream.d_stream.begin(); + } - bsls::TimeInterval delay; - const mqbu::StorageKey key(state.upstreamSubQueueId()); - mqbi::Storage* virtualStorage = d_subStreamMessages_p->virtualStorage(key); + storageIter_mp.load(new (*d_allocator_p) + VirtualPushStreamIterator(upstreamSubQueueId, + storage(), + &d_pushStream, + it), + d_allocator_p); - BSLS_ASSERT_SAFE(virtualStorage); + start = storageIter_mp.get(); + } - state.processDeliveryLists(&delay, key, *virtualStorage, appId); + bsls::TimeInterval delay; + app->deliverMessages(&delay, + d_realStorageIter_mp.get(), + start, + d_storageIter_mp.get()); if (delay != bsls::TimeInterval()) { - state.scheduleThrottle( + app->scheduleThrottle( mwcsys::Time::nowMonotonicClock() + delay, bdlf::BindUtil::bind(&RelayQueueEngine::processAppRedelivery, this, - bsl::ref(state), - appId)); + upstreamSubQueueId, + app)); } - else if (state.redeliveryListSize() == 0) { - deliverMessages(); + + if (app->isReadyForDelivery()) { + // can continue delivering + const bmqt::MessageGUID dummy; + afterNewMessage(dummy, 0); } } @@ -775,7 +803,7 @@ void RelayQueueEngine::rebuildUpstreamState(Routers::AppContext* context, iter->second.d_downstreamSubQueueId, upstreamSubQueueId, streamParameters, - appState->d_routing_sp.get()); + appState->routing().get()); if (errorStream.length() > 0) { BALL_LOG_WARN << "#BMQ_SUBSCRIPTION_FAILURE for queue '" @@ -812,7 +840,7 @@ void RelayQueueEngine::applyConfiguration(App_State& app, app.undoRouting(); - app.d_routing_sp = context.d_routing_sp; + app.routing() = context.d_routing_sp; if (!d_queueState_p->isDeliverConsumerPriority()) { if (d_queueState_p->hasMultipleSubStreams()) { @@ -833,9 +861,9 @@ void RelayQueueEngine::applyConfiguration(App_State& app, } } - app.d_routing_sp->apply(); + app.routing()->apply(); - Routers::Consumers& consumers = app.d_routing_sp->d_consumers; + Routers::Consumers& consumers = app.routing()->d_consumers; for (Routers::Consumers::const_iterator itConsumer = consumers.begin(); itConsumer != consumers.end(); @@ -850,7 +878,7 @@ void RelayQueueEngine::applyConfiguration(App_State& app, // cancelled. itConsumer->second.lock()->invalidate(); - app.d_priorityCount = app.d_routing_sp->finalize(); + app.routing()->finalize(); continue; // CONTINUE } @@ -864,25 +892,24 @@ void RelayQueueEngine::applyConfiguration(App_State& app, } // CREATORS -RelayQueueEngine::RelayQueueEngine( - QueueState* queueState, - mqbs::VirtualStorageCatalog* subStreamMessages, - const mqbconfm::Domain& domainConfig, - bslma::Allocator* allocator) +RelayQueueEngine::RelayQueueEngine(QueueState* queueState, + const mqbconfm::Domain& domainConfig, + bslma::Allocator* allocator) : d_queueState_p(queueState) -, d_subStreamMessages_p(subStreamMessages) +, d_pushStream(queueState->pushElementsPool(), allocator) , d_domainConfig(domainConfig, allocator) , d_apps(allocator) +, d_appIds(allocator) , d_self(this) // use default allocator -, d_scheduler_p(queueState->scheduler()) , d_appsDeliveryContext(d_queueState_p->queue(), allocator) +, d_storageIter_mp() +, d_realStorageIter_mp() , d_allocator_p(allocator) { // PRECONDITIONS BSLS_ASSERT_SAFE(queueState); BSLS_ASSERT_SAFE(queueState->queue()); BSLS_ASSERT_SAFE(queueState->storage()); - BSLS_ASSERT_SAFE(subStreamMessages); d_throttledRejectedMessages.initialize( 1, @@ -905,7 +932,6 @@ RelayQueueEngine::~RelayQueueEngine() int RelayQueueEngine::configure( BSLS_ANNOTATION_UNUSED bsl::ostream& errorDescription) { - BSLS_ASSERT_SAFE(d_subStreamMessages_p); return 0; } @@ -914,13 +940,22 @@ void RelayQueueEngine::resetState(bool isShuttingDown) for (AppsMap::iterator it = d_apps.begin(); it != d_apps.end(); ++it) { it->second->undoRouting(); if (isShuttingDown) { - it->second->d_routing_sp->reset(); + it->second->routing()->reset(); } // else, keep the routing which new engine can reuse } if (!isShuttingDown) { d_apps.clear(); } + + d_realStorageIter_mp = storage()->getIterator( + mqbu::StorageKey::k_NULL_KEY); + + d_storageIter_mp.load( + new (*d_allocator_p) PushStreamIterator(storage(), + &d_pushStream, + d_pushStream.d_stream.begin()), + d_allocator_p); } int RelayQueueEngine::rebuildInternalState( @@ -1031,30 +1066,25 @@ mqbi::QueueHandle* RelayQueueEngine::getHandle( const bmqp_ctrlmsg::SubQueueIdInfo& downstreamInfo = bmqp::QueueUtil::extractSubQueueInfo(handleParameters); - AppsMap::iterator itApp = d_apps.find(upstreamSubQueueId); - if (itApp == d_apps.end()) { + App_State* app = findApp(upstreamSubQueueId); + if (app == 0) { // Create and insert new App_State // Correlate downstreamInfo with upstream - mqbu::StorageKey appKey = mqbu::StorageKey(upstreamSubQueueId); - bsl::ostringstream errorDesc; - d_subStreamMessages_p->addVirtualStorage(errorDesc, - downstreamInfo.appId(), - appKey); - - AppStateSp appStateSp(new (*d_allocator_p) App_State( - upstreamSubQueueId, - downstreamInfo.appId(), - d_subStreamMessages_p->getIterator(appKey), - d_queueState_p->queue(), - d_scheduler_p, - appKey, - d_queueState_p->routingContext(), - d_allocator_p), + + AppStateSp appStateSp(new (*d_allocator_p) + App_State(upstreamSubQueueId, + downstreamInfo.appId(), + d_queueState_p->queue(), + d_queueState_p->scheduler(), + d_queueState_p->routingContext(), + d_allocator_p), d_allocator_p); - BSLS_ASSERT_SAFE(appStateSp->d_storageIter_mp); - itApp = d_apps.insert(bsl::make_pair(upstreamSubQueueId, appStateSp)) - .first; + app = appStateSp.get(); + + d_apps.insert(bsl::make_pair(upstreamSubQueueId, appStateSp)); + + d_appIds.insert(bsl::make_pair(app->appId(), appStateSp)); d_queueState_p->adopt(appStateSp); @@ -1065,8 +1095,11 @@ mqbi::QueueHandle* RelayQueueEngine::getHandle( << " upstreamSubQueueId: [" << upstreamSubQueueId << "]"; } - BSLS_ASSERT_SAFE(itApp != d_apps.end()); - App_State& app(*itApp->second); + BSLS_ASSERT_SAFE(app); + + if (!app->isAuthorized()) { + app->authorize(); + } queueHandle->registerSubStream( downstreamInfo, @@ -1077,7 +1110,7 @@ mqbi::QueueHandle* RelayQueueEngine::getHandle( // If a new reader/write, insert its (default-valued) stream parameters // into our map of consumer stream parameters advertised upstream. bsl::pair insertResult = - app.d_cache.insert(bsl::make_pair( + app->d_cache.insert(bsl::make_pair( queueHandle, RelayQueueEngine_AppState::CachedParameters(handleParameters, downstreamInfo.subId(), @@ -1147,12 +1180,12 @@ void RelayQueueEngine::configureHandle( } unsigned int upstreamSubQueueId = it->second.d_upstreamSubQueueId; - AppsMap::iterator itApp(d_apps.find(upstreamSubQueueId)); - BSLS_ASSERT_SAFE(itApp != d_apps.end()); + App_State* app = findApp(upstreamSubQueueId); + BSLS_ASSERT_SAFE(app); context->initializeRouting(d_queueState_p->routingContext()); - configureApp(*itApp->second, handle, streamParameters, context); + configureApp(*app, handle, streamParameters, context); } void RelayQueueEngine::releaseHandle( @@ -1202,12 +1235,12 @@ void RelayQueueEngine::releaseHandleImpl( BSLS_ASSERT_SAFE(it != handle->subStreamInfos().end()); unsigned int upstreamSubQueueId = it->second.d_upstreamSubQueueId; - AppsMap::iterator itApp = d_apps.find(upstreamSubQueueId); + App_State* app = findApp(upstreamSubQueueId); - BSLS_ASSERT_SAFE(itApp != d_apps.end()); - App_State::CachedParametersMap::iterator itHandle = - itApp->second->d_cache.find(handle); - BSLS_ASSERT_SAFE(itHandle != itApp->second->d_cache.end()); + BSLS_ASSERT_SAFE(app); + App_State::CachedParametersMap::iterator itHandle = app->d_cache.find( + handle); + BSLS_ASSERT_SAFE(itHandle != app->d_cache.end()); bmqp_ctrlmsg::QueueHandleParameters& cachedHandleParameters = itHandle->second.d_handleParameters; @@ -1310,15 +1343,10 @@ void RelayQueueEngine::onHandleUsable(mqbi::QueueHandle* handle, if (d_queueState_p->routingContext().onUsable(&upstreamSubQueueId, upstreamSubscriptionId)) { - // TEMPORARILY - // Call the same 'deliverMessages' as in the case of 'afterNewMessage' - // which attempts to delivery every message for every application - // (which is '__defaut' in non-fanout case. While the logic is - // simplified, it may be suboptimal if the case of large number of - // applications and small capacity limits. + App_State* app = findApp(upstreamSubQueueId); - (void)upstreamSubQueueId; - deliverMessages(); + BSLS_ASSERT_SAFE(app); + processAppRedelivery(upstreamSubQueueId, app); } } @@ -1355,33 +1383,21 @@ int RelayQueueEngine::onConfirmMessage(mqbi::QueueHandle* handle, // Specified 'subscriptionId' is the downstream one. Need to calculate // corresponding appKey. This is done by retrieving the upstream subId // and using it to create the appKey. - AppsMap::iterator itApp = d_apps.find(upstreamSubQueueId); - - BSLS_ASSERT_SAFE(itApp != d_apps.end()); - - // Inform the 'app' that 'msgGUID' is about to be removed from its virtual - // storage, so that app can advance its iterator etc if required. - - QueueEngineUtil_AppState& app = *itApp->second; + App_State* app = findApp(upstreamSubQueueId); - app.beforeMessageRemoved(msgGUID, false); + BSLS_ASSERT_SAFE(app); - // Create appKey and remove message. - const mqbu::StorageKey appKey(upstreamSubQueueId); - d_subStreamMessages_p->remove(msgGUID, appKey); - - app.tryCancelThrottle(handle, msgGUID); + app->tryCancelThrottle(handle, msgGUID); // If proxy, also inform the physical storage. if (d_queueState_p->domain()->cluster()->isRemote()) { mqbi::Storage* storage = d_queueState_p->storage(); - mqbi::StorageResult::Enum rc = storage->releaseRef(msgGUID, - appKey, - 0ULL); + mqbi::StorageResult::Enum rc = storage->confirm(msgGUID, + app->appKey(), + 0ULL); if (mqbi::StorageResult::e_ZERO_REFERENCES == rc) { // Since there are no references, there should be no app holding // msgGUID and no need to call `beforeMessageRemoved` - BSLS_ASSERT_SAFE(!d_subStreamMessages_p->hasMessage(msgGUID)); return rc_NO_MORE_REFERENCES; // RETURN } @@ -1408,60 +1424,50 @@ int RelayQueueEngine::onRejectMessage( // Retrieve upstream subId and use it to create an appKey to reject the // message. - AppsMap::iterator itApp = d_apps.find(upstreamSubQueueId); + App_State* app = findApp(upstreamSubQueueId); - BSLS_ASSERT_SAFE(itApp != d_apps.end()); + BSLS_ASSERT_SAFE(app); // Inform the 'app' that 'msgGUID' is about to be removed from its virtual // storage, so that app can advance its iterator etc if required. - QueueEngineUtil_AppState& app = *itApp->second; - const mqbu::StorageKey appKey(upstreamSubQueueId); - int result = 0; + int result = 0; bslma::ManagedPtr message; - - mqbi::StorageResult::Enum rc = d_subStreamMessages_p->getIterator(&message, - appKey, - msgGUID); + mqbi::StorageResult::Enum rc = storage()->getIterator(&message, + app->appKey(), + msgGUID); if (rc == mqbi::StorageResult::e_SUCCESS) { - result = message->rdaInfo().counter(); + bmqp::RdaInfo& rda = + message->appMessageState(app->ordinal()).d_rdaInfo; + + result = rda.counter(); if (d_throttledRejectedMessages.requestPermission()) { BALL_LOG_INFO << "[THROTTLED] Queue '" << d_queueState_p->uri() << "' rejecting PUSH [GUID: '" << msgGUID - << "', subQueueId: " << app.upstreamSubQueueId() - << "] with the counter: [" << message->rdaInfo() - << "]"; + << "', subQueueId: " << app->upstreamSubQueueId() + << "] with the counter: [" << rda << "]"; } - if (!message->rdaInfo().isUnlimited()) { + if (!rda.isUnlimited()) { BSLS_ASSERT_SAFE(result); - message->rdaInfo().setPotentiallyPoisonous(true); - message->rdaInfo().setCounter(--result); + rda.setPotentiallyPoisonous(true); + rda.setCounter(--result); if (result == 0) { - // Inform the 'app' that 'msgGUID' is about to be removed from - // its virtual storage, so that app can advance its iterator - // etc if required. - - app.beforeMessageRemoved(msgGUID, false); - - d_subStreamMessages_p->remove(msgGUID, appKey); + // TODO d_subStreamMessages_p->remove(msgGUID, appKey); if (d_queueState_p->domain()->cluster()->isRemote()) { - rc = d_queueState_p->storage()->releaseRef(msgGUID, - appKey, - 0ULL, - true); + rc = + storage()->confirm(msgGUID, app->appKey(), 0ULL, true); if (mqbi::StorageResult::e_ZERO_REFERENCES == rc) { // Since there are no references, there should be no // app holding msgGUID and no need to call // `beforeMessageRemoved`. - BSLS_ASSERT_SAFE( - !d_subStreamMessages_p->hasMessage(msgGUID)); - d_queueState_p->storage()->remove(msgGUID, 0, true); + + storage()->remove(msgGUID, 0); } } } @@ -1471,7 +1477,7 @@ int RelayQueueEngine::onRejectMessage( BALL_LOG_INFO << "[THROTTLED] Queue '" << d_queueState_p->uri() << "' got reject for an unknown message [GUID: '" << msgGUID - << "', subQueueId: " << app.upstreamSubQueueId() << "]"; + << "', subQueueId: " << app->upstreamSubQueueId() << "]"; } return result; @@ -1485,18 +1491,8 @@ void RelayQueueEngine::beforeMessageRemoved(const bmqt::MessageGUID& msgGUID) BSLS_ASSERT_SAFE(d_queueState_p->queue()->dispatcher()->inDispatcherThread( d_queueState_p->queue())); - for (AppsMap::iterator iter = d_apps.begin(); iter != d_apps.end(); - ++iter) { - App_State& app(*iter->second); - - app.beforeMessageRemoved(msgGUID, true); - - mqbu::StorageKey appKey = mqbi::QueueEngine::k_DEFAULT_APP_KEY; - - if (d_queueState_p->hasMultipleSubStreams()) { - appKey = app.d_appKey; - } - d_subStreamMessages_p->remove(msgGUID, appKey); + if (!d_storageIter_mp->atEnd() && (d_storageIter_mp->guid() == msgGUID)) { + d_storageIter_mp->advance(); } } @@ -1526,43 +1522,19 @@ void RelayQueueEngine::afterQueuePurged(const bsl::string& appId, BSLS_ASSERT_SAFE(appId == bmqp::ProtocolUtil::k_NULL_APP_ID); // Purge all virtual storages, and reset all iterators. - d_subStreamMessages_p->removeAll(mqbu::StorageKey::k_NULL_KEY); + + d_pushStream.removeAll(); for (AppsMap::iterator it = d_apps.begin(); it != d_apps.end(); ++it) { - it->second->d_storageIter_mp->reset(); - it->second->d_redeliveryList.clear(); + it->second->clear(); } + d_storageIter_mp->reset(); } else { - // Purge virtual storage and reset iterator corresponding to - // the specified 'appKey', which can be constructed by using - // either upstream subId or storage-level appKey. - - // Lookup in 'd_subStreamMessages_p' first. - bsl::string appIdFound; - mqbu::StorageKey subStreamAppKey; - bool foundAppKey = true; - - if (d_subStreamMessages_p->hasVirtualStorage(appKey, &appIdFound)) { - BSLS_ASSERT_SAFE(appIdFound == appId); - - subStreamAppKey = appKey; - } - else if (d_queueState_p->storage()->hasVirtualStorage(appKey, - &appIdFound)) { - // Note that 'appId' may not exist in - // 'd_subStreamMessages_p' if that consumer is not - // connected (directly or indirectly) to this relay node. - - BSLS_ASSERT_SAFE(appIdFound == appId); - - foundAppKey = d_subStreamMessages_p->hasVirtualStorage( - appIdFound, - &subStreamAppKey); - } - else { - foundAppKey = false; + // Purge virtual storage corresponding to the specified 'appKey'. + AppIds::const_iterator itApp = d_appIds.find(appId); + if (itApp == d_appIds.end()) { BALL_LOG_ERROR << "#QUEUE_STORAGE_NOTFOUND " << "For queue '" << d_queueState_p->uri() << "', queueKey '" << d_queueState_p->key() @@ -1571,19 +1543,12 @@ void RelayQueueEngine::afterQueuePurged(const bsl::string& appId, << "' while attempting to purge its virtual " << "storage."; } + else { + // Clear out virtual storage corresponding to the App. + beforeOneAppRemoved(itApp->second->upstreamSubQueueId()); + d_pushStream.removeApp(itApp->second->upstreamSubQueueId()); - if (foundAppKey) { - // Clear out virtual storage corresponding to the retrieved - // 'subStreamAppKey'. - d_subStreamMessages_p->removeAll(subStreamAppKey); - - for (AppsMap::iterator it = d_apps.begin(); it != d_apps.end(); - ++it) { - if (it->second->d_appId == appIdFound) { - it->second->d_storageIter_mp->reset(); - it->second->d_redeliveryList.clear(); - } - } + itApp->second->clear(); } } } @@ -1626,14 +1591,10 @@ void RelayQueueEngine::loadInternals(mqbcmd::QueueEngine* out) const int numSubStreams = 0; mqbi::Storage::AppIdKeyPairs appIdKeyPairs; - if (d_subStreamMessages_p) { - numSubStreams = d_subStreamMessages_p->numVirtualStorages(); - d_subStreamMessages_p->loadVirtualStorageDetails(&appIdKeyPairs); - } - else { - numSubStreams = d_queueState_p->storage()->numVirtualStorages(); - d_queueState_p->storage()->loadVirtualStorageDetails(&appIdKeyPairs); - } + + numSubStreams = storage()->numVirtualStorages(); + storage()->loadVirtualStorageDetails(&appIdKeyPairs); + BSLS_ASSERT_SAFE(static_cast(appIdKeyPairs.size()) == numSubStreams); relayQueueEngine.numSubstreams() = numSubStreams; @@ -1650,10 +1611,8 @@ void RelayQueueEngine::loadInternals(mqbcmd::QueueEngine* out) const mwcu::MemOutStream appKey; appKey << p.second; subStream.appKey() = appKey.str(); - subStream.numMessages() = - (d_subStreamMessages_p - ? d_subStreamMessages_p->numMessages(p.second) - : d_queueState_p->storage()->numMessages(p.second)); + subStream.numMessages() = d_queueState_p->storage()->numMessages( + p.second); } } @@ -1671,9 +1630,64 @@ void RelayQueueEngine::loadInternals(mqbcmd::QueueEngine* out) const &relayQueueEngine.routing()); } +void RelayQueueEngine::registerStorage(const bsl::string& appId, + const mqbu::StorageKey& appKey, + unsigned int appOrdinal) +{ + // executed by the *QUEUE DISPATCHER* thread + + // PRECONDITIONS + BSLS_ASSERT_SAFE(d_queueState_p->queue()->dispatcher()->inDispatcherThread( + d_queueState_p->queue())); + + AppIds::iterator iter = d_appIds.find(appId); + + if (iter == d_appIds.end()) { + // No consumer has opened the queue with 'appId'. + + return; // RETURN + } + + // A consumer has already opened the queue with 'appId'. + + BALL_LOG_INFO << "Remote queue: " << d_queueState_p->uri() + << " (id: " << d_queueState_p->id() + << ") now has storage: [App Id: " << appId + << ", key: " << appKey << ", ordinal: " << appOrdinal << "]"; + + iter->second->authorize(appKey, appOrdinal); +} + +void RelayQueueEngine::unregisterStorage(const bsl::string& appId, + const mqbu::StorageKey& appKey, + unsigned int appOrdinal) +{ + // executed by the *QUEUE DISPATCHER* thread + + // PRECONDITIONS + BSLS_ASSERT_SAFE(d_queueState_p->queue()->dispatcher()->inDispatcherThread( + d_queueState_p->queue())); + + AppIds::iterator iter = d_appIds.find(appId); + mqbu::StorageKey key; + + if (iter == d_appIds.end()) { + // No consumer has opened the queue with 'appId'. + } + else { + // A consumer has already opened the queue with 'appId'. + BSLS_ASSERT_SAFE(iter->second->appKey() == appKey); + + iter->second->unauthorize(); + } + + (void)appOrdinal; +} + bool RelayQueueEngine::subscriptionId2upstreamSubQueueId( - unsigned int* subQueueId, - unsigned int subscriptionId) const + const bmqt::MessageGUID& msgGUID, + unsigned int* subQueueId, + unsigned int subscriptionId) const { if (subscriptionId == bmqp::Protocol::k_DEFAULT_SUBSCRIPTION_ID) { BSLS_ASSERT_SAFE(d_apps.find(bmqp::QueueId::k_DEFAULT_SUBQUEUE_ID) != @@ -1684,11 +1698,26 @@ bool RelayQueueEngine::subscriptionId2upstreamSubQueueId( const Routers::SubscriptionIds::SharedItem itId = d_queueState_p->routingContext().d_groupIds.find(subscriptionId); if (!itId) { + BMQ_LOGTHROTTLE_ERROR() + << "#QUEUE_UNKNOWN_SUBSCRIPTION_ID " + << "Remote queue: " << d_queueState_p->uri() + << " (id: " << d_queueState_p->id() + << ") received a PUSH message for guid " << msgGUID + << ", with unknown Subscription Id " << subscriptionId; + return false; // RETURN } if (itId->value().d_priorityGroup == 0) { // The 'd_queueContext.d_groupIds' may contain new ids for which // configure response is not received yet. + + BMQ_LOGTHROTTLE_ERROR() + << "#QUEUE_UNKNOWN_SUBSCRIPTION_ID " + << "Remote queue: " << d_queueState_p->uri() + << " (id: " << d_queueState_p->id() + << ") received a PUSH message for guid " << msgGUID + << ", with unconfigured Subscription Id " << subscriptionId; + return false; } *subQueueId = itId->value().upstreamSubQueueId(); @@ -1696,5 +1725,200 @@ bool RelayQueueEngine::subscriptionId2upstreamSubQueueId( return true; } +unsigned int +RelayQueueEngine::push(mqbi::StorageMessageAttributes* attributes, + const bmqt::MessageGUID& msgGUID, + const bsl::shared_ptr& appData, + const bmqp::Protocol::SubQueueInfosArray& subscriptions, + bool isOutOfOrder) +{ + if (isOutOfOrder) { + BSLS_ASSERT_SAFE(subscriptions.size() == 1); + + // No guarantee of uniqueness. Cannot use PushStream. + unsigned int upstreamSubQueueId; + + if (subscriptionId2upstreamSubQueueId(msgGUID, + &upstreamSubQueueId, + subscriptions.begin()->id())) { + App_State* app = findApp(upstreamSubQueueId); + + if (app == 0) { + BMQ_LOGTHROTTLE_ERROR() + << "#QUEUE_UNKNOWN_SUBSCRIPTION_ID " + << "Remote queue: " << d_queueState_p->uri() + << " (id: " << d_queueState_p->id() + << ") discarding a PUSH message for guid " << msgGUID + << ", with unknown App Id " << upstreamSubQueueId; + + return 0; // RETURN + } + + if (!checkForDuplicate(app, msgGUID)) { + return 0; // RETURN + } + + app->putForRedelivery(msgGUID); + + attributes->setRefCount(1); + + storePush(attributes, msgGUID, appData, true); + + // Attempt to deliver + processAppRedelivery(upstreamSubQueueId, app); + + return 1; // RETURN + } + + return 0; // RETURN + } + + // Count only those subQueueIds which 'storage' is aware of. + + PushStream::iterator itGuid = d_pushStream.findOrAppendMessage(msgGUID); + unsigned int count = 0; + + for (bmqp::Protocol::SubQueueInfosArray::const_iterator cit = + subscriptions.begin(); + cit != subscriptions.end(); + ++cit) { + const bmqp::SubQueueInfo& subscription = *cit; + + unsigned int subQueueId; + + if (!subscriptionId2upstreamSubQueueId(msgGUID, + &subQueueId, + subscription.id())) { + continue; // CONTINUE + } + + PushStream::Apps::iterator itApp = d_pushStream.d_apps.find( + subQueueId); + if (itApp == d_pushStream.d_apps.end()) { + AppsMap::const_iterator cit = d_apps.find(subQueueId); + + if (cit == d_apps.end()) { + BMQ_LOGTHROTTLE_ERROR() + << "#QUEUE_UNKNOWN_SUBSCRIPTION_ID " + << "Remote queue: " << d_queueState_p->uri() + << " (id: " << d_queueState_p->id() + << ") discarding a PUSH message for guid " << msgGUID + << ", with unknown App Id " << subscription.id(); + continue; // CONTINUE + } + + itApp = d_pushStream.d_apps.emplace(subQueueId, cit->second).first; + } + else if (!checkForDuplicate(itApp->second.d_app.get(), msgGUID)) { + continue; // CONTINUE + } + + PushStream::Element* element = d_pushStream.create(subscription, + itGuid, + itApp); + + d_pushStream.add(element); + ++count; + } + + if (count) { + // Pass correct ref count + attributes->setRefCount(count); + storePush(attributes, msgGUID, appData, false); + } + return count; +} + +bool RelayQueueEngine::checkForDuplicate(const App_State* app, + const bmqt::MessageGUID& msgGUID) +{ + // Check the storage if this is duplicate PUSH + // Currently, only Proxies can do this because they clear their storage + // once all readers are gone. Therefore, a Proxy can detect duplicate PUSH + // when there is a client (potentially) processing the same PUSH. + // Replicas do not clear their (replicated) storage. They do not check if + // there is a client (potentially) processing the same PUSH. + // Also, Proxies use InMemoryStorage which relies on PUSH uniqueness for + // calculating the refCount. + // (Replicas receive the refCount by replication). + + if (d_queueState_p->domain()->cluster()->isRemote()) { + d_realStorageIter_mp->reset(msgGUID); + if (!d_realStorageIter_mp->atEnd()) { + mqbi::AppMessage& appState = d_realStorageIter_mp->appMessageState( + app->ordinal()); + + if (!appState.isPushing()) { + appState.setPushState(); + } + else { + BMQ_LOGTHROTTLE_INFO() + << "Remote queue: " << d_queueState_p->uri() + << " (id: " << d_queueState_p->id() << ", App '" + << app->appId() + << "') discarding a duplicate PUSH for guid " << msgGUID; + return false; // RETURN + } + } + } + return true; +} + +void RelayQueueEngine::storePush(mqbi::StorageMessageAttributes* attributes, + const bmqt::MessageGUID& msgGUID, + const bsl::shared_ptr& appData, + bool isOutOfOrder) +{ + if (d_queueState_p->domain()->cluster()->isRemote()) { + // Save the message along with the subIds in the storage. Note that + // for now, we will assume that in fanout mode, the only option present + // in 'options' is subQueueInfos, and we won't store the specified + // 'options' in the storage. + + mqbi::StorageResult::Enum result = mqbi::StorageResult::e_SUCCESS; + result = storage()->put(attributes, + msgGUID, + appData, + bsl::shared_ptr() // No options + ); + + if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( + result != mqbi::StorageResult::e_SUCCESS)) { + BSLS_PERFORMANCEHINT_UNLIKELY_HINT; + + if (result != mqbi::StorageResult::e_GUID_NOT_UNIQUE || + isOutOfOrder) { + BMQ_LOGTHROTTLE_INFO() + << d_queueState_p->uri() << " failed to store GUID [" + << msgGUID << "], result = " << result; + } + // A redelivery PUSH for one App in the presence of another App + // can result in 'e_GUID_NOT_UNIQUE'. + } + } +} + +void RelayQueueEngine::beforeOneAppRemoved(unsigned int upstreamSubQueueId) +{ + while (!d_storageIter_mp->atEnd()) { + if (d_storageIter_mp->numApps() > 1) { + // Removal of App's elements will not invalidate 'd_storageIter_mp' + break; + } + + const PushStream::Element* element = d_storageIter_mp->element(0); + if (element->app().d_app->upstreamSubQueueId() != upstreamSubQueueId) { + break; + } + + d_storageIter_mp->advance(); + } +} + +mqbi::Storage* RelayQueueEngine::storage() const +{ + return d_queueState_p->storage(); +} + } // close package namespace } // close enterprise namespace diff --git a/src/groups/mqb/mqbblp/mqbblp_relayqueueengine.h b/src/groups/mqb/mqbblp/mqbblp_relayqueueengine.h index 3f306cf24..1515c3ecf 100644 --- a/src/groups/mqb/mqbblp/mqbblp_relayqueueengine.h +++ b/src/groups/mqb/mqbblp/mqbblp_relayqueueengine.h @@ -38,6 +38,7 @@ // MQB +#include #include #include #include @@ -127,14 +128,12 @@ struct RelayQueueEngine_AppState : QueueEngineUtil_AppState { // CREATORS explicit RelayQueueEngine_AppState( - unsigned int upstreamSubQueueId, - const bsl::string& appId, - bslma::ManagedPtr iterator, - mqbi::Queue* queue, - bdlmt::EventScheduler* scheduler, - const mqbu::StorageKey& appKey, - Routers::QueueRoutingContext& queueContext, - bslma::Allocator* allocator = 0); + unsigned int upstreamSubQueueId, + const bsl::string& appId, + mqbi::Queue* queue, + bdlmt::EventScheduler* scheduler, + Routers::QueueRoutingContext& queueContext, + bslma::Allocator* allocator = 0); }; // ====================== @@ -161,6 +160,9 @@ class RelayQueueEngine : public mqbi::QueueEngine { /// (subId) -> App_State map typedef bsl::unordered_map AppsMap; + /// (appId) -> App_State map + typedef bsl::unordered_map AppIds; + class AutoPurger; // A guard helper class. @@ -210,36 +212,33 @@ class RelayQueueEngine : public mqbi::QueueEngine { // DATA QueueState* d_queueState_p; - mqbs::VirtualStorageCatalog* d_subStreamMessages_p; - // List of messages that need to be - // delivered to sub-streams, as indicated - // by the upstream node. Note that this - // variable is non null only if self node - // is a replica *and* queue is not in - // broadcast mode. If non null, the object - // itself is owned by the associated - // RemoteQueue instance. + PushStream d_pushStream; const mqbconfm::Domain d_domainConfig; AppsMap d_apps; // Map of (appId) to App_State. + AppIds d_appIds; + // (appId) -> App_State map + mwcu::SharedResource d_self; // Used to avoid executing a callback if // the engine has been destroyed. For // example, upon queue converting to local. - bdlmt::EventScheduler* d_scheduler_p; - // Event scheduler currently used for - // message throttling. Held, not owned. - bdlmt::Throttle d_throttledRejectedMessages; // Throttler for REJECTs. QueueEngineUtil_AppsDeliveryContext d_appsDeliveryContext; // Reusable apps delivery context + bslma::ManagedPtr d_storageIter_mp; + // Storage iterator to the PushStream + + bslma::ManagedPtr d_realStorageIter_mp; + // Storage iterator to access storage state. + bslma::Allocator* d_allocator_p; // Allocator to use. private: @@ -312,8 +311,7 @@ class RelayQueueEngine : public mqbi::QueueEngine { /// consumers. Behavior is undefined unless there is at least one /// active consumer. void deliverMessages(); - - void processAppRedelivery(App_State& state, const bsl::string& appId); + void processAppRedelivery(unsigned int upstreamSubQueueId, App_State* app); /// Configure the specified `handle` with the specified /// `streamParameters` for the specified `appState`. When the specified @@ -347,6 +345,20 @@ class RelayQueueEngine : public mqbi::QueueEngine { /// THREAD: This method is called from the Queue's dispatcher thread. void applyConfiguration(App_State& appState, ConfigureContext& context); + mqbi::Storage* storage() const; + + App_State* findApp(unsigned int upstreamSubQueueId) const; + + bool checkForDuplicate(const App_State* app, + const bmqt::MessageGUID& msgGUID); + + void storePush(mqbi::StorageMessageAttributes* attributes, + const bmqt::MessageGUID& msgGUID, + const bsl::shared_ptr& appData, + bool isOutOfOrder); + + void beforeOneAppRemoved(unsigned int upstreamSubQueueId); + private: // NOT IMPLEMENTED @@ -360,10 +372,9 @@ class RelayQueueEngine : public mqbi::QueueEngine { public: // CREATORS - RelayQueueEngine(QueueState* queueState, - mqbs::VirtualStorageCatalog* subStreamMessages, - const mqbconfm::Domain& domainConfig, - bslma::Allocator* allocator); + RelayQueueEngine(QueueState* queueState, + const mqbconfm::Domain& domainConfig, + bslma::Allocator* allocator); /// Destructor virtual ~RelayQueueEngine() BSLS_KEYWORD_OVERRIDE; @@ -501,6 +512,24 @@ class RelayQueueEngine : public mqbi::QueueEngine { virtual void onTimer(bsls::Types::Int64 currentTimer) BSLS_KEYWORD_OVERRIDE; + /// Called after creation of a new storage for the specified + /// `appIdKeyPair`. + /// + /// THREAD: This method is called from the Queue's dispatcher thread. + virtual void + registerStorage(const bsl::string& appId, + const mqbu::StorageKey& appKey, + unsigned int appOrdinal) BSLS_KEYWORD_OVERRIDE; + + /// Called after removal of the storage for the specified + /// `appIdKeyPair`. + /// + /// THREAD: This method is called from the Queue's dispatcher thread. + virtual void + unregisterStorage(const bsl::string& appId, + const mqbu::StorageKey& appKey, + unsigned int appOrdinal) BSLS_KEYWORD_OVERRIDE; + /// Not valid for 'RelayQueueEngine' mqbi::StorageResult::Enum evaluateAutoSubscriptions( const bmqp::PutHeader& putHeader, @@ -508,6 +537,19 @@ class RelayQueueEngine : public mqbi::QueueEngine { const bmqp::MessagePropertiesInfo& mpi, bsls::Types::Uint64 timestamp) BSLS_KEYWORD_OVERRIDE; + // MANIPULATORS + + /// If the specified `isOutOfOrder` is `true`, insert the specified + // `msgGUID` into the corresponding App redelivery list. Otherwise, insert + // the `msgGUID` into the PushStream; insert PushStream Elements + // (`mqbi::AppMessage`, `upstreamSubQueueId`) pairs for each recognized App + /// in the specified `subscriptions`. + /// Return number of inserted PushStream Elements. + unsigned int push(mqbi::StorageMessageAttributes* attributes, + const bmqt::MessageGUID& msgGUID, + const bsl::shared_ptr& appData, + const bmqp::Protocol::SubQueueInfosArray& subscriptions, + bool isOutOfOrder); // ACCESSORS /// Return the reference count that should be applied to a message @@ -521,13 +563,12 @@ class RelayQueueEngine : public mqbi::QueueEngine { virtual void loadInternals(mqbcmd::QueueEngine* out) const BSLS_KEYWORD_OVERRIDE; - // ACCESSORS - /// Load upstream subQueue id into the specified `subQueueId` given the /// specified upstream `subscriptionId`. /// Each subStream has unique Subscription ids. - bool subscriptionId2upstreamSubQueueId(unsigned int* subQueueId, - unsigned int subscriptionId) const; + bool subscriptionId2upstreamSubQueueId(const bmqt::MessageGUID& msgGUID, + unsigned int* subQueueId, + unsigned int subscriptionId) const; }; // ============================================================================ @@ -539,22 +580,18 @@ class RelayQueueEngine : public mqbi::QueueEngine { // -------------------------------- inline RelayQueueEngine_AppState::RelayQueueEngine_AppState( - unsigned int upstreamSubQueueId, - const bsl::string& appId, - bslma::ManagedPtr iterator, - mqbi::Queue* queue, - bdlmt::EventScheduler* scheduler, - const mqbu::StorageKey& appKey, - Routers::QueueRoutingContext& queueContext, - bslma::Allocator* allocator) -: QueueEngineUtil_AppState(iterator, - queue, + unsigned int upstreamSubQueueId, + const bsl::string& appId, + mqbi::Queue* queue, + bdlmt::EventScheduler* scheduler, + Routers::QueueRoutingContext& queueContext, + bslma::Allocator* allocator) +: QueueEngineUtil_AppState(queue, scheduler, - true, queueContext, upstreamSubQueueId, appId, - appKey, + mqbu::StorageKey::k_NULL_KEY, allocator) , d_cache(allocator) { @@ -634,6 +671,19 @@ inline void RelayQueueEngine::ConfigureContext::initializeRouting( // class RelayQueueEngine // ---------------------- +inline RelayQueueEngine::App_State* +RelayQueueEngine::findApp(unsigned int upstreamSubQueueId) const +{ + AppsMap::const_iterator cit = d_apps.find(upstreamSubQueueId); + + if (cit == d_apps.end()) { + return 0; + } + else { + return cit->second.get(); + } +} + inline unsigned int RelayQueueEngine::messageReferenceCount() const { // Irrespective of number of worker-consumers or their status (dead or diff --git a/src/groups/mqb/mqbblp/mqbblp_relayqueueengine.t.cpp b/src/groups/mqb/mqbblp/mqbblp_relayqueueengine.t.cpp index 5aea4b5df..49dcb33db 100644 --- a/src/groups/mqb/mqbblp/mqbblp_relayqueueengine.t.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_relayqueueengine.t.cpp @@ -109,7 +109,7 @@ static void test1_breathingTest() // 2) Post 3 messages to the queue, and invoke the engine to deliver them // to the highest priority consumers - tester.post("1,2,3"); + tester.post("1,2,3", guard.engine()); tester.afterNewMessage(3); @@ -194,7 +194,7 @@ static void test2_aggregateDownstream() // C1: 1 // C2: 2 // C3: 1 - tester.post("1,2,3,4"); + tester.post("1,2,3,4", guard.engine()); tester.afterNewMessage(4); ASSERT_EQ(C1->_numMessages(), 1); @@ -220,7 +220,7 @@ static void test2_aggregateDownstream() // C1: 1 // C2: 2 // C3: 3 - tester.post("5,6,7,8,9,10"); + tester.post("5,6,7,8,9,10", guard.engine()); tester.afterNewMessage(6); ASSERT_EQ(C1->_numMessages(), 1); @@ -244,7 +244,7 @@ static void test2_aggregateDownstream() // C1: 1 // C2: 2 // C3: 2 - tester.post("11,12,13,14,15"); + tester.post("11,12,13,14,15", guard.engine()); tester.afterNewMessage(5); ASSERT_EQ(C1->_numMessages(), 1); @@ -268,7 +268,7 @@ static void test2_aggregateDownstream() // C1: 1 // C2: 1 // C3: 2 - tester.post("16,17,18,19"); + tester.post("16,17,18,19", guard.engine()); tester.afterNewMessage(4); ASSERT_EQ(C1->_numMessages(), 1); @@ -291,7 +291,7 @@ static void test2_aggregateDownstream() // C1: 1 // C2: 0 // C3: 2 - tester.post("20,21,22,23,24,25"); + tester.post("20,21,22,23,24,25", guard.engine()); tester.afterNewMessage(6); ASSERT_EQ(C1->_numMessages(), 2); @@ -359,7 +359,7 @@ static void test3_reconfigure() // C1: 1 // C2: 1 - tester.post("1,2"); + tester.post("1,2", guard.engine()); tester.afterNewMessage(2); ASSERT_EQ(C1->_numMessages(), 1); @@ -377,7 +377,7 @@ static void test3_reconfigure() // C1: 0 // C2: 1 - tester.post("3,4"); + tester.post("3,4", guard.engine()); tester.afterNewMessage(2); ASSERT_EQ(C1->_numMessages(), 0); @@ -392,7 +392,7 @@ static void test3_reconfigure() // C1: 1 // C2: 1 - tester.post("5,6,7,8"); + tester.post("5,6,7,8", guard.engine()); tester.afterNewMessage(4); ASSERT_EQ(C1->_numMessages(), 2); @@ -412,7 +412,7 @@ static void test3_reconfigure() // C1: 0 // C2: 0 // C3: 1 - tester.post("9,10"); + tester.post("9,10", guard.engine()); tester.afterNewMessage(2); ASSERT_EQ(C1->_numMessages(), 0); @@ -430,7 +430,7 @@ static void test3_reconfigure() // C1: 2 // C2: 1 // C3: 1 - tester.post("11,12,13,14,15,16,17,18"); + tester.post("11,12,13,14,15,16,17,18", guard.engine()); tester.afterNewMessage(8); ASSERT_EQ(C1->_numMessages(), 4); @@ -498,7 +498,7 @@ static void test4_cannotDeliver() // C1: 1 // C2: 1 - tester.post("1,2"); + tester.post("1,2", guard.engine()); tester.afterNewMessage(2); ASSERT_EQ(C1->_numMessages(), 1); @@ -516,7 +516,7 @@ static void test4_cannotDeliver() // C1: N.A. // C2: 1 - tester.post("3,4"); + tester.post("3,4", guard.engine()); tester.afterNewMessage(2); ASSERT_EQ(C1->_numMessages(), 0); @@ -531,7 +531,7 @@ static void test4_cannotDeliver() // C1: N.A. // C2: N.A. - tester.post("5"); + tester.post("5", guard.engine()); tester.afterNewMessage(1); ASSERT_EQ(C1->_numMessages(), 0); @@ -555,7 +555,7 @@ static void test4_cannotDeliver() // C1: 1 // C2: 1 - tester.post("6,7,8,9"); + tester.post("6,7,8,9", guard.engine()); tester.afterNewMessage(4); ASSERT_EQ(C1->_numMessages(), 2); @@ -613,7 +613,7 @@ static void test5_localRedelivery() // C1: 1 // C2: 1 - tester.post("1,2,3,4"); + tester.post("1,2,3,4", guard.engine()); tester.afterNewMessage(4); ASSERT_EQ(C1->_numMessages(), 2); @@ -689,7 +689,7 @@ static void test6_clearDeliveryStateWhenLostReaders() tester.configureHandle("C1 consumerPriority=1 consumerPriorityCount=1"); // C1: 2 - tester.post("1,2"); + tester.post("1,2", guard.engine()); tester.afterNewMessage(2); ASSERT_EQ(C1->_numMessages(), 2); @@ -701,7 +701,7 @@ static void test6_clearDeliveryStateWhenLostReaders() C1->_setCanDeliver(false); // C1: N.A. - tester.post("3"); + tester.post("3", guard.engine()); tester.afterNewMessage(1); ASSERT_EQ(C1->_numMessages(), 1); @@ -763,9 +763,9 @@ static void test7_broadcastMode() // Note: For the RelayQueueEngine, one has to call 'afterNewMessage()' // after each single message. We couldn't do 'tester.post("1,2");'. - tester.post("1"); + tester.post("1", guard.engine()); tester.afterNewMessage(1); - tester.post("2"); + tester.post("2", guard.engine()); tester.afterNewMessage(1); ASSERT_EQ(C1->_messages(), "1,2"); @@ -776,7 +776,7 @@ static void test7_broadcastMode() // 2) C1: Can't deliver C1->_setCanDeliver(false); - tester.post("3"); + tester.post("3", guard.engine()); tester.afterNewMessage(1); ASSERT_EQ(C1->_messages(), ""); @@ -787,7 +787,7 @@ static void test7_broadcastMode() // 3) C1: Can deliver again C1->_setCanDeliver(true); - tester.post("4"); + tester.post("4", guard.engine()); tester.afterNewMessage(1); ASSERT_EQ(C1->_messages(), "4"); @@ -798,7 +798,7 @@ static void test7_broadcastMode() // 4) C1: Dropped tester.dropHandle("C1"); - tester.post("5"); + tester.post("5", guard.engine()); tester.afterNewMessage(1); ASSERT_EQ(C2->_messages(), "5"); @@ -852,7 +852,7 @@ static void test8_priority_beforeMessageRemoved_garbageCollection() tester.configureHandle("C1 consumerPriority=1 consumerPriorityCount=1"); PV(L_ << ": post ['1','2','3','4']"); - tester.post("1,2,3,4"); + tester.post("1,2,3,4", guard.engine()); PVV(L_ << ": C1 Messages: " << C1->_messages()); ASSERT_EQ(C1->_numMessages(), 0); @@ -1063,7 +1063,12 @@ static void test11_roundRobinAndRedelivery() mwctst::TestHelper::printTestName("ROUND-ROBIN AND REDELIVERY"); mqbconfm::Domain config = fanoutConfig(); - mqbblp::QueueEngineTester tester(fanoutConfig(), + bsl::vector& appIDs = config.mode().fanout().appIDs(); + appIDs.push_back("a"); + appIDs.push_back("b"); + appIDs.push_back("c"); + + mqbblp::QueueEngineTester tester(config, false, // start scheduler s_allocator_p); mqbblp::QueueEngineTesterGuard guard(&tester); @@ -1100,7 +1105,7 @@ static void test11_roundRobinAndRedelivery() 0); // 2. Post 3 messages (2 + 1) - tester.post("1,2,3"); + tester.post("1,2,3", guard.engine()); tester.afterNewMessage(3); // 3. Verify that each consumer received messages according to @@ -1175,7 +1180,7 @@ static void test12_redeliverAfterGc() tester.configureHandle("C2 consumerPriority=1 consumerPriorityCount=1" " maxUnconfirmedMessages=0"); - tester.post("1,2,3,4"); + tester.post("1,2,3,4", guard.engine()); tester.afterNewMessage(4); PVV(L_ << ": C1 Messages: " << C1->_messages()); @@ -1313,7 +1318,7 @@ static void test14_throttleRedeliveryPriority() mqbmock::QueueHandle* C2 = tester.getHandle("C2 readCount=1"); tester.configureHandle("C2 consumerPriority=1 consumerPriorityCount=1"); - tester.post("1,2,3,4"); + tester.post("1,2,3,4", guard.engine()); tester.afterNewMessage(4); PVV(L_ << ": C1 Messages: " << C1->_messages()); @@ -1420,7 +1425,7 @@ static void test15_throttleRedeliveryFanout() tester.configureHandle("C4@c maxUnconfirmedMessages=4" " consumerPriority=1 consumerPriorityCount=1"); - tester.post("1,2,3,4"); + tester.post("1,2,3,4", guard.engine()); tester.afterNewMessage(4); PVV(L_ << ": C1 Messages: " << C1->_messages("a")); @@ -1551,7 +1556,7 @@ static void test16_throttleRedeliveryCancelledDelay() mqbmock::QueueHandle* C2 = tester.getHandle("C2 readCount=1"); tester.configureHandle("C2 consumerPriority=1 consumerPriorityCount=1"); - tester.post("1,2,3,4"); + tester.post("1,2,3,4", guard.engine()); tester.afterNewMessage(4); // 2) @@ -1631,7 +1636,7 @@ static void test17_throttleRedeliveryNewHandle() mqbmock::QueueHandle* C2 = tester.getHandle("C2 readCount=1"); tester.configureHandle("C2 consumerPriority=1 consumerPriorityCount=1"); - tester.post("1,2"); + tester.post("1,2", guard.engine()); tester.afterNewMessage(2); // 2) @@ -1706,7 +1711,7 @@ static void test18_throttleRedeliveryNoMoreHandles() mqbmock::QueueHandle* C2 = tester.getHandle("C2 readCount=1"); tester.configureHandle("C2 consumerPriority=1 consumerPriorityCount=1"); - tester.post("1,2"); + tester.post("1,2", guard.engine()); tester.afterNewMessage(2); // 2) diff --git a/src/groups/mqb/mqbblp/mqbblp_remotequeue.cpp b/src/groups/mqb/mqbblp/mqbblp_remotequeue.cpp index 76de65acb..bbd579fa8 100644 --- a/src/groups/mqb/mqbblp/mqbblp_remotequeue.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_remotequeue.cpp @@ -106,7 +106,6 @@ int RemoteQueue::configureAsProxy(bsl::ostream& errorDescription, mqbs::DataStore::k_INVALID_PARTITION_ID, domainCfg, d_state_p->domain()->capacityMeter(), - bmqp::RdaInfo(), d_allocator_p), d_allocator_p); @@ -138,20 +137,12 @@ int RemoteQueue::configureAsProxy(bsl::ostream& errorDescription, d_state_p->setStorage(storageMp); - // Create SubStreamMessages list. - d_subStreamMessages_mp.load( + // Create the queueEngine. + d_queueEngine_mp.load( new (*d_allocator_p) - mqbs::VirtualStorageCatalog(d_state_p->storage(), d_allocator_p), + RelayQueueEngine(d_state_p, mqbconfm::Domain(), d_allocator_p), d_allocator_p); - // Create the queueEngine. - d_queueEngine_mp.load(new (*d_allocator_p) - RelayQueueEngine(d_state_p, - d_subStreamMessages_mp.get(), - mqbconfm::Domain(), - d_allocator_p), - d_allocator_p); - rc = d_queueEngine_mp->configure(errorDescription); if (rc != 0) { return 10 * rc + rc_QUEUE_ENGINE_CFG_FAILURE; // RETURN @@ -224,12 +215,6 @@ int RemoteQueue::configureAsClusterMember(bsl::ostream& errorDescription, storageMp->capacityMeter()->disable(); } - // Create SubStreamMessages list. - d_subStreamMessages_mp.load( - new (*d_allocator_p) - mqbs::VirtualStorageCatalog(storageMp.get(), d_allocator_p), - d_allocator_p); - if (!d_state_p->isStorageCompatible(storageMp)) { MWCTSK_ALARMLOG_ALARM("CLUSTER_STATE") << d_state_p->domain()->cluster()->name() << ": Partition [" @@ -244,10 +229,8 @@ int RemoteQueue::configureAsClusterMember(bsl::ostream& errorDescription, // Create the queueEngine. d_queueEngine_mp.load( - new (*d_allocator_p) RelayQueueEngine(d_state_p, - d_subStreamMessages_mp.get(), - domainCfg, - d_allocator_p), + new (*d_allocator_p) + RelayQueueEngine(d_state_p, domainCfg, d_allocator_p), d_allocator_p); } else { @@ -332,6 +315,26 @@ bool RemoteQueue::loadSubQueueInfos( return subQueueInfos->size(); } +// We have two VirtualStorageCatalog in the case of RemoteQueue (one for PUTs, +// one for PUSHes. +// And one in the case of LocalQueue. +// +// LocalQueue ----------------> Storage +// / | +// RemoteQueue --------------- | +// | | +// V V +// VirtualStorageCatalog (2) VirtualStorageCatalog (1) +// + +// PUT Replication Broadcast PUSH non-Broadcast PUSH +// -------------------------------------------------------------------------- +// LocalQueue (1) +// Replica (1) (1) (2 (some Apps)) (2 (some Apps)) +// Proxy (1 (some Apps)) (1 (some Apps)) + +// Redundant storage at Proxy + void RemoteQueue::pushMessage( const bmqt::MessageGUID& msgGUID, const bsl::shared_ptr& appData, @@ -361,10 +364,20 @@ void RemoteQueue::pushMessage( BSLS_ASSERT_SAFE(appData); result = storage->put(&attributes, msgGUID, appData, options); + + if (result != mqbi::StorageResult::e_SUCCESS) { + if (d_throttledFailedPushMessages.requestPermission()) { + BALL_LOG_WARN << d_state_p->uri() + << " failed to store broadcast PUSH [" + << msgGUID << "], result = " << result; + } + } } else { // In a replica, 'appData' must be empty in non-broadcast mode. BSLS_ASSERT_SAFE(!appData); + + // Insert into the ShortList } // 'msgGUID' must be present in the storage. @@ -416,90 +429,11 @@ void RemoteQueue::pushMessage( BSLS_ASSERT_SAFE(d_state_p->hasMultipleSubStreams() || subQueueInfos.size() == 1); - // Collect only those subQueueIds which 'storage' is aware of. - storageKeys.reserve(subQueueInfos.size()); - - for (bmqp::Protocol::SubQueueInfosArray::const_iterator cit = - subQueueInfos.begin(); - cit != subQueueInfos.end(); - ++cit) { - unsigned int subQueueId; - - if (!d_queueEngine_mp->subscriptionId2upstreamSubQueueId(&subQueueId, - cit->id())) { - if (d_throttledFailedPushMessages.requestPermission()) { - BALL_LOG_ERROR << "#QUEUE_UNKNOWN_SUBSCRIPTION_ID " - << "Remote queue: " << d_state_p->uri() - << " (id: " << d_state_p->id() - << ") received a PUSH message for guid " - << msgGUID << ", with unknown Subscription Id " - << cit->id(); - } - return; // RETURN - } - - const mqbu::StorageKey appKey(subQueueId); - - if (BSLS_PERFORMANCEHINT_PREDICT_LIKELY( - d_subStreamMessages_mp->hasVirtualStorage(appKey))) { - storageKeys.push_back(appKey); - BSLS_ASSERT_SAFE(cit->rdaInfo().isUnlimited() || - cit->rdaInfo().counter()); - - result = d_subStreamMessages_mp->put(msgGUID, - msgSize, - cit->rdaInfo(), - cit->id(), - appKey); - - if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( - result != mqbi::StorageResult::e_SUCCESS)) { - BSLS_PERFORMANCEHINT_UNLIKELY_HINT; - if (d_throttledFailedPushMessages.requestPermission()) { - BALL_LOG_WARN << d_state_p->uri() - << " failed to store PUSH message [" - << msgGUID << "], app [" << *cit - << "], result = " << result; - } - } - } - else { - BSLS_PERFORMANCEHINT_UNLIKELY_HINT; - BALL_LOG_ERROR << "#QUEUE_STORAGE_NOTFOUND " - << "For a PUSH message for queue [" - << d_state_p->description() << "] and GUID [" - << msgGUID << "], received a subQueueId [" - << subQueueId << "] in SubQueueIdsOption" - << " for which virtual storage does not exist in " - << "the storage. AppKey corresponding to the " - << "subQueueId: [" << appKey << "]."; - } - } - - if (d_state_p->domain()->cluster()->isRemote()) { - // Save the message along with the subIds in the storage. Note that - // for now, we will assume that in fanout mode, the only option present - // in 'options' is subQueueInfos, and we won't store the specified - // 'options' in the storage. - - attributes.setRefCount(storageKeys.size()); - // Pass correct ref count - - result = storage->put(&attributes, - msgGUID, - appData, - bsl::shared_ptr(), // No options - storageKeys); - - if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( - result != mqbi::StorageResult::e_SUCCESS)) { - BSLS_PERFORMANCEHINT_UNLIKELY_HINT; - if (d_throttledFailedPushMessages.requestPermission()) { - BALL_LOG_INFO << d_state_p->uri() << " failed to store GUID [" - << msgGUID << "], result = " << result; - } - } - } + d_queueEngine_mp->push(&attributes, + msgGUID, + appData, + subQueueInfos, + isOutOfOrder); // 'flush' will inform the queue engine so it delivers the message. } @@ -517,7 +451,6 @@ RemoteQueue::RemoteQueue(QueueState* state, , d_optionsView(allocator) , d_pendingPutsTimeoutNs(deduplicationTimeMs * bdlt::TimeUnitRatio::k_NANOSECONDS_PER_MILLISECOND) -, d_scheduler_p(state->scheduler()) , d_pendingMessagesTimerEventHandle() , d_ackWindowSize(ackWindowSize) , d_unackedPutCounter(0) @@ -598,7 +531,7 @@ void RemoteQueue::resetState() erasePendingMessages(d_pendingMessages.end()); if (d_pendingMessagesTimerEventHandle) { - d_scheduler_p->cancelEventAndWait(&d_pendingMessagesTimerEventHandle); + scheduler()->cancelEventAndWait(&d_pendingMessagesTimerEventHandle); // 'expirePendingMessagesDispatched' does not restart timer if // 'd_pendingMessages' is empty } @@ -630,7 +563,7 @@ void RemoteQueue::close() BSLS_ASSERT_SAFE(d_pendingMessages.size() == 0); if (d_pendingMessagesTimerEventHandle) { - d_scheduler_p->cancelEventAndWait(&d_pendingMessagesTimerEventHandle); + scheduler()->cancelEventAndWait(&d_pendingMessagesTimerEventHandle); // 'expirePendingMessagesDispatched' does not restart timer if // 'd_pendingMessages' is empty } @@ -648,16 +581,12 @@ void RemoteQueue::getHandle( BSLS_ASSERT_SAFE(d_state_p->queue()->dispatcher()->inDispatcherThread( d_state_p->queue())); - mqbi::QueueHandle* handle = d_queueEngine_mp->getHandle(clientContext, - handleParameters, - upstreamSubQueueId, - callback); - mqbi::Cluster* cluster = d_state_p->domain()->cluster(); + if (cluster->isRemote()) { - if (d_state_p->hasMultipleSubStreams() && handle && + if (d_state_p->hasMultipleSubStreams() && !handleParameters.subIdInfo().isNull()) { - // Successful 'getHandle' in fanout mode at proxy: need to add + // In fanout mode at proxy need to add // virtual storage to the physical storage bsl::ostringstream errorDesc; mqbu::StorageKey appKey(upstreamSubQueueId); @@ -679,6 +608,11 @@ void RemoteQueue::getHandle( mqbi::QueueEngine::k_DEFAULT_APP_KEY); } } + + d_queueEngine_mp->getHandle(clientContext, + handleParameters, + upstreamSubQueueId, + callback); } void RemoteQueue::configureHandle( @@ -1011,7 +945,7 @@ void RemoteQueue::postMessage(const bmqp::PutHeader& putHeaderIn, if (!d_pendingMessagesTimerEventHandle) { bsls::TimeInterval time; time.setTotalNanoseconds(now + d_pendingPutsTimeoutNs); - d_scheduler_p->scheduleEvent( + scheduler()->scheduleEvent( &d_pendingMessagesTimerEventHandle, time, bdlf::BindUtil::bind(&RemoteQueue::expirePendingMessages, @@ -1395,7 +1329,7 @@ void RemoteQueue::expirePendingMessagesDispatched() bsls::TimeInterval time; time.setTotalNanoseconds(nextTime); - d_scheduler_p->scheduleEvent( + scheduler()->scheduleEvent( &d_pendingMessagesTimerEventHandle, time, bdlf::BindUtil::bind(&RemoteQueue::expirePendingMessages, this)); @@ -1525,7 +1459,7 @@ void RemoteQueue::cleanPendingMessages(mqbi::QueueHandle* handle) } if (d_pendingMessages.size() == 0 && d_pendingMessagesTimerEventHandle) { - d_scheduler_p->cancelEventAndWait(&d_pendingMessagesTimerEventHandle); + scheduler()->cancelEventAndWait(&d_pendingMessagesTimerEventHandle); // 'expirePendingMessagesDispatched' does not restart timer if // 'd_pendingMessages' is empty } @@ -1622,7 +1556,7 @@ void RemoteQueue::onOpenFailure(unsigned int upstreamSubQueueId) BSLS_ASSERT_SAFE(d_pendingMessages.size() == 0); if (d_pendingMessagesTimerEventHandle) { - d_scheduler_p->cancelEventAndWait( + scheduler()->cancelEventAndWait( &d_pendingMessagesTimerEventHandle); // 'expirePendingMessagesDispatched' does not restart timer if // 'd_pendingMessages' is empty diff --git a/src/groups/mqb/mqbblp/mqbblp_remotequeue.h b/src/groups/mqb/mqbblp/mqbblp_remotequeue.h index 7d0360c41..2b063cfe4 100644 --- a/src/groups/mqb/mqbblp/mqbblp_remotequeue.h +++ b/src/groups/mqb/mqbblp/mqbblp_remotequeue.h @@ -226,12 +226,6 @@ class RemoteQueue { // stop. Must be Less or equal to // the deduplication timeout. - bdlmt::EventScheduler* d_scheduler_p; - // Pointer, held not owned, to the - // scheduler to use for pending - // expiration and message - // throttling. - bdlmt::EventScheduler::EventHandle d_pendingMessagesTimerEventHandle; // Broadcast PUT can be retransmitted if it is guaranteed that the PUT did @@ -474,17 +468,15 @@ class RemoteQueue { /// THREAD: This method is called from the Queue's dispatcher thread. void onOpenFailure(unsigned int upstreamSubQueueId); + /// Return the event scheduler associated with this remote queue. + bdlmt::EventScheduler* scheduler(); + // ACCESSORS /// Load into the specified `out` object the internal details about this /// queue. void loadInternals(mqbcmd::RemoteQueue* out) const; - - /// Return the event scheduler associated with this remote queue. - bdlmt::EventScheduler* scheduler(); - - /// Set this remote queue's event scheduler. - void setEventScheduler(bdlmt::EventScheduler* scheduler); + const bmqt::MessageGUID& resumePoint() const; }; // ============================================================================ @@ -576,12 +568,7 @@ RemoteQueue::subStreamContext(unsigned int upstreamSubQueueId) inline bdlmt::EventScheduler* RemoteQueue::scheduler() { - return d_scheduler_p; -} - -inline void RemoteQueue::setEventScheduler(bdlmt::EventScheduler* scheduler) -{ - d_scheduler_p = scheduler; + return d_state_p->scheduler(); } } // close package namespace diff --git a/src/groups/mqb/mqbblp/mqbblp_remotequeue.t.cpp b/src/groups/mqb/mqbblp/mqbblp_remotequeue.t.cpp index a2fb9ee78..6e6e9b475 100644 --- a/src/groups/mqb/mqbblp/mqbblp_remotequeue.t.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_remotequeue.t.cpp @@ -96,7 +96,6 @@ void verifyBroadfcastPut( ++(*count); } - } // ============================================================================ @@ -140,8 +139,6 @@ class TestBench { bmqp_ctrlmsg::QueueHandleParameters d_params; bmqt::AckResult::Enum d_status; bsl::queue d_puts; - bdlmt::EventScheduler d_scheduler; - bdlmt::EventSchedulerTestTimeSource d_timeSource; TestClock d_testClock; StateSpPool d_stateSpPool; bslma::Allocator* d_allocator_p; @@ -175,9 +172,7 @@ TestBench::TestBench(bslma::Allocator* allocator_p) , d_params(allocator_p) , d_status(bmqt::AckResult::e_UNKNOWN) , d_puts(allocator_p) -, d_scheduler(bsls::SystemClockType::e_MONOTONIC, allocator_p) -, d_timeSource(&d_scheduler) -, d_testClock(d_timeSource) +, d_testClock(d_cluster._timeSource()) , d_stateSpPool(8192, allocator_p) , d_allocator_p(allocator_p) { @@ -195,14 +190,16 @@ TestBench::TestBench(bslma::Allocator* allocator_p) bdlf::BindUtil::bind(&TestClock::monotonicClock, &d_testClock), bdlf::BindUtil::bind(&TestClock::highResTimer, &d_testClock)); - d_scheduler.start(); + mwcu::MemOutStream errorDescription(allocator_p); + + d_cluster.start(errorDescription); } TestBench::~TestBench() { bmqt::UriParser::shutdown(); - d_scheduler.stop(); + d_cluster.stop(); d_event.reset(); @@ -272,7 +269,7 @@ void TestBench::dropPuts() void TestBench::advanceTime(const bsls::TimeInterval& step) { - d_timeSource.advanceTime(step); + d_cluster.advanceTime(step.totalSeconds()); } TestBench::TestRemoteQueue::TestRemoteQueue( @@ -289,6 +286,7 @@ TestBench::TestRemoteQueue::TestRemoteQueue( d_storageKey, 1, // partition &theBench.d_domain, + theBench.d_cluster._resources(), theBench.d_allocator_p) , d_remoteQueue(&d_queueState, timeout, @@ -297,8 +295,6 @@ TestBench::TestRemoteQueue::TestRemoteQueue( theBench.d_allocator_p) { d_queueState.setRoutingConfig(routingConfig); - d_queueState.setEventScheduler(&theBench.d_scheduler); - d_remoteQueue.setEventScheduler(&theBench.d_scheduler); d_queue_sp->_setDispatcherEventHandler( bdlf::BindUtil::bind(&mqbblp::RemoteQueue::onDispatcherEvent, &d_remoteQueue, @@ -1033,16 +1029,10 @@ static void test4_buffering() ASSERT_EQ(0U, x.count()); ASSERT_EQ(0U, y.count()); - theBench.d_timeSource.advanceTime(bsls::TimeInterval(timeout + 1, 0)); + theBench.advanceTime(bsls::TimeInterval(timeout + 1, 0)); bslmt::Semaphore sem; - typedef void (bslmt::Semaphore::*PostFn)(); - - theBench.d_scheduler.scheduleEvent( - theBench.d_timeSource.now(), - bdlf::BindUtil::bind(static_cast(&bslmt::Semaphore::post), - &sem)); - sem.wait(); + theBench.d_cluster.waitForScheduler(); } static void test5_reopen_failure() diff --git a/src/groups/mqb/mqbblp/mqbblp_rootqueueengine.cpp b/src/groups/mqb/mqbblp/mqbblp_rootqueueengine.cpp index c7fc0146a..80cb2fce9 100644 --- a/src/groups/mqb/mqbblp/mqbblp_rootqueueengine.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_rootqueueengine.cpp @@ -29,7 +29,6 @@ #include #include #include -#include // BMQ #include @@ -82,9 +81,7 @@ const bsls::Types::Int64 k_NS_PER_MESSAGE = // --------------------- // PRIVATE MANIPULATORS -size_t RootQueueEngine::deliverMessages(AppState* app, - const bsl::string& appId, - const mqbu::StorageKey& key) +void RootQueueEngine::deliverMessages(AppState* app) { // executed by the *QUEUE DISPATCHER* thread // PRECONDITIONS @@ -93,49 +90,73 @@ size_t RootQueueEngine::deliverMessages(AppState* app, BSLS_ASSERT_SAFE(d_queueState_p->storage()); + if (!app->isAuthorized()) { + return; // RETURN + } + + // Position to the resumePoint + bslma::ManagedPtr storageIter_mp; + mqbi::StorageIterator* start = 0; + + if (!app->resumePoint().isUnset()) { + if (d_queueState_p->storage()->getIterator(&storageIter_mp, + app->appKey(), + app->resumePoint()) != + mqbi::StorageResult::e_SUCCESS) { + // The message is gone because of either GC or purge. + // In either case, start at the beginning. + // This code relies on TTL per Queue (Domain), not per message - if + // 'resumePoint()' has exceeded the TTL, everything before that had + // as well. + storageIter_mp = d_queueState_p->storage()->getIterator( + app->appKey()); + } + start = storageIter_mp.get(); + // 'start' points at either the resume point (if found) or the first + // unconfirmed message of the 'app' (if not found). + } + else { + start = d_storageIter_mp.get(); + // 'start' points at the next message in the logical stream (common + // for all apps). + } + bsls::TimeInterval delay; - size_t numMessages = - app->deliverMessages(&delay, key, *d_queueState_p->storage(), appId); + const size_t numMessages = app->deliverMessages(&delay, + d_realStorageIter_mp.get(), + start, + d_storageIter_mp.get()); if (delay != bsls::TimeInterval()) { app->scheduleThrottle( mwcsys::Time::nowMonotonicClock() + delay, bdlf::BindUtil::bind(&RootQueueEngine::deliverMessages, this, - app, - appId, - key)); + app)); } if (BSLS_PERFORMANCEHINT_PREDICT_LIKELY(numMessages > 0)) { - d_consumptionMonitor.onMessageSent(key); + d_consumptionMonitor.onMessageSent(app->appKey()); } - return numMessages; + if (app->isReadyForDelivery()) { + // If the 'app' has caught up with the queue data stream, need to + // continue the delivery from the queue position in the stream. + // Cannot rely on 'LocalQueue' calling 'afterNewMessage' since it turns + // off 'd_hasNewMessages'. Just call it explicitly. + const bmqt::MessageGUID dummy; + afterNewMessage(dummy, 0); + } } RootQueueEngine::Apps::iterator RootQueueEngine::makeSubStream(const bsl::string& appId, const AppKeyCount& appKey, - bool isAuthorized, - bool hasStorage, unsigned int upstreamSubQueueId) { - bslma::ManagedPtr storageIterMp; - - if (hasStorage) { - storageIterMp = d_queueState_p->storage()->getIterator(appKey.first); - } - else { - storageIterMp = bslma::ManagedPtr( - new (*d_allocator_p) mqbs::VoidStorageIterator(), - d_allocator_p); - } AppStateSp app(new (*d_allocator_p) - AppState(storageIterMp, - d_queueState_p->queue(), + AppState(d_queueState_p->queue(), d_scheduler_p, - isAuthorized, d_queueState_p->routingContext(), upstreamSubQueueId, appId, @@ -143,12 +164,6 @@ RootQueueEngine::makeSubStream(const bsl::string& appId, d_allocator_p), d_allocator_p); - if (hasStorage) { - d_consumptionMonitor.registerSubStream( - appKey.first, - bdlf::BindUtil::bind(&AppState::head, app.get())); - } - bsl::pair rc = d_apps.insert(appId, appKey, app); @@ -397,7 +412,10 @@ int RootQueueEngine::initializeAppId(const bsl::string& appId, } mqbu::StorageKey appKey; - if (!d_queueState_p->storage()->hasVirtualStorage(appId, &appKey)) { + unsigned int ordinal = 0; + if (!d_queueState_p->storage()->hasVirtualStorage(appId, + &appKey, + &ordinal)) { BALL_LOG_ERROR << "#QUEUE_STORAGE_NOTFOUND " << "Virtual storage does not exist for AppId '" << appId << "', queue: '" @@ -412,15 +430,19 @@ int RootQueueEngine::initializeAppId(const bsl::string& appId, } BSLS_ASSERT_SAFE(!appKey.isNull()); - makeSubStream(appId, - AppKeyCount(appKey, 0), - true, - true, - upstreamSubQueueId); + Apps::iterator iter = makeSubStream(appId, + AppKeyCount(appKey, 0), + upstreamSubQueueId); + + iter->value()->authorize(appKey, ordinal); + + d_consumptionMonitor.registerSubStream( + appKey, + bdlf::BindUtil::bind(&RootQueueEngine::head, this, iter->value())); BALL_LOG_INFO << "Found virtual storage for appId [" << appId << "], queue [" << d_queueState_p->uri() << "], appKey [" - << appKey << "]"; + << appKey << "], ordinal [" << ordinal << "]"; return 0; } @@ -429,13 +451,22 @@ void RootQueueEngine::resetState(bool isShuttingDown) { for (Apps::iterator it = d_apps.begin(); it != d_apps.end(); ++it) { it->value()->undoRouting(); - it->value()->d_routing_sp->reset(); + it->value()->routing()->reset(); } d_consumptionMonitor.reset(); if (!isShuttingDown) { d_apps.clear(); + d_storageIter_mp = d_queueState_p->storage()->getIterator( + mqbu::StorageKey::k_NULL_KEY); + d_realStorageIter_mp = d_queueState_p->storage()->getIterator( + mqbu::StorageKey::k_NULL_KEY); + + if (!d_storageIter_mp->atEnd()) { + BALL_LOG_INFO << "Queue [" << d_queueState_p->uri() + << "] starting at " << d_storageIter_mp->guid(); + } } } @@ -449,15 +480,15 @@ void RootQueueEngine::rebuildSelectedApp( const AppStateSp& app = itApp->value(); - BSLS_ASSERT_SAFE(app->d_routing_sp); + BSLS_ASSERT_SAFE(app->routing()); mwcu::MemOutStream errorStream(d_allocator_p); - app->d_routing_sp->loadApp(itApp->key1().c_str(), - handle, - &errorStream, - info, - previous); + app->routing()->loadApp(itApp->key1().c_str(), + handle, + &errorStream, + info, + previous); if (errorStream.length() > 0) { BALL_LOG_WARN << "#BMQ_SUBSCRIPTION_FAILURE for queue '" @@ -500,7 +531,7 @@ int RootQueueEngine::rebuildInternalState(bsl::ostream& errorDescription) if (!previous) { continue; // CONTINUE } - Apps::iterator itApp = d_apps.findByKey1(previous->d_appId); + Apps::iterator itApp = d_apps.findByKey1(previous->appId()); unsigned int upstreamSubQueueId = previous->upstreamSubQueueId(); if (itApp == d_apps.end()) { @@ -521,30 +552,32 @@ int RootQueueEngine::rebuildInternalState(bsl::ostream& errorDescription) else { key2 = AppKeyCount( d_queueState_p->appKeyGenerator()->generateAppKey( - previous->d_appId, + previous->appId(), d_queueState_p->partitionId()), 0); } - itApp = makeSubStream(previous->d_appId, + itApp = makeSubStream(previous->appId(), key2, - false, - false, bmqp::QueueId::k_UNASSIGNED_SUBQUEUE_ID); } AppStateSp& app = itApp->value(); - app->d_routing_sp = previous->d_routing_sp; - app->d_priorityCount = previous->d_priorityCount; + app->routing() = previous->routing(); app->setUpstreamSubQueueId(upstreamSubQueueId); + // Do not copy resumePoint. New RootQueueEngine redelivers everything + // unconfirmed; its iterator is at the beginning. + // This relates to how RelayQueueEngine checks for duplicates; it could + // limit the check to OOO PUSH messages only except the case when new + // Primary redelivers everything. d_queueState_p->abandon(upstreamSubQueueId); d_queueState_p->adopt(app); } BALL_LOG_INFO << "Rebuilt internal state of queue engine for queue [" << d_queueState_p->queue()->description() << "] having " - << d_apps.size() << " consumers " + << d_apps.size() << " apps / substreams " << "[handleParameters: " << d_queueState_p->handleParameters() << ", streamParameters: " @@ -754,8 +787,6 @@ mqbi::QueueHandle* RootQueueEngine::getHandle( } iter = makeSubStream(appId, key2, - false, - false, bmqp::QueueId::k_UNASSIGNED_SUBQUEUE_ID); } BSLS_ASSERT_SAFE(iter != d_apps.end()); @@ -766,6 +797,16 @@ mqbi::QueueHandle* RootQueueEngine::getHandle( d_queueState_p->adopt(iter->value()); upstreamSubQueueId = iter->value()->upstreamSubQueueId(); + + if (!iter->value()->isAuthorized()) { + if (iter->value()->authorize()) { + d_consumptionMonitor.registerSubStream( + iter->value()->appKey(), + bdlf::BindUtil::bind(&RootQueueEngine::head, + this, + iter->value())); + } + } } else { upstreamSubQueueId = bmqp::QueueId::k_DEFAULT_SUBQUEUE_ID; @@ -892,6 +933,9 @@ void RootQueueEngine::configureHandle( BSLS_ASSERT_SAFE(iter != d_apps.end()); const AppStateSp& affectedApp = iter->value(); + + BSLS_ASSERT_SAFE(affectedApp->appKey() == iter->key2().first); + // prepare the App for rebuilding consumers affectedApp->undoRouting(); @@ -900,12 +944,13 @@ void RootQueueEngine::configureHandle( BSLS_ASSERT_SAFE( bmqt::QueueFlagsUtil::isReader(handle->handleParameters().flags())); - bsl::shared_ptr previous = affectedApp->d_routing_sp; + const bsl::shared_ptr previous = + affectedApp->routing(); - affectedApp->d_routing_sp.reset(new (*d_allocator_p) Routers::AppContext( - d_queueState_p->routingContext(), - d_allocator_p), - d_allocator_p); + affectedApp->routing().reset(new (*d_allocator_p) Routers::AppContext( + d_queueState_p->routingContext(), + d_allocator_p), + d_allocator_p); d_queueState_p->handleCatalog().iterateConsumers( bdlf::BindUtil::bind(&RootQueueEngine::rebuildSelectedApp, @@ -915,15 +960,15 @@ void RootQueueEngine::configureHandle( iter, previous.get())); - affectedApp->d_priorityCount = affectedApp->d_routing_sp->finalize(); - affectedApp->d_routing_sp->apply(); - affectedApp->d_routing_sp->registerSubscriptions(); + affectedApp->routing()->finalize(); + affectedApp->routing()->apply(); + affectedApp->routing()->registerSubscriptions(); BALL_LOG_INFO << "Rebuilt active consumers of the highest " << "priority for queue '" << d_queueState_p->queue()->description() << "', appId = '" << iter->key1() << "'. Now there are " - << affectedApp->d_priorityCount << " consumers."; + << affectedApp->routing()->priorityCount() << " consumers."; // Inform the requester of the success before attempting to deliver new // messages. @@ -933,7 +978,7 @@ void RootQueueEngine::configureHandle( streamParameters); // Now triggering message delivery for affected apps - deliverMessages(affectedApp.get(), iter->key1(), iter->key2().first); + deliverMessages(affectedApp.get()); } void RootQueueEngine::releaseHandle( @@ -1154,9 +1199,7 @@ void RootQueueEngine::releaseHandle( if (app->transferUnconfirmedMessages(handle, currSubStreamInfo)) { // There are potential consumers to redeliver to - deliverMessages(app, - currSubStreamInfo.appId(), - itApp->key2().first); + deliverMessages(app); } if (result.isQueueStreamEmpty()) { @@ -1166,7 +1209,7 @@ void RootQueueEngine::releaseHandle( // state. On the surface it results in alarm being // (re)generated if the unauthorized app is used again // after all previous clients are gone. - if (!itApp->value()->d_isAuthorized) { + if (!itApp->value()->isAuthorized()) { BALL_LOG_INFO << "There are no more clients for the unauthorized" << " appId [" << itApp->key1() @@ -1234,7 +1277,7 @@ void RootQueueEngine::onHandleUsable(mqbi::QueueHandle* handle, const AppStateSp app = subQueue(upstreamSubQueueId); BSLS_ASSERT_SAFE(app); - deliverMessages(app.get(), app->d_appId, app->d_appKey); + deliverMessages(app.get()); } } @@ -1249,20 +1292,35 @@ void RootQueueEngine::afterNewMessage( d_queueState_p->queue())); // Deliver new messages to active (alive and capable to deliver) consumers - do { - d_appsDeliveryContext.reset(); + d_appsDeliveryContext.start(); + while (d_appsDeliveryContext.reset(d_storageIter_mp.get())) { + // Assume, all Apps need to deliver (some may be at capacity) for (Apps::iterator iter = d_apps.begin(); iter != d_apps.end(); ++iter) { AppStateSp& app = iter->value(); - if (app->redeliveryListSize() == 0) { - if (d_appsDeliveryContext.processApp(*app)) { - d_consumptionMonitor.onMessageSent(iter->key2().first); - } + + if (d_appsDeliveryContext.processApp(*app, app->ordinal())) { + // Consider this message as sent out + + d_consumptionMonitor.onMessageSent(iter->key2().first); + + // Report queue time metric per App + // Report 'queue time' metric for all active appIds + d_queueState_p->queue()->stats()->onEvent( + mqbstat::QueueStatsDomain::EventType::e_QUEUE_TIME, + d_appsDeliveryContext.timeDelta(), + app->appId()); } } + if (!d_appsDeliveryContext.isEmpty()) { + // Report 'queue time' metric for the entire queue + d_queueState_p->queue()->stats()->onEvent( + mqbstat::QueueStatsDomain::EventType::e_QUEUE_TIME, + d_appsDeliveryContext.timeDelta()); + } d_appsDeliveryContext.deliverMessage(); - } while (d_appsDeliveryContext.d_doRepeat); + } if (QueueEngineUtil::isBroadcastMode(d_queueState_p->queue())) { // Clear storage status @@ -1273,6 +1331,8 @@ void RootQueueEngine::afterNewMessage( // used while calling 'removeAll()', it should always succeed. BSLS_ASSERT_SAFE(mqbi::StorageResult::e_SUCCESS == rc); (void)rc; // Compiler happiness + + d_storageIter_mp->reset(); } } @@ -1303,15 +1363,7 @@ int RootQueueEngine::onConfirmMessage(mqbi::QueueHandle* handle, // TODO: handle missing SubQueue? QueueEngineUtil_AppState& app = *subQueue(subQueueId); - // Inform app that a message from its virtual storage is getting removed, - // so that it can advance its iterator etc if required. - app.beforeMessageRemoved(msgGUID, false); - - const mqbu::StorageKey& appKey = app.d_appKey; - BSLS_ASSERT_SAFE(!appKey.isNull()); - - if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( - !d_queueState_p->storage()->hasVirtualStorage(appKey))) { + if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY(!app.isAuthorized())) { // If an appId was dynamically unregistered, it is possible that the // client may still attempt at confirming outstanding messages, which // we need to guard against. @@ -1319,8 +1371,11 @@ int RootQueueEngine::onConfirmMessage(mqbi::QueueHandle* handle, return rc_ERROR; // RETURN } + const mqbu::StorageKey& appKey = app.appKey(); + BSLS_ASSERT_SAFE(!appKey.isNull()); + // Release from storage - mqbi::StorageResult::Enum rc = d_queueState_p->storage()->releaseRef( + mqbi::StorageResult::Enum rc = d_queueState_p->storage()->confirm( msgGUID, appKey, bdlt::EpochUtil::convertToTimeT64(bdlt::CurrentTime::utc())); @@ -1332,11 +1387,12 @@ int RootQueueEngine::onConfirmMessage(mqbi::QueueHandle* handle, } if (rc == mqbi::StorageResult::e_ZERO_REFERENCES) { + beforeMessageRemoved(msgGUID); return rc_NO_MORE_REFERENCES; // RETURN } BALL_LOG_INFO << "'" << d_queueState_p->queue()->description() - << "', appId = '" << app.d_appId + << "', appId = '" << app.appId() << "' failed to release references upon CONFIRM " << msgGUID << "' [reason: " << mqbi::StorageResult::toAscii(rc) << "]"; @@ -1365,7 +1421,7 @@ int RootQueueEngine::onRejectMessage(mqbi::QueueHandle* handle, // TODO: handle missing SubQueue? QueueEngineUtil_AppState& app = *subQueue(subQueueId); - const mqbu::StorageKey& appKey = app.d_appKey; + const mqbu::StorageKey& appKey = app.appKey(); BSLS_ASSERT_SAFE(!appKey.isNull()); @@ -1414,46 +1470,44 @@ int RootQueueEngine::onRejectMessage(mqbi::QueueHandle* handle, const int maxDeliveryAttempts = d_queueState_p->domain()->config().maxDeliveryAttempts(); const bool domainIsUnlimited = (maxDeliveryAttempts == 0); - if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( - domainIsUnlimited != message->rdaInfo().isUnlimited())) { + bmqp::RdaInfo& rda = message->appMessageState(app.ordinal()).d_rdaInfo; + + if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY(domainIsUnlimited != + rda.isUnlimited())) { BSLS_PERFORMANCEHINT_UNLIKELY_HINT; BALL_LOGTHROTTLE_WARN(k_MAX_INSTANT_MESSAGES, k_NS_PER_MESSAGE) - << "[THROTTLED] Mismatch between the message's RdaInfo " - << message->rdaInfo() << " and the domain's " - << "'maxDeliveryAttempts' setting [" << maxDeliveryAttempts - << "], updating message's RdaInfo"; + << "[THROTTLED] Mismatch between the message's RdaInfo " << rda + << " and the domain's 'maxDeliveryAttempts' setting [" + << maxDeliveryAttempts << "], updating message's RdaInfo"; if (maxDeliveryAttempts > 0) { - message->rdaInfo().setCounter(maxDeliveryAttempts); + rda.setCounter(maxDeliveryAttempts); } else { - message->rdaInfo().setUnlimited(); + rda.setUnlimited(); } } - counter = message->rdaInfo().counter(); + counter = rda.counter(); if (d_throttledRejectedMessages.requestPermission()) { BALL_LOG_INFO << "[THROTTLED] Queue '" << d_queueState_p->uri() << "' rejecting PUSH [GUID: '" << msgGUID - << "', appId: " << app.d_appId + << "', appId: " << app.appId() << ", subQueueId: " << app.upstreamSubQueueId() - << "] with the counter: [" << message->rdaInfo() - << "]"; + << "] with the counter: [" << rda << "]"; } - if (!message->rdaInfo().isUnlimited()) { + if (!rda.isUnlimited()) { BSLS_ASSERT_SAFE(counter); - message->rdaInfo().setPotentiallyPoisonous(true); - message->rdaInfo().setCounter(--counter); + rda.setPotentiallyPoisonous(true); + rda.setCounter(--counter); if (counter == 0) { - // Purge the message from corresponding app. - app.beforeMessageRemoved(msgGUID, false); - storageRc = d_queueState_p->storage()->releaseRef(msgGUID, - appKey, - 0ULL, - true); + storageRc = d_queueState_p->storage()->confirm(msgGUID, + appKey, + 0ULL, + true); // Log the rejected message and raise an alarm, in a throttled // manner. @@ -1471,7 +1525,7 @@ int RootQueueEngine::onRejectMessage(mqbi::QueueHandle* handle, d_miscWorkThreadPool_p->enqueueJob(bdlf::BindUtil::bind( &QueueEngineUtil::logRejectMessage, msgGUID, - app.d_appId, + app.appId(), app.upstreamSubQueueId(), appData, attributes, @@ -1491,7 +1545,7 @@ int RootQueueEngine::onRejectMessage(mqbi::QueueHandle* handle, // holding msgGUID and no need to call // `beforeMessageRemoved`. beforeMessageRemoved(msgGUID); - d_queueState_p->storage()->remove(msgGUID, 0, true); + d_queueState_p->storage()->remove(msgGUID, 0); } } } @@ -1513,8 +1567,10 @@ void RootQueueEngine::beforeMessageRemoved(const bmqt::MessageGUID& msgGUID) // PRECONDITIONS BSLS_ASSERT_SAFE(d_queueState_p->queue()->dispatcher()->inDispatcherThread( d_queueState_p->queue())); - for (Apps::iterator iter = d_apps.begin(); iter != d_apps.end(); ++iter) { - iter->value()->beforeMessageRemoved(msgGUID, true); + BSLS_ASSERT_SAFE(d_storageIter_mp); + + if (!d_storageIter_mp->atEnd() && (d_storageIter_mp->guid() == msgGUID)) { + d_storageIter_mp->advance(); } } @@ -1535,12 +1591,7 @@ void RootQueueEngine::afterQueuePurged(const bsl::string& appId, // add an additional assert that the appId must be empty. BSLS_ASSERT_SAFE(appId == bmqp::ProtocolUtil::k_NULL_APP_ID); - for (Apps::iterator iter = d_apps.begin(Apps::e_SECOND_KEY); - iter != d_apps.end(); - ++iter) { - BSLS_ASSERT_SAFE(iter->value()->d_storageIter_mp); - iter->value()->d_storageIter_mp->reset(); - } + d_storageIter_mp->reset(); return; // RETURN } @@ -1548,7 +1599,7 @@ void RootQueueEngine::afterQueuePurged(const bsl::string& appId, Apps::iterator iter = d_apps.findByKey2(AppKeyCount(appKey, 0)); BSLS_ASSERT_SAFE(iter != d_apps.end()); BSLS_ASSERT_SAFE(iter->key1() == appId); - iter->value()->d_storageIter_mp->reset(); + iter->value()->clear(); } void RootQueueEngine::onTimer(bsls::Types::Int64 currentTimer) @@ -1611,8 +1662,6 @@ void RootQueueEngine::afterAppIdRegistered( iter = makeSubStream(appId, AppKeyCount(key, 0), - true, - false, bmqp::QueueId::k_UNASSIGNED_SUBQUEUE_ID); } else { @@ -1639,11 +1688,10 @@ void RootQueueEngine::afterAppIdRegistered( key = appKey; - AppStateSp consumerStateSp = iter->value(); + AppStateSp appStateSp = iter->value(); d_apps.erase(iter); - iter = d_apps.insert(appId, AppKeyCount(key, 0), consumerStateSp) - .first; + iter = d_apps.insert(appId, AppKeyCount(key, 0), appStateSp).first; } else { // TODO_CSL Remove this snippet when pre-CSL workflow has been @@ -1656,7 +1704,6 @@ void RootQueueEngine::afterAppIdRegistered( // the appId has been registered. This is critical to ensure that the // now-registered appId continues to be available across restarts of // its consumers. - iter->value()->d_isAuthorized = true; } BSLS_ASSERT_SAFE(!key.isNull()); @@ -1668,15 +1715,6 @@ void RootQueueEngine::afterAppIdRegistered( mqbi::Storage::AppIdKeyPairs(1, mqbi::Storage::AppIdKeyPair(appId, key)), mqbi::Storage::AppIdKeyPairs()); - - bslma::ManagedPtr storageIterMp; - storageIterMp = d_queueState_p->storage()->getIterator(iter->key2().first); - - d_consumptionMonitor.registerSubStream( - key, - bdlf::BindUtil::bind(&AppState::head, iter->value())); - - iter->value()->d_storageIter_mp = storageIterMp; } void RootQueueEngine::afterAppIdUnregistered( @@ -1705,10 +1743,8 @@ void RootQueueEngine::afterAppIdUnregistered( : iter->key2().first; BSLS_ASSERT_SAFE(iter->key2().first == appKey); - // we still keep the app but invalidate the iterator - iter->value()->d_storageIter_mp.load(new (*d_allocator_p) - mqbs::VoidStorageIterator(), - d_allocator_p); + // we still keep the app but invalidate the authorization + iter->value()->unauthorize(); // Do a best effort to confirm the messages and remove the storage. If // either fails, just log the condition. @@ -1739,6 +1775,52 @@ void RootQueueEngine::afterAppIdUnregistered( d_consumptionMonitor.unregisterSubStream(appKey); } +void RootQueueEngine::registerStorage(const bsl::string& appId, + const mqbu::StorageKey& appKey, + unsigned int appOrdinal) +{ + // executed by the *QUEUE DISPATCHER* thread + + // PRECONDITIONS + BSLS_ASSERT_SAFE(d_queueState_p->queue()->dispatcher()->inDispatcherThread( + d_queueState_p->queue())); + + BALL_LOG_INFO << "Local queue: " << d_queueState_p->uri() + << " (id: " << d_queueState_p->id() + << ") now has storage: [App Id: " << appId + << ", key: " << appKey << ", ordinal: " << appOrdinal << "]"; + + Apps::iterator iter = d_apps.findByKey1(appId); + BSLS_ASSERT_SAFE(iter != d_apps.end()); + BSLS_ASSERT_SAFE(iter->key2().first == appKey); + + iter->value()->authorize(appKey, appOrdinal); + + d_consumptionMonitor.registerSubStream( + appKey, + bdlf::BindUtil::bind(&RootQueueEngine::head, this, iter->value())); +} + +void RootQueueEngine::unregisterStorage(const bsl::string& appId, + const mqbu::StorageKey& appKey, + unsigned int appOrdinal) +{ + // executed by the *QUEUE DISPATCHER* thread + + // PRECONDITIONS + BSLS_ASSERT_SAFE(d_queueState_p->queue()->dispatcher()->inDispatcherThread( + d_queueState_p->queue())); + + Apps::iterator iter = d_apps.findByKey1(appId); + BSLS_ASSERT_SAFE(iter != d_apps.end()); + BSLS_ASSERT_SAFE(iter->key2().first == appKey); + + // we still keep the app but invalidate the authorization + iter->value()->unauthorize(); + + (void)appOrdinal; +} + mqbi::StorageResult::Enum RootQueueEngine::evaluateAutoSubscriptions( const bmqp::PutHeader& putHeader, const bsl::shared_ptr& appData, @@ -1780,6 +1862,25 @@ mqbi::StorageResult::Enum RootQueueEngine::evaluateAutoSubscriptions( return result; } +bslma::ManagedPtr +RootQueueEngine::head(const AppStateSp app) const +{ + bslma::ManagedPtr out; + + if (!app->putAsideList().empty()) { + d_queueState_p->storage()->getIterator(&out, + app->appKey(), + app->putAsideList().first()); + } + else if (!d_storageIter_mp->atEnd()) { + d_queueState_p->storage()->getIterator(&out, + app->appKey(), + d_storageIter_mp->guid()); + } + + return out; +} + // ACCESSORS // (virtual mqbi::QueueEngine) unsigned int RootQueueEngine::messageReferenceCount() const @@ -1832,8 +1933,10 @@ void RootQueueEngine::loadInternals(mqbcmd::QueueEngine* out) const consumerState.appId() = iter->key1(); if (d_queueState_p->storage()->hasVirtualStorage(iter->key1())) { - consumerState.isAtEndOfStorage().makeValue( - iter->value()->d_storageIter_mp->atEnd()); + const bool isAtEndOfStorage = iter->value()->isAtEndOfStorage() && + d_storageIter_mp->atEnd(); + + consumerState.isAtEndOfStorage().makeValue(isAtEndOfStorage); consumerState.status() = (!iter->value()->hasConsumers() ? mqbcmd::ConsumerStatus::REGISTERED : mqbcmd::ConsumerStatus::ALIVE); diff --git a/src/groups/mqb/mqbblp/mqbblp_rootqueueengine.h b/src/groups/mqb/mqbblp/mqbblp_rootqueueengine.h index 4fe859db8..d4c369110 100644 --- a/src/groups/mqb/mqbblp/mqbblp_rootqueueengine.h +++ b/src/groups/mqb/mqbblp/mqbblp_rootqueueengine.h @@ -146,6 +146,15 @@ class RootQueueEngine BSLS_KEYWORD_FINAL : public mqbi::QueueEngine { QueueEngineUtil_AppsDeliveryContext d_appsDeliveryContext; // Reusable apps delivery context + bslma::ManagedPtr d_storageIter_mp; + // Storage iterator to the logical stream of messages. + // Queue Engine iterates this one sequentially. + + bslma::ManagedPtr d_realStorageIter_mp; + // Storage iterator to access storage state. + // Queue Engine uses this one to access random message (as in the case of + // redelivery). + bslma::Allocator* d_allocator_p; // Allocator to use private: @@ -159,15 +168,12 @@ class RootQueueEngine BSLS_KEYWORD_FINAL : public mqbi::QueueEngine { // PRIVATE MANIPULATORS /// Attempt to deliver outstanding messages, if any, to the consumers - /// of the Fanout appId corresponding to the specified `app`. Return - /// total number of re-routed messages. If at least one message has - /// been delivered, update `d_consumptionMonitor` for the specified - /// `key`. + /// of the Fanout appId corresponding to the specified `app`. If at least + /// one message has been delivered, update `d_consumptionMonitor` for the + /// key of the 'app'. /// /// THREAD: This method is called from the Queue's dispatcher thread. - size_t deliverMessages(AppState* app, - const bsl::string& appId, - const mqbu::StorageKey& key); + void deliverMessages(AppState* app); // PRIVATE ACCESSORS @@ -195,8 +201,6 @@ class RootQueueEngine BSLS_KEYWORD_FINAL : public mqbi::QueueEngine { Apps::iterator makeSubStream(const bsl::string& appId, const AppKeyCount& appKey, - bool isAuthorized, - bool hasStorage, unsigned int upstreamSubQueueId); bool validate(unsigned int upstreamSubQueueId) const; @@ -401,6 +405,24 @@ class RootQueueEngine BSLS_KEYWORD_FINAL : public mqbi::QueueEngine { virtual void afterAppIdUnregistered( const mqbi::Storage::AppIdKeyPair& appIdKeyPair) BSLS_KEYWORD_OVERRIDE; + /// Called after creation of a new storage for the specified + /// `appIdKeyPair`. + /// + /// THREAD: This method is called from the Queue's dispatcher thread. + virtual void + registerStorage(const bsl::string& appId, + const mqbu::StorageKey& appKey, + unsigned int appOrdinal) BSLS_KEYWORD_OVERRIDE; + + /// Called after removal of the storage for the specified + /// `appIdKeyPair`. + /// + /// THREAD: This method is called from the Queue's dispatcher thread. + virtual void + unregisterStorage(const bsl::string& appId, + const mqbu::StorageKey& appKey, + unsigned int appOrdinal) BSLS_KEYWORD_OVERRIDE; + /// Given the specified 'putHeader', 'appData', 'mpi', and 'timestamp', /// evaluate all Auto (Application) subscriptions and exclude applications /// with negative results from message delivery. @@ -413,6 +435,10 @@ class RootQueueEngine BSLS_KEYWORD_FINAL : public mqbi::QueueEngine { const bmqp::MessagePropertiesInfo& mpi, bsls::Types::Uint64 timestamp) BSLS_KEYWORD_OVERRIDE; + /// Return storage iterator to the 1st un-delivered message including + /// 'put-aside' messages (those without matching Subscriptions). + bslma::ManagedPtr head(const AppStateSp app) const; + // ACCESSORS // (virtual mqbi::QueueEngine) diff --git a/src/groups/mqb/mqbblp/mqbblp_rootqueueengine.t.cpp b/src/groups/mqb/mqbblp/mqbblp_rootqueueengine.t.cpp index 20ef19f17..ffcf4cc8b 100644 --- a/src/groups/mqb/mqbblp/mqbblp_rootqueueengine.t.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_rootqueueengine.t.cpp @@ -5131,7 +5131,7 @@ static void test46_throttleRedeliveryNoMoreHandles() // 4) Advance the time by the throttling delay and verify the second C3 // received the second message. // Testing: -// mqbblp::QueueEngine the last handle disappering for a particular app +// mqbblp::QueueEngine the last handle disappearing for a particular app // should end the delay for the current message. // ------------------------------------------------------------------------ { diff --git a/src/groups/mqb/mqbblp/mqbblp_routers.cpp b/src/groups/mqb/mqbblp/mqbblp_routers.cpp index 21e3cc11f..079a2c8ef 100644 --- a/src/groups/mqb/mqbblp/mqbblp_routers.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_routers.cpp @@ -381,9 +381,9 @@ void Routers::AppContext::load( } } -size_t Routers::AppContext::finalize() +unsigned int Routers::AppContext::finalize() { - size_t count = 0; + d_priorityCount = 0; clean(); @@ -443,7 +443,7 @@ size_t Routers::AppContext::finalize() group.d_highestSubscriptions.emplace_back(&subscription); level.d_count += n; - count += n; + d_priorityCount += n; } else { // This 'subscription' has the 'expression' which is used @@ -460,7 +460,7 @@ size_t Routers::AppContext::finalize() } } - return count; + return d_priorityCount; } void Routers::AppContext::registerSubscriptions() @@ -602,18 +602,18 @@ void Routers::AppContext::reset() Routers::Result Routers::AppContext::selectConsumer( const Visitor& visitor, - const mqbi::StorageIterator* currentMessage) + const mqbi::StorageIterator* currentMessage, + unsigned int subscriptionId) { BSLS_ASSERT_SAFE(currentMessage); - unsigned int sId = currentMessage->subscriptionId(); - PriorityGroup* group = 0; d_queue.d_evaluationContext.setPropertiesReader(d_queue.d_preader.get()); ScopeExit scope(d_queue, currentMessage); - if (sId != bmqp::Protocol::k_DEFAULT_SUBSCRIPTION_ID) { - SubscriptionIds::SharedItem itId = d_queue.d_groupIds.find(sId); + if (subscriptionId != bmqp::Protocol::k_DEFAULT_SUBSCRIPTION_ID) { + SubscriptionIds::SharedItem itId = d_queue.d_groupIds.find( + subscriptionId); if (itId) { // Use already selected existing Group diff --git a/src/groups/mqb/mqbblp/mqbblp_routers.h b/src/groups/mqb/mqbblp/mqbblp_routers.h index 056f36193..c23f6f28b 100644 --- a/src/groups/mqb/mqbblp/mqbblp_routers.h +++ b/src/groups/mqb/mqbblp/mqbblp_routers.h @@ -622,6 +622,8 @@ class Routers { e_NO_CAPACITY_ALL = 3 // All Subscription(s) are without capacity , e_DELAY = 4 // Delay due to Potentially Poisonous data + , + e_INVALID = 5 // Not valid anymore due to Confirm/Purge }; /// Class that implements round-robin routing policy. @@ -684,6 +686,8 @@ class Routers { bmqeval::CompilationContext d_compilationContext; + unsigned int d_priorityCount; + bslma::Allocator* d_allocator_p; AppContext(QueueRoutingContext& queue, bslma::Allocator* allocator); @@ -710,7 +714,7 @@ class Routers { /// Make a pass on results of previous parsing and build round-robin /// lists of highest priority `Subscription`s. - size_t finalize(); + unsigned int finalize(); void registerSubscriptions(); @@ -736,7 +740,8 @@ class Routers { /// `true`. Routers::Result selectConsumer(const Visitor& visitor, - const mqbi::StorageIterator* currentMessage); + const mqbi::StorageIterator* currentMessage, + unsigned int subscriptionId); /// Iterate all highest priority `Subscriber`s and call the /// specified `visitor` for each highest priority `Subscription` @@ -765,6 +770,8 @@ class Routers { void generate(bmqp_ctrlmsg::StreamParameters* streamParameters) const; bool hasHandle(mqbi::QueueHandle* handle) const; + + unsigned int priorityCount() const; }; }; @@ -816,6 +823,7 @@ inline Routers::AppContext::AppContext(QueueRoutingContext& queue, , d_queue(queue) , d_router(d_priorities) , d_compilationContext(allocator) +, d_priorityCount(0) , d_allocator_p(allocator) { // NOTHING @@ -832,6 +840,11 @@ inline bool Routers::AppContext::hasHandle(mqbi::QueueHandle* handle) const return d_consumers.hasItem(handle); } +inline unsigned int Routers::AppContext::priorityCount() const +{ + return d_priorityCount; +} + // ----------------------------------- // struct Routers::QueueRoutingContext // ----------------------------------- diff --git a/src/groups/mqb/mqbblp/mqbblp_routers.t.cpp b/src/groups/mqb/mqbblp/mqbblp_routers.t.cpp index 1ee68c6fb..9fb123d04 100644 --- a/src/groups/mqb/mqbblp/mqbblp_routers.t.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_routers.t.cpp @@ -72,18 +72,16 @@ struct TestStorage { 1, d_domainCfg, &d_capacityMeter, - bmqp::RdaInfo(), allocator) , d_iterator(d_storage.getIterator(mqbu::StorageKey())) , d_bufferFactory(32, allocator) - , d_queue_sp(new (*allocator) mqbmock::Queue(0, allocator), allocator) + , d_queue_sp(new(*allocator) mqbmock::Queue(0, allocator), allocator) , d_allocator_p(allocator) { bmqt::MessageGUID guid; guid.fromHex("00000000000000000000000000000001"); mqbi::StorageMessageAttributes attributes; - const mqbi::Storage::StorageKeys storageKeys(allocator); const bsl::shared_ptr appData( new (*allocator) bdlbb::Blob(&d_bufferFactory, allocator), allocator); @@ -93,15 +91,12 @@ struct TestStorage { // TODO: put data for Expression evaluation mqbi::StorageResult::Enum rc = - d_storage.put(&attributes, guid, appData, options, storageKeys); + d_storage.put(&attributes, guid, appData, options); ASSERT_EQ(rc, mqbi::StorageResult::e_SUCCESS); } - ~TestStorage() - { - // d_storage.removeAll(mqbu::StorageKey()); - } + ~TestStorage() { d_storage.removeAll(mqbu::StorageKey()); } mqbmock::QueueHandle getHandle() { diff --git a/src/groups/mqb/mqbblp/package/mqbblp.mem b/src/groups/mqb/mqbblp/package/mqbblp.mem index ebcdb28fd..ca21ebcc8 100644 --- a/src/groups/mqb/mqbblp/package/mqbblp.mem +++ b/src/groups/mqb/mqbblp/package/mqbblp.mem @@ -9,6 +9,7 @@ mqbblp_domain mqbblp_localqueue mqbblp_messagegroupidhelper mqbblp_messagegroupidmanager +mqbblp_pushstream mqbblp_queue mqbblp_queueconsumptionmonitor mqbblp_queueenginetester diff --git a/src/groups/mqb/mqbc/mqbc_clusterdata.cpp b/src/groups/mqb/mqbc/mqbc_clusterdata.cpp index 590e929ae..b19e97a56 100644 --- a/src/groups/mqb/mqbc/mqbc_clusterdata.cpp +++ b/src/groups/mqb/mqbc/mqbc_clusterdata.cpp @@ -95,9 +95,7 @@ mqbc::ClusterDataIdentity clusterIdentity(const bslstl::StringRef& name, // CREATORS ClusterData::ClusterData( const bslstl::StringRef& name, - bdlmt::EventScheduler* scheduler, - bdlbb::BlobBufferFactory* bufferFactory, - BlobSpPool* blobSpPool, + const mqbi::ClusterResources& resources, const mqbcfg::ClusterDefinition& clusterConfig, const mqbcfg::ClusterProxyDefinition& clusterProxyConfig, bslma::ManagedPtr netCluster, @@ -108,9 +106,7 @@ ClusterData::ClusterData( const StatContextsMap& statContexts, bslma::Allocator* allocator) : d_allocator_p(allocator) -, d_scheduler_p(scheduler) -, d_bufferFactory_p(bufferFactory) -, d_blobSpPool_p(blobSpPool) +, d_resources(resources) , d_dispatcherClientData() , d_clusterConfig(clusterConfig) , d_clusterProxyConfig(clusterProxyConfig) @@ -122,10 +118,13 @@ ClusterData::ClusterData( cluster->isRemote(), allocator)) , d_cluster_p(cluster) -, d_messageTransmitter(bufferFactory, cluster, transportManager, allocator) +, d_messageTransmitter(resources.bufferFactory(), + cluster, + transportManager, + allocator) , d_requestManager(bmqp::EventType::e_CONTROL, - bufferFactory, - scheduler, + resources.bufferFactory(), + resources.scheduler(), false, // lateResponseMode allocator) , d_multiRequestManager(&d_requestManager, allocator) @@ -146,12 +145,10 @@ ClusterData::ClusterData( { // PRECONDITIONS BSLS_ASSERT_SAFE(d_allocator_p); - BSLS_ASSERT_SAFE(d_scheduler_p); - BSLS_ASSERT_SAFE(d_bufferFactory_p); - BSLS_ASSERT_SAFE(d_blobSpPool_p); BSLS_ASSERT_SAFE(d_cluster_p); BSLS_ASSERT_SAFE(d_transportManager_p); - BSLS_ASSERT(scheduler->clockType() == bsls::SystemClockType::e_MONOTONIC); + BSLS_ASSERT(resources.scheduler()->clockType() == + bsls::SystemClockType::e_MONOTONIC); // Initialize the clusterStats object - under the hood this creates a new // subcontext to be held by this object to be used by all lower level diff --git a/src/groups/mqb/mqbc/mqbc_clusterdata.h b/src/groups/mqb/mqbc/mqbc_clusterdata.h index 8fad5a13c..8b7e2565c 100644 --- a/src/groups/mqb/mqbc/mqbc_clusterdata.h +++ b/src/groups/mqb/mqbc/mqbc_clusterdata.h @@ -153,14 +153,7 @@ class ClusterData { bslma::Allocator* d_allocator_p; // Allocator to use - bdlmt::EventScheduler* d_scheduler_p; - // EventScheduler to use - - bdlbb::BlobBufferFactory* d_bufferFactory_p; - // Blob buffer factory to use - - BlobSpPool* d_blobSpPool_p; // from mqbblp::Cluster - // Pool of shared pointers to blob to use + const mqbi::ClusterResources d_resources; mqbi::DispatcherClientData d_dispatcherClientData; // Dispatcher client data associated to this @@ -224,14 +217,12 @@ class ClusterData { // CREATORS - /// Create a `mqbc::ClusterData` with the specified `name`, `scheduler`, - /// `bufferFactory`, `blobSpPool`, `clusterConfig`, 'clusterProxyConfig', - /// `netCluster`, `cluster`, `clustersStatContext` and `statContexts` - /// values. Use the specified `allocator` for any memory allocation. + /// Create a `mqbc::ClusterData` with the specified `name`, `resources`, + /// `clusterConfig`, `clusterProxyConfig`, `netCluster`, `cluster`, + /// `clustersStatContext` and `statContexts` values. Use the specified + /// `allocator` for any memory allocation. ClusterData(const bslstl::StringRef& name, - bdlmt::EventScheduler* scheduler, - bdlbb::BlobBufferFactory* bufferFactory, - BlobSpPool* blobSpPool, + const mqbi::ClusterResources& resources, const mqbcfg::ClusterDefinition& clusterConfig, const mqbcfg::ClusterProxyDefinition& clusterProxyConfig, bslma::ManagedPtr netCluster, @@ -301,6 +292,7 @@ class ClusterData { // ACCESSORS /// Return the value of the corresponding member of this object. + const mqbi::ClusterResources& resources() const; const mqbi::DispatcherClientData& dispatcherClientData() const; const mqbcfg::ClusterDefinition& clusterConfig() const; const mqbcfg::ClusterProxyDefinition& clusterProxyConfig() const; @@ -355,17 +347,17 @@ ClusterDataIdentity::identity() const // MANIPULATORS inline bdlmt::EventScheduler& ClusterData::scheduler() { - return *d_scheduler_p; + return *d_resources.scheduler(); } inline bdlbb::BlobBufferFactory& ClusterData::bufferFactory() { - return *d_bufferFactory_p; + return *d_resources.bufferFactory(); } inline ClusterData::BlobSpPool& ClusterData::blobSpPool() { - return *d_blobSpPool_p; + return *d_resources.blobSpPool(); } inline mqbi::DispatcherClientData& ClusterData::dispatcherClientData() @@ -443,6 +435,12 @@ inline bdlmt::FixedThreadPool& ClusterData::miscWorkThreadPool() return d_miscWorkThreadPool; } +// ACCESSORS +inline const mqbi::ClusterResources& ClusterData::resources() const +{ + return d_resources; +} + // ACCESSORS inline const mqbi::DispatcherClientData& ClusterData::dispatcherClientData() const diff --git a/src/groups/mqb/mqbi/mqbi_cluster.h b/src/groups/mqb/mqbi/mqbi_cluster.h index 0ab4d05c9..a211c36e7 100644 --- a/src/groups/mqb/mqbi/mqbi_cluster.h +++ b/src/groups/mqb/mqbi/mqbi_cluster.h @@ -56,6 +56,7 @@ // BDE #include #include +#include #include #include #include @@ -460,6 +461,63 @@ class Cluster : public DispatcherClient { int partitionId) const = 0; }; +struct ClusterResources { + // Resources to use for all queues in all clusters + public: + // TYPES + + /// Pool of shared pointers to Blobs + typedef bdlcc::SharedObjectPool< + bdlbb::Blob, + bdlcc::ObjectPoolFunctors::DefaultCreator, + bdlcc::ObjectPoolFunctors::RemoveAll > + BlobSpPool; + + private: + // PRIVATE DATA + + /// EventScheduler to use + bdlmt::EventScheduler* d_scheduler_p; + + /// Blob buffer factory to use + bdlbb::BlobBufferFactory* d_bufferFactory_p; + + /// Pool of shared pointers to blob to + /// use. + BlobSpPool* d_blobSpPool_p; + + /// Pool of PushStream elements for Proxy/Replica QueueEngine. + bsl::optional d_pushElementsPool; + + public: + // CREATORS + + explicit ClusterResources(bdlmt::EventScheduler* scheduler, + bdlbb::BlobBufferFactory* bufferFactory, + BlobSpPool* blobSpPool); + + explicit ClusterResources(bdlmt::EventScheduler* scheduler, + bdlbb::BlobBufferFactory* bufferFactory, + BlobSpPool* blobSpPool, + bdlma::ConcurrentPool* pushElementsPool); + + ClusterResources(const ClusterResources& copy); + + // ACCESSORS + + /// Returns a pointer to the event scheduler + bdlmt::EventScheduler* scheduler() const; + + /// Returns a pointer to the blob buffer factory + bdlbb::BlobBufferFactory* bufferFactory() const; + + /// Returns a pointer to the shared blob objects pool + BlobSpPool* blobSpPool() const; + + /// Returns a pointer to the concurrent pool for Push elements + const bsl::optional& pushElementsPool() const; +}; + // ============================================================================ // INLINE DEFINITIONS // ============================================================================ @@ -474,6 +532,70 @@ inline bool Cluster::isFSMWorkflow() const return false; } +inline ClusterResources::ClusterResources( + bdlmt::EventScheduler* scheduler, + bdlbb::BlobBufferFactory* bufferFactory, + BlobSpPool* blobSpPool) +: d_scheduler_p(scheduler) +, d_bufferFactory_p(bufferFactory) +, d_blobSpPool_p(blobSpPool) +, d_pushElementsPool() +{ + BSLS_ASSERT_SAFE(d_scheduler_p); + BSLS_ASSERT_SAFE(d_bufferFactory_p); + BSLS_ASSERT_SAFE(d_blobSpPool_p); +} + +inline ClusterResources::ClusterResources( + bdlmt::EventScheduler* scheduler, + bdlbb::BlobBufferFactory* bufferFactory, + BlobSpPool* blobSpPool, + bdlma::ConcurrentPool* pushElementsPool) +: d_scheduler_p(scheduler) +, d_bufferFactory_p(bufferFactory) +, d_blobSpPool_p(blobSpPool) +, d_pushElementsPool(pushElementsPool) +{ + BSLS_ASSERT_SAFE(d_scheduler_p); + BSLS_ASSERT_SAFE(d_bufferFactory_p); + BSLS_ASSERT_SAFE(d_blobSpPool_p); + BSLS_ASSERT_SAFE(d_pushElementsPool); +} + +inline ClusterResources::ClusterResources(const ClusterResources& copy) +: d_scheduler_p(copy.d_scheduler_p) +, d_bufferFactory_p(copy.d_bufferFactory_p) +, d_blobSpPool_p(copy.d_blobSpPool_p) +, d_pushElementsPool(copy.d_pushElementsPool) +{ + // NOTHING +} + +inline bdlmt::EventScheduler* ClusterResources::scheduler() const +{ + return d_scheduler_p; +} +// EventScheduler to use + +inline bdlbb::BlobBufferFactory* ClusterResources::bufferFactory() const +{ + return d_bufferFactory_p; +} +// Blob buffer factory to use + +inline ClusterResources::BlobSpPool* ClusterResources::blobSpPool() const +{ + return d_blobSpPool_p; +} +// Pool of shared pointers to blob to +// use. + +inline const bsl::optional& +ClusterResources::pushElementsPool() const +{ + return d_pushElementsPool; +} + } // close package namespace // ----------------------- diff --git a/src/groups/mqb/mqbi/mqbi_queueengine.cpp b/src/groups/mqb/mqbi/mqbi_queueengine.cpp index ac6b2d1be..582956c6b 100644 --- a/src/groups/mqb/mqbi/mqbi_queueengine.cpp +++ b/src/groups/mqb/mqbi/mqbi_queueengine.cpp @@ -47,5 +47,21 @@ void QueueEngine::afterAppIdUnregistered( // NOTHING } +void QueueEngine::registerStorage( + BSLS_ANNOTATION_UNUSED const bsl::string& appId, + BSLS_ANNOTATION_UNUSED const mqbu::StorageKey& appKey, + BSLS_ANNOTATION_UNUSED unsigned int appOrdinal) +{ + // NOTHING +} + +void QueueEngine::unregisterStorage( + BSLS_ANNOTATION_UNUSED const bsl::string& appId, + BSLS_ANNOTATION_UNUSED const mqbu::StorageKey& appKey, + BSLS_ANNOTATION_UNUSED unsigned int appOrdinal) +{ + // NOTHING +} + } // close package namespace } // close enterprise namespace diff --git a/src/groups/mqb/mqbi/mqbi_queueengine.h b/src/groups/mqb/mqbi/mqbi_queueengine.h index ee3944f85..cf6f27195 100644 --- a/src/groups/mqb/mqbi/mqbi_queueengine.h +++ b/src/groups/mqb/mqbi/mqbi_queueengine.h @@ -207,6 +207,22 @@ class QueueEngine { virtual void afterAppIdUnregistered(const mqbi::Storage::AppIdKeyPair& appIdKeyPair); + /// Called after creation of a new storage for the specified + /// `appIdKeyPair`. + /// + /// THREAD: This method is called from the Queue's dispatcher thread. + virtual void registerStorage(const bsl::string& appId, + const mqbu::StorageKey& appKey, + unsigned int appOrdinal); + + /// Called after removal of the storage for the specified + /// `appIdKeyPair`. + /// + /// THREAD: This method is called from the Queue's dispatcher thread. + virtual void unregisterStorage(const bsl::string& appId, + const mqbu::StorageKey& appKey, + unsigned int appOrdinal); + /// Given the specified 'putHeader', 'appData', 'mpi', and 'timestamp', /// evaluate all Auto (Application) subscriptions and exclude applications /// with negative results from message delivery. diff --git a/src/groups/mqb/mqbi/mqbi_storage.h b/src/groups/mqb/mqbi/mqbi_storage.h index fbbce4961..73748f33e 100644 --- a/src/groups/mqb/mqbi/mqbi_storage.h +++ b/src/groups/mqb/mqbi/mqbi_storage.h @@ -261,6 +261,38 @@ bool operator==(const StorageMessageAttributes& lhs, bool operator!=(const StorageMessageAttributes& lhs, const StorageMessageAttributes& rhs); +struct AppMessage { + // VST to track the state associated with (GUID, App) pair. + + // PULIC TYOES + enum State { e_NONE = 0, e_PUT = 1, e_PUSH = 2, e_CONFIRM = 3 }; + + bmqp::RdaInfo d_rdaInfo; + unsigned int d_subscriptionId; + State d_state; + + AppMessage(const bmqp::RdaInfo& rdaInfo); + + /// Set this object state to indicate a delivery by (Replica or Proxy). + void setPushState(); + + /// Set this object state to indicate a corresponding CONFIRM. + void setConfirmState(); + + /// Set this object state to indicate a corresponding CONFIRM. + void setRemovedState(); + + /// Return `true` if this object is expecting CONFIRM or purge. + bool isPending() const; + + /// Return `true` if this object is expecting CONFIRM or purge but + /// had not been sent out (by Replica or Proxy). + bool isNew() const; + + /// Return `true` if this object is submitted for delivery. + bool isPushing() const; +}; + // ===================== // class StorageIterator // ===================== @@ -286,9 +318,11 @@ class StorageIterator { /// items' collection. virtual bool advance() = 0; - /// Reset the iterator to point to first item, if any, in the underlying - /// storage. - virtual void reset() = 0; + /// If the specified `atEnd` is `true`, reset the iterator to point to the + /// to the end of the underlying storage. Otherwise, reset the iterator to + /// point first item, if any, in the underlying storage. + virtual void + reset(const bmqt::MessageGUID& where = bmqt::MessageGUID()) = 0; // ACCESSORS @@ -297,15 +331,16 @@ class StorageIterator { /// behavior is undefined unless `atEnd` returns `false`. virtual const bmqt::MessageGUID& guid() const = 0; - /// Return a reference offering modifiable access to the mutable RdaInfo + /// Return a reference offering non-modifiable access to to the App state /// associated to the item currently pointed at by this iterator. The /// behavior is undefined unless `atEnd` returns `false`. - virtual bmqp::RdaInfo& rdaInfo() const = 0; + virtual const AppMessage& + appMessageView(unsigned int appOrdinal) const = 0; - /// Return subscription id associated to the item currently pointed at - /// by this iterator. - /// The behavior is undefined unless `atEnd` returns `false`. - virtual unsigned int subscriptionId() const = 0; + /// Return a reference offering modifiable access to the App state + /// associated to the item currently pointed at by this iterator. The + /// behavior is undefined unless `atEnd` returns `false`. + virtual AppMessage& appMessageState(unsigned int appOrdinal) = 0; /// Return a reference offering non-modifiable access to the application /// data associated with the item currently pointed at by this iterator. @@ -352,6 +387,8 @@ class Storage { bmqp::Protocol::k_SUBID_ARRAY_STATIC_LEN> StorageKeys; + static const size_t k_INVALID_ORDINAL = 999999; + public: // CREATORS @@ -374,30 +411,25 @@ class Storage { virtual void setQueue(mqbi::Queue* queue) = 0; - virtual mqbi::Queue* queue() = 0; - /// Close this storage. virtual void close() = 0; - /// Get an iterator for items stored in the virtual storage identified - /// by the specified `appKey`. Iterator will point to point to the - /// oldest item, if any, or to the end of the collection if empty. Note - /// that if `appKey` is null, an iterator over the underlying physical - /// storage will be returned. Also note that because `Storage` and - /// `StorageIterator` are interfaces, the implementation of this method - /// will allocate, so it's recommended to keep the iterator. + /// Get an iterator for data stored in the virtual storage identified by + /// the specified `appKey`. + /// If the `appKey` is null, the returned iterator can iterate states of + /// all Apps; otherwise, the iterator can iterate states of the App + /// corresponding to the `appKey`. virtual bslma::ManagedPtr getIterator(const mqbu::StorageKey& appKey) = 0; - /// Load into the specified `out` an iterator for items stored in - /// the virtual storage identified by the specified `appKey`, initially - /// pointing to the item associated with the specified `msgGUID`. + /// Load into the specified `out` an iterator for data stored in the + /// virtual storage initially pointing to the message associated with the + /// specified `msgGUID`. + /// If the `appKey` is null, the returned iterator can iterate states of + /// all Apps; otherwise, the iterator can iterate states of the App + /// corresponding to the `appKey`. /// Return zero on success, and a non-zero code if `msgGUID` was not - /// found in the storage. Note that if `appKey` is null, an iterator - /// over the underlying physical storage will be returned. Also note - /// that because `Storage` and `StorageIterator` are interfaces, the - /// implementation of this method will allocate, so it's recommended to - /// keep the iterator. + /// found in the storage. virtual StorageResult::Enum getIterator(bslma::ManagedPtr* out, const mqbu::StorageKey& appKey, @@ -405,48 +437,71 @@ class Storage { /// Save the message contained in the specified `appData`, `options` and /// the associated `attributes` and `msgGUID` into this storage and the - /// associated virtual storages, if any. The `attributes` is an in/out - /// parameter and storage layer can populate certain fields of that - /// struct. Return 0 on success or an non-zero error code on failure. + /// associated virtual storage. The `attributes` is an in/out parameter + /// and storage layer can populate certain fields of that struct. + /// Return 0 on success or an non-zero error code on failure. virtual StorageResult::Enum put(StorageMessageAttributes* attributes, const bmqt::MessageGUID& msgGUID, const bsl::shared_ptr& appData, - const bsl::shared_ptr& options, - const StorageKeys& storageKeys = StorageKeys()) = 0; - - // TBD: Have this method invoke 'beforeMessageRemoved' on the assoicated - // QueueEngine to notify it that a message is "released" for the - // subStream associated with the specified 'appKey'. - - /// Release the reference of the specified `appKey` on the message - /// identified by the specified `msgGUID`, and record this event in the - /// storage. Return one of the return codes from: - /// * **e_GUID_NOT_FOUND** : `msgGUID` was not found - /// * **e_ZERO_REFERENCES** : message refCount has become zero - /// * **e_NON_ZERO_REFERENCES** : message refCount is still not zero - /// * **e_WRITE_FAILURE** : failed to record this event in storage - virtual StorageResult::Enum releaseRef(const bmqt::MessageGUID& msgGUID, - const mqbu::StorageKey& appKey, - bsls::Types::Int64 timestamp, - bool onReject = false) = 0; + const bsl::shared_ptr& options) = 0; + + /// Update the App state corresponding to the specified `msgGUID` and the + /// specified `appKey` in the DataStream. Decrement the reference count of + /// the message identified by the `msgGUID`, and record the CONFIRM in the + /// storage. + /// Return one of the return codes from: + /// * e_SUCCESS : success + /// * e_GUID_NOT_FOUND : `msgGUID` was not found + /// * e_ZERO_REFERENCES : message refCount has become zero + /// * e_NON_ZERO_REFERENCES : message refCount is still not zero + /// * e_WRITE_FAILURE : failed to record this event in storage + /// + /// Behavior is undefined unless there is an App with the `appKey`. + /// + /// On CONFIRM, the caller of `confirm` is responsible to follow with + /// `remove` call. `releaseRef` is an alternative way to remove message in + /// one call. + virtual StorageResult::Enum confirm(const bmqt::MessageGUID& msgGUID, + const mqbu::StorageKey& appKey, + bsls::Types::Int64 timestamp, + bool onReject = false) = 0; + + /// Decrement the reference count of the message identified by the + /// `msgGUID`. If the resulting value is zero, delete the message data and + /// record the event in the storage. + /// Return one of the return codes from: + /// * e_SUCCESS : success + /// * e_GUID_NOT_FOUND : `msgGUID` was not found + /// * e_INVALID_OPERATION : the value is invalid (already zero) + /// * e_ZERO_REFERENCES : message refCount has become zero + /// * e_NON_ZERO_REFERENCE : message refCount is still not zero + /// + /// On CONFIRM, the caller of `confirm` is responsible to follow with + /// `remove` call. `releaseRef` is an alternative way to remove message in + /// one call. + virtual StorageResult::Enum + releaseRef(const bmqt::MessageGUID& msgGUID) = 0; /// Remove from the storage the message having the specified `msgGUID` - /// and store it's size, in bytes, in the optionally specified `msgSize` - /// if the `msgGUID` was found. Return 0 on success, or a non-zero - /// return code if the `msgGUID` was not found. If the optionally - /// specified `clearAll` is true, remove the message from all virtual - /// storages as well. + /// and store it's size, in bytes, in the optionally specified `msgSize`. + /// Record the event in the storage. + /// Return 0 on success, or a non-zero return code if the `msgGUID` was not + /// found or if has failed to record this event in storage. + /// + /// On CONFIRM, the caller of `confirm` is responsible to follow with + /// `remove` call. `releaseRef` is an alternative way to remove message in + /// one call. virtual StorageResult::Enum remove(const bmqt::MessageGUID& msgGUID, - int* msgSize = 0, - bool clearAll = false) = 0; - - /// Remove all messages from this storage for the client identified by - /// the specified `appKey`. If `appKey` is null, then remove messages - /// for all clients. Return one of the return codes from: - /// * **e_SUCCESS** : `msgGUID` was not found - /// * **e_WRITE_FAILURE** : failed to record this event in storage - /// * **e_APPKEY_NOT_FOUND** : Invalid appKey specified + int* msgSize = 0) = 0; + + /// Remove all messages from this storage for the App identified by the + /// specified `appKey` if `appKey` is not null. Otherwise, remove messages + /// for all Apps. Record the event in the storage. + /// Return one of the return codes from: + /// * e_SUCCESS : success + /// * e_WRITE_FAILURE : failed to record this event in storage + /// * e_APPKEY_NOT_FOUND : Invalid `appKey` specified virtual StorageResult::Enum removeAll(const mqbu::StorageKey& appKey) = 0; /// If the specified `storage` is `true`, flush any buffered replication @@ -493,29 +548,24 @@ class Storage { virtual void selectForAutoConfirming(const bmqt::MessageGUID& msgGUID) = 0; virtual StorageResult::Enum autoConfirm(const mqbu::StorageKey& appKey, bsls::Types::Uint64 timestamp) = 0; - /// The sequence of calls is 'selectForAutoConfirming', then zero or more - /// 'autoConfirm', then 'put' - all for the same specified 'msgGUID'. - /// 'autoConfirm' replicates ephemeral auto CONFIRM for the specified - /// 'appKey' in persistent storage. + /// The sequence of calls is `selectForAutoConfirming`, then zero or more + /// `autoConfirm`, then `put` - all for the same specified `msgGUID`. + /// `autoConfirm` replicates ephemeral auto CONFIRM for the specified + /// `appKey` in persistent storage. /// Any other sequence removes auto CONFIRMs. /// Auto-confirmed Apps do not PUSH the message. // ACCESSORS + /// Return the queue this storage is associated with. + virtual mqbi::Queue* queue() const = 0; + /// Return the URI of the queue this storage is associated with. virtual const bmqt::Uri& queueUri() const = 0; /// Return the storage key associated with this instance. virtual const mqbu::StorageKey& queueKey() const = 0; - /// Return the appId associated with this storage instance. If there is - /// not appId associated, return an empty string. - virtual const bsl::string& appId() const = 0; - - /// Return the app key, if any, associated with this storage instance. - /// If there is no appKey associated, return a null key. - virtual const mqbu::StorageKey& appKey() const = 0; - /// Return the current configuration used by this storage. The behavior /// is undefined unless `configure` was successfully called. virtual const mqbconfm::Storage& config() const = 0; @@ -587,10 +637,11 @@ class Storage { /// Return true if virtual storage identified by the specified `appId` /// exists, otherwise return false. Load into the optionally specified - /// `appKey` the appKey associated with `appId` if the virtual storage - /// exists, otherwise set it to 0. + /// `appKey` and `ordinal` the appKey and ordinal associated with `appId` + /// if the virtual storage exists, otherwise set it to 0. virtual bool hasVirtualStorage(const bsl::string& appId, - mqbu::StorageKey* appKey = 0) const = 0; + mqbu::StorageKey* appKey = 0, + unsigned int* ordinal = 0) const = 0; /// Return `true` if there was Replication Receipt for the specified /// `msgGUID`. @@ -608,6 +659,48 @@ class Storage { // INLINE DEFINITIONS // ============================================================================ +// ---------------- +// class AppMessage +// ---------------- + +inline AppMessage::AppMessage(const bmqp::RdaInfo& rdaInfo) +: d_rdaInfo(rdaInfo) +, d_subscriptionId(bmqp::Protocol::k_DEFAULT_SUBSCRIPTION_ID) +, d_state(e_PUT) +{ + // NOTHING +} + +inline void AppMessage::setPushState() +{ + d_state = e_PUSH; +} + +inline void AppMessage::setConfirmState() +{ + d_state = e_CONFIRM; +} + +inline void AppMessage::setRemovedState() +{ + d_state = e_NONE; +} + +inline bool AppMessage::isPending() const +{ + return d_state == e_PUT || d_state == e_PUSH; +} + +inline bool AppMessage::isNew() const +{ + return d_state == e_PUT; +} + +inline bool AppMessage::isPushing() const +{ + return d_state == e_PUSH; +} + // ------------------------------ // class StorageMessageAttributes // ------------------------------ diff --git a/src/groups/mqb/mqbmock/mqbmock_cluster.cpp b/src/groups/mqb/mqbmock/mqbmock_cluster.cpp index 9b14f2e91..3d9b39e4f 100644 --- a/src/groups/mqb/mqbmock/mqbmock_cluster.cpp +++ b/src/groups/mqb/mqbmock/mqbmock_cluster.cpp @@ -242,6 +242,7 @@ Cluster::Cluster(bdlbb::BlobBufferFactory* bufferFactory, , d_isLeader(isLeader) , d_isRestoringState(false) , d_processor() +, d_resources(&d_scheduler, bufferFactory, &d_blobSpPool) { // PRECONDITIONS if (isClusterMember) { @@ -268,9 +269,7 @@ Cluster::Cluster(bdlbb::BlobBufferFactory* bufferFactory, d_clusterData_mp.load(new (*d_allocator_p) mqbc::ClusterData( d_clusterDefinition.name(), - &d_scheduler, - d_bufferFactory_p, - &d_blobSpPool, + d_resources, d_clusterDefinition, mqbcfg::ClusterProxyDefinition(d_allocator_p), d_netCluster_mp, diff --git a/src/groups/mqb/mqbmock/mqbmock_cluster.h b/src/groups/mqb/mqbmock/mqbmock_cluster.h index bcdd63a34..d8cfc5cc6 100644 --- a/src/groups/mqb/mqbmock/mqbmock_cluster.h +++ b/src/groups/mqb/mqbmock/mqbmock_cluster.h @@ -220,6 +220,8 @@ class Cluster : public mqbi::Cluster { EventProcessor d_processor; + mqbi::ClusterResources d_resources; + private: // NOT IMPLEMENTED Cluster(const Cluster&) BSLS_CPP11_DELETED; @@ -440,6 +442,9 @@ class Cluster : public mqbi::Cluster { /// Move the test timer forward the specified `seconds`. void advanceTime(int seconds); + /// Move the test timer forward the specified `milliseconds`. + void advanceTime(const bsls::TimeInterval& interval); + /// Block until scheduler executes all the scheduled callbacks. void waitForScheduler(); @@ -526,6 +531,7 @@ class Cluster : public mqbi::Cluster { const bdlmt::EventSchedulerTestTimeSource& _timeSource() const; const TestChannelMap& _channels() const; const mqbc::ClusterData* _clusterData() const; + const mqbi::ClusterResources& _resources() const; /// Return the value of the corresponding member of this object. const mqbc::ClusterState& _state() const; @@ -600,6 +606,11 @@ inline void Cluster::advanceTime(int seconds) d_timeSource.advanceTime(bsls::TimeInterval(seconds)); } +inline void Cluster::advanceTime(const bsls::TimeInterval& interval) +{ + d_timeSource.advanceTime(interval); +} + inline void Cluster::getPrimaryNodes(int* rc, bsl::ostream& errorDescription, bsl::vector* nodes, @@ -657,6 +668,11 @@ inline const mqbc::ClusterState& Cluster::_state() const return d_state; } +inline const mqbi::ClusterResources& Cluster::_resources() const +{ + return d_resources; +} + inline bsls::TimeInterval Cluster::getTime() const { return d_timeSource.now(); diff --git a/src/groups/mqb/mqbs/mqbs_filebackedstorage.cpp b/src/groups/mqb/mqbs/mqbs_filebackedstorage.cpp index 9e1c53a01..31a48a145 100644 --- a/src/groups/mqb/mqbs/mqbs_filebackedstorage.cpp +++ b/src/groups/mqb/mqbs/mqbs_filebackedstorage.cpp @@ -90,8 +90,8 @@ void FileBackedStorage::purgeCommon(const mqbu::StorageKey& appKey) // Update stats d_capacityMeter.clear(); - if (d_queue_p) { - d_queue_p->stats()->onEvent( + if (queue()) { + queue()->stats()->onEvent( mqbstat::QueueStatsDomain::EventType::e_PURGE, 0); } @@ -105,12 +105,10 @@ FileBackedStorage::FileBackedStorage( const mqbu::StorageKey& queueKey, const mqbconfm::Domain& config, mqbu::CapacityMeter* parentCapacityMeter, - const bmqp::RdaInfo& defaultRdaInfo, bslma::Allocator* allocator, mwcma::CountingAllocatorStore* allocatorStore) : d_allocator_p(allocator) , d_store_p(dataStore) -, d_queue_p(0) , d_queueKey(queueKey) , d_config() , d_queueUri(queueUri, allocator) @@ -126,22 +124,13 @@ FileBackedStorage::FileBackedStorage( .totalNanoseconds(), allocatorStore ? allocatorStore->get("Handles") : d_allocator_p) , d_queueOpRecordHandles(allocator) -, d_emptyAppId(allocator) -, d_nullAppKey() , d_isEmpty(1) -, d_defaultRdaInfo(defaultRdaInfo) , d_hasReceipts(!config.consistency().isStrongValue()) , d_currentlyAutoConfirming() -, d_ephemeralConfirms(d_allocator_p) +, d_autoConfirms(d_allocator_p) { BSLS_ASSERT(d_store_p); - if (config.maxDeliveryAttempts()) { - d_defaultRdaInfo.setCounter(config.maxDeliveryAttempts()); - } - else { - d_defaultRdaInfo.setUnlimited(); - } // Note that the specified 'parentCapacityMeter' (and thus // 'd_capacityMeter.parent()') can be zero, so we can't assert on it being // non zero. This is possible when a node comes up, recovers a queue, @@ -151,6 +140,8 @@ FileBackedStorage::FileBackedStorage( // instance associated with it (instead of a 'mqbblp::Cluster' instance), // and domain instance will return a zero capacity meter when queries to be // passed to the 'FileBackedStorage' instance. + + d_virtualStorageCatalog.setDefaultRda(config.maxDeliveryAttempts()); } FileBackedStorage::~FileBackedStorage() @@ -176,7 +167,7 @@ FileBackedStorage::get(bsl::shared_ptr* appData, if (handles[0].primaryLeaseId() < d_store_p->primaryLeaseId()) { // Consider this the past that needs translation - bmqp::SchemaLearner& learner = d_queue_p->schemaLearner(); + bmqp::SchemaLearner& learner = queue()->schemaLearner(); attributes->setMessagePropertiesInfo(learner.multiplex( learner.createContext(handles[0].primaryLeaseId()), @@ -191,7 +182,7 @@ mqbi::StorageResult::Enum FileBackedStorage::get(mqbi::StorageMessageAttributes* attributes, const bmqt::MessageGUID& msgGUID) const { - BSLS_ASSERT_SAFE(d_queue_p); + BSLS_ASSERT_SAFE(queue()); RecordHandleMap::const_iterator it = d_handles.find(msgGUID); if (it == d_handles.end()) { @@ -204,7 +195,7 @@ FileBackedStorage::get(mqbi::StorageMessageAttributes* attributes, if (handles[0].primaryLeaseId() < d_store_p->primaryLeaseId()) { // Consider this the past that needs translation - bmqp::SchemaLearner& learner = d_queue_p->schemaLearner(); + bmqp::SchemaLearner& learner = queue()->schemaLearner(); attributes->setMessagePropertiesInfo(learner.multiplex( learner.createContext(handles[0].primaryLeaseId()), @@ -237,7 +228,7 @@ int FileBackedStorage::configure( const mqbconfm::Storage& config, const mqbconfm::Limits& limits, const bsls::Types::Int64 messageTtl, - const int maxDeliveryAttempts) + int maxDeliveryAttempts) { d_config = config; d_capacityMeter.setLimits(limits.messages(), limits.bytes()) @@ -245,28 +236,24 @@ int FileBackedStorage::configure( limits.bytesWatermarkRatio()); d_ttlSeconds = messageTtl; - if (maxDeliveryAttempts > 0) { - d_defaultRdaInfo.setCounter(maxDeliveryAttempts); - } - else { - d_defaultRdaInfo.setUnlimited(); - } + d_virtualStorageCatalog.setDefaultRda(maxDeliveryAttempts); + return 0; } void FileBackedStorage::setQueue(mqbi::Queue* queue) { - d_queue_p = queue; + d_virtualStorageCatalog.setQueue(queue); // Update queue stats if a queue has been associated with the storage. - if (d_queue_p) { + if (queue) { const bsls::Types::Int64 numMessage = numMessages( mqbu::StorageKey::k_NULL_KEY); const bsls::Types::Int64 numByte = numBytes( mqbu::StorageKey::k_NULL_KEY); - d_queue_p->stats()->setQueueContentRaw(numMessage, numByte); + queue->stats()->setQueueContentRaw(numMessage, numByte); BALL_LOG_INFO << "Associated queue [" << queue->uri() << "] with key [" << queueKey() << "] and Partition [" @@ -287,101 +274,81 @@ mqbi::StorageResult::Enum FileBackedStorage::put(mqbi::StorageMessageAttributes* attributes, const bmqt::MessageGUID& msgGUID, const bsl::shared_ptr& appData, - const bsl::shared_ptr& options, - const StorageKeys& storageKeys) + const bsl::shared_ptr& options) { const int msgSize = appData->length(); - if (storageKeys.empty()) { - // Store the specified message in the 'physical' as well as *all* - // virtual storages. + // Store the specified message in the 'physical' as well as *all* + // virtual storages. - if (d_handles.isInHistory(msgGUID)) { - return mqbi::StorageResult::e_DUPLICATE; - } + if (d_handles.isInHistory(msgGUID)) { + return mqbi::StorageResult::e_DUPLICATE; + } - // Verify if we have enough capacity. - mqbu::CapacityMeter::CommitResult capacity = - d_capacityMeter.commitUnreserved(1, msgSize); + // Verify if we have enough capacity. + mqbu::CapacityMeter::CommitResult capacity = + d_capacityMeter.commitUnreserved(1, msgSize); - if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( - capacity != mqbu::CapacityMeter::e_SUCCESS)) { - BSLS_PERFORMANCEHINT_UNLIKELY_HINT; + if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( + capacity != mqbu::CapacityMeter::e_SUCCESS)) { + BSLS_PERFORMANCEHINT_UNLIKELY_HINT; - return (capacity == mqbu::CapacityMeter::e_LIMIT_MESSAGES - ? mqbi::StorageResult::e_LIMIT_MESSAGES - : mqbi::StorageResult::e_LIMIT_BYTES); // RETURN - } + return (capacity == mqbu::CapacityMeter::e_LIMIT_MESSAGES + ? mqbi::StorageResult::e_LIMIT_MESSAGES + : mqbi::StorageResult::e_LIMIT_BYTES); // RETURN + } - // Update - DataStoreRecordHandle handle; - int rc = d_store_p->writeMessageRecord(attributes, - &handle, - msgGUID, - appData, - options, - d_queueKey); - if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY(rc != 0)) { - BSLS_PERFORMANCEHINT_UNLIKELY_HINT; - - // Rollback reserved capacity. - d_capacityMeter.remove(1, msgSize); - return mqbi::StorageResult::e_WRITE_FAILURE; // RETURN - } + // Update + DataStoreRecordHandle handle; + int rc = d_store_p->writeMessageRecord(attributes, + &handle, + msgGUID, + appData, + options, + d_queueKey); + if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY(rc != 0)) { + BSLS_PERFORMANCEHINT_UNLIKELY_HINT; + + // Rollback reserved capacity. + d_capacityMeter.remove(1, msgSize); + return mqbi::StorageResult::e_WRITE_FAILURE; // RETURN + } - InsertRc irc = d_handles.insert(bsl::make_pair(msgGUID, Item()), - attributes->arrivalTimepoint()); + InsertRc irc = d_handles.insert(bsl::make_pair(msgGUID, Item()), + attributes->arrivalTimepoint()); - irc.first->second.d_array.push_back(handle); - irc.first->second.d_refCount = attributes->refCount(); + irc.first->second.d_array.push_back(handle); + irc.first->second.d_refCount = attributes->refCount(); - // Looks like extra lookup in - // VirtualStorageIterator::loadMessageAndAttributes() can be avoided - // if we keep `irc` (like we keep 'DataStoreRecordHandle'). + // Looks like extra lookup in + // VirtualStorageIterator::loadMessageAndAttributes() can be avoided + // if we keep `irc` (like we keep 'DataStoreRecordHandle'). - d_virtualStorageCatalog.put(msgGUID, - msgSize, - d_defaultRdaInfo, - bmqp::Protocol::k_DEFAULT_SUBSCRIPTION_ID, - mqbu::StorageKey::k_NULL_KEY); + if (d_autoConfirms.empty()) { + d_virtualStorageCatalog.put(msgGUID, msgSize); + } + else { + VirtualStorage::DataStreamMessage* dataStreamMessage = 0; + d_virtualStorageCatalog.put(msgGUID, msgSize, &dataStreamMessage); // Move auto confirms to the data record - for (unsigned int i = 0; i < d_ephemeralConfirms.size(); ++i) { - irc.first->second.d_array.push_back(d_ephemeralConfirms[i]); + for (AutoConfirms::const_iterator it = d_autoConfirms.begin(); + it != d_autoConfirms.end(); + ++it) { + irc.first->second.d_array.push_back(it->d_confirmRecordHandle); + d_virtualStorageCatalog.autoConfirm(dataStreamMessage, + it->d_appKey); } - d_ephemeralConfirms.clear(); - d_currentlyAutoConfirming = bmqt::MessageGUID(); - - BSLS_ASSERT_SAFE(d_queue_p); - d_queue_p->stats()->onEvent( - mqbstat::QueueStatsDomain::EventType::e_ADD_MESSAGE, - msgSize); - - d_isEmpty.storeRelaxed(0); - - return mqbi::StorageResult::e_SUCCESS; // RETURN - } - // 'storageKeys' is not empty only when proxy receives PUSH. - // Auto confirming does no apply then. - - // Store the specified message only in the virtual storages identified by - // the specified 'storageKeys'. Note that since message is not added to - // the 'physical' storage, we don't modify 'd_capacityMeter', 'd_isEmpty', - // etc variables. - BSLS_ASSERT(hasMessage(msgGUID)); - - for (size_t i = 0; i < storageKeys.size(); ++i) { - d_virtualStorageCatalog.put(msgGUID, - msgSize, - d_defaultRdaInfo, - bmqp::Protocol::k_DEFAULT_SUBSCRIPTION_ID, - storageKeys[i]); + d_autoConfirms.clear(); } + d_currentlyAutoConfirming = bmqt::MessageGUID(); - // Note that unlike 'InMemoryStorage', we don't add the message to the - // 'physical' storage in this case, because proxies don't use - // FileBackedStorage. - // TBD: this logic needs to be cleaned up. + BSLS_ASSERT_SAFE(queue()); + queue()->stats()->onEvent( + mqbstat::QueueStatsDomain::EventType::e_ADD_MESSAGE, + msgSize); + + d_isEmpty.storeRelaxed(0); return mqbi::StorageResult::e_SUCCESS; // RETURN } @@ -389,15 +356,6 @@ FileBackedStorage::put(mqbi::StorageMessageAttributes* attributes, bslma::ManagedPtr FileBackedStorage::getIterator(const mqbu::StorageKey& appKey) { - if (appKey.isNull()) { - bslma::ManagedPtr mp( - new (*d_allocator_p) - FileBackedStorageIterator(this, d_handles.begin()), - d_allocator_p); - - return mp; // RETURN - } - return d_virtualStorageCatalog.getIterator(appKey); } @@ -406,52 +364,41 @@ FileBackedStorage::getIterator(bslma::ManagedPtr* out, const mqbu::StorageKey& appKey, const bmqt::MessageGUID& msgGUID) { - if (appKey.isNull()) { - RecordHandleMap::const_iterator it = d_handles.find(msgGUID); - if (it == d_handles.end()) { - return mqbi::StorageResult::e_GUID_NOT_FOUND; // RETURN - } - - out->load(new (*d_allocator_p) FileBackedStorageIterator(this, it), - d_allocator_p); - - return mqbi::StorageResult::e_SUCCESS; // RETURN - } - return d_virtualStorageCatalog.getIterator(out, appKey, msgGUID); } mqbi::StorageResult::Enum -FileBackedStorage::releaseRef(const bmqt::MessageGUID& msgGUID, - const mqbu::StorageKey& appKey, - bsls::Types::Int64 timestamp, - bool onReject) +FileBackedStorage::confirm(const bmqt::MessageGUID& msgGUID, + const mqbu::StorageKey& appKey, + bsls::Types::Int64 timestamp, + bool onReject) { + // PRECONDITIONS + BSLS_ASSERT_SAFE(!appKey.isNull()); + RecordHandleMap::iterator it = d_handles.find(msgGUID); if (it == d_handles.end()) { return mqbi::StorageResult::e_GUID_NOT_FOUND; // RETURN } - if (!appKey.isNull()) { - mqbi::StorageResult::Enum rc = d_virtualStorageCatalog.remove(msgGUID, - appKey); - if (mqbi::StorageResult::e_SUCCESS != rc) { - return rc; // RETURN - } + mqbi::StorageResult::Enum rc = d_virtualStorageCatalog.confirm(msgGUID, + appKey); + if (mqbi::StorageResult::e_SUCCESS != rc) { + return rc; // RETURN } RecordHandlesArray& handles = it->second.d_array; BSLS_ASSERT_SAFE(!handles.empty()); DataStoreRecordHandle handle; - int rc = d_store_p->writeConfirmRecord( + int writeResult = d_store_p->writeConfirmRecord( &handle, msgGUID, d_queueKey, appKey, timestamp, onReject ? ConfirmReason::e_REJECTED : ConfirmReason::e_CONFIRMED); - if (0 != rc) { + if (0 != writeResult) { // If 'appKey' isn't null, we have already removed 'msgGUID' from the // virtual storage of 'appKey'. This is ok, because if above 'write' // has failed, its game over for this node anyways. @@ -469,20 +416,83 @@ FileBackedStorage::releaseRef(const bmqt::MessageGUID& msgGUID, } mqbi::StorageResult::Enum -FileBackedStorage::remove(const bmqt::MessageGUID& msgGUID, - int* msgSize, - bool clearAll) +FileBackedStorage::releaseRef(const bmqt::MessageGUID& guid) { - RecordHandleMap::iterator it = d_handles.find(msgGUID); + RecordHandleMapIter it = d_handles.find(guid); if (it == d_handles.end()) { - return mqbi::StorageResult::e_GUID_NOT_FOUND; // RETURN + return mqbi::StorageResult::e_GUID_NOT_FOUND; } - if (clearAll) { - d_virtualStorageCatalog.remove(msgGUID, mqbu::StorageKey::k_NULL_KEY); + if (0 == it->second.d_refCount) { + // Outstanding refCount for this message is already zero. + + return mqbi::StorageResult::e_INVALID_OPERATION; } - BSLS_ASSERT_SAFE(!d_virtualStorageCatalog.hasMessage(msgGUID)); + const RecordHandlesArray& handles = it->second.d_array; + BSLS_ASSERT_SAFE(!handles.empty()); + + if (0 == --it->second.d_refCount) { + // This appKey was the last outstanding client for this message. + // Message can now be deleted. + + int msgLen = static_cast(d_store_p->getMessageLenRaw(handles[0])); + + int rc = d_store_p->writeDeletionRecord( + guid, + d_queueKey, + DeletionRecordFlag::e_NONE, + bdlt::EpochUtil::convertToTimeT64(bdlt::CurrentTime::utc())); + + if (0 != rc) { + MWCTSK_ALARMLOG_ALARM("FILE_IO") + << "PartitionId [" << partitionId() << "] failed to write " + << "DELETION record for GUID: " << guid << ", for queue '" + << d_queueUri << "', queueKey '" << d_queueKey + << "' while attempting to purge the message, rc: " << rc + << MWCTSK_ALARMLOG_END; + } + + // If a queue is associated, inform it about the message being + // deleted, and update queue stats. + // The same 'e_DEL_MESSAGE' is about 3 cases: TTL, no SC quorum, + // and a purge. + if (queue()) { + queue()->queueEngine()->beforeMessageRemoved(guid); + queue()->stats()->onEvent( + mqbstat::QueueStatsDomain::EventType::e_DEL_MESSAGE, + msgLen); + } + + // There is not really a need to remove the guid from all virtual + // storages, because we can be here only if guid doesn't exist in + // any virtual storage apart from 'vs' (because updated outstanding + // refCount is zero). So we just delete records associated with + // the guid from the underlying (this) storage. + + for (unsigned int i = 0; i < handles.size(); ++i) { + d_store_p->removeRecordRaw(handles[i]); + } + + d_capacityMeter.remove(1, msgLen); + d_handles.erase(it); + + return mqbi::StorageResult::e_ZERO_REFERENCES; + } + else { + return mqbi::StorageResult::e_NON_ZERO_REFERENCES; + } +} + +mqbi::StorageResult::Enum +FileBackedStorage::remove(const bmqt::MessageGUID& msgGUID, int* msgSize) +{ + RecordHandleMap::iterator it = d_handles.find(msgGUID); + if (it == d_handles.end()) { + return mqbi::StorageResult::e_GUID_NOT_FOUND; // RETURN + } + + d_virtualStorageCatalog.remove(msgGUID); const RecordHandlesArray& handles = it->second.d_array; BSLS_ASSERT_SAFE(!handles.empty()); @@ -510,8 +520,8 @@ FileBackedStorage::remove(const bmqt::MessageGUID& msgGUID, // Update stats d_capacityMeter.remove(1, msgLen); - BSLS_ASSERT_SAFE(d_queue_p); - d_queue_p->stats()->onEvent( + BSLS_ASSERT_SAFE(queue()); + queue()->stats()->onEvent( mqbstat::QueueStatsDomain::EventType::e_DEL_MESSAGE, msgLen); @@ -553,99 +563,15 @@ FileBackedStorage::removeAll(const mqbu::StorageKey& appKey) purgeCommon(appKey); // or 'mqbu::StorageKey::k_NULL_KEY' dispatcherFlush(true, false); d_isEmpty.storeRelaxed(1); + return mqbi::StorageResult::e_SUCCESS; // RETURN } // A specific appKey is being purged. - bslma::ManagedPtr iter = - d_virtualStorageCatalog.getIterator(appKey); - while (!iter->atEnd()) { - const bmqt::MessageGUID& guid = iter->guid(); - RecordHandleMapIter it = d_handles.find(guid); - if (it == d_handles.end()) { - BALL_LOG_WARN - << "#STORAGE_PURGE_ERROR " << "Partition [" << partitionId() - << "]" << ": Attempting to purge GUID '" << guid - << "' from virtual storage with appId '" << appId - << "' & appKey '" << appKey << "' for queue '" << queueUri() - << "' & queueKey '" << queueKey() - << "', but GUID does not exist in the underlying storage."; - iter->advance(); - continue; // CONTINUE - } - - if (0 == it->second.d_refCount) { - // Outstanding refCount for this message is already zero. - - MWCTSK_ALARMLOG_ALARM("REPLICATION") - << "Partition [" << partitionId() << "]" - << ": Attempting to purge GUID '" << guid - << "' from virtual storage with appId '" << appId - << "' & appKey '" << appKey << "] for queue '" << queueUri() - << "' & queueKey '" << queueKey() - << "', for which refCount is already zero." - << MWCTSK_ALARMLOG_END; - iter->advance(); - continue; // CONTINUE - } - - const RecordHandlesArray& handles = it->second.d_array; - BSLS_ASSERT_SAFE(!handles.empty()); - - if (0 == --it->second.d_refCount) { - // This appKey was the last outstanding client for this message. - // Message can now be deleted. - - int msgLen = static_cast( - d_store_p->getMessageLenRaw(handles[0])); - - rc = d_store_p->writeDeletionRecord( - guid, - d_queueKey, - DeletionRecordFlag::e_NONE, - bdlt::EpochUtil::convertToTimeT64(bdlt::CurrentTime::utc())); - - if (0 != rc) { - MWCTSK_ALARMLOG_ALARM("FILE_IO") - << "Partition [" << partitionId() << "] failed to write " - << "DELETION record for GUID: " << guid << ", for queue '" - << d_queueUri << "', queueKey '" << d_queueKey - << "' while attempting to purge the message, rc: " << rc - << MWCTSK_ALARMLOG_END; - iter->advance(); - continue; // CONTINUE - } - - // If a queue is associated, inform it about the message being - // deleted, and update queue stats. - // The same 'e_DEL_MESSAGE' is about 3 cases: TTL, no SC quorum, - // and a purge. - if (d_queue_p) { - d_queue_p->queueEngine()->beforeMessageRemoved(guid); - d_queue_p->stats()->onEvent( - mqbstat::QueueStatsDomain::EventType::e_DEL_MESSAGE, - msgLen); - } - - // There is not really a need to remove the guid from all virtual - // storages, because we can be here only if guid doesn't exist in - // any virtual storage apart from 'vs' (because updated outstanding - // refCount is zero). So we just delete records associated with - // the guid from the underlying (this) storage. - - for (unsigned int i = 0; i < handles.size(); ++i) { - d_store_p->removeRecordRaw(handles[i]); - } - - d_capacityMeter.remove(1, msgLen); - d_handles.erase(it); - } - - iter->advance(); - } + d_virtualStorageCatalog.removeAll(appKey); + // This will call back 'releaseRef' - purgeCommon(appKey); dispatcherFlush(true, false); if (d_handles.empty()) { @@ -677,9 +603,9 @@ int FileBackedStorage::gcExpiredMessages( bsls::Types::Int64 now = mwcsys::Time::highResolutionTimer(); int limit = k_GC_MESSAGES_BATCH_SIZE; bsls::Types::Int64 deduplicationTimeNs = - d_queue_p ? d_queue_p->domain()->config().deduplicationTimeMs() * - bdlt::TimeUnitRatio::k_NANOSECONDS_PER_MILLISECOND - : 0; + queue() ? queue()->domain()->config().deduplicationTimeMs() * + bdlt::TimeUnitRatio::k_NANOSECONDS_PER_MILLISECOND + : 0; for (RecordHandleMapIter next = d_handles.begin(), cit; next != d_handles.end() && --limit;) { @@ -735,16 +661,15 @@ int FileBackedStorage::gcExpiredMessages( // and update queue stats. // The same 'e_DEL_MESSAGE' is about 3 cases: TTL, no SC quorum, purge. - if (d_queue_p) { - d_queue_p->queueEngine()->beforeMessageRemoved(cit->first); - d_queue_p->stats()->onEvent( + if (queue()) { + queue()->queueEngine()->beforeMessageRemoved(cit->first); + queue()->stats()->onEvent( mqbstat::QueueStatsDomain::EventType::e_DEL_MESSAGE, msgLen); } // Remove message from all virtual storages. - d_virtualStorageCatalog.remove(cit->first, - mqbu::StorageKey::k_NULL_KEY); + d_virtualStorageCatalog.gc(cit->first); // Delete all items pointed by all handles for this GUID (i.e., delete // message from the underlying storage). @@ -758,14 +683,14 @@ int FileBackedStorage::gcExpiredMessages( ++numMsgsDeleted; } - if (d_queue_p && numMsgsDeleted > 0) { + if (queue() && numMsgsDeleted > 0) { if (numMsgsDeleted > numMsgsUnreceipted) { - d_queue_p->stats()->onEvent( + queue()->stats()->onEvent( mqbstat::QueueStatsDomain::EventType::e_GC_MESSAGE, numMsgsDeleted - numMsgsUnreceipted); } if (numMsgsUnreceipted) { - d_queue_p->stats()->onEvent( + queue()->stats()->onEvent( mqbstat::QueueStatsDomain::EventType::e_NO_SC_MESSAGE, numMsgsUnreceipted); } @@ -800,33 +725,42 @@ void FileBackedStorage::processMessageRecord( irc.first->second.d_array.push_back(handle); irc.first->second.d_refCount = refCount; - // Add 'guid' to all virtual storages, if any. - d_virtualStorageCatalog.put(guid, - msgLen, - d_defaultRdaInfo, - bmqp::Protocol::k_DEFAULT_SUBSCRIPTION_ID, - mqbu::StorageKey::k_NULL_KEY); - - if (!d_currentlyAutoConfirming.isUnset()) { - if (d_currentlyAutoConfirming == guid) { - // Move auto confirms to the data record - for (unsigned int i = 0; i < d_ephemeralConfirms.size(); ++i) { - irc.first->second.d_array.push_back( - d_ephemeralConfirms[i]); + if (d_autoConfirms.empty()) { + d_virtualStorageCatalog.put(guid, msgLen); + } + else { + if (!d_currentlyAutoConfirming.isUnset()) { + if (d_currentlyAutoConfirming == guid) { + VirtualStorage::DataStreamMessage* dataStreamMessage = 0; + d_virtualStorageCatalog.put(guid, + msgLen, + &dataStreamMessage); + + // Move auto confirms to the data record + for (AutoConfirms::const_iterator it = + d_autoConfirms.begin(); + it != d_autoConfirms.end(); + ++it) { + irc.first->second.d_array.push_back( + it->d_confirmRecordHandle); + d_virtualStorageCatalog.autoConfirm(dataStreamMessage, + it->d_appKey); + } + } + else { + clearSelection(); } - d_ephemeralConfirms.clear(); d_currentlyAutoConfirming = bmqt::MessageGUID(); } - else { - clearSelection(); - } + d_autoConfirms.clear(); } + d_currentlyAutoConfirming = bmqt::MessageGUID(); // Update the messages & bytes monitors, and the stats. d_capacityMeter.forceCommit(1, msgLen); // Return value ignored. - if (d_queue_p) { - d_queue_p->stats()->onEvent( + if (queue()) { + queue()->stats()->onEvent( mqbstat::QueueStatsDomain::EventType::e_ADD_MESSAGE, msgLen); } @@ -862,8 +796,7 @@ void FileBackedStorage::processConfirmRecord( d_currentlyAutoConfirming = guid; } - d_virtualStorageCatalog.autoConfirm(d_currentlyAutoConfirming, appKey); - d_ephemeralConfirms.push_back(handle); + d_autoConfirms.emplace_back(appKey, handle); return; // RETURN } @@ -897,8 +830,8 @@ void FileBackedStorage::processConfirmRecord( --it->second.d_refCount; // Update outstanding refCount if (!appKey.isNull()) { - mqbi::StorageResult::Enum rc = d_virtualStorageCatalog.remove(guid, - appKey); + mqbi::StorageResult::Enum rc = d_virtualStorageCatalog.confirm(guid, + appKey); if (mqbi::StorageResult::e_SUCCESS != rc) { BALL_LOG_ERROR << "#STORAGE_INVALID_CONFIRM " << "Partition [" << partitionId() << "]" @@ -941,9 +874,9 @@ void FileBackedStorage::processDeletionRecord(const bmqt::MessageGUID& guid) const RecordHandlesArray& handles = it->second.d_array; const unsigned int msgLen = d_store_p->getMessageLenRaw(handles[0]); - if (d_queue_p) { - d_queue_p->queueEngine()->beforeMessageRemoved(guid); - d_queue_p->stats()->onEvent( + if (queue()) { + queue()->queueEngine()->beforeMessageRemoved(guid); + queue()->stats()->onEvent( mqbstat::QueueStatsDomain::EventType::e_DEL_MESSAGE, msgLen); } @@ -953,7 +886,7 @@ void FileBackedStorage::processDeletionRecord(const bmqt::MessageGUID& guid) // records were received earlier for each appKey, but we remove the guid // again, just in case. When the code is mature enough, we could remove // this. - d_virtualStorageCatalog.remove(guid, mqbu::StorageKey::k_NULL_KEY); + d_virtualStorageCatalog.remove(guid); d_capacityMeter.remove(1, msgLen, true /* silent mode; don't log */); @@ -993,7 +926,7 @@ void FileBackedStorage::purge(const mqbu::StorageKey& appKey) { purgeCommon(appKey); - if (d_queue_p) { + if (queue()) { bsl::string appId; if (appKey.isNull()) { appId = bmqp::ProtocolUtil::k_NULL_APP_ID; @@ -1005,7 +938,7 @@ void FileBackedStorage::purge(const mqbu::StorageKey& appKey) static_cast(rc); } - d_queue_p->queueEngine()->afterQueuePurged(appId, appKey); + queue()->queueEngine()->afterQueuePurged(appId, appKey); } } @@ -1033,139 +966,22 @@ FileBackedStorage::autoConfirm(const mqbu::StorageKey& appKey, if (0 != rc) { return mqbi::StorageResult::e_WRITE_FAILURE; // RETURN } - d_ephemeralConfirms.push_back(handle); - d_virtualStorageCatalog.autoConfirm(d_currentlyAutoConfirming, appKey); + d_autoConfirms.emplace_back(appKey, handle); return mqbi::StorageResult::e_SUCCESS; } void FileBackedStorage::clearSelection() { - for (unsigned int i = 0; i < d_ephemeralConfirms.size(); ++i) { - d_store_p->removeRecordRaw(d_ephemeralConfirms[i]); + for (AutoConfirms::const_iterator it = d_autoConfirms.begin(); + it != d_autoConfirms.end(); + ++it) { + d_store_p->removeRecordRaw(it->d_confirmRecordHandle); } - d_ephemeralConfirms.clear(); + d_autoConfirms.clear(); d_currentlyAutoConfirming = bmqt::MessageGUID(); } -// ------------------------------- -// class FileBackedStorageIterator -// ------------------------------- - -// PRIVATE MANIPULATORS -void FileBackedStorageIterator::clear() -{ - // Clear previous state, if any. This is required so that new state can be - // loaded in 'appData', 'options' or 'attributes' routines. - d_appData_sp.reset(); - d_options_sp.reset(); - d_attributes.reset(); -} - -// PRIVATE ACCESSORS -void FileBackedStorageIterator::loadMessageAndAttributes() const -{ - BSLS_ASSERT_SAFE(!atEnd()); - if (!d_appData_sp) { - const RecordHandlesArray& array = d_iterator->second.d_array; - BSLS_ASSERT_SAFE(!array.empty()); - d_storage_p->d_store_p->loadMessageRaw(&d_appData_sp, - &d_options_sp, - &d_attributes, - array[0]); - } -} - -// CREATORS -FileBackedStorageIterator::FileBackedStorageIterator() -: d_storage_p(0) -, d_iterator() -, d_attributes() -, d_appData_sp() -, d_options_sp() -{ - // NOTHING -} - -FileBackedStorageIterator::FileBackedStorageIterator( - const FileBackedStorage* storage, - const RecordHandleMapConstIter& initialPosition) -: d_storage_p(storage) -, d_iterator(initialPosition) -, d_attributes() -{ -} - -FileBackedStorageIterator::~FileBackedStorageIterator() -{ - // NOTHING -} - -// MANIPULATORS -bool FileBackedStorageIterator::advance() -{ - // PRECONDITIONS - BSLS_ASSERT_SAFE(!atEnd()); - - clear(); - ++d_iterator; - return !atEnd(); -} - -void FileBackedStorageIterator::reset() -{ - clear(); - - // Reset iterator to beginning. - d_iterator = d_storage_p->d_handles.begin(); -} - -// ACCESSORS -const bmqt::MessageGUID& FileBackedStorageIterator::guid() const -{ - return d_iterator->first; -} - -bmqp::RdaInfo& FileBackedStorageIterator::rdaInfo() const -{ - static bmqp::RdaInfo dummy; - return dummy; -} - -unsigned int FileBackedStorageIterator::subscriptionId() const -{ - return bmqp::Protocol::k_DEFAULT_SUBSCRIPTION_ID; -} - -const bsl::shared_ptr& FileBackedStorageIterator::appData() const -{ - loadMessageAndAttributes(); - return d_appData_sp; -} - -const bsl::shared_ptr& FileBackedStorageIterator::options() const -{ - loadMessageAndAttributes(); - return d_options_sp; -} - -const mqbi::StorageMessageAttributes& -FileBackedStorageIterator::attributes() const -{ - loadMessageAndAttributes(); - return d_attributes; -} - -bool FileBackedStorageIterator::atEnd() const -{ - return d_iterator == d_storage_p->d_handles.end(); -} - -bool FileBackedStorageIterator::hasReceipt() const -{ - return atEnd() ? false : d_iterator->second.d_array[0].hasReceipt(); -} - } // close package namespace } // close enterprise namespace diff --git a/src/groups/mqb/mqbs/mqbs_filebackedstorage.h b/src/groups/mqb/mqbs/mqbs_filebackedstorage.h index 5e7b70648..8c37e08ec 100644 --- a/src/groups/mqb/mqbs/mqbs_filebackedstorage.h +++ b/src/groups/mqb/mqbs/mqbs_filebackedstorage.h @@ -79,9 +79,6 @@ class CountingAllocatorStore; namespace mqbs { -// FORWARD DECLARATION -class FileBackedStorageIterator; - // ======================= // class FileBackedStorage // ======================= @@ -113,6 +110,22 @@ class FileBackedStorage BSLS_KEYWORD_FINAL : public ReplicatedStorage { void reset(); }; + struct AutoConfirm { + // Transient state tracking auto-confirm status for the current + // message being replicated or put. + + const mqbu::StorageKey d_appKey; + const DataStoreRecordHandle d_confirmRecordHandle; + + AutoConfirm(const mqbu::StorageKey& appKey, + const DataStoreRecordHandle& confirmRecordHandle); + + const mqbu::StorageKey& appKey(); + const DataStoreRecordHandle& confirmRecordHandle(); + }; + + typedef bsl::list AutoConfirms; + public: // TYPES typedef mqbi::Storage::AppIdKeyPair AppIdKeyPair; @@ -148,10 +161,6 @@ class FileBackedStorage BSLS_KEYWORD_FINAL : public ReplicatedStorage { DataStore* d_store_p; - mqbi::Queue* d_queue_p; - // This could be null if a local or remote - // queue instance has not been created. - mqbu::StorageKey d_queueKey; mqbconfm::Storage d_config; @@ -183,23 +192,10 @@ class FileBackedStorage BSLS_KEYWORD_FINAL : public ReplicatedStorage { // records of type 'ADDITION' could be // present only if queue is in fanout mode. - bsl::string d_emptyAppId; - // This field is unused, but needs to be a - // member variable so that 'appId()' - // routine can return a ref. - - mqbu::StorageKey d_nullAppKey; - // This field is unused, but needs to be a - // member variable so that 'appKey()' - // routine can return a ref. - bsls::AtomicInt d_isEmpty; // Flag indicating if storage is empty. // This flag can be checked from any - // thread. - - bmqp::RdaInfo d_defaultRdaInfo; - // Use in all 'put' operations. + // thread.. bmqp::SchemaLearner::Context d_schemaLearnerContext; // Context for replicated data. @@ -209,7 +205,7 @@ class FileBackedStorage BSLS_KEYWORD_FINAL : public ReplicatedStorage { bmqt::MessageGUID d_currentlyAutoConfirming; // Message being evaluated and possibly auto confirmed. - RecordHandlesArray d_ephemeralConfirms; + AutoConfirms d_autoConfirms; // Auto CONFIRMs waiting for 'put' or 'processMessageRecord' private: @@ -241,7 +237,6 @@ class FileBackedStorage BSLS_KEYWORD_FINAL : public ReplicatedStorage { const mqbu::StorageKey& queueKey, const mqbconfm::Domain& config, mqbu::CapacityMeter* parentCapacityMeter, - const bmqp::RdaInfo& defaultRdaInfo, bslma::Allocator* allocator, mwcma::CountingAllocatorStore* allocatorStore = 0); @@ -257,20 +252,16 @@ class FileBackedStorage BSLS_KEYWORD_FINAL : public ReplicatedStorage { /// otherwise return false. virtual bool isPersistent() const BSLS_KEYWORD_OVERRIDE; + /// Return the queue this storage is associated with. + /// Storage exists without a queue before `setQueue`. + virtual mqbi::Queue* queue() const BSLS_KEYWORD_OVERRIDE; + /// Return the URI of the queue this storage is associated with. virtual const bmqt::Uri& queueUri() const BSLS_KEYWORD_OVERRIDE; /// Return the queueKey associated with this storage instance. virtual const mqbu::StorageKey& queueKey() const BSLS_KEYWORD_OVERRIDE; - /// Return the appId associated with this storage instance. If there is - /// not appId associated, return an empty string. - virtual const bsl::string& appId() const BSLS_KEYWORD_OVERRIDE; - - /// Return the app key, if any, associated with this storage instance. - /// If there is no appKey associated, return a null key. - virtual const mqbu::StorageKey& appKey() const BSLS_KEYWORD_OVERRIDE; - /// Return the number of messages in the virtual storage associated with /// the specified `appKey`. If `appKey` is null, number of messages in /// the `physical` storage is returned. Behavior is undefined if @@ -316,39 +307,40 @@ class FileBackedStorage BSLS_KEYWORD_FINAL : public ReplicatedStorage { /// Return the number of virtual storages registered with this instance. virtual int numVirtualStorages() const BSLS_KEYWORD_OVERRIDE; + /// Return true if virtual storage identified by the specified 'appKey' + /// exists, otherwise return false. Load into the optionally specified + /// 'appId' the appId associated with 'appKey' if the virtual storage + /// exists, otherwise set it to 0. virtual bool hasVirtualStorage(const mqbu::StorageKey& appKey, bsl::string* appId = 0) const BSLS_KEYWORD_OVERRIDE; - // Return true if virtual storage identified by the specified 'appKey' - // exists, otherwise return false. Load into the optionally specified - // 'appId' the appId associated with 'appKey' if the virtual storage - // exists, otherwise set it to 0. /// Return `true` if there was Replication Receipt for the specified /// `msgGUID`. virtual bool hasReceipt(const bmqt::MessageGUID& msgGUID) const BSLS_KEYWORD_OVERRIDE; - virtual bool hasVirtualStorage(const bsl::string& appId, - mqbu::StorageKey* appKey = 0) const - BSLS_KEYWORD_OVERRIDE; - // Return true if virtual storage identified by the specified 'appId' - // exists, otherwise return false. Load into the optionally specified - // 'appKey' the appKey associated with 'appId' if the virtual storage - // exists, otherwise set it to 0. + /// Return true if virtual storage identified by the specified 'appId' + /// exists, otherwise return false. Load into the optionally specified + /// 'appKey' and 'ordinal' the appKey and ordinal associated with 'appId' + /// if the virtual storage exists, otherwise set it to 0. + virtual bool + hasVirtualStorage(const bsl::string& appId, + mqbu::StorageKey* appKey = 0, + unsigned int* ordinal = 0) const BSLS_KEYWORD_OVERRIDE; + /// Load into the specified 'buffer' the list of pairs of appId and appKey + // for all the virtual storages registered with this instance. virtual void loadVirtualStorageDetails(AppIdKeyPairs* buffer) const BSLS_KEYWORD_OVERRIDE; - // Load into the specified 'buffer' the list of pairs of appId and - // appKey for all the virtual storages registered with this instance. + /// Store in the specified 'msgSize' the size, in bytes, of the message + /// having the specified 'msgGUID' if found and return success, or return + /// a non-zero return code and leave 'msgSize' untouched if no message for + /// the 'msgGUID' was found. virtual mqbi::StorageResult::Enum getMessageSize( int* msgSize, const bmqt::MessageGUID& msgGUID) const BSLS_KEYWORD_OVERRIDE; - // Store in the specified 'msgSize' the size, in bytes, of the message - // having the specified 'msgGUID' if found and return success, or - // return a non-zero return code and leave 'msgSize' untouched if no - // message with 'msgGUID' were found. // MANIPULATORS @@ -362,84 +354,106 @@ class FileBackedStorage BSLS_KEYWORD_FINAL : public ReplicatedStorage { const mqbconfm::Storage& config, const mqbconfm::Limits& limits, const bsls::Types::Int64 messageTtl, - const int maxDeliveryAttempts) BSLS_KEYWORD_OVERRIDE; + int maxDeliveryAttempts) BSLS_KEYWORD_OVERRIDE; /// Return the resource capacity meter associated to this storage. virtual mqbu::CapacityMeter* capacityMeter() BSLS_KEYWORD_OVERRIDE; virtual void setQueue(mqbi::Queue* queue) BSLS_KEYWORD_OVERRIDE; - virtual mqbi::Queue* queue() BSLS_KEYWORD_OVERRIDE; - /// Close this storage. virtual void close() BSLS_KEYWORD_OVERRIDE; - /// Save the message contained in the specified `appData`, `options` and - /// the associated `attributes` and `msgGUID` into this storage and the - /// associated virtual storages, if any. The `attributes` is an in/out - /// parameter and storage layer can populate certain fields of that - /// struct. Return 0 on success or an non-zero error code on failure. + /// Save the message contained in the specified 'appData', 'options' and + /// the associated 'attributes' and 'msgGUID' into this storage and the + /// associated virtual storage. The 'attributes' is an in/out parameter + /// and storage layer can populate certain fields of that struct. + /// Return 0 on success or an non-zero error code on failure. virtual mqbi::StorageResult::Enum put(mqbi::StorageMessageAttributes* attributes, const bmqt::MessageGUID& msgGUID, const bsl::shared_ptr& appData, - const bsl::shared_ptr& options, - const StorageKeys& storageKeys = StorageKeys()) BSLS_KEYWORD_OVERRIDE; - - /// Get an iterator for items stored in the virtual storage identified - /// by the specified `appKey`. Iterator will point to point to the - /// oldest item, if any, or to the end of the collection if empty. Note - /// that if `appKey` is null, an iterator over the underlying physical - /// storage will be returned. Also note that because `Storage` and - /// `StorageIterator` are interfaces, the implementation of this method - /// will allocate, so it's recommended to keep the iterator. + const bsl::shared_ptr& options) BSLS_KEYWORD_OVERRIDE; + + /// Get an iterator for data stored in the virtual storage identified by + /// the specified 'appKey'. + /// If the 'appKey' is null, the returned iterator can iterate states of + /// all Apps; otherwise, the iterator can iterate states of the App + /// corresponding to the 'appKey'. virtual bslma::ManagedPtr getIterator(const mqbu::StorageKey& appKey) BSLS_KEYWORD_OVERRIDE; - /// Load into the specified `out` an iterator for items stored in - /// the virtual storage identified by the specified `appKey`, initially - /// pointing to the item associated with the specified `msgGUID`. - /// Return zero on success, and a non-zero code if `msgGUID` was not - /// found in the storage. Note that if `appKey` is null, an iterator - /// over the underlying physical storage will be returned. Also note - /// that because `Storage` and `StorageIterator` are interfaces, the - /// implementation of this method will allocate, so it's recommended to - /// keep the iterator. + /// Load into the specified 'out' an iterator for data stored in the + /// virtual storage initially pointing to the message associated with the + /// specified 'msgGUID'. + /// If the 'appKey' is null, the returned iterator can iterate states of + /// all Apps; otherwise, the iterator can iterate states of the App + /// corresponding to the 'appKey'. + /// Return zero on success, and a non-zero code if 'msgGUID' was not + /// found in the storage. virtual mqbi::StorageResult::Enum getIterator(bslma::ManagedPtr* out, const mqbu::StorageKey& appKey, const bmqt::MessageGUID& msgGUID) BSLS_KEYWORD_OVERRIDE; - /// Release the reference of the specified `appKey` on the message - /// identified by the specified `msgGUID`, and record this event in the - /// storage. Return one of the return codes from: - /// * **e_GUID_NOT_FOUND** : `msgGUID` was not found - /// * **e_ZERO_REFERENCES** : message refCount has become zero - /// * **e_NON_ZERO_REFERENCES** : message refCount is still not zero - /// * **e_WRITE_FAILURE** : failed to record this event in storage + /// Update the App state corresponding to the specified 'msgGUID' and the + /// specified 'appKey' in the DataStream. Decrement the reference count of + /// the message identified by the 'msgGUID', and record the CONFIRM in the + /// storage. + /// Return one of the return codes from: + /// * e_SUCCESS : success + /// * e_GUID_NOT_FOUND : 'msgGUID' was not found + /// * e_ZERO_REFERENCES : message refCount has become zero + /// * e_NON_ZERO_REFERENCES : message refCount is still not zero + /// * e_WRITE_FAILURE : failed to record this event in storage + /// + /// Behavior is undefined unless there is an App with the 'appKey'. + /// + /// On CONFIRM, the caller of 'confirm' is responsible to follow with + /// 'remove' call. 'releaseRef' is an alternative way to remove message in + /// one call. + mqbi::StorageResult::Enum + confirm(const bmqt::MessageGUID& msgGUID, + const mqbu::StorageKey& appKey, + bsls::Types::Int64 timestamp, + bool onReject = false) BSLS_KEYWORD_OVERRIDE; + + /// Decrement the reference count of the message identified by the + /// 'msgGUID'. If the resulting value is zero, delete the message data and + /// record the event in the storage. + /// Return one of the return codes from: + /// * e_SUCCESS : success + /// * e_GUID_NOT_FOUND : 'msgGUID' was not found + /// * e_INVALID_OPERATION : the value is invalid (already zero) + /// * e_ZERO_REFERENCES : message refCount has become zero + /// * e_NON_ZERO_REFERENCE : message refCount is still not zero + /// + /// On CONFIRM, the caller of 'confirm' is responsible to follow with + /// 'remove' call. 'releaseRef' is an alternative way to remove message in + /// one call. mqbi::StorageResult::Enum - releaseRef(const bmqt::MessageGUID& msgGUID, - const mqbu::StorageKey& appKey, - bsls::Types::Int64 timestamp, - bool onReject = false) BSLS_KEYWORD_OVERRIDE; - - /// Remove from the storage the message having the specified `msgGUID` - /// and store it's size, in bytes, in the optionally specified `msgSize` - /// if the `msgGUID` was found. Return 0 on success, or a non-zero - /// return code if the `msgGUID` was not found. If the optionally - /// specified `clearAll` is true, remove the message from all virtual - /// storages as well. + releaseRef(const bmqt::MessageGUID& msgGUID) BSLS_KEYWORD_OVERRIDE; + + /// Remove from the storage the message having the specified 'msgGUID' + /// and store it's size, in bytes, in the optionally specified 'msgSize'. + /// Record the event in the storage. + /// Return 0 on success, or a non-zero return code if the 'msgGUID' was not + /// found or if has failed to record this event in storage. + /// + /// On CONFIRM, the caller of 'confirm' is responsible to follow with + /// 'remove' call. 'releaseRef' is an alternative way to remove message in + /// one call. virtual mqbi::StorageResult::Enum remove(const bmqt::MessageGUID& msgGUID, - int* msgSize = 0, - bool clearAll = false) BSLS_KEYWORD_OVERRIDE; - - /// Remove all messages from this storage for the client identified by - /// the specified `appKey`. If `appKey` is null, then remove messages - /// for all clients. Return one of the return codes from: - /// * **e_SUCCESS** : `msgGUID` was not found - /// * **e_WRITE_FAILURE** : failed to record this event in storage - /// * **e_APPKEY_NOT_FOUND** : Invalid `appKey` specified + int* msgSize = 0) BSLS_KEYWORD_OVERRIDE; + + /// Remove all messages from this storage for the App identified by the + /// specified 'appKey' if 'appKey' is not null. Otherwise, remove messages + /// for all Apps. Record the event in the storage. + /// Return one of the return codes from: + /// * e_SUCCESS : success + /// * e_WRITE_FAILURE : failed to record this event in storage + /// * e_APPKEY_NOT_FOUND : Invalid 'appKey' specified virtual mqbi::StorageResult::Enum removeAll(const mqbu::StorageKey& appKey) BSLS_KEYWORD_OVERRIDE; @@ -530,115 +544,6 @@ class FileBackedStorage BSLS_KEYWORD_FINAL : public ReplicatedStorage { virtual unsigned int numAutoConfirms() const BSLS_KEYWORD_OVERRIDE; }; -// =============================== -// class FileBackedStorageIterator -// =============================== - -/// TBD: -class FileBackedStorageIterator : public mqbi::StorageIterator { - private: - // PRIVATE TYPES - typedef FileBackedStorage::RecordHandlesArray RecordHandlesArray; - - typedef FileBackedStorage::RecordHandleMap RecordHandleMap; - - typedef FileBackedStorage::RecordHandleMapConstIter - RecordHandleMapConstIter; - - private: - // DATA - const FileBackedStorage* d_storage_p; - - RecordHandleMapConstIter d_iterator; - - mutable mqbi::StorageMessageAttributes d_attributes; - - mutable bsl::shared_ptr d_appData_sp; - // If this variable is empty, it is - // assumed that attributes and message - // have not been loaded in this - // iteration (see also - // 'loadMessageAndAttributes' impl). - - mutable bsl::shared_ptr d_options_sp; - - private: - // PRIVATE MANIPULATORS - void clear(); - - // PRIVATE ACCESSORS - void loadMessageAndAttributes() const; - - public: - // CREATORS - - /// Create an invalid iterator. `atEnd()` will return false. Only valid - /// operations are `reset` and destruction. - FileBackedStorageIterator(); - - /// Create an iterator instance over the specified `storage` and - /// initially pointing to `initialPosition`. - FileBackedStorageIterator(const FileBackedStorage* storage, - const RecordHandleMapConstIter& initialPosition); - - /// Destroy this object. - ~FileBackedStorageIterator() BSLS_KEYWORD_OVERRIDE; - - // MANIPULATORS - - /// Advance the iterator to the next item. The behavior is undefined - /// unless `atEnd` returns `false`. Return `true` if the iterator then - /// points to a valid item, or `false` if it now is at the end of the - /// items' collection. - bool advance() BSLS_KEYWORD_OVERRIDE; - - /// Reset the iterator to point to first item, if any, in the underlying - /// storage. - void reset() BSLS_KEYWORD_OVERRIDE; - - // ACCESSORS - - /// Return a reference offering non-modifiable access to the guid - /// associated to the item currently pointed at by this iterator. The - /// behavior is undefined unless `atEnd` returns `false`. - const bmqt::MessageGUID& guid() const BSLS_KEYWORD_OVERRIDE; - - /// Return a reference offering modifiable access to the RdaInfo - /// associated to the item currently pointed at by this iterator. The - /// behavior is undefined unless `atEnd` returns `false`. - bmqp::RdaInfo& rdaInfo() const BSLS_KEYWORD_OVERRIDE; - - /// Return subscription id associated to the item currently pointed at - /// by this iterator. - /// The behavior is undefined unless `atEnd` returns `false`. - unsigned int subscriptionId() const BSLS_KEYWORD_OVERRIDE; - - /// Return a reference offering non-modifiable access to the attributes - /// associated with the message currently pointed at by this iterator. - /// The behavior is undefined unless `atEnd` returns `false`. - const mqbi::StorageMessageAttributes& - attributes() const BSLS_KEYWORD_OVERRIDE; - - /// Return a reference offering non-modifiable access to the application - /// data associated with the item currently pointed at by this iterator. - /// The behavior is undefined unless `atEnd` returns `false`. - const bsl::shared_ptr& appData() const BSLS_KEYWORD_OVERRIDE; - - /// Return a reference offering non-modifiable access to the options - /// associated with the item currently pointed at by this iterator. The - /// behavior is undefined unless `atEnd` returns `false`. - const bsl::shared_ptr& options() const BSLS_KEYWORD_OVERRIDE; - - /// Return `true` if this iterator is currently at the end of the items' - /// collection, and hence doesn't reference a valid item. - bool atEnd() const BSLS_KEYWORD_OVERRIDE; - - /// Return `true` if this iterator is currently not at the end of the - /// `items` collection and the message currently pointed at by this - /// iterator has received replication factor Receipts. - bool hasReceipt() const BSLS_KEYWORD_OVERRIDE; -}; - // ============================================================================ // INLINE DEFINITIONS // ============================================================================ @@ -653,14 +558,34 @@ inline void FileBackedStorage::Item::reset() d_refCount = 0; } +inline FileBackedStorage::AutoConfirm::AutoConfirm( + const mqbu::StorageKey& appKey, + const DataStoreRecordHandle& confirmRecordHandle) +: d_appKey(appKey) +, d_confirmRecordHandle(confirmRecordHandle) +{ + // NOTHING +} + +inline const mqbu::StorageKey& FileBackedStorage::AutoConfirm::appKey() +{ + return d_appKey; +} + +inline const DataStoreRecordHandle& +FileBackedStorage::AutoConfirm::confirmRecordHandle() +{ + return d_confirmRecordHandle; +} + // ----------------- // FileBackedStorage // ----------------- // MANIPULATORS -inline mqbi::Queue* FileBackedStorage::queue() +inline mqbi::Queue* FileBackedStorage::queue() const { - return d_queue_p; + return d_virtualStorageCatalog.queue(); } inline int FileBackedStorage::addVirtualStorage(bsl::ostream& errorDescription, @@ -706,16 +631,6 @@ inline const mqbu::StorageKey& FileBackedStorage::queueKey() const return d_queueKey; } -inline const bsl::string& FileBackedStorage::appId() const -{ - return d_emptyAppId; -} - -inline const mqbu::StorageKey& FileBackedStorage::appKey() const -{ - return d_nullAppKey; -} - inline bsls::Types::Int64 FileBackedStorage::numMessages(const mqbu::StorageKey& appKey) const { @@ -794,11 +709,11 @@ FileBackedStorage::hasVirtualStorage(const mqbu::StorageKey& appKey, return d_virtualStorageCatalog.hasVirtualStorage(appKey, appId); } -inline bool -FileBackedStorage::hasVirtualStorage(const bsl::string& appId, - mqbu::StorageKey* appKey) const +inline bool FileBackedStorage::hasVirtualStorage(const bsl::string& appId, + mqbu::StorageKey* appKey, + unsigned int* ordinal) const { - return d_virtualStorageCatalog.hasVirtualStorage(appId, appKey); + return d_virtualStorageCatalog.hasVirtualStorage(appId, appKey, ordinal); } inline void @@ -809,7 +724,7 @@ FileBackedStorage::loadVirtualStorageDetails(AppIdKeyPairs* buffer) const inline unsigned int FileBackedStorage::numAutoConfirms() const { - return d_ephemeralConfirms.size(); + return d_autoConfirms.size(); } } // close package namespace diff --git a/src/groups/mqb/mqbs/mqbs_filestore.cpp b/src/groups/mqb/mqbs/mqbs_filestore.cpp index 906008dee..6b873ee4a 100644 --- a/src/groups/mqb/mqbs/mqbs_filestore.cpp +++ b/src/groups/mqb/mqbs/mqbs_filestore.cpp @@ -5356,16 +5356,6 @@ void FileStore::createStorage(bsl::shared_ptr* storageSp, BSLS_ASSERT_SAFE(!storageCfg.isUndefinedValue()); - bmqp::RdaInfo rdaInfo; - if (domain->config().maxDeliveryAttempts()) { - rdaInfo.setCounter(domain->config().maxDeliveryAttempts()); - } - else { - // For Initialization of RdaInfo, value of 0 for config - // maxDeliveryAttempts symbolizes Infinity so we dont set the counter - // as constructor for rdaInfo initializes counter with Unlimited. - } - bslma::Allocator* storageAlloc = d_storageAllocatorStore.baseAllocator(); if (storageCfg.isInMemoryValue()) { storageSp->reset(new (*storageAlloc) @@ -5374,7 +5364,6 @@ void FileStore::createStorage(bsl::shared_ptr* storageSp, config().partitionId(), domain->config(), domain->capacityMeter(), - rdaInfo, storageAlloc, &d_storageAllocatorStore), storageAlloc); @@ -5386,7 +5375,6 @@ void FileStore::createStorage(bsl::shared_ptr* storageSp, queueKey, domain->config(), domain->capacityMeter(), - rdaInfo, storageAlloc, &d_storageAllocatorStore), storageAlloc); diff --git a/src/groups/mqb/mqbs/mqbs_inmemorystorage.cpp b/src/groups/mqb/mqbs/mqbs_inmemorystorage.cpp index 896ea87a6..75b3f6a57 100644 --- a/src/groups/mqb/mqbs/mqbs_inmemorystorage.cpp +++ b/src/groups/mqb/mqbs/mqbs_inmemorystorage.cpp @@ -54,11 +54,9 @@ InMemoryStorage::InMemoryStorage(const bmqt::Uri& uri, int partitionId, const mqbconfm::Domain& config, mqbu::CapacityMeter* parentCapacityMeter, - const bmqp::RdaInfo& defaultRdaInfo, bslma::Allocator* allocator, mwcma::CountingAllocatorStore* allocatorStore) : d_allocator_p(allocator) -, d_queue_p(0) , d_key(queueKey) , d_uri(uri, allocator) , d_partitionId(partitionId) @@ -74,14 +72,13 @@ InMemoryStorage::InMemoryStorage(const bmqt::Uri& uri, this, allocatorStore ? allocatorStore->get("VirtualHandles") : d_allocator_p) , d_ttlSeconds(config.messageTtl()) -, d_emptyAppId(allocator) -, d_nullAppKey() , d_isEmpty(1) -, d_defaultRdaInfo(defaultRdaInfo) , d_currentlyAutoConfirming() -, d_numAutoConfirms(0) +, d_autoConfirms(allocator) { BSLS_ASSERT_SAFE(0 <= d_ttlSeconds); // Broadcast queues can use 0 for TTL + + d_virtualStorageCatalog.setDefaultRda(config.maxDeliveryAttempts()); } InMemoryStorage::~InMemoryStorage() @@ -96,7 +93,7 @@ int InMemoryStorage::configure( const mqbconfm::Storage& config, const mqbconfm::Limits& limits, const bsls::Types::Int64 messageTtl, - BSLS_ANNOTATION_UNUSED const int maxDeliveryAttempts) + const int maxDeliveryAttempts) { d_config = config; d_capacityMeter.setLimits(limits.messages(), limits.bytes()) @@ -104,22 +101,24 @@ int InMemoryStorage::configure( limits.bytesWatermarkRatio()); d_ttlSeconds = messageTtl; + d_virtualStorageCatalog.setDefaultRda(maxDeliveryAttempts); + return 0; } void InMemoryStorage::setQueue(mqbi::Queue* queue) { - d_queue_p = queue; + d_virtualStorageCatalog.setQueue(queue); // Update queue stats if a queue has been associated with the storage. - if (d_queue_p) { + if (queue) { const bsls::Types::Int64 numMessage = numMessages( mqbu::StorageKey::k_NULL_KEY); const bsls::Types::Int64 numByte = numBytes( mqbu::StorageKey::k_NULL_KEY); - d_queue_p->stats()->setQueueContentRaw(numMessage, numByte); + queue->stats()->setQueueContentRaw(numMessage, numByte); BALL_LOG_INFO << "Associated queue [" << queue->uri() << "] with key [" << queueKey() << "] and Partition [" @@ -139,15 +138,6 @@ void InMemoryStorage::close() bslma::ManagedPtr InMemoryStorage::getIterator(const mqbu::StorageKey& appKey) { - if (appKey.isNull()) { - bslma::ManagedPtr mp( - new (*d_allocator_p) - InMemoryStorageIterator(this, d_items.begin()), - d_allocator_p); - - return mp; // RETURN - } - return d_virtualStorageCatalog.getIterator(appKey); } @@ -156,18 +146,6 @@ InMemoryStorage::getIterator(bslma::ManagedPtr* out, const mqbu::StorageKey& appKey, const bmqt::MessageGUID& msgGUID) { - if (appKey.isNull()) { - ItemsMapConstIter it = d_items.find(msgGUID); - if (it == d_items.end()) { - return mqbi::StorageResult::e_GUID_NOT_FOUND; // RETURN - } - - out->load(new (*d_allocator_p) InMemoryStorageIterator(this, it), - d_allocator_p); - - return mqbi::StorageResult::e_SUCCESS; // RETURN - } - return d_virtualStorageCatalog.getIterator(out, appKey, msgGUID); } @@ -175,15 +153,19 @@ mqbi::StorageResult::Enum InMemoryStorage::put(mqbi::StorageMessageAttributes* attributes, const bmqt::MessageGUID& msgGUID, const bsl::shared_ptr& appData, - const bsl::shared_ptr& options, - const StorageKeys& storageKeys) + const bsl::shared_ptr& options) { const int msgSize = appData->length(); - if (storageKeys.empty()) { - // Store the specified message in the 'physical' as well as *all* - // virtual storages. + // Proxies are unaware of the number of apps unlike Replicas. + // The latter can check for duplicates. + // The former can receive more than one PUSH message for the same GUID for + // different apps. + // For example, PUT, one app connects, receives PUSH, another app connects, + // receives PUSH. + // For this reason, Proxies sum up all incoming apps in the refCount. + if (!isProxy()) { if (d_items.isInHistory(msgGUID)) { return mqbi::StorageResult::e_DUPLICATE; } @@ -205,17 +187,27 @@ InMemoryStorage::put(mqbi::StorageMessageAttributes* attributes, Item(appData, options, *attributes)), attributes->arrivalTimepoint()); - d_virtualStorageCatalog.put(msgGUID, - msgSize, - d_defaultRdaInfo, - bmqp::Protocol::k_DEFAULT_SUBSCRIPTION_ID, - mqbu::StorageKey::k_NULL_KEY); + if (d_autoConfirms.empty()) { + d_virtualStorageCatalog.put(msgGUID, msgSize); + } + else { + VirtualStorage::DataStreamMessage* dataStreamMessage = 0; + d_virtualStorageCatalog.put(msgGUID, msgSize, &dataStreamMessage); + + // Move auto confirms to the data record + for (AutoConfirms::const_iterator it = d_autoConfirms.begin(); + it != d_autoConfirms.end(); + ++it) { + d_virtualStorageCatalog.autoConfirm(dataStreamMessage, + it->d_appKey); + } + d_autoConfirms.clear(); + } d_currentlyAutoConfirming = bmqt::MessageGUID(); - d_numAutoConfirms = 0; - if (d_queue_p) { - d_queue_p->stats()->onEvent( + if (queue()) { + queue()->stats()->onEvent( mqbstat::QueueStatsDomain::EventType::e_ADD_MESSAGE, msgSize); } @@ -227,19 +219,6 @@ InMemoryStorage::put(mqbi::StorageMessageAttributes* attributes, // crashes, the primary may deliver again the same messages to us. return mqbi::StorageResult::e_SUCCESS; // RETURN } - // 'storageKeys' is not empty only when proxy receives PUSH. - // Auto confirming does no apply then. - - // Specific appKeys have been specified. Insert the guid in the - // corresponding virtual storages. - - for (size_t i = 0; i < storageKeys.size(); ++i) { - d_virtualStorageCatalog.put(msgGUID, - msgSize, - d_defaultRdaInfo, - bmqp::Protocol::k_DEFAULT_SUBSCRIPTION_ID, - storageKeys[i]); - } // If the guid also exists in the 'physical' storage, bump up its reference // count by appropriate value. Note that in-memory storage is used at the @@ -248,24 +227,28 @@ InMemoryStorage::put(mqbi::StorageMessageAttributes* attributes, ItemsMapIter it = d_items.find(msgGUID); if (it != d_items.end()) { - mqbi::StorageMessageAttributes& attribs = it->second.attributes(); - attribs.setRefCount(attribs.refCount() + - storageKeys.size()); // Bump up + mqbi::StorageMessageAttributes& existing = it->second.attributes(); + existing.setRefCount(existing.refCount() + + attributes->refCount()); // Bump up } else { d_items.insert(bsl::make_pair(msgGUID, Item(appData, options, *attributes)), attributes->arrivalTimepoint()); + d_virtualStorageCatalog.put(msgGUID, msgSize); } + // We don't verify uniqueness of the insertion because in the case of a + // proxy, it uses this inMemoryStorage, and when some upstream node + // crashes, the primary may deliver again the same messages to us. return mqbi::StorageResult::e_SUCCESS; // RETURN } -mqbi::StorageResult::Enum InMemoryStorage::releaseRef( - const bmqt::MessageGUID& msgGUID, - const mqbu::StorageKey& appKey, - BSLS_ANNOTATION_UNUSED bsls::Types::Int64 timestamp, - BSLS_ANNOTATION_UNUSED bool onReject) +mqbi::StorageResult::Enum +InMemoryStorage::confirm(const bmqt::MessageGUID& msgGUID, + const mqbu::StorageKey& appKey, + BSLS_ANNOTATION_UNUSED bsls::Types::Int64 timestamp, + BSLS_ANNOTATION_UNUSED bool onReject) { ItemsMapIter it = d_items.find(msgGUID); if (it == d_items.end()) { @@ -273,8 +256,8 @@ mqbi::StorageResult::Enum InMemoryStorage::releaseRef( } if (!appKey.isNull()) { - mqbi::StorageResult::Enum rc = d_virtualStorageCatalog.remove(msgGUID, - appKey); + mqbi::StorageResult::Enum rc = d_virtualStorageCatalog.confirm(msgGUID, + appKey); if (mqbi::StorageResult::e_SUCCESS != rc) { return rc; // RETURN } @@ -291,20 +274,57 @@ mqbi::StorageResult::Enum InMemoryStorage::releaseRef( } mqbi::StorageResult::Enum -InMemoryStorage::remove(const bmqt::MessageGUID& msgGUID, - int* msgSize, - bool clearAll) +InMemoryStorage::releaseRef(const bmqt::MessageGUID& guid) { - ItemsMapIter it = d_items.find(msgGUID); + ItemsMapIter it = d_items.find(guid); if (it == d_items.end()) { return mqbi::StorageResult::e_GUID_NOT_FOUND; // RETURN } - if (clearAll) { - d_virtualStorageCatalog.remove(msgGUID, mqbu::StorageKey::k_NULL_KEY); + unsigned int refCount = it->second.attributes().refCount(); + if (0 == refCount) { + // Outstanding refCount for this message is already zero. + return mqbi::StorageResult::e_INVALID_OPERATION; // RETURN + } + it->second.attributes().setRefCount(--refCount); + + if (0 == refCount) { + // This appKey was the last outstanding client for this message. + // Message can now be deleted. + + int msgLen = it->second.appData()->length(); + d_capacityMeter.remove(1, msgLen); + if (queue()) { + queue()->queueEngine()->beforeMessageRemoved(guid); + queue()->stats()->onEvent( + mqbstat::QueueStatsDomain::EventType::e_DEL_MESSAGE, + msgLen); + } + + // There is not really a need to remove the guid from all virtual + // storages, because we can be here only if guid doesn't exist in + // any virtual storage apart from the one associated with the + // specified 'appKey' (because updated outstanding refCount is + // zero). So we just delete the guid from the underlying (this) + // storage. + + d_items.erase(it); + + return mqbi::StorageResult::e_ZERO_REFERENCES; // RETURN + } + + return mqbi::StorageResult::e_NON_ZERO_REFERENCES; +} + +mqbi::StorageResult::Enum +InMemoryStorage::remove(const bmqt::MessageGUID& msgGUID, int* msgSize) +{ + ItemsMapIter it = d_items.find(msgGUID); + if (it == d_items.end()) { + return mqbi::StorageResult::e_GUID_NOT_FOUND; // RETURN } - BSLS_ASSERT_SAFE(!d_virtualStorageCatalog.hasMessage(msgGUID)); + d_virtualStorageCatalog.remove(msgGUID); int msgLen = it->second.appData()->length(); @@ -313,8 +333,8 @@ InMemoryStorage::remove(const bmqt::MessageGUID& msgGUID, // Update resource usage d_capacityMeter.remove(1, msgLen); - if (d_queue_p) { - d_queue_p->stats()->onEvent( + if (queue()) { + queue()->stats()->onEvent( mqbstat::QueueStatsDomain::EventType::e_DEL_MESSAGE, msgLen); } @@ -340,8 +360,8 @@ InMemoryStorage::removeAll(const mqbu::StorageKey& appKey) d_items.clear(); d_capacityMeter.clear(); - if (d_queue_p) { - d_queue_p->stats()->onEvent( + if (queue()) { + queue()->stats()->onEvent( mqbstat::QueueStatsDomain::EventType::e_PURGE, 0); } @@ -351,8 +371,8 @@ InMemoryStorage::removeAll(const mqbu::StorageKey& appKey) return mqbi::StorageResult::e_SUCCESS; // RETURN } - bsl::string appId; - if (!d_virtualStorageCatalog.hasVirtualStorage(appKey, &appId)) { + VirtualStorage* vs = d_virtualStorageCatalog.virtualStorage(appKey); + if (!vs) { return mqbi::StorageResult::e_APPKEY_NOT_FOUND; // RETURN } @@ -361,63 +381,6 @@ InMemoryStorage::removeAll(const mqbu::StorageKey& appKey) // refCount, and if updated refCount is zero, delete that msg from the // underlying (this) storage. - bslma::ManagedPtr iter = - d_virtualStorageCatalog.getIterator(appKey); - while (!iter->atEnd()) { - const bmqt::MessageGUID& guid = iter->guid(); - ItemsMapIter it = d_items.find(guid); - if (it == d_items.end()) { - BALL_LOG_WARN - << "#STORAGE_PURGE_ERROR " - << "Attempting to purge GUID '" << guid - << "' from virtual storage with appId '" << appId - << "' & appKey '" << appKey << "' for queue '" << queueUri() - << "' & queueKey '" << queueKey() - << "', but GUID does not exist in the underlying storage."; - iter->advance(); - continue; // CONTINUE - } - - unsigned int refCount = it->second.attributes().refCount(); - if (0 == refCount) { - // Outstanding refCount for this message is already zero. - BALL_LOG_WARN << "#STORAGE_PURGE_ERROR " - << "Attempting to purge GUID '" << guid - << "' from virtual storage with appId '" << appId - << "' & appKey '" << appKey << "' for queue '" - << queueUri() << "' & queueKey '" << queueKey() - << "], for which refCount is already zero."; - iter->advance(); - continue; // CONTINUE - } - it->second.attributes().setRefCount(--refCount); - - if (0 == refCount) { - // This appKey was the last outstanding client for this message. - // Message can now be deleted. - - int msgLen = it->second.appData()->length(); - d_capacityMeter.remove(1, msgLen); - if (d_queue_p) { - d_queue_p->queueEngine()->beforeMessageRemoved(guid); - d_queue_p->stats()->onEvent( - mqbstat::QueueStatsDomain::EventType::e_DEL_MESSAGE, - msgLen); - } - - // There is not really a need to remove the guid from all virtual - // storages, because we can be here only if guid doesn't exist in - // any virtual storage apart from the one associated with the - // specified 'appKey' (because updated outstanding refCount is - // zero). So we just delete the guid from the underlying (this) - // storage. - - d_items.erase(it); - } - - iter->advance(); - } - // Clear out the virtual storage associated with the specified 'appKey'. // Note that this cannot be done while iterating over the it in the above // 'while' loop for obvious reasons. @@ -461,23 +424,22 @@ int InMemoryStorage::gcExpiredMessages( int msgLen = cit->second.appData()->length(); d_capacityMeter.remove(1, msgLen); - if (d_queue_p) { - d_queue_p->queueEngine()->beforeMessageRemoved(cit->first); - d_queue_p->stats()->onEvent( + if (queue()) { + queue()->queueEngine()->beforeMessageRemoved(cit->first); + queue()->stats()->onEvent( mqbstat::QueueStatsDomain::EventType::e_DEL_MESSAGE, msgLen); } // Remove message from all virtual storages and the physical (this) // storage. - d_virtualStorageCatalog.remove(cit->first, - mqbu::StorageKey::k_NULL_KEY); + d_virtualStorageCatalog.remove(cit->first); d_items.erase(cit, now); ++numMsgsDeleted; } - if (d_queue_p && (numMsgsDeleted > 0)) { - d_queue_p->stats()->onEvent( + if (queue() && (numMsgsDeleted > 0)) { + queue()->stats()->onEvent( mqbstat::QueueStatsDomain::EventType::e_GC_MESSAGE, numMsgsDeleted); } @@ -497,7 +459,7 @@ bool InMemoryStorage::gcHistory() void InMemoryStorage::selectForAutoConfirming(const bmqt::MessageGUID& msgGUID) { - d_numAutoConfirms = 0; + d_autoConfirms.clear(); d_currentlyAutoConfirming = msgGUID; } @@ -506,9 +468,8 @@ InMemoryStorage::autoConfirm(const mqbu::StorageKey& appKey, bsls::Types::Uint64 timestamp) { (void)timestamp; - d_virtualStorageCatalog.autoConfirm(d_currentlyAutoConfirming, appKey); - ++d_numAutoConfirms; + d_autoConfirms.emplace_back(appKey); return mqbi::StorageResult::e_SUCCESS; } @@ -615,23 +576,5 @@ bool InMemoryStorage::isStrongConsistency() const return false; } -// ----------------------------- -// class InMemoryStorageIterator -// ----------------------------- - -InMemoryStorageIterator::InMemoryStorageIterator( - InMemoryStorage* storage, - const ItemsMapConstIter& initialPosition) -: d_storage_p(storage) -, d_iterator(initialPosition) -{ - // NOTHING -} - -InMemoryStorageIterator::~InMemoryStorageIterator() -{ - // NOTHING -} - } // close package namespace } // close enterprise namespace diff --git a/src/groups/mqb/mqbs/mqbs_inmemorystorage.h b/src/groups/mqb/mqbs/mqbs_inmemorystorage.h index d52d8c7de..e48ea3b21 100644 --- a/src/groups/mqb/mqbs/mqbs_inmemorystorage.h +++ b/src/groups/mqb/mqbs/mqbs_inmemorystorage.h @@ -148,6 +148,16 @@ class InMemoryStorage BSLS_KEYWORD_FINAL : public ReplicatedStorage { typedef mqbi::Storage::StorageKeys StorageKeys; + struct AutoConfirm { + const mqbu::StorageKey d_appKey; + + AutoConfirm(const mqbu::StorageKey& appKey); + + const mqbu::StorageKey& appKey(); + }; + + typedef bsl::list AutoConfirms; + public: // CLASS METHODS @@ -168,8 +178,6 @@ class InMemoryStorage BSLS_KEYWORD_FINAL : public ReplicatedStorage { // DATA bslma::Allocator* d_allocator_p; - mqbi::Queue* d_queue_p; - mqbu::StorageKey d_key; bmqt::Uri d_uri; @@ -204,29 +212,16 @@ class InMemoryStorage BSLS_KEYWORD_FINAL : public ReplicatedStorage { bsls::Types::Int64 d_ttlSeconds; - bsl::string d_emptyAppId; - // This field is unused, but needs to - // be a member variable so that - // 'appId()' routine can return a ref. - - mqbu::StorageKey d_nullAppKey; - // This field is unused, but needs to - // be a member variable so that - // 'appKey()' routine can return a ref. - bsls::AtomicInt d_isEmpty; // Flag indicating if storage is empty. // This flag can be checked from any // thread. - bmqp::RdaInfo d_defaultRdaInfo; - // Use in all 'put' operations. - bmqt::MessageGUID d_currentlyAutoConfirming; // Message being evaluated and possibly auto confirmed. - unsigned int d_numAutoConfirms; - // Current number of auto confirmed Apps for 'd_currentlyAutoConfirming'. + AutoConfirms d_autoConfirms; + // Current auto confirmed Apps for 'd_currentlyAutoConfirming'. private: // NOT IMPLEMENTED @@ -250,7 +245,6 @@ class InMemoryStorage BSLS_KEYWORD_FINAL : public ReplicatedStorage { int partitionId, const mqbconfm::Domain& config, mqbu::CapacityMeter* parentCapacityMeter, - const bmqp::RdaInfo& defaultRdaInfo, bslma::Allocator* allocator, mwcma::CountingAllocatorStore* allocatorStore = 0); @@ -273,76 +267,99 @@ class InMemoryStorage BSLS_KEYWORD_FINAL : public ReplicatedStorage { virtual void setQueue(mqbi::Queue* queue) BSLS_KEYWORD_OVERRIDE; - virtual mqbi::Queue* queue() BSLS_KEYWORD_OVERRIDE; - /// Close this storage. virtual void close() BSLS_KEYWORD_OVERRIDE; - /// Get an iterator for items stored in the virtual storage identified - /// by the specified `appKey`. Iterator will point to point to the - /// oldest item, if any, or to the end of the collection if empty. Note - /// that if `appKey` is null, an iterator over the underlying physical - /// storage will be returned. Also note that because `Storage` and - /// `StorageIterator` are interfaces, the implementation of this method - /// will allocate, so it's recommended to keep the iterator. + /// Get an iterator for data stored in the virtual storage identified by + /// the specified 'appKey'. + /// If the 'appKey' is null, the returned iterator can iterate states of + /// all Apps; otherwise, the iterator can iterate states of the App + /// corresponding to the 'appKey'. virtual bslma::ManagedPtr getIterator(const mqbu::StorageKey& appKey) BSLS_KEYWORD_OVERRIDE; - /// Load into the specified `out` an iterator for items stored in - /// the virtual storage identified by the specified `appKey`, initially - /// pointing to the item associated with the specified `msgGUID`. - /// Return zero on success, and a non-zero code if `msgGUID` was not - /// found in the storage. Note that if `appKey` is null, an iterator - /// over the underlying physical storage will be returned. Also note - /// that because `Storage` and `StorageIterator` are interfaces, the - /// implementation of this method will allocate, so it's recommended to - /// keep the iterator. + /// Load into the specified 'out' an iterator for data stored in the + /// virtual storage initially pointing to the message associated with the + /// specified 'msgGUID'. + /// If the 'appKey' is null, the returned iterator can iterate states of + /// all Apps; otherwise, the iterator can iterate states of the App + /// corresponding to the 'appKey'. + /// Return zero on success, and a non-zero code if 'msgGUID' was not + /// found in the storage. virtual mqbi::StorageResult::Enum getIterator(bslma::ManagedPtr* out, const mqbu::StorageKey& appKey, const bmqt::MessageGUID& msgGUID) BSLS_KEYWORD_OVERRIDE; - /// Save the message contained in the specified `appData`, `options` and - /// the associated `attributes` and `msgGUID` into this storage and the - /// associated virtual storages, if any. The `attributes` is an in/out - /// parameter and storage layer can populate certain fields of that - /// struct. Return 0 on success or an non-zero error code on failure. + /// Save the message contained in the specified 'appData', 'options' and + /// the associated 'attributes' and 'msgGUID' into this storage and the + /// associated virtual storage. The 'attributes' is an in/out parameter + /// and storage layer can populate certain fields of that struct. + /// Return 0 on success or an non-zero error code on failure. virtual mqbi::StorageResult::Enum put(mqbi::StorageMessageAttributes* attributes, const bmqt::MessageGUID& msgGUID, const bsl::shared_ptr& appData, - const bsl::shared_ptr& options, - const StorageKeys& storageKeys = StorageKeys()) BSLS_KEYWORD_OVERRIDE; - - /// Release the reference of the specified `appKey` on the message - /// identified by the specified `msgGUID`, and record this event in the - /// storage. Return one of the return codes from: - /// * **e_GUID_NOT_FOUND** : `msgGUID` was not found - /// * **e_ZERO_REFERENCES** : message refCount has become zero - /// * **e_NON_ZERO_REFERENCES** : message refCount is still not zero - /// * **e_WRITE_FAILURE** : failed to record this event in storage + const bsl::shared_ptr& options) BSLS_KEYWORD_OVERRIDE; + + /// Update the App state corresponding to the specified 'msgGUID' and the + /// specified 'appKey' in the DataStream. Decrement the reference count of + /// the message identified by the 'msgGUID', and record the CONFIRM in the + /// storage. + /// Return one of the return codes from: + /// * e_SUCCESS : success + /// * e_GUID_NOT_FOUND : 'msgGUID' was not found + /// * e_ZERO_REFERENCES : message refCount has become zero + /// * e_NON_ZERO_REFERENCES : message refCount is still not zero + /// * e_WRITE_FAILURE : failed to record this event in storage + /// + /// Behavior is undefined unless there is an App with the 'appKey'. + /// + /// On CONFIRM, the caller of 'confirm' is responsible to follow with + /// 'remove' call. 'releaseRef' is an alternative way to remove message in + /// one call. mqbi::StorageResult::Enum - releaseRef(const bmqt::MessageGUID& msgGUID, - const mqbu::StorageKey& appKey, - bsls::Types::Int64 timestamp, - bool onReject = false) BSLS_KEYWORD_OVERRIDE; - - /// Remove from the storage the message having the specified `msgGUID` - /// and store it's size, in bytes, in the optionally specified `msgSize` - /// if the `msgGUID` was found. Return 0 on success, or a non-zero - /// return code if the `msgGUID` was not found. If the optionally - /// specified `clearAll` is true, remove the message from all virtual - /// storages as well. + confirm(const bmqt::MessageGUID& msgGUID, + const mqbu::StorageKey& appKey, + bsls::Types::Int64 timestamp, + bool onReject = false) BSLS_KEYWORD_OVERRIDE; + + /// Decrement the reference count of the message identified by the + /// 'msgGUID'. If the resulting value is zero, delete the message data and + /// record the event in the storage. + /// Return one of the return codes from: + /// * e_SUCCESS : success + /// * e_GUID_NOT_FOUND : 'msgGUID' was not found + /// * e_INVALID_OPERATION : the value is invalid (already zero) + /// * e_ZERO_REFERENCES : message refCount has become zero + /// * e_NON_ZERO_REFERENCE : message refCount is still not zero + /// + /// On CONFIRM, the caller of 'confirm' is responsible to follow with + /// 'remove' call. 'releaseRef' is an alternative way to remove message in + /// one call. + mqbi::StorageResult::Enum + releaseRef(const bmqt::MessageGUID& msgGUID) BSLS_KEYWORD_OVERRIDE; + + /// Remove from the storage the message having the specified 'msgGUID' + /// and store it's size, in bytes, in the optionally specified 'msgSize'. + /// Record the event in the storage. + /// Return 0 on success, or a non-zero return code if the 'msgGUID' was not + /// found or if has failed to record this event in storage. + /// + /// On CONFIRM, the caller of 'confirm' is responsible to follow with + /// 'remove' call. 'releaseRef' is an alternative way to remove message in + /// one call. virtual mqbi::StorageResult::Enum remove(const bmqt::MessageGUID& msgGUID, - int* msgSize = 0, - bool clearAll = false) BSLS_KEYWORD_OVERRIDE; - - /// Remove all messages from this storage for the client identified by - /// the specified `appKey`. If `appKey` is null, then remove messages - /// for all clients. Return one of the return codes from: - /// * **e_SUCCESS** : `msgGUID` was not found - /// * **e_APPKEY_NOT_FOUND** : Invalid `appKey` specified + int* msgSize = 0) BSLS_KEYWORD_OVERRIDE; + + /// Remove all messages from this storage for the App identified by the + /// specified 'appKey' if 'appKey' is not null. Otherwise, remove messages + /// for all Apps. Record the event in the storage. + /// Return one of the return codes from: + /// * e_SUCCESS : success + /// * e_WRITE_FAILURE : failed to record this event in storage + /// * e_APPKEY_NOT_FOUND : Invalid 'appKey' specified virtual mqbi::StorageResult::Enum removeAll(const mqbu::StorageKey& appKey) BSLS_KEYWORD_OVERRIDE; @@ -404,20 +421,16 @@ class InMemoryStorage BSLS_KEYWORD_FINAL : public ReplicatedStorage { // ACCESSORS // (virtual mqbi::Storage) + /// Return the queue this storage is associated with. + /// Storage exists without a queue before `setQueue`. + virtual mqbi::Queue* queue() const BSLS_KEYWORD_OVERRIDE; + /// Return the URI of the queue this storage is associated with. virtual const bmqt::Uri& queueUri() const BSLS_KEYWORD_OVERRIDE; /// Return the storage key associated with this instance. virtual const mqbu::StorageKey& queueKey() const BSLS_KEYWORD_OVERRIDE; - /// Return the appId associated with this storage instance. If there is - /// not appId associated, return an empty string. - virtual const bsl::string& appId() const BSLS_KEYWORD_OVERRIDE; - - /// Return the app key, if any, associated with this storage instance. - /// If there is no appKey associated, return a null key. - virtual const mqbu::StorageKey& appKey() const BSLS_KEYWORD_OVERRIDE; - /// Return the current configuration used by this storage. The behavior /// is undefined unless `configure` was successfully called. virtual const mqbconfm::Storage& config() const BSLS_KEYWORD_OVERRIDE; @@ -468,42 +481,43 @@ class InMemoryStorage BSLS_KEYWORD_FINAL : public ReplicatedStorage { get(mqbi::StorageMessageAttributes* attributes, const bmqt::MessageGUID& msgGUID) const BSLS_KEYWORD_OVERRIDE; + /// Store in the specified 'msgSize' the size, in bytes, of the message + /// having the specified 'msgGUID' if found and return success, or return a + /// non-zero return code and leave 'msgSize' untouched if no message for + /// the 'msgGUID' was found. virtual mqbi::StorageResult::Enum getMessageSize( int* msgSize, const bmqt::MessageGUID& msgGUID) const BSLS_KEYWORD_OVERRIDE; - // Store in the specified 'msgSize' the size, in bytes, of the message - // having the specified 'msgGUID' if found and return success, or - // return a non-zero return code and leave 'msgSize' untouched if no - // message with 'msgGUID' were found. /// Return the number of virtual storages registered with this instance. virtual int numVirtualStorages() const BSLS_KEYWORD_OVERRIDE; + /// Return true if virtual storage identified by the specified 'appKey' + /// exists, otherwise return false. Load into the optionally specified + /// 'appId' the appId associated with 'appKey' if the virtual storage + /// exists, otherwise set it to 0. virtual bool hasVirtualStorage(const mqbu::StorageKey& appKey, bsl::string* appId = 0) const BSLS_KEYWORD_OVERRIDE; - // Return true if virtual storage identified by the specified 'appKey' - // exists, otherwise return false. Load into the optionally specified - // 'appId' the appId associated with 'appKey' if the virtual storage - // exists, otherwise set it to 0. - virtual bool hasVirtualStorage(const bsl::string& appId, - mqbu::StorageKey* appKey = 0) const - BSLS_KEYWORD_OVERRIDE; - // Return true if virtual storage identified by the specified 'appId' - // exists, otherwise return false. Load into the optionally specified - // 'appKey' the appKey associated with 'appId' if the virtual storage - // exists, otherwise set it to 0. + /// Return true if virtual storage identified by the specified 'appId' + /// exists, otherwise return false. Load into the optionally specified + /// 'appKey' and 'ordinal' the appKey and ordinal associated with 'appId' + /// if the virtual storage exists, otherwise set it to 0. + virtual bool + hasVirtualStorage(const bsl::string& appId, + mqbu::StorageKey* appKey = 0, + unsigned int* ordinal = 0) const BSLS_KEYWORD_OVERRIDE; + /// Return 'true' if there was Replication Receipt for the specified + /// 'msgGUID'. virtual bool hasReceipt(const bmqt::MessageGUID& msgGUID) const BSLS_KEYWORD_OVERRIDE; - // Return 'true' if there was Replication Receipt for the specified - // 'msgGUID'. + /// Load into the specified 'buffer' the list of pairs of appId and + /// appKey for all the virtual storages registered with this instance. virtual void loadVirtualStorageDetails(AppIdKeyPairs* buffer) const BSLS_KEYWORD_OVERRIDE; - // Load into the specified 'buffer' the list of pairs of appId and - // appKey for all the virtual storages registered with this instance. virtual unsigned int numAutoConfirms() const BSLS_KEYWORD_OVERRIDE; @@ -537,91 +551,9 @@ class InMemoryStorage BSLS_KEYWORD_FINAL : public ReplicatedStorage { queueOpRecordHandles() const BSLS_KEYWORD_OVERRIDE; virtual bool isStrongConsistency() const BSLS_KEYWORD_OVERRIDE; -}; - -// ============================= -// class InMemoryStorageIterator -// ============================= - -class InMemoryStorageIterator : public mqbi::StorageIterator { - // TBD - - private: - // PRIVATE TYPES - typedef InMemoryStorage::ItemsMapConstIter ItemsMapConstIter; - - private: - // DATA - InMemoryStorage* d_storage_p; // Storage this Iterator is associated - // with - - ItemsMapConstIter d_iterator; // Internal iterator representing the - // current position - - public: - // CREATORS - - /// Create a new InMemoryStorageIterator from the specified `storage` - /// and pointing at the specified `initialPosition`. - InMemoryStorageIterator(InMemoryStorage* storage, - const ItemsMapConstIter& initialPosition); - - /// Destructor - ~InMemoryStorageIterator() BSLS_KEYWORD_OVERRIDE; // ACCESSORS - - /// Return a reference offering non-modifiable access to the guid - /// associated to the item currently pointed at by this iterator. The - /// behavior is undefined unless `atEnd` returns `false`. - const bmqt::MessageGUID& guid() const BSLS_KEYWORD_OVERRIDE; - - /// Return a reference offering modifiable access to the RdaInfo - /// associated to the item currently pointed at by this iterator. The - /// behavior is undefined unless `atEnd` returns `false`. - bmqp::RdaInfo& rdaInfo() const BSLS_KEYWORD_OVERRIDE; - - /// Return subscription id associated to the item currently pointed at - /// by this iterator. - /// The behavior is undefined unless `atEnd` returns `false`. - unsigned int subscriptionId() const BSLS_KEYWORD_OVERRIDE; - - /// Return a reference offering non-modifiable access to the application - /// data associated with the item currently pointed at by this iterator. - /// The behavior is undefined unless `atEnd` returns `false`. - const bsl::shared_ptr& appData() const BSLS_KEYWORD_OVERRIDE; - - /// Return a reference offering non-modifiable access to the options - /// associated with the item currently pointed at by this iterator. The - /// behavior is undefined unless `atEnd` returns `false`. - const bsl::shared_ptr& options() const BSLS_KEYWORD_OVERRIDE; - - const mqbi::StorageMessageAttributes& - attributes() const BSLS_KEYWORD_OVERRIDE; - // Return a reference offering non-modifiable access to the attributes - // associated with the message currently pointed at by this iterator. - // The behavior is undefined unless 'atEnd' returns 'false'. - - /// Return `true` if this iterator is currently at the end of the items' - /// collection, and hence doesn't reference a valid item. - bool atEnd() const BSLS_KEYWORD_OVERRIDE; - - /// Return `true` if this iterator is currently not at the end of the - /// `items` collection and the message currently pointed at by this - /// iterator has received replication factor Receipts. - bool hasReceipt() const BSLS_KEYWORD_OVERRIDE; - - // MANIPULATORS - - /// Advance the iterator to the next item. The behavior is undefined - /// unless `atEnd` returns `false`. Return `true` if the iterator then - /// points to a valid item, or `false` if it now is at the end of the - /// items' collection. - bool advance() BSLS_KEYWORD_OVERRIDE; - - /// Reset the iterator to point to first item, if any, in the underlying - /// storage. - void reset() BSLS_KEYWORD_OVERRIDE; + bool isProxy() const; }; // ============================================================================ @@ -702,15 +634,27 @@ InMemoryStorage_Item::attributes() const return d_attributes; } +inline InMemoryStorage::AutoConfirm::AutoConfirm( + const mqbu::StorageKey& appKey) +: d_appKey(appKey) +{ + // NOTHING +} + +inline const mqbu::StorageKey& InMemoryStorage::AutoConfirm::appKey() +{ + return d_appKey; +} + // --------------------- // class InMemoryStorage // --------------------- // MANIPULATORS // (virtual mqbi::Storage) -inline mqbi::Queue* InMemoryStorage::queue() +inline mqbi::Queue* InMemoryStorage::queue() const { - return d_queue_p; + return d_virtualStorageCatalog.queue(); } inline int InMemoryStorage::addVirtualStorage(bsl::ostream& errorDescription, @@ -742,16 +686,6 @@ inline const mqbu::StorageKey& InMemoryStorage::queueKey() const return d_key; } -inline const bsl::string& InMemoryStorage::appId() const -{ - return d_emptyAppId; -} - -inline const mqbu::StorageKey& InMemoryStorage::appKey() const -{ - return d_nullAppKey; -} - inline const mqbconfm::Storage& InMemoryStorage::config() const { return d_config; @@ -822,9 +756,10 @@ inline bool InMemoryStorage::hasVirtualStorage(const mqbu::StorageKey& appKey, } inline bool InMemoryStorage::hasVirtualStorage(const bsl::string& appId, - mqbu::StorageKey* appKey) const + mqbu::StorageKey* appKey, + unsigned int* ordinal) const { - return d_virtualStorageCatalog.hasVirtualStorage(appId, appKey); + return d_virtualStorageCatalog.hasVirtualStorage(appId, appKey, ordinal); } inline bool InMemoryStorage::hasReceipt(const bmqt::MessageGUID&) const @@ -841,7 +776,7 @@ InMemoryStorage::loadVirtualStorageDetails(AppIdKeyPairs* buffer) const inline unsigned int InMemoryStorage::numAutoConfirms() const { - return d_numAutoConfirms; + return d_autoConfirms.size(); } inline mqbu::CapacityMeter* InMemoryStorage::capacityMeter() @@ -856,77 +791,10 @@ inline int InMemoryStorage::partitionId() const return d_partitionId; } -// ----------------------------- -// class InMemoryStorageIterator -// ----------------------------- - // ACCESSORS -inline const bmqt::MessageGUID& InMemoryStorageIterator::guid() const -{ - return d_iterator->first; -} - -inline bmqp::RdaInfo& InMemoryStorageIterator::rdaInfo() const -{ - static bmqp::RdaInfo dummy; - return dummy; -} - -inline unsigned int InMemoryStorageIterator::subscriptionId() const -{ - return bmqp::Protocol::k_DEFAULT_SUBSCRIPTION_ID; -} - -inline const bsl::shared_ptr& -InMemoryStorageIterator::appData() const -{ - // PRECONDITIONS - BSLS_ASSERT_SAFE(!atEnd()); - - return d_iterator->second.appData(); -} - -inline const bsl::shared_ptr& -InMemoryStorageIterator::options() const -{ - // PRECONDITIONS - BSLS_ASSERT_SAFE(!atEnd()); - - return d_iterator->second.options(); -} - -inline const mqbi::StorageMessageAttributes& -InMemoryStorageIterator::attributes() const -{ - // PRECONDITIONS - BSLS_ASSERT_SAFE(!atEnd()); - - return d_iterator->second.attributes(); -} - -inline bool InMemoryStorageIterator::atEnd() const -{ - return d_iterator == d_storage_p->d_items.end(); -} - -inline bool InMemoryStorageIterator::hasReceipt() const -{ - return !atEnd(); -} - -// MANIPULATORS -inline bool InMemoryStorageIterator::advance() -{ - // PRECONDITIONS - BSLS_ASSERT_SAFE(!atEnd()); - - ++d_iterator; - return !atEnd(); -} - -inline void InMemoryStorageIterator::reset() +inline bool InMemoryStorage::isProxy() const { - d_iterator = d_storage_p->d_items.begin(); + return d_partitionId == mqbs::DataStore::k_INVALID_PARTITION_ID; } } // close package namespace diff --git a/src/groups/mqb/mqbs/mqbs_inmemorystorage.t.cpp b/src/groups/mqb/mqbs/mqbs_inmemorystorage.t.cpp index ccdca3ca2..f4c7788d7 100644 --- a/src/groups/mqb/mqbs/mqbs_inmemorystorage.t.cpp +++ b/src/groups/mqb/mqbs/mqbs_inmemorystorage.t.cpp @@ -98,6 +98,7 @@ namespace { // CONSTANTS const int k_PARTITION_ID = 1; +const int k_PROXY_PARTITION_ID = mqbs::DataStore::k_INVALID_PARTITION_ID; const char k_HEX_QUEUE[] = "ABCDEF1234"; const bsls::Types::Int64 k_DEFAULT_MSG = 20; const bsls::Types::Int64 k_DEFAULT_BYTES = 2048; @@ -202,7 +203,6 @@ struct Tester { partitionId, domainCfg, d_mockDomain.capacityMeter(), - bmqp::RdaInfo(), d_allocator_p), d_allocator_p); d_inMemoryStorage_mp->setQueue(&d_mockQueue); @@ -254,11 +254,10 @@ struct Tester { /// as incrementing values from 0 or from dataOffset provided /// Thus acting like a vector clock, also easier to verify. mqbi::StorageResult::Enum - addMessages(bsl::vector* guidHolder, - const mqbi::Storage::StorageKeys& storageKeys, - const int msgCount, - int dataOffset = 0, - bool useSameGuids = false) + addMessages(bsl::vector* guidHolder, + const int msgCount, + int dataOffset = 0, + bool useSameGuids = false) { if (!useSameGuids) { guidHolder->reserve(msgCount); @@ -293,8 +292,7 @@ struct Tester { &attributes, guid, appDataPtr, - appDataPtr, - storageKeys); + appDataPtr); if (rc != mqbi::StorageResult::e_SUCCESS) { return rc; // RETURN @@ -346,7 +344,7 @@ struct BasicTest : mwctst::Test { BasicTest::BasicTest() : d_tester(k_URI_STR, k_QUEUE_KEY, - k_PARTITION_ID, + k_PROXY_PARTITION_ID, k_INT64_MAX, // ttlSeconds s_allocator_p, false) // toConfigure @@ -469,14 +467,12 @@ TEST_F(BasicTest, breathingTest) ASSERT_EQ(d_tester.storage().queueUri().asString(), k_URI_STR); ASSERT_EQ(d_tester.storage().queueKey(), k_QUEUE_KEY); - ASSERT_EQ(d_tester.storage().appId(), ""); - ASSERT_EQ(d_tester.storage().appKey(), k_NULL_KEY); ASSERT_EQ(d_tester.storage().config(), mqbconfm::Storage()); ASSERT_EQ(d_tester.storage().isPersistent(), false); ASSERT_EQ(d_tester.storage().numMessages(k_NULL_KEY), k_INT64_ZERO); ASSERT_EQ(d_tester.storage().numBytes(k_NULL_KEY), k_INT64_ZERO); ASSERT_EQ(d_tester.storage().isEmpty(), true); - ASSERT_EQ(d_tester.storage().partitionId(), k_PARTITION_ID); + ASSERT_EQ(d_tester.storage().partitionId(), k_PROXY_PARTITION_ID); ASSERT_NE(d_tester.storage().queue(), static_cast(0)); // Queue has been set via call to 'setQueue' @@ -568,13 +564,12 @@ TEST_F(Test, put_noVirtualStorage) mwctst::TestHelper::printTestName("PUT - WITH NO VIRTUAL STORAGES"); mwcu::MemOutStream errDescription(s_allocator_p); - const mqbi::Storage::StorageKeys storageKeys; bsl::vector guids(s_allocator_p); const int k_MSG_COUNT = 10; // Check 'put' - To physical storage (StorageKeys = NULL) - ASSERT_EQ(d_tester.addMessages(&guids, storageKeys, k_MSG_COUNT), + ASSERT_EQ(d_tester.addMessages(&guids, k_MSG_COUNT), mqbi::StorageResult::e_SUCCESS); // Check accessors and manipulators for Messages @@ -606,12 +601,11 @@ TEST_F(Test, getMessageSize) mwctst::TestHelper::printTestName("GET MESSAGE SIZE"); mwcu::MemOutStream errDescription(s_allocator_p); - const mqbi::Storage::StorageKeys storageKeys; bsl::vector guids(s_allocator_p); const int k_MSG_COUNT = 10; - ASSERT_EQ(d_tester.addMessages(&guids, storageKeys, k_MSG_COUNT), + ASSERT_EQ(d_tester.addMessages(&guids, k_MSG_COUNT), mqbi::StorageResult::e_SUCCESS); // Check getMessageSize @@ -644,13 +638,12 @@ TEST_F(Test, get_noVirtualStorages) mwctst::TestHelper::printTestName("GET - WITH NO VIRTUAL STORAGES"); mwcu::MemOutStream errDescription(s_allocator_p); - const mqbi::Storage::StorageKeys storageKeys; bsl::vector guids(s_allocator_p); const int k_MSG_COUNT = 5; // Put to physical storage - StorageKeys NULL - BSLS_ASSERT_OPT(d_tester.addMessages(&guids, storageKeys, k_MSG_COUNT) == + BSLS_ASSERT_OPT(d_tester.addMessages(&guids, k_MSG_COUNT) == mqbi::StorageResult::e_SUCCESS); // Check 'get' overloads @@ -732,11 +725,10 @@ TEST_F(Test, removeMessage) const int k_MSG_COUNT = 10; mwcu::MemOutStream errDescription(s_allocator_p); - const mqbi::Storage::StorageKeys storageKeys; bsl::vector guids(s_allocator_p); // Check 'put' - To physical storage (StorageKeys = NULL) - BSLS_ASSERT_OPT(d_tester.addMessages(&guids, storageKeys, k_MSG_COUNT) == + BSLS_ASSERT_OPT(d_tester.addMessages(&guids, k_MSG_COUNT) == mqbi::StorageResult::e_SUCCESS); // Remove messages one by one @@ -860,7 +852,7 @@ TEST_F(Test, removeVirtualStorage) ASSERT_EQ(d_tester.storage().numVirtualStorages(), 0); } -TEST_F(BasicTest, put_withVirtualStorages) +TEST_F(Test, put_withVirtualStorages) // ------------------------------------------------------------------------ // Put Test with virtual storages // @@ -891,35 +883,24 @@ TEST_F(BasicTest, put_withVirtualStorages) // Scenario: // Two Virtual Storages. - // - First 20 Messages 'put' using 'mqbu::StorageKey::k_NULL_KEY' applies - // to physical as well as both virtual storages. - // - Next 20 messages only to 'k_APP_KEY2'. - // Capacity Meter updates only for the first 20 messages. Refer to 'put' - // implementation for logic. + const int k_MSG_COUNT = 20; const bsls::Types::Int64 k_MSG_COUNT_INT64 = static_cast(k_MSG_COUNT); const bsls::Types::Int64 k_BYTE_PER_MSG = static_cast( sizeof(int)); - mqbi::Storage::StorageKeys storageKeys; bsl::vector guids(s_allocator_p); // Check 'put'- To physical storage (StorageKeys = NULL) - ASSERT_EQ(d_tester.addMessages(&guids, storageKeys, k_MSG_COUNT), - mqbi::StorageResult::e_SUCCESS); - - storageKeys.push_back(k_APP_KEY2); - - ASSERT_EQ(d_tester.addMessages(&guids, storageKeys, k_MSG_COUNT), + ASSERT_EQ(d_tester.addMessages(&guids, k_MSG_COUNT), mqbi::StorageResult::e_SUCCESS); // Verify number of messages for each Virtual storage ASSERT_EQ(d_tester.storage().numMessages(mqbu::StorageKey::k_NULL_KEY), - 2 * k_MSG_COUNT_INT64); + k_MSG_COUNT_INT64); ASSERT_EQ(d_tester.storage().numMessages(k_APP_KEY1), k_MSG_COUNT_INT64); - ASSERT_EQ(d_tester.storage().numMessages(k_APP_KEY2), - 2 * k_MSG_COUNT_INT64); + ASSERT_EQ(d_tester.storage().numMessages(k_APP_KEY2), k_MSG_COUNT_INT64); // Verify number of bytes for each Virtual storage ASSERT_EQ(d_tester.storage().numBytes(mqbu::StorageKey::k_NULL_KEY), @@ -930,7 +911,7 @@ TEST_F(BasicTest, put_withVirtualStorages) ASSERT_EQ(d_tester.storage().numBytes(k_APP_KEY1), k_MSG_COUNT_INT64 * k_BYTE_PER_MSG); ASSERT_EQ(d_tester.storage().numBytes(k_APP_KEY2), - 2 * k_MSG_COUNT_INT64 * k_BYTE_PER_MSG); + k_MSG_COUNT_INT64 * k_BYTE_PER_MSG); // Verify capacity meter updates only on 'put' to physical storage ASSERT_EQ(d_tester.storage().capacityMeter()->bytes(), @@ -941,7 +922,7 @@ TEST_F(BasicTest, put_withVirtualStorages) mqbi::StorageResult::e_SUCCESS); } -TEST_F(BasicTest, removeAllMessages_appKeyNotFound) +TEST_F(Test, removeAllMessages_appKeyNotFound) // ------------------------------------------------------------------------ // REMOVE ALL MESSAGES - APPKEY NOT FOUND // @@ -966,9 +947,7 @@ TEST_F(BasicTest, removeAllMessages_appKeyNotFound) k_APP_KEY1) == 0); // Scenario - // One virtual Storage: - // - First 20 Messages 'put' using 'mqbu::StorageKey::k_NULL_KEY' applies - // to physical as well as the virtual storage represented by 'APP_KEY1'. + // One virtual Storage // - Attempt to 'removeAll' for non-existent 'k_APP_KEY2' and verify // appropriate return code. const int k_MSG_COUNT = 20; @@ -977,9 +956,8 @@ TEST_F(BasicTest, removeAllMessages_appKeyNotFound) const bsls::Types::Int64 k_BYTE_PER_MSG = static_cast( sizeof(int)); - mqbi::Storage::StorageKeys storageKeys; bsl::vector guids(s_allocator_p); - d_tester.addMessages(&guids, storageKeys, k_MSG_COUNT); + d_tester.addMessages(&guids, k_MSG_COUNT); // Verify 'removeAll' operation BSLS_ASSERT_OPT(d_tester.storage().numMessages(k_NULL_KEY) == @@ -1022,9 +1000,6 @@ TEST_F(BasicTest, removeAllMessages) // Scenario // Two Virtual Storages - // - First 20 Messages 'put' using 'mqbu::StorageKey::k_NULL_KEY' applies - // to physical as well as both virtual storages. - // - Next 20 messages only to 'k_APP_KEY2'. // Check 'removeAll' using these appKeys. const int k_MSG_COUNT = 20; const bsls::Types::Int64 k_MSG_COUNT_INT64 = @@ -1032,22 +1007,15 @@ TEST_F(BasicTest, removeAllMessages) const bsls::Types::Int64 k_BYTE_PER_MSG = static_cast( sizeof(int)); - mqbi::Storage::StorageKeys storageKeys; bsl::vector guids(s_allocator_p); - ASSERT_EQ(d_tester.addMessages(&guids, storageKeys, k_MSG_COUNT), - mqbi::StorageResult::e_SUCCESS); - - storageKeys.push_back(k_APP_KEY2); - - ASSERT_EQ(d_tester.addMessages(&guids, storageKeys, k_MSG_COUNT), + ASSERT_EQ(d_tester.addMessages(&guids, k_MSG_COUNT), mqbi::StorageResult::e_SUCCESS); // Verify 'removeAll' operation - ASSERT_EQ(d_tester.storage().numMessages(k_APP_KEY2), - 2 * k_MSG_COUNT_INT64); + ASSERT_EQ(d_tester.storage().numMessages(k_APP_KEY2), k_MSG_COUNT_INT64); ASSERT_EQ(d_tester.storage().numBytes(k_APP_KEY2), - 2 * k_MSG_COUNT_INT64 * k_BYTE_PER_MSG); + k_MSG_COUNT_INT64 * k_BYTE_PER_MSG); ASSERT_EQ(d_tester.storage().numMessages(k_APP_KEY1), k_MSG_COUNT_INT64); ASSERT_EQ(d_tester.storage().numBytes(k_APP_KEY1), k_MSG_COUNT_INT64 * k_BYTE_PER_MSG); @@ -1092,22 +1060,14 @@ TEST_F(BasicTest, get_withVirtualStorages) // Scenario // Single Virtual Storages - // - First 20 Messages 'put' using 'mqbu::StorageKey::k_NULL_KEY' applies - // to physical storage. - // - The 20 messages are 'put' again with storage key 'k_APP_KEY1'. - // 'get' operation to test references in message attributes. - mqbi::Storage::StorageKeys storageKeys; bsl::vector guids(s_allocator_p); // Check 'put'- To physical storage (StorageKeys = NULL) - ASSERT_EQ(d_tester.addMessages(&guids, storageKeys, 20), + ASSERT_EQ(d_tester.addMessages(&guids, 20), mqbi::StorageResult::e_SUCCESS); - // 'put' to Virtual Storage 'k_APP_KEY1' - storageKeys.push_back(k_APP_KEY1); - - ASSERT_EQ(d_tester.addMessages(&guids, storageKeys, 20, 0, true), + ASSERT_EQ(d_tester.addMessages(&guids, 20, 0, true), mqbi::StorageResult::e_SUCCESS); // Verify 'get' operation @@ -1137,7 +1097,7 @@ TEST_F(BasicTest, get_withVirtualStorages) mqbi::StorageResult::e_SUCCESS); } -TEST_F(BasicTest, releaseRef) +TEST_F(BasicTest, confirm) // ------------------------------------------------------------------------ // RELEASE REF // @@ -1161,9 +1121,6 @@ TEST_F(BasicTest, releaseRef) // Scenario: // Single Virtual Storage - // - First 20 Messages 'put' using 'mqbu::StorageKey::k_NULL_KEY' applies - // to physical storage. - // - The 20 messages are 'put' again with storage key 'k_APP_KEY1'. // 'get' operation to test references in message attributes. // We then use the releaseRef to check the decrease in refCount for the // particular message. @@ -1171,17 +1128,13 @@ TEST_F(BasicTest, releaseRef) const bsls::Types::Int64 k_BYTE_PER_MSG = static_cast( sizeof(int)); - mqbi::Storage::StorageKeys storageKeys; bsl::vector guids(s_allocator_p); // Check 'put': To physical storage (StorageKeys = NULL) - ASSERT_EQ(d_tester.addMessages(&guids, storageKeys, k_MSG_COUNT), + ASSERT_EQ(d_tester.addMessages(&guids, k_MSG_COUNT), mqbi::StorageResult::e_SUCCESS); - // 'put' to Virtual Storage: 'k_APP_KEY1' - storageKeys.push_back(k_APP_KEY1); - - ASSERT_EQ(d_tester.addMessages(&guids, storageKeys, k_MSG_COUNT, 0, true), + ASSERT_EQ(d_tester.addMessages(&guids, k_MSG_COUNT, 0, true), mqbi::StorageResult::e_SUCCESS); mqbi::StorageMessageAttributes attributes; @@ -1190,13 +1143,12 @@ TEST_F(BasicTest, releaseRef) BSLS_ASSERT_OPT(attributes.refCount() == 2U); // Attempt 'releaseRef' with non-existent GUID - ASSERT_EQ(d_tester.storage().releaseRef(generateUniqueGUID(guids), - k_APP_KEY1, - 0), - mqbi::StorageResult::e_GUID_NOT_FOUND); + ASSERT_EQ( + d_tester.storage().confirm(generateUniqueGUID(guids), k_APP_KEY1, 0), + mqbi::StorageResult::e_GUID_NOT_FOUND); // 'releaseRef' on 'APP_KEY1' and verify refCount decreased by 1 - ASSERT_EQ(d_tester.storage().releaseRef(guids[5], k_APP_KEY1, 0), + ASSERT_EQ(d_tester.storage().confirm(guids[5], k_APP_KEY1, 0), mqbi::StorageResult::e_NON_ZERO_REFERENCES); BSLS_ASSERT_OPT(d_tester.storage().get(&attributes, guids[5]) == @@ -1206,7 +1158,7 @@ TEST_F(BasicTest, releaseRef) ASSERT_EQ(d_tester.storage().numBytes(k_APP_KEY1), 19 * k_BYTE_PER_MSG); // 'releaseRef' on 'APP_KEY1' *with the same guid* and verify no effect - ASSERT_EQ(d_tester.storage().releaseRef(guids[5], k_APP_KEY1, 0), + ASSERT_EQ(d_tester.storage().confirm(guids[5], k_APP_KEY1, 0), mqbi::StorageResult::e_GUID_NOT_FOUND); BSLS_ASSERT_OPT(d_tester.storage().get(&attributes, guids[5]) == @@ -1216,7 +1168,7 @@ TEST_F(BasicTest, releaseRef) ASSERT_EQ(d_tester.storage().numBytes(k_APP_KEY1), 19 * k_BYTE_PER_MSG); // 'releaseRef' on the physical storage and verify refCount decreased to 0 - ASSERT_EQ(d_tester.storage().releaseRef(guids[5], k_NULL_KEY, 0), + ASSERT_EQ(d_tester.storage().confirm(guids[5], k_NULL_KEY, 0), mqbi::StorageResult::e_ZERO_REFERENCES); BSLS_ASSERT_OPT(d_tester.storage().removeAll(k_NULL_KEY) == @@ -1238,11 +1190,10 @@ TEST_F(Test, getIterator_noVirtualStorages) const int k_MSG_COUNT = 10; - const mqbi::Storage::StorageKeys storageKeys; bsl::vector guids(s_allocator_p); // Put to physical storage: StorageKeys NULL - ASSERT_EQ(d_tester.addMessages(&guids, storageKeys, k_MSG_COUNT), + ASSERT_EQ(d_tester.addMessages(&guids, k_MSG_COUNT), mqbi::StorageResult::e_SUCCESS); // Check Iterator @@ -1323,30 +1274,18 @@ TEST_F(BasicTest, getIterator_withVirtualStorages) // Scenario: // Two Virtual Storages - // - First 20 Messages 'put' using 'mqbu::StorageKey::k_NULL_KEY' applies - // to physical as well as both virtual storages. - // - Next 20 messages only to 'k_APP_KEY2'. - // - And the next 20 using keys to both storages 'k_APP_KEY1' and - // 'k_APP_KEY2'. // Try iterator for physical storage as well as both of these storages. - mqbi::Storage::StorageKeys storageKeys; bsl::vector guids(s_allocator_p); // Check 'put': To physical storage (StorageKeys = NULL) - ASSERT_EQ(d_tester.addMessages(&guids, storageKeys, 20), + ASSERT_EQ(d_tester.addMessages(&guids, 20), mqbi::StorageResult::e_SUCCESS); - // 'put' to Virtual Storage: 'k_APP_KEY2' - storageKeys.push_back(k_APP_KEY2); - - ASSERT_EQ(d_tester.addMessages(&guids, storageKeys, 20, 20), + ASSERT_EQ(d_tester.addMessages(&guids, 20, 20), mqbi::StorageResult::e_SUCCESS); - // 'put' to both virtual storages - storageKeys.push_back(k_APP_KEY1); - - ASSERT_EQ(d_tester.addMessages(&guids, storageKeys, 20, 40), + ASSERT_EQ(d_tester.addMessages(&guids, 20, 40), mqbi::StorageResult::e_SUCCESS); // Check Iterator @@ -1388,8 +1327,12 @@ TEST_F(BasicTest, getIterator_withVirtualStorages) msgData = 0; iterator = d_tester.storage().getIterator(k_APP_KEY1); + for (int i = 20; i < 40; ++i) { + d_tester.storage().confirm(guids[i], k_APP_KEY1, 0); + } + do { - // skip the 20 in betwee + // skip the 20 in between if (msgData == 20) { msgData += 20; } @@ -1420,11 +1363,10 @@ TEST_F(Test, capacityMeter_limitMessages) mwcu::MemOutStream errDescription(s_allocator_p); - const mqbi::Storage::StorageKeys storageKeys; bsl::vector guids(s_allocator_p); // Put to physical storage: StorageKeys NULL - ASSERT_EQ(d_tester.addMessages(&guids, storageKeys, k_DEFAULT_MSG), + ASSERT_EQ(d_tester.addMessages(&guids, k_DEFAULT_MSG), mqbi::StorageResult::e_SUCCESS); // Access CapacityMeter @@ -1434,15 +1376,14 @@ TEST_F(Test, capacityMeter_limitMessages) ASSERT_EQ(d_tester.storage().capacityMeter()->messages(), k_DEFAULT_MSG); // Try to insert more than Capacity Meter - Check success first time - ASSERT_EQ(d_tester.addMessages(&guids, storageKeys, 1), - mqbi::StorageResult::e_SUCCESS); + ASSERT_EQ(d_tester.addMessages(&guids, 1), mqbi::StorageResult::e_SUCCESS); ASSERT_EQ(d_tester.storage().capacityMeter()->messages(), k_DEFAULT_MSG + 1); // Try to insert more than Capacity Meter - Check failure after it's // already full - ASSERT_EQ(d_tester.addMessages(&guids, storageKeys, 1), + ASSERT_EQ(d_tester.addMessages(&guids, 1), mqbi::StorageResult::e_LIMIT_MESSAGES); ASSERT_EQ(d_tester.storage().capacityMeter()->messages(), @@ -1454,7 +1395,7 @@ TEST_F(Test, capacityMeter_limitMessages) ASSERT_EQ(d_tester.storage().capacityMeter()->messages(), 0); } -TEST_F(BasicTest, capacityMeter_limitBytes) +TEST_F(Test, capacityMeter_limitBytes) // ------------------------------------------------------------------------ // Capacity Meter Test // @@ -1472,22 +1413,20 @@ TEST_F(BasicTest, capacityMeter_limitBytes) BSLS_ASSERT_OPT(d_tester.configure(k_MSG_LIMIT, k_BYTES_LIMIT) == 0); - const mqbi::Storage::StorageKeys storageKeys; bsl::vector guids(s_allocator_p); // Insert Max messages possible in 80bytes const int k_MSG_COUNT = 20; - ASSERT_EQ(d_tester.addMessages(&guids, storageKeys, k_MSG_COUNT), + ASSERT_EQ(d_tester.addMessages(&guids, k_MSG_COUNT), mqbi::StorageResult::e_SUCCESS); // Try to insert more than Capacity Meter - Check success first time - ASSERT_EQ(d_tester.addMessages(&guids, storageKeys, 1), - mqbi::StorageResult::e_SUCCESS); + ASSERT_EQ(d_tester.addMessages(&guids, 1), mqbi::StorageResult::e_SUCCESS); ASSERT_EQ(d_tester.storage().capacityMeter()->bytes(), 84); // Try to insert more than Capacity Meter, check failure after it's full - ASSERT_EQ(d_tester.addMessages(&guids, storageKeys, 1), + ASSERT_EQ(d_tester.addMessages(&guids, 1), mqbi::StorageResult::e_LIMIT_BYTES); ASSERT_EQ(d_tester.storage().removeAll(mqbu::StorageKey::k_NULL_KEY), @@ -1514,14 +1453,13 @@ TEST_F(GCTest, garbageCollect) configure(k_TTL); mwcu::MemOutStream errDescription(s_allocator_p); - const mqbi::Storage::StorageKeys storageKeys; bsl::vector guids(s_allocator_p); int k_MSG_COUNT = 10; // Use offset of '1' so the first message at timestamp 1 and increments // from there - ASSERT_EQ(tester().addMessages(&guids, storageKeys, k_MSG_COUNT, 1), + ASSERT_EQ(tester().addMessages(&guids, k_MSG_COUNT, 1), mqbi::StorageResult::e_SUCCESS); bsls::Types::Uint64 latestMsgTimestamp; diff --git a/src/groups/mqb/mqbs/mqbs_storageprintutil.t.cpp b/src/groups/mqb/mqbs/mqbs_storageprintutil.t.cpp index 843f459f6..3ee3a92f4 100644 --- a/src/groups/mqb/mqbs/mqbs_storageprintutil.t.cpp +++ b/src/groups/mqb/mqbs/mqbs_storageprintutil.t.cpp @@ -145,7 +145,6 @@ struct Tester { 0, domainCfg, &d_capacityMeter, - bmqp::RdaInfo(), s_allocator_p), s_allocator_p); @@ -188,16 +187,12 @@ struct Tester { appDataPtr->setLength(i * 10); mqbi::StorageMessageAttributes attributes; - d_storage_mp->put(&attributes, - guid, - appDataPtr, - appDataPtr, - mqbi::Storage::StorageKeys()); + d_storage_mp->put(&attributes, guid, appDataPtr, appDataPtr); } for (int i = 0; i < 5; ++i) { - d_storage_mp->releaseRef(d_guids[i * 2], k_APP_KEY1, 0); - d_storage_mp->releaseRef(d_guids[i * 2 + 1], k_APP_KEY2, 0); + d_storage_mp->confirm(d_guids[i * 2], k_APP_KEY1, 0); + d_storage_mp->confirm(d_guids[i * 2 + 1], k_APP_KEY2, 0); } } diff --git a/src/groups/mqb/mqbs/mqbs_virtualstorage.cpp b/src/groups/mqb/mqbs/mqbs_virtualstorage.cpp index 61900d01f..dc9b9594c 100644 --- a/src/groups/mqb/mqbs/mqbs_virtualstorage.cpp +++ b/src/groups/mqb/mqbs/mqbs_virtualstorage.cpp @@ -15,6 +15,7 @@ // mqbs_virtualstorage.cpp -*-C++-*- #include +#include #include // BDE @@ -34,14 +35,15 @@ namespace mqbs { VirtualStorage::VirtualStorage(mqbi::Storage* storage, const bsl::string& appId, const mqbu::StorageKey& appKey, + unsigned int ordinal, bslma::Allocator* allocator) : d_allocator_p(allocator) , d_storage_p(storage) , d_appId(appId, allocator) , d_appKey(appKey) -, d_guids(allocator) -, d_totalBytes(0) -, d_autoConfirm() +, d_removedBytes(0) +, d_numRemoved(0) +, d_ordinal(ordinal) { BSLS_ASSERT_SAFE(d_storage_p); BSLS_ASSERT_SAFE(allocator); @@ -56,204 +58,59 @@ VirtualStorage::~VirtualStorage() // MANIPULATORS mqbi::StorageResult::Enum -VirtualStorage::get(bsl::shared_ptr* appData, - bsl::shared_ptr* options, - mqbi::StorageMessageAttributes* attributes, - const bmqt::MessageGUID& msgGUID) const +VirtualStorage::confirm(DataStreamMessage* dataStreamMessage) { - if (!hasMessage(msgGUID)) { - return mqbi::StorageResult::e_GUID_NOT_FOUND; // RETURN - } - - return d_storage_p->get(appData, options, attributes, msgGUID); -} - -mqbi::StorageResult::Enum -VirtualStorage::get(mqbi::StorageMessageAttributes* attributes, - const bmqt::MessageGUID& msgGUID) const -{ - if (!hasMessage(msgGUID)) { - return mqbi::StorageResult::e_GUID_NOT_FOUND; // RETURN - } - - return d_storage_p->get(attributes, msgGUID); -} - -int VirtualStorage::configure( - BSLS_ANNOTATION_UNUSED bsl::ostream& errorDescription, - BSLS_ANNOTATION_UNUSED const mqbconfm::Storage& config, - BSLS_ANNOTATION_UNUSED const mqbconfm::Limits& limits, - BSLS_ANNOTATION_UNUSED const bsls::Types::Int64 messageTtl, - BSLS_ANNOTATION_UNUSED const int maxDeliveryAttempts) -{ - // NOTHING - return 0; -} - -void VirtualStorage::setQueue(BSLS_ANNOTATION_UNUSED mqbi::Queue* queue) -{ - BSLS_ASSERT_OPT(false && "Should not be invoked."); -} - -mqbi::Queue* VirtualStorage::queue() -{ - BSLS_ASSERT_OPT(false && "Should not be invoked."); - return static_cast(0); -} - -void VirtualStorage::close() -{ - // NOTHING -} + mqbi::AppMessage& appMessage = dataStreamMessage->app(ordinal()); -mqbi::StorageResult::Enum VirtualStorage::put(const bmqt::MessageGUID& msgGUID, - int msgSize, - const bmqp::RdaInfo& rdaInfo, - unsigned int subScriptionId) -{ - if (!d_autoConfirm.isUnset()) { - const bool isAutoConfirmed = (d_autoConfirm == msgGUID); + if (appMessage.isPending()) { + appMessage.setConfirmState(); - d_autoConfirm = bmqt::MessageGUID(); - if (isAutoConfirmed) { - return mqbi::StorageResult::e_SUCCESS; // RETURN - } - } + d_removedBytes += dataStreamMessage->d_size; + ++d_numRemoved; - if (d_guids - .insert(bsl::make_pair( - msgGUID, - MessageContext(msgSize, rdaInfo, subScriptionId))) - .second == false) { - // Duplicate GUID - return mqbi::StorageResult::e_GUID_NOT_UNIQUE; // RETURN + return mqbi::StorageResult::e_SUCCESS; } - - // Success: new GUID - d_totalBytes += msgSize; - return mqbi::StorageResult::e_SUCCESS; -} - -mqbi::StorageResult::Enum VirtualStorage::put( - BSLS_ANNOTATION_UNUSED mqbi::StorageMessageAttributes* attributes, - BSLS_ANNOTATION_UNUSED const bmqt::MessageGUID& msgGUID, - BSLS_ANNOTATION_UNUSED const bsl::shared_ptr& appData, - BSLS_ANNOTATION_UNUSED const bsl::shared_ptr& options, - BSLS_ANNOTATION_UNUSED const mqbi::Storage::StorageKeys& storageKeys) -{ - BSLS_ASSERT_OPT(false && "Should not be invoked."); - return mqbi::StorageResult::e_INVALID_OPERATION; -} - -bslma::ManagedPtr -VirtualStorage::getIterator(const mqbu::StorageKey& appKey) -{ - BSLS_ASSERT_SAFE(d_appKey == appKey); - static_cast(appKey); - - bslma::ManagedPtr mp( - new (*d_allocator_p) VirtualStorageIterator(this, d_guids.begin()), - d_allocator_p); - - return mp; -} - -mqbi::StorageResult::Enum -VirtualStorage::getIterator(bslma::ManagedPtr* out, - const mqbu::StorageKey& appKey, - const bmqt::MessageGUID& msgGUID) -{ - // PRECONDITIONS - BSLS_ASSERT_SAFE(d_appKey == appKey); - static_cast(appKey); - - GuidListIter it = d_guids.find(msgGUID); - if (it == d_guids.end()) { + else { + // already deleted return mqbi::StorageResult::e_GUID_NOT_FOUND; // RETURN } - - out->load(new (*d_allocator_p) VirtualStorageIterator(this, it), - d_allocator_p); - - return mqbi::StorageResult::e_SUCCESS; -} - -mqbi::StorageResult::Enum VirtualStorage::releaseRef( - BSLS_ANNOTATION_UNUSED const bmqt::MessageGUID& msgGUID, - BSLS_ANNOTATION_UNUSED const mqbu::StorageKey& appKey, - BSLS_ANNOTATION_UNUSED bsls::Types::Int64 timestamp, - BSLS_ANNOTATION_UNUSED bool onReject) -{ - BSLS_ASSERT_OPT(false && "Should not be invoked."); - return mqbi::StorageResult::e_INVALID_OPERATION; } mqbi::StorageResult::Enum -VirtualStorage::remove(const bmqt::MessageGUID& msgGUID, - int* msgSize, - BSLS_ANNOTATION_UNUSED bool clearAll) - +VirtualStorage::remove(DataStreamMessage* dataStreamMessage) { - GuidList::const_iterator it = d_guids.find(msgGUID); - if (it == d_guids.end()) { - return mqbi::StorageResult::e_GUID_NOT_FOUND; // RETURN - } - - if (msgSize) { - *msgSize = it->second.d_size; - } - d_totalBytes -= it->second.d_size; - d_guids.erase(it); - return mqbi::StorageResult::e_SUCCESS; -} + mqbi::AppMessage& appMessage = dataStreamMessage->app(ordinal()); -mqbi::StorageResult::Enum VirtualStorage::removeAll( - BSLS_ANNOTATION_UNUSED const mqbu::StorageKey& appKey) -{ - d_guids.clear(); - d_totalBytes = 0; - return mqbi::StorageResult::e_SUCCESS; -} + if (appMessage.isPending()) { + appMessage.setRemovedState(); -void VirtualStorage::dispatcherFlush(bool, bool) -{ - BSLS_ASSERT_OPT(false && "Should not be invoked."); -} + d_removedBytes += dataStreamMessage->d_size; + ++d_numRemoved; -// ACCESSORS -mqbi::StorageResult::Enum -VirtualStorage::getMessageSize(int* msgSize, - const bmqt::MessageGUID& msgGUID) const -{ - GuidList::const_iterator cit = d_guids.find(msgGUID); - if (cit == d_guids.end()) { + return mqbi::StorageResult::e_SUCCESS; + } + else { + // already deleted return mqbi::StorageResult::e_GUID_NOT_FOUND; // RETURN } - - *msgSize = cit->second.d_size; - return mqbi::StorageResult::e_SUCCESS; } -int VirtualStorage::numVirtualStorages() const +void VirtualStorage::onGC(const DataStreamMessage& dataStreamMessage) { - BSLS_ASSERT_OPT(false && "Should not be invoked."); - return 0; -} + if (!dataStreamMessage.d_apps.empty()) { + const mqbi::AppMessage& appMessage = dataStreamMessage.app(ordinal()); -bool VirtualStorage::hasVirtualStorage( - BSLS_ANNOTATION_UNUSED const mqbu::StorageKey& appKey, - BSLS_ANNOTATION_UNUSED bsl::string* appId) const -{ - BSLS_ASSERT_OPT(false && "Should not be invoked."); - return false; + if (!appMessage.isPending()) { + d_removedBytes -= dataStreamMessage.d_size; + --d_numRemoved; + } + } } -bool VirtualStorage::hasVirtualStorage( - BSLS_ANNOTATION_UNUSED const bsl::string& appId, - BSLS_ANNOTATION_UNUSED mqbu::StorageKey* appKey) const +void VirtualStorage::resetStats() { - BSLS_ASSERT_OPT(false && "Should not be invoked."); - return false; + d_removedBytes = 0; + d_numRemoved = 0; } bool VirtualStorage::hasReceipt(const bmqt::MessageGUID& msgGUID) const @@ -261,66 +118,9 @@ bool VirtualStorage::hasReceipt(const bmqt::MessageGUID& msgGUID) const return d_storage_p->hasReceipt(msgGUID); } -void VirtualStorage::loadVirtualStorageDetails( - BSLS_ANNOTATION_UNUSED AppIdKeyPairs* buffer) const -{ - BSLS_ASSERT_OPT(false && "Should not be invoked."); -} - -unsigned int VirtualStorage::numAutoConfirms() const -{ - BSLS_ASSERT_OPT(false && "Should not be invoked."); - return 0; -} - -int VirtualStorage::gcExpiredMessages( - BSLS_ANNOTATION_UNUSED bsls::Types::Uint64* latestGcMsgTimestampEpoch, - BSLS_ANNOTATION_UNUSED bsls::Types::Int64* configuredTtlValue, - BSLS_ANNOTATION_UNUSED bsls::Types::Uint64 secondsFromEpoch) -{ - BSLS_ASSERT_OPT(false && "Should not be invoked."); - return 0; -} - -bool VirtualStorage::gcHistory() -{ - BSLS_ASSERT_OPT(false && "Should not be invoked."); - return false; -} - -int VirtualStorage::addVirtualStorage( - BSLS_ANNOTATION_UNUSED bsl::ostream& errorDescription, - BSLS_ANNOTATION_UNUSED const bsl::string& appId, - BSLS_ANNOTATION_UNUSED const mqbu::StorageKey& appKey) -{ - BSLS_ASSERT_OPT(false && "Should not be invoked."); - return -1; -} - -bool VirtualStorage::removeVirtualStorage( - BSLS_ANNOTATION_UNUSED const mqbu::StorageKey& appKey) -{ - BSLS_ASSERT_OPT(false && "Should not be invoked."); - return false; -} - -void VirtualStorage::selectForAutoConfirming(const bmqt::MessageGUID& msgGUID) -{ - BSLS_ASSERT_SAFE(false && "Should not be invoked."); -} - -mqbi::StorageResult::Enum -VirtualStorage::autoConfirm(const mqbu::StorageKey& appKey, - bsls::Types::Uint64 timestamp) -{ - BSLS_ASSERT_SAFE(false && "Should not be invoked."); - - return mqbi::StorageResult::e_INVALID_OPERATION; -} - -void VirtualStorage::autoConfirm(const bmqt::MessageGUID& msgGUID) +unsigned int VirtualStorage::ordinal() const { - d_autoConfirm = msgGUID; + return d_ordinal; } // ---------------------------- @@ -328,7 +128,7 @@ void VirtualStorage::autoConfirm(const bmqt::MessageGUID& msgGUID) // ---------------------------- // PRIVATE MANIPULATORS -void VirtualStorageIterator::clear() +void StorageIterator::clear() { // Clear previous state, if any. This is required so that new state can be // loaded in 'appData', 'options' or 'attributes' routines. @@ -339,16 +139,15 @@ void VirtualStorageIterator::clear() } // PRIVATE ACCESSORS -bool VirtualStorageIterator::loadMessageAndAttributes() const +bool StorageIterator::loadMessageAndAttributes() const { BSLS_ASSERT_SAFE(!atEnd()); if (!d_appData_sp) { - mqbi::StorageResult::Enum rc = d_virtualStorage_p->d_storage_p->get( - &d_appData_sp, - &d_options_sp, - &d_attributes, - d_iterator->first); + mqbi::StorageResult::Enum rc = d_storage_p->get(&d_appData_sp, + &d_options_sp, + &d_attributes, + d_iterator->first); BSLS_ASSERT_SAFE(mqbi::StorageResult::e_SUCCESS == rc); static_cast(rc); // suppress compiler warning return true; // RETURN @@ -357,26 +156,28 @@ bool VirtualStorageIterator::loadMessageAndAttributes() const } // CREATORS -VirtualStorageIterator::VirtualStorageIterator( - VirtualStorage* storage, - const VirtualStorage::GuidList::const_iterator& initialPosition) -: d_virtualStorage_p(storage) +StorageIterator::StorageIterator( + mqbi::Storage* storage, + VirtualStorageCatalog* owner, + const VirtualStorage::DataStream::iterator& initialPosition) +: d_storage_p(storage) +, d_owner_p(owner) , d_iterator(initialPosition) , d_attributes() , d_appData_sp() , d_options_sp() , d_haveReceipt(false) { - BSLS_ASSERT_SAFE(d_virtualStorage_p); + BSLS_ASSERT_SAFE(d_owner_p); } -VirtualStorageIterator::~VirtualStorageIterator() +StorageIterator::~StorageIterator() { // NOTHING } // MANIPULATORS -bool VirtualStorageIterator::advance() +bool StorageIterator::advance() { // PRECONDITIONS BSLS_ASSERT_SAFE(!atEnd()); @@ -386,16 +187,16 @@ bool VirtualStorageIterator::advance() return !atEnd(); } -void VirtualStorageIterator::reset() +void StorageIterator::reset(const bmqt::MessageGUID& where) { clear(); // Reset iterator to beginning - d_iterator = d_virtualStorage_p->d_guids.begin(); + d_iterator = d_owner_p->begin(where); } // ACCESSORS -const bmqt::MessageGUID& VirtualStorageIterator::guid() const +const bmqt::MessageGUID& StorageIterator::guid() const { // PRECONDITIONS BSLS_ASSERT_SAFE(!atEnd()); @@ -403,45 +204,54 @@ const bmqt::MessageGUID& VirtualStorageIterator::guid() const return d_iterator->first; } -bmqp::RdaInfo& VirtualStorageIterator::rdaInfo() const +const mqbi::AppMessage& +StorageIterator::appMessageView(unsigned int appOrdinal) const { // PRECONDITIONS BSLS_ASSERT_SAFE(!atEnd()); - return d_iterator->second.d_rdaInfo; + const VirtualStorage::DataStreamMessage& dataStreamMessage = + d_iterator->second; + + if (dataStreamMessage.d_apps.size() > appOrdinal) { + return d_iterator->second.app(appOrdinal); + } + return d_owner_p->defaultAppMessage(); } -unsigned int VirtualStorageIterator::subscriptionId() const +mqbi::AppMessage& StorageIterator::appMessageState(unsigned int appOrdinal) { // PRECONDITIONS BSLS_ASSERT_SAFE(!atEnd()); - return d_iterator->second.d_subscriptionId; + VirtualStorage::DataStreamMessage* dataStreamMessage = &d_iterator->second; + + d_owner_p->setup(dataStreamMessage); + + return dataStreamMessage->app(appOrdinal); } -const bsl::shared_ptr& VirtualStorageIterator::appData() const +const bsl::shared_ptr& StorageIterator::appData() const { loadMessageAndAttributes(); return d_appData_sp; } -const bsl::shared_ptr& VirtualStorageIterator::options() const +const bsl::shared_ptr& StorageIterator::options() const { loadMessageAndAttributes(); return d_options_sp; } -const mqbi::StorageMessageAttributes& -VirtualStorageIterator::attributes() const +const mqbi::StorageMessageAttributes& StorageIterator::attributes() const { // Do not load memory-mapped file message (expensive). if (d_attributes.refCount() == 0) { // No loaded Attributes for the current message yet. - mqbi::StorageResult::Enum rc = d_virtualStorage_p->d_storage_p->get( - &d_attributes, - d_iterator->first); + mqbi::StorageResult::Enum rc = d_storage_p->get(&d_attributes, + d_iterator->first); BSLS_ASSERT_SAFE(mqbi::StorageResult::e_SUCCESS == rc); (void)rc; } @@ -450,19 +260,19 @@ VirtualStorageIterator::attributes() const return d_attributes; } -bool VirtualStorageIterator::atEnd() const +bool StorageIterator::atEnd() const { - return (d_iterator == d_virtualStorage_p->d_guids.end()); + return (d_iterator == d_owner_p->end()); } -bool VirtualStorageIterator::hasReceipt() const +bool StorageIterator::hasReceipt() const { if (atEnd()) { return false; // RETURN } if (!d_haveReceipt) { // 'd_attributes.hasReceipt' can be stale. Double check by reloading - if (d_virtualStorage_p->d_storage_p->hasReceipt(d_iterator->first)) { + if (d_storage_p->hasReceipt(d_iterator->first)) { d_haveReceipt = true; } } @@ -470,5 +280,38 @@ bool VirtualStorageIterator::hasReceipt() const return d_haveReceipt; } +// CREATORS +VirtualStorageIterator::VirtualStorageIterator( + VirtualStorage* virtualStorage, + mqbi::Storage* storage, + VirtualStorageCatalog* owner, + const VirtualStorage::DataStream::iterator& initialPosition) +: StorageIterator(storage, owner, initialPosition) +, d_virtualStorage_p(virtualStorage) +{ + BSLS_ASSERT_SAFE(d_virtualStorage_p); +} + +VirtualStorageIterator::~VirtualStorageIterator() +{ + // NOTHING +} + +// MANIPULATORS +bool VirtualStorageIterator::advance() +{ + // PRECONDITIONS + BSLS_ASSERT_SAFE(d_virtualStorage_p); + + while (StorageIterator::advance()) { + if (StorageIterator::appMessageView(d_virtualStorage_p->ordinal()) + .isPending()) { + return true; // RETURN + } + } + + return false; +} + } // close package namespace } // close enterprise namespace diff --git a/src/groups/mqb/mqbs/mqbs_virtualstorage.h b/src/groups/mqb/mqbs/mqbs_virtualstorage.h index bc900cda4..ef4dded4c 100644 --- a/src/groups/mqb/mqbs/mqbs_virtualstorage.h +++ b/src/groups/mqb/mqbs/mqbs_virtualstorage.h @@ -62,41 +62,43 @@ namespace BloombergLP { namespace mqbs { // FORWARD DECLARATION -class VirtualStorageIterator; +class VirtualStorageCatalog; // ==================== // class VirtualStorage // ==================== -class VirtualStorage : public mqbi::Storage { - // TBD +class VirtualStorage { + // This Mechanism represents one App in a Storage (FileBased or InMemory) - private: - // FRIENDS - friend class VirtualStorageIterator; - - // PRIVATE TYPES - struct MessageContext { - int d_size; - mutable bmqp::RdaInfo d_rdaInfo; - unsigned int d_subscriptionId; - - MessageContext(int size, - const bmqp::RdaInfo& rdaInfo, - unsigned int subScriptionId); + public: + struct DataStreamMessage { + int d_size; + // The message size + + bsl::vector d_apps; + // App states for the message + + DataStreamMessage(int size, bslma::Allocator* allocator); + + /// Return reference to the modifiable state of the App corresponding + /// to the specified 'ordinal. + mqbi::AppMessage& app(unsigned int appOrdinal); + + /// Return reference to the non-modifiable state of the App + /// corresponding to the specified 'ordinal. + const mqbi::AppMessage& app(unsigned int appOrdinal) const; }; /// msgGUID -> MessageContext /// Must be a container in which iteration order is same as insertion /// order. typedef mwcc::OrderedHashMap > - GuidList; - - typedef GuidList::iterator GuidListIter; + DataStream; - typedef mqbi::Storage::StorageKeys StorageKeys; + typedef DataStream::iterator DataStreamIterator; private: // DATA @@ -106,21 +108,21 @@ class VirtualStorage : public mqbi::Storage { // underlying 'real' storage. Held. bsl::string d_appId; - // AppId of the consumer client which this - // instance represents. + // AppId of the App which this instance represents. mqbu::StorageKey d_appKey; // Storage key of the associated 'appId'. - GuidList d_guids; - // List of guids that are part of this storage. + bsls::Types::Int64 d_removedBytes; + // Cumulative count of all bytes _removed_ from this App + // The owner 'VirtualStorageCatalog' keeps track of all bytes - bsls::Types::Int64 d_totalBytes; - // Total size (in bytes) of all the messages that - // it holds. + bsls::Types::Int64 d_numRemoved; + // Cumulative count of all messages _removed_ from this App + // The owner 'VirtualStorageCatalog' keeps track of all messages - bmqt::MessageGUID d_autoConfirm; - // This App should not 'put' this guid because it is auto confirmed. + const unsigned int d_ordinal; + // The ordinal to locate corresponding state in 'DataStreamMessage' private: // NOT IMPLEMENTED @@ -134,272 +136,81 @@ class VirtualStorage : public mqbi::Storage { // CREATORS /// Create an instance of virtual storage backed by the specified real - /// `storage`, and having the specified `appId` and `appKey`, and use + /// `storage`, and having the specified `appId`, and `appKey`, and use /// the specified `allocator` for any memory allocations. Behavior is /// undefined unless `storage` is non-null, `appId` is non-empty and /// `appKey` is non-null. Note that the specified real `storage` must /// outlive this virtual storage instance. + /// The specified 'ordinal' uniquely identifies an offset for this instance + /// in 'DataStreamMessage'. VirtualStorage(mqbi::Storage* storage, const bsl::string& appId, const mqbu::StorageKey& appKey, + unsigned int ordinal, bslma::Allocator* allocator); /// Destructor. - ~VirtualStorage() BSLS_KEYWORD_OVERRIDE; + ~VirtualStorage(); // ACCESSORS - /// Return the URI of the queue this storage is associated with. - const bmqt::Uri& queueUri() const BSLS_KEYWORD_OVERRIDE; - - /// Return the storage key associated with this instance. - const mqbu::StorageKey& queueKey() const BSLS_KEYWORD_OVERRIDE; - /// Return the appId associated with this storage instance. Note that /// the returned string is always non-empty. - const bsl::string& appId() const BSLS_KEYWORD_OVERRIDE; + const bsl::string& appId() const; /// Return the app key, if any, associated with this storage instance. /// Note that the returned key is always non-null. - const mqbu::StorageKey& appKey() const BSLS_KEYWORD_OVERRIDE; - - /// Return the current configuration used by this storage. The behavior - /// is undefined unless `configure` was successfully called. - const mqbconfm::Storage& config() const BSLS_KEYWORD_OVERRIDE; + const mqbu::StorageKey& appKey() const; - /// Return the partitionId associated with this storage. - int partitionId() const BSLS_KEYWORD_OVERRIDE; - - /// Return true if storage is backed by a persistent data store, - /// otherwise return false. - bool isPersistent() const BSLS_KEYWORD_OVERRIDE; - - bsls::Types::Int64 - numMessages(const mqbu::StorageKey& appKey) const BSLS_KEYWORD_OVERRIDE; + bsls::Types::Int64 numRemoved() const; // Return the number of messages in the storage. - bsls::Types::Int64 - numBytes(const mqbu::StorageKey& appKey) const BSLS_KEYWORD_OVERRIDE; + bsls::Types::Int64 removedBytes() const; // Return the number of bytes in the storage. - /// Behavior is undefined if this method is ever invoked. This method - /// needs to be implemented as its part of base protocol. - bool isEmpty() const BSLS_KEYWORD_OVERRIDE; - - bool - hasMessage(const bmqt::MessageGUID& msgGUID) const BSLS_KEYWORD_OVERRIDE; - // Return true if this storage has message with the specified - // 'msgGUID', false otherwise. - - /// Behavior is undefined if this method is ever invoked. This method - /// needs to be implemented as its part of base protocol. - int numVirtualStorages() const BSLS_KEYWORD_OVERRIDE; - - /// Behavior is undefined if this method is ever invoked. This method - /// needs to be implemented as its part of base protocol. - bool hasVirtualStorage(const mqbu::StorageKey& appKey, - bsl::string* appId = 0) const BSLS_KEYWORD_OVERRIDE; - - /// Behavior is undefined if this method is ever invoked. This method - /// needs to be implemented as its part of base protocol. - bool hasVirtualStorage(const bsl::string& appId, - mqbu::StorageKey* appKey = 0) const - BSLS_KEYWORD_OVERRIDE; - /// Return `true` if there was Replication Receipt for the specified /// `msgGUID`. - bool - hasReceipt(const bmqt::MessageGUID& msgGUID) const BSLS_KEYWORD_OVERRIDE; - - /// Behavior is undefined if this method is ever invoked. This method - /// needs to be implemented as its part of base protocol. - void loadVirtualStorageDetails(AppIdKeyPairs* buffer) const - BSLS_KEYWORD_OVERRIDE; + bool hasReceipt(const bmqt::MessageGUID& msgGUID) const; - /// Behavior is undefined if these methods are ever invoked. These methods - /// need to be implemented as their part of the base protocol. - unsigned int numAutoConfirms() const BSLS_KEYWORD_OVERRIDE; - - /// Store in the specified `msgSize` the size, in bytes, of the message - /// having the specified `msgGUID` if found and return success, or - /// return a non-zero return code and leave `msgSize` untouched if no - /// message with `msgGUID` were found. - mqbi::StorageResult::Enum getMessageSize( - int* msgSize, - const bmqt::MessageGUID& msgGUID) const BSLS_KEYWORD_OVERRIDE; + /// Return the unique offset of this instance in 'DataStreamMessage'. + unsigned int ordinal() const; // MANIPULATORS - /// Behavior is undefined if this method is ever invoked. This method - /// needs to be implemented as its part of base protocol. - mqbu::CapacityMeter* capacityMeter() BSLS_KEYWORD_OVERRIDE; - - // ACCESSORS - - /// Retrieve the message and its metadata having the specified `msgGUID` - /// in the specified `appData`, `options` and `attributes` from this - /// storage. Return zero on success or a non-zero error code on - /// failure. - mqbi::StorageResult::Enum - get(bsl::shared_ptr* appData, - bsl::shared_ptr* options, - mqbi::StorageMessageAttributes* attributes, - const bmqt::MessageGUID& msgGUID) const BSLS_KEYWORD_OVERRIDE; - - /// Populate the specified `attributes` buffer with attributes of the - /// message having the specified `msgGUID`. Return zero on success or a - /// non-zero error code on failure. - mqbi::StorageResult::Enum - get(mqbi::StorageMessageAttributes* attributes, - const bmqt::MessageGUID& msgGUID) const BSLS_KEYWORD_OVERRIDE; + /// Change the state of this App in the specified 'dataStreamMessage' to + /// indicate CONFIRM. + mqbi::StorageResult::Enum confirm(DataStreamMessage* dataStreamMessage); - // MANIPULATORS + /// Change the state of this App in the specified 'dataStreamMessage' to + /// indicate removal (by a purge or unregistration). + mqbi::StorageResult::Enum remove(DataStreamMessage* dataStreamMessage); - /// Configure this storage using the specified `config` and `limits`. - /// Return 0 on success, or an non-zero return code and fill in a - /// description of the error in the specified `errorDescription` - /// otherwise. Note that calling `configure` on an already configured - /// storage should atomically reconfigure that storage with the new - /// configuration (or fail and leave the storage untouched). - int configure(bsl::ostream& errorDescription, - const mqbconfm::Storage& config, - const mqbconfm::Limits& limits, - const bsls::Types::Int64 messageTtl, - const int maxDeliveryAttempts) BSLS_KEYWORD_OVERRIDE; - - /// Behavior is undefined if this method is ever invoked. This method - /// needs to be implemented as its part of base protocol. - void setQueue(mqbi::Queue* queue) BSLS_KEYWORD_OVERRIDE; - - /// Behavior is undefined if this method is ever invoked. This method - /// needs to be implemented as its part of base protocol. - mqbi::Queue* queue() BSLS_KEYWORD_OVERRIDE; - - /// Close this storage. - void close() BSLS_KEYWORD_OVERRIDE; - - /// Save the message having the specified `msgGUID`, `msgSize`, and - /// `rdaInfo` into this virtual storage. Return 0 on success or an - /// non-zero error code on failure. - mqbi::StorageResult::Enum put(const bmqt::MessageGUID& msgGUID, - int msgSize, - const bmqp::RdaInfo& rdaInfo, - unsigned int subScriptionId); - - /// Behavior is undefined if this method is ever invoked. This method - /// needs to be implemented as its part of base protocol. Please call - /// put(const bmqt::MessageGUID& msgGUID, const int msgSize) instead. - mqbi::StorageResult::Enum - put(mqbi::StorageMessageAttributes* attributes, - const bmqt::MessageGUID& msgGUID, - const bsl::shared_ptr& appData, - const bsl::shared_ptr& options, - const StorageKeys& storageKeys = StorageKeys()) BSLS_KEYWORD_OVERRIDE; - - /// Get an iterator for items stored in the virtual storage identified - /// by the specified `appKey`. Iterator will point to point to the - /// oldest item, if any, or to the end of the collection if empty. Note - /// that if `appKey` is null, an iterator over the underlying physical - /// storage will be returned. Also note that because `Storage` and - /// `StorageIterator` are interfaces, the implementation of this method - /// will allocate, so it's recommended to keep the iterator. - bslma::ManagedPtr - getIterator(const mqbu::StorageKey& appKey) BSLS_KEYWORD_OVERRIDE; - - /// Load into the specified `out` an iterator for items stored in - /// the virtual storage identified by the specified `appKey`, initially - /// pointing to the item associated with the specified `msgGUID`. - /// Return zero on success, and a non-zero code if `msgGUID` was not - /// found in the storage. Note that if `appKey` is null, an iterator - /// over the underlying physical storage will be returned. Also note - /// that because `Storage` and `StorageIterator` are interfaces, the - /// implementation of this method will allocate, so it's recommended to - /// keep the iterator. - mqbi::StorageResult::Enum - getIterator(bslma::ManagedPtr* out, - const mqbu::StorageKey& appKey, - const bmqt::MessageGUID& msgGUID) BSLS_KEYWORD_OVERRIDE; - - /// Remove from the storage the message having the specified `msgGUID` - /// and store it's size, in bytes, in the optionally specified `msgSize` - /// if the `msgGUID` was found. Return 0 on success, or a non-zero - /// return code if the `msgGUID` was not found. The optionally - /// specified `clearAll` is ignored. - mqbi::StorageResult::Enum - remove(const bmqt::MessageGUID& msgGUID, - int* msgSize = 0, - bool clearAll = false) BSLS_KEYWORD_OVERRIDE; - - /// Behavior is undefined if this method is ever invoked. This method - /// needs to be implemented as its part of base protocol. - mqbi::StorageResult::Enum - releaseRef(const bmqt::MessageGUID& msgGUID, - const mqbu::StorageKey& appKey, - bsls::Types::Int64 timestamp, - bool onReject = false) BSLS_KEYWORD_OVERRIDE; - - /// Remove all entries from this storage. Specified `appKey` is unused. - /// This routine always returns success. - mqbi::StorageResult::Enum - removeAll(const mqbu::StorageKey& appKey) BSLS_KEYWORD_OVERRIDE; - - /// Behavior is undefined if this method is ever invoked. This method - /// needs to be implemented as its part of base protocol. - void dispatcherFlush(bool storage, bool queues) BSLS_KEYWORD_OVERRIDE; - - /// Attempt to garbage-collect messages for which TTL has expired, and - /// return the number of messages garbage-collected. Populate the - /// specified `latestGcMsgTimestampEpoch` with the timestamp, as seconds - /// from epoch, of the latest message garbage-collected due to TTL - /// expiration, and the specified `configuredTtlValue` with the TTL - /// value (in seconds) with which this storage instance is configured. - virtual int gcExpiredMessages( - bsls::Types::Uint64* latestGcMsgTimestampEpoch, - bsls::Types::Int64* configuredTtlValue, - bsls::Types::Uint64 secondsFromEpoch) BSLS_KEYWORD_OVERRIDE; - - /// Garbage-collect those messages from the deduplication history which - /// have expired the deduplication window. Return `true`, if there are - /// expired items unprocessed because of the batch limit. - virtual bool gcHistory() BSLS_KEYWORD_OVERRIDE; - - /// Behavior is undefined if this method is ever invoked. This method - /// needs to be implemented as its part of base protocol. - int - addVirtualStorage(bsl::ostream& errorDescription, - const bsl::string& appId, - const mqbu::StorageKey& appKey) BSLS_KEYWORD_OVERRIDE; - - /// Behavior is undefined if this method is ever invoked. This method - /// needs to be implemented as its part of base protocol. - bool - removeVirtualStorage(const mqbu::StorageKey& appKey) BSLS_KEYWORD_OVERRIDE; - - /// Behavior is undefined if these methods are ever invoked. These methods - /// need to be implemented as their part of the base protocol. - void selectForAutoConfirming(const bmqt::MessageGUID& msgGUID) - BSLS_KEYWORD_OVERRIDE; - mqbi::StorageResult::Enum - autoConfirm(const mqbu::StorageKey& appKey, - bsls::Types::Uint64 timestamp) BSLS_KEYWORD_OVERRIDE; + /// Observe removal of this App from the specified 'dataStreamMessage' by + /// GC and update bytes and messages counts if needed. + void onGC(const DataStreamMessage& dataStreamMessage); - /// Ignore the specified 'msgGUID' in the subsequent 'put' call because the - /// App has auto confirmed it. - void autoConfirm(const bmqt::MessageGUID& msgGUID); + /// Reset bytes and messages counts as in the case of purging all Apps. + void resetStats(); }; -// ============================ -// class VirtualStorageIterator -// ============================ +// ===================== +// class StorageIterator +// ===================== -class VirtualStorageIterator : public mqbi::StorageIterator { - // TBD +class StorageIterator : public mqbi::StorageIterator { + // Mechanism to provide access to both underlying real storage (FileBased + // or InMemory) and all App states in Virtual Storage. private: // DATA - VirtualStorage* d_virtualStorage_p; + mqbi::Storage* d_storage_p; + // underlying 'real' storage (FileBased or InMemory). + + VirtualStorageCatalog* d_owner_p; + // The owner and creator. - VirtualStorage::GuidList::const_iterator d_iterator; + VirtualStorage::DataStreamIterator d_iterator; + // Access to App states in Virtual Storages. mutable mqbi::StorageMessageAttributes d_attributes; @@ -413,13 +224,12 @@ class VirtualStorageIterator : public mqbi::StorageIterator { mutable bsl::shared_ptr d_options_sp; mutable bool d_haveReceipt; - // Avoid reading Attributes. + // Cached value. private: // NOT IMPLEMENTED - VirtualStorageIterator(const VirtualStorageIterator&); // = delete - VirtualStorageIterator& - operator=(const VirtualStorageIterator&); // = delete + StorageIterator(const StorageIterator&); // = delete + StorageIterator& operator=(const StorageIterator&); // = delete private: // PRIVATE MANIPULATORS @@ -441,18 +251,25 @@ class VirtualStorageIterator : public mqbi::StorageIterator { public: // CREATORS - /// Create a new VirtualStorageIterator from the specified `storage` and - /// pointing at the specified `initialPosition`. - VirtualStorageIterator( - VirtualStorage* storage, - const VirtualStorage::GuidList::const_iterator& initialPosition); + /// Create a new VirtualStorageIterator for the specified 'storage' and + /// 'owner' pointing at the specified 'initialPosition'. + StorageIterator( + mqbi::Storage* storage, + VirtualStorageCatalog* owner, + const VirtualStorage::DataStream::iterator& initialPosition); /// Destructor - ~VirtualStorageIterator() BSLS_KEYWORD_OVERRIDE; + ~StorageIterator() BSLS_KEYWORD_OVERRIDE; // MANIPULATORS bool advance() BSLS_KEYWORD_OVERRIDE; - void reset() BSLS_KEYWORD_OVERRIDE; + + /// If the specified 'where' is unset, reset the iterator to point to the + /// to the beginning of the Virtual Storage. Otherwise, reset the + /// iterator to point to the item corresponding to the 'where'. If the + /// item is not found, reset the iterator to the end of the storage. + void reset(const bmqt::MessageGUID& where = bmqt::MessageGUID()) + BSLS_KEYWORD_OVERRIDE; // ACCESSORS @@ -461,15 +278,19 @@ class VirtualStorageIterator : public mqbi::StorageIterator { /// behavior is undefined unless `atEnd` returns `false`. const bmqt::MessageGUID& guid() const BSLS_KEYWORD_OVERRIDE; - /// Return a reference offering modifiable access to the RdaInfo - /// associated to the item currently pointed at by this iterator. The - /// behavior is undefined unless `atEnd` returns `false`. - bmqp::RdaInfo& rdaInfo() const BSLS_KEYWORD_OVERRIDE; + /// Return a reference offering non-modifiable access to the App state + /// corresponding to the specified 'ordinal' and the item currently pointed + /// at by this iterator. + /// The behavior is undefined unless `atEnd` returns `false`. + const mqbi::AppMessage& + appMessageView(unsigned int appOrdinal) const BSLS_KEYWORD_OVERRIDE; - /// Return subscription id associated to the item currently pointed at - /// by this iterator. + /// Return a reference offering modifiable access to the App state + /// corresponding to the specified 'ordinal' and the item currently pointed + /// at by this iterator. /// The behavior is undefined unless `atEnd` returns `false`. - unsigned int subscriptionId() const BSLS_KEYWORD_OVERRIDE; + mqbi::AppMessage& + appMessageState(unsigned int appOrdinal) BSLS_KEYWORD_OVERRIDE; /// Return a reference offering non-modifiable access to the application /// data associated with the item currently pointed at by this iterator. @@ -497,97 +318,102 @@ class VirtualStorageIterator : public mqbi::StorageIterator { bool hasReceipt() const BSLS_KEYWORD_OVERRIDE; }; +// ============================ +// class VirtualStorageIterator +// ============================ + +class VirtualStorageIterator : public StorageIterator { + // Mechanism to provide access to both underlying real storage (FileBased + // or InMemory) and one App states in Virtual Storage. + private: + // DATA + VirtualStorage* d_virtualStorage_p; + + private: + // NOT IMPLEMENTED + VirtualStorageIterator(const VirtualStorageIterator&); // = delete + VirtualStorageIterator& + operator=(const VirtualStorageIterator&); // = delete + + public: + // CREATORS + + /// Create a new VirtualStorageIterator for the specified 'storage' and + /// 'owner' pointing at the specified 'initialPosition'. The specified + /// 'virtualStorage' identifies the App which states this object iterates. + VirtualStorageIterator( + VirtualStorage* virtualStorage, + mqbi::Storage* storage, + VirtualStorageCatalog* owner, + const VirtualStorage::DataStream::iterator& initialPosition); + + /// Destructor + ~VirtualStorageIterator() BSLS_KEYWORD_OVERRIDE; + + // MANIPULATORS + + /// Advance the iterator to the next messages which has the App state as + /// pending. + bool advance() BSLS_KEYWORD_OVERRIDE; +}; + // ============================================================================ // INLINE DEFINITIONS // ============================================================================ -// ------------------------------------ -// class VirtualStorage::MessageContext -// ------------------------------------ +// --------------------------------------- +// class VirtualStorage::DataStreamMessage +// --------------------------------------- -inline VirtualStorage::MessageContext::MessageContext( - int size, - const bmqp::RdaInfo& rdaInfo, - unsigned int subScriptionId) +inline VirtualStorage::DataStreamMessage::DataStreamMessage( + int size, + bslma::Allocator* allocator) : d_size(size) -, d_rdaInfo(rdaInfo) -, d_subscriptionId(subScriptionId) +, d_apps(allocator) { // NOTHING } -// -------------------- -// class VirtualStorage -// -------------------- - -// ACCESSORS -inline const bmqt::Uri& VirtualStorage::queueUri() const +inline mqbi::AppMessage& +VirtualStorage::DataStreamMessage::app(unsigned int appOrdinal) { - return d_storage_p->queueUri(); -} + BSLS_ASSERT_SAFE(appOrdinal < d_apps.size()); -inline const mqbu::StorageKey& VirtualStorage::queueKey() const -{ - return d_storage_p->queueKey(); + return d_apps[appOrdinal]; } -inline const bsl::string& VirtualStorage::appId() const +inline const mqbi::AppMessage& +VirtualStorage::DataStreamMessage::app(unsigned int appOrdinal) const { - return d_appId; -} + BSLS_ASSERT_SAFE(appOrdinal < d_apps.size()); -inline const mqbu::StorageKey& VirtualStorage::appKey() const -{ - return d_appKey; + return d_apps[appOrdinal]; } -inline const mqbconfm::Storage& VirtualStorage::config() const -{ - return d_storage_p->config(); -} - -inline int VirtualStorage::partitionId() const -{ - return mqbs::DataStore::k_INVALID_PARTITION_ID; -} - -inline bool VirtualStorage::isPersistent() const -{ - return d_storage_p->isPersistent(); -} +// -------------------- +// class VirtualStorage +// -------------------- -inline mqbu::CapacityMeter* VirtualStorage::capacityMeter() -{ - BSLS_ASSERT_OPT(false && "Should not be invoked."); - return static_cast(0); -} +// ACCESSORS -inline bsls::Types::Int64 VirtualStorage::numMessages( - BSLS_ANNOTATION_UNUSED const mqbu::StorageKey& appKey) const +inline const bsl::string& VirtualStorage::appId() const { - return d_guids.size(); + return d_appId; } -inline bsls::Types::Int64 VirtualStorage::numBytes( - BSLS_ANNOTATION_UNUSED const mqbu::StorageKey& appKey) const +inline const mqbu::StorageKey& VirtualStorage::appKey() const { - return d_totalBytes; + return d_appKey; } -inline bool VirtualStorage::isEmpty() const +inline bsls::Types::Int64 VirtualStorage::numRemoved() const { - // executed by *ANY* thread - - BSLS_ASSERT_OPT(false && "Should not be invoked."); - // If needed, can be implemented in a manner similar to - // 'mqbs::InMemoryStorage' and 'mqbs::FileBackedStorage'. - - return false; + return d_numRemoved; // TODO } -inline bool VirtualStorage::hasMessage(const bmqt::MessageGUID& msgGUID) const +inline bsls::Types::Int64 VirtualStorage::removedBytes() const { - return 1 == d_guids.count(msgGUID); + return d_removedBytes; } } // close package namespace diff --git a/src/groups/mqb/mqbs/mqbs_virtualstorage.t.cpp b/src/groups/mqb/mqbs/mqbs_virtualstorage.t.cpp deleted file mode 100644 index c65e440e1..000000000 --- a/src/groups/mqb/mqbs/mqbs_virtualstorage.t.cpp +++ /dev/null @@ -1,802 +0,0 @@ -// Copyright 2023 Bloomberg Finance L.P. -// SPDX-License-Identifier: Apache-2.0 -// -// 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 -// -// http://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. - -// mqbs_virtualstorage.t.cpp -*-C++-*- -#include - -// MQB -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -// BMQ -#include -#include - -// BDE -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -// MWC -#include - -// TEST DRIVER -#include - -// CONVENIENCE -using namespace BloombergLP; -using namespace bsl; - -//============================================================================= -// TEST PLAN -//----------------------------------------------------------------------------- -// - breathingTest -// - unsupportedOperations -// - put -// - hasMessage -// - getMessageSize -// - get -// - remove -// - removeAll -// - getIterator -//----------------------------------------------------------------------------- - -// ============================================================================ -// TEST HELPERS UTILITY -// ---------------------------------------------------------------------------- - -namespace { - -// CONSTANTS -const int k_PARTITION_ID = 1; -const char k_HEX_QUEUE[] = "ABCDEF1234"; -const char k_URI_STR[] = "bmq://mydomain/testqueue"; -const char k_APP_ID[] = "ABCDEF1111"; -const mqbu::StorageKey k_QUEUE_KEY(mqbu::StorageKey::HexRepresentation(), - k_HEX_QUEUE); -const mqbu::StorageKey k_APP_KEY(mqbu::StorageKey::HexRepresentation(), - k_APP_ID); -const unsigned int k_DEFAULT_MSG_SIZE = 25; - -// ALIASES -typedef bsl::vector MessageGuids; -typedef MessageGuids::iterator MessageGuidsI; -typedef MessageGuids::const_iterator MessageGuidsCI; - -const bsls::Types::Int64 k_INT64_ZERO = 0; -const bsls::Types::Int64 k_INT64_MAX = - bsl::numeric_limits::max(); - -// FUNCTIONS - -/// Load into the specified `guids` a new message GUID unique from all -/// existing GUIDs in `guids`, and return it. -static bmqt::MessageGUID generateUniqueGUID(MessageGuids* guids) -{ - bmqt::MessageGUID uniqueGUID; - do { - mqbu::MessageGUIDUtil::generateGUID(&uniqueGUID); - } while (bsl::find(guids->begin(), guids->end(), uniqueGUID) != - guids->end()); - - guids->push_back(uniqueGUID); - return uniqueGUID; -} - -// CLASSES -// ============= -// struct Tester -// ============= -struct Tester { - private: - // DATA - bdlbb::PooledBlobBufferFactory d_bufferFactory; - mqbmock::Cluster d_mockCluster; - mqbmock::Domain d_mockDomain; - mqbmock::Queue d_mockQueue; - mqbmock::QueueEngine d_mockQueueEngine; - bsl::shared_ptr d_storage_sp; - bslma::ManagedPtr d_virtualStorage_mp; - bslma::Allocator* d_allocator_p; - - public: - // CREATORS - Tester(const bslstl::StringRef& uri = k_URI_STR, - const mqbu::StorageKey& queueKey = k_QUEUE_KEY, - int partitionId = k_PARTITION_ID, - bsls::Types::Int64 ttlSeconds = k_INT64_MAX, - bslma::Allocator* allocator = s_allocator_p) - : d_bufferFactory(1024, allocator) - , d_mockCluster(&d_bufferFactory, allocator) - , d_mockDomain(&d_mockCluster, allocator) - , d_mockQueue(&d_mockDomain, allocator) - , d_mockQueueEngine(allocator) - , d_allocator_p(allocator) - { - d_mockDomain.capacityMeter()->setLimits(k_INT64_MAX, k_INT64_MAX); - d_mockQueue._setQueueEngine(&d_mockQueueEngine); - - mqbconfm::Domain domainCfg; - domainCfg.deduplicationTimeMs() = 0; // No history - domainCfg.messageTtl() = ttlSeconds; - - d_storage_sp.load(new (*d_allocator_p) mqbs::InMemoryStorage( - bmqt::Uri(uri, d_allocator_p), - queueKey, - partitionId, - domainCfg, - d_mockDomain.capacityMeter(), - bmqp::RdaInfo(), - d_allocator_p), - d_allocator_p); - d_storage_sp->setQueue(&d_mockQueue); - BSLS_ASSERT_OPT(d_storage_sp->queue() == &d_mockQueue); - - d_virtualStorage_mp.load(new (*d_allocator_p) - mqbs::VirtualStorage(d_storage_sp.get(), - k_APP_ID, - k_APP_KEY, - d_allocator_p), - d_allocator_p); - } - - ~Tester() - { - d_virtualStorage_mp->removeAll(mqbu::StorageKey::k_NULL_KEY); - d_virtualStorage_mp->close(); - d_storage_sp->removeAll(mqbu::StorageKey::k_NULL_KEY); - d_storage_sp->close(); - } - - // MANIPULATORS - int configure(bsls::Types::Int64 msgCapacity, - bsls::Types::Int64 byteCapacity, - double msgWatermarkRatio = 0.8, - double byteWatermarkRatio = 0.8, - bsls::Types::Int64 messageTtl = k_INT64_MAX) - { - // PRECONDITIONS - BSLS_ASSERT_OPT(d_storage_sp && "Storage was not created"); - - mqbconfm::Storage config; - mqbconfm::Limits limits; - - config.makeInMemory(); - - limits.messages() = msgCapacity; - limits.messagesWatermarkRatio() = msgWatermarkRatio; - limits.bytes() = byteCapacity; - limits.bytesWatermarkRatio() = byteWatermarkRatio; - - mwcu::MemOutStream errDescription(s_allocator_p); - return d_storage_sp->configure(errDescription, - config, - limits, - messageTtl, - 0); // maxDeliveryAttempts - } - - mqbi::StorageResult::Enum addPhysicalMessages(const MessageGuids guids, - const int dataOffset = 0) - { - for (size_t i = 0; i != guids.size(); i++) { - const bmqt::MessageGUID& guid = guids[i]; - - mqbi::StorageMessageAttributes attributes( - static_cast(dataOffset + i), - 1, - bmqp::MessagePropertiesInfo::makeNoSchema(), - bmqt::CompressionAlgorithmType::e_NONE); - - const bsl::shared_ptr appDataPtr( - new (*d_allocator_p) - bdlbb::Blob(&d_bufferFactory, d_allocator_p), - d_allocator_p); - const int data = i + dataOffset; - bdlbb::BlobUtil::append(&(*appDataPtr), - reinterpret_cast(&data), - static_cast(sizeof(int))); - - mqbi::StorageResult::Enum rc = - d_storage_sp->put(&attributes, guid, appDataPtr, appDataPtr); - PV("rc = " << rc); - if (rc != mqbi::StorageResult::e_SUCCESS) { - return rc; // RETURN - } - } - - return mqbi::StorageResult::e_SUCCESS; - } - - mqbs::VirtualStorage& vStorage() { return *d_virtualStorage_mp.ptr(); } - - mqbi::Storage& storage() { return *d_storage_sp.ptr(); } - - mqbmock::Queue& mockQueue() { return d_mockQueue; } -}; - -} // close anonymous namespace - -// ============================================================================ -// TESTS -// ---------------------------------------------------------------------------- - -static void test1_breathingTest() -// ------------------------------------------------------------------------ -// BREATHING TEST -// -// Concerns: -// Exercise the basic functionality of the component. -// -// Testing: -// Basic functionality -// ------------------------------------------------------------------------ -{ - mwctst::TestHelper::printTestName("BREATHING TEST"); - Tester tester; - - ASSERT_EQ(tester.vStorage().queueUri().asString(), k_URI_STR); - ASSERT_EQ(tester.vStorage().queueKey(), k_QUEUE_KEY); - ASSERT_EQ(tester.vStorage().appId(), k_APP_ID); - ASSERT_EQ(tester.vStorage().appKey(), k_APP_KEY); - ASSERT_EQ(tester.vStorage().config(), mqbconfm::Storage()); - ASSERT_EQ(tester.vStorage().isPersistent(), false); - ASSERT_EQ(tester.vStorage().numMessages(k_APP_KEY), k_INT64_ZERO); - ASSERT_EQ(tester.vStorage().numBytes(k_APP_KEY), k_INT64_ZERO); -} - -static void test2_unsupportedOperations() -// ------------------------------------------------------------------------ -// UNSUPPORTED OPERATIONS -// -// Concerns: -// A 'mqbs::VirtualStorage' implements, but does not adhere to, some -// operations declared in its interface, in the sense that these methods -// fail to execute. -// -// Testing: -// queue() -// capacityMeter() -// isEmpty() -// setQueue(...) -// put(msgGUID, attributes, appData, options, storageKeys) -// releaseRef(...) -// dispatcherFlush() -// numVirtualStorages() -// hasVirtualStorage(...) -// loadVirtualStorageDetails(...) -// gc(...) -// addVirtualStorage(...) -// removeVirtualStorage(...) -// ------------------------------------------------------------------------ -{ - mwctst::TestHelper::printTestName("UNSUPPORTED OPRATIONS"); - Tester tester; - - bmqt::MessageGUID guid; - mqbi::StorageMessageAttributes attributes; - bsl::shared_ptr appData; - bsl::shared_ptr options; - mqbi::Storage::AppIdKeyPairs appIdKeyPairs; - bsls::Types::Uint64 latestMsgTimestamp; - bsls::Types::Int64 configuredTtlValue; - bsls::Types::Uint64 secondsFromEpoch = 5; - mwcu::MemOutStream errDescription(s_allocator_p); - - ASSERT_OPT_FAIL(tester.vStorage().queue()); - ASSERT_OPT_FAIL(tester.vStorage().capacityMeter()); - ASSERT_OPT_FAIL(tester.vStorage().isEmpty()); - ASSERT_OPT_FAIL(tester.vStorage().setQueue(&tester.mockQueue())); - ASSERT_OPT_FAIL( - tester.vStorage().put(&attributes, guid, appData, options)); - ASSERT_OPT_FAIL(tester.vStorage().releaseRef(guid, k_APP_KEY, 0)); - ASSERT_OPT_FAIL(tester.vStorage().dispatcherFlush(true, false)); - ASSERT_OPT_FAIL(tester.vStorage().numVirtualStorages()); - ASSERT_OPT_FAIL(tester.vStorage().hasVirtualStorage(k_APP_KEY)); - ASSERT_OPT_FAIL(tester.vStorage().hasVirtualStorage(k_APP_ID)); - ASSERT_OPT_FAIL( - tester.vStorage().loadVirtualStorageDetails(&appIdKeyPairs)); - ASSERT_OPT_FAIL(tester.vStorage().gcExpiredMessages(&latestMsgTimestamp, - &configuredTtlValue, - secondsFromEpoch)); - ASSERT_OPT_FAIL(tester.vStorage().addVirtualStorage(errDescription, - k_APP_ID, - k_APP_KEY)); - ASSERT_OPT_FAIL(tester.vStorage().removeVirtualStorage(k_APP_KEY)); -} - -static void test3_put() -// ------------------------------------------------------------------------ -// PUT -// -// Concerns: -// Verify that 'put' works as intended. -// -// Testing: -// put(...) -// ------------------------------------------------------------------------ -{ - mwctst::TestHelper::printTestName("PUT"); - Tester tester; - - // Put 20 new messages - MessageGuids guids; - bmqt::MessageGUID newGuid; - const int k_MSG_COUNT = 20; - for (int i = 1; i <= k_MSG_COUNT; ++i) { - newGuid = generateUniqueGUID(&guids); - ASSERT_EQ( - tester.vStorage().put(newGuid, - k_DEFAULT_MSG_SIZE, - bmqp::RdaInfo(), - bmqp::Protocol::k_DEFAULT_SUBSCRIPTION_ID), - mqbi::StorageResult::e_SUCCESS); - ASSERT_EQ(tester.vStorage().numMessages(k_APP_KEY), i); - ASSERT_EQ(tester.vStorage().numBytes(k_APP_KEY), - i * k_DEFAULT_MSG_SIZE); - } - - int msgSize; - for (MessageGuidsCI cit = guids.cbegin(); cit != guids.cend(); ++cit) { - const bmqt::MessageGUID& guid = *cit; - ASSERT(tester.vStorage().hasMessage(guid)); - ASSERT_EQ(tester.vStorage().getMessageSize(&msgSize, guid), - mqbi::StorageResult::e_SUCCESS); - ASSERT_EQ(static_cast(msgSize), k_DEFAULT_MSG_SIZE); - } - - // Inserting duplicated messageGUID should not succeed and should have no - // side effect - for (MessageGuidsCI cit = guids.cbegin(); cit != guids.cend(); ++cit) { - const bmqt::MessageGUID& guid = *cit; - ASSERT_EQ( - tester.vStorage().put(guid, - k_DEFAULT_MSG_SIZE, - bmqp::RdaInfo(), - bmqp::Protocol::k_DEFAULT_SUBSCRIPTION_ID), - mqbi::StorageResult::e_GUID_NOT_UNIQUE); - // For now, this still returns e_SUCCESS. - ASSERT_EQ(tester.vStorage().numMessages(k_APP_KEY), k_MSG_COUNT); - ASSERT_EQ(tester.vStorage().numBytes(k_APP_KEY), - k_MSG_COUNT * k_DEFAULT_MSG_SIZE); - ASSERT(tester.vStorage().hasMessage(guid)); - ASSERT_EQ(tester.vStorage().getMessageSize(&msgSize, guid), - mqbi::StorageResult::e_SUCCESS); - ASSERT_EQ(static_cast(msgSize), k_DEFAULT_MSG_SIZE); - } -} - -static void test4_hasMessage() -// ------------------------------------------------------------------------ -// HAS MESSAGE -// -// Concerns: -// Verify that 'hasMessage' works as intended. -// -// Testing: -// hasMessage(...) -// ------------------------------------------------------------------------ -{ - mwctst::TestHelper::printTestName("HAS MESSAGE"); - Tester tester; - - // Put 10 new messages - MessageGuids guids; - bmqt::MessageGUID newGuid; - const int k_MSG_COUNT = 10; - for (int i = 1; i <= k_MSG_COUNT; ++i) { - newGuid = generateUniqueGUID(&guids); - BSLS_ASSERT_OPT( - tester.vStorage().put(newGuid, - k_DEFAULT_MSG_SIZE, - bmqp::RdaInfo(), - bmqp::Protocol::k_DEFAULT_SUBSCRIPTION_ID) == - mqbi::StorageResult::e_SUCCESS); - BSLS_ASSERT_OPT(tester.vStorage().numMessages(k_APP_KEY) == i); - BSLS_ASSERT_OPT(tester.vStorage().numBytes(k_APP_KEY) == - i * k_DEFAULT_MSG_SIZE); - } - - for (MessageGuidsCI cit = guids.cbegin(); cit != guids.cend(); ++cit) { - ASSERT(tester.vStorage().hasMessage(*cit)); - } - - // Check 'hasMessage' with non-existent GUIDs - for (int i = 1; i <= k_MSG_COUNT; ++i) { - newGuid = generateUniqueGUID(&guids); - ASSERT(!tester.vStorage().hasMessage(newGuid)); - } -} - -static void test5_getMessageSize() -// ------------------------------------------------------------------------ -// GET MESSAGE SIZE -// -// Concerns: -// Verify that 'getMessageSize' works as intended. -// -// Testing: -// getMessageSize(...) -// ------------------------------------------------------------------------ -{ - mwctst::TestHelper::printTestName("GET MESSAGE SIZE"); - Tester tester; - - // Put 10 new messages - MessageGuids guids; - bmqt::MessageGUID newGuid; - const int k_MSG_COUNT = 10; - for (int i = 1; i <= k_MSG_COUNT; ++i) { - newGuid = generateUniqueGUID(&guids); - BSLS_ASSERT_OPT( - tester.vStorage().put(newGuid, - k_DEFAULT_MSG_SIZE, - bmqp::RdaInfo(), - bmqp::Protocol::k_DEFAULT_SUBSCRIPTION_ID) == - mqbi::StorageResult::e_SUCCESS); - BSLS_ASSERT_OPT(tester.vStorage().numMessages(k_APP_KEY) == i); - BSLS_ASSERT_OPT(tester.vStorage().numBytes(k_APP_KEY) == - i * k_DEFAULT_MSG_SIZE); - } - - int msgSize; - for (MessageGuidsCI cit = guids.cbegin(); cit != guids.cend(); - ++cit, msgSize = 0) { - const bmqt::MessageGUID& guid = *cit; - BSLS_ASSERT_OPT(tester.vStorage().hasMessage(guid)); - ASSERT_EQ(tester.vStorage().getMessageSize(&msgSize, guid), - mqbi::StorageResult::e_SUCCESS); - ASSERT_EQ(static_cast(msgSize), k_DEFAULT_MSG_SIZE); - } - - // Check 'getMessageSize' with non-existent GUIDs - for (int i = 1; i <= k_MSG_COUNT; ++i, msgSize = 0) { - newGuid = generateUniqueGUID(&guids); - BSLS_ASSERT_OPT(!tester.vStorage().hasMessage(newGuid)); - ASSERT_EQ(tester.vStorage().getMessageSize(&msgSize, newGuid), - mqbi::StorageResult::e_GUID_NOT_FOUND); - ASSERT_EQ(msgSize, 0); - } -} - -static void test6_get() -// ------------------------------------------------------------------------ -// GET -// -// Concerns: -// Verify that 'get' works as intended. -// -// Testing: -// get(...) -// ------------------------------------------------------------------------ -{ - mwctst::TestHelper::printTestName("GET"); - Tester tester; - - // Put 10 new messages - MessageGuids guids; - bmqt::MessageGUID newGuid; - const int k_MSG_COUNT = 10; - for (int i = 1; i <= k_MSG_COUNT; ++i) { - newGuid = generateUniqueGUID(&guids); - BSLS_ASSERT_OPT( - tester.vStorage().put(newGuid, - k_DEFAULT_MSG_SIZE, - bmqp::RdaInfo(), - bmqp::Protocol::k_DEFAULT_SUBSCRIPTION_ID) == - mqbi::StorageResult::e_SUCCESS); - BSLS_ASSERT_OPT(tester.vStorage().numMessages(k_APP_KEY) == i); - BSLS_ASSERT_OPT(tester.vStorage().numBytes(k_APP_KEY) == - i * k_DEFAULT_MSG_SIZE); - } - - // Since mqbs::VirtualStorage::get() retrieves the physical message - // from its underlying physical storage, we must store the message - // there as well. - BSLS_ASSERT_OPT(tester.configure(k_INT64_MAX, k_INT64_MAX) == 0); - BSLS_ASSERT_OPT(tester.addPhysicalMessages(guids) == - mqbi::StorageResult::e_SUCCESS); - - // Check 'get' overloads - for (int i = 0; i < k_MSG_COUNT; ++i) { - { - mqbi::StorageMessageAttributes attributes; - ASSERT_EQ(tester.storage().get(&attributes, guids[i]), - mqbi::StorageResult::e_SUCCESS); - ASSERT_EQ(attributes.arrivalTimestamp(), - static_cast(i)); - ASSERT_EQ(attributes.refCount(), static_cast(1)); - ASSERT(attributes.messagePropertiesInfo().isPresent()); - } - - { - mqbi::StorageMessageAttributes attributes; - bsl::shared_ptr appData; - bsl::shared_ptr options; - ASSERT_EQ(tester.storage().get(&appData, - &options, - &attributes, - guids[i]), - mqbi::StorageResult::e_SUCCESS); - - ASSERT_EQ(attributes.arrivalTimestamp(), - static_cast(i)); - ASSERT_EQ(attributes.refCount(), static_cast(1)); - ASSERT(attributes.messagePropertiesInfo().isPresent()); - ASSERT_EQ(*(reinterpret_cast(appData->buffer(0).data())), i); - } - } - - // Check 'get' with a non-existent GUID - mqbi::StorageMessageAttributes attributes; - bsl::shared_ptr appData; - bsl::shared_ptr options; - ASSERT_EQ(tester.storage().get(&attributes, generateUniqueGUID(&guids)), - mqbi::StorageResult::e_GUID_NOT_FOUND); - ASSERT_EQ(tester.storage().get(&appData, - &options, - &attributes, - generateUniqueGUID(&guids)), - mqbi::StorageResult::e_GUID_NOT_FOUND); -} - -static void test7_remove() -// ------------------------------------------------------------------------ -// REMOVE -// -// Concerns: -// Verify that 'remove' works as intended. -// -// Testing: -// remove(...) -// ------------------------------------------------------------------------ -{ - mwctst::TestHelper::printTestName("REMOVE"); - Tester tester; - - // Put 20 new messages - MessageGuids guids; - bmqt::MessageGUID newGuid; - const int k_MSG_COUNT = 20; - for (int i = 1; i <= k_MSG_COUNT; ++i) { - newGuid = generateUniqueGUID(&guids); - - BSLS_ASSERT_OPT( - tester.vStorage().put(newGuid, - k_DEFAULT_MSG_SIZE, - bmqp::RdaInfo(), - bmqp::Protocol::k_DEFAULT_SUBSCRIPTION_ID) == - mqbi::StorageResult::e_SUCCESS); - BSLS_ASSERT_OPT(tester.vStorage().numMessages(k_APP_KEY) == i); - BSLS_ASSERT_OPT(tester.vStorage().numBytes(k_APP_KEY) == - i * k_DEFAULT_MSG_SIZE); - } - - // Remove half of those messages - int msgSize; - for (int i = 0; i < k_MSG_COUNT / 2; ++i) { - const bmqt::MessageGUID& guid = guids[i]; - BSLS_ASSERT_OPT(tester.vStorage().hasMessage(guid)); - BSLS_ASSERT_OPT(tester.vStorage().getMessageSize(&msgSize, guid) == - mqbi::StorageResult::e_SUCCESS); - BSLS_ASSERT_OPT(static_cast(msgSize) == - k_DEFAULT_MSG_SIZE); - - ASSERT_EQ(tester.vStorage().remove(guid, &msgSize), - mqbi::StorageResult::e_SUCCESS) - ASSERT_EQ(static_cast(msgSize), k_DEFAULT_MSG_SIZE); - ASSERT(!tester.vStorage().hasMessage(guid)); - ASSERT_EQ(tester.vStorage().getMessageSize(&msgSize, guid), - mqbi::StorageResult::e_GUID_NOT_FOUND); - ASSERT_EQ(tester.vStorage().numMessages(k_APP_KEY), - k_MSG_COUNT - i - 1); - ASSERT_EQ(tester.vStorage().numBytes(k_APP_KEY), - (k_MSG_COUNT - i - 1) * k_DEFAULT_MSG_SIZE); - } - - // Sanity check: the remaining half is not removed - for (int i = k_MSG_COUNT / 2; i < k_MSG_COUNT; ++i) { - const bmqt::MessageGUID& guid = guids[i]; - ASSERT(tester.vStorage().hasMessage(guid)); - } -} - -static void test8_removeAll() -// ------------------------------------------------------------------------ -// REMOVE ALL -// -// Concerns: -// Verify that 'removeAll' works as intended. -// -// Testing: -// removeAll(...) -// ------------------------------------------------------------------------ -{ - mwctst::TestHelper::printTestName("REMOVE ALL"); - Tester tester; - - // Put 20 new messages - MessageGuids guids; - bmqt::MessageGUID newGuid; - const int k_MSG_COUNT = 20; - for (int i = 1; i <= k_MSG_COUNT; ++i) { - newGuid = generateUniqueGUID(&guids); - - BSLS_ASSERT_OPT( - tester.vStorage().put(newGuid, - k_DEFAULT_MSG_SIZE, - bmqp::RdaInfo(), - bmqp::Protocol::k_DEFAULT_SUBSCRIPTION_ID) == - mqbi::StorageResult::e_SUCCESS); - BSLS_ASSERT_OPT(tester.vStorage().numMessages(k_APP_KEY) == i); - BSLS_ASSERT_OPT(tester.vStorage().numBytes(k_APP_KEY) == - i * k_DEFAULT_MSG_SIZE); - } - - // Remove all messages - ASSERT_EQ(tester.vStorage().removeAll(k_APP_KEY), - mqbi::StorageResult::e_SUCCESS); - ASSERT_EQ(tester.vStorage().numMessages(k_APP_KEY), 0); - ASSERT_EQ(tester.vStorage().numBytes(k_APP_KEY), 0); - - int msgSize; - for (MessageGuidsCI cit = guids.cbegin(); cit != guids.cend(); ++cit) { - const bmqt::MessageGUID& guid = *cit; - ASSERT(!tester.vStorage().hasMessage(guid)); - ASSERT_EQ(tester.vStorage().getMessageSize(&msgSize, guid), - mqbi::StorageResult::e_GUID_NOT_FOUND); - } -} - -static void test9_getIterator() -// ------------------------------------------------------------------------ -// GET ITERATOR -// -// Concerns: -// Verify that 'getIterator' works as intended. -// -// Testing: -// getIterator(...) -// ------------------------------------------------------------------------ -{ - mwctst::TestHelper::printTestName("GET ITERATOR"); - Tester tester; - - // Put 10 new messages - MessageGuids guids; - bmqt::MessageGUID newGuid; - const int k_MSG_COUNT = 10; - for (int i = 1; i <= k_MSG_COUNT; ++i) { - newGuid = generateUniqueGUID(&guids); - BSLS_ASSERT_OPT( - tester.vStorage().put(newGuid, - k_DEFAULT_MSG_SIZE, - bmqp::RdaInfo(), - bmqp::Protocol::k_DEFAULT_SUBSCRIPTION_ID) == - mqbi::StorageResult::e_SUCCESS); - BSLS_ASSERT_OPT(tester.vStorage().numMessages(k_APP_KEY) == i); - BSLS_ASSERT_OPT(tester.vStorage().numBytes(k_APP_KEY) == - i * k_DEFAULT_MSG_SIZE); - } - - // Since mqbs::VirtualStorageIterator iterates through physical messages - // from its underlying physical storage, we must store the message - // there as well. - BSLS_ASSERT_OPT(tester.configure(k_INT64_MAX, k_INT64_MAX) == 0); - BSLS_ASSERT_OPT(tester.addPhysicalMessages(guids) == - mqbi::StorageResult::e_SUCCESS); - - // Check Iterator - bslma::ManagedPtr iterator; - iterator = tester.vStorage().getIterator(k_APP_KEY); - - int msgData = 0; - do { - ASSERT_EQ(iterator->guid(), guids[msgData]); - ASSERT_EQ( - *(reinterpret_cast(iterator->appData()->buffer(0).data())), - msgData); - ASSERT_EQ( - *(reinterpret_cast(iterator->options()->buffer(0).data())), - msgData); - ASSERT_EQ(iterator->attributes().arrivalTimestamp(), - static_cast(msgData)); - msgData++; - iterator->advance(); - } while (!iterator->atEnd()); - - // Check iterator's 'reset' - iterator->reset(); - ASSERT_EQ(iterator->guid(), guids[0]); - - // Check Iterator from specific point - msgData = 5; - ASSERT_EQ(tester.vStorage().getIterator(&iterator, k_APP_KEY, guids[5]), - mqbi::StorageResult::e_SUCCESS); - - do { - ASSERT_EQ(iterator->guid(), guids[msgData]); - ASSERT_EQ( - *(reinterpret_cast(iterator->appData()->buffer(0).data())), - msgData); - msgData++; - iterator->advance(); - } while (!iterator->atEnd()); - - // Check iterator with random GUID - bmqt::MessageGUID randomGuid = generateUniqueGUID(&guids); - ASSERT_EQ(tester.vStorage().getIterator(&iterator, k_APP_KEY, randomGuid), - mqbi::StorageResult::e_GUID_NOT_FOUND); - - ASSERT_EQ(tester.vStorage().removeAll(k_APP_KEY), - mqbi::StorageResult::e_SUCCESS); -} - -// ============================================================================ -// MAIN PROGRAM -// ---------------------------------------------------------------------------- - -int main(int argc, char* argv[]) -{ - TEST_PROLOG(mwctst::TestHelper::e_CHECK_GBL_ALLOC); - - bmqt::UriParser::initialize(s_allocator_p); - - { - mqbcfg::AppConfig brokerConfig(s_allocator_p); - mqbcfg::BrokerConfig::set(brokerConfig); - - bsl::shared_ptr statContext = - mqbstat::BrokerStatsUtil::initializeStatContext(30, s_allocator_p); - - switch (_testCase) { - case 0: - case 9: test9_getIterator(); break; - case 8: test8_removeAll(); break; - case 7: test7_remove(); break; - case 6: test6_get(); break; - case 5: test5_getMessageSize(); break; - case 4: test4_hasMessage(); break; - case 3: test3_put(); break; - case 2: test2_unsupportedOperations(); break; - case 1: test1_breathingTest(); break; - default: { - cerr << "WARNING: CASE '" << _testCase << "' NOT FOUND." << endl; - s_testStatus = -1; - } break; - } - } - - bmqt::UriParser::shutdown(); - - TEST_EPILOG(mwctst::TestHelper::e_CHECK_GBL_ALLOC); -} diff --git a/src/groups/mqb/mqbs/mqbs_virtualstoragecatalog.cpp b/src/groups/mqb/mqbs/mqbs_virtualstoragecatalog.cpp index 58f87073a..61971c72b 100644 --- a/src/groups/mqb/mqbs/mqbs_virtualstoragecatalog.cpp +++ b/src/groups/mqb/mqbs/mqbs_virtualstoragecatalog.cpp @@ -20,6 +20,8 @@ // MQB #include +#include + // BDE #include #include @@ -34,11 +36,71 @@ namespace mqbs { // class VirtualStorageCatalog // --------------------------- +// When the 'd_dataStream.erase(msgGUID)' is called +// +// Queues | QueueEngines | Storage | VSC +// ———————————+———————————————————+———————————————————+——————————————————— +// +// confirmMessage +// | +// |—————> onConfirmMessage +// | | onRejectMessage +// | | | +// | +-----------+—————> confirm ——————————> confirm +// | | +// | | +// | | processDeletionRecord (1) +// | | | +// | | V +// +———————————————————+—————> remove ———————————> remove +// | +// +—————————————> erase +// +// Proxy +// onHandleReleased +// | +// +—————————————————————————> removeVirtualStorage +// | +// BroadcastMode +—————————————————> removeVirtualStorage +// | | +// | Primary | +// | afterAppIdUnregistered | +// | | | +// | | | +// +———————+—————————————> removeAll | +// | purge | +// | | | +// V V | +// purgeCommon | +// | V +// +—————————————————> removeAll +// | +// releaseRef (1) <————+—————————————> erase +// +// +// +// gcExpiredMessages (1) +// | +// | +// +—————————————————> gc +// | +// +—————————————> erase +// +// (1) QE::beforeMessageRemoved +// + // CREATORS VirtualStorageCatalog::VirtualStorageCatalog(mqbi::Storage* storage, bslma::Allocator* allocator) : d_storage_p(storage) , d_virtualStorages(allocator) +, d_availableOrdinals(allocator) +, d_nextOrdinal(0) +, d_dataStream(allocator) +, d_totalBytes(0) +, d_numMessages(0) +, d_defaultAppMessage(defaultAppMessage().d_rdaInfo) +, d_queue_p(0) , d_allocator_p(allocator) { // PRECONDITIONS @@ -53,79 +115,197 @@ VirtualStorageCatalog::~VirtualStorageCatalog() } // MANIPULATORS -mqbi::StorageResult::Enum -VirtualStorageCatalog::put(const bmqt::MessageGUID& msgGUID, - int msgSize, - const bmqp::RdaInfo& rdaInfo, - unsigned int subScriptionId, - const mqbu::StorageKey& appKey) + +VirtualStorageCatalog::DataStreamIterator +VirtualStorageCatalog::begin(const bmqt::MessageGUID& where) { - if (!appKey.isNull()) { - VirtualStoragesIter it = d_virtualStorages.findByKey2(appKey); - BSLS_ASSERT_SAFE(it != d_virtualStorages.end()); + if (where.isUnset()) { + return d_dataStream.begin(); + } + else { + return d_dataStream.find(where); + } +} + +VirtualStorageCatalog::DataStreamIterator VirtualStorageCatalog::end() +{ + return d_dataStream.end(); +} + +VirtualStorageCatalog::DataStreamIterator +VirtualStorageCatalog::get(const bmqt::MessageGUID& msgGUID) +{ + DataStreamIterator it = d_dataStream.find(msgGUID); - return it->value()->put(msgGUID, - msgSize, - rdaInfo, - subScriptionId); // RETURN + if (it != d_dataStream.end()) { + setup(&it->second); } - // Add guid to all virtual storages. - mqbi::StorageResult::Enum lastRc = mqbi::StorageResult::e_SUCCESS; - for (VirtualStoragesIter it = d_virtualStorages.begin(); - it != d_virtualStorages.end(); - ++it) { - mqbi::StorageResult::Enum rc = - it->value()->put(msgGUID, msgSize, rdaInfo, subScriptionId); - if (rc != mqbi::StorageResult::e_SUCCESS) { - lastRc = rc; - } + return it; +} + +void VirtualStorageCatalog::setup(VirtualStorage::DataStreamMessage* data) +{ + // The only case for subsequent resize is proxy receiving subsequent PUSH + // messages for the same GUID and different apps + if (data->d_apps.size() < d_nextOrdinal) { + data->d_apps.resize(d_nextOrdinal, defaultAppMessage()); + } +} + +mqbi::StorageResult::Enum +VirtualStorageCatalog::put(const bmqt::MessageGUID& msgGUID, + int msgSize, + VirtualStorage::DataStreamMessage** out) +{ + bsl::pair insertResult = + d_dataStream.insert(bsl::make_pair( + msgGUID, + VirtualStorage::DataStreamMessage(msgSize, d_allocator_p))); + + if (!insertResult.second) { + // Duplicate GUID + return mqbi::StorageResult::e_GUID_NOT_UNIQUE; // RETURN + } + + d_totalBytes += msgSize; + ++d_numMessages; + + if (out) { + // The auto-confirm case when we need to update App states. + *out = &insertResult.first->second; + + setup(*out); } - return lastRc; // RETURN + return mqbi::StorageResult::e_SUCCESS; // RETURN } bslma::ManagedPtr VirtualStorageCatalog::getIterator(const mqbu::StorageKey& appKey) { - // PRECONDITIONS - BSLS_ASSERT_SAFE(!appKey.isNull()); + bslma::ManagedPtr mp; - VirtualStoragesIter it = d_virtualStorages.findByKey2(appKey); - BSLS_ASSERT_SAFE(it != d_virtualStorages.end()); - return it->value()->getIterator(appKey); + if (!appKey.isNull()) { + VirtualStoragesIter it = d_virtualStorages.findByKey2(appKey); + BSLS_ASSERT_SAFE(it != d_virtualStorages.end()); + + VirtualStorage* vs = it->value().get(); + + mp.load(new (*d_allocator_p) + VirtualStorageIterator(it->value().get(), + d_storage_p, + this, + d_dataStream.begin()), + d_allocator_p); + + if (!mp->atEnd()) { + if (!mp->appMessageView(vs->ordinal()).isPending()) { + // By contract, the iterator iterates only pending states. + mp->advance(); + } + } + } + else { + mp.load(new (*d_allocator_p) + StorageIterator(d_storage_p, this, d_dataStream.begin()), + d_allocator_p); + } + + return mp; } mqbi::StorageResult::Enum VirtualStorageCatalog::getIterator( bslma::ManagedPtr* out, const mqbu::StorageKey& appKey, const bmqt::MessageGUID& msgGUID) +{ + DataStreamIterator data = d_dataStream.find(msgGUID); + + if (data == d_dataStream.end()) { + return mqbi::StorageResult::e_GUID_NOT_FOUND; // RETURN + } + + if (!appKey.isNull()) { + VirtualStoragesIter it = d_virtualStorages.findByKey2(appKey); + BSLS_ASSERT_SAFE(it != d_virtualStorages.end()); + + VirtualStorage* vs = it->value().get(); + + out->load(new (*d_allocator_p) + VirtualStorageIterator(vs, d_storage_p, this, data), + d_allocator_p); + + const mqbi::AppMessage& appView = (*out)->appMessageView( + vs->ordinal()); + + if (!appView.isPending()) { + // By contract, the iterator iterates only pending states. + + return mqbi::StorageResult::e_GUID_NOT_FOUND; // RETURN + } + } + else { + out->load(new (*d_allocator_p) + StorageIterator(d_storage_p, this, data), + d_allocator_p); + } + + return mqbi::StorageResult::e_SUCCESS; +} + +mqbi::StorageResult::Enum +VirtualStorageCatalog::confirm(const bmqt::MessageGUID& msgGUID, + const mqbu::StorageKey& appKey) { // PRECONDITIONS BSLS_ASSERT_SAFE(!appKey.isNull()); + VirtualStorage::DataStreamIterator data = get(msgGUID); + if (data == d_dataStream.end()) { + return mqbi::StorageResult::e_GUID_NOT_FOUND; // RETURN + } + VirtualStoragesIter it = d_virtualStorages.findByKey2(appKey); BSLS_ASSERT_SAFE(it != d_virtualStorages.end()); - return it->value()->getIterator(out, appKey, msgGUID); + + setup(&data->second); + + return it->value()->confirm(&data->second); } mqbi::StorageResult::Enum -VirtualStorageCatalog::remove(const bmqt::MessageGUID& msgGUID, - const mqbu::StorageKey& appKey) +VirtualStorageCatalog::remove(const bmqt::MessageGUID& msgGUID) { - if (!appKey.isNull()) { - VirtualStoragesIter it = d_virtualStorages.findByKey2(appKey); - BSLS_ASSERT_SAFE(it != d_virtualStorages.end()); - return it->value()->remove(msgGUID); // RETURN + // Remove all Apps states at once. + if (0 == d_dataStream.erase(msgGUID)) { + return mqbi::StorageResult::e_GUID_NOT_FOUND; // RETURN } - // Remove guid from all virtual storages. + return mqbi::StorageResult::e_SUCCESS; +} + +mqbi::StorageResult::Enum +VirtualStorageCatalog::gc(const bmqt::MessageGUID& msgGUID) +{ + VirtualStorage::DataStreamIterator data = d_dataStream.find(msgGUID); + if (data == d_dataStream.end()) { + return mqbi::StorageResult::e_GUID_NOT_FOUND; // RETURN + } + + // Update each App so the math of numMessages/numBytes stays correct. + // REVISIT. for (VirtualStoragesIter it = d_virtualStorages.begin(); it != d_virtualStorages.end(); ++it) { - it->value()->remove(msgGUID); // ignore rc + it->value()->onGC(data->second); } + d_totalBytes -= data->second.d_size; + --d_numMessages; + + d_dataStream.erase(data); + return mqbi::StorageResult::e_SUCCESS; } @@ -133,16 +313,68 @@ mqbi::StorageResult::Enum VirtualStorageCatalog::removeAll(const mqbu::StorageKey& appKey) { if (!appKey.isNull()) { - VirtualStoragesIter it = d_virtualStorages.findByKey2(appKey); - BSLS_ASSERT_SAFE(it != d_virtualStorages.end()); - return it->value()->removeAll(appKey); // RETURN + VirtualStoragesIter itVs = d_virtualStorages.findByKey2(appKey); + BSLS_ASSERT_SAFE(itVs != d_virtualStorages.end()); + + for (DataStreamIterator itData = d_dataStream.begin(); + itData != d_dataStream.end();) { + mqbi::StorageResult::Enum result = mqbi::StorageResult::e_SUCCESS; + + VirtualStorage::DataStreamMessage* data = &itData->second; + setup(data); + + if (itVs->value()->remove(data) == + mqbi::StorageResult::e_SUCCESS) { + // The 'data' was not already removed or confirmed. + result = d_storage_p->releaseRef(itData->first); + } + + if (result == mqbi::StorageResult::e_ZERO_REFERENCES) { + itData = d_dataStream.erase(itData); + } + else { + if (result == mqbi::StorageResult::e_GUID_NOT_FOUND) { + BALL_LOG_WARN + << "#STORAGE_PURGE_ERROR " << "PartitionId [" + << d_storage_p->partitionId() << "]" + << ": Attempting to purge GUID '" << itData->first + << "' from virtual storage with appId '" + << itVs->value()->appId() << "' & appKey '" << appKey + << "' for queue '" << d_storage_p->queueUri() + << "' & queueKey '" << d_storage_p->queueKey() + << "', but GUID does not exist in the underlying " + "storage."; + } + else if (result == + mqbi::StorageResult::e_NON_ZERO_REFERENCES) { + } + else if (result == mqbi::StorageResult::e_SUCCESS) { + } + else { + MWCTSK_ALARMLOG_ALARM("STORAGE_PURGE_ERROR") + << "PartitionId [" << d_storage_p->partitionId() << "]" + << ": Attempting to purge GUID '" << itData->first + << "' from virtual storage with appId '" + << itVs->value()->appId() << "' & appKey '" << appKey + << "] for queue '" << d_storage_p->queueUri() + << "' & queueKey '" << d_storage_p->queueKey() + << "', with invalid context (refCount is already " + "zero)." + << MWCTSK_ALARMLOG_END; + } + ++itData; + } + } } - - // Clear all virtual storages. - for (VirtualStoragesIter it = d_virtualStorages.begin(); - it != d_virtualStorages.end(); - ++it) { - it->value()->removeAll(it->key2()); // ignore rc + else { + for (VirtualStoragesIter it = d_virtualStorages.begin(); + it != d_virtualStorages.end(); + ++it) { + it->value()->resetStats(); + } + d_dataStream.clear(); + d_numMessages = 0; + d_totalBytes = 0; } return mqbi::StorageResult::e_SUCCESS; @@ -168,14 +400,36 @@ int VirtualStorageCatalog::addVirtualStorage(bsl::ostream& errorDescription, return -1; // RETURN } + Ordinal appOrdinal; + + // Ordinals ever grow. + if (d_availableOrdinals.empty()) { + appOrdinal = d_nextOrdinal++; + } + else { + appOrdinal = d_availableOrdinals.front(); + // There is no conflict because everything 'appOrdinal' was removed. + d_availableOrdinals.pop_front(); + } + + BSLS_ASSERT_SAFE(appOrdinal <= d_virtualStorages.size()); + VirtualStorageSp vsp; vsp.createInplace(d_allocator_p, d_storage_p, appId, appKey, + appOrdinal, d_allocator_p); d_virtualStorages.insert(appId, appKey, vsp); + if (d_queue_p) { + BSLS_ASSERT_SAFE(d_queue_p->queueEngine()); + // QueueEngines use the key to look up the id + + d_queue_p->queueEngine()->registerStorage(appId, appKey, appOrdinal); + } + return 0; } @@ -183,13 +437,33 @@ bool VirtualStorageCatalog::removeVirtualStorage( const mqbu::StorageKey& appKey) { if (appKey.isNull()) { + // Make sure there is no AppMessage in the pending states + removeAll(appKey); + // Remove all virtual storages d_virtualStorages.clear(); + d_availableOrdinals.clear(); + d_nextOrdinal = 0; return true; // RETURN } VirtualStoragesConstIter it = d_virtualStorages.findByKey2(appKey); if (it != d_virtualStorages.end()) { + // Make sure there is no AppMessage in the pending states + removeAll(appKey); + + const VirtualStorage& vs = *it->value(); + d_availableOrdinals.push_back(vs.ordinal()); + + if (d_queue_p) { + BSLS_ASSERT_SAFE(d_queue_p->queueEngine()); + // QueueEngines use the key to look up the id + + d_queue_p->queueEngine()->unregisterStorage(vs.appId(), + appKey, + vs.ordinal()); + } + d_virtualStorages.erase(it); return true; // RETURN } @@ -197,27 +471,33 @@ bool VirtualStorageCatalog::removeVirtualStorage( return false; } -mqbi::Storage* +VirtualStorage* VirtualStorageCatalog::virtualStorage(const mqbu::StorageKey& appKey) { // PRECONDITIONS BSLS_ASSERT_SAFE(!appKey.isNull()); VirtualStoragesIter it = d_virtualStorages.findByKey2(appKey); - BSLS_ASSERT_SAFE(it != d_virtualStorages.end()); - return it->value().get(); + if (it == d_virtualStorages.end()) { + return 0; + } + else { + return it->value().get(); + } } -void VirtualStorageCatalog::autoConfirm(const bmqt::MessageGUID& msgGUID, - const mqbu::StorageKey& appKey) +void VirtualStorageCatalog::autoConfirm( + VirtualStorage::DataStreamMessage* dataStreamMessage, + const mqbu::StorageKey& appKey) { // PRECONDITIONS + BSLS_ASSERT_SAFE(dataStreamMessage); BSLS_ASSERT_SAFE(!appKey.isNull()); VirtualStoragesIter it = d_virtualStorages.findByKey2(appKey); BSLS_ASSERT_SAFE(it != d_virtualStorages.end()); - it->value()->autoConfirm(msgGUID); + it->value()->confirm(dataStreamMessage); } // ACCESSORS @@ -233,17 +513,18 @@ bool VirtualStorageCatalog::hasVirtualStorage(const mqbu::StorageKey& appKey, if (appId) { if (hasVs) { *appId = cit->value()->appId(); - return true; // RETURN } - - *appId = ""; + else { + *appId = ""; + } } return hasVs; } bool VirtualStorageCatalog::hasVirtualStorage(const bsl::string& appId, - mqbu::StorageKey* appKey) const + mqbu::StorageKey* appKey, + unsigned int* ordinal) const { // PRECONDITIONS BSLS_ASSERT_SAFE(!appId.empty()); @@ -254,26 +535,19 @@ bool VirtualStorageCatalog::hasVirtualStorage(const bsl::string& appId, if (appKey) { if (hasVs) { *appKey = cit->key2(); - return true; // RETURN } - - *appKey = mqbu::StorageKey::k_NULL_KEY; + else { + *appKey = mqbu::StorageKey::k_NULL_KEY; + } } - return hasVs; -} - -bool VirtualStorageCatalog::hasMessage(const bmqt::MessageGUID& msgGUID) const -{ - for (VirtualStoragesConstIter it = d_virtualStorages.begin(); - it != d_virtualStorages.end(); - ++it) { - if (it->value()->hasMessage(msgGUID)) { - return true; // RETURN + if (ordinal) { + if (hasVs) { + *ordinal = cit->value()->ordinal(); } } - return false; + return hasVs; } void VirtualStorageCatalog::loadVirtualStorageDetails( diff --git a/src/groups/mqb/mqbs/mqbs_virtualstoragecatalog.h b/src/groups/mqb/mqbs/mqbs_virtualstoragecatalog.h index 42489a83f..0a2642b8b 100644 --- a/src/groups/mqb/mqbs/mqbs_virtualstoragecatalog.h +++ b/src/groups/mqb/mqbs/mqbs_virtualstoragecatalog.h @@ -38,6 +38,7 @@ #include // BDE +#include #include #include #include @@ -54,13 +55,42 @@ namespace mqbs { // class VirtualStorageCatalog // =========================== -/// A catalog of virtual storages associated with a queue. +// The owner of so-called Virtual Storage(s) implemented as +// 'VirtualStorage::DataStream'. Both 'FileBasedStorage' and 'InMemoryStorage' +// own an instance of 'VirtualStorageCatalog'. The access to which is done in +// two ways. +// 1) While the 'mgbi::Storage' does not expose a reference to this +// instance, most of the 'mgbi::Storage' calls result in calling the +// corresponding 'VirtualStorageCatalog' method. +// 2) Both 'RootQueueEngine' and 'RelayQueueEngine' access the Virtual Storage +// by an 'mqbi::StorageIterator' implemented by 'mqbs::StorageIterator'. +// +// The purpose of Virtual Storage is to keep state of (guid, App) pairs for +// delivery by QueueEngines. 'App' is identified by 'appKey' and an ordinal - +// offset in the consecutive memory ('VirtualStorage::DataStreamMessage') +// holding all Apps states ('mqbi::AppMessage') for each guid. + class VirtualStorageCatalog { + private: + // CLASS-SCOPE CATEGORY + BALL_LOG_SET_CLASS_CATEGORY("MQBS.VIRTUALSTORAGE"); + + public: + // TYPES + typedef mqbi::Storage::AppIdKeyPair AppIdKeyPair; + + typedef mqbi::Storage::AppIdKeyPairs AppIdKeyPairs; + + typedef unsigned int Ordinal; + private: // PRIVATE TYPES typedef bsl::shared_ptr VirtualStorageSp; - /// Any(appId, appKey) -> virtualStorage + /// List of available ordinal values for Virtual Storages. + typedef bsl::list AvailableOrdinals; + + /// appKey -> virtualStorage typedef mwcc:: TwoKeyHashMap VirtualStorages; @@ -69,6 +99,12 @@ class VirtualStorageCatalog { typedef VirtualStorages::const_iterator VirtualStoragesConstIter; + public: + // TYPES + + /// Access to the DataStream + typedef VirtualStorage::DataStream::iterator DataStreamIterator; + private: // DATA mqbi::Storage* d_storage_p; // Physical storage underlying all @@ -79,6 +115,28 @@ class VirtualStorageCatalog { // Map of appKey to corresponding // virtual storage + AvailableOrdinals d_availableOrdinals; + // available ordinal values for Virtual Storages. + + Ordinal d_nextOrdinal; + // Monotonically increasing value to generate new ordinal. + + VirtualStorage::DataStream d_dataStream; + // The DataStream tracking all Apps states. + + bsls::Types::Int64 d_totalBytes; + // Cumulative count of all bytes. + + bsls::Types::Int64 d_numMessages; + // Cumulative count of all messages. + + mqbi::AppMessage d_defaultAppMessage; + // The default App state + + mqbi::Queue* d_queue_p; + // This could be null if a local or remote + // queue instance has not been created. + bslma::Allocator* d_allocator_p; // Allocator to use private: @@ -93,15 +151,10 @@ class VirtualStorageCatalog { bslma::UsesBslmaAllocator) public: - // TYPES - typedef mqbi::Storage::AppIdKeyPair AppIdKeyPair; - - typedef mqbi::Storage::AppIdKeyPairs AppIdKeyPairs; - // CREATORS /// Create an instance of virtual storage catalog with the specified - /// `allocator`. + /// 'defaultRdaInfo' and 'allocator'. VirtualStorageCatalog(mqbi::Storage* storage, bslma::Allocator* allocator); /// Destructor @@ -109,116 +162,141 @@ class VirtualStorageCatalog { // MANIPULATORS - /// Save the message having the specified `msgGUID`, `msgSize` and - /// `rdaInfo` to the virtual storage associated with the specified - /// `appKey`. Note that if `appKey` is null, the message will be added - /// to all virtual storages maintained by this instance. - mqbi::StorageResult::Enum put(const bmqt::MessageGUID& msgGUID, - int msgSize, - const bmqp::RdaInfo& rdaInfo, - unsigned int subScriptionId, - const mqbu::StorageKey& appKey); - - /// Get an iterator for items stored in the virtual storage identified - /// by the specified `appKey`. Iterator will point to point to the - /// oldest item, if any, or to the end of the collection if empty. Note - /// that if `appKey` is null, an iterator over the underlying physical - /// storage will be returned. Also note that because `Storage` and - /// `StorageIterator` are interfaces, the implementation of this method - /// will allocate, so it's recommended to keep the iterator. - /// TBD: Is the behavior undefined if `appKey` is null? + /// If the specified 'where' is unset, return reference to the beginning of + /// the DataStream. Otherwise, return reference to the corresponding item + /// in the DataStream. + /// If item is not found, return reference to the end of the DataStream. + DataStreamIterator + begin(const bmqt::MessageGUID& where = bmqt::MessageGUID()); + + /// Return reference to the end of the DataStream. + DataStreamIterator end(); + + /// Return reference to the item in the DataStream corresponding to the + /// specified 'msgGUID' and allocate space for all Apps states if needed. + DataStreamIterator get(const bmqt::MessageGUID& msgGUID); + + /// Allocate space for all Apps states in the specified 'data' if needed. + void setup(VirtualStorage::DataStreamMessage* data); + + /// Save the message having the specified 'msgGUID' and 'msgSize' to the + /// DataStream. If the specified 'out' is not '0', allocate space for all + /// Apps states and load the created object into the 'out'. + mqbi::StorageResult::Enum put(const bmqt::MessageGUID& msgGUID, + int msgSize, + VirtualStorage::DataStreamMessage** out = 0); + + /// Get an iterator for items stored in the DataStream identified by the + /// specified 'appKey'. + /// If the 'appKey' is null, the returned iterator can iterate states of + /// all Apps; otherwise, the iterator can iterate states of the App + /// corresponding to the 'appKey'. bslma::ManagedPtr getIterator(const mqbu::StorageKey& appKey); - /// Load into the specified `out` an iterator for items stored in - /// the virtual storage identified by the specified `appKey`, initially - /// pointing to the item associated with the specified `msgGUID`. - /// Return zero on success, and a non-zero code if `msgGUID` was not - /// found in the storage. Note that if `appKey` is null, an iterator - /// over the underlying physical storage will be returned. Also note - /// that because `Storage` and `StorageIterator` are interfaces, the - /// implementation of this method will allocate, so it's recommended to - /// keep the iterator. - /// TBD: Is the behavior undefined if `appKey` is null? + /// Load into the specified 'out' an iterator for items stored in the + /// DataStream initially pointing to the item associated with the specified + /// 'msgGUID'. + /// If the 'appKey' is null, the returned iterator can iterate states of + /// all Apps; otherwise, the iterator can iterate states of the App + /// corresponding to the 'appKey'. + /// Return zero on success, and a non-zero code if 'msgGUID' was not + /// found in the storage. mqbi::StorageResult::Enum getIterator(bslma::ManagedPtr* out, const mqbu::StorageKey& appKey, const bmqt::MessageGUID& msgGUID); - /// Remove the message having the specified `msgGUID` from the storage - /// for the client identified by the specified `appKey`. If `appKey` is - /// null, then remove the message from the storages for all clients. - /// Return 0 on success, or a non-zero return code if the `msgGUID` was - /// not found or the `appKey` is invalid. - mqbi::StorageResult::Enum remove(const bmqt::MessageGUID& msgGUID, - const mqbu::StorageKey& appKey); - - /// Remove all messages from the storage for the client identified by - /// the specified `appKey`. If `appKey` is null, then remove messages - /// for all clients. Return one of the return codes from: - /// * **e_SUCCESS** : `msgGUID` was not found - /// * **e_APPKEY_NOT_FOUND** : Invalid `appKey` specified + /// Remove the message having the specified 'msgGUID' from the DataStream. + /// Return 0 on success, or a non-zero return code if the 'msgGUID' was + /// not found. + mqbi::StorageResult::Enum remove(const bmqt::MessageGUID& msgGUID); + + /// Remove the message having the specified 'msgGUID' from the DataStream + /// and update the counts of bytes and messages according to GC logic. + /// Return 0 on success, or a non-zero return code if the 'msgGUID' was + /// not found. + mqbi::StorageResult::Enum gc(const bmqt::MessageGUID& msgGUID); + + /// Update the App state corresponding to the specified 'msgGUID' and the + /// specified 'appKey' in the DataStream. + /// Return 0 on success, or a non-zero return code if the 'msgGUID' was + /// not found. + /// Behavior is undefined unless there is an App with the 'appKey'. + mqbi::StorageResult::Enum confirm(const bmqt::MessageGUID& msgGUID, + const mqbu::StorageKey& appKey); + + /// If the specified 'appKey' is null, erase the entire DataStream; + /// otherwise, erase all states of the App corresponding to the 'appKey'. mqbi::StorageResult::Enum removeAll(const mqbu::StorageKey& appKey); /// Create, if it doesn't exist already, a virtual storage instance with - /// the specified `appId` and `appKey`. Return zero upon success and a + /// the specified 'appId' and 'appKey'. Return zero upon success and a /// non-zero value otherwise, and populate the specified - /// `errorDescription` with a brief reason in case of failure. + /// 'errorDescription' with a brief reason in case of failure. + /// Behavior is undefined if the 'appKey' is not valid. int addVirtualStorage(bsl::ostream& errorDescription, const bsl::string& appId, const mqbu::StorageKey& appKey); - /// Remove the virtual storage identified by the specified `appKey`. - /// Return true if a virtual storage with `appKey` was found and - /// deleted, false if a virtual storage with `appKey` does not exist. - /// Behavior is undefined unless `appKey` is non-null. Note that this - /// method will delete the virtual storage, and any reference to it will - /// become invalid after this method returns. + /// If the specified 'appKey' is null, erase the entire DataStream and + /// all Virtual Storage instances; erase all states of the App + /// corresponding to the 'appKey' and remove the corresponding Virtual + /// Storage instance. bool removeVirtualStorage(const mqbu::StorageKey& appKey); - mqbi::Storage* virtualStorage(const mqbu::StorageKey& appKey); + /// Return the Virtual Storage instance corresponding to the specified + /// 'appKey'. + VirtualStorage* virtualStorage(const mqbu::StorageKey& appKey); + + /// (Auto)Confirm the specified 'msgGUID' for the specified 'appKey'. + /// Behavior is undefined unless there is an App with the 'appKey'. + void autoConfirm(VirtualStorage::DataStreamMessage* dataStreamMessage, + const mqbu::StorageKey& appKey); - /// Ignore the specified 'msgGUID' in the subsequent 'put' call for the - /// specified 'appKey' because the App has auto confirmed it. - void autoConfirm(const bmqt::MessageGUID& msgGUID, - const mqbu::StorageKey& appKey); + /// Set the default RDA according to the specified 'maxDeliveryAttempts'. + void setDefaultRda(int maxDeliveryAttempts); + + void setQueue(mqbi::Queue* queue); // ACCESSORS /// Return the number of virtual storages registered with this instance. int numVirtualStorages() const; - /// Return true if virtual storage identified by the specified `appKey` + /// Return true if virtual storage identified by the specified 'appKey' /// exists, otherwise return false. Load into the optionally specified - /// `appId` the appId associated with `appKey` if the virtual storage + /// 'appId' the appId associated with 'appKey' if the virtual storage /// exists, otherwise set it to the empty string. bool hasVirtualStorage(const mqbu::StorageKey& appKey, bsl::string* appId = 0) const; - /// Return true if virtual storage identified by the specified `appId` + /// Return true if virtual storage identified by the specified 'appId' /// exists, otherwise return false. Load into the optionally specified - /// `appKey` the appKey associated with `appId` if the virtual storage - /// exists, otherwise set it to the null key. + /// 'appKey' and 'ordinal' the appKey and ordinal associated with 'appId' + /// if the virtual storage exists, otherwise set it to the null key. bool hasVirtualStorage(const bsl::string& appId, - mqbu::StorageKey* appKey = 0) const; + mqbu::StorageKey* appKey = 0, + unsigned int* ordinal = 0) const; - /// Load into the specified `buffer` the list of pairs of appId and + /// Load into the specified 'buffer' the list of pairs of appId and /// appKey for all the virtual storages registered with this instance. void loadVirtualStorageDetails(AppIdKeyPairs* buffer) const; /// Return the number of messages in the virtual storage associated with - /// the specified `appKey`. Behavior is undefined unless a virtual - /// storage associated with the `appKey` exists in this catalog. + /// the specified 'appKey'. Behavior is undefined unless a virtual + /// storage associated with the 'appKey' exists in this'og. bsls::Types::Int64 numMessages(const mqbu::StorageKey& appKey) const; /// Return the number of bytes in the virtual storage associated with - /// the specified `appKey`. Behavior is undefined unless a virtual - /// storage associated with the `appKey` exists in this catalog. + /// the specified 'appKey'. Behavior is undefined unless a virtual + /// storage associated with the 'appKey' exists in this catalog. bsls::Types::Int64 numBytes(const mqbu::StorageKey& appKey) const; - /// Return true if there is a virtual storage associated with any appKey - /// which contains the specified `msgGUID`. Return false otherwise. - bool hasMessage(const bmqt::MessageGUID& msgGUID) const; + /// Return the default App state. + const mqbi::AppMessage& defaultAppMessage() const; + + mqbi::Queue* queue() const; }; // ============================================================================ @@ -229,6 +307,21 @@ class VirtualStorageCatalog { // class VirtualStorageCatalog // --------------------------- +inline void VirtualStorageCatalog::setDefaultRda(int maxDeliveryAttempts) +{ + if (maxDeliveryAttempts > 0) { + d_defaultAppMessage.d_rdaInfo.setCounter(maxDeliveryAttempts); + } + else { + d_defaultAppMessage.d_rdaInfo.setUnlimited(); + } +} + +inline void VirtualStorageCatalog::setQueue(mqbi::Queue* queue) +{ + d_queue_p = queue; +} + // ACCESSORS inline int VirtualStorageCatalog::numVirtualStorages() const { @@ -238,9 +331,12 @@ inline int VirtualStorageCatalog::numVirtualStorages() const inline bsls::Types::Int64 VirtualStorageCatalog::numMessages(const mqbu::StorageKey& appKey) const { + // PRECONDITIONS + BSLS_ASSERT_SAFE(!appKey.isNull()); + VirtualStoragesConstIter cit = d_virtualStorages.findByKey2(appKey); BSLS_ASSERT_SAFE(cit != d_virtualStorages.end()); - return cit->value()->numMessages(appKey); + return d_numMessages - cit->value()->numRemoved(); } inline bsls::Types::Int64 @@ -248,7 +344,18 @@ VirtualStorageCatalog::numBytes(const mqbu::StorageKey& appKey) const { VirtualStoragesConstIter cit = d_virtualStorages.findByKey2(appKey); BSLS_ASSERT_SAFE(cit != d_virtualStorages.end()); - return cit->value()->numBytes(appKey); + + return d_totalBytes - cit->value()->removedBytes(); +} + +inline const mqbi::AppMessage& VirtualStorageCatalog::defaultAppMessage() const +{ + return d_defaultAppMessage; +} + +inline mqbi::Queue* VirtualStorageCatalog::queue() const +{ + return d_queue_p; } } // close package namespace diff --git a/src/groups/mqb/mqbs/mqbs_voidstorageiterator.cpp b/src/groups/mqb/mqbs/mqbs_voidstorageiterator.cpp deleted file mode 100644 index c38da406c..000000000 --- a/src/groups/mqb/mqbs/mqbs_voidstorageiterator.cpp +++ /dev/null @@ -1,108 +0,0 @@ -// Copyright 2018-2023 Bloomberg Finance L.P. -// SPDX-License-Identifier: Apache-2.0 -// -// 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 -// -// http://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. - -// mqbs_voidstorageiterator.cpp -*-C++-*- -#include - -// BDE -#include - -namespace BloombergLP { -namespace mqbs { - -namespace { -bsls::ObjectBuffer > s_blob_sp; -mqbi::StorageMessageAttributes s_attr; - -const bsl::shared_ptr& blob() -{ - BSLMT_ONCE_DO - { - new (s_blob_sp.buffer()) bsl::shared_ptr(new bdlbb::Blob); - } - - return s_blob_sp.object(); -} -} -// ------------------------- -// class VoidStorageIterator -// ------------------------- - -// CREATORS -VoidStorageIterator::VoidStorageIterator() -{ - // NOTHING -} - -VoidStorageIterator::~VoidStorageIterator() -{ - // NOTHING -} - -// ACCESSORS -const bmqt::MessageGUID& VoidStorageIterator::guid() const -{ - return d_invalidGuid; -} - -bmqp::RdaInfo& VoidStorageIterator::rdaInfo() const -{ - static bmqp::RdaInfo dummy; - return dummy; -} - -unsigned int VoidStorageIterator::subscriptionId() const -{ - return bmqp::Protocol::k_DEFAULT_SUBSCRIPTION_ID; -} - -const bsl::shared_ptr& VoidStorageIterator::appData() const -{ - return blob(); -} - -const bsl::shared_ptr& VoidStorageIterator::options() const -{ - return blob(); -} - -const mqbi::StorageMessageAttributes& VoidStorageIterator::attributes() const -{ - return s_attr; -} - -bool VoidStorageIterator::atEnd() const -{ - return true; -} - -bool VoidStorageIterator::hasReceipt() const -{ - return false; -} - -// MANIPULATORS -bool VoidStorageIterator::advance() -{ - return false; -} - -void VoidStorageIterator::reset() -{ - // NOTHING -} - -} // close package namespace -} // close enterprise namespace diff --git a/src/groups/mqb/mqbs/mqbs_voidstorageiterator.h b/src/groups/mqb/mqbs/mqbs_voidstorageiterator.h deleted file mode 100644 index c530f0e42..000000000 --- a/src/groups/mqb/mqbs/mqbs_voidstorageiterator.h +++ /dev/null @@ -1,138 +0,0 @@ -// Copyright 2018-2023 Bloomberg Finance L.P. -// SPDX-License-Identifier: Apache-2.0 -// -// 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 -// -// http://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. - -// mqbs_voidstorageiterator.h -*-C++-*- -#ifndef INCLUDED_MQBS_VOIDSTORAGEITERATOR -#define INCLUDED_MQBS_VOIDSTORAGEITERATOR - -//@PURPOSE: Provide an iterator over an empty storage. -// -//@CLASSES: -// mqbs::VoidStorageIterator -// -//@DESCRIPTION: 'mqbs::VoidStorageIterator' provides a concrete implementation -// of the 'mqbi::Storage' interface that iterates over an empty range of -// messages. This is used by substreams created with an unauthorized app id, -// until it is dynamically registered. - -// MQB -#include - -// BMQ -#include - -// BDE -#include -#include -#include - -namespace BloombergLP { - -namespace bmqt { -class MessageGUID; -} -namespace mqbi { -class StorageMessageAttributes; -} - -namespace mqbs { - -// ========================= -// class VoidStorageIterator -// ========================= - -class VoidStorageIterator : public mqbi::StorageIterator { - private: - // DATA - bmqt::MessageGUID d_invalidGuid; - // Constant representing a null message GUID. - - private: - // NOT IMPLEMENTED - VoidStorageIterator(const VoidStorageIterator&) BSLS_KEYWORD_DELETED; - VoidStorageIterator& - operator=(const VoidStorageIterator&) BSLS_KEYWORD_DELETED; - - public: - // CREATORS - - /// Create a `VoidStorageIterator` object. - VoidStorageIterator(); - - /// Destroy this object. - ~VoidStorageIterator() BSLS_KEYWORD_OVERRIDE; - - // ACCESSORS - // (virtual mqbi::StorageIterator) - - /// Return a reference offering non-modifiable access to the guid - /// associated to the item currently pointed at by this iterator. The - /// behavior is undefined unless `atEnd` returns `false`. - virtual const bmqt::MessageGUID& guid() const BSLS_KEYWORD_OVERRIDE; - - /// Return a reference offering modifiable access to the RdaInfo - /// associated to the item currently pointed at by this iterator. The - /// behavior is undefined unless `atEnd` returns `false`. - bmqp::RdaInfo& rdaInfo() const BSLS_KEYWORD_OVERRIDE; - - /// Return subscription id associated to the item currently pointed at - /// by this iterator. - /// The behavior is undefined unless `atEnd` returns `false`. - unsigned int subscriptionId() const BSLS_KEYWORD_OVERRIDE; - - /// Return a reference offering non-modifiable access to the application - /// data associated with the item currently pointed at by this iterator. - /// The behavior is undefined unless `atEnd` returns `false`. - virtual const bsl::shared_ptr& - appData() const BSLS_KEYWORD_OVERRIDE; - - /// Return a reference offering non-modifiable access to the options - /// associated with the item currently pointed at by this iterator. The - /// behavior is undefined unless `atEnd` returns `false`. - virtual const bsl::shared_ptr& - options() const BSLS_KEYWORD_OVERRIDE; - - /// Return a reference offering non-modifiable access to the attributes - /// associated with the message currently pointed at by this iterator. - /// The behavior is undefined unless `atEnd` returns `false`. - virtual const mqbi::StorageMessageAttributes& - attributes() const BSLS_KEYWORD_OVERRIDE; - - /// Return `true` if this iterator is currently at the end of the items' - /// collection, and hence doesn't reference a valid item. - virtual bool atEnd() const BSLS_KEYWORD_OVERRIDE; - - /// Return `true` if this iterator is currently not at the end of the - /// `items` collection and the message currently pointed at by this - /// iterator has received replication factor Receipts. - virtual bool hasReceipt() const BSLS_KEYWORD_OVERRIDE; - - // MANIPULATORS - - /// Advance the iterator to the next item. The behavior is undefined - /// unless `atEnd` returns `false`. Return `true` if the iterator then - /// points to a valid item, or `false` if it now is at the end of the - /// items' collection. - virtual bool advance() BSLS_KEYWORD_OVERRIDE; - - /// Reset the iterator to point to first item, if any, in the underlying - /// storage. - virtual void reset() BSLS_KEYWORD_OVERRIDE; -}; - -} // close package namespace -} // close enterprise namespace - -#endif diff --git a/src/groups/mqb/mqbs/mqbs_voidstorageiterator.t.cpp b/src/groups/mqb/mqbs/mqbs_voidstorageiterator.t.cpp deleted file mode 100644 index ddca5a1f9..000000000 --- a/src/groups/mqb/mqbs/mqbs_voidstorageiterator.t.cpp +++ /dev/null @@ -1,51 +0,0 @@ -// Copyright 2023 Bloomberg Finance L.P. -// SPDX-License-Identifier: Apache-2.0 -// -// 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 -// -// http://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. - -// mqbs_voidstorageiterator.t.cpp -*-C++-*- -#include - -// MWC -#include - -using namespace BloombergLP; -using namespace bsl; - -// ============================================================================ -// TESTS -// ---------------------------------------------------------------------------- - -TEST(functionality) -{ - mqbs::VoidStorageIterator it; - ASSERT_EQ(it.atEnd(), true); - ASSERT_EQ(it.advance(), false); - ASSERT_EQ(it.guid(), bmqt::MessageGUID()); - ASSERT_EQ(it.attributes(), mqbi::StorageMessageAttributes()); -} - -// ============================================================================ -// MAIN PROGRAM -// ---------------------------------------------------------------------------- - -int main(int argc, char* argv[]) -{ - mqbs::VoidStorageIterator().appData(); // Allocate the shared_ptr - - TEST_PROLOG(mwctst::TestHelper::e_DEFAULT); - - mwctst::runTest(_testCase); - - TEST_EPILOG(mwctst::TestHelper::e_CHECK_DEF_GBL_ALLOC); -} diff --git a/src/groups/mqb/mqbs/package/mqbs.mem b/src/groups/mqb/mqbs/package/mqbs.mem index 3bc9571c8..8d527be8a 100644 --- a/src/groups/mqb/mqbs/package/mqbs.mem +++ b/src/groups/mqb/mqbs/package/mqbs.mem @@ -24,4 +24,3 @@ mqbs_storageprintutil mqbs_storageutil mqbs_virtualstorage mqbs_virtualstoragecatalog -mqbs_voidstorageiterator diff --git a/src/integration-tests/test_breathing.py b/src/integration-tests/test_breathing.py index c624b1a87..4679302e3 100644 --- a/src/integration-tests/test_breathing.py +++ b/src/integration-tests/test_breathing.py @@ -523,8 +523,8 @@ def test_verify_broadcast(cluster: Cluster): def test_verify_redelivery(cluster: Cluster): """Drop one consumer having unconfirmed message while there is another consumer unable to take the message (due to max_unconfirmed_messages - limit). Then start new consumer and make sure it does not crash (Ticket - 156808957) and receives that unconfirmed message. + limit). Then start new consumer and make sure it does not crash and + receives that unconfirmed message. """ proxies = cluster.proxy_cycle() @@ -558,6 +558,8 @@ def test_verify_redelivery(cluster: Cluster): consumer1.wait_push_event() before = consumer1.list(tc.URI_FANOUT_FOO, block=True) + assert len(before) == 1 + consumer2.wait_push_event() consumer1.stop_session(block=True) @@ -574,6 +576,8 @@ def test_verify_redelivery(cluster: Cluster): consumer1.wait_push_event() after = consumer1.list(tc.URI_FANOUT_FOO, block=True) + assert len(after) == 1 + assert before[0].payload == after[0].payload _stop_clients([producer, consumer1, consumer2]) diff --git a/src/integration-tests/test_subscriptions.py b/src/integration-tests/test_subscriptions.py index ef990545f..eeb0ff4c3 100644 --- a/src/integration-tests/test_subscriptions.py +++ b/src/integration-tests/test_subscriptions.py @@ -1539,7 +1539,7 @@ def test_no_capacity_all_optimization(cluster: Cluster): # Extra message 'm3' is good for 'consumer1' but cannot be delivered: # no capacity all condition observed. - assert optimization_monitor.has_new_message() + assert not optimization_monitor.has_new_message() consumer1.confirm_all() consumer1.expect_messages([m2], confirm=False) @@ -1557,8 +1557,9 @@ def test_no_capacity_all_optimization(cluster: Cluster): # pending messages: [m3, m4] # Posting the message 'm4' triggers routing, but the only consumer is full: - # no capacity all condition observed. - assert optimization_monitor.has_new_message() + # but the no capacity all condition is NOT observed because the app has + # memory ('resumePoint') + assert not optimization_monitor.has_new_message() consumer2 = Consumer(cluster, uri, ["y < 0"], max_unconfirmed_messages=1) @@ -1648,8 +1649,9 @@ def test_no_capacity_all_fanout(cluster: Cluster): # pending messages (bar): [m2, m3] # Extra message 'm3' is good for 'consumer_bar' but cannot be delivered: - # no capacity all condition observed. - assert optimization_monitor.has_new_message("bar") + # but the no capacity all condition is NOT observed because the app has + # memory ('resumePoint') + assert not optimization_monitor.has_new_message("bar") consumer_bar.confirm_all() consumer_bar.expect_messages([m2], confirm=False)