diff --git a/src/groups/mqb/mqba/mqba_clientsession.cpp b/src/groups/mqb/mqba/mqba_clientsession.cpp index 88eca8783e..f7fc90d36e 100644 --- a/src/groups/mqb/mqba/mqba_clientsession.cpp +++ b/src/groups/mqb/mqba/mqba_clientsession.cpp @@ -351,16 +351,16 @@ struct BuildAckOverflowFunctor { // ------------------------- ClientSessionState::ClientSessionState( - bslma::ManagedPtr& clientStatContext, - BlobSpPool* blobSpPool, - bdlbb::BlobBufferFactory* bufferFactory, - bmqp::EncodingType::Enum encodingType, - bslma::Allocator* allocator) + const bsl::shared_ptr& clientStatContext, + BlobSpPool* blobSpPool, + bdlbb::BlobBufferFactory* bufferFactory, + bmqp::EncodingType::Enum encodingType, + bslma::Allocator* allocator) : d_allocator_p(allocator) , d_channelBufferQueue(allocator) , d_unackedMessageInfos(d_allocator_p) , d_dispatcherClientData() -, d_statContext_mp(clientStatContext) +, d_statContext_sp(clientStatContext) , d_bufferFactory_p(bufferFactory) , d_blobSpPool_p(blobSpPool) , d_schemaEventBuilder(blobSpPool, encodingType, allocator) @@ -668,7 +668,7 @@ void ClientSession::sendAck(bmqt::AckResult::Enum status, queueStats = invalidQueueStats(); } else { - queueStats = subQueueCiter->value().d_stats.get(); + queueStats = subQueueCiter->value().d_stats_sp.get(); } } @@ -1667,23 +1667,10 @@ bool ClientSession::validateMessage(mqbi::QueueHandle** queueHandle, return false; // RETURN } - StreamsMap::iterator subQueueIt = - queueIt->second.d_subQueueInfosMap.findBySubIdSafe(queueId.subId()); - if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( - subQueueIt == queueIt->second.d_subQueueInfosMap.end())) { - BSLS_PERFORMANCEHINT_UNLIKELY_HINT; - - if (eventType == bmqp::EventType::e_CONFIRM) { - // Update invalid queue stats - invalidQueueStats()->onEvent( - mqbstat::QueueStatsClient::EventType::e_CONFIRM, - 1); - } - - *errorStream << "for an unknown subQueueId"; - - return false; // RETURN - } + // Do not lookup 'queueId.subId()'. + // 'QueueHandle::confirmMessageDispatched' does the check. + // Note, that it does not update stats (on "bmq://invalid/queue"). + // It does log warnings. *queueHandle = queueIt->second.d_handle_p; BSLS_ASSERT_SAFE(queueHandle); @@ -1729,13 +1716,6 @@ bool ClientSession::validateMessage(mqbi::QueueHandle** queueHandle, return false; // RETURN } - if (eventType == bmqp::EventType::e_CONFIRM) { - // Update stats for the queue (or subStream of the queue) - subQueueIt->value().d_stats->onEvent( - mqbstat::QueueStatsClient::EventType::e_CONFIRM, - 1); - } - return true; } @@ -1877,7 +1857,7 @@ void ClientSession::onPushEvent(const mqbi::DispatcherPushEvent& event) blob->length()); } else { - subQueueCiter->value().d_stats->onEvent( + subQueueCiter->value().onEvent( mqbstat::QueueStatsClient::EventType::e_PUSH, blob->length()); } @@ -2028,9 +2008,8 @@ void ClientSession::onPutEvent(const mqbi::DispatcherPutEvent& event) BSLS_ASSERT_SAFE(queueStatePtr && subQueueInfoPtr); BSLS_ASSERT_SAFE(queueStatePtr->d_handle_p); - subQueueInfoPtr->d_stats->onEvent( - mqbstat::QueueStatsClient::EventType::e_PUT, - appDataSp->length()); + subQueueInfoPtr->onEvent(mqbstat::QueueStatsClient::EventType::e_PUT, + appDataSp->length()); const bool isAtMostOnce = queueStatePtr->d_handle_p->queue()->isAtMostOnce(); @@ -2197,7 +2176,7 @@ mqbstat::QueueStatsClient* ClientSession::invalidQueueStats() d_state.d_invalidQueueStats.makeValue(); d_state.d_invalidQueueStats.value().initialize( "bmq://invalid/queue", - d_state.d_statContext_mp.get(), + d_state.d_statContext_sp.get(), d_state.d_allocator_p); // TBD: The queue uri should be '** INVALID QUEUE **', but that can // only be done once the stats UI panel has been updated to @@ -2400,17 +2379,17 @@ bool ClientSession::validatePutMessage(QueueState** queueState, // CREATORS ClientSession::ClientSession( - const bsl::shared_ptr& channel, - const bmqp_ctrlmsg::NegotiationMessage& negotiationMessage, - const bsl::string& sessionDescription, - mqbi::Dispatcher* dispatcher, - mqbblp::ClusterCatalog* clusterCatalog, - mqbi::DomainFactory* domainFactory, - bslma::ManagedPtr& clientStatContext, - ClientSessionState::BlobSpPool* blobSpPool, - bdlbb::BlobBufferFactory* bufferFactory, - bdlmt::EventScheduler* scheduler, - bslma::Allocator* allocator) + const bsl::shared_ptr& channel, + const bmqp_ctrlmsg::NegotiationMessage& negotiationMessage, + const bsl::string& sessionDescription, + mqbi::Dispatcher* dispatcher, + mqbblp::ClusterCatalog* clusterCatalog, + mqbi::DomainFactory* domainFactory, + const bsl::shared_ptr& clientStatContext, + ClientSessionState::BlobSpPool* blobSpPool, + bdlbb::BlobBufferFactory* bufferFactory, + bdlmt::EventScheduler* scheduler, + bslma::Allocator* allocator) : d_self(this) // use default allocator , d_operationState(e_RUNNING) , d_isDisconnecting(false) @@ -2431,7 +2410,7 @@ ClientSession::ClientSession( allocator) , d_queueSessionManager(this, *d_clientIdentity_p, - d_state.d_statContext_mp.get(), + d_state.d_statContext_sp, domainFactory, allocator) , d_clusterCatalog_p(clusterCatalog) diff --git a/src/groups/mqb/mqba/mqba_clientsession.h b/src/groups/mqb/mqba/mqba_clientsession.h index cfe5fec201..60a78e3e5c 100644 --- a/src/groups/mqb/mqba/mqba_clientsession.h +++ b/src/groups/mqb/mqba/mqba_clientsession.h @@ -149,8 +149,6 @@ struct ClientSessionState { typedef bsl::pair UnackedMessageInfoMapInsertRc; - typedef bslma::ManagedPtr StatContextMp; - public: // PUBLIC DATA @@ -173,7 +171,7 @@ struct ClientSessionState { /// Stat context dedicated to this domain, to use as the parent stat /// context for any queue in this domain. - StatContextMp d_statContext_mp; + const bsl::shared_ptr d_statContext_sp; /// Blob buffer factory to use. /// @@ -225,11 +223,11 @@ struct ClientSessionState { /// builder will use. Memory allocations are performed using the /// specified `allocator`. ClientSessionState( - bslma::ManagedPtr& clientStatContext, - BlobSpPool* blobSpPool, - bdlbb::BlobBufferFactory* bufferFactory, - bmqp::EncodingType::Enum encodingType, - bslma::Allocator* allocator); + const bsl::shared_ptr& clientStatContext, + BlobSpPool* blobSpPool, + bdlbb::BlobBufferFactory* bufferFactory, + bmqp::EncodingType::Enum encodingType, + bslma::Allocator* allocator); }; // =================== @@ -577,11 +575,11 @@ class ClientSession : public mqbnet::Session, mqbi::Dispatcher* dispatcher, mqbblp::ClusterCatalog* clusterCatalog, mqbi::DomainFactory* domainFactory, - bslma::ManagedPtr& clientStatContext, - ClientSessionState::BlobSpPool* blobSpPool, - bdlbb::BlobBufferFactory* bufferFactory, - bdlmt::EventScheduler* scheduler, - bslma::Allocator* allocator); + const bsl::shared_ptr& clientStatContext, + ClientSessionState::BlobSpPool* blobSpPool, + bdlbb::BlobBufferFactory* bufferFactory, + bdlmt::EventScheduler* scheduler, + bslma::Allocator* allocator); /// Destructor ~ClientSession() BSLS_KEYWORD_OVERRIDE; diff --git a/src/groups/mqb/mqba/mqba_clientsession.t.cpp b/src/groups/mqb/mqba/mqba_clientsession.t.cpp index 28e8f54fa4..9632b152a8 100644 --- a/src/groups/mqb/mqba/mqba_clientsession.t.cpp +++ b/src/groups/mqb/mqba/mqba_clientsession.t.cpp @@ -585,7 +585,7 @@ class MyMockDomain : public mqbmock::Domain { /// calls the specified `callback` with a new queue handle created /// using the specified `handleParameters`. The specified `uri` and /// `clientContext` are ignored. - void openQueue(BSLA_UNUSED const bmqt::Uri& uri, + void openQueue(const bmqt::Uri& uri, const bsl::shared_ptr& clientContext, const bmqp_ctrlmsg::QueueHandleParameters& handleParameters, @@ -605,8 +605,15 @@ class MyMockDomain : public mqbmock::Domain { handleParameters, d_allocator_p); - OpenQueueConfirmationCookie confirmationCookie; - confirmationCookie.createInplace(d_allocator_p, d_queueHandle.get()); + mqbi::OpenQueueConfirmationCookieSp confirmationCookie; + confirmationCookie.createInplace(d_allocator_p); + confirmationCookie->d_handle = d_queueHandle.get(); + + confirmationCookie->d_stats_sp.createInplace(d_allocator_p); + confirmationCookie->d_stats_sp->initialize( + uri, + clientContext->statContext().get(), + d_allocator_p); bmqp_ctrlmsg::Status status(d_allocator_p); status.category() = bmqp_ctrlmsg::StatusCategory::E_SUCCESS; @@ -648,18 +655,18 @@ class TestBench { public: // DATA - bdlbb::PooledBlobBufferFactory d_bufferFactory; - BlobSpPool d_blobSpPool; + bdlbb::PooledBlobBufferFactory d_bufferFactory; + BlobSpPool d_blobSpPool; bsl::shared_ptr d_channel; - mqbmock::Cluster d_cluster; - mqbmock::Dispatcher d_mockDispatcher; - MyMockDomain d_domain; - mqbmock::DomainFactory d_mockDomainFactory; - bslma::ManagedPtr d_clientStatContext_mp; - bdlmt::EventScheduler d_scheduler; - TestClock d_testClock; - mqba::ClientSession d_cs; - bslma::Allocator* d_allocator_p; + mqbmock::Cluster d_cluster; + mqbmock::Dispatcher d_mockDispatcher; + MyMockDomain d_domain; + mqbmock::DomainFactory d_mockDomainFactory; + const bsl::shared_ptr d_clientStatContext_sp; + bdlmt::EventScheduler d_scheduler; + TestClock d_testClock; + mqba::ClientSession d_cs; + bslma::Allocator* d_allocator_p; static const int k_PAYLOAD_LENGTH = 36; @@ -682,9 +689,8 @@ class TestBench { , d_mockDispatcher(allocator) , d_domain(&d_mockDispatcher, &d_cluster, atMostOnce, allocator) , d_mockDomainFactory(d_domain, allocator) - , d_clientStatContext_mp( - mqbstat::QueueStatsUtil::initializeStatContextClients(10, allocator) - .managedPtr()) + , d_clientStatContext_sp( + mqbstat::QueueStatsUtil::initializeStatContextClients(10, allocator)) , d_scheduler(bsls::SystemClockType::e_MONOTONIC, allocator) , d_testClock(d_scheduler) , d_cs(d_channel, @@ -693,7 +699,7 @@ class TestBench { setInDispatcherThread(&d_mockDispatcher), 0, // ClusterCatalog &d_mockDomainFactory, - d_clientStatContext_mp, + d_clientStatContext_sp, &d_blobSpPool, &d_bufferFactory, &d_scheduler, diff --git a/src/groups/mqb/mqbblp/mqbblp_cluster.cpp b/src/groups/mqb/mqbblp/mqbblp_cluster.cpp index 5495999524..b6f2be60d8 100644 --- a/src/groups/mqb/mqbblp/mqbblp_cluster.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_cluster.cpp @@ -389,8 +389,7 @@ void Cluster::sendAck(bmqt::AckResult::Enum status, const bmqt::MessageGUID& messageGUID, int queueId, const bslstl::StringRef& source, - mqbc::ClusterNodeSession* nodeSession, - bool isSelfGenerated) + mqbc::ClusterNodeSession* nodeSession) { // executed by the *DISPATCHER* thread @@ -417,21 +416,10 @@ void Cluster::sendAck(bmqt::AckResult::Enum status, uri_p = &it->second.d_handle_p->queue()->uri(); // If queue exists, report self generated NACK - if (isSelfGenerated) { - it->second.d_handle_p->queue() - ->stats() - ->onEvent(1); - } - } - else if (!isSelfGenerated) { - BMQU_THROTTLEDACTION_THROTTLE( - d_throttledFailedAckMessages, - BALL_LOG_WARN - << description() - << ": ACK message for queue with unknown queueId [" - << queueId << ", guid: " << messageGUID << ", for node: " - << nodeSession->clusterNode()->nodeDescription()); - return; // RETURN + + it->second.d_handle_p->queue() + ->stats() + ->onEvent(1); } // Throttle error log if this is a 'failed Ack': note that we log at @@ -468,9 +456,9 @@ void Cluster::sendAck(bmqt::AckResult::Enum status, cit->second.d_subQueueInfosMap.findBySubIdSafe( bmqp::QueueId::k_DEFAULT_SUBQUEUE_ID); if (subQueueCiter != cit->second.d_subQueueInfosMap.end()) { - subQueueCiter->value() - .d_clientStats - ->onEvent(1); + subQueueCiter->value().onEvent( + mqbstat::QueueStatsClient::EventType::e_ACK, + 1); } // In the case of Strong Consistency, a Receipt can arrive and trigger // an ACK after Producer closes subStream. @@ -506,8 +494,7 @@ void Cluster::sendAck(bmqt::AckResult::Enum status, const bmqt::MessageGUID& messageGUID, int queueId, const bslstl::StringRef& source, - mqbnet::ClusterNode* destination, - bool isSelfGenerated) + mqbnet::ClusterNode* destination) { // executed by the *DISPATCHER* thread @@ -519,49 +506,7 @@ void Cluster::sendAck(bmqt::AckResult::Enum status, d_clusterData.membership().getClusterNodeSession(destination); BSLS_ASSERT(nodeSession); - sendAck(status, - correlationId, - messageGUID, - queueId, - source, - nodeSession, - isSelfGenerated); -} - -void Cluster::generateNack(bmqt::AckResult::Enum status, - const bslstl::StringRef& nackReason, - const bmqp::PutHeader& putHeader, - mqbi::Queue* queue, - DispatcherClient* source, - const bsl::shared_ptr& appData, - const bsl::shared_ptr& options, - bool raiseAlarm) -{ - // executed by the *DISPATCHER* thread - // PRECONDITIONS - BSLS_ASSERT_SAFE(dispatcher()->inDispatcherThread(this)); - - mqbc::ClusterUtil::generateNack(status, - putHeader, - source, - dispatcher(), - appData, - options); - - // Report locally generated NACK - queue->stats()->onEvent(1); - - bmqu::MemOutStream os; - os << description() << ": Failed to relay PUT message " - << "[queueId: " << putHeader.queueId() - << ", GUID: " << putHeader.messageGUID() << "]. " - << "Reason: " << nackReason; - BMQU_THROTTLEDACTION_THROTTLE( - d_throttledFailedPutMessages, - if (raiseAlarm) { - BMQTSK_ALARMLOG_ALARM("CLUSTER") - << os.str() << BMQTSK_ALARMLOG_END; - } else { BALL_LOG_WARN << os.str(); }); + sendAck(status, correlationId, messageGUID, queueId, source, nodeSession); } void Cluster::processCommandDispatched(mqbcmd::ClusterResult* result, @@ -752,177 +697,6 @@ void Cluster::continueShutdownDispatched( d_clusterOrchestrator.queueHelper().processShutdownEvent(); } -void Cluster::onRelayPutEvent(const mqbi::DispatcherEvent& event) -{ - // executed by the *DISPATCHER* thread - - // PRECONDITIONS - BSLS_ASSERT_SAFE(dispatcher()->inDispatcherThread(this)); - BSLS_ASSERT_SAFE(mqbi::DispatcherEventType::e_PUT == event.type()); - BSLS_ASSERT_SAFE(event.asPutEvent()->isRelay()); - - const mqbi::DispatcherPutEvent* realEvent = event.asPutEvent(); - - // This relay-PUT message is enqueued by the RemoteQueue on either cluster - // (in case of replica) or clusterProxy (in case of proxy). This is a - // replica so this node just needs to forward the message to queue's - // partition's primary node (after appropriate checks). - // - // Note: Due to internal knowledge of the RemoteQueue component, we can be - // sure that there is exactly one PUT message contained in 'event'. - - const bmqp::PutHeader& ph = realEvent->putHeader(); - const int pid = realEvent->partitionId(); - bsls::Types::Uint64 genCount = realEvent->genCount(); - bsls::Types::Uint64 leaseId = d_state.partition(pid).primaryLeaseId(); - - if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY(genCount != leaseId)) { - BMQU_THROTTLEDACTION_THROTTLE( - d_throttledSkippedPutMessages, - BALL_LOG_WARN << description() - << ": skipping relay-PUT message [ queueId: " - << ph.queueId() << ", GUID: " << ph.messageGUID() - << "], genCount: " << genCount << " vs " << leaseId - << "."); - return; // RETURN - } - - mqbi::Queue* queue = d_clusterOrchestrator.queueHelper().lookupQueue( - ph.queueId()); - BSLS_ASSERT_SAFE(queue); - - if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( - mqbs::DataStore::k_INVALID_PARTITION_ID == pid)) { - BSLS_PERFORMANCEHINT_UNLIKELY_HINT; - - // Inform event's source of relay-PUT failure - bmqu::MemOutStream os; - os << "invalid partition [" << pid << "]"; - generateNack(bmqt::AckResult::e_INVALID_ARGUMENT, - os.str(), - ph, - queue, - event.source(), - 0, - 0, - true); // raiseAlarm - - return; // RETURN - } - - // DO process PUTs in the E_STOPPING state. This is for broadcast PUTs - // that "cross" StopRequest. Since 'RemoteQueue' does not buffer broadcast - // PUTs data, "crossed" PUTs will be lost. - // But do NOT send to E_STOPPING upstream. See below. - bmqp_ctrlmsg::NodeStatus::Value selfStatus = - d_clusterData.membership().selfNodeStatus(); - if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( - bmqp_ctrlmsg::NodeStatus::E_AVAILABLE != selfStatus && - bmqp_ctrlmsg::NodeStatus::E_STOPPING != selfStatus)) { - BSLS_PERFORMANCEHINT_UNLIKELY_HINT; - - // Inform event's source of relay-PUT failure - bmqu::MemOutStream os; - os << "self (replica node) not available. Self status: " << selfStatus; - generateNack(bmqt::AckResult::e_NOT_READY, - os.str(), - ph, - queue, - event.source(), - 0, - 0, - false); // raiseAlarm - - return; // RETURN - } - - BSLS_ASSERT(pid < static_cast(d_state.partitions().size())); - const ClusterStatePartitionInfo& pinfo = d_state.partition(pid); - - if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( - 0 == pinfo.primaryNode() || - bmqp_ctrlmsg::PrimaryStatus::E_ACTIVE != pinfo.primaryStatus())) { - BSLS_PERFORMANCEHINT_UNLIKELY_HINT; - - bmqu::MemOutStream os; - os << "no or non-active primary for partition [" << pid << "]"; - generateNack(bmqt::AckResult::e_NOT_READY, - os.str(), - ph, - queue, - event.source(), - realEvent->blob(), - realEvent->options(), - false); // raiseAlarm - - return; // RETURN - } - - if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( - pinfo.primaryNode()->nodeId() == - d_clusterData.membership().netCluster()->selfNodeId())) { - BSLS_PERFORMANCEHINT_UNLIKELY_HINT; - - // This should not occur - bmqu::MemOutStream os; - os << "self is primary for partition [" << pid << "]"; - generateNack(bmqt::AckResult::e_UNKNOWN, - os.str(), - ph, - queue, - event.source(), - 0, - 0, - false); // raiseAlarm - - return; // RETURN - } - - mqbc::ClusterNodeSession* ns = - d_clusterData.membership().getClusterNodeSession(pinfo.primaryNode()); - BSLS_ASSERT_SAFE(ns); - - bmqp_ctrlmsg::NodeStatus::Value primaryStatus = ns->nodeStatus(); - - // Do not send to E_STOPPING upstream. Self-NACK with data. - if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( - bmqp_ctrlmsg::NodeStatus::E_AVAILABLE != primaryStatus)) { - BSLS_PERFORMANCEHINT_UNLIKELY_HINT; - - bmqu::MemOutStream os; - os << "primary not available. Primary status: " << primaryStatus; - generateNack(bmqt::AckResult::e_NOT_READY, - os.str(), - ph, - queue, - event.source(), - realEvent->blob(), - realEvent->options(), - false); // raiseAlarm - - return; // RETURN - } - - bmqt::GenericResult::Enum rc = ns->clusterNode()->channel().writePut( - realEvent->putHeader(), - realEvent->blob(), - realEvent->state()); - if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( - rc != bmqt::GenericResult::e_SUCCESS)) { - BSLS_PERFORMANCEHINT_UNLIKELY_HINT; - - // This is non-recoverable, so we drop the PUT msg and log it. - - BMQU_THROTTLEDACTION_THROTTLE( - d_throttledSkippedPutMessages, - BALL_LOG_ERROR << description() << ": skipping relay-PUT message [" - << "queueId: " << ph.queueId() << ", GUID: " - << ph.messageGUID() << "] to primary node: " - << ns->clusterNode()->nodeDescription() - << ", rc: " << rc << "."); - } -} - void Cluster::onPutEvent(const mqbi::DispatcherPutEvent& event) { // executed by the *DISPATCHER* thread @@ -978,8 +752,7 @@ void Cluster::onPutEvent(const mqbi::DispatcherPutEvent& event) putIt.header().messageGUID(), putIt.header().queueId(), "Node unavailable", - ns, - true); // isSelfGenerated + ns); }; return; // RETURN } @@ -1009,9 +782,8 @@ void Cluster::onPutEvent(const mqbi::DispatcherPutEvent& event) putIt.header().messageGUID(), queueId.id(), "putEvent::UnknownQueue", - ns, - true); // isSelfGenerated - continue; // CONTINUE + ns); + continue; // CONTINUE } const QueueState& queueState = queueIt->second; @@ -1040,9 +812,8 @@ void Cluster::onPutEvent(const mqbi::DispatcherPutEvent& event) putIt.header().messageGUID(), queueId.id(), "putEvent::notWritable", - ns, - true); // isSelfGenerated - continue; // CONTINUE + ns); + continue; // CONTINUE } // Retrieve the payload of that message @@ -1066,8 +837,7 @@ void Cluster::onPutEvent(const mqbi::DispatcherPutEvent& event) putIt.header().messageGUID(), queueId.id(), "putEvent::failedLoadApplicationData", - ns, - true); // isSelfGenerated + ns); continue; // CONTINUE } @@ -1076,11 +846,9 @@ void Cluster::onPutEvent(const mqbi::DispatcherPutEvent& event) queueState.d_subQueueInfosMap.findBySubId( bmqp::QueueId::k_DEFAULT_SUBQUEUE_ID); - subQueueCiter->value() - .d_clientStats - ->onEvent( - - appDataSp->length()); + subQueueCiter->value().onEvent( + mqbstat::QueueStatsClient::EventType::e_PUT, + appDataSp->length()); // TBD: groupId: Similar to 'appDataSp' above, load 'optionsSp' here, // using something like PutMessageIterator::loadOptions(). @@ -1110,8 +878,7 @@ void Cluster::onPutEvent(const mqbi::DispatcherPutEvent& event) putIt.header().messageGUID(), queueId.id(), "putEvent::auto-ACK", - ns, - true); // isSelfGenerated + ns); } bmqp::PutHeaderFlagUtil::unsetFlag( &flags, @@ -1143,74 +910,6 @@ void Cluster::onPutEvent(const mqbi::DispatcherPutEvent& event) } } -void Cluster::onAckEvent(const mqbi::DispatcherAckEvent& event) -{ - // executed by the *DISPATCHER* thread - - // PRECONDITIONS - BSLS_ASSERT_SAFE(dispatcher()->inDispatcherThread(this)); - BSLS_ASSERT_SAFE(!event.isRelay()); - - // This ACK message is enqueued by mqbblp::Queue on this node, and needs to - // be forwarded to 'event.clusterNode()' (the replica node). - - // NOTE: we do not log anything here, all logging is done in 'sendAck'. - - const bmqp::AckMessage& ackMessage = event.ackMessage(); - bmqp_ctrlmsg::NodeStatus::Value selfStatus = - d_clusterData.membership().selfNodeStatus(); - if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( - bmqp_ctrlmsg::NodeStatus::E_AVAILABLE != selfStatus && - bmqp_ctrlmsg::NodeStatus::E_STOPPING != selfStatus)) { - BSLS_PERFORMANCEHINT_UNLIKELY_HINT; - // Drop ACK coz self is unavailable - BMQU_THROTTLEDACTION_THROTTLE( - d_throttledFailedAckMessages, - BALL_LOG_WARN << "Dropping an ACK for queue [queueId: " - << ackMessage.queueId() - << ", guid: " << ackMessage.messageGUID() - << ", status: " << ackMessage.status() - << "] for node " - << event.clusterNode()->nodeDescription() - << ". Reason: self (primary node) not available. " - << "Node status: " << selfStatus); - return; // RETURN - } - - mqbc::ClusterNodeSession* ns = - d_clusterData.membership().getClusterNodeSession(event.clusterNode()); - BSLS_ASSERT_SAFE(ns); - - bmqp_ctrlmsg::NodeStatus::Value downstreamStatus = ns->nodeStatus(); - - if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( - bmqp_ctrlmsg::NodeStatus::E_AVAILABLE != downstreamStatus && - bmqp_ctrlmsg::NodeStatus::E_STOPPING != downstreamStatus)) { - BSLS_PERFORMANCEHINT_UNLIKELY_HINT; - // Drop the ACK because downstream node is either starting, in the - // maintenance mode, or shut down. - BMQU_THROTTLEDACTION_THROTTLE( - d_throttledFailedAckMessages, - BALL_LOG_WARN << "Dropping an ACK for queue [queueId: " - << ackMessage.queueId() - << ", guid: " << ackMessage.messageGUID() - << ", status: " << ackMessage.status() - << "] for node " - << event.clusterNode()->nodeDescription() - << ". Reason: target node not available. " - << "Node status: " << downstreamStatus); - return; // RETURN - } - - sendAck(bmqp::ProtocolUtil::ackResultFromCode(ackMessage.status()), - ackMessage.correlationId(), - ackMessage.messageGUID(), - ackMessage.queueId(), - "onAckEvent", - event.clusterNode(), - false); // isSelfGenerated -} - void Cluster::onRelayAckEvent(const mqbi::DispatcherAckEvent& event) { // executed by the *DISPATCHER* thread @@ -1494,18 +1193,8 @@ Cluster::validateMessage(mqbi::QueueHandle** queueHandle, return ValidationResult::k_UNKNOWN_QUEUE; // RETURN } - const QueueState& queueState = queueIt->second; - StreamsMap::const_iterator subQueueIt = - queueState.d_subQueueInfosMap.findBySubIdSafe(queueId.subId()); - - *queueHandle = queueState.d_handle_p; - - if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( - subQueueIt == queueState.d_subQueueInfosMap.end())) { - BSLS_PERFORMANCEHINT_UNLIKELY_HINT; - - return ValidationResult::k_UNKNOWN_SUBQUEUE; // RETURN - } + const QueueState& queueState = queueIt->second; + *queueHandle = queueState.d_handle_p; if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( queueState.d_isFinalCloseQueueReceived)) { @@ -1514,13 +1203,6 @@ Cluster::validateMessage(mqbi::QueueHandle** queueHandle, return ValidationResult::k_FINAL; // RETURN } - if (eventType == bmqp::EventType::e_CONFIRM) { - // Update client stats - subQueueIt->value() - .d_clientStats - ->onEvent(1); - } - return ValidationResult::k_SUCCESS; } @@ -1583,63 +1265,137 @@ void Cluster::onRelayRejectEvent(const mqbi::DispatcherRejectEvent& event) } } -void Cluster::onRelayConfirmEvent(const mqbi::DispatcherConfirmEvent& event) +mqbi::InlineResult::Enum +Cluster::sendConfirmInline(int partitionId, + const bmqp::ConfirmMessage& message) { - // executed by the *DISPATCHER* thread + // executed by *ANY* thread - // PRECONDITIONS - BSLS_ASSERT_SAFE(dispatcher()->inDispatcherThread(this)); - BSLS_ASSERT_SAFE(event.isRelay()); + // Replacing 'onRelayConfirmEvent' // This relay-CONFIRM message is enqueued by the RemoteQueue on either // cluster (in case of replica) or clusterProxy (in case of proxy). This // is a replica so this node just needs to forward the message to queue's // partition's primary node (after appropriate checks). - const bmqp::ConfirmMessage& confirmMsg = event.confirmMessage(); - const int pid = event.partitionId(); + if (mqbs::DataStore::k_INVALID_PARTITION_ID == partitionId) { + return mqbi::InlineResult::e_INVALID_PARTITION; // RETURN + } - const int id = confirmMsg.queueId(); - const unsigned int subId = static_cast( - confirmMsg.subQueueId()); - const bmqp::QueueId queueId(id, subId); - mqbc::ClusterNodeSession* ns = 0; + mqbc::GateKeeper::Status primaryStatus(d_state.gatePrimary(partitionId)); - bdlma::LocalSequentialAllocator<256> localAllocator(d_allocator_p); - bmqu::MemOutStream errorStream(&localAllocator); + if (!primaryStatus.isOpen()) { + return mqbi::InlineResult::e_INVALID_PRIMARY; // RETURN + } - bool isValid = validateRelayMessage(&ns, &errorStream, pid); - if (isValid) { - bmqt::GenericResult::Enum rc = - ns->clusterNode()->channel().writeConfirm( - queueId.id(), - queueId.subId(), - confirmMsg.messageGUID()); - if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( - rc != bmqt::GenericResult::e_SUCCESS)) { - BSLS_PERFORMANCEHINT_UNLIKELY_HINT; + // This assumes thread-safe access to partition info - // This is non-recoverable, so we drop the CONFIRM msg and log it. - BMQU_THROTTLEDACTION_THROTTLE( - d_throttledDroppedConfirmMessages, - BALL_LOG_ERROR << description() << ": dropping CONFIRM message" - << " [queueId: " << queueId.id() - << ", subQueuId: " << queueId.subId() - << ", GUID: " << confirmMsg.messageGUID() - << "] to node " - << ns->clusterNode()->nodeDescription() - << ", ConfirmBuilder rc: " << rc << "."); - } + const ClusterStatePartitionInfo& pinfo = d_state.partition(partitionId); + mqbc::ClusterNodeSession* ns = pinfo.primaryNodeSession(); + BSLS_ASSERT_SAFE(ns); + + mqbc::GateKeeper::Status nodeStatus(ns->gateConfirm()); + + if (!nodeStatus.isOpen()) { + return mqbi::InlineResult::e_UNAVAILABLE; // RETURN } - else { - BMQU_THROTTLEDACTION_THROTTLE( - d_throttledFailedRejectMessages, - BALL_LOG_WARN << "Failed to relay CONFIRM message " - << "[queueId: " << queueId.id() - << ", subQueueId: " << queueId.subId() - << ", GUID: " << confirmMsg.messageGUID() << "]. " - << errorStream.str()); + + mqbnet::ClusterNode* primary = ns->clusterNode(); + BSLS_ASSERT_SAFE(primary); + + // This assumes thread-safe access to + // d_clusterData.membership().netCluster()->selfNodeId() + if (primary->nodeId() == + d_clusterData.membership().netCluster()->selfNodeId()) { + return mqbi::InlineResult::e_SELF_PRIMARY; // RETURN + } + + bmqt::GenericResult::Enum rc = primary->channel().writeConfirm( + message.queueId(), + message.subQueueId(), + message.messageGUID()); + + if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( + rc != bmqt::GenericResult::e_SUCCESS)) { + BSLS_PERFORMANCEHINT_UNLIKELY_HINT; + + return mqbi::InlineResult::e_CHANNEL_ERROR; // RETURN + } + + return mqbi::InlineResult::e_SUCCESS; +} + +mqbi::InlineResult::Enum +Cluster::sendPutInline(int partitionId, + const bmqp::PutHeader& putHeader, + const bsl::shared_ptr& appData, + BSLA_UNUSED const bsl::shared_ptr& options, + const bsl::shared_ptr& state, + bsls::Types::Uint64 genCount) +{ + // executed by *ANY* thread + + // Replacing 'Cluster::onRelayPutEvent' + + // This relay-PUT message is enqueued by the RemoteQueue on either cluster + // (in case of replica) or clusterProxy (in case of proxy). This is a + // replica so this node just needs to forward the message to queue's + // partition's primary node (after appropriate checks). + + if (mqbs::DataStore::k_INVALID_PARTITION_ID == partitionId) { + return mqbi::InlineResult::e_INVALID_PARTITION; // RETURN } + + mqbc::GateKeeper::Status primaryStatus(d_state.gatePrimary(partitionId)); + + if (!primaryStatus.isOpen()) { + return mqbi::InlineResult::e_INVALID_PRIMARY; // RETURN + } + + // This assumes thread-safe access to partition info + + const ClusterStatePartitionInfo& pinfo = d_state.partition(partitionId); + bsls::Types::Uint64 leaseId = + d_state.partition(partitionId).primaryLeaseId(); + + if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY(genCount != leaseId)) { + return mqbi::InlineResult::e_INVALID_GEN_COUNT; // RETURN + } + + mqbnet::ClusterNode* primary = pinfo.primaryNode(); + BSLS_ASSERT_SAFE(primary); + + mqbc::ClusterNodeSession* primaryNodeSession = + d_clusterData.membership().getClusterNodeSession(primary); + + BSLS_ASSERT_SAFE(primaryNodeSession); + + mqbc::GateKeeper::Status nodeStatus(primaryNodeSession->gatePut()); + + if (!nodeStatus.isOpen()) { + // This checks both self status and the destination status + return mqbi::InlineResult::e_UNAVAILABLE; // RETURN + } + + // This assumes thread-safe access to + // d_clusterData.membership().netCluster()->selfNodeId() + if (primary->nodeId() == + d_clusterData.membership().netCluster()->selfNodeId()) { + return mqbi::InlineResult::e_SELF_PRIMARY; // RETURN + } + + bmqt::GenericResult::Enum rc = primary->channel().writePut(putHeader, + appData, + state); + + if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( + rc != bmqt::GenericResult::e_SUCCESS)) { + BSLS_PERFORMANCEHINT_UNLIKELY_HINT; + + return mqbi::InlineResult::e_CHANNEL_ERROR; // RETURN + } + + return mqbi::InlineResult::e_SUCCESS; } bool Cluster::validateRelayMessage(mqbc::ClusterNodeSession** ns, @@ -1703,145 +1459,6 @@ bool Cluster::validateRelayMessage(mqbc::ClusterNodeSession** ns, return true; } -void Cluster::onPushEvent(const mqbi::DispatcherPushEvent& event) -{ - // executed by the *DISPATCHER* thread - - // PRECONDITIONS - BSLS_ASSERT_SAFE(dispatcher()->inDispatcherThread(this)); - BSLS_ASSERT_SAFE(!event.isRelay()); - - // This PUSH message is enqueued by mqbblp::Queue/QueueHandle on this node, - // and needs to be forwarded to 'event.clusterNode()' (the replica node, - // which is the client). Note that replica is already expected to have the - // payload, and so, primary (this node) sends only the guid and, if - // applicable, the associated subQueueIds. - - bmqp_ctrlmsg::NodeStatus::Value selfStatus = - d_clusterData.membership().selfNodeStatus(); - if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( - bmqp_ctrlmsg::NodeStatus::E_AVAILABLE != selfStatus)) { - BSLS_PERFORMANCEHINT_UNLIKELY_HINT; - - // Drop PUSH coz self is going down - BMQU_THROTTLEDACTION_THROTTLE( - d_throttledFailedPushMessages, - BALL_LOG_WARN << "Dropping a PUSH for queue [queueId: " - << event.queueId() << ", guid: " << event.guid() - << "] for node " - << event.clusterNode()->nodeDescription() - << ". Reason: self (primary node) not available." - << " Node status: " << selfStatus); - return; // RETURN - } - - mqbc::ClusterNodeSession* ns = - d_clusterData.membership().getClusterNodeSession(event.clusterNode()); - BSLS_ASSERT_SAFE(ns); - - if (bmqp_ctrlmsg::NodeStatus::E_AVAILABLE != ns->nodeStatus()) { - // Target node is not AVAILABLE, so we don't send this PUSH msg to it. - // Note that this PUSH msg was dispatched by the queue handle - // representing the target node, and will be in its 'pending list'. - - BMQU_THROTTLEDACTION_THROTTLE( - d_throttledFailedPushMessages, - BALL_LOG_WARN << description() - << ": Failed to send PUSH message [queueId: " - << event.queueId() << ", GUID: " << event.guid() - << "] to target node: " - << event.clusterNode()->nodeDescription() - << ". Reason: node not available. " - << "Target node status: " << ns->nodeStatus()); - - return; // RETURN - } - - QueueHandleMap& queueHandles = ns->queueHandles(); - QueueHandleMapIter queueIt = queueHandles.find(event.queueId()); - if (queueIt == queueHandles.end()) { - BMQU_THROTTLEDACTION_THROTTLE( - d_throttledFailedPushMessages, - BALL_LOG_WARN << description() - << ": PUSH message for queue with unknown queueId [" - << event.queueId() << ", guid: " << event.guid() - << "] to target node: " - << event.clusterNode()->nodeDescription()); - - return; // RETURN - } - - // Build push event using PushEventBuilder. - const QueueState& queueState = queueIt->second; - - // Update stats - // TODO: Extract this and the version from 'mqba::ClientSession' to a - // function - for (bmqp::Protocol::SubQueueInfosArray::size_type i = 0; - i < event.subQueueInfos().size(); - ++i) { - StreamsMap::const_iterator subQueueCiter = - queueState.d_subQueueInfosMap.findBySubscriptionId( - event.subQueueInfos()[i].id()); - - subQueueCiter->value() - .d_clientStats - ->onEvent( - event.blob() ? event.blob()->length() : 0); - } - - bmqt::GenericResult::Enum rc = bmqt::GenericResult::e_SUCCESS; - // TBD: groupId: also pass options to the 'PushEventBuilder::packMessage' - // routine below. - - if (queueState.d_handle_p->queue()->isAtMostOnce()) { - // If it's at most once, then we explicitly send the payload since it's - // in-mem mode and there's been no replication (i.e. no preceding - // STORAGE message). - BSLS_ASSERT_SAFE(event.blob()); - rc = ns->clusterNode()->channel().writePush( - event.blob(), - event.queueId(), - event.guid(), - 0, - event.compressionAlgorithmType(), - event.messagePropertiesInfo(), - event.subQueueInfos()); - } - else { - int flags = 0; - - if (event.isOutOfOrderPush()) { - bmqp::PushHeaderFlagUtil::setFlag( - &flags, - bmqp::PushHeaderFlags::e_OUT_OF_ORDER); - } - - rc = ns->clusterNode()->channel().writePush( - event.queueId(), - event.guid(), - flags, - event.compressionAlgorithmType(), - event.messagePropertiesInfo(), - event.subQueueInfos()); - } - - if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( - rc != bmqt::GenericResult::e_SUCCESS)) { - BSLS_PERFORMANCEHINT_UNLIKELY_HINT; - - // This is non-recoverable, so we drop the PUSH msg and log it. - - BMQU_THROTTLEDACTION_THROTTLE( - d_throttledDroppedPushMessages, - BALL_LOG_ERROR << description() << ": dropping PUSH message " - << "[queueId: " << event.queueId() << ", guid: " - << event.guid() << "] to target node: " - << event.clusterNode()->nodeDescription() - << ", PushBuilder rc: " << rc << "."); - } -} - void Cluster::onRelayPushEvent(const mqbi::DispatcherPushEvent& event) { // executed by the *DISPATCHER* thread @@ -2416,14 +2033,12 @@ Cluster::Cluster(const bslstl::StringRef& name, &d_state, d_allocators.get("ClusterOrchestrator")) , d_clusterMonitor(&d_clusterData, &d_state, d_allocator_p) -, d_throttledFailedPutMessages(5000, 5) // 5 logs per 5s interval -, d_throttledSkippedPutMessages(5000, 5) // 5 logs per 5s interval -, d_throttledFailedAckMessages(5000, 5) // 5 logs per 5s interval -, d_throttledDroppedAckMessages(5000, 5) // 5 logs per 5s interval -, d_throttledFailedConfirmMessages(5000, 5) // 5 logs per 5s interval -, d_throttledDroppedConfirmMessages(5000, 5) // 5 logs per 5s interval -, d_throttledFailedPushMessages(5000, 5) // 5 logs per 5s interval -, d_throttledDroppedPushMessages(5000, 5) // 5 logs per 5s interval +, d_throttledFailedPutMessages(5000, 5) // 5 logs per 5s interval +, d_throttledFailedAckMessages(5000, 5) // 5 logs per 5s interval +, d_throttledDroppedAckMessages(5000, 5) // 5 logs per 5s interval +, d_throttledFailedConfirmMessages(5000, 5) // 5 logs per 5s interval +, d_throttledFailedPushMessages(5000, 5) // 5 logs per 5s interval +, d_throttledDroppedPushMessages(5000, 5) // 5 logs per 5s interval , d_logSummarySchedulerHandle() , d_queueGcSchedulerHandle() , d_stopRequestsManager_p(stopRequestsManager) @@ -2449,22 +2064,23 @@ Cluster::Cluster(const bslstl::StringRef& name, NodeListIter endIter = netCluster_p->nodes().end(); for (; nodeIter != endIter; ++nodeIter) { + // Create stat context for each cluster node + bmqst::StatContextConfiguration config((*nodeIter)->hostName()); + + StatContextMp statContextMp = + d_clusterData.clusterNodesStatContext()->addSubcontext(config); + StatContextSp statContextSp(statContextMp, d_allocator_p); + mqbc::ClusterMembership::ClusterNodeSessionSp nodeSessionSp; nodeSessionSp.createInplace(d_allocator_p, this, *nodeIter, d_clusterData.identity().name(), d_clusterData.identity().identity(), + statContextSp, d_allocator_p); - nodeSessionSp->setNodeStatus(bmqp_ctrlmsg::NodeStatus::E_UNKNOWN); - - // Create stat context for each cluster node - bmqst::StatContextConfiguration config((*nodeIter)->hostName()); - - StatContextMp statContextMp = - d_clusterData.clusterNodesStatContext()->addSubcontext(config); - StatContextSp statContextSp(statContextMp, d_allocator_p); - nodeSessionSp->statContext() = statContextSp; + nodeSessionSp->setNodeStatus(bmqp_ctrlmsg::NodeStatus::E_UNKNOWN, + bmqp_ctrlmsg::NodeStatus::E_UNKNOWN); nodeSessionMap.insert(bsl::make_pair(*nodeIter, nodeSessionSp)); @@ -3197,14 +2813,9 @@ void Cluster::onDispatcherEvent(const mqbi::DispatcherEvent& event) } break; // BREAK case mqbi::DispatcherEventType::e_PUT: { const mqbi::DispatcherPutEvent& realEvent = *event.asPutEvent(); - if (realEvent.isRelay()) { - // We pass a parent object event here because the implementation - // uses `source()` field from this parent object - onRelayPutEvent(event); - } - else { - onPutEvent(realEvent); - } + BSLS_ASSERT_SAFE(!realEvent.isRelay()); + + onPutEvent(realEvent); } break; // BREAK case mqbi::DispatcherEventType::e_ACK: { const mqbi::DispatcherAckEvent& realEvent = *event.asAckEvent(); @@ -3212,18 +2823,14 @@ void Cluster::onDispatcherEvent(const mqbi::DispatcherEvent& event) onRelayAckEvent(realEvent); } else { - onAckEvent(realEvent); + BALL_LOG_ERROR << "Received unexpected ACK event"; } } break; // BREAK case mqbi::DispatcherEventType::e_CONFIRM: { const mqbi::DispatcherConfirmEvent& realEvent = *event.asConfirmEvent(); - if (realEvent.isRelay()) { - onRelayConfirmEvent(realEvent); - } - else { - onConfirmEvent(realEvent); - } + BSLS_ASSERT_SAFE(!realEvent.isRelay()); + onConfirmEvent(realEvent); } break; case mqbi::DispatcherEventType::e_REJECT: { const mqbi::DispatcherRejectEvent& realEvent = *event.asRejectEvent(); @@ -3255,7 +2862,7 @@ void Cluster::onDispatcherEvent(const mqbi::DispatcherEvent& event) onRelayPushEvent(realEvent); } else { - onPushEvent(realEvent); + BALL_LOG_ERROR << "Received unexpected PUSH event"; } } break; // BREAK case mqbi::DispatcherEventType::e_REPLICATION_RECEIPT: diff --git a/src/groups/mqb/mqbblp/mqbblp_cluster.h b/src/groups/mqb/mqbblp/mqbblp_cluster.h index 57f1537827..e17f4d0f68 100644 --- a/src/groups/mqb/mqbblp/mqbblp_cluster.h +++ b/src/groups/mqb/mqbblp/mqbblp_cluster.h @@ -271,9 +271,6 @@ class Cluster : public mqbi::Cluster, /// Throttling parameters for failed PUT messages. bmqu::ThrottledActionParams d_throttledFailedPutMessages; - /// Throttling parameters for dropped PUT messages. - bmqu::ThrottledActionParams d_throttledSkippedPutMessages; - /// Throttling parameters for failed ACK messages. bmqu::ThrottledActionParams d_throttledFailedAckMessages; @@ -286,9 +283,6 @@ class Cluster : public mqbi::Cluster, /// Throttling parameters for failed REJECT messages. bmqu::ThrottledActionParams d_throttledFailedRejectMessages; - /// Throttling parameters for dropped CONFIRM messages. - bmqu::ThrottledActionParams d_throttledDroppedConfirmMessages; - /// Throttling parameters for dropped REJECT messages. bmqu::ThrottledActionParams d_throttledDroppedRejectMessages; @@ -336,47 +330,26 @@ class Cluster : public mqbi::Cluster, /// Append an ACK message to the session's ack builder, with the /// specified `status`, `correlationId`, `messageGUID` and `queueId` to /// the specified `destination` node. The specified `source` is used - /// when logging, to indicate the origin of the ACK. The specified - /// `isSelfGenerated` flag indicates whether the ACK is originally - /// generated from this object, or just relayed through it. + /// when logging, to indicate the origin of the ACK. void sendAck(bmqt::AckResult::Enum status, int correlationId, const bmqt::MessageGUID& messageGUID, int queueId, const bslstl::StringRef& source, - mqbnet::ClusterNode* destination, - bool isSelfGenerated); + mqbnet::ClusterNode* destination); /// Append an ACK message to the session's ack builder, with the /// specified `status`, `correlationId`, `messageGUID` and `queueId` to /// the cluster node identified by the specified cluster `nodeSession`. /// The specified `source` is used when logging, to indicate the origin - /// of the ACK. The specified `isSelfGenerated` flag indicates whether - /// the ACK is originally generated from this object, or just relayed - /// through it. + /// of the ACK. void sendAck(bmqt::AckResult::Enum status, int correlationId, const bmqt::MessageGUID& messageGUID, int queueId, const bslstl::StringRef& source, - mqbc::ClusterNodeSession* nodeSession, - bool isSelfGenerated); - - /// Generate a nack with the specified `status` and `nackReason` for a - /// PUT message having the specified `putHeader` for the specified - /// `queue` from the specified `source`. The nack is replied to the - /// `source`. The specified `raiseAlarm` flag determines whether an - /// alarm should be raised for this nack. - void generateNack(bmqt::AckResult::Enum status, - const bslstl::StringRef& nackReason, - const bmqp::PutHeader& putHeader, - mqbi::Queue* queue, - DispatcherClient* source, - const bsl::shared_ptr& appData, - const bsl::shared_ptr& options, - bool raiseAlarm); + mqbc::ClusterNodeSession* nodeSession); - /// Executed by dispatcher thread. void processCommandDispatched(mqbcmd::ClusterResult* result, const mqbcmd::ClusterCommand& command); @@ -397,22 +370,14 @@ class Cluster : public mqbi::Cluster, void onPutEvent(const mqbi::DispatcherPutEvent& event); - void onRelayPutEvent(const mqbi::DispatcherEvent& event); - - void onAckEvent(const mqbi::DispatcherAckEvent& event); - void onRelayAckEvent(const mqbi::DispatcherAckEvent& event); void onConfirmEvent(const mqbi::DispatcherConfirmEvent& event); - void onRelayConfirmEvent(const mqbi::DispatcherConfirmEvent& event); - void onRejectEvent(const mqbi::DispatcherRejectEvent& event); void onRelayRejectEvent(const mqbi::DispatcherRejectEvent& event); - void onPushEvent(const mqbi::DispatcherPushEvent& event); - void onRelayPushEvent(const mqbi::DispatcherPushEvent& event); /// Validate a message of the specified `eventType` using the specified @@ -615,6 +580,25 @@ class Cluster : public mqbi::Cluster, /// Load the cluster state to the specified `out` object. void loadClusterStatus(mqbcmd::ClusterResult* out) BSLS_KEYWORD_OVERRIDE; + /// Send the specified CONFIRM 'message' for the specified 'partitionId' + /// without switching thread context. + /// 'onRelayConfirmEvent' replacement. + mqbi::InlineResult::Enum sendConfirmInline( + int partitionId, + const bmqp::ConfirmMessage& message) BSLS_KEYWORD_OVERRIDE; + + /// Send PUT message for the specified 'partitionId' using the specified + /// 'putHeader', 'appData', 'options', 'state', 'genCount' without + /// switching thread context. + /// 'onRelayPutEvent' replacement. + mqbi::InlineResult::Enum + sendPutInline(int partitionId, + const bmqp::PutHeader& putHeader, + const bsl::shared_ptr& appData, + const bsl::shared_ptr& options, + const bsl::shared_ptr& state, + bsls::Types::Uint64 genCount) BSLS_KEYWORD_OVERRIDE; + /// Purge and force GC queues in this cluster on a given domain. void purgeAndGCQueueOnDomain(mqbcmd::ClusterResult* result, const bsl::string& domainName) diff --git a/src/groups/mqb/mqbblp/mqbblp_clusterorchestrator.cpp b/src/groups/mqb/mqbblp/mqbblp_clusterorchestrator.cpp index 485d2dcf09..87ab17c7a2 100644 --- a/src/groups/mqb/mqbblp/mqbblp_clusterorchestrator.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_clusterorchestrator.cpp @@ -856,7 +856,9 @@ void ClusterOrchestrator::processStopRequest( << ", current status: " << ns->nodeStatus() << ", new status: " << bmqp_ctrlmsg::NodeStatus::E_STOPPING; - ns->setNodeStatus(bmqp_ctrlmsg::NodeStatus::E_STOPPING); + bmqp_ctrlmsg::NodeStatus::Value selfStatus = + d_clusterData_p->membership().selfNodeStatus(); + ns->setNodeStatus(bmqp_ctrlmsg::NodeStatus::E_STOPPING, selfStatus); processNodeStoppingNotification(ns, &request); } @@ -1043,7 +1045,9 @@ void ClusterOrchestrator::processNodeStatusAdvisory( << ", current status: " << ns->nodeStatus() << ", new status: " << nsAdvisory.status(); - ns->setNodeStatus(nsAdvisory.status()); + bmqp_ctrlmsg::NodeStatus::Value selfStatus = + d_clusterData_p->membership().selfNodeStatus(); + ns->setNodeStatus(nsAdvisory.status(), selfStatus); if (bmqp_ctrlmsg::NodeStatus::E_STARTING == nsAdvisory.status()) { return; // RETURN @@ -1063,8 +1067,7 @@ void ClusterOrchestrator::processNodeStatusAdvisory( } if (bmqp_ctrlmsg::NodeStatus::E_AVAILABLE == nsAdvisory.status()) { - if (bmqp_ctrlmsg::NodeStatus::E_STOPPING == - d_clusterData_p->membership().selfNodeStatus()) { + if (bmqp_ctrlmsg::NodeStatus::E_STOPPING == selfStatus) { return; // RETURN } @@ -1148,6 +1151,8 @@ void ClusterOrchestrator::processNodeStateChangeEvent( d_clusterData_p->membership().getClusterNodeSession(node); BSLS_ASSERT_SAFE(ns); + bmqp_ctrlmsg::NodeStatus::Value selfStatus = + d_clusterData_p->membership().selfNodeStatus(); if (isAvailable) { if (bmqp_ctrlmsg::NodeStatus::E_UNAVAILABLE == ns->nodeStatus()) { // Current status of the peer node is unavailable, which means we @@ -1167,7 +1172,7 @@ void ClusterOrchestrator::processNodeStateChangeEvent( // Node is connected, but we don't know its status as of yet so we mark // it appropriately. - ns->setNodeStatus(bmqp_ctrlmsg::NodeStatus::E_UNKNOWN); + ns->setNodeStatus(bmqp_ctrlmsg::NodeStatus::E_UNKNOWN, selfStatus); // Send self's status to the node. @@ -1177,13 +1182,12 @@ void ClusterOrchestrator::processNodeStateChangeEvent( bmqp_ctrlmsg::NodeStatusAdvisory& advisory = clusterMsg.choice().makeNodeStatusAdvisory(); - advisory.status() = d_clusterData_p->membership().selfNodeStatus(); + advisory.status() = selfStatus; d_clusterData_p->messageTransmitter().sendMessage(controlMsg, node); updateDatumStats(ns); - if (bmqp_ctrlmsg::NodeStatus::E_STOPPING == - d_clusterData_p->membership().selfNodeStatus()) { + if (bmqp_ctrlmsg::NodeStatus::E_STOPPING == selfStatus) { return; // RETURN } @@ -1222,7 +1226,7 @@ void ClusterOrchestrator::processNodeStateChangeEvent( // 'node' has gone down. Mark it as UNAVAILABLE. - ns->setNodeStatus(bmqp_ctrlmsg::NodeStatus::E_UNAVAILABLE); + ns->setNodeStatus(bmqp_ctrlmsg::NodeStatus::E_UNAVAILABLE, selfStatus); // Cancel all outstanding requests to that node, with the 'CANCELED' // category and the 'e_NODE_DOWN' code. diff --git a/src/groups/mqb/mqbblp/mqbblp_clusterproxy.cpp b/src/groups/mqb/mqbblp/mqbblp_clusterproxy.cpp index 8676a3e94b..59d8fc4cc4 100644 --- a/src/groups/mqb/mqbblp/mqbblp_clusterproxy.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_clusterproxy.cpp @@ -86,30 +86,6 @@ void completeShutDown(const CompletionCallback& callback) // ------------------ // PRIVATE MANIPULATORS -void ClusterProxy::generateNack(bmqt::AckResult::Enum status, - const bmqp::PutHeader& putHeader, - DispatcherClient* source, - const bsl::shared_ptr& appData, - const bsl::shared_ptr& options, - bmqt::GenericResult::Enum rc) -{ - // executed by the *DISPATCHER* thread - // PRECONDITIONS - BSLS_ASSERT_SAFE(dispatcher()->inDispatcherThread(this)); - - mqbc::ClusterUtil::generateNack(status, - putHeader, - source, - dispatcher(), - appData, - options); - - BMQU_THROTTLEDACTION_THROTTLE( - d_throttledSkippedPutMessages, - BALL_LOG_ERROR << description() << ": skipping relay-PUT message [" - << "queueId: " << putHeader.queueId() << ", GUID: " - << putHeader.messageGUID() << "], rc: " << rc << "."); -} void ClusterProxy::startDispatched() { @@ -335,6 +311,8 @@ void ClusterProxy::onActiveNodeUp(mqbnet::ClusterNode* activeNode) return; // RETURN } + d_activeNode_p = activeNode; + // Indicate we have a new leader (in proxy, the active is equal to the // leader); this will trigger a reopen of the queues that will eventually // invoke the queueHelper's 'stateRestoredFn' (i.e., 'onQueuesReopened'). @@ -348,6 +326,8 @@ void ClusterProxy::onActiveNodeUp(mqbnet::ClusterNode* activeNode) // to e_ACTIVE. Hence, we do: e_UNDEFINED -> e_PASSIVE -> e_ACTIVE d_clusterData.electorInfo().setLeaderStatus( mqbc::ElectorInfoLeaderStatus::e_ACTIVE); + + d_gateActiveNode.open(); } void ClusterProxy::onActiveNodeDown(const mqbnet::ClusterNode* node) @@ -358,6 +338,9 @@ void ClusterProxy::onActiveNodeDown(const mqbnet::ClusterNode* node) BSLS_ASSERT_SAFE(dispatcher()->inDispatcherThread(this)); BSLS_ASSERT_SAFE(node); + d_gateActiveNode.close(); + d_activeNode_p = 0; + if (d_isStopping) { BALL_LOG_INFO << description() << ": Ignoring 'ActiveNodeDown' " << "notification because self is stopping."; @@ -493,95 +476,6 @@ void ClusterProxy::onAckEvent(const mqbi::DispatcherAckEvent& event) } } -void ClusterProxy::onRelayPutEvent(const mqbi::DispatcherPutEvent& event, - mqbi::DispatcherClient* source) -{ - // executed by the *DISPATCHER* thread - - // PRECONDITIONS - BSLS_ASSERT_SAFE(dispatcher()->inDispatcherThread(this)); - BSLS_ASSERT_SAFE(event.isRelay() == true); - - // This event is invoked as a result of RemoteQueue asking cluster proxy to - // relay PUT message to cluster on it's behalf. Note that we don't check - // if we have an active node connection, and simply add the PUT message to - // the builder. If there is no active connection right now, these PUT - // messages will be relayed whenever a connection with new active is - // established and state is restored (ie, queues are re-opened). - - const bmqp::PutHeader& ph = event.putHeader(); - bsls::Types::Uint64 genCount = event.genCount(); - bsls::Types::Uint64 term = d_clusterData.electorInfo().electorTerm(); - if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY(genCount != term)) { - BMQU_THROTTLEDACTION_THROTTLE( - d_throttledSkippedPutMessages, - BALL_LOG_WARN << description() << ": skipping relay-PUT message [" - << "queueId: " << ph.queueId() << ", GUID: " - << ph.messageGUID() << "], genCount: " << genCount - << " vs " << term << "."); - return; // RETURN - } - - mqbnet::ClusterNode* activeNode = d_activeNodeManager.activeNode(); - bmqt::GenericResult::Enum rc; - - if (activeNode) { - rc = activeNode->channel().writePut(event.putHeader(), - event.blob(), - event.state()); - } - else { - rc = bmqt::GenericResult::e_NOT_CONNECTED; - } - - if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( - rc != bmqt::GenericResult::e_SUCCESS)) { - BSLS_PERFORMANCEHINT_UNLIKELY_HINT; - - generateNack(bmqt::AckResult::e_NOT_READY, - ph, - source, - event.blob(), - event.options(), - rc); - } -} - -void ClusterProxy::onRelayConfirmEvent( - const mqbi::DispatcherConfirmEvent& event) -{ - // executed by the *DISPATCHER* thread - - // PRECONDITIONS - BSLS_ASSERT_SAFE(dispatcher()->inDispatcherThread(this)); - BSLS_ASSERT_SAFE(event.isRelay() == true); - - // This event is invoked as a result of RemoteQueue asking cluster proxy to - // relay CONFIRM message to cluster on it's behalf. - - mqbnet::ClusterNode* activeNode = d_activeNodeManager.activeNode(); - const bmqp::ConfirmMessage& confirmMsg = event.confirmMessage(); - bmqt::GenericResult::Enum rc; - - if (activeNode) { - rc = activeNode->channel().writeConfirm(confirmMsg.queueId(), - confirmMsg.subQueueId(), - confirmMsg.messageGUID()); - } - else { - rc = bmqt::GenericResult::e_NOT_CONNECTED; - } - - if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( - rc != bmqt::GenericResult::e_SUCCESS)) { - BSLS_PERFORMANCEHINT_UNLIKELY_HINT; - BALL_LOG_ERROR << "Failed to relay Confirm message [rc: " << rc - << ", GUID: " << confirmMsg.messageGUID() - << ", queue Id: " << confirmMsg.queueId() << "" - << ", subqueue Id: " << confirmMsg.subQueueId() << ""; - } -} - void ClusterProxy::onRelayRejectEvent(const mqbi::DispatcherRejectEvent& event) { // executed by the *DISPATCHER* thread @@ -599,14 +493,13 @@ void ClusterProxy::onRelayRejectEvent(const mqbi::DispatcherRejectEvent& event) // these REJECT messages will be relayed whenever a connection with new // active is established and state is restored (ie, queues are re-opened). - const bmqp::RejectMessage& rejectMsg = event.rejectMessage(); - mqbnet::ClusterNode* activeNode = d_activeNodeManager.activeNode(); + const bmqp::RejectMessage& rejectMsg = event.rejectMessage(); bmqt::GenericResult::Enum rc; - if (activeNode) { - rc = activeNode->channel().writeReject(rejectMsg.queueId(), - rejectMsg.subQueueId(), - rejectMsg.messageGUID()); + if (d_activeNode_p) { + rc = d_activeNode_p->channel().writeReject(rejectMsg.queueId(), + rejectMsg.subQueueId(), + rejectMsg.messageGUID()); } else { rc = bmqt::GenericResult::e_NOT_CONNECTED; @@ -756,9 +649,7 @@ ClusterProxy::sendRequest(const RequestManagerType::RequestSp& request, // PRECONDITIONS BSLS_ASSERT_SAFE(dispatcher()->inDispatcherThread(this)); - mqbnet::ClusterNode* activeNode = d_activeNodeManager.activeNode(); - - if (!activeNode) { + if (!d_activeNode_p) { // Not connected to any node, can't deliver the request. ball::Severity::Level severity = ball::Severity::e_ERROR; if (d_activeNodeLookupEventHandle) { @@ -774,7 +665,7 @@ ClusterProxy::sendRequest(const RequestManagerType::RequestSp& request, return bmqt::GenericResult::e_NOT_CONNECTED; // RETURN } - if (target && target != activeNode) { + if (target && target != d_activeNode_p) { if (!d_isStopping) { // Don't warn/alarm if self is stopping. BALL_LOG_WARN << description() << ": Unable to send request " @@ -782,20 +673,20 @@ ClusterProxy::sendRequest(const RequestManagerType::RequestSp& request, << ", because specified target " << target->nodeDescription() << " is different from " << "the upstream node as perceived by the proxy " - << activeNode->nodeDescription() << "."; + << d_activeNode_p->nodeDescription() << "."; } return bmqt::GenericResult::e_REFUSED; // RETURN } - request->setGroupId(activeNode->nodeId()); + request->setGroupId(d_activeNode_p->nodeId()); return d_clusterData.requestManager().sendRequest( request, bdlf::BindUtil::bind(&mqbnet::ClusterNode::write, - activeNode, + d_activeNode_p, bdlf::PlaceHolders::_1, bmqp::EventType::e_CONTROL), - activeNode->nodeDescription(), + d_activeNode_p->nodeDescription(), timeout); } @@ -995,12 +886,13 @@ ClusterProxy::ClusterProxy( mqbcfg::BrokerConfig::get().hostDataCenter()) , d_queueHelper(&d_clusterData, &d_state, 0, allocator) , d_nodeStatsMap(allocator) -, d_throttledFailedAckMessages(5000, 1) // 1 log per 5s interval -, d_throttledSkippedPutMessages(5000, 1) // 1 log per 5s interval +, d_throttledFailedAckMessages(5000, 1) // 1 log per 5s interval , d_clusterMonitor(&d_clusterData, &d_state, d_allocator_p) , d_activeNodeLookupEventHandle() , d_shutdownChain(d_allocator_p) , d_stopRequestsManager_p(stopRequestsManager) +, d_activeNode_p(0) +, d_gateActiveNode() { // PRECONDITIONS mqbnet::Cluster* netCluster_p = d_clusterData.membership().netCluster(); @@ -1223,9 +1115,9 @@ void ClusterProxy::loadClusterStatus(mqbcmd::ClusterResult* out) out->makeClusterProxyStatus(); clusterProxyStatus.description() = description(); - if (d_activeNodeManager.activeNode()) { + if (d_activeNode_p) { clusterProxyStatus.activeNodeDescription().makeValue( - d_activeNodeManager.activeNode()->nodeDescription()); + d_activeNode_p->nodeDescription()); } clusterProxyStatus.isHealthy() = d_clusterMonitor.isHealthy(); @@ -1246,6 +1138,83 @@ void ClusterProxy::purgeAndGCQueueOnDomain( result->makeError().message() = os.str(); } +mqbi::InlineResult::Enum +ClusterProxy::sendConfirmInline(BSLA_UNUSED int partitionId, + const bmqp::ConfirmMessage& message) +{ + // executed by the *ANY* thread + + // This event is invoked as a result of RemoteQueue asking cluster proxy to + // relay CONFIRM message to cluster on it's behalf. + + mqbc::GateKeeper::Status primaryStatus(d_gateActiveNode); + + if (!primaryStatus.isOpen()) { + return mqbi::InlineResult::e_INVALID_PRIMARY; // RETURN + } + + BSLS_ASSERT_SAFE(d_activeNode_p); + + bmqt::GenericResult::Enum rc = d_activeNode_p->channel().writeConfirm( + message.queueId(), + message.subQueueId(), + message.messageGUID()); + + if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( + rc != bmqt::GenericResult::e_SUCCESS)) { + BSLS_PERFORMANCEHINT_UNLIKELY_HINT; + return mqbi::InlineResult::e_CHANNEL_ERROR; // RETURN + } + + return mqbi::InlineResult::e_SUCCESS; +} + +mqbi::InlineResult::Enum ClusterProxy::sendPutInline( + BSLA_UNUSED int partitionId, + const bmqp::PutHeader& putHeader, + const bsl::shared_ptr& appData, + BSLA_UNUSED const bsl::shared_ptr& options, + const bsl::shared_ptr& state, + bsls::Types::Uint64 genCount) +{ + // This event is invoked as a result of RemoteQueue asking cluster proxy to + // relay PUT message to cluster on it's behalf. Note that we don't check + // if we have an active node connection, and simply add the PUT message to + // the builder. If there is no active connection right now, these PUT + // messages will be relayed whenever a connection with new active is + // established and state is restored (ie, queues are re-opened). + + // This event is invoked as a result of RemoteQueue asking cluster proxy to + // relay CONFIRM message to cluster on it's behalf. + + mqbc::GateKeeper::Status primaryStatus(d_gateActiveNode); + + if (!primaryStatus.isOpen()) { + return mqbi::InlineResult::e_INVALID_PRIMARY; // RETURN + } + + BSLS_ASSERT_SAFE(d_activeNode_p); + + // This assumes thread-safe access to + // d_clusterData.electorInfo().electorTerm() if the gate is open + + bsls::Types::Uint64 term = d_clusterData.electorInfo().electorTerm(); + if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY(genCount != term)) { + return mqbi::InlineResult::e_INVALID_PRIMARY; // RETURN + } + + bmqt::GenericResult::Enum rc = + d_activeNode_p->channel().writePut(putHeader, appData, state); + + if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( + rc != bmqt::GenericResult::e_SUCCESS)) { + BSLS_PERFORMANCEHINT_UNLIKELY_HINT; + return mqbi::InlineResult::e_CHANNEL_ERROR; // RETURN + } + + return mqbi::InlineResult::e_SUCCESS; +} + // MANIPULATORS // (virtual: mqbi::DispatcherClient) void ClusterProxy::onDispatcherEvent(const mqbi::DispatcherEvent& event) @@ -1260,24 +1229,11 @@ void ClusterProxy::onDispatcherEvent(const mqbi::DispatcherEvent& event) switch (event.type()) { case mqbi::DispatcherEventType::e_PUT: { - const mqbi::DispatcherPutEvent* realEvent = event.asPutEvent(); - if (realEvent->isRelay()) { - onRelayPutEvent(*realEvent, event.source()); - } - else { - BALL_LOG_ERROR << "#UNEXPECTED_EVENT " << description() - << "PUT event not yet implemented"; - } + BALL_LOG_ERROR << "#UNEXPECTED_EVENT " << description() + << "PUT event not yet implemented"; } break; case mqbi::DispatcherEventType::e_CONFIRM: { - const mqbi::DispatcherConfirmEvent* realEvent = event.asConfirmEvent(); - if (realEvent->isRelay()) { - onRelayConfirmEvent(*realEvent); - } - else { - BALL_LOG_ERROR << "#UNEXPECTED_EVENT " << description() - << "CONFIRM event not yet implemented"; - } + BALL_LOG_ERROR << "#UNEXPECTED_EVENT (CONFIRM)" << description(); } break; case mqbi::DispatcherEventType::e_REJECT: { const mqbi::DispatcherRejectEvent* realEvent = event.asRejectEvent(); diff --git a/src/groups/mqb/mqbblp/mqbblp_clusterproxy.h b/src/groups/mqb/mqbblp/mqbblp_clusterproxy.h index 9994eb2441..87b787557d 100644 --- a/src/groups/mqb/mqbblp/mqbblp_clusterproxy.h +++ b/src/groups/mqb/mqbblp/mqbblp_clusterproxy.h @@ -222,9 +222,6 @@ class ClusterProxy : public mqbc::ClusterStateObserver, /// Throttling parameters for failed ACK messages. bmqu::ThrottledActionParams d_throttledFailedAckMessages; - /// Throttling parameters for skipped PUT messages. - bmqu::ThrottledActionParams d_throttledSkippedPutMessages; - /// Cluster state monitor. ClusterStateMonitor d_clusterMonitor; @@ -241,6 +238,14 @@ class ClusterProxy : public mqbc::ClusterStateObserver, /// @note Should be part of `ClusterResources`. StopRequestManagerType* d_stopRequestsManager_p; + mqbnet::ClusterNode* d_activeNode_p; + // Protected by d_gateActiveNode - it is safe to use while + // 'mqbc::GateKeeper::Status(d_gateActiveNode).isOpen()' + // 'd_activeNodeManager.activeNode()' does not provide such guarantee. + // Also, updated after 'd_clusterData.electorInfo().setElectorInfo' + + mqbc::GateKeeper d_gateActiveNode; + private: // PRIVATE MANIPULATORS @@ -300,26 +305,10 @@ class ClusterProxy : public mqbc::ClusterStateObserver, // PRIVATE MANIPULATORS // (Event processing) - /// Generate a NACK with the specified `status` for a PUT message having - /// the specified `putHeader` from the specified `source`. The nack is - /// replied to the `source`. Log the specified `rc` as a reason for the - /// NACK. - void generateNack(bmqt::AckResult::Enum status, - const bmqp::PutHeader& putHeader, - DispatcherClient* source, - const bsl::shared_ptr& appData, - const bsl::shared_ptr& options, - bmqt::GenericResult::Enum rc); - void onPushEvent(const mqbi::DispatcherPushEvent& event); void onAckEvent(const mqbi::DispatcherAckEvent& event); - void onRelayPutEvent(const mqbi::DispatcherPutEvent& event, - mqbi::DispatcherClient* source); - - void onRelayConfirmEvent(const mqbi::DispatcherConfirmEvent& event); - void onRelayRejectEvent(const mqbi::DispatcherRejectEvent& event); // PRIVATE MANIPULATORS @@ -504,6 +493,25 @@ class ClusterProxy : public mqbc::ClusterStateObserver, /// Load the cluster state in the specified `out` object. void loadClusterStatus(mqbcmd::ClusterResult* out) BSLS_KEYWORD_OVERRIDE; + /// Send the specified CONFIRM 'message' for the specified 'partitionId' + /// without switching thread context. + /// 'onRelayConfirmEvent' replacement. + mqbi::InlineResult::Enum sendConfirmInline( + int partitionId, + const bmqp::ConfirmMessage& message) BSLS_KEYWORD_OVERRIDE; + + /// Send PUT message for the specified 'partitionId' using the specified + /// 'putHeader', 'appData', 'options', 'state', 'genCount' without + /// switching thread context. + /// 'onRelayPutEvent' replacement. + mqbi::InlineResult::Enum + sendPutInline(int partitionId, + const bmqp::PutHeader& putHeader, + const bsl::shared_ptr& appData, + const bsl::shared_ptr& options, + const bsl::shared_ptr& state, + bsls::Types::Uint64 genCount) BSLS_KEYWORD_OVERRIDE; + /// Purge and force GC queues in this cluster on a given domain. void purgeAndGCQueueOnDomain(mqbcmd::ClusterResult* result, const bsl::string& domainName) diff --git a/src/groups/mqb/mqbblp/mqbblp_clusterqueuehelper.cpp b/src/groups/mqb/mqbblp/mqbblp_clusterqueuehelper.cpp index 00a92b99ce..df7ba0ebaa 100644 --- a/src/groups/mqb/mqbblp/mqbblp_clusterqueuehelper.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_clusterqueuehelper.cpp @@ -206,7 +206,7 @@ void ClusterQueueHelper::QueueContext::respond( (*cIt)->d_callback(status, 0, bmqp_ctrlmsg::OpenQueueResponse(), - mqbi::Cluster::OpenQueueConfirmationCookie()); + mqbi::OpenQueueConfirmationCookieSp()); } } @@ -1099,7 +1099,7 @@ void ClusterQueueHelper::sendOpenQueueRequest( context->d_callback(failure, \ 0, \ bmqp_ctrlmsg::OpenQueueResponse(), \ - mqbi::Cluster::OpenQueueConfirmationCookie()); \ + mqbi::OpenQueueConfirmationCookieSp()); \ } while (0) if (bmqp::QueueUtil::isEmpty(context->d_handleParameters)) { @@ -1358,7 +1358,7 @@ void ClusterQueueHelper::onOpenQueueResponse( context->d_callback(requestContext->response().choice().status(), 0, bmqp_ctrlmsg::OpenQueueResponse(), - mqbi::Cluster::OpenQueueConfirmationCookie()); + mqbi::OpenQueueConfirmationCookieSp()); return; // RETURN } @@ -1969,13 +1969,13 @@ void ClusterQueueHelper::onReopenQueueRetryDispatched( } void ClusterQueueHelper::onOpenQueueConfirmationCookieReleased( - mqbi::QueueHandle** value, + mqbi::OpenQueueContext* value, const bmqp_ctrlmsg::QueueHandleParameters& handleParameters) { // TBD: NOT REVIEWED // executed by *ANY* thread - mqbi::QueueHandle* handle = *value; + mqbi::QueueHandle* handle = value->d_handle; d_allocator_p->deleteObject(value); if (!handle) { @@ -2069,8 +2069,8 @@ bool ClusterQueueHelper::createQueue( << ", context.d_handleParameters: " << context->d_handleParameters << "]"; - mqbi::Cluster::OpenQueueConfirmationCookie confirmationCookie( - new (*d_allocator_p) mqbi::QueueHandle * (0), + mqbi::OpenQueueConfirmationCookieSp confirmationCookie( + new (*d_allocator_p) mqbi::OpenQueueContext(), bdlf::BindUtil::bind( &ClusterQueueHelper::onOpenQueueConfirmationCookieReleased, this, @@ -2108,6 +2108,7 @@ bool ClusterQueueHelper::createQueue( bmqp::QueueUtil::extractSubQueueId(parameters); queue->getHandle( + confirmationCookie, context->d_clientContext, context->d_handleParameters, upstreamSubQueueId, @@ -2137,7 +2138,7 @@ bool ClusterQueueHelper::createQueue( 0, context, openQueueResponse, - mqbi::Cluster::OpenQueueConfirmationCookie()); + mqbi::OpenQueueConfirmationCookieSp()); if (isPrimary) { // No further cleanup required. @@ -2449,29 +2450,12 @@ void ClusterQueueHelper::onHandleConfigured( { // executed by *ANY* thread - d_cluster_p->dispatcher()->execute( - bdlf::BindUtil::bind(&ClusterQueueHelper::onHandleConfiguredDispatched, - this, - status, - streamParameters, - request, - requester), - d_cluster_p); -} - -void ClusterQueueHelper::onHandleConfiguredDispatched( - const bmqp_ctrlmsg::Status& status, - const bmqp_ctrlmsg::StreamParameters& streamParameters, - const bmqp_ctrlmsg::ControlMessage& request, - mqbc::ClusterNodeSession* requester) -{ - // executed by the *CLUSTER* dispatcher thread + // Make sure the response is out; otherwise inline PUSH (in queue thread) + // can get to the channel before the response. - // PRECONDITIONS - BSLS_ASSERT_SAFE( - d_cluster_p->dispatcher()->inDispatcherThread(d_cluster_p)); + // 'synchronize' (Queue with the Cluster) would result in a dead-lock if + // 'Queue::configure' synchronizes Cluster with Queue. - // Send the response (always success) bdlma::LocalSequentialAllocator<1024> localAllocator(d_allocator_p); bmqp_ctrlmsg::ControlMessage response(&localAllocator); @@ -2511,25 +2495,47 @@ void ClusterQueueHelper::onHandleConfiguredDispatched( configureStream.streamParameters() = streamParameters; } - // Need to rebuild Subscriptions - CNSQueueHandleMap::iterator it = requester->queueHandles().find(qId); - if (it == requester->queueHandles().end()) { - // Failure. + // Need to rebuild Subscriptions in the Cluster thread. + d_cluster_p->dispatcher()->execute( + bdlf::BindUtil::bind( + &ClusterQueueHelper::onHandleConfiguredDispatched, + this, + qId, + streamParameters, + requester), + d_cluster_p); + } + + d_clusterData_p->messageTransmitter().sendMessageSafe( + response, + requester->clusterNode()); +} + +void ClusterQueueHelper::onHandleConfiguredDispatched( + int qId, + const bmqp_ctrlmsg::StreamParameters& streamParameters, + mqbc::ClusterNodeSession* requester) +{ + // executed by the *CLUSTER* dispatcher thread + + // PRECONDITIONS + BSLS_ASSERT_SAFE( + d_cluster_p->dispatcher()->inDispatcherThread(d_cluster_p)); + + // Need to rebuild Subscriptions + CNSQueueHandleMap::iterator it = requester->queueHandles().find(qId); + if (it == requester->queueHandles().end()) { + // Failure. BMQ_LOGTHROTTLE_WARN << d_cluster_p->description() << ": Received configureStream response from [" << requester->description() << "] for a queue with unknown Id " << "(" << qId << ")."; - } - else { - it->second.d_subQueueInfosMap.addSubscriptions(streamParameters); - } } - - d_clusterData_p->messageTransmitter().sendMessage( - response, - requester->clusterNode()); + else { + it->second.d_subQueueInfosMap.addSubscriptions(streamParameters); + } } void ClusterQueueHelper::onGetDomain( @@ -2641,11 +2647,11 @@ void ClusterQueueHelper::onGetDomainDispatched( } void ClusterQueueHelper::onGetQueueHandle( - const bmqp_ctrlmsg::Status& status, - mqbi::QueueHandle* queueHandle, - const OpenQueueContextSp& context, - const bmqp_ctrlmsg::OpenQueueResponse& openQueueResponse, - const mqbi::Domain::OpenQueueConfirmationCookie& confirmationCookie) + const bmqp_ctrlmsg::Status& status, + mqbi::QueueHandle* queueHandle, + const OpenQueueContextSp& context, + const bmqp_ctrlmsg::OpenQueueResponse& openQueueResponse, + const mqbi::OpenQueueConfirmationCookieSp& confirmationCookie) { // executed by *ANY* thread @@ -2660,7 +2666,7 @@ void ClusterQueueHelper::onGetQueueHandle( // 'requester'. if (confirmationCookie) { - *confirmationCookie = queueHandle; + confirmationCookie->d_handle = queueHandle; } if (context->d_clientContext->isClusterMember()) { @@ -2681,13 +2687,13 @@ void ClusterQueueHelper::onGetQueueHandle( } void ClusterQueueHelper::onGetQueueHandleDispatched( - const bmqp_ctrlmsg::Status& status, - mqbi::QueueHandle* queueHandle, - const bmqp_ctrlmsg::OpenQueueResponse& openQueueResponse, - const mqbi::Domain::OpenQueueConfirmationCookie& confirmationCookie, - const bmqp_ctrlmsg::ControlMessage& request, - mqbc::ClusterNodeSession* requester, - const int peerInstanceId) + const bmqp_ctrlmsg::Status& status, + mqbi::QueueHandle* queueHandle, + const bmqp_ctrlmsg::OpenQueueResponse& openQueueResponse, + const mqbi::OpenQueueConfirmationCookieSp& confirmationCookie, + const bmqp_ctrlmsg::ControlMessage& request, + mqbc::ClusterNodeSession* requester, + const int peerInstanceId) { // executed by the cluster *DISPATCHER* thread @@ -2744,12 +2750,14 @@ void ClusterQueueHelper::onGetQueueHandleDispatched( BSLS_ASSERT_SAFE(queueHandle); BSLS_ASSERT_SAFE(confirmationCookie); // in case of success, the cookie // must be a valid shared_ptr + BSLS_ASSERT_SAFE(confirmationCookie->d_handle); // Update the cookie to point to a null queue handle, which indicates that // 'requester' has successfully received and processed the open-queue // response. - *confirmationCookie = 0; // Indicate proper response of the queueHandle + confirmationCookie->d_handle = 0; + // Indicate proper response of the queueHandle const bmqp_ctrlmsg::OpenQueue& openQueue = request.choice().openQueue(); const bmqp_ctrlmsg::QueueHandleParameters& handleParams = @@ -2774,10 +2782,7 @@ void ClusterQueueHelper::onGetQueueHandleDispatched( if (subQueueIter == iter->second.d_subQueueInfosMap.end()) { // New subStream for this queueHandle mqbc::ClusterNodeSession::SubQueueInfo subQueueInfo; - subQueueInfo.d_clientStats->initialize( - queueHandle->queue()->uri(), - requester->statContext().get(), - d_allocator_p); + subQueueInfo.d_clientStats_sp = confirmationCookie->d_stats_sp; iter->second.d_subQueueInfosMap.insert(handleParams, subQueueInfo); } @@ -2794,9 +2799,7 @@ void ClusterQueueHelper::onGetQueueHandleDispatched( queueContext.d_handle_p = queueHandle; CNSSubQueueInfo subQueueInfo; - subQueueInfo.d_clientStats->initialize(queueHandle->queue()->uri(), - requester->statContext().get(), - d_allocator_p); + subQueueInfo.d_clientStats_sp = confirmationCookie->d_stats_sp; queueContext.d_subQueueInfosMap.insert(handleParams, subQueueInfo); @@ -4703,10 +4706,8 @@ void ClusterQueueHelper::openQueue( failure.category() = bmqp_ctrlmsg::StatusCategory::E_REFUSED; \ failure.code() = CODE; \ failure.message() = REASON; \ - callback(failure, \ - 0, \ - bmqp_ctrlmsg::OpenQueueResponse(), \ - mqbi::Cluster::OpenQueueConfirmationCookie()); \ + mqbi::OpenQueueConfirmationCookieSp temp; \ + callback(failure, 0, bmqp_ctrlmsg::OpenQueueResponse(), temp); \ } while (0) if (d_cluster_p->isStopping()) { diff --git a/src/groups/mqb/mqbblp/mqbblp_clusterqueuehelper.h b/src/groups/mqb/mqbblp/mqbblp_clusterqueuehelper.h index 35fdb25013..1a0d2587e0 100644 --- a/src/groups/mqb/mqbblp/mqbblp_clusterqueuehelper.h +++ b/src/groups/mqb/mqbblp/mqbblp_clusterqueuehelper.h @@ -583,7 +583,7 @@ class ClusterQueueHelper BSLS_KEYWORD_FINAL /// Custom deleter of the openQueue confirmationCookie (in the specified /// `value`), for an open queue from the specified `request`. void onOpenQueueConfirmationCookieReleased( - mqbi::QueueHandle** value, + mqbi::OpenQueueContext* value, const bmqp_ctrlmsg::QueueHandleParameters& handleParameters); /// Final part of the open queue pipeline for the specified `context`: @@ -626,16 +626,15 @@ class ClusterQueueHelper BSLS_KEYWORD_FINAL const bmqp_ctrlmsg::ControlMessage& request, mqbc::ClusterNodeSession* requester); void onHandleConfiguredDispatched( - const bmqp_ctrlmsg::Status& status, + int qId, const bmqp_ctrlmsg::StreamParameters& streamParameters, - const bmqp_ctrlmsg::ControlMessage& request, mqbc::ClusterNodeSession* requester); void onGetDomain(const bmqp_ctrlmsg::Status& status, mqbi::Domain* domain, const bmqp_ctrlmsg::ControlMessage& request, mqbc::ClusterNodeSession* requester, - const int peerInstanceId); + int peerInstanceId); /// Callback invoked in response to an open domain query (in the /// specified `request`) made to the domain factory on behalf of the @@ -651,11 +650,11 @@ class ClusterQueueHelper BSLS_KEYWORD_FINAL const int peerInstanceId); void onGetQueueHandle( - const bmqp_ctrlmsg::Status& status, - mqbi::QueueHandle* queueHandle, - const OpenQueueContextSp& context, - const bmqp_ctrlmsg::OpenQueueResponse& openQueueResponse, - const mqbi::Domain::OpenQueueConfirmationCookie& confirmationCookie); + const bmqp_ctrlmsg::Status& status, + mqbi::QueueHandle* queueHandle, + const OpenQueueContextSp& context, + const bmqp_ctrlmsg::OpenQueueResponse& openQueueResponse, + const mqbi::OpenQueueConfirmationCookieSp& confirmationCookie); /// Callback invoked in response to an open queue request to the domain /// (in the specified `request`). If the specified `status` is SUCCESS, @@ -667,13 +666,13 @@ class ClusterQueueHelper BSLS_KEYWORD_FINAL /// `status` contains the category, error code and description of the /// failure. The `queueHandle` must be released once no longer needed. void onGetQueueHandleDispatched( - const bmqp_ctrlmsg::Status& status, - mqbi::QueueHandle* queueHandle, - const bmqp_ctrlmsg::OpenQueueResponse& openQueueResponse, - const mqbi::Domain::OpenQueueConfirmationCookie& confirmationCookie, - const bmqp_ctrlmsg::ControlMessage& request, - mqbc::ClusterNodeSession* requester, - const int peerInstanceId); + const bmqp_ctrlmsg::Status& status, + mqbi::QueueHandle* queueHandle, + const bmqp_ctrlmsg::OpenQueueResponse& openQueueResponse, + const mqbi::OpenQueueConfirmationCookieSp& confirmationCookie, + const bmqp_ctrlmsg::ControlMessage& request, + mqbc::ClusterNodeSession* requester, + const int peerInstanceId); void reconfigureCallback( const bmqp_ctrlmsg::Status& status, diff --git a/src/groups/mqb/mqbblp/mqbblp_clusterstatemanager.cpp b/src/groups/mqb/mqbblp/mqbblp_clusterstatemanager.cpp index dac7a8d14a..3dcb2c2ca4 100644 --- a/src/groups/mqb/mqbblp/mqbblp_clusterstatemanager.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_clusterstatemanager.cpp @@ -497,6 +497,8 @@ void ClusterStateManager::onLeaderSyncDataQueryResponse( proposedPrimaryNode); BSLS_ASSERT_SAFE(proposedPrimaryNs); + mqbc::ClusterNodeSession* effectivePrimaryNs = proposedPrimaryNs; + if (bmqp_ctrlmsg::NodeStatus::E_AVAILABLE != proposedPrimaryNs->nodeStatus()) { // Self node does not perceive proposed primary node as AVAILABLE, @@ -507,6 +509,7 @@ void ClusterStateManager::onLeaderSyncDataQueryResponse( // continuing with next iteration. effectivePrimaryNode = 0; + effectivePrimaryNs = 0; BALL_LOG_WARN << d_clusterData_p->identity().description() << ": specified primary node " @@ -551,7 +554,7 @@ void ClusterStateManager::onLeaderSyncDataQueryResponse( // TODO CSL Please review this code path during node startup sequence. d_state_p->setPartitionPrimary(peerPinfo.partitionId(), effectiveLeaseId, - effectivePrimaryNode); // Could be null + effectivePrimaryNs); // Could be null // Update primary node-specific list of assigned partitions only if it // needs to be. @@ -783,7 +786,11 @@ void ClusterStateManager::setPrimary(int partitionId, d_state_p->partitions().size()); BSLS_ASSERT_SAFE(primary); - d_state_p->setPartitionPrimary(partitionId, leaseId, primary); + mqbc::ClusterNodeSession* ns = + d_clusterData_p->membership().getClusterNodeSession(primary); + BSLS_ASSERT_SAFE(ns); + + d_state_p->setPartitionPrimary(partitionId, leaseId, ns); } void ClusterStateManager::setPrimaryStatus( diff --git a/src/groups/mqb/mqbblp/mqbblp_clusterstatemonitor.t.cpp b/src/groups/mqb/mqbblp/mqbblp_clusterstatemonitor.t.cpp index 9db1ad75dd..3316d937db 100644 --- a/src/groups/mqb/mqbblp/mqbblp_clusterstatemonitor.t.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_clusterstatemonitor.t.cpp @@ -240,8 +240,14 @@ struct TestHelper { bmqp_ctrlmsg::PrimaryStatus::Value status = isActive ? bmqp_ctrlmsg::PrimaryStatus::E_ACTIVE : bmqp_ctrlmsg::PrimaryStatus::E_PASSIVE; + + mqbc::ClusterNodeSession* ns = + d_cluster_mp->_clusterData()->membership().getClusterNodeSession( + node); + BSLS_ASSERT_OPT(ns); + d_cluster_mp->_state() - ->setPartitionPrimary(partition, 1, node) + ->setPartitionPrimary(partition, 1, ns) .setPartitionPrimaryStatus(partition, status); } @@ -255,7 +261,7 @@ struct TestHelper { ->membership() .clusterNodeSessionMap() .find(node) - ->second->setNodeStatus(status); + ->second->setNodeStatus(status, status); } }; diff --git a/src/groups/mqb/mqbblp/mqbblp_domain.cpp b/src/groups/mqb/mqbblp/mqbblp_domain.cpp index 161d29377e..44ffb14e09 100644 --- a/src/groups/mqb/mqbblp/mqbblp_domain.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_domain.cpp @@ -229,11 +229,11 @@ int normalizeConfig(mqbconfm::Domain* defn, // ------------ void Domain::onOpenQueueResponse( - const bmqp_ctrlmsg::Status& status, - mqbi::QueueHandle* queuehandle, - const bmqp_ctrlmsg::OpenQueueResponse& openQueueResponse, - const mqbi::Cluster::OpenQueueConfirmationCookie& confirmationCookie, - const mqbi::Domain::OpenQueueCallback& callback) + const bmqp_ctrlmsg::Status& status, + mqbi::QueueHandle* queuehandle, + const bmqp_ctrlmsg::OpenQueueResponse& openQueueResponse, + const mqbi::OpenQueueConfirmationCookieSp& confirmationCookie, + const mqbi::Domain::OpenQueueCallback& callback) { // executed by *ANY* thread @@ -486,10 +486,11 @@ void Domain::openQueue( status.message() = k_NODE_IS_STOPPING; } + mqbi::OpenQueueConfirmationCookieSp temp; callback(status, static_cast(0), bmqp_ctrlmsg::OpenQueueResponse(), - mqbi::Cluster::OpenQueueConfirmationCookie()); + temp); return; // RETURN } diff --git a/src/groups/mqb/mqbblp/mqbblp_domain.h b/src/groups/mqb/mqbblp/mqbblp_domain.h index 2a120c18d7..990b9806b3 100644 --- a/src/groups/mqb/mqbblp/mqbblp_domain.h +++ b/src/groups/mqb/mqbblp/mqbblp_domain.h @@ -186,11 +186,11 @@ class Domain BSLS_KEYWORD_FINAL : public mqbi::Domain { /// Invoke the specified `callback` with the specified /// `confirmationCookie` to propagate the result to the requester. void onOpenQueueResponse( - const bmqp_ctrlmsg::Status& status, - mqbi::QueueHandle* queuehandle, - const bmqp_ctrlmsg::OpenQueueResponse& openQueueResponse, - const mqbi::Cluster::OpenQueueConfirmationCookie& confirmationCookie, - const mqbi::Domain::OpenQueueCallback& callback); + const bmqp_ctrlmsg::Status& status, + mqbi::QueueHandle* queuehandle, + const bmqp_ctrlmsg::OpenQueueResponse& openQueueResponse, + const mqbi::OpenQueueConfirmationCookieSp& confirmationCookie, + const mqbi::Domain::OpenQueueCallback& callback); // PRIVATE MANIPULATORS // (virtual: mqbc::ClusterStateObserver) diff --git a/src/groups/mqb/mqbblp/mqbblp_localqueue.cpp b/src/groups/mqb/mqbblp/mqbblp_localqueue.cpp index c5d943da36..328ff5eaa5 100644 --- a/src/groups/mqb/mqbblp/mqbblp_localqueue.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_localqueue.cpp @@ -251,6 +251,7 @@ void LocalQueue::close() } void LocalQueue::getHandle( + const mqbi::OpenQueueConfirmationCookieSp& context, const bsl::shared_ptr& clientContext, const bmqp_ctrlmsg::QueueHandleParameters& handleParameters, unsigned int upstreamSubQueueId, @@ -262,7 +263,8 @@ void LocalQueue::getHandle( BSLS_ASSERT_SAFE(d_state_p->queue()->dispatcher()->inDispatcherThread( d_state_p->queue())); - d_queueEngine_mp->getHandle(clientContext, + d_queueEngine_mp->getHandle(context, + clientContext, handleParameters, upstreamSubQueueId, callback); diff --git a/src/groups/mqb/mqbblp/mqbblp_localqueue.h b/src/groups/mqb/mqbblp/mqbblp_localqueue.h index c81ea77348..b9d8ef3350 100644 --- a/src/groups/mqb/mqbblp/mqbblp_localqueue.h +++ b/src/groups/mqb/mqbblp/mqbblp_localqueue.h @@ -125,11 +125,13 @@ class LocalQueue BSLS_CPP11_FINAL { /// result. /// /// THREAD: This method is called from the Queue's dispatcher thread. - void getHandle(const bsl::shared_ptr& - clientContext, - const bmqp_ctrlmsg::QueueHandleParameters& handleParameters, - unsigned int upstreamSubQueueId, - const mqbi::QueueHandle::GetHandleCallback& callback); + void + getHandle(const mqbi::OpenQueueConfirmationCookieSp& confirmationCookie, + const bsl::shared_ptr& + clientContext, + const bmqp_ctrlmsg::QueueHandleParameters& handleParameters, + unsigned int upstreamSubQueueId, + const mqbi::QueueHandle::GetHandleCallback& callback); /// Configure the specified `handle` with the specified /// `streamParameters`, and invoke the specified `configuredCb` callback diff --git a/src/groups/mqb/mqbblp/mqbblp_queue.cpp b/src/groups/mqb/mqbblp/mqbblp_queue.cpp index bd8f8ec7a7..4cb74a1b93 100644 --- a/src/groups/mqb/mqbblp/mqbblp_queue.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_queue.cpp @@ -123,6 +123,7 @@ void Queue::configureDispatched(bool isReconfigure) } void Queue::getHandleDispatched( + const mqbi::OpenQueueConfirmationCookieSp& context, const bsl::shared_ptr& clientContext, const bmqp_ctrlmsg::QueueHandleParameters& handleParameters, unsigned int upstreamSubQueueId, @@ -137,13 +138,15 @@ void Queue::getHandleDispatched( mqbi::QueueHandleRequesterContext::k_INVALID_REQUESTER_ID); if (d_localQueue_mp) { - d_localQueue_mp->getHandle(clientContext, + d_localQueue_mp->getHandle(context, + clientContext, handleParameters, upstreamSubQueueId, callback); } else if (d_remoteQueue_mp) { - d_remoteQueue_mp->getHandle(clientContext, + d_remoteQueue_mp->getHandle(context, + clientContext, handleParameters, upstreamSubQueueId, callback); @@ -659,6 +662,7 @@ int Queue::configure(bsl::ostream* errorDescription_p, } void Queue::getHandle( + const mqbi::OpenQueueConfirmationCookieSp& context, const bsl::shared_ptr& clientContext, const bmqp_ctrlmsg::QueueHandleParameters& handleParameters, unsigned int upstreamSubQueueId, @@ -672,6 +676,7 @@ void Queue::getHandle( dispatcher()->execute(bdlf::BindUtil::bind(&Queue::getHandleDispatched, this, + context, clientContext, handleParameters, upstreamSubQueueId, diff --git a/src/groups/mqb/mqbblp/mqbblp_queue.h b/src/groups/mqb/mqbblp/mqbblp_queue.h index b3b7a33b91..916141d2db 100644 --- a/src/groups/mqb/mqbblp/mqbblp_queue.h +++ b/src/groups/mqb/mqbblp/mqbblp_queue.h @@ -129,6 +129,7 @@ class Queue BSLS_CPP11_FINAL : public mqbi::Queue { void configureDispatched(bool isReconfigure); void getHandleDispatched( + const mqbi::OpenQueueConfirmationCookieSp& confirmationCookie, const bsl::shared_ptr& clientContext, const bmqp_ctrlmsg::QueueHandleParameters& handleParameters, @@ -213,11 +214,13 @@ class Queue BSLS_CPP11_FINAL : public mqbi::Queue { /// specified `clientContext` and using the specified `handleParameters` /// and `upstreamSubQueueId`. Invoke the specified `callback` with the /// result. - void getHandle(const bsl::shared_ptr& - clientContext, - const bmqp_ctrlmsg::QueueHandleParameters& handleParameters, - unsigned int upstreamSubQueueId, - const mqbi::QueueHandle::GetHandleCallback& callback) + void + getHandle(const mqbi::OpenQueueConfirmationCookieSp& confirmationCookie, + const bsl::shared_ptr& + clientContext, + const bmqp_ctrlmsg::QueueHandleParameters& handleParameters, + unsigned int upstreamSubQueueId, + const mqbi::QueueHandle::GetHandleCallback& callback) BSLS_KEYWORD_OVERRIDE; /// Configure the specified `handle` with the specified diff --git a/src/groups/mqb/mqbblp/mqbblp_queueenginetester.cpp b/src/groups/mqb/mqbblp/mqbblp_queueenginetester.cpp index f9ee137f08..3800eb9055 100644 --- a/src/groups/mqb/mqbblp/mqbblp_queueenginetester.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_queueenginetester.cpp @@ -777,7 +777,11 @@ QueueEngineTester::getHandle(const bsl::string& clientText) bdlf::PlaceHolders::_1, // status bdlf::PlaceHolders::_2); // handle - d_mockQueue_sp->getHandle(clientContext, + mqbi::OpenQueueConfirmationCookieSp confirmationCookie; + confirmationCookie.createInplace(d_allocator_p); + + d_mockQueue_sp->getHandle(confirmationCookie, + clientContext, handleParams, upstreamSubQueueId, callback); @@ -788,15 +792,21 @@ QueueEngineTester::getHandle(const bsl::string& clientText) d_clientContexts.end()); // Client data + ClientContextSp clientContext( new (*d_allocator_p) mqbi::QueueHandleRequesterContext(d_allocator_p), d_allocator_p); + clientContext->setClient(d_mockDispatcherClient_mp.get()) .setDescription("test.tsk:1") .setIsClusterMember(true) .setRequesterId(mqbi::QueueHandleRequesterContext :: - generateUniqueRequesterId()); + generateUniqueRequesterId()) + .setStatContext( + mqbstat::QueueStatsUtil::initializeStatContextClients( + 10, + d_allocator_p)); mqbi::QueueHandle::GetHandleCallback callback = bdlf::BindUtil::bindS( d_allocator_p, @@ -806,7 +816,11 @@ QueueEngineTester::getHandle(const bsl::string& clientText) bdlf::PlaceHolders::_1, bdlf::PlaceHolders::_2); - d_mockQueue_sp->getHandle(clientContext, + mqbi::OpenQueueConfirmationCookieSp confirmationCookie; + confirmationCookie.createInplace(d_allocator_p); + + d_mockQueue_sp->getHandle(confirmationCookie, + clientContext, handleParams, upstreamSubQueueId, callback); diff --git a/src/groups/mqb/mqbblp/mqbblp_queueengineutil.cpp b/src/groups/mqb/mqbblp/mqbblp_queueengineutil.cpp index 86cf331650..1d7c77d66e 100644 --- a/src/groups/mqb/mqbblp/mqbblp_queueengineutil.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_queueengineutil.cpp @@ -210,7 +210,7 @@ bool QueueEngineUtil::consumerAndProducerLimitsAreValid( int QueueEngineUtil::validateUri( const bmqp_ctrlmsg::QueueHandleParameters& handleParameters, mqbi::QueueHandle* handle, - const mqbi::QueueHandleRequesterContext& clientContext) + const mqbi::QueueHandleRequesterContext* clientContext) { bmqt::Uri uri; bsl::string error; @@ -224,11 +224,10 @@ int QueueEngineUtil::validateUri( << "#CLIENT_IMPROPER_BEHAVIOR " << "Mismatched queue URIs for same queueId for a " << "client. Rejecting request."; - if (clientContext.requesterId() != - mqbi::QueueHandleRequesterContext::k_INVALID_REQUESTER_ID) { + if (clientContext) { BALL_LOG_OUTPUT_STREAM - << " ClientPtr '" << clientContext.client() - << "', requesterId '" << clientContext.requesterId() + << " ClientPtr '" << clientContext->client() + << "', requesterId '" << clientContext->requesterId() << "',"; } BALL_LOG_OUTPUT_STREAM diff --git a/src/groups/mqb/mqbblp/mqbblp_queueengineutil.h b/src/groups/mqb/mqbblp/mqbblp_queueengineutil.h index 7a50fbfea2..2e46cd33e6 100644 --- a/src/groups/mqb/mqbblp/mqbblp_queueengineutil.h +++ b/src/groups/mqb/mqbblp/mqbblp_queueengineutil.h @@ -111,8 +111,7 @@ struct QueueEngineUtil { static int validateUri(const bmqp_ctrlmsg::QueueHandleParameters& handleParameters, mqbi::QueueHandle* handle, - const mqbi::QueueHandleRequesterContext& clientContext = - mqbi::QueueHandleRequesterContext()); + const mqbi::QueueHandleRequesterContext* clientContext = 0); /// Return true is the specified `queue` is of the broadcast type. static bool isBroadcastMode(const mqbi::Queue* queue); diff --git a/src/groups/mqb/mqbblp/mqbblp_queuehandle.cpp b/src/groups/mqb/mqbblp/mqbblp_queuehandle.cpp index 4177d3276f..e7212ef2c8 100644 --- a/src/groups/mqb/mqbblp/mqbblp_queuehandle.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_queuehandle.cpp @@ -114,13 +114,13 @@ void onHandleDeconfigured(const bmqp_ctrlmsg::Status&, // CREATORS QueueHandle::Subscription::Subscription( - unsigned int subId, + unsigned int downstreamSubQueueId, const bsl::shared_ptr& downstream, unsigned int upstreamId) : d_unconfirmedMonitor(0, 0, 0, 0, 0, 0) // Set later , d_downstream(downstream) -, d_downstreamSubQueueId(subId) +, d_downstreamSubQueueId(downstreamSubQueueId) , d_upstreamId(upstreamId) { // NOTHING @@ -131,6 +131,11 @@ const bsl::string& QueueHandle::Subscription::appId() const return d_downstream->d_appId; } +const QueueHandle::StatsSp& QueueHandle::Subscription::stats() const +{ + return d_downstream->d_stats_sp; +} + // ---------------------------------- // struct QueueHandle::ConfirmFunctor // ---------------------------------- @@ -202,6 +207,11 @@ void QueueHandle::confirmMessageDispatched(const bmqt::MessageGUID& msgGUID, downstreamSubQueueId); unsigned int upstreamSubQueueId = subStream->d_upstreamSubQueueId; + // Update client stats + subStream->d_stats_sp->onEvent( + mqbstat::QueueStatsClient::EventType::e_CONFIRM, + 1); + // If we previously hit the maxUnconfirmed and are now back to below the // lowWatermark for BOTH messages and bytes, then we will schedule a // delivery by indicating to the associated queue engine that this handle @@ -531,7 +541,7 @@ void QueueHandle::deliverMessageImpl( const bmqt::MessageGUID& msgGUID, const mqbi::StorageMessageAttributes& attributes, const bmqp::Protocol::MsgGroupId& msgGroupId, - const bmqp::Protocol::SubQueueInfosArray& subQueueInfos, + const bmqp::Protocol::SubQueueInfosArray& subscriptions, bool isOutOfOrder) { // executed by the *QUEUE_DISPATCHER* thread @@ -541,33 +551,67 @@ void QueueHandle::deliverMessageImpl( d_queue_sp->dispatcher()->inDispatcherThread(d_queue_sp.get())); BSLS_ASSERT_SAFE( bmqt::QueueFlagsUtil::isReader(handleParameters().flags())); - BSLS_ASSERT_SAFE(subQueueInfos.size() >= 1 && - subQueueInfos.size() <= d_subscriptions.size()); + BSLS_ASSERT_SAFE(subscriptions.size() >= 1 && + subscriptions.size() <= d_subscriptions.size()); d_domainStats_p->onEvent( attributes.appDataLen()); - // Create an event to dispatch delivery of the message to the client - mqbi::DispatcherClient* client = d_clientContext_sp->client(); - mqbi::DispatcherEvent* event = client->dispatcher()->getEvent(client); - (*event) - .setType(mqbi::DispatcherEventType::e_PUSH) - .setSource(d_queue_sp.get()) - .setGuid(msgGUID) - .setQueueId(id()) - .setMessagePropertiesInfo(d_queue_sp->schemaLearner().demultiplex( - d_schemaLearnerPushContext, - attributes.messagePropertiesInfo())) - .setSubQueueInfos(subQueueInfos) - .setMsgGroupId(msgGroupId) - .setCompressionAlgorithmType(attributes.compressionAlgorithmType()) - .setOutOfOrderPush(isOutOfOrder); - - if (message) { - event->setBlob(message); + if (d_clientContext_sp->inlineClient() == 0) { + // Create an event to dispatch delivery of the message to the client + mqbi::DispatcherClient* client = d_clientContext_sp->client(); + mqbi::DispatcherEvent* event = client->dispatcher()->getEvent(client); + (*event) + .setType(mqbi::DispatcherEventType::e_PUSH) + .setSource(d_queue_sp.get()) + .setGuid(msgGUID) + .setQueueId(id()) + .setMessagePropertiesInfo(d_queue_sp->schemaLearner().demultiplex( + d_schemaLearnerPushContext, + attributes.messagePropertiesInfo())) + .setSubQueueInfos(subscriptions) + .setMsgGroupId(msgGroupId) + .setCompressionAlgorithmType(attributes.compressionAlgorithmType()) + .setOutOfOrderPush(isOutOfOrder); + + if (message) { + event->setBlob(message); + } + + client->dispatcher()->dispatchEvent(event, client); + return; // RETURN } - client->dispatcher()->dispatchEvent(event, client); + mqbi::InlineResult::Enum result = + d_clientContext_sp->inlineClient()->sendPush( + msgGUID, + id(), + message, + attributes, + d_queue_sp->schemaLearner().demultiplex( + d_schemaLearnerPushContext, + attributes.messagePropertiesInfo()), + subscriptions); + + if (result == mqbi::InlineResult::e_SUCCESS) { + for (bmqp::Protocol::SubQueueInfosArray::size_type i = 0; + i < subscriptions.size(); + ++i) { + const bsl::shared_ptr& subscription = + d_subscriptions[subscriptions[i].id()]; + + if (subscription->stats()) { + subscription->stats()->onEvent( + mqbstat::QueueStatsClient::EventType::e_PUSH, + message ? message->length() : 0); + } + } + } + else if (d_throttledDroppedPutMessages.requestPermission()) { + BALL_LOG_WARN << "Queue '" << d_queue_sp->description() + << "' failed to PUSH " << msgGUID + << " with error: " << result; + } } QueueHandle::QueueHandle( @@ -589,6 +633,7 @@ QueueHandle::QueueHandle( d_queue_sp ? d_queue_sp->schemaLearner().createContext() : 0) , d_schemaLearnerPushContext( d_queue_sp ? d_queue_sp->schemaLearner().createContext() : 0) +, d_producerStats() , d_allocator_p(allocator) { // PRECONDITIONS @@ -624,9 +669,10 @@ QueueHandle::~QueueHandle() // QueueHandle. } -void QueueHandle::registerSubStream(const bmqp_ctrlmsg::SubQueueIdInfo& stream, - unsigned int upstreamSubQueueId, - const mqbi::QueueCounts& counts) +mqbi::QueueHandle::SubStreams::const_iterator +QueueHandle::registerSubStream(const bmqp_ctrlmsg::SubQueueIdInfo& stream, + unsigned int upstreamSubQueueId, + const mqbi::QueueCounts& counts) { // executed by the *QUEUE_DISPATCHER* thread @@ -647,21 +693,44 @@ void QueueHandle::registerSubStream(const bmqp_ctrlmsg::SubQueueIdInfo& stream, // Update the 'upstreamSubQueueId'. The previously registered stream // could be a producer and the new one - consumer with a new id. downstream(stream.subId())->d_upstreamSubQueueId = upstreamSubQueueId; - return; // RETURN } - // Allocate spot + else { + BSLS_ASSERT_SAFE(!validateDownstreamId(stream.subId())); + + StatsSp stats; - BSLS_ASSERT_SAFE(!validateDownstreamId(stream.subId())); + if (d_clientContext_sp->statContext()) { + stats.createInplace(d_allocator_p); - makeSubStream(stream.appId(), stream.subId(), upstreamSubQueueId); + stats->initialize(queue()->uri(), + d_clientContext_sp->statContext().get(), + d_allocator_p); - BALL_LOG_INFO << "QueueHandle [" << this << "] registering subQueue [" - << stream << "] with upstreamSubQueueId [" - << upstreamSubQueueId << "]"; + if (upstreamSubQueueId == bmqp::QueueId::k_DEFAULT_SUBQUEUE_ID) { + // cache producer stats to avoid lookup + d_producerStats = stats; + } + } + makeSubStream(stream.appId(), + stream.subId(), + upstreamSubQueueId, + stats); + + BALL_LOG_INFO << "QueueHandle [" << this << "] registering subQueue [" + << stream << "] with upstreamSubQueueId [" + << upstreamSubQueueId << "]"; + + infoIter = d_subStreamInfos + .emplace(stream.appId(), + StreamInfo(counts, + stream.subId(), + upstreamSubQueueId, + d_allocator_p)) + .first; + infoIter->second.d_clientStats_sp = stats; + } - d_subStreamInfos.emplace( - stream.appId(), - StreamInfo(counts, stream.subId(), upstreamSubQueueId, d_allocator_p)); + return infoIter; } void QueueHandle::registerSubscription(unsigned int downstreamSubId, @@ -786,6 +855,11 @@ bool QueueHandle::unregisterSubStream( } d_downstreams[downstreamSubQueueId].reset(); d_subStreamInfos.erase(infoIter); + + if (subStreamInfo.subId() == bmqp::QueueId::k_DEFAULT_SUBQUEUE_ID) { + d_producerStats.reset(); + } + return true; // RETURN } return false; @@ -1198,6 +1272,8 @@ void QueueHandle::onAckMessage(const bmqp::AckMessage& ackMessage) // PRECONDITIONS BSLS_ASSERT_SAFE( d_queue_sp->dispatcher()->inDispatcherThread(d_queue_sp.get())); + bmqt::AckResult::Enum status = bmqp::ProtocolUtil::ackResultFromCode( + ackMessage.status()); // NOTE: ACK comes from upstream, and client may have gone away, so we // check for it here. @@ -1224,22 +1300,75 @@ void QueueHandle::onAckMessage(const bmqp::AckMessage& ackMessage) // was requested or queue failed to post the message or both. Also note // that we need to reset the queueId in ack message to the one which is // known downstream. - mqbi::DispatcherClient* client = d_clientContext_sp->client(); - mqbi::DispatcherEvent* event = client->dispatcher()->getEvent(client); - (*event) - .setType(mqbi::DispatcherEventType::e_ACK) - .setSource(d_queue_sp.get()) - .setAckMessage(ackMessage); + d_domainStats_p->onEvent(1); - // Override with correct downstream queueId - const mqbi::DispatcherAckEvent* ackEvent = event->asAckEvent(); + mqbi::InlineClient* inlineClient = d_clientContext_sp->inlineClient(); - bmqp::AckMessage& ackMsg = const_cast( - ackEvent->ackMessage()); - ackMsg.setQueueId(id()); + if (inlineClient == 0) { + mqbi::DispatcherClient* client = d_clientContext_sp->client(); + mqbi::DispatcherEvent* event = client->dispatcher()->getEvent(client); + (*event) + .setType(mqbi::DispatcherEventType::e_ACK) + .setSource(d_queue_sp.get()) + .setAckMessage(ackMessage); - client->dispatcher()->dispatchEvent(event, client); - d_domainStats_p->onEvent(1); + // Override with correct downstream queueId + const mqbi::DispatcherAckEvent* ackEvent = event->asAckEvent(); + + bmqp::AckMessage& ackMsg = const_cast( + ackEvent->ackMessage()); + ackMsg.setQueueId(id()); + + client->dispatcher()->dispatchEvent(event, client); + return; // RETURN + } + + if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY(status != + bmqt::AckResult::e_SUCCESS)) { + BSLS_PERFORMANCEHINT_UNLIKELY_HINT; + + // Throttle error log if this is a 'failed Ack': note that we log at + // INFO level in order not to overwhelm the dashboard, if a queue is + // full, every post will nack, which could be a lot. + if (d_throttledFailedAckMessages.requestPermission()) { + BALL_LOG_INFO << "Queue '" << d_queue_sp->description() + << "' NACK [status: " << status + << ", GUID: " << ackMessage.messageGUID() + << ", node '" << d_clientContext_sp->description() + << "']"; + } + } + + // Always print at trace level + BALL_LOG_TRACE << "Queue '" << d_queue_sp->description() + << "' sending ACK [status: " << status + << ", GUID: " << ackMessage.messageGUID() << ", node '" + << d_clientContext_sp->description() << "']"; + + mqbi::InlineResult::Enum result = inlineClient->sendAck(id(), ackMessage); + // Override with correct downstream queueId + + if (result == mqbi::InlineResult::e_SUCCESS) { + // Update stats for the queue (or subStream of the queue) + // TBD: We should collect all invalid stats (i.e. stats for queues that + // were not found). We could collect these under a new 'invalid queue' + // stat context. + if (d_producerStats) { + d_producerStats->onEvent( + mqbstat::QueueStatsClient::EventType::e_ACK, + 1); + } + // In the case of Strong Consistency, a Receipt can arrive and trigger + // an ACK after Producer closes subStream. + } + else { + // Drop ACK + if (d_throttledFailedAckMessages.requestPermission()) { + BALL_LOG_INFO << "Queue '" << d_queue_sp->description() + << "' dropping ACK " << ackMessage.messageGUID() + << " with error: " << result; + } + } } bool QueueHandle::canDeliver(unsigned int downstreamSubscriptionId) const diff --git a/src/groups/mqb/mqbblp/mqbblp_queuehandle.h b/src/groups/mqb/mqbblp/mqbblp_queuehandle.h index 3203845c96..95fc5d2c76 100644 --- a/src/groups/mqb/mqbblp/mqbblp_queuehandle.h +++ b/src/groups/mqb/mqbblp/mqbblp_queuehandle.h @@ -69,6 +69,7 @@ class QueueHandle : public mqbi::QueueHandle { private: // TYPES + typedef bsl::shared_ptr StatsSp; /// VST representing downstream substream with all redelivery data. /// (Re)created upon `registerSubStream` @@ -77,9 +78,11 @@ class QueueHandle : public mqbi::QueueHandle { const bsl::string d_appId; unsigned int d_upstreamSubQueueId; mqbi::QueueHandle::RedeliverySp d_data; + const StatsSp d_stats_sp; Downstream(const bsl::string& appId, unsigned int upstreamSubQueueId, + const StatsSp& stats, bslma::Allocator* allocator_p); }; @@ -116,8 +119,9 @@ class QueueHandle : public mqbi::QueueHandle { const bsl::string& appId() const; - static unsigned int nextStamp(); + const StatsSp& stats() const; }; + typedef bsl::shared_ptr SubscriptionSp; typedef bmqc::Array, bmqp::Protocol::k_SUBID_ARRAY_STATIC_LEN> @@ -189,7 +193,9 @@ class QueueHandle : public mqbi::QueueHandle { bmqp::SchemaLearner::Context d_schemaLearnerPushContext; - /// Allocator to use. + /// cache producer stats to avoid lookup + StatsSp d_producerStats; + bslma::Allocator* d_allocator_p; private: @@ -225,10 +231,10 @@ class QueueHandle : public mqbi::QueueHandle { /// Called by the `Queue` to deliver the specified `message` with the /// specified `msgSize`, `msgGUID`, `attributes`, `isOutOfOrder`, and - /// `msgGroupId` for the specified `subQueueInfos` streams of the queue. + /// `msgGroupId` for the specified `subscriptions` streams of the queue. /// The behavior is undefined unless the queueHandle can send a message at /// this time for all of the `subQueueInfos` streams (see - /// 'canDeliver(unsigned int subQueueId)' for more information). + /// 'canDeliver(unsigned int subscription)' for more information). /// /// THREAD: This method is called from the Queue's dispatcher thread. void @@ -236,12 +242,13 @@ class QueueHandle : public mqbi::QueueHandle { const bmqt::MessageGUID& msgGUID, const mqbi::StorageMessageAttributes& attributes, const bmqp::Protocol::MsgGroupId& msgGroupId, - const bmqp::Protocol::SubQueueInfosArray& subQueueInfos, + const bmqp::Protocol::SubQueueInfosArray& subscriptions, bool isOutOfOrder); void makeSubStream(const bsl::string& appId, unsigned int downstreamSubQueueId, - unsigned int upstreamSubQueueId); + unsigned int upstreamSubQueueId, + const StatsSp& stats); const bsl::shared_ptr& downstream(unsigned int subQueueId) const; @@ -314,7 +321,7 @@ class QueueHandle : public mqbi::QueueHandle { /// `upstreamSubQueueId`. /// /// THREAD: This method is called from the Queue's dispatcher thread. - void + mqbi::QueueHandle::SubStreams::const_iterator registerSubStream(const bmqp_ctrlmsg::SubQueueIdInfo& subStreamInfo, unsigned int upstreamSubQueueId, const mqbi::QueueCounts& counts) BSLS_KEYWORD_OVERRIDE; @@ -551,14 +558,15 @@ inline mqbi::DispatcherClient* QueueHandle::client() inline void QueueHandle::makeSubStream(const bsl::string& appId, unsigned int downstreamSubQueueId, - unsigned int upstreamSubQueueId) + unsigned int upstreamSubQueueId, + const StatsSp& stats) { if (downstreamSubQueueId >= d_downstreams.size()) { d_downstreams.resize(downstreamSubQueueId + 1); } d_downstreams[downstreamSubQueueId].reset( new (*d_allocator_p) - Downstream(appId, upstreamSubQueueId, d_allocator_p), + Downstream(appId, upstreamSubQueueId, stats, d_allocator_p), d_allocator_p); } @@ -631,13 +639,15 @@ inline bmqp::SchemaLearner::Context& QueueHandle::schemaLearnerContext() const } inline QueueHandle::Downstream::Downstream(const bsl::string& appId, - unsigned int upstreamSubQueueId, + unsigned int upstreamSubQueueId, + const StatsSp& stats, bslma::Allocator* allocator_p) : d_appId(appId) , d_upstreamSubQueueId(upstreamSubQueueId) , d_data(new(*allocator_p) mqbi::QueueHandle::UnconfirmedMessageInfoMap(allocator_p), allocator_p) +, d_stats_sp(stats) { // NOTHING } diff --git a/src/groups/mqb/mqbblp/mqbblp_queuesessionmanager.cpp b/src/groups/mqb/mqbblp/mqbblp_queuesessionmanager.cpp index 7acb0bba15..46d86c8af1 100644 --- a/src/groups/mqb/mqbblp/mqbblp_queuesessionmanager.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_queuesessionmanager.cpp @@ -26,7 +26,6 @@ #include #include #include -#include // BMQ #include @@ -210,10 +209,10 @@ void QueueSessionManager::onDomainOpenCb( d_requesterContext_sp, request.choice().openQueue().handleParameters(), bdlf::BindUtil::bind(&QueueSessionManager::onQueueOpenCb, - this, // requester + this, bdlf::PlaceHolders::_1, // status bdlf::PlaceHolders::_2, // queueHandle - bdlf::PlaceHolders::_3, // routingCfg + bdlf::PlaceHolders::_3, // openQueueResponse bdlf::PlaceHolders::_4, // confirmationCookie successCallback, request, @@ -221,13 +220,13 @@ void QueueSessionManager::onDomainOpenCb( } void QueueSessionManager::onQueueOpenCb( - const bmqp_ctrlmsg::Status& status, - mqbi::QueueHandle* queueHandle, - const bmqp_ctrlmsg::OpenQueueResponse& openQueueResponse, - const mqbi::Domain::OpenQueueConfirmationCookie& confirmationCookie, - const GetHandleCallback& responseCallback, - const bmqp_ctrlmsg::ControlMessage& request, - const bmqu::AtomicValidatorSp& validator) + const bmqp_ctrlmsg::Status& status, + mqbi::QueueHandle* queueHandle, + const bmqp_ctrlmsg::OpenQueueResponse& openQueueResponse, + const mqbi::OpenQueueConfirmationCookieSp& confirmationCookie, + const GetHandleCallback& responseCallback, + const bmqp_ctrlmsg::ControlMessage& request, + const bmqu::AtomicValidatorSp& validator) { // executed by *ANY* thread @@ -257,12 +256,12 @@ void QueueSessionManager::onQueueOpenCb( } void QueueSessionManager::onQueueOpenCbDispatched( - const bmqp_ctrlmsg::Status& status, - mqbi::QueueHandle* queueHandle, - const bmqp_ctrlmsg::OpenQueueResponse& openQueueResponse, - const mqbi::Domain::OpenQueueConfirmationCookie& confirmationCookie, - const GetHandleCallback& responseCallback, - const bmqp_ctrlmsg::ControlMessage& request) + const bmqp_ctrlmsg::Status& status, + mqbi::QueueHandle* queueHandle, + const bmqp_ctrlmsg::OpenQueueResponse& openQueueResponse, + const mqbi::OpenQueueConfirmationCookieSp& confirmationCookie, + const GetHandleCallback& responseCallback, + const bmqp_ctrlmsg::ControlMessage& request) { // executed by the *CLIENT* dispatcher thread @@ -283,14 +282,13 @@ void QueueSessionManager::onQueueOpenCbDispatched( if (status.category() == bmqp_ctrlmsg::StatusCategory::E_SUCCESS) { BSLS_ASSERT_SAFE(queueHandle); - BSLS_ASSERT_SAFE( - confirmationCookie); // in case of success, the cookie - // must be a valid shared_ptr + BSLS_ASSERT_SAFE(confirmationCookie->d_handle); + // in case of success, the cookie must be a valid shared_ptr // Update the cookie to point to a null queue handle, which indicates // that requester (this client session) has successfully received and // processed the open-queue response. - *confirmationCookie = 0; + confirmationCookie->d_handle = 0; // Success, configure the handle and the session bsl::pair ins = d_queues.emplace( @@ -312,15 +310,9 @@ void QueueSessionManager::onQueueOpenCbDispatched( QueueState::StreamsMap::iterator subQueueInfo = qs.d_subQueueInfosMap.insert(apppId, queueId.subId(), queueId); - bmqst::StatContext* statContext = - subQueueInfo->value().d_stats->statContext(); - - if (!statContext) { - subQueueInfo->value().d_stats->initialize( - queueHandle->queue()->uri(), - d_statContext_p, - d_allocator_p); - statContext = subQueueInfo->value().d_stats->statContext(); + if (!subQueueInfo->value().d_stats_sp && + confirmationCookie->d_stats_sp) { + subQueueInfo->value().d_stats_sp = confirmationCookie->d_stats_sp; const bsl::string& domain = queueHandle->queue()->domain()->name(); const bsl::string& cluster = @@ -333,7 +325,11 @@ void QueueSessionManager::onQueueOpenCbDispatched( const bsls::Types::Int64 queueFlags = queueHandle->handleParameters().flags(); - createQueueStatsDatum(statContext, domain, cluster, queueFlags); + createQueueStatsDatum( + subQueueInfo->value().d_stats_sp->statContext(), + domain, + cluster, + queueFlags); } } @@ -506,15 +502,13 @@ void QueueSessionManager::dispatchErrorCallback( } QueueSessionManager::QueueSessionManager( - mqbi::DispatcherClient* dispatcherClient, - const bmqp_ctrlmsg::ClientIdentity& clientIdentity, - bmqst::StatContext* statContext, - mqbi::DomainFactory* domainFactory, - bslma::Allocator* allocator) + mqbi::DispatcherClient* dispatcherClient, + const bmqp_ctrlmsg::ClientIdentity& clientIdentity, + const bsl::shared_ptr& statContext_sp, + mqbi::DomainFactory* domainFactory, + bslma::Allocator* allocator) : d_dispatcherClient_p(dispatcherClient) -, d_statContext_p(statContext) , d_domainFactory_p(domainFactory) -, d_allocator_p(allocator) , d_shutdownInProgress(false) , d_validator_sp(new(*allocator) bmqu::AtomicValidator(), allocator) , d_requesterContext_sp(new(*allocator) @@ -523,7 +517,7 @@ QueueSessionManager::QueueSessionManager( { // PRECONDITIONS BSLS_ASSERT_SAFE(d_dispatcherClient_p != 0); - BSLS_ASSERT_SAFE(d_statContext_p != 0); + BSLS_ASSERT_SAFE(statContext_sp); BSLS_ASSERT_SAFE(d_domainFactory_p != 0); d_requesterContext_sp->setClient(dispatcherClient) @@ -531,7 +525,8 @@ QueueSessionManager::QueueSessionManager( .setDescription(dispatcherClient->description()) .setIsClusterMember(false) .setRequesterId( - mqbi::QueueHandleRequesterContext ::generateUniqueRequesterId()); + mqbi::QueueHandleRequesterContext::generateUniqueRequesterId()) + .setStatContext(statContext_sp); } QueueSessionManager::~QueueSessionManager() diff --git a/src/groups/mqb/mqbblp/mqbblp_queuesessionmanager.h b/src/groups/mqb/mqbblp/mqbblp_queuesessionmanager.h index f99780e9b6..914d310c9c 100644 --- a/src/groups/mqb/mqbblp/mqbblp_queuesessionmanager.h +++ b/src/groups/mqb/mqbblp/mqbblp_queuesessionmanager.h @@ -27,6 +27,7 @@ // MQB #include #include +#include // BMQ #include @@ -56,12 +57,6 @@ class DomainQualification; namespace mqbi { class DispatcherClient; } -namespace mqbstat { -class QueueStatsClient; -} -namespace bmqst { -class StatContext; -} namespace mqbblp { @@ -93,7 +88,7 @@ class QueueSessionManager { /// opened in the session struct SubQueueInfo { /// Stats of this `SubQueue`, with regards to the client. - bsl::shared_ptr d_stats; + bsl::shared_ptr d_stats_sp; // queueId (id, subId) of the `SubQueue` bmqp::QueueId d_queueId; @@ -103,6 +98,9 @@ class QueueSessionManager { /// Constructor of a new object, initializes all data members to /// default values. SubQueueInfo(const bmqp::QueueId& queueId); + + void onEvent(mqbstat::QueueStatsClient::EventType::Enum type, + bsls::Types::Int64 value) const; }; /// Struct holding the state associated to a queue opened in the session @@ -161,15 +159,9 @@ class QueueSessionManager { // Dispatcher client to use, held not owned. mqbi::DispatcherClient* d_dispatcherClient_p; - // StatContext to use, held not owned. - bmqst::StatContext* d_statContext_p; - // DomainFactory to use, held not owned. mqbi::DomainFactory* d_domainFactory_p; - // Allocator to use. - bslma::Allocator* d_allocator_p; - // Once this flag is set, either the channel has been destroyed and is no // longer valid or we sent the `DisconnectResponse` to the client; in // either way, *NO* messages of any sort should be delivered to the client. @@ -238,13 +230,13 @@ class QueueSessionManager { const bmqu::AtomicValidatorSp& validator); void onQueueOpenCb( - const bmqp_ctrlmsg::Status& status, - mqbi::QueueHandle* queueHandle, - const bmqp_ctrlmsg::OpenQueueResponse& openQueueResponse, - const mqbi::Domain::OpenQueueConfirmationCookie& confirmationCookie, - const GetHandleCallback& responseCallback, - const bmqp_ctrlmsg::ControlMessage& request, - const bmqu::AtomicValidatorSp& validator); + const bmqp_ctrlmsg::Status& status, + mqbi::QueueHandle* queueHandle, + const bmqp_ctrlmsg::OpenQueueResponse& openQueueResponse, + const mqbi::OpenQueueConfirmationCookieSp& confirmationCookie, + const GetHandleCallback& responseCallback, + const bmqp_ctrlmsg::ControlMessage& request, + const bmqu::AtomicValidatorSp& validator); /// Callback invoked in response to an open queue request (in the /// specified `controlMessage`). If the specified `status` is SUCCESS, @@ -256,12 +248,12 @@ class QueueSessionManager { /// of the failure. The `queueHandle` must be released once no longer /// needed. void onQueueOpenCbDispatched( - const bmqp_ctrlmsg::Status& status, - mqbi::QueueHandle* queueHandle, - const bmqp_ctrlmsg::OpenQueueResponse& openQueueResponse, - const mqbi::Domain::OpenQueueConfirmationCookie& confirmationCookie, - const GetHandleCallback& responseCallback, - const bmqp_ctrlmsg::ControlMessage& request); + const bmqp_ctrlmsg::Status& status, + mqbi::QueueHandle* queueHandle, + const bmqp_ctrlmsg::OpenQueueResponse& openQueueResponse, + const mqbi::OpenQueueConfirmationCookieSp& confirmationCookie, + const GetHandleCallback& responseCallback, + const bmqp_ctrlmsg::ControlMessage& request); void onHandleReleased(const bsl::shared_ptr& handle, const mqbi::QueueHandleReleaseResult& result, @@ -300,11 +292,12 @@ class QueueSessionManager { // CREATORS /// Create a `QueueSessionManager` object. - QueueSessionManager(mqbi::DispatcherClient* dispatcherClient, - const bmqp_ctrlmsg::ClientIdentity& clientIdentity, - bmqst::StatContext* statContext, - mqbi::DomainFactory* domainFactory, - bslma::Allocator* allocator); + QueueSessionManager( + mqbi::DispatcherClient* dispatcherClient, + const bmqp_ctrlmsg::ClientIdentity& clientIdentity, + const bsl::shared_ptr& statContext_sp, + mqbi::DomainFactory* domainFactory, + bslma::Allocator* allocator); ~QueueSessionManager(); @@ -343,10 +336,18 @@ class QueueSessionManager { inline QueueSessionManager::SubQueueInfo::SubQueueInfo( const bmqp::QueueId& queueId) -: d_stats() +: d_stats_sp() , d_queueId(queueId) { - d_stats.createInplace(); +} + +inline void QueueSessionManager::SubQueueInfo::onEvent( + mqbstat::QueueStatsClient::EventType::Enum type, + bsls::Types::Int64 value) const +{ + if (d_stats_sp) { + d_stats_sp->onEvent(type, value); + } } // -------------------------------------- diff --git a/src/groups/mqb/mqbblp/mqbblp_relayqueueengine.cpp b/src/groups/mqb/mqbblp/mqbblp_relayqueueengine.cpp index 1322a1e31e..30af68dbee 100644 --- a/src/groups/mqb/mqbblp/mqbblp_relayqueueengine.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_relayqueueengine.cpp @@ -1012,6 +1012,7 @@ int RelayQueueEngine::rebuildInternalState( } mqbi::QueueHandle* RelayQueueEngine::getHandle( + const mqbi::OpenQueueConfirmationCookieSp& context, const bsl::shared_ptr& clientContext, const bmqp_ctrlmsg::QueueHandleParameters& handleParameters, unsigned int upstreamSubQueueId, @@ -1064,7 +1065,7 @@ mqbi::QueueHandle* RelayQueueEngine::getHandle( // Already aware of this queueId from this client. if (QueueEngineUtil::validateUri(handleParameters, queueHandle, - *clientContext) != 0) { + clientContext.get()) != 0) { CALLBACK(bmqp_ctrlmsg::StatusCategory::E_INVALID_ARGUMENT, -1, "Queue URI mismatch for same queueId.", @@ -1161,11 +1162,16 @@ mqbi::QueueHandle* RelayQueueEngine::getHandle( } } - queueHandle->registerSubStream( - downstreamInfo, - upstreamSubQueueId, - mqbi::QueueCounts(handleParameters.readCount(), - handleParameters.writeCount())); + { + mqbi::QueueHandle::SubStreams::const_iterator citSubStream = + queueHandle->registerSubStream( + downstreamInfo, + upstreamSubQueueId, + mqbi::QueueCounts(handleParameters.readCount(), + handleParameters.writeCount())); + + context->d_stats_sp = citSubStream->second.d_clientStats_sp; + } // If a new reader/write, insert its (default-valued) stream parameters // into our map of consumer stream parameters advertised upstream. @@ -1180,6 +1186,7 @@ mqbi::QueueHandle* RelayQueueEngine::getHandle( &insertResult.first->second.d_handleParameters, handleParameters); } + // Inform the requester of the success CALLBACK(bmqp_ctrlmsg::StatusCategory::E_SUCCESS, 0, "", queueHandle); diff --git a/src/groups/mqb/mqbblp/mqbblp_relayqueueengine.h b/src/groups/mqb/mqbblp/mqbblp_relayqueueengine.h index 9636866587..6999626413 100644 --- a/src/groups/mqb/mqbblp/mqbblp_relayqueueengine.h +++ b/src/groups/mqb/mqbblp/mqbblp_relayqueueengine.h @@ -411,7 +411,8 @@ class RelayQueueEngine BSLS_KEYWORD_FINAL : public mqbi::QueueEngine { /// /// THREAD: This method is called from the Queue's dispatcher thread. mqbi::QueueHandle* - getHandle(const bsl::shared_ptr& + getHandle(const mqbi::OpenQueueConfirmationCookieSp& context, + const bsl::shared_ptr& clientContext, const bmqp_ctrlmsg::QueueHandleParameters& handleParameters, unsigned int upstreamSubQueueId, diff --git a/src/groups/mqb/mqbblp/mqbblp_remotequeue.cpp b/src/groups/mqb/mqbblp/mqbblp_remotequeue.cpp index 4335f20e9c..6a428d0a1a 100644 --- a/src/groups/mqb/mqbblp/mqbblp_remotequeue.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_remotequeue.cpp @@ -620,6 +620,7 @@ void RemoteQueue::close() } void RemoteQueue::getHandle( + const mqbi::OpenQueueConfirmationCookieSp& context, const bsl::shared_ptr& clientContext, const bmqp_ctrlmsg::QueueHandleParameters& handleParameters, unsigned int upstreamSubQueueId, @@ -659,7 +660,8 @@ void RemoteQueue::getHandle( } } - d_queueEngine_mp->getHandle(clientContext, + d_queueEngine_mp->getHandle(context, + clientContext, handleParameters, upstreamSubQueueId, callback); @@ -1023,8 +1025,65 @@ void RemoteQueue::postMessage(const bmqp::PutHeader& putHeaderIn, } } - if (ctx.d_state == SubStreamContext::e_OPENED && - d_state_p->isAtMostOnce()) { + // Update the domain's PUT stats (note that ideally this should be done at + // the time the message is actually sent upstream, i.e. in + // cluster/clusterProxy) for the most exact accuracy, but doing it here is + // good enough. + d_state_p->stats()->onEvent( + appData->length()); + + mqbi::InlineResult::Enum rc = mqbi::InlineResult::e_UNAVAILABLE; + + if (ctx.d_state == SubStreamContext::e_OPENED) { + putHeader.setQueueId(d_state_p->id()); + + mqbi::Cluster* cluster = d_state_p->domain()->cluster(); + + rc = cluster->sendPutInline(d_state_p->partitionId(), + putHeader, + appData, + options, + state, + ctx.d_genCount); + + if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( + mqbi::InlineResult::isPermanentError(rc))) { + BSLS_PERFORMANCEHINT_UNLIKELY_HINT; + + // Cannot make progress with this PUT. NACK and drop it + + // Use the same throttle as for incoming PUTs + if (d_throttledFailedPutMessages.requestPermission()) { + BALL_LOG_WARN + << "Failed to relay PUT message " + << "[queueId: " << d_state_p->id() + << ", GUID: " << putHeader.messageGUID() + << "]. Reason: " << mqbi::InlineResult::toAscii(rc); + } + bmqp::AckMessage ackMessage; + bmqt::AckResult::Enum ackResult = mqbi::InlineResult::toAckResult( + rc); + ackMessage.setStatus( + bmqp::ProtocolUtil::ackResultToCode(ackResult)); + + ackMessage.setMessageGUID(putHeader.messageGUID()); + + d_state_p->stats() + ->onEvent(1); + + // CorrelationId & QueueId are left unset as those fields + // will be filled downstream. + source->onAckMessage(ackMessage); + + // REVISIT: still want to increment the stats? + return; // RETURN + } + } + + // Ignore transient errors since retransmission is still possible even for + // broadcast queue since the PUT did not go upstream. + + if (rc == mqbi::InlineResult::e_SUCCESS && d_state_p->isAtMostOnce()) { d_pendingMessages.insert( bsl::make_pair(putHeader.messageGUID(), PutMessage(source, putHeader, 0, 0, 0, state))); @@ -1035,18 +1094,6 @@ void RemoteQueue::postMessage(const bmqp::PutHeader& putHeaderIn, PutMessage(source, putHeader, appData, options, now, state))); } - if (ctx.d_state == SubStreamContext::e_OPENED) { - sendPutMessage(putHeader, appData, options, state, ctx.d_genCount); - } - - // Update the domain's PUT stats (note that ideally this should be done at - // the time the message is actually sent upstream, i.e. in - // cluster/clusterProxy) for the most exact accuracy, but doing it here is - // good enough. - - d_state_p->stats()->onEvent( - appData->length()); - if (d_queueEngine_mp) { d_queueEngine_mp->afterPostMessage(); } @@ -1177,23 +1224,28 @@ void RemoteQueue::sendConfirmMessage(const bmqt::MessageGUID& msgGUID, << msgGUID << "']"; // Relay the CONFIRM message via clusterProxy/cluster. - mqbi::Queue* queue = d_state_p->queue(); mqbi::Cluster* cluster = d_state_p->domain()->cluster(); bmqp::ConfirmMessage confirmMessage; + confirmMessage.setQueueId(d_state_p->id()) .setSubQueueId(upstreamSubQueueId) .setMessageGUID(msgGUID); - mqbi::Dispatcher* dispatcher = queue->dispatcher(); - mqbi::DispatcherEvent* dispEvent = dispatcher->getEvent(cluster); - (*dispEvent) - .setType(mqbi::DispatcherEventType::e_CONFIRM) - .setSource(queue) - .setConfirmMessage(confirmMessage) - .setPartitionId(d_state_p->partitionId()) - .setIsRelay(true); // Relay message - // partitionId is needed only by replica - dispatcher->dispatchEvent(dispEvent, cluster); + mqbi::InlineResult::Enum rc = + cluster->sendConfirmInline(d_state_p->partitionId(), confirmMessage); + + if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY(rc != + mqbi::InlineResult::e_SUCCESS)) { + BSLS_PERFORMANCEHINT_UNLIKELY_HINT; + + if (d_throttledFailedConfirmMessages.requestPermission()) { + BALL_LOG_WARN << "Failed to relay CONFIRM message " + << "[queueId: " << d_state_p->id() + << ", subQueueId: " << upstreamSubQueueId + << ", GUID: " << confirmMessage.messageGUID() + << "]. Reason: " << mqbi::InlineResult::toAscii(rc); + } + } } void RemoteQueue::onAckMessageDispatched(const mqbi::DispatcherAckEvent& event) @@ -1446,6 +1498,7 @@ void RemoteQueue::retransmitPendingMessagesDispatched( size_t numMessages = d_pendingMessages.size(); const bmqt::MessageGUID firstGUID = d_pendingMessages.begin()->second.d_header.messageGUID(); + mqbi::Cluster* cluster = d_state_p->domain()->cluster(); for (Puts::iterator it = d_pendingMessages.begin(); it != d_pendingMessages.end();) { @@ -1455,20 +1508,37 @@ void RemoteQueue::retransmitPendingMessagesDispatched( it->second.d_state_sp->cancel(); it->second.d_state_sp = d_statePool_p->getObject(); } - sendPutMessage(it->second.d_header, - it->second.d_appData, - it->second.d_options, - it->second.d_state_sp, - genCount); - ++numTransmitted; - if (0 == it->second.d_timeReceived) { - // This is broadcast (not time-controlled); no more - // retransmission unless NOT_READY NACK is received in - // which case NACK will supply the data. - it->second.d_appData.clear(); - it->second.d_options.clear(); + + mqbi::InlineResult::Enum rc = cluster->sendPutInline( + d_state_p->partitionId(), + it->second.d_header, + it->second.d_appData, + it->second.d_options, + it->second.d_state_sp, + genCount); + + if (BSLS_PERFORMANCEHINT_PREDICT_UNLIKELY( + mqbi::InlineResult::isPermanentError(rc))) { + BSLS_PERFORMANCEHINT_UNLIKELY_HINT; + + bmqp::AckMessage ackMessage; + bmqt::AckResult::Enum ackResult = + mqbi::InlineResult::toAckResult(rc); + ackMessage.setStatus( + bmqp::ProtocolUtil::ackResultToCode(ackResult)); + it = nack(it, ackMessage); + } + else { + ++numTransmitted; + if (0 == it->second.d_timeReceived) { + // This is broadcast (not time-controlled); no more + // retransmission unless NOT_READY NACK is received in + // which case NACK will supply the data. + it->second.d_appData.clear(); + it->second.d_options.clear(); + } + ++it; } - ++it; } else { // This is previously transmitted broadcast for which there was @@ -1569,34 +1639,6 @@ RemoteQueue::Puts::iterator& RemoteQueue::nack(Puts::iterator& it, return it; } -void RemoteQueue::sendPutMessage( - const bmqp::PutHeader& putHeader, - const bsl::shared_ptr& appData, - const bsl::shared_ptr& options, - const bsl::shared_ptr& state, - bsls::Types::Uint64 genCount) -{ - mqbi::Cluster* cluster = d_state_p->domain()->cluster(); - - // Replica or Proxy. Update queueId to the one known upstream. - bmqp::PutHeader& ph = const_cast(putHeader); - ph.setQueueId(d_state_p->id()); - - mqbi::Dispatcher* dispatcher = d_state_p->queue()->dispatcher(); - mqbi::DispatcherEvent* dispEvent = dispatcher->getEvent(cluster); - (*dispEvent) - .setType(mqbi::DispatcherEventType::e_PUT) - .setIsRelay(true) // Relay message - .setSource(d_state_p->queue()) - .setPutHeader(ph) - .setPartitionId(d_state_p->partitionId()) // Only replica uses - .setBlob(appData) - .setOptions(options) - .setGenCount(genCount) - .setState(state); - dispatcher->dispatchEvent(dispEvent, cluster); -} - void RemoteQueue::onOpenFailure(unsigned int upstreamSubQueueId) { // executed by the *DISPATCHER* thread diff --git a/src/groups/mqb/mqbblp/mqbblp_remotequeue.h b/src/groups/mqb/mqbblp/mqbblp_remotequeue.h index c63606751e..b4fe54cba7 100644 --- a/src/groups/mqb/mqbblp/mqbblp_remotequeue.h +++ b/src/groups/mqb/mqbblp/mqbblp_remotequeue.h @@ -291,12 +291,6 @@ class RemoteQueue { void cleanPendingMessages(mqbi::QueueHandle* handle); Puts::iterator& nack(Puts::iterator& it, bmqp::AckMessage& ackMessage); - void sendPutMessage(const bmqp::PutHeader& putHeader, - const bsl::shared_ptr& appData, - const bsl::shared_ptr& options, - const bsl::shared_ptr& state, - bsls::Types::Uint64 genCount); - void sendConfirmMessage(const bmqt::MessageGUID& msgGUID, unsigned int upstreamSubQueueId, mqbi::QueueHandle* source); @@ -356,7 +350,8 @@ class RemoteQueue { /// result. /// /// THREAD: This method is called from the Queue's dispatcher thread. - void getHandle(const bsl::shared_ptr& + void getHandle(const mqbi::OpenQueueConfirmationCookieSp& context, + const bsl::shared_ptr& clientContext, const bmqp_ctrlmsg::QueueHandleParameters& handleParameters, unsigned int upstreamSubQueueId, diff --git a/src/groups/mqb/mqbblp/mqbblp_remotequeue.t.cpp b/src/groups/mqb/mqbblp/mqbblp_remotequeue.t.cpp index 7af661659b..007021bc0a 100644 --- a/src/groups/mqb/mqbblp/mqbblp_remotequeue.t.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_remotequeue.t.cpp @@ -107,13 +107,14 @@ class TestBench { public: struct PutEvent { const bmqp::PutHeader d_header; - mqbi::DispatcherClient* d_source; bsl::shared_ptr d_appData; bsl::shared_ptr d_options; - PutEvent(const mqbi::DispatcherPutEvent& event, - mqbi::DispatcherClient* source); + PutEvent(const bmqp::PutHeader& header, + const bsl::shared_ptr& appData, + const bsl::shared_ptr& options); }; + class TestRemoteQueue { public: bsl::shared_ptr d_queue_sp; @@ -149,10 +150,16 @@ class TestBench { bmqp_ctrlmsg::RoutingConfiguration& routingConfig); /// Install to handle (record) Cluster events - void eventProcessor(const mqbi::DispatcherEvent& event); + mqbi::InlineResult::Enum + putProcessor(int partitionId, + const bmqp::PutHeader& putHeader, + const bsl::shared_ptr& appData, + const bsl::shared_ptr& options, + const bsl::shared_ptr& state, + bsls::Types::Uint64 genCount); /// Ack all recorded PUTs - void ackPuts(bmqt::AckResult::Enum status); + void ackPuts(mqbi::Queue* queue, bmqt::AckResult::Enum status); void dropPuts(); @@ -176,10 +183,14 @@ TestBench::TestBench(bslma::Allocator* allocator_p) d_dispatcher._setInDispatcherThread(true); - d_cluster._setEventProcessor( - bdlf::BindUtil::bind(&TestBench::eventProcessor, - this, - bdlf::PlaceHolders::_1)); + d_cluster._setPutFunctor(bdlf::BindUtil::bind(&TestBench::putProcessor, + this, + bdlf::PlaceHolders::_1, + bdlf::PlaceHolders::_2, + bdlf::PlaceHolders::_3, + bdlf::PlaceHolders::_4, + bdlf::PlaceHolders::_5, + bdlf::PlaceHolders::_6)); bmqsys::Time::initialize( bdlf::BindUtil::bind(&TestClock::realtimeClock, &d_testClock), @@ -213,28 +224,30 @@ bsl::shared_ptr TestBench::getQueue() return queue_sp; } -TestBench::PutEvent::PutEvent(const mqbi::DispatcherPutEvent& event, - mqbi::DispatcherClient* source) -: d_header(event.putHeader()) -, d_source(source) -, d_appData(event.blob()) -, d_options(event.blob()) +TestBench::PutEvent::PutEvent(const bmqp::PutHeader& header, + const bsl::shared_ptr& appData, + const bsl::shared_ptr& options) +: d_header(header) +, d_appData(appData) +, d_options(options) { // NOTHING } -void TestBench::eventProcessor(const mqbi::DispatcherEvent& event) +mqbi::InlineResult::Enum TestBench::putProcessor( + BSLS_ANNOTATION_UNUSED int partitionId, + const bmqp::PutHeader& putHeader, + const bsl::shared_ptr& appData, + const bsl::shared_ptr& options, + BSLS_ANNOTATION_UNUSED const bsl::shared_ptr& state, + BSLS_ANNOTATION_UNUSED bsls::Types::Uint64 genCount) { - if (event.type() == mqbi::DispatcherEventType::e_PUT) { - const mqbi::DispatcherPutEvent* realEvent = event.asPutEvent(); + d_puts.push(PutEvent(putHeader, appData, options)); - if (realEvent->isRelay()) { - d_puts.push(PutEvent(*realEvent, event.source())); - } - } + return mqbi::InlineResult::e_SUCCESS; } -void TestBench::ackPuts(bmqt::AckResult::Enum status) +void TestBench::ackPuts(mqbi::Queue* queue, bmqt::AckResult::Enum status) { while (!d_puts.empty()) { const bmqp::PutHeader& ph = d_puts.front().d_header; @@ -252,7 +265,7 @@ void TestBench::ackPuts(bmqt::AckResult::Enum status) .setAckMessage(ackMessage) .setBlob(d_puts.front().d_appData) .setOptions(d_puts.front().d_options); - d_puts.front().d_source->onDispatcherEvent(ackEvent); + queue->onDispatcherEvent(ackEvent); } d_puts.pop(); } @@ -509,7 +522,7 @@ static void test1_fanoutBasic() y.postOneMessage(&theQueue.d_remoteQueue); // everything is ACK'ed with e_SUCCESS - theBench.ackPuts(bmqt::AckResult::e_SUCCESS); + theBench.ackPuts(theQueue.d_queue_sp.get(), bmqt::AckResult::e_SUCCESS); BMQTST_ASSERT_EQ(0U, x.count()); BMQTST_ASSERT_EQ(0U, y.count()); @@ -521,7 +534,7 @@ static void test1_fanoutBasic() y.postOneMessage(&theQueue.d_remoteQueue); // Everything is pending - theBench.ackPuts(bmqt::AckResult::e_NOT_READY); + theBench.ackPuts(theQueue.d_queue_sp.get(), bmqt::AckResult::e_NOT_READY); // RemoteQueue should terminate 'e_NOT_READY' BMQTST_ASSERT_EQ(2U, x.count()); BMQTST_ASSERT_EQ(3U, y.count()); @@ -535,7 +548,7 @@ static void test1_fanoutBasic() bmqp::QueueId::k_DEFAULT_SUBQUEUE_ID); // everything is ACK'ed with e_SUCCESS - theBench.ackPuts(bmqt::AckResult::e_SUCCESS); + theBench.ackPuts(theQueue.d_queue_sp.get(), bmqt::AckResult::e_SUCCESS); BMQTST_ASSERT_EQ(0U, x.count()); BMQTST_ASSERT_EQ(0U, y.count()); @@ -579,7 +592,7 @@ static void test1_fanoutBasic() x.d_status = bmqt::AckResult::e_SUCCESS; y.d_status = bmqt::AckResult::e_SUCCESS; - theBench.ackPuts(bmqt::AckResult::e_SUCCESS); + theBench.ackPuts(theQueue.d_queue_sp.get(), bmqt::AckResult::e_SUCCESS); BMQTST_ASSERT_EQ(0U, x.count()); BMQTST_ASSERT_EQ(0U, y.count()); @@ -663,7 +676,7 @@ static void test2_broadcastBasic() // everything but broadcast is ACK'ed with e_SUCCESS // One (Nth) broadcast is ACK'ed resulting in removal of all previously // broadcasted PUTs. - theBench.ackPuts(bmqt::AckResult::e_SUCCESS); + theBench.ackPuts(theQueue.d_queue_sp.get(), bmqt::AckResult::e_SUCCESS); BMQTST_ASSERT_EQ(0U, z.count()); pendingBroadcastPutsWithData = 0; @@ -685,7 +698,7 @@ static void test2_broadcastBasic() } // Everything is pending including all N broadcast PUTs - theBench.ackPuts(bmqt::AckResult::e_NOT_READY); + theBench.ackPuts(theQueue.d_queue_sp.get(), bmqt::AckResult::e_NOT_READY); // RemoteQueue should terminate 'e_NOT_READY' BMQTST_ASSERT_EQ(0U, z.count()); @@ -727,7 +740,7 @@ static void test2_broadcastBasic() // everything but broadcast is ACK'ed with e_SUCCESS // One (N + 1) broadcast is ACK'ed resulting in removal of all previously // broadcasted PUTs. - theBench.ackPuts(bmqt::AckResult::e_SUCCESS); + theBench.ackPuts(theQueue.d_queue_sp.get(), bmqt::AckResult::e_SUCCESS); BMQTST_ASSERT_EQ(0U, z.count()); @@ -797,7 +810,7 @@ static void test2_broadcastBasic() 3, bmqp::QueueId::k_DEFAULT_SUBQUEUE_ID); - theBench.ackPuts(bmqt::AckResult::e_SUCCESS); + theBench.ackPuts(theQueue.d_queue_sp.get(), bmqt::AckResult::e_SUCCESS); BMQTST_ASSERT_EQ(0U, z.count()); @@ -841,7 +854,7 @@ static void test2_broadcastBasic() 2, bmqp::QueueId::k_DEFAULT_SUBQUEUE_ID); - theBench.ackPuts(bmqt::AckResult::e_SUCCESS); + theBench.ackPuts(theQueue.d_queue_sp.get(), bmqt::AckResult::e_SUCCESS); BMQTST_ASSERT_EQ(0U, z.count()); pendingBroadcastPutsWithData = 0; @@ -1023,7 +1036,7 @@ static void test4_buffering() BMQTST_ASSERT_EQ(5U, theBench.d_puts.size()); // ACK with e_SUCCESS - theBench.ackPuts(bmqt::AckResult::e_SUCCESS); + theBench.ackPuts(theQueue.d_queue_sp.get(), bmqt::AckResult::e_SUCCESS); BMQTST_ASSERT_EQ(2U, x.count()); BMQTST_ASSERT_EQ(3U, y.count()); @@ -1044,7 +1057,7 @@ static void test4_buffering() BMQTST_ASSERT_EQ(10U, theBench.d_puts.size()); // everything is ACK'ed with e_SUCCESS - theBench.ackPuts(bmqt::AckResult::e_SUCCESS); + theBench.ackPuts(theQueue.d_queue_sp.get(), bmqt::AckResult::e_SUCCESS); BMQTST_ASSERT_EQ(0U, x.count()); BMQTST_ASSERT_EQ(0U, y.count()); diff --git a/src/groups/mqb/mqbblp/mqbblp_rootqueueengine.cpp b/src/groups/mqb/mqbblp/mqbblp_rootqueueengine.cpp index c774abd141..32d1b7187b 100644 --- a/src/groups/mqb/mqbblp/mqbblp_rootqueueengine.cpp +++ b/src/groups/mqb/mqbblp/mqbblp_rootqueueengine.cpp @@ -609,6 +609,7 @@ int RootQueueEngine::rebuildInternalState(bsl::ostream& errorDescription) } mqbi::QueueHandle* RootQueueEngine::getHandle( + const mqbi::OpenQueueConfirmationCookieSp& context, const bsl::shared_ptr& clientContext, const bmqp_ctrlmsg::QueueHandleParameters& handleParameters, unsigned int upstreamSubQueueId, @@ -837,11 +838,14 @@ mqbi::QueueHandle* RootQueueEngine::getHandle( d_queueState_p->handleParameters())); // Register substream - queueHandle->registerSubStream( - subStreamInfo, - upstreamSubQueueId, - mqbi::QueueCounts(handleParameters.readCount(), - handleParameters.writeCount())); + mqbi::QueueHandle::SubStreams::const_iterator citSubStream = + queueHandle->registerSubStream( + subStreamInfo, + upstreamSubQueueId, + mqbi::QueueCounts(handleParameters.readCount(), + handleParameters.writeCount())); + + context->d_stats_sp = citSubStream->second.d_clientStats_sp; // Inform the requester of the success CALLBACK(bmqp_ctrlmsg::StatusCategory::E_SUCCESS, 0, "", queueHandle); diff --git a/src/groups/mqb/mqbblp/mqbblp_rootqueueengine.h b/src/groups/mqb/mqbblp/mqbblp_rootqueueengine.h index 0c8df63ce0..92460c661d 100644 --- a/src/groups/mqb/mqbblp/mqbblp_rootqueueengine.h +++ b/src/groups/mqb/mqbblp/mqbblp_rootqueueengine.h @@ -280,7 +280,8 @@ class RootQueueEngine BSLS_KEYWORD_FINAL : public mqbi::QueueEngine { /// /// THREAD: This method is called from the Queue's dispatcher thread. mqbi::QueueHandle* - getHandle(const bsl::shared_ptr& + getHandle(const mqbi::OpenQueueConfirmationCookieSp& context, + const bsl::shared_ptr& clientContext, const bmqp_ctrlmsg::QueueHandleParameters& handleParameters, unsigned int upstreamSubQueueId, diff --git a/src/groups/mqb/mqbc/mqbc_clustermembership.cpp b/src/groups/mqb/mqbc/mqbc_clustermembership.cpp index c0d83a4133..34883ed8f9 100644 --- a/src/groups/mqb/mqbc/mqbc_clustermembership.cpp +++ b/src/groups/mqb/mqbc/mqbc_clustermembership.cpp @@ -79,7 +79,16 @@ ClusterMembership::setSelfNodeStatus(bmqp_ctrlmsg::NodeStatus::Value value) bmqp_ctrlmsg::NodeStatus::Value oldVal = selfSession->nodeStatus(); - selfSession->setNodeStatus(value); + selfSession->setNodeStatus(value, value); + + for (ClusterNodeSessionMapConstIter cit = clusterNodeSessionMap().begin(); + cit != clusterNodeSessionMap().end(); + ++cit) { + mqbc::ClusterNodeSession* session = cit->second.get(); + if (session != selfSession) { + session->setNodeStatus(session->nodeStatus(), value); + } + } if (oldVal == value) { return *this; // RETURN diff --git a/src/groups/mqb/mqbc/mqbc_clusternodesession.cpp b/src/groups/mqb/mqbc/mqbc_clusternodesession.cpp index f336c46fc5..8bf5d934df 100644 --- a/src/groups/mqb/mqbc/mqbc_clusternodesession.cpp +++ b/src/groups/mqb/mqbc/mqbc_clusternodesession.cpp @@ -33,11 +33,12 @@ namespace mqbc { // CREATORS ClusterNodeSession::ClusterNodeSession( - mqbi::DispatcherClient* cluster, - mqbnet::ClusterNode* netNode, - const bsl::string& clusterName, - const bmqp_ctrlmsg::ClientIdentity& identity, - bslma::Allocator* allocator) + mqbi::DispatcherClient* cluster, + mqbnet::ClusterNode* netNode, + const bsl::string& clusterName, + const bmqp_ctrlmsg::ClientIdentity& identity, + const bsl::shared_ptr& statContext, + bslma::Allocator* allocator) : d_cluster_p(cluster) , d_clusterNode_p(netNode) , d_peerInstanceId(0) // There is no invalid value for this field @@ -45,9 +46,13 @@ ClusterNodeSession::ClusterNodeSession( new(*allocator) mqbi::QueueHandleRequesterContext(allocator), allocator) , d_nodeStatus(bmqp_ctrlmsg::NodeStatus::E_UNAVAILABLE) // See note in ctor -, d_statContext_sp() , d_primaryPartitions(allocator) , d_queueHandles(allocator) +, d_gatePush() +, d_gateAck() +, d_gatePut() +, d_gateConfirm() +, d_statContext_sp(statContext) { // Note regarding 'd_nodeStatus': it must be initialized with E_UNAVAILABLE // because this value indicates that self node is not connected to this @@ -58,7 +63,9 @@ ClusterNodeSession::ClusterNodeSession( .setDescription(description()) .setIsClusterMember(true) .setRequesterId( - mqbi::QueueHandleRequesterContext ::generateUniqueRequesterId()); + mqbi::QueueHandleRequesterContext ::generateUniqueRequesterId()) + .setInlineClient(this) + .setStatContext(statContext); // TBD: The passed in 'queueHandleRequesterIdentity' is currently the // 'clusterState->identity()' (representing the identity of self node // in the cluster); and it should instead represent the identity of @@ -92,6 +99,11 @@ void ClusterNodeSession::teardown() qit = d_queueHandles.erase(qit); } + d_gatePush.close(); + d_gateAck.close(); + d_gatePut.close(); + d_gateConfirm.close(); + // TBD: Synchronize on the dispatcher ? } @@ -182,5 +194,90 @@ void ClusterNodeSession::removeAllPartitions() d_primaryPartitions.clear(); } +mqbi::InlineResult::Enum ClusterNodeSession::sendPush( + const bmqt::MessageGUID& msgGUID, + int queueId, + const bsl::shared_ptr& message, + const mqbi::StorageMessageAttributes& attributes, + const bmqp::MessagePropertiesInfo& mps, + const bmqp::Protocol::SubQueueInfosArray& subQueueInfos) +{ + // executed by the *QUEUE DISPATCHER* thread + // This PUSH message is enqueued by mqbblp::Queue/QueueHandle on this node, + // and needs to be forwarded to 'event.clusterNode()' (the replica node, + // which is the client). Note that replica is already expected to have the + // payload, and so, primary (this node) sends only the guid and, if + // applicable, the associated subQueueIds. + + GateKeeper::Status status(d_gatePush); + + if (!status.isOpen()) { + // Target node (or self) is not AVAILABLE, so we don't send this PUSH + // to it. Note that this PUSH msg was dispatched by the queue handle + // representing the target node, and will be in its 'pending list'. + + return mqbi::InlineResult::e_UNAVAILABLE; // RETURN + } + + bmqt::GenericResult::Enum rc = bmqt::GenericResult::e_SUCCESS; + // TBD: groupId: also pass options to the 'PushEventBuilder::packMessage' + // routine below. + + if (message) { + // If it's at most once, then we explicitly send the payload since it's + // in-mem mode and there's been no replication (i.e. no preceding + // STORAGE message). + rc = clusterNode()->channel().writePush( + message, + queueId, + msgGUID, + 0, + attributes.compressionAlgorithmType(), + mps, + subQueueInfos); + } + else { + rc = clusterNode()->channel().writePush( + queueId, + msgGUID, + 0, + attributes.compressionAlgorithmType(), + mps, + subQueueInfos); + } + + return rc == bmqt::GenericResult::e_SUCCESS + ? mqbi::InlineResult::e_SUCCESS + : mqbi::InlineResult::e_CHANNEL_ERROR; +} + +mqbi::InlineResult::Enum +ClusterNodeSession::sendAck(int queueId, const bmqp::AckMessage& ackMessage) +{ + // executed by the *QUEUE DISPATCHER* thread + + // This ACK message is enqueued by mqbblp::Queue on this node, and needs to + // be forwarded to 'clusterNode()' (the replica node). + + GateKeeper::Status status(d_gateAck); + + if (!status.isOpen()) { + // Drop the ACK because downstream node (or self) is either starting, + // or shut down. + + return mqbi::InlineResult::e_UNAVAILABLE; // RETURN + } + + bmqt::GenericResult::Enum rc = clusterNode()->channel().writeAck( + ackMessage.status(), + ackMessage.correlationId(), + ackMessage.messageGUID(), + queueId); + + return rc == bmqt::GenericResult::e_SUCCESS + ? mqbi::InlineResult::e_SUCCESS + : mqbi::InlineResult::e_CHANNEL_ERROR; +} + } // close package namespace } // close enterprise namespace diff --git a/src/groups/mqb/mqbc/mqbc_clusternodesession.h b/src/groups/mqb/mqbc/mqbc_clusternodesession.h index 660d2fdeb8..c3ebc65b9a 100644 --- a/src/groups/mqb/mqbc/mqbc_clusternodesession.h +++ b/src/groups/mqb/mqbc/mqbc_clusternodesession.h @@ -28,7 +28,7 @@ #include #include #include -#include +#include // BMQ #include @@ -51,8 +51,92 @@ namespace BloombergLP { namespace mqbc { -// FORWARD DECLARATION -class ClusterState; +// ================ +// class AtomicGate +// ================ + +class AtomicGate { + // This thread-safe mechanism maintains binary state (open/close) allowing + // efficiently check the state ('tryEnter'/'leave'), change it to open + // ('open'), and to close ('closeAndDrain') waiting for all 'leave' calls + // matching 'tryEnter' calls. + + private: + // PRIVATE TYPES + + /// The Enum values have arithmetical meaning and cannot be changed, or + /// extended, or reordered. + enum Enum { e_INIT = 0, e_CLOSE = 1, e_ENTER = 2 }; + + // PRIVATE DATA + bsls::AtomicInt d_value; + + public: + // CREATORS + AtomicGate(bool isOpen); + ~AtomicGate(); + + // MANIPULATORS + void closeAndDrain(); + // Write lock. + + void open(); + // Undo 'closeAndDrain'. + + bool tryEnter(); + // Return true if 'closeAndDrain' has not been called. + + void leave(); + // Undo 'tryEnter'. The behavior is undefined if 'tryEnter' + // has returned 'false', or if called more than once. +}; + +// ================ +// class GateKeeper +// ================ + +class GateKeeper { + // This mechanism is a wrapper around 'AtomicGate' allowing 'tryEnter' + // and 'leave' using RAII ('Status'). + // 'open' and 'close' are not thread-safe and can be called from the thread + // maintaining the status, while 'Status' ('tryEnter'/'leave') is + // thread-safe and can be called from any thread attempting to 'enter' the + // gate. + + private: + // DATA + AtomicGate d_gate; + bool d_isOpen; + + public: + // TYPES + class Status { + private: + // DATA + AtomicGate& d_gate; + bool d_isOpen; + + private: + // NOT IMPLEMENTED + Status(const Status& other) BSLS_KEYWORD_DELETED; + + public: + // CREATORS + Status(GateKeeper& lock); + ~Status(); + + // ACCESSORS + bool isOpen() const; + }; + + public: + // CREATORS + GateKeeper(); + + // MANIPULATORS + void open(); + void close(); +}; // ======================== // class ClusterNodeSession @@ -60,7 +144,8 @@ class ClusterState; /// Provide a session for interaction with BlazingMQ cluster node. class ClusterNodeSession : public mqbi::DispatcherClient, - public mqbi::QueueHandleRequester { + public mqbi::QueueHandleRequester, + public mqbi::InlineClient { private: // CLASS-SCOPE CATEGORY BALL_LOG_SET_CLASS_CATEGORY("MQBC.CLUSTERNODESESSION"); @@ -72,13 +157,17 @@ class ClusterNodeSession : public mqbi::DispatcherClient, /// opened in this session. struct SubQueueInfo { /// Stats of this SubQueue, with regards to the client. - bsl::shared_ptr d_clientStats; + bsl::shared_ptr d_clientStats_sp; // CREATORS /// Constructor of a new object, initializes all data members to /// default values. SubQueueInfo(); + + // ACCESSORS + void onEvent(mqbstat::QueueStatsClient::EventType::Enum type, + bsls::Types::Int64 value) const; }; /// Struct holding the state associated to a queue opened in by the @@ -129,8 +218,6 @@ class ClusterNodeSession : public mqbi::DispatcherClient, typedef QueueHandleMap::const_iterator QueueHandleMapConstIter; - typedef bsl::shared_ptr StatContextSp; - typedef QueueState::StreamsMap StreamsMap; private: @@ -160,14 +247,21 @@ class ClusterNodeSession : public mqbi::DispatcherClient, /// Node status. bmqp_ctrlmsg::NodeStatus::Value d_nodeStatus; - StatContextSp d_statContext_sp; - /// PartitionIds for which this node is the primary. bsl::vector d_primaryPartitions; /// List of queue handles opened on this node by `d_clusterNode_p`. QueueHandleMap d_queueHandles; + /// Gates controlling sending messages inline (from multiple threads). + GateKeeper d_gatePush; + GateKeeper d_gateAck; + GateKeeper d_gatePut; + GateKeeper d_gateConfirm; + + // Extra reference to the stat context + bsl::shared_ptr d_statContext_sp; + private: // NOT IMPLEMENTED ClusterNodeSession(const ClusterNodeSession&); // = delete; @@ -179,11 +273,13 @@ class ClusterNodeSession : public mqbi::DispatcherClient, bslma::UsesBslmaAllocator) // CREATORS - explicit ClusterNodeSession(mqbi::DispatcherClient* cluster, - mqbnet::ClusterNode* netNode, - const bsl::string& clusterName, - const bmqp_ctrlmsg::ClientIdentity& identity, - bslma::Allocator* allocator); + explicit ClusterNodeSession( + mqbi::DispatcherClient* cluster, + mqbnet::ClusterNode* netNode, + const bsl::string& clusterName, + const bmqp_ctrlmsg::ClientIdentity& identity, + const bsl::shared_ptr& statContext, + bslma::Allocator* allocator); /// Destructor. ~ClusterNodeSession() BSLS_KEYWORD_OVERRIDE; @@ -211,13 +307,14 @@ class ClusterNodeSession : public mqbi::DispatcherClient, /// Return a reference to the dispatcherClientData. mqbi::DispatcherClientData& dispatcherClientData() BSLS_KEYWORD_OVERRIDE; - void setNodeStatus(bmqp_ctrlmsg::NodeStatus::Value value); + /// @brief Update the status of this node session. + /// @param other Node status of the node this session is connected to. + /// @param self Node status of the node this session belongs to. + void setNodeStatus(bmqp_ctrlmsg::NodeStatus::Value other, + bmqp_ctrlmsg::NodeStatus::Value self); void setPeerInstanceId(int value); - /// Return the associated stat context. - StatContextSp& statContext(); - /// Return a reference to the modifiable list of queue handles. QueueHandleMap& queueHandles(); @@ -240,6 +337,31 @@ class ClusterNodeSession : public mqbi::DispatcherClient, /// node is the primary. void removeAllPartitions(); + // (virtual: mqbi::InlineClient) + mqbi::InlineResult::Enum + sendPush(const bmqt::MessageGUID& msgGUID, + int queueId, + const bsl::shared_ptr& message, + const mqbi::StorageMessageAttributes& attributes, + const bmqp::MessagePropertiesInfo& mps, + const bmqp::Protocol::SubQueueInfosArray& subQueueInfos) + BSLS_KEYWORD_OVERRIDE; + // Called by the 'queueId' to deliver the specified 'message' with the + // specified 'message', 'msgGUID', 'attributes' and 'mps' for the + // specified 'subQueueInfos' streams of the queue. + // + // THREAD: This method is called from the Queue's dispatcher thread. + + mqbi::InlineResult::Enum + sendAck(int queueId, + const bmqp::AckMessage& ackMessage) BSLS_KEYWORD_OVERRIDE; + // Called by the 'queueId' to send the specified 'ackMessage'. + // + // THREAD: This method is called from the Queue's dispatcher thread. + + GateKeeper& gatePut(); + GateKeeper& gateConfirm(); + // ACCESSORS /// Return a pointer to the dispatcher this client is associated with. @@ -265,7 +387,7 @@ class ClusterNodeSession : public mqbi::DispatcherClient, int peerInstanceId() const; /// Return the associated stat context. - const StatContextSp& statContext() const; + const bsl::shared_ptr& statContext() const; /// Return a reference to the non-modifiable list of partitions for /// which this cluster node is the primary. @@ -291,14 +413,132 @@ class ClusterNodeSession : public mqbi::DispatcherClient, // INLINE DEFINITIONS // ============================================================================ +// ---------------- +// class AtomicGate +// ---------------- + +inline AtomicGate::AtomicGate(bool isOpen) +: d_value(isOpen ? e_INIT : e_CLOSE) +{ + // NOTHING +} + +inline AtomicGate::~AtomicGate() +{ + BSLS_ASSERT_SAFE(d_value <= e_CLOSE); +} + +// MANIPULATORS +inline void AtomicGate::closeAndDrain() +{ + int result = d_value.add(e_CLOSE); + + BSLS_ASSERT_SAFE(result & e_CLOSE); + // Do not support more than one writer + + // Spin while locked result > e_CLOSE + + while (result > e_CLOSE) { + bslmt::ThreadUtil::yield(); + result = d_value; + } +} + +inline bool AtomicGate::tryEnter() +{ + const int result = d_value.add(e_ENTER); + + if (result & e_CLOSE) { + d_value.subtract(e_ENTER); + return false; // RETURN + } + else { + return true; // RETURN + } +} + +inline void AtomicGate::open() +{ + BSLA_MAYBE_UNUSED const int result = d_value.subtract(e_CLOSE); + + BSLS_ASSERT_SAFE(result >= e_INIT); + BSLS_ASSERT_SAFE((result & e_CLOSE) == 0); +} + +inline void AtomicGate::leave() +{ + BSLA_MAYBE_UNUSED const int result = d_value.subtract(e_ENTER); + + BSLS_ASSERT_SAFE(result >= e_INIT); +} + +// ------------------------ +// class GateKeeper::Status +// ------------------------ + +inline GateKeeper::Status::Status(GateKeeper& gateKeeper) +: d_gate(gateKeeper.d_gate) +, d_isOpen(d_gate.tryEnter()) +{ + // NOTHING +} + +inline GateKeeper::Status::~Status() +{ + if (d_isOpen) { + d_gate.leave(); + } +} + +inline bool GateKeeper::Status::isOpen() const +{ + return d_isOpen; +} + +// ---------------- +// class GateKeeper +// ---------------- + +inline GateKeeper::GateKeeper() +: d_gate(false) +, d_isOpen(false) +{ + // NOTHING +} + +inline void GateKeeper::open() +{ + if (!d_isOpen) { + d_isOpen = true; + d_gate.open(); + } +} + +inline void GateKeeper::close() +{ + if (d_isOpen) { + d_isOpen = false; + d_gate.closeAndDrain(); + } +} + // --------------------------------------- // struct ClientSessionState::SubQueueInfo // --------------------------------------- inline ClusterNodeSession::SubQueueInfo::SubQueueInfo() -: d_clientStats() +: d_clientStats_sp() { - d_clientStats.createInplace(); +} + +// ACCESSORS +inline void ClusterNodeSession::SubQueueInfo::onEvent( + mqbstat::QueueStatsClient::EventType::Enum type, + bsls::Types::Int64 value) const +{ + if (d_clientStats_sp) { + d_clientStats_sp->onEvent(type, value); + } } // ------------------------------------ @@ -340,9 +580,50 @@ inline mqbi::DispatcherClientData& ClusterNodeSession::dispatcherClientData() } inline void -ClusterNodeSession::setNodeStatus(bmqp_ctrlmsg::NodeStatus::Value value) +ClusterNodeSession::setNodeStatus(bmqp_ctrlmsg::NodeStatus::Value other, + bmqp_ctrlmsg::NodeStatus::Value self) { - d_nodeStatus = value; + // executed by the *DISPATCHER* thread + + d_nodeStatus = other; + + if ((other != bmqp_ctrlmsg::NodeStatus::E_AVAILABLE && + other != bmqp_ctrlmsg::NodeStatus::E_STOPPING) || + (self != bmqp_ctrlmsg::NodeStatus::E_AVAILABLE && + self != bmqp_ctrlmsg::NodeStatus::E_STOPPING)) { + d_gatePush.close(); + d_gateAck.close(); + d_gatePut.close(); + d_gateConfirm.close(); + } + else { + // Both are either E_AVAILABLE or E_STOPPING + d_gateAck.open(); + d_gateConfirm.open(); + + if (other == bmqp_ctrlmsg::NodeStatus::E_AVAILABLE) { + // Even if self is E_STOPPING, do allow PUTs. + // Otherwise, broadcast PUTs can be lost in the following scenario: + // This node sends StopRequest to a downstream while some broadcast + // PUTs "cross" the StopRequest. + + d_gatePut.open(); + if (self == bmqp_ctrlmsg::NodeStatus::E_AVAILABLE) { + d_gatePush.open(); + } + else { + // Do NOT process PUSH in the E_STOPPING state. + d_gatePush.close(); + } + } + else { + // But do NOT send to E_STOPPING upstream. + d_gatePut.close(); + + // Do NOT send PUSH to E_STOPPING upstream. + d_gatePush.close(); + } + } } inline void ClusterNodeSession::setPeerInstanceId(int value) @@ -369,11 +650,6 @@ ClusterNodeSession::onDispatcherEvent(const mqbi::DispatcherEvent& event) d_cluster_p->onDispatcherEvent(event); } -inline ClusterNodeSession::StatContextSp& ClusterNodeSession::statContext() -{ - return d_statContext_sp; -} - inline ClusterNodeSession::QueueHandleMap& ClusterNodeSession::queueHandles() { return d_queueHandles; @@ -385,6 +661,16 @@ inline mqbi::Dispatcher* ClusterNodeSession::dispatcher() return d_cluster_p->dispatcher(); } +inline GateKeeper& ClusterNodeSession::gatePut() +{ + return d_gatePut; +} + +inline GateKeeper& ClusterNodeSession::gateConfirm() +{ + return d_gateConfirm; +} + inline const mqbi::Dispatcher* ClusterNodeSession::dispatcher() const { return d_cluster_p->dispatcher(); @@ -429,7 +715,7 @@ inline int ClusterNodeSession::peerInstanceId() const return d_peerInstanceId; } -inline const ClusterNodeSession::StatContextSp& +inline const bsl::shared_ptr& ClusterNodeSession::statContext() const { return d_statContext_sp; diff --git a/src/groups/mqb/mqbc/mqbc_clusterstate.cpp b/src/groups/mqb/mqbc/mqbc_clusterstate.cpp index 2096587df5..9b084a2e25 100644 --- a/src/groups/mqb/mqbc/mqbc_clusterstate.cpp +++ b/src/groups/mqb/mqbc/mqbc_clusterstate.cpp @@ -302,7 +302,7 @@ ClusterState& ClusterState::unregisterObserver(ClusterStateObserver* observer) ClusterState& ClusterState::setPartitionPrimary(int partitionId, unsigned int leaseId, - mqbnet::ClusterNode* node) + ClusterNodeSession* ns) { // executed by the cluster *DISPATCHER* thread @@ -312,21 +312,25 @@ ClusterState& ClusterState::setPartitionPrimary(int partitionId, BSLS_ASSERT_SAFE(partitionId < static_cast(d_partitionsInfo.size())); ClusterStatePartitionInfo& pinfo = d_partitionsInfo[partitionId]; - mqbnet::ClusterNode* oldPrimary = pinfo.primaryNode(); + ClusterNodeSession* oldPrimary = pinfo.primaryNodeSession(); const unsigned int oldLeaseId = pinfo.primaryLeaseId(); BSLS_ASSERT_SAFE(leaseId >= oldLeaseId); - pinfo.setPrimaryNode(node); - if (node) { + mqbnet::ClusterNode* node = 0; + mqbnet::ClusterNode* oldPrimaryNode = oldPrimary + ? oldPrimary->clusterNode() + : 0; + if (ns) { + node = ns->clusterNode(); + BSLS_ASSERT_SAFE(node); pinfo.setPrimaryNodeId(node->nodeId()); } else { pinfo.setPrimaryNodeId(mqbnet::Cluster::k_INVALID_NODE_ID); } - pinfo.setPrimaryLeaseId(leaseId); - if (node == oldPrimary) { + if (ns == oldPrimary) { // We are being notified about the same primary. Check leaseId. Note // that leader can bump up just the leaseId while keeping the primary // node unchanged. @@ -339,21 +343,30 @@ ClusterState& ClusterState::setPartitionPrimary(int partitionId, } } + BALL_LOG_INFO << "Cluster [" << d_cluster_p->name() + << "]: closing the gate " << partitionId; + d_gatePrimary[partitionId].close(); + + pinfo.setPrimaryNodeSession(ns); + + pinfo.setPrimaryLeaseId(leaseId); + bmqp_ctrlmsg::PrimaryStatus::Value primaryStatus = bmqp_ctrlmsg::PrimaryStatus::E_UNDEFINED; - if (node) { + if (ns) { // By default, a new primary is PASSIVE. primaryStatus = bmqp_ctrlmsg::PrimaryStatus::E_PASSIVE; } pinfo.setPrimaryStatus(primaryStatus); - BALL_LOG_INFO << "Cluster [" << d_cluster_p->name() << "]: " - << "Setting primary of Partition [" << partitionId << "] to " - << "[" << (node ? node->nodeDescription() : "** NULL **") + BALL_LOG_INFO << "Cluster [" << d_cluster_p->name() + << "]: Setting primary of Partition [" << partitionId + << "] to [" + << (node ? node->nodeDescription() : "** NULL **") << "], leaseId: [" << leaseId << "], primaryStatus: [" << primaryStatus << "], oldPrimary: [" - << (oldPrimary ? oldPrimary->nodeDescription() - : "** NULL **") + << (oldPrimaryNode ? oldPrimaryNode->nodeDescription() + : "** NULL **") << "], oldLeaseId: [" << oldLeaseId << "]."; for (ObserversSetIter it = d_observers.begin(); it != d_observers.end(); @@ -362,7 +375,7 @@ ClusterState& ClusterState::setPartitionPrimary(int partitionId, node, leaseId, pinfo.primaryStatus(), - oldPrimary, + oldPrimaryNode, oldLeaseId); } @@ -381,9 +394,9 @@ ClusterState& ClusterState::setPartitionPrimaryStatus( BSLS_ASSERT_SAFE(partitionId < static_cast(d_partitionsInfo.size())); ClusterStatePartitionInfo& pinfo = d_partitionsInfo[partitionId]; - if (0 == pinfo.primaryNode()) { - BALL_LOG_ERROR << "Cluster [" << d_cluster_p->name() << "]: " - << "Failed to set the primary status of Partition [" + if (0 == pinfo.primaryNodeSession()) { + BALL_LOG_ERROR << "Cluster [" << d_cluster_p->name() + << "]: Failed to set the primary status of Partition [" << partitionId << "] to [" << value << "], reason: primary node is ** NULL **."; @@ -393,13 +406,17 @@ ClusterState& ClusterState::setPartitionPrimaryStatus( BSLS_ASSERT_SAFE(bmqp_ctrlmsg::PrimaryStatus::E_UNDEFINED != pinfo.primaryStatus()); + mqbnet::ClusterNode* node = pinfo.primaryNode(); + + BSLS_ASSERT_SAFE(node); + bmqp_ctrlmsg::PrimaryStatus::Value oldStatus = pinfo.primaryStatus(); pinfo.setPrimaryStatus(value); - BALL_LOG_INFO << "Cluster [" << d_cluster_p->name() << "]: " - << "Setting status of primary [" - << pinfo.primaryNode()->nodeDescription() - << "] of Partition [" << partitionId << "] to [" << value + BALL_LOG_INFO << "Cluster [" << d_cluster_p->name() + << "]: Setting status of primary [" + << node->nodeDescription() << "] of Partition [" + << partitionId << "] to [" << value << "], oldPrimaryStatus: [" << oldStatus << "], leaseId: [" << pinfo.primaryLeaseId() << "]."; @@ -411,14 +428,29 @@ ClusterState& ClusterState::setPartitionPrimaryStatus( it != d_observers.end(); ++it) { (*it)->onPartitionPrimaryAssignment(partitionId, - pinfo.primaryNode(), + node, pinfo.primaryLeaseId(), value, - pinfo.primaryNode(), + node, pinfo.primaryLeaseId()); } } + // TODO: this code assumes that it is safe to send PUTs and CONFIRMS at + // this point if the status is E_ACTIVE or close the gate otherwise. + // May need to open the gate later/close earlier by a separate call. + + if (bmqp_ctrlmsg::PrimaryStatus::E_ACTIVE == value) { + BALL_LOG_INFO << "Cluster [" << d_cluster_p->name() + << "]: opening the gate " << partitionId; + d_gatePrimary[partitionId].open(); + } + else { + BALL_LOG_INFO << "Cluster [" << d_cluster_p->name() + << "]: closing the gate " << partitionId; + d_gatePrimary[partitionId].close(); + } + return *this; } diff --git a/src/groups/mqb/mqbc/mqbc_clusterstate.h b/src/groups/mqb/mqbc/mqbc_clusterstate.h index 63776dd9fe..835dcdeec9 100644 --- a/src/groups/mqb/mqbc/mqbc_clusterstate.h +++ b/src/groups/mqb/mqbc/mqbc_clusterstate.h @@ -26,6 +26,7 @@ /// observers, implementing the @bbref{mqbblp::ClusterStateObserver} interface. // MQB +#include #include #include #include @@ -102,7 +103,7 @@ class ClusterStatePartitionInfo { /// Pointer to primary node for the partition; null if no primary /// associated. - mqbnet::ClusterNode* d_primaryNode_p; + ClusterNodeSession* d_primaryNodeSession_p; // Status of the primary. bmqp_ctrlmsg::PrimaryStatus::Value d_primaryStatus; @@ -119,7 +120,8 @@ class ClusterStatePartitionInfo { ClusterStatePartitionInfo& setPrimaryNodeId(int value); ClusterStatePartitionInfo& setNumQueuesMapped(int value); ClusterStatePartitionInfo& setNumActiveQueues(int value); - ClusterStatePartitionInfo& setPrimaryNode(mqbnet::ClusterNode* value); + ClusterStatePartitionInfo& + setPrimaryNodeSession(ClusterNodeSession* value); /// Set the corresponding member to the specified `value` and return a /// reference offering modifiable access to this object. @@ -132,6 +134,7 @@ class ClusterStatePartitionInfo { int primaryNodeId() const; int numQueuesMapped() const; int numActiveQueues() const; + ClusterNodeSession* primaryNodeSession() const; mqbnet::ClusterNode* primaryNode() const; /// Return the value of the corresponding member of this object. @@ -555,7 +558,8 @@ class ClusterState { /// Observers of the cluster state. ObserversSet d_observers; - /// Regexp wrapper used to get partition Id. + bsl::vector d_gatePrimary; + PartitionIdExtractor d_partitionIdExtractor; /// TODO (FSM); remove after switching to FSM @@ -612,15 +616,15 @@ class ClusterState { // ----------------- /// Update the status of the specified `partitionId`, to indicate that - /// the specified `node` is the primary, with the specified `leaseId`. - /// If `node` is a null pointer, this means the partition has no + /// the node of the specified `ns` is the primary, with the specified + /// `leaseId`. If `ns` is a null pointer, this means the partition has no /// primary. This will notify all active observers by invoking /// `onPartitionPrimaryAssignment()` on each of them, with the /// `partitionId` and `node` as parameters. The bahavior is undefined /// unless `partitionId >= 0` and `partitionId < partitionsCount`. - ClusterState& setPartitionPrimary(int partitionId, - unsigned int leaseId, - mqbnet::ClusterNode* node); + ClusterState& setPartitionPrimary(int partitionId, + unsigned int leaseId, + ClusterNodeSession* ns); /// Set the status of the primary of the specified `partitionId` to the /// specified `value`. @@ -679,6 +683,8 @@ class ClusterState { /// Clear this cluster state object, without firing any observers. void clear(); + GateKeeper& gatePrimary(int partitionId); + /// TODO (FSM); remove after switching to FSM bool cacheDoubleAssignment(const bmqt::Uri& uri, int partitionId); @@ -768,7 +774,7 @@ inline ClusterStatePartitionInfo::ClusterStatePartitionInfo() , d_primaryNodeId(mqbnet::Cluster::k_INVALID_NODE_ID) , d_numQueuesMapped(0) , d_numActiveQueues(0) -, d_primaryNode_p(0) +, d_primaryNodeSession_p(0) , d_primaryStatus(bmqp_ctrlmsg::PrimaryStatus::E_UNDEFINED) { // NOTHING @@ -811,9 +817,9 @@ ClusterStatePartitionInfo::setNumActiveQueues(int value) } inline ClusterStatePartitionInfo& -ClusterStatePartitionInfo::setPrimaryNode(mqbnet::ClusterNode* value) +ClusterStatePartitionInfo::setPrimaryNodeSession(ClusterNodeSession* value) { - d_primaryNode_p = value; + d_primaryNodeSession_p = value; return *this; } @@ -852,7 +858,13 @@ inline int ClusterStatePartitionInfo::numActiveQueues() const inline mqbnet::ClusterNode* ClusterStatePartitionInfo::primaryNode() const { - return d_primaryNode_p; + return d_primaryNodeSession_p ? d_primaryNodeSession_p->clusterNode() : 0; +} + +inline ClusterNodeSession* +ClusterStatePartitionInfo::primaryNodeSession() const +{ + return d_primaryNodeSession_p; } inline bmqp_ctrlmsg::PrimaryStatus::Value @@ -995,6 +1007,7 @@ inline ClusterState::ClusterState(mqbi::Cluster* cluster, , d_domainStates(allocator) , d_queueKeys(allocator) , d_observers(allocator) +, d_gatePrimary(partitionsCount, allocator) , d_partitionIdExtractor(allocator) , d_doubleAssignments(allocator) { @@ -1007,6 +1020,7 @@ inline ClusterState::ClusterState(mqbi::Cluster* cluster, d_partitionsInfo.resize(partitionsCount); for (int i = 0; i < partitionsCount; ++i) { d_partitionsInfo[i].setPartitionId(i); + d_gatePrimary[i].close(); } } @@ -1026,6 +1040,12 @@ inline ClusterState::QueueKeys& ClusterState::queueKeys() return d_queueKeys; } +inline GateKeeper& ClusterState::gatePrimary(int partitionId) +{ + // This assumes thread-safe access to d_gatePrimary vector. + return d_gatePrimary[partitionId]; +} + // ACCESSORS inline const mqbi::Cluster* ClusterState::cluster() const { @@ -1071,9 +1091,14 @@ inline bool ClusterState::isSelfPrimary(int partitionId) const const ClusterStatePartitionInfo& partitionInfo = partition(partitionId); - return (partitionInfo.primaryNode() && - (partitionInfo.primaryNode()->nodeId() == - cluster()->netCluster().selfNodeId())); + if (partitionInfo.primaryNodeSession()) { + const mqbnet::ClusterNode* node = + partitionInfo.primaryNodeSession()->clusterNode(); + BSLS_ASSERT_SAFE(node); + return node->nodeId() == cluster()->netCluster().selfNodeId(); + } + + return false; } inline bool ClusterState::isSelfActivePrimary(int partitionId) const @@ -1133,7 +1158,7 @@ inline bool ClusterState::isSelfActivePrimary() const inline bool ClusterState::hasActivePrimary(int partitionId) const { - return 0 != partition(partitionId).primaryNode() && + return 0 != partition(partitionId).primaryNodeSession() && bmqp_ctrlmsg::PrimaryStatus::E_ACTIVE == partition(partitionId).primaryStatus(); } diff --git a/src/groups/mqb/mqbc/mqbc_clusterstatemanager.cpp b/src/groups/mqb/mqbc/mqbc_clusterstatemanager.cpp index 40c41940fd..1e55e446a1 100644 --- a/src/groups/mqb/mqbc/mqbc_clusterstatemanager.cpp +++ b/src/groups/mqb/mqbc/mqbc_clusterstatemanager.cpp @@ -1416,7 +1416,12 @@ void ClusterStateManager::setPrimary(int partitionId, d_state_p->partitions().size()); BSLS_ASSERT_SAFE(primary); - d_state_p->setPartitionPrimary(partitionId, leaseId, primary); + ClusterNodeSession* ns = + d_clusterData_p->membership().getClusterNodeSession(primary); + + BSLS_ASSERT_SAFE(ns); + + d_state_p->setPartitionPrimary(partitionId, leaseId, ns); } void ClusterStateManager::setPrimaryStatus( diff --git a/src/groups/mqb/mqbc/mqbc_clusterutil.cpp b/src/groups/mqb/mqbc/mqbc_clusterutil.cpp index b37df743a2..e8665fd869 100644 --- a/src/groups/mqb/mqbc/mqbc_clusterutil.cpp +++ b/src/groups/mqb/mqbc/mqbc_clusterutil.cpp @@ -89,9 +89,13 @@ void applyPartitionPrimary( mqbnet::ClusterNode* proposedPrimaryNode = clusterData.membership().netCluster()->lookupNode( info.primaryNodeId()); + ClusterNodeSession* ns = + clusterData.membership().getClusterNodeSession( + proposedPrimaryNode); + clusterState->setPartitionPrimary(info.partitionId(), info.primaryLeaseId(), - proposedPrimaryNode); + ns); } } diff --git a/src/groups/mqb/mqbc/mqbc_clusterutil.h b/src/groups/mqb/mqbc/mqbc_clusterutil.h index 2201be4c7e..930dd4f3a5 100644 --- a/src/groups/mqb/mqbc/mqbc_clusterutil.h +++ b/src/groups/mqb/mqbc/mqbc_clusterutil.h @@ -113,16 +113,6 @@ struct ClusterUtil { public: // FUNCTIONS - /// Generate a nack with the specified `status` for a PUT message having - /// the specified `putHeader` from the specified `source`. The nack is - /// replied to the `source`. - static void generateNack(bmqt::AckResult::Enum status, - const bmqp::PutHeader& putHeader, - mqbi::DispatcherClient* source, - mqbi::Dispatcher* dispatcher, - const bsl::shared_ptr& appData, - const bsl::shared_ptr& options); - /// Return true if the specified `syncPoint` is valid, false otherwise. static bool isValid(const bmqp_ctrlmsg::SyncPoint& syncPoint); @@ -376,34 +366,6 @@ struct ClusterUtil { // ------------------ // struct ClusterUtil // ------------------ -inline void -ClusterUtil::generateNack(bmqt::AckResult::Enum status, - const bmqp::PutHeader& putHeader, - mqbi::DispatcherClient* source, - mqbi::Dispatcher* dispatcher, - const bsl::shared_ptr& appData, - const bsl::shared_ptr& options) -{ - BSLS_ASSERT_SAFE(status != bmqt::AckResult::e_SUCCESS); - - bmqp::AckMessage ackMessage(bmqp::ProtocolUtil::ackResultToCode(status), - putHeader.correlationId(), - putHeader.messageGUID(), - putHeader.queueId()); - - mqbi::DispatcherEvent* ev = dispatcher->getEvent(source); - (*ev).setType(mqbi::DispatcherEventType::e_ACK).setAckMessage(ackMessage); - - if (appData) { - (*ev).setBlob(appData); - (*ev).setOptions(options); - } - else { - BSLS_ASSERT_SAFE(!options); - } - - dispatcher->dispatchEvent(ev, source); -} inline bool ClusterUtil::isValid(const bmqp_ctrlmsg::SyncPoint& syncPoint) { diff --git a/src/groups/mqb/mqbc/mqbc_incoreclusterstateledger.t.cpp b/src/groups/mqb/mqbc/mqbc_incoreclusterstateledger.t.cpp index a9dba1b897..d503b49f49 100644 --- a/src/groups/mqb/mqbc/mqbc_incoreclusterstateledger.t.cpp +++ b/src/groups/mqb/mqbc/mqbc_incoreclusterstateledger.t.cpp @@ -277,7 +277,11 @@ struct Tester { for (mqbnet::Cluster::NodesList::iterator iter = nodes.begin(); iter != nodes.end(); ++iter) { - d_cluster_mp->_state()->setPartitionPrimary(pid, 1, *iter); + mqbc::ClusterNodeSession* ns = d_cluster_mp->_clusterData() + ->membership() + .getClusterNodeSession(*iter); + BSLS_ASSERT_OPT(ns); + d_cluster_mp->_state()->setPartitionPrimary(pid, 1, ns); ++pid; } diff --git a/src/groups/mqb/mqbc/mqbc_storagemanager.t.cpp b/src/groups/mqb/mqbc/mqbc_storagemanager.t.cpp index eefac42cf9..a6c2f1d829 100644 --- a/src/groups/mqb/mqbc/mqbc_storagemanager.t.cpp +++ b/src/groups/mqb/mqbc/mqbc_storagemanager.t.cpp @@ -203,9 +203,11 @@ struct TestHelper { unsigned int leaseId, mqbnet::ClusterNode* node) { - d_cluster_mp->_state()->setPartitionPrimary(partitionId, - leaseId, - node); + d_cluster_mp->_state()->setPartitionPrimary( + partitionId, + leaseId, + d_cluster_mp->_clusterData()->membership().getClusterNodeSession( + node)); storageManager->setPrimaryForPartition(partitionId, node, leaseId); } diff --git a/src/groups/mqb/mqbi/mqbi_cluster.h b/src/groups/mqb/mqbi/mqbi_cluster.h index 6903a22879..0f61dd2b16 100644 --- a/src/groups/mqb/mqbi/mqbi_cluster.h +++ b/src/groups/mqb/mqbi/mqbi_cluster.h @@ -206,18 +206,6 @@ class Cluster : public DispatcherClient { public: // TYPES - /// Type of a `cookie` provided in the `OpenQueueCallback` to confirm - /// processing of the `openQueue` response by the requester. Opening a - /// queue is fully async, and it could happen that the requester went - /// down before the `openQueue` response got delivered to it. In this - /// case, we must rollback upstream state. This cookie is used for - /// that: it is initialized to zero (in the `Cluster` implementation), - /// and carried over to the original requester of the `openQueue`. If - /// the requester is not able to process the openQueue response, it - /// needs to set this cookie to the queue handle which it received, so - /// that the operation can be rolled back. - typedef bsl::shared_ptr OpenQueueConfirmationCookie; - /// Signature of the callback passed to the `openQueue()` method: if the /// specified `status` is SUCCESS, the operation was a success and the /// specified `queueHandle` contains the queue handle, and the specified @@ -232,7 +220,7 @@ class Cluster : public DispatcherClient { const bmqp_ctrlmsg::Status& status, QueueHandle* queueHandle, const bmqp_ctrlmsg::OpenQueueResponse& openQueueResponse, - const OpenQueueConfirmationCookie& confirmationCookie)> + const OpenQueueConfirmationCookieSp& confirmationCookie)> OpenQueueCallback; // TYPES @@ -364,6 +352,25 @@ class Cluster : public DispatcherClient { /// Load the cluster state to the specified `out` object. virtual void loadClusterStatus(mqbcmd::ClusterResult* out) = 0; + /// Send the specified CONFIRM 'message' for the specified 'partitionId' + /// without switching thread context. + /// 'onRelayConfirmEvent' replacement. + virtual mqbi::InlineResult::Enum + sendConfirmInline(int partitionId, + const bmqp::ConfirmMessage& message) = 0; + + /// Send PUT message for the specified 'partitionId' using the specified + /// 'putHeader', 'appData', 'options', 'state', 'genCount' without + /// switching thread context. + /// 'onRelayPutEvent' replacement. + virtual mqbi::InlineResult::Enum + sendPutInline(int partitionId, + const bmqp::PutHeader& putHeader, + const bsl::shared_ptr& appData, + const bsl::shared_ptr& options, + const bsl::shared_ptr& state, + bsls::Types::Uint64 genCount) = 0; + /// Purge and force GC queues in this cluster on a given domain. virtual void purgeAndGCQueueOnDomain(mqbcmd::ClusterResult* result, const bsl::string& domainName) = 0; diff --git a/src/groups/mqb/mqbi/mqbi_domain.h b/src/groups/mqb/mqbi/mqbi_domain.h index 897623ffdb..a6a110918e 100644 --- a/src/groups/mqb/mqbi/mqbi_domain.h +++ b/src/groups/mqb/mqbi/mqbi_domain.h @@ -82,6 +82,9 @@ class DispatcherClient; class Queue; class QueueHandle; class QueueHandleRequesterContext; +struct OpenQueueContext; + +typedef bsl::shared_ptr OpenQueueConfirmationCookieSp; // ============ // class Domain @@ -96,9 +99,6 @@ class Domain { /// domain with the specified `domainName` has stopped. typedef bsl::function TeardownCb; - /// Refer to the similar typedef in `mqbi::Cluster` for description. - typedef bsl::shared_ptr OpenQueueConfirmationCookie; - /// Signature of the callback function for the `openQueue` method. If /// the specified `status` is SUCCESS, then the specified `handle` is /// populated with the resulting queue handle to use by the callee; and @@ -111,10 +111,10 @@ class Domain { /// requester (see meaning in the /// `mqbi::Cluster::OpenQueueConfirmationCookie` typedef). typedef bsl::function + const bmqp_ctrlmsg::Status& status, + QueueHandle* handle, + const bmqp_ctrlmsg::OpenQueueResponse& openQueueResponse, + const mqbi::OpenQueueConfirmationCookieSp& confirmationCookie)> OpenQueueCallback; public: diff --git a/src/groups/mqb/mqbi/mqbi_queue.cpp b/src/groups/mqb/mqbi/mqbi_queue.cpp index a7c60b3c22..04bcf48e71 100644 --- a/src/groups/mqb/mqbi/mqbi_queue.cpp +++ b/src/groups/mqb/mqbi/mqbi_queue.cpp @@ -48,6 +48,15 @@ bsl::ostream& operator<<(bsl::ostream& stream, // starts from zero. bsls::AtomicInt64 QueueHandleRequesterContext::s_previousRequesterId(-1); +// ------------------ +// class InlineClient +// ------------------ + +InlineClient::~InlineClient() +{ + // NOTHING +} + // -------------------------- // class QueueHandleRequester // -------------------------- diff --git a/src/groups/mqb/mqbi/mqbi_queue.h b/src/groups/mqb/mqbi/mqbi_queue.h index 5206d1fd45..ec8984055e 100644 --- a/src/groups/mqb/mqbi/mqbi_queue.h +++ b/src/groups/mqb/mqbi/mqbi_queue.h @@ -86,6 +86,9 @@ namespace BloombergLP { namespace bmqt { class Uri; } +namespace bmqst { +class StatContext; +} namespace mqbcmd { class QueueCommand; } @@ -107,6 +110,9 @@ class ResourceUsageMonitor; namespace mqbstat { class QueueStatsDomain; } +namespace mqbstat { +class QueueStatsClient; +} namespace mqbi { @@ -116,6 +122,57 @@ class Domain; class Queue; class QueueEngine; +// =================== +// struct InlineResult +// =================== +struct InlineResult { + enum Enum { + e_SUCCESS = 0, + e_UNAVAILABLE = 1, + e_INVALID_PRIMARY = 2, + e_INVALID_GEN_COUNT = 3, + e_CHANNEL_ERROR = 4, + e_INVALID_PARTITION = 5, + e_SELF_PRIMARY = 6 + }; + // CLASS METHODS + static const char* toAscii(InlineResult::Enum value); + static bool isPermanentError(InlineResult::Enum value); + static bmqt::AckResult::Enum toAckResult(InlineResult::Enum value); +}; + +// ================== +// class InlineClient +// ================== +class InlineClient { + // Interface for PUSH and ACK. + + public: + // CREATORS + virtual ~InlineClient(); + + virtual mqbi::InlineResult::Enum + sendPush(const bmqt::MessageGUID& msgGUID, + int queueId, + const bsl::shared_ptr& message, + const mqbi::StorageMessageAttributes& attributes, + const bmqp::MessagePropertiesInfo& mps, + const bmqp::Protocol::SubQueueInfosArray& subQueues) = 0; + // Called by the 'queueId' to deliver the specified 'message' with the + // specified 'message', 'msgGUID', 'attributes' and 'mps' for the + // specified 'subQueues' streams of the queue. + // Return 'InlineResult::Enum'. + // + // THREAD: This method is called from the Queue's dispatcher thread. + + virtual mqbi::InlineResult::Enum + sendAck(int queueId, const bmqp::AckMessage& ackMessage) = 0; + // Called by the 'Queue' to send the specified 'ackMessage'. + // Return 'InlineResult::Enum'. + // + // THREAD: This method is called from the Queue's dispatcher thread. +}; + // ================================= // class QueueHandleRequesterContext // ================================= @@ -170,6 +227,18 @@ class QueueHandleRequesterContext { // Unique ID associated with the requester // of a queue handle. + bsl::shared_ptr d_statContext_sp; + + InlineClient* d_inlineClient_p; + + // NOT IMPLEMENTED + QueueHandleRequesterContext(const QueueHandleRequesterContext&) + BSLS_CPP11_DELETED; + + /// Copy constructor and assignment operator are not implemented. + QueueHandleRequesterContext& + operator=(const QueueHandleRequesterContext&) BSLS_CPP11_DELETED; + public: // TRAITS BSLMF_NESTED_TRAIT_DECLARATION(QueueHandleRequesterContext, @@ -186,12 +255,6 @@ class QueueHandleRequesterContext { /// Default constructor explicit QueueHandleRequesterContext(bslma::Allocator* allocator = 0); - /// Create a `QueueHandleRequesterContext` object having the same value - /// as the specified `original` object, and using the specified - /// `allocator`. - QueueHandleRequesterContext(const QueueHandleRequesterContext& original, - bslma::Allocator* allocator); - // MANIPULATORS QueueHandleRequesterContext& setClient(DispatcherClient* value); QueueHandleRequesterContext& @@ -202,6 +265,10 @@ class QueueHandleRequesterContext { /// Set the corresponding data member to the specified `value` and /// return a reference offering modifiable access to this object. QueueHandleRequesterContext& setRequesterId(RequesterId value); + QueueHandleRequesterContext& + setStatContext(const bsl::shared_ptr& value); + + QueueHandleRequesterContext& setInlineClient(InlineClient* inlineClient); // ACCESSORS DispatcherClient* client() const; @@ -212,6 +279,10 @@ class QueueHandleRequesterContext { /// Return the corresponding data member's value. RequesterId requesterId() const; + const bsl::shared_ptr& statContext() const; + + InlineClient* inlineClient() const; + /// Return true if the requester node is first hop after the client (or /// last hop before the client). bool isFirstHop() const; @@ -237,6 +308,30 @@ class QueueHandleRequester { handleRequesterContext() const = 0; }; +// ======================= +// struct OpenQueueContext +// ======================= + +struct OpenQueueContext { + // Type of a 'cookie' provided in the 'OpenQueueCallback' to confirm + // processing of the 'openQueue' response by the requester. Opening a + // queue is fully async, and it could happen that the requester went + // down before the 'openQueue' response got delivered to it. In this + // case, we must rollback upstream state. This cookie is used for + // that: it is initialized to zero (in the 'Cluster' implementation), + // and carried over to the original requester of the 'openQueue'. If + // the requester is not able to process the openQueue response, it + // needs to set this cookie to the queue handle which it received, so + // that the operation can be rolled back. + + QueueHandle* d_handle; + bsl::shared_ptr d_stats_sp; + + OpenQueueContext(); +}; + +typedef bsl::shared_ptr OpenQueueConfirmationCookieSp; + // ================== // struct QueueCounts // ================== @@ -447,6 +542,8 @@ class QueueHandle { unsigned int d_upstreamSubQueueId; bmqp_ctrlmsg::StreamParameters d_streamParameters; + bsl::shared_ptr d_clientStats_sp; + StreamInfo(const QueueCounts& counts, unsigned int downstreamSubQueueId, unsigned int upstreamSubQueueId, @@ -476,9 +573,10 @@ class QueueHandle { /// the specified `counts`. Create new context for the `subStreamInfo` /// if it is not registered. Associate the subStream with the specified /// `upstreamSubQueueId`. + /// Return an iterator pointing to the context. /// /// THREAD: This method is called from the Queue's dispatcher thread. - virtual void + virtual SubStreams::const_iterator registerSubStream(const bmqp_ctrlmsg::SubQueueIdInfo& subStreamInfo, unsigned int upstreamSubQueueId, const mqbi::QueueCounts& counts) = 0; @@ -742,9 +840,11 @@ class Queue : public DispatcherClient { /// Obtain a handle to this queue, for the client represented by the /// specified `clientContext` and using the specified `handleParameters` - /// and `upstreamSubQueueId`. Invoke the specified `callback` with the - /// result. + /// and `upstreamSubQueueId`. Load a reference to the corresponding + /// `mqbstat::QueueStatsClient` into the specified `context`. + /// Invoke the specified `callback` with the result. virtual void getHandle( + const mqbi::OpenQueueConfirmationCookieSp& context, const bsl::shared_ptr& clientContext, const bmqp_ctrlmsg::QueueHandleParameters& handleParameters, unsigned int upstreamSubQueueId, @@ -979,6 +1079,63 @@ class QueueHandleFactory { // INLINE DEFINITIONS // ============================================================================ +inline const char* InlineResult::toAscii(InlineResult::Enum value) +{ +#define CASE(X) \ + case e_##X: return #X; + + switch (value) { + CASE(SUCCESS) + CASE(UNAVAILABLE) + CASE(CHANNEL_ERROR) + CASE(INVALID_PARTITION) + CASE(INVALID_PRIMARY) + CASE(INVALID_GEN_COUNT) + CASE(SELF_PRIMARY) + default: return "(* UNKNOWN *)"; + } +#undef CASE +} + +inline bool InlineResult::isPermanentError(InlineResult::Enum value) +{ + if (value == InlineResult::Enum::e_INVALID_PARTITION || + value == InlineResult::Enum::e_SELF_PRIMARY) { + return true; // RETURN + } + else { + return false; // RETURN + } +} + +inline bmqt::AckResult::Enum +InlineResult::toAckResult(InlineResult::Enum value) +{ + bmqt::AckResult::Enum ackResult = bmqt::AckResult::e_SUCCESS; + + if (value == InlineResult::Enum::e_SUCCESS) { + ackResult = bmqt::AckResult::e_SUCCESS; + } + else if (value == InlineResult::Enum::e_INVALID_PARTITION) { + ackResult = bmqt::AckResult::e_INVALID_ARGUMENT; + } + else { + ackResult = bmqt::AckResult::e_UNKNOWN; + } + return ackResult; +} + +// ----------------------- +// struct OpenQueueContext +// ----------------------- + +inline OpenQueueContext::OpenQueueContext() +: d_handle() +, d_stats_sp() +{ + // NOTHING +} + // ------------------ // struct QueueCounts // ------------------ @@ -1119,18 +1276,8 @@ inline QueueHandleRequesterContext::QueueHandleRequesterContext( , d_description(allocator) , d_isClusterMember(false) , d_requesterId(k_INVALID_REQUESTER_ID) -{ - // NOTHING -} - -inline QueueHandleRequesterContext::QueueHandleRequesterContext( - const QueueHandleRequesterContext& original, - bslma::Allocator* allocator) -: d_client_p(original.d_client_p) -, d_identity(original.d_identity, allocator) -, d_description(original.d_description, allocator) -, d_isClusterMember(original.d_isClusterMember) -, d_requesterId(original.d_requesterId) +, d_statContext_sp() +, d_inlineClient_p(0) { // NOTHING } @@ -1170,6 +1317,23 @@ QueueHandleRequesterContext::setRequesterId(RequesterId value) return *this; } +inline QueueHandleRequesterContext& +QueueHandleRequesterContext::setStatContext( + const bsl::shared_ptr& value) +{ + d_statContext_sp = value; + return *this; +} + +inline QueueHandleRequesterContext& +QueueHandleRequesterContext::setInlineClient(InlineClient* inlineClient) +{ + BSLS_ASSERT_SAFE(inlineClient); + + d_inlineClient_p = inlineClient; + return *this; +} + inline DispatcherClient* QueueHandleRequesterContext::client() const { return d_client_p; @@ -1202,6 +1366,17 @@ QueueHandleRequesterContext::requesterId() const return d_requesterId; } +inline const bsl::shared_ptr& +QueueHandleRequesterContext::statContext() const +{ + return d_statContext_sp; +} + +inline InlineClient* QueueHandleRequesterContext::inlineClient() const +{ + return d_inlineClient_p; +} + } // 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 990a831925..fa3bfcf852 100644 --- a/src/groups/mqb/mqbi/mqbi_queueengine.h +++ b/src/groups/mqb/mqbi/mqbi_queueengine.h @@ -95,11 +95,13 @@ class QueueEngine { /// Obtain and return a handle to this queue for the client identified /// with the specified `clientContext`, using the specified /// `handleParameters` and `upstreamSubQueueId`, and invoke the - /// specified `callback` when finished. In case of error, return a null - /// pointer. + /// specified `callback` when finished. Load a reference to the + /// corresponding `mqbstat::QueueStatsClient` into the specified `context`. + /// In case of error, return a null pointer. /// /// THREAD: This method is called from the Queue's dispatcher thread. virtual QueueHandle* getHandle( + const mqbi::OpenQueueConfirmationCookieSp& context, const bsl::shared_ptr& clientContext, const bmqp_ctrlmsg::QueueHandleParameters& handleParameters, unsigned int upstreamSubQueueId, diff --git a/src/groups/mqb/mqbmock/mqbmock_cluster.cpp b/src/groups/mqb/mqbmock/mqbmock_cluster.cpp index 9c3ca06457..c857b062ef 100644 --- a/src/groups/mqb/mqbmock/mqbmock_cluster.cpp +++ b/src/groups/mqb/mqbmock/mqbmock_cluster.cpp @@ -177,19 +177,24 @@ void Cluster::_initializeNodeSessions() NodesListIter nodeIter = clusterMembership.netCluster()->nodes().begin(); NodesListIter endIter = clusterMembership.netCluster()->nodes().end(); for (; nodeIter != endIter; ++nodeIter) { + // Create stat context for each cluster node + bmqst::StatContextConfiguration config((*nodeIter)->hostName()); + + StatContextSp statContextSp( + d_clusterData_mp->clusterNodesStatContext()->addSubcontext(config), + d_allocator_p); + mqbc::ClusterMembership::ClusterNodeSessionSp nodeSessionSp; nodeSessionSp.createInplace(d_allocator_p, this, *nodeIter, d_clusterData_mp->identity().name(), d_clusterData_mp->identity().identity(), + statContextSp, d_allocator_p); - nodeSessionSp->setNodeStatus(bmqp_ctrlmsg::NodeStatus::E_AVAILABLE); - // Create stat context for each cluster node - bmqst::StatContextConfiguration config((*nodeIter)->hostName()); - nodeSessionSp->statContext() = - d_clusterData_mp->clusterNodesStatContext()->addSubcontext(config); + nodeSessionSp->setNodeStatus(bmqp_ctrlmsg::NodeStatus::E_AVAILABLE, + bmqp_ctrlmsg::NodeStatus::E_AVAILABLE); clusterMembership.clusterNodeSessionMap().insert( bsl::make_pair(*nodeIter, nodeSessionSp)); @@ -244,6 +249,7 @@ Cluster::Cluster(bslma::Allocator* allocator, , d_isLeader(isLeader) , d_isRestoringState(false) , d_processor() +, d_putFunctor(bsl::allocator_arg, d_allocator_p) , d_resources(&d_scheduler, &d_bufferFactory, &d_blobSpPool) { // PRECONDITIONS @@ -473,6 +479,31 @@ void Cluster::loadClusterStatus(mqbcmd::ClusterResult* out) out->makeClusterStatus(); } +mqbi::InlineResult::Enum +Cluster::sendConfirmInline(BSLA_UNUSED int partitionId, + BSLA_UNUSED const bmqp::ConfirmMessage& message) +{ + return mqbi::InlineResult::e_UNAVAILABLE; +} + +mqbi::InlineResult::Enum +Cluster::sendPutInline(int partitionId, + const bmqp::PutHeader& putHeader, + const bsl::shared_ptr& appData, + const bsl::shared_ptr& options, + const bsl::shared_ptr& state, + bsls::Types::Uint64 genCount) +{ + BSLS_ASSERT_SAFE(d_putFunctor); + + return d_putFunctor(partitionId, + putHeader, + appData, + options, + state, + genCount); +} + void Cluster::purgeAndGCQueueOnDomain( mqbcmd::ClusterResult* result, BSLA_UNUSED const bsl::string& domainName) diff --git a/src/groups/mqb/mqbmock/mqbmock_cluster.h b/src/groups/mqb/mqbmock/mqbmock_cluster.h index dc42f09a7e..61dbfe5780 100644 --- a/src/groups/mqb/mqbmock/mqbmock_cluster.h +++ b/src/groups/mqb/mqbmock/mqbmock_cluster.h @@ -147,6 +147,15 @@ class Cluster : public mqbi::Cluster { typedef TestChannelMap::iterator TestChannelMapIter; typedef TestChannelMap::const_iterator TestChannelMapCIter; + typedef bsl::function& appData, + const bsl::shared_ptr& options, + const bsl::shared_ptr& state, + bsls::Types::Uint64 genCount)> + PutFunctor; + public: // CONSTANTS @@ -218,6 +227,8 @@ class Cluster : public mqbi::Cluster { EventProcessor d_processor; + PutFunctor d_putFunctor; + mqbi::ClusterResources d_resources; private: @@ -280,6 +291,7 @@ class Cluster : public mqbi::Cluster { mqbi::Dispatcher* dispatcher() BSLS_KEYWORD_OVERRIDE; void _setEventProcessor(const EventProcessor& processor); + void _setPutFunctor(const PutFunctor& f); /// Return a reference to the dispatcherClientData. mqbi::DispatcherClientData& dispatcherClientData() BSLS_KEYWORD_OVERRIDE; @@ -401,6 +413,18 @@ class Cluster : public mqbi::Cluster { const bsl::string& domainName) BSLS_KEYWORD_OVERRIDE; + mqbi::InlineResult::Enum sendConfirmInline( + int partitionId, + const bmqp::ConfirmMessage& message) BSLS_KEYWORD_OVERRIDE; + + mqbi::InlineResult::Enum + sendPutInline(int partitionId, + const bmqp::PutHeader& putHeader, + const bsl::shared_ptr& appData, + const bsl::shared_ptr& options, + const bsl::shared_ptr& state, + bsls::Types::Uint64 genCount) BSLS_KEYWORD_OVERRIDE; + // MANIPULATORS // (specific to mqbmock::Cluster) @@ -570,6 +594,11 @@ inline void Cluster::_setEventProcessor(const EventProcessor& processor) d_processor = processor; } +inline void Cluster::_setPutFunctor(const PutFunctor& f) +{ + d_putFunctor = f; +} + inline Cluster::BlobSpPool* Cluster::_blobSpPool() { return &d_blobSpPool; diff --git a/src/groups/mqb/mqbmock/mqbmock_queue.cpp b/src/groups/mqb/mqbmock/mqbmock_queue.cpp index a80c407c50..d4ee293d0f 100644 --- a/src/groups/mqb/mqbmock/mqbmock_queue.cpp +++ b/src/groups/mqb/mqbmock/mqbmock_queue.cpp @@ -112,6 +112,7 @@ int Queue::configure(BSLA_UNUSED bsl::ostream* errorDescription_p, } void Queue::getHandle( + const mqbi::OpenQueueConfirmationCookieSp& context, const bsl::shared_ptr& clientContext, const bmqp_ctrlmsg::QueueHandleParameters& handleParameters, unsigned int upstreamSubQueueId, @@ -123,7 +124,8 @@ void Queue::getHandle( BSLS_ASSERT_OPT(d_queueEngine_p && "Queue Engine has not been set"); BSLS_ASSERT_SAFE(dispatcher()->inDispatcherThread(this)); - d_queueEngine_p->getHandle(clientContext, + d_queueEngine_p->getHandle(context, + clientContext, handleParameters, upstreamSubQueueId, callback); diff --git a/src/groups/mqb/mqbmock/mqbmock_queue.h b/src/groups/mqb/mqbmock/mqbmock_queue.h index 0519e4faa4..5e4e1bcfda 100644 --- a/src/groups/mqb/mqbmock/mqbmock_queue.h +++ b/src/groups/mqb/mqbmock/mqbmock_queue.h @@ -199,7 +199,8 @@ class Queue : public mqbi::Queue { /// specified `clientContext` and using the specified `handleParameters` /// and `upstreamSubQueueId`. Invoke the specified `callback` with the /// result. - void getHandle(const bsl::shared_ptr& + void getHandle(const mqbi::OpenQueueConfirmationCookieSp& context, + const bsl::shared_ptr& clientContext, const bmqp_ctrlmsg::QueueHandleParameters& handleParameters, unsigned int upstreamSubQueueId, diff --git a/src/groups/mqb/mqbmock/mqbmock_queueengine.cpp b/src/groups/mqb/mqbmock/mqbmock_queueengine.cpp index fc61bb85b8..33dce9aa53 100644 --- a/src/groups/mqb/mqbmock/mqbmock_queueengine.cpp +++ b/src/groups/mqb/mqbmock/mqbmock_queueengine.cpp @@ -58,6 +58,7 @@ int QueueEngine::rebuildInternalState( } mqbi::QueueHandle* QueueEngine::getHandle( + BSLA_UNUSED const mqbi::OpenQueueConfirmationCookieSp& context, BSLA_UNUSED const bsl::shared_ptr& clientContext, BSLA_UNUSED const bmqp_ctrlmsg::QueueHandleParameters& handleParameters, diff --git a/src/groups/mqb/mqbmock/mqbmock_queueengine.h b/src/groups/mqb/mqbmock/mqbmock_queueengine.h index efac05c92f..d6f72b12f3 100644 --- a/src/groups/mqb/mqbmock/mqbmock_queueengine.h +++ b/src/groups/mqb/mqbmock/mqbmock_queueengine.h @@ -111,7 +111,8 @@ class QueueEngine : public mqbi::QueueEngine { /// `handleParameters`, and invoke the specified `callback` when /// finished. mqbi::QueueHandle* - getHandle(const bsl::shared_ptr& + getHandle(const mqbi::OpenQueueConfirmationCookieSp& context, + const bsl::shared_ptr& clientContext, const bmqp_ctrlmsg::QueueHandleParameters& handleParameters, unsigned int upstreamSubQueueId, diff --git a/src/groups/mqb/mqbmock/mqbmock_queuehandle.cpp b/src/groups/mqb/mqbmock/mqbmock_queuehandle.cpp index a540eb269b..ad103c1e7b 100644 --- a/src/groups/mqb/mqbmock/mqbmock_queuehandle.cpp +++ b/src/groups/mqb/mqbmock/mqbmock_queuehandle.cpp @@ -127,9 +127,10 @@ mqbi::DispatcherClient* QueueHandle::client() return d_client_p; } -void QueueHandle::registerSubStream(const bmqp_ctrlmsg::SubQueueIdInfo& stream, - unsigned int upstreamSubQueueId, - const mqbi::QueueCounts& counts) +mqbi::QueueHandle::SubStreams::const_iterator +QueueHandle::registerSubStream(const bmqp_ctrlmsg::SubQueueIdInfo& stream, + unsigned int upstreamSubQueueId, + const mqbi::QueueCounts& counts) { assertConsistentSubStreamInfo(stream.appId(), stream.subId()); @@ -138,15 +139,19 @@ void QueueHandle::registerSubStream(const bmqp_ctrlmsg::SubQueueIdInfo& stream, if (it == d_subStreamInfos.end()) { d_downstreams.emplace(stream.subId(), Downstream(upstreamSubQueueId, d_allocator_p)); - d_subStreamInfos.insert(bsl::make_pair(stream.appId(), - StreamInfo(counts, - stream.subId(), - upstreamSubQueueId, - d_allocator_p))); - return; // RETURN + it = d_subStreamInfos + .insert(bsl::make_pair(stream.appId(), + StreamInfo(counts, + stream.subId(), + upstreamSubQueueId, + d_allocator_p))) + .first; + } + else { + it->second.d_counts += counts; } - it->second.d_counts += counts; + return it; } void QueueHandle::registerSubscription(unsigned int downstreamSubId, diff --git a/src/groups/mqb/mqbmock/mqbmock_queuehandle.h b/src/groups/mqb/mqbmock/mqbmock_queuehandle.h index ea2e716fb5..fe41796b57 100644 --- a/src/groups/mqb/mqbmock/mqbmock_queuehandle.h +++ b/src/groups/mqb/mqbmock/mqbmock_queuehandle.h @@ -229,9 +229,10 @@ class QueueHandle : public mqbi::QueueHandle { /// specified `subStreamInfo` by `readCount` and `writeCount` in /// the specified `counts`. Create new context for the `subStreamInfo` /// if it is not registered. + /// Return an iterator pointing to the context. /// /// THREAD: This method is called from the Queue's dispatcher thread. - void + mqbi::QueueHandle::SubStreams::const_iterator registerSubStream(const bmqp_ctrlmsg::SubQueueIdInfo& subStreamInfo, unsigned int upstreamSubQueueId, const mqbi::QueueCounts& counts) BSLS_KEYWORD_OVERRIDE; diff --git a/src/groups/mqb/mqbstat/mqbstat_clusterstats.cpp b/src/groups/mqb/mqbstat/mqbstat_clusterstats.cpp index 8e907d53ee..567161e1ea 100644 --- a/src/groups/mqb/mqbstat/mqbstat_clusterstats.cpp +++ b/src/groups/mqb/mqbstat/mqbstat_clusterstats.cpp @@ -116,10 +116,6 @@ struct ClusterStatsIndex { }; }; -//------------------------- -// struct ClusterStatsIndex -//------------------------- - /// Namespace for the constants of stat values that applies to the queues /// from the clients struct ClusterStatus { @@ -536,104 +532,6 @@ const char* ClusterStats::Role::toAscii(Enum value) #undef case } -// ---------------------- -// class ClusterNodeStats -// ---------------------- - -bsls::Types::Int64 -ClusterNodeStats::getValue(const bmqst::StatContext& context, - int snapshotId, - const Stat::Enum& stat) - -{ - // invoked from the SNAPSHOT thread - - const bmqst::StatValue::SnapshotLocation latestSnapshot(0, 0); - const bmqst::StatValue::SnapshotLocation oldestSnapshot(0, snapshotId); - -#define STAT_SINGLE(OPERATION, STAT) \ - bmqst::StatUtil::OPERATION( \ - context.value(bmqst::StatContext::e_DIRECT_VALUE, STAT), \ - latestSnapshot) - -#define STAT_RANGE(OPERATION, STAT) \ - bmqst::StatUtil::OPERATION( \ - context.value(bmqst::StatContext::e_DIRECT_VALUE, STAT), \ - latestSnapshot, \ - oldestSnapshot) - - switch (stat) { - case Stat::e_PUSH_MESSAGES_DELTA: { - return STAT_RANGE(incrementsDifference, - ClusterNodeStatsIndex::e_STAT_PUSH); - } - case Stat::e_PUT_MESSAGES_DELTA: { - return STAT_RANGE(incrementsDifference, - ClusterNodeStatsIndex::e_STAT_PUT); - } - case Stat::e_ACK_DELTA: { - return STAT_RANGE(incrementsDifference, - ClusterNodeStatsIndex::e_STAT_ACK); - } - case Stat::e_CONFIRM_DELTA: { - return STAT_RANGE(incrementsDifference, - ClusterNodeStatsIndex::e_STAT_CONFIRM); - } - case Stat::e_PUSH_BYTES_DELTA: { - return STAT_RANGE(valueDifference, ClusterNodeStatsIndex::e_STAT_PUSH); - } - case Stat::e_PUT_BYTES_DELTA: { - return STAT_RANGE(valueDifference, ClusterNodeStatsIndex::e_STAT_PUT); - } - case Stat::e_PUSH_MESSAGES_ABS: { - return STAT_SINGLE(increments, ClusterNodeStatsIndex::e_STAT_PUSH); - } - case Stat::e_PUT_MESSAGES_ABS: { - return STAT_SINGLE(increments, ClusterNodeStatsIndex::e_STAT_PUT); - } - case Stat::e_ACK_ABS: { - return STAT_SINGLE(increments, ClusterNodeStatsIndex::e_STAT_ACK); - } - case Stat::e_CONFIRM_ABS: { - return STAT_SINGLE(increments, ClusterNodeStatsIndex::e_STAT_CONFIRM); - } - case Stat::e_PUSH_BYTES_ABS: { - return STAT_SINGLE(value, ClusterNodeStatsIndex::e_STAT_PUSH); - } - case Stat::e_PUT_BYTES_ABS: { - return STAT_SINGLE(value, ClusterNodeStatsIndex::e_STAT_PUT); - } - default: { - BSLS_ASSERT_SAFE(false && "Attempting to access an unknown stat"); - } - } - - return 0; - -#undef STAT_RANGE -#undef STAT_SINGLE -} - -ClusterNodeStats::ClusterNodeStats() -: d_statContext_mp(0) -{ - // NOTHING -} - -void ClusterNodeStats::initialize(const bmqt::Uri& uri, - bmqst::StatContext* clientStatContext, - bslma::Allocator* allocator) -{ - // PRECONDITIONS - BSLS_ASSERT_SAFE(!d_statContext_mp && "initialize called twice"); - - // Create subContext - bdlma::LocalSequentialAllocator<2048> localAllocator(allocator); - - d_statContext_mp = clientStatContext->addSubcontext( - bmqst::StatContextConfiguration(uri.asString(), &localAllocator)); -} - // ---------------------- // class ClusterStatsUtil // ---------------------- diff --git a/src/groups/mqb/mqbstat/mqbstat_clusterstats.h b/src/groups/mqb/mqbstat/mqbstat_clusterstats.h index c2b9b0c93a..fc9cfe0f92 100644 --- a/src/groups/mqb/mqbstat/mqbstat_clusterstats.h +++ b/src/groups/mqb/mqbstat/mqbstat_clusterstats.h @@ -310,115 +310,6 @@ class ClusterStats { bmqst::StatContext* statContext(); }; -// ====================== -// class ClusterNodeStats -// ====================== - -/// Mechanism to keep track of individual overall statistics of a cluster -/// node. -class ClusterNodeStats { - public: - // TYPES - - /// Enum representing the various type of events for which statistics - /// are monitored. - struct EventType { - // TYPES - enum Enum { e_ACK, e_CONFIRM, e_PUT, e_PUSH }; - }; - - /// Enum representing the various type of stats that can be obtained - /// from this object. - struct Stat { - // TYPES - enum Enum { - e_PUT_MESSAGES_DELTA, - e_PUT_BYTES_DELTA, - e_PUT_MESSAGES_ABS, - e_PUT_BYTES_ABS, - e_PUSH_MESSAGES_DELTA, - e_PUSH_BYTES_DELTA, - e_PUSH_MESSAGES_ABS, - e_PUSH_BYTES_ABS, - e_ACK_DELTA, - e_ACK_ABS, - e_CONFIRM_DELTA, - e_CONFIRM_ABS - }; - }; - - private: - // DATA - - /// StatContext - bslma::ManagedPtr d_statContext_mp; - - // PRIVATE TYPES - - /// Namespace for the constants of stat values that applies to the - /// cluster node - struct ClusterNodeStatsIndex { - enum Enum { - /// Value: Number of ack messages delivered to the client - e_STAT_ACK, - /// Value: Number of confirm messages delivered to the client - e_STAT_CONFIRM, - /// Value: Accumulated bytes of all messages ever pushed to - /// the client - /// Increments: Number of messages ever pushed to the client - e_STAT_PUSH, - /// Value: Accumulated bytes of all messages ever received - /// from the client - /// Increments: Number of messages ever received from the client - e_STAT_PUT - }; - }; - - private: - // NOT IMPLEMENTED - ClusterNodeStats(const ClusterNodeStats&) BSLS_CPP11_DELETED; - - /// Copy constructor and assignment operator are not implemented. - ClusterNodeStats& operator=(const ClusterNodeStats&) BSLS_CPP11_DELETED; - - public: - // CLASS METHODS - - /// Get the value of the specified `stat` reported to the cluster node - /// represented by its associated specified `context` as the difference - /// between the latest snapshot-ed value (i.e., `snapshotId == 0`) and - /// the value that was recorded at the specified `snapshotId` snapshots - /// ago. - /// - /// THREAD: This method can only be invoked from the `snapshot` thread. - static bsls::Types::Int64 getValue(const bmqst::StatContext& context, - int snapshotId, - const Stat::Enum& stat); - - // CREATORS - - /// Create a new object in an uninitialized state. - ClusterNodeStats(); - - // MANIPULATORS - - /// Initialize this object for the queue with the specified `uri`, and - /// register it as a subcontext of the specified - /// `clusterNodesStatContext`, using the specified `allocator`. - void initialize(const bmqt::Uri& uri, - bmqst::StatContext* clusterNodesStatContext, - bslma::Allocator* allocator); - - /// Update statistics for the event of the specified `type` and with the - /// specified `value` (depending on the `type`, `value` can represent - /// the number of bytes, a counter, ... - template - void onEvent(bsls::Types::Int64 value); - - /// Return a pointer to the statcontext. - bmqst::StatContext* statContext(); -}; - // ======================= // struct ClusterStatsUtil // ======================= @@ -456,43 +347,6 @@ inline bmqst::StatContext* ClusterStats::statContext() return d_statContext_mp.get(); } -// ---------------------- -// class ClusterNodeStats -// ---------------------- - -inline bmqst::StatContext* ClusterNodeStats::statContext() -{ - return d_statContext_mp.get(); -} - -template <> -inline void ClusterNodeStats::onEvent( - BSLA_UNUSED bsls::Types::Int64 value) -{ - d_statContext_mp->adjustValue(ClusterNodeStatsIndex::e_STAT_ACK, 1); -} - -template <> -inline void ClusterNodeStats::onEvent( - BSLA_UNUSED bsls::Types::Int64 value) -{ - d_statContext_mp->adjustValue(ClusterNodeStatsIndex::e_STAT_CONFIRM, 1); -} - -template <> -inline void ClusterNodeStats::onEvent( - bsls::Types::Int64 value) -{ - d_statContext_mp->adjustValue(ClusterNodeStatsIndex::e_STAT_PUSH, value); -} - -template <> -inline void ClusterNodeStats::onEvent( - bsls::Types::Int64 value) -{ - d_statContext_mp->adjustValue(ClusterNodeStatsIndex::e_STAT_PUT, value); -} - } // close package namespace } // close enterprise namespace