From b631a080726ac6364ff5c3ee55131bcaa9697695 Mon Sep 17 00:00:00 2001 From: Bernard Normier Date: Thu, 17 Oct 2024 17:01:06 -0400 Subject: [PATCH] Move Timer to IceInternal, add InlineTimerTask (#2923) --- cpp/include/Ice/Communicator.h | 2 +- cpp/include/Ice/Initialize.h | 2 +- cpp/include/Ice/OutgoingAsync.h | 2 +- cpp/include/Ice/Timer.h | 21 ++++++++++++++- cpp/src/DataStorm/Instance.cpp | 2 +- cpp/src/DataStorm/Instance.h | 4 +-- cpp/src/DataStorm/NodeSessionManager.cpp | 16 +++++------ cpp/src/DataStorm/SessionI.cpp | 6 ++--- cpp/src/DataStorm/SessionI.h | 2 +- cpp/src/DataStorm/TimerTaskI.h | 27 ------------------- cpp/src/Ice/ConnectionFactory.cpp | 2 +- cpp/src/Ice/ConnectionI.cpp | 6 ++--- cpp/src/Ice/ConnectionI.h | 4 +-- .../Ice/IdleTimeoutTransceiverDecorator.cpp | 4 +-- cpp/src/Ice/IdleTimeoutTransceiverDecorator.h | 8 +++--- cpp/src/Ice/Initialize.cpp | 2 +- cpp/src/Ice/Instance.cpp | 8 +++--- cpp/src/Ice/Instance.h | 2 +- cpp/src/Ice/RetryQueue.h | 2 +- cpp/src/IceDiscovery/LookupI.h | 6 ++--- cpp/src/IceGrid/Allocatable.h | 2 +- cpp/src/IceGrid/IceGridNode.cpp | 4 +-- cpp/src/IceGrid/NodeI.cpp | 4 +-- cpp/src/IceGrid/NodeI.h | 6 ++--- cpp/src/IceGrid/RegistryI.cpp | 2 +- cpp/src/IceGrid/RegistryI.h | 2 +- cpp/src/IceGrid/ServerI.cpp | 8 +++--- cpp/src/IceGrid/ServerI.h | 10 +++---- cpp/src/IceGrid/SessionI.cpp | 4 +-- cpp/src/IceGrid/SessionI.h | 10 +++---- cpp/src/IceLocatorDiscovery/PluginI.cpp | 4 +-- cpp/src/IceStorm/Instance.cpp | 4 +-- cpp/src/IceStorm/Instance.h | 4 +-- cpp/src/IceStorm/NodeI.cpp | 8 +++--- cpp/src/IceStorm/NodeI.h | 10 +++---- cpp/test/Ice/hold/Server.cpp | 2 +- cpp/test/Ice/hold/TestI.cpp | 6 ++--- cpp/test/Ice/hold/TestI.h | 4 +-- cpp/test/IceUtil/timer/Client.cpp | 16 +++++------ 39 files changed, 114 insertions(+), 124 deletions(-) delete mode 100644 cpp/src/DataStorm/TimerTaskI.h diff --git a/cpp/include/Ice/Communicator.h b/cpp/include/Ice/Communicator.h index 9b181cc66dd..9fbde3452a6 100644 --- a/cpp/include/Ice/Communicator.h +++ b/cpp/include/Ice/Communicator.h @@ -409,7 +409,7 @@ namespace Ice friend ICE_API CommunicatorPtr initialize(StringSeq&, const InitializationData&, std::int32_t); friend ICE_API CommunicatorPtr initialize(const InitializationData&, std::int32_t); friend ICE_API IceInternal::InstancePtr IceInternal::getInstance(const Ice::CommunicatorPtr&); - friend ICE_API ::Ice::TimerPtr IceInternal::getInstanceTimer(const Ice::CommunicatorPtr&); + friend ICE_API IceInternal::TimerPtr IceInternal::getInstanceTimer(const Ice::CommunicatorPtr&); const IceInternal::InstancePtr _instance; }; diff --git a/cpp/include/Ice/Initialize.h b/cpp/include/Ice/Initialize.h index 4e6bd4695bc..3389e0c3c92 100644 --- a/cpp/include/Ice/Initialize.h +++ b/cpp/include/Ice/Initialize.h @@ -703,7 +703,7 @@ namespace IceInternal // to be used by modules such as Freeze. // ICE_API InstancePtr getInstance(const Ice::CommunicatorPtr&); - ICE_API Ice::TimerPtr getInstanceTimer(const Ice::CommunicatorPtr&); + ICE_API TimerPtr getInstanceTimer(const Ice::CommunicatorPtr&); } #endif diff --git a/cpp/include/Ice/OutgoingAsync.h b/cpp/include/Ice/OutgoingAsync.h index f9a4aaf6f80..adab8f5cc0e 100644 --- a/cpp/include/Ice/OutgoingAsync.h +++ b/cpp/include/Ice/OutgoingAsync.h @@ -144,7 +144,7 @@ namespace IceInternal // correct notified of failures and make sure the retry task is // correctly canceled when the invocation completes. // - class ICE_API ProxyOutgoingAsyncBase : public OutgoingAsyncBase, public Ice::TimerTask + class ICE_API ProxyOutgoingAsyncBase : public OutgoingAsyncBase, public TimerTask { public: virtual AsyncStatus invokeRemote(const Ice::ConnectionIPtr&, bool, bool) = 0; diff --git a/cpp/include/Ice/Timer.h b/cpp/include/Ice/Timer.h index c3ac4feb160..448b15ad588 100644 --- a/cpp/include/Ice/Timer.h +++ b/cpp/include/Ice/Timer.h @@ -18,7 +18,7 @@ #include #include -namespace Ice +namespace IceInternal { class Timer; using TimerPtr = std::shared_ptr; @@ -32,6 +32,18 @@ namespace Ice }; using TimerTaskPtr = std::shared_ptr; + // Adapts a function to a TimerTask. + class InlineTimerTask final : public TimerTask + { + public: + InlineTimerTask(std::function function) : _function(std::move(function)) {} + + void runTimerTask() final { _function(); } + + private: + std::function _function; + }; + // The timer class is used to schedule tasks for one-time execution or repeated execution. Tasks are executed by a // dedicated timer thread sequentially. class ICE_API Timer @@ -43,6 +55,13 @@ namespace Ice // Destroy the timer and join the timer execution thread. Must not be called from a timer task. void destroy(); + // Schedule a function-task for execution after a given delay. + template + void schedule(std::function function, const std::chrono::duration& delay) + { + schedule(std::make_shared(std::move(function)), delay); + } + // Schedule task for execution after a given delay. template void schedule(const TimerTaskPtr& task, const std::chrono::duration& delay) diff --git a/cpp/src/DataStorm/Instance.cpp b/cpp/src/DataStorm/Instance.cpp index 15014bb9f81..f7fcca8f550 100644 --- a/cpp/src/DataStorm/Instance.cpp +++ b/cpp/src/DataStorm/Instance.cpp @@ -100,7 +100,7 @@ Instance::Instance(const Ice::CommunicatorPtr& communicator) : _communicator(com _executor = make_shared(); _connectionManager = make_shared(_executor); - _timer = make_shared(); + _timer = make_shared(); _traceLevels = make_shared(_communicator); } diff --git a/cpp/src/DataStorm/Instance.h b/cpp/src/DataStorm/Instance.h index 1a99e47b637..9fcf6507261 100644 --- a/cpp/src/DataStorm/Instance.h +++ b/cpp/src/DataStorm/Instance.h @@ -93,7 +93,7 @@ namespace DataStormI return _executor; } - Ice::TimerPtr getTimer() const + IceInternal::TimerPtr getTimer() const { assert(_timer); return _timer; @@ -126,7 +126,7 @@ namespace DataStormI std::optional _lookup; std::shared_ptr _traceLevels; std::shared_ptr _executor; - Ice::TimerPtr _timer; + IceInternal::TimerPtr _timer; std::chrono::milliseconds _retryDelay; int _retryMultiplier; int _retryCount; diff --git a/cpp/src/DataStorm/NodeSessionManager.cpp b/cpp/src/DataStorm/NodeSessionManager.cpp index 065d00223c8..32c8f3f99c9 100644 --- a/cpp/src/DataStorm/NodeSessionManager.cpp +++ b/cpp/src/DataStorm/NodeSessionManager.cpp @@ -8,7 +8,6 @@ #include "Instance.h" #include "NodeI.h" #include "NodeSessionI.h" -#include "TimerTaskI.h" #include "TopicFactoryI.h" #include "TraceUtil.h" @@ -423,15 +422,14 @@ NodeSessionManager::disconnected(optional node, optional loo else { instance->getTimer()->schedule( - make_shared( - [=, this, self = shared_from_this()] + [=, this, self = shared_from_this()] + { + auto instance = _instance.lock(); + if (instance) { - auto instance = _instance.lock(); - if (instance) - { - self->connect(lookup, _nodePrx); - } - }), + self->connect(lookup, _nodePrx); + } + }, instance->getRetryDelay(_retryCount++)); } } diff --git a/cpp/src/DataStorm/SessionI.cpp b/cpp/src/DataStorm/SessionI.cpp index 0c1fc1be8d5..c1ee24390f8 100644 --- a/cpp/src/DataStorm/SessionI.cpp +++ b/cpp/src/DataStorm/SessionI.cpp @@ -7,7 +7,6 @@ #include "ConnectionManager.h" #include "Instance.h" #include "NodeI.h" -#include "TimerTaskI.h" #include "TopicFactoryI.h" #include "TopicI.h" #include "TraceUtil.h" @@ -693,7 +692,7 @@ SessionI::retry(optional node, exception_ptr exception) << " (ms) for peer to reconnect"; } - _retryTask = make_shared([self = shared_from_this()] { self->remove(); }); + _retryTask = make_shared([self = shared_from_this()] { self->remove(); }); _instance->getTimer()->schedule(_retryTask, delay); } else @@ -736,7 +735,8 @@ SessionI::retry(optional node, exception_ptr exception) return false; } - _retryTask = make_shared([node, self = shared_from_this()] { self->reconnect(node); }); + _retryTask = + make_shared([node, self = shared_from_this()] { self->reconnect(node); }); _instance->getTimer()->schedule(_retryTask, delay); } return true; diff --git a/cpp/src/DataStorm/SessionI.h b/cpp/src/DataStorm/SessionI.h index 66283e73cfd..52441b60e43 100644 --- a/cpp/src/DataStorm/SessionI.h +++ b/cpp/src/DataStorm/SessionI.h @@ -346,7 +346,7 @@ namespace DataStormI bool _destroyed; int _sessionInstanceId; int _retryCount; - Ice::TimerTaskPtr _retryTask; + IceInternal::TimerTaskPtr _retryTask; std::map _topics; std::unique_lock* _topicLock; diff --git a/cpp/src/DataStorm/TimerTaskI.h b/cpp/src/DataStorm/TimerTaskI.h deleted file mode 100644 index f4397f33fa4..00000000000 --- a/cpp/src/DataStorm/TimerTaskI.h +++ /dev/null @@ -1,27 +0,0 @@ -// -// Copyright (c) ZeroC, Inc. All rights reserved. -// - -#ifndef DATASTORMI_TIMER_TASKI_H -#define DATASTORMI_TIMER_TASKI_H - -#include "Ice/Ice.h" - -#include - -namespace DataStormI -{ - // TODO temporary until we fix https://github.com/zeroc-ice/ice/issues/2877 - class TimerTaskI : public Ice::TimerTask - { - public: - TimerTaskI(std::function task) : _task(std::move(task)) {} - - void runTimerTask() override { _task(); } - - private: - std::function _task; - }; -} - -#endif diff --git a/cpp/src/Ice/ConnectionFactory.cpp b/cpp/src/Ice/ConnectionFactory.cpp index 090b8ec5af4..2ea7ce38951 100644 --- a/cpp/src/Ice/ConnectionFactory.cpp +++ b/cpp/src/Ice/ConnectionFactory.cpp @@ -69,7 +69,7 @@ namespace return nullptr; } - class StartAcceptor : public Ice::TimerTask, public std::enable_shared_from_this + class StartAcceptor : public TimerTask, public std::enable_shared_from_this { public: StartAcceptor(const IncomingConnectionFactoryPtr& factory, const InstancePtr& instance) diff --git a/cpp/src/Ice/ConnectionI.cpp b/cpp/src/Ice/ConnectionI.cpp index 6ee7d279c42..11274c48f08 100644 --- a/cpp/src/Ice/ConnectionI.cpp +++ b/cpp/src/Ice/ConnectionI.cpp @@ -38,7 +38,7 @@ using namespace IceInternal; namespace { - class ConnectTimerTask final : public Ice::TimerTask + class ConnectTimerTask final : public TimerTask { public: ConnectTimerTask(const Ice::ConnectionIPtr& connection) : _connection(connection) {} @@ -55,7 +55,7 @@ namespace const weak_ptr _connection; }; - class CloseTimerTask final : public Ice::TimerTask + class CloseTimerTask final : public TimerTask { public: CloseTimerTask(const Ice::ConnectionIPtr& connection) : _connection(connection) {} @@ -72,7 +72,7 @@ namespace const weak_ptr _connection; }; - class InactivityTimerTask final : public Ice::TimerTask + class InactivityTimerTask final : public TimerTask { public: InactivityTimerTask(const Ice::ConnectionIPtr& connection) : _connection(connection) {} diff --git a/cpp/src/Ice/ConnectionI.h b/cpp/src/Ice/ConnectionI.h index 362e7f6f713..afa05307d17 100644 --- a/cpp/src/Ice/ConnectionI.h +++ b/cpp/src/Ice/ConnectionI.h @@ -347,13 +347,13 @@ namespace Ice const IceInternal::TraceLevelsPtr _traceLevels; const IceInternal::ThreadPoolPtr _threadPool; - const Ice::TimerPtr _timer; + const IceInternal::TimerPtr _timer; const std::chrono::seconds _connectTimeout; const std::chrono::seconds _closeTimeout; const std::chrono::seconds _inactivityTimeout; - Ice::TimerTaskPtr _inactivityTimerTask; + IceInternal::TimerTaskPtr _inactivityTimerTask; bool _inactivityTimerTaskScheduled; std::function _connectionStartCompleted; diff --git a/cpp/src/Ice/IdleTimeoutTransceiverDecorator.cpp b/cpp/src/Ice/IdleTimeoutTransceiverDecorator.cpp index 3d6e4247d5d..209381ac1c4 100644 --- a/cpp/src/Ice/IdleTimeoutTransceiverDecorator.cpp +++ b/cpp/src/Ice/IdleTimeoutTransceiverDecorator.cpp @@ -13,7 +13,7 @@ using namespace IceInternal; namespace { - class HeartbeatTimerTask final : public Ice::TimerTask + class HeartbeatTimerTask final : public TimerTask { public: HeartbeatTimerTask(const ConnectionIPtr& connection) : _connection(connection) {} @@ -31,7 +31,7 @@ namespace const weak_ptr _connection; }; - class IdleCheckTimerTask final : public Ice::TimerTask + class IdleCheckTimerTask final : public TimerTask { public: IdleCheckTimerTask(const ConnectionIPtr& connection, const chrono::seconds& idleTimeout) diff --git a/cpp/src/Ice/IdleTimeoutTransceiverDecorator.h b/cpp/src/Ice/IdleTimeoutTransceiverDecorator.h index 45b34bbd788..27c880a1920 100644 --- a/cpp/src/Ice/IdleTimeoutTransceiverDecorator.h +++ b/cpp/src/Ice/IdleTimeoutTransceiverDecorator.h @@ -21,7 +21,7 @@ namespace IceInternal IdleTimeoutTransceiverDecorator( const TransceiverPtr& decoratee, const std::chrono::seconds& idleTimeout, - const Ice::TimerPtr& timer) + const IceInternal::TimerPtr& timer) : _decoratee(decoratee), _idleTimeout(idleTimeout), _timer(timer) @@ -71,13 +71,13 @@ namespace IceInternal private: const TransceiverPtr _decoratee; const std::chrono::seconds _idleTimeout; - const Ice::TimerPtr _timer; + const IceInternal::TimerPtr _timer; bool _idleCheckEnabled = false; // Set by decoratorInit - Ice::TimerTaskPtr _heartbeatTimerTask; - Ice::TimerTaskPtr _idleCheckTimerTask; + IceInternal::TimerTaskPtr _heartbeatTimerTask; + IceInternal::TimerTaskPtr _idleCheckTimerTask; }; } diff --git a/cpp/src/Ice/Initialize.cpp b/cpp/src/Ice/Initialize.cpp index af936d73791..a383fed9047 100644 --- a/cpp/src/Ice/Initialize.cpp +++ b/cpp/src/Ice/Initialize.cpp @@ -412,7 +412,7 @@ IceInternal::getInstance(const CommunicatorPtr& communicator) return communicator->_instance; } -Ice::TimerPtr +IceInternal::TimerPtr IceInternal::getInstanceTimer(const CommunicatorPtr& communicator) { return communicator->_instance->timer(); diff --git a/cpp/src/Ice/Instance.cpp b/cpp/src/Ice/Instance.cpp index ddea5a1b463..2d4545795e2 100644 --- a/cpp/src/Ice/Instance.cpp +++ b/cpp/src/Ice/Instance.cpp @@ -180,7 +180,7 @@ namespace IceInternal // Required because ObserverUpdaterI is a friend of Instan // // Timer specialization which supports the thread observer // - class ThreadObserverTimer final : public Ice::Timer + class ThreadObserverTimer final : public IceInternal::Timer { public: ThreadObserverTimer() : _hasObserver(false) {} @@ -188,7 +188,7 @@ namespace IceInternal // Required because ObserverUpdaterI is a friend of Instan void updateObserver(const Ice::Instrumentation::CommunicatorObserverPtr&); private: - void runTimerTask(const Ice::TimerTaskPtr&) final; + void runTimerTask(const TimerTaskPtr&) final; std::mutex _mutex; std::atomic _hasObserver; @@ -210,7 +210,7 @@ ThreadObserverTimer::updateObserver(const Ice::Instrumentation::CommunicatorObse } void -ThreadObserverTimer::runTimerTask(const Ice::TimerTaskPtr& task) +ThreadObserverTimer::runTimerTask(const TimerTaskPtr& task) { if (_hasObserver) { @@ -443,7 +443,7 @@ IceInternal::Instance::retryQueue() return _retryQueue; } -Ice::TimerPtr +IceInternal::TimerPtr IceInternal::Instance::timer() { lock_guard lock(_mutex); diff --git a/cpp/src/Ice/Instance.h b/cpp/src/Ice/Instance.h index f8b71ba5cce..59a0cd0de47 100644 --- a/cpp/src/Ice/Instance.h +++ b/cpp/src/Ice/Instance.h @@ -83,7 +83,7 @@ namespace IceInternal EndpointHostResolverPtr endpointHostResolver(); RetryQueuePtr retryQueue(); const std::vector& retryIntervals() const { return _retryIntervals; } - Ice::TimerPtr timer(); + IceInternal::TimerPtr timer(); EndpointFactoryManagerPtr endpointFactoryManager() const; Ice::PluginManagerPtr pluginManager() const; size_t messageSizeMax() const { return _messageSizeMax; } diff --git a/cpp/src/Ice/RetryQueue.h b/cpp/src/Ice/RetryQueue.h index 405f30995ea..97ea776e927 100644 --- a/cpp/src/Ice/RetryQueue.h +++ b/cpp/src/Ice/RetryQueue.h @@ -20,7 +20,7 @@ namespace IceInternal using OutgoingAsyncBasePtr = std::shared_ptr; using ProxyOutgoingAsyncBasePtr = std::shared_ptr; - class RetryTask : public Ice::TimerTask, public CancellationHandler, public std::enable_shared_from_this + class RetryTask : public TimerTask, public CancellationHandler, public std::enable_shared_from_this { public: RetryTask(const InstancePtr&, const RetryQueuePtr&, const ProxyOutgoingAsyncBasePtr&); diff --git a/cpp/src/IceDiscovery/LookupI.h b/cpp/src/IceDiscovery/LookupI.h index 89f1e1ce1b2..3ce020ad67f 100644 --- a/cpp/src/IceDiscovery/LookupI.h +++ b/cpp/src/IceDiscovery/LookupI.h @@ -19,7 +19,7 @@ namespace IceDiscovery { class LookupI; - class Request : public Ice::TimerTask + class Request : public IceInternal::TimerTask { public: Request(const LookupIPtr&, int); @@ -137,7 +137,7 @@ namespace IceDiscovery void objectRequestTimedOut(const ObjectRequestPtr&); void objectRequestException(const ObjectRequestPtr&, std::exception_ptr); - const Ice::TimerPtr& timer() { return _timer; } + const IceInternal::TimerPtr& timer() { return _timer; } int latencyMultiplier() { return _latencyMultiplier; } @@ -150,7 +150,7 @@ namespace IceDiscovery const int _latencyMultiplier; const std::string _domainId; - Ice::TimerPtr _timer; + IceInternal::TimerPtr _timer; bool _warnOnce; std::map _objectRequests; diff --git a/cpp/src/IceGrid/Allocatable.h b/cpp/src/IceGrid/Allocatable.h index dadcae770db..d698d1a70c0 100644 --- a/cpp/src/IceGrid/Allocatable.h +++ b/cpp/src/IceGrid/Allocatable.h @@ -17,7 +17,7 @@ namespace IceGrid class Allocatable; class SessionI; - class AllocationRequest : public Ice::TimerTask, public std::enable_shared_from_this + class AllocationRequest : public IceInternal::TimerTask, public std::enable_shared_from_this { public: virtual ~AllocationRequest() = default; diff --git a/cpp/src/IceGrid/IceGridNode.cpp b/cpp/src/IceGrid/IceGridNode.cpp index 99499eac4f8..50dfe1b737c 100644 --- a/cpp/src/IceGrid/IceGridNode.cpp +++ b/cpp/src/IceGrid/IceGridNode.cpp @@ -61,7 +61,7 @@ namespace void usage(const std::string&); shared_ptr _activator; - Ice::TimerPtr _timer; + IceInternal::TimerPtr _timer; shared_ptr _registry; shared_ptr _node; unique_ptr _sessions; @@ -417,7 +417,7 @@ NodeService::startImpl(int argc, char* argv[], int& status) // // Create a new timer to handle server activation/deactivation timeouts. // - _timer = make_shared(); + _timer = make_shared(); // // The IceGrid instance name. diff --git a/cpp/src/IceGrid/NodeI.cpp b/cpp/src/IceGrid/NodeI.cpp index 2ec82f24ebd..059c241cb43 100644 --- a/cpp/src/IceGrid/NodeI.cpp +++ b/cpp/src/IceGrid/NodeI.cpp @@ -48,7 +48,7 @@ NodeI::NodeI( const Ice::ObjectAdapterPtr& adapter, NodeSessionManager& sessions, const shared_ptr& activator, - const Ice::TimerPtr& timer, + const IceInternal::TimerPtr& timer, const shared_ptr& traceLevels, NodePrx proxy, const string& name, @@ -277,7 +277,7 @@ NodeI::getActivator() const return _activator; } -Ice::TimerPtr +IceInternal::TimerPtr NodeI::getTimer() const { return _timer; diff --git a/cpp/src/IceGrid/NodeI.h b/cpp/src/IceGrid/NodeI.h index 9ac92da4f74..e078662e706 100644 --- a/cpp/src/IceGrid/NodeI.h +++ b/cpp/src/IceGrid/NodeI.h @@ -44,7 +44,7 @@ namespace IceGrid const Ice::ObjectAdapterPtr&, NodeSessionManager&, const std::shared_ptr&, - const Ice::TimerPtr&, + const IceInternal::TimerPtr&, const std::shared_ptr&, NodePrx, const std::string&, @@ -100,7 +100,7 @@ namespace IceGrid void shutdown(); - Ice::TimerPtr getTimer() const; + IceInternal::TimerPtr getTimer() const; Ice::CommunicatorPtr getCommunicator() const; Ice::ObjectAdapterPtr getAdapter() const; std::shared_ptr getActivator() const; @@ -164,7 +164,7 @@ namespace IceGrid const Ice::ObjectAdapterPtr _adapter; NodeSessionManager& _sessions; const std::shared_ptr _activator; - const Ice::TimerPtr _timer; + const IceInternal::TimerPtr _timer; const std::shared_ptr _traceLevels; const std::string _name; const NodePrx _proxy; diff --git a/cpp/src/IceGrid/RegistryI.cpp b/cpp/src/IceGrid/RegistryI.cpp index ca0a059b14f..bc353d2fe02 100644 --- a/cpp/src/IceGrid/RegistryI.cpp +++ b/cpp/src/IceGrid/RegistryI.cpp @@ -729,7 +729,7 @@ RegistryI::setupClientSessionFactory(const IceGrid::LocatorPrx& locator) } assert(_reaper); - _timer = make_shared(); // Used for session allocation timeout. + _timer = make_shared(); // Used for session allocation timeout. _clientSessionFactory = make_shared(servantManager, _database, _timer, _reaper); if (servantManager && _master) // Slaves don't support client session manager objects. diff --git a/cpp/src/IceGrid/RegistryI.h b/cpp/src/IceGrid/RegistryI.h index dd57afbcf84..99bc9867918 100644 --- a/cpp/src/IceGrid/RegistryI.h +++ b/cpp/src/IceGrid/RegistryI.h @@ -103,7 +103,7 @@ namespace IceGrid bool _master; std::string _replicaName; std::shared_ptr _reaper; - Ice::TimerPtr _timer; + IceInternal::TimerPtr _timer; std::shared_ptr _servantManager; std::chrono::seconds _sessionTimeout; std::unique_ptr _session; diff --git a/cpp/src/IceGrid/ServerI.cpp b/cpp/src/IceGrid/ServerI.cpp index d38ae6c6877..ecfc52d5771 100644 --- a/cpp/src/IceGrid/ServerI.cpp +++ b/cpp/src/IceGrid/ServerI.cpp @@ -167,7 +167,7 @@ namespace IceGrid return props; } - class CommandTimeoutTimerTask final : public Ice::TimerTask + class CommandTimeoutTimerTask final : public IceInternal::TimerTask { public: CommandTimeoutTimerTask(const shared_ptr& command) : _command(command) {} @@ -178,7 +178,7 @@ namespace IceGrid const shared_ptr _command; }; - class DelayedStart : public Ice::TimerTask + class DelayedStart : public IceInternal::TimerTask { public: DelayedStart(const shared_ptr& server, const shared_ptr& traceLevels) @@ -381,7 +381,7 @@ ServerCommand::ServerCommand(const shared_ptr& server) : _server(server TimedServerCommand::TimedServerCommand( const shared_ptr& server, - const Ice::TimerPtr& timer, + const IceInternal::TimerPtr& timer, chrono::seconds timeout) : ServerCommand(server), _timer(timer), @@ -634,7 +634,7 @@ StartCommand::finished() StopCommand::StopCommand( const shared_ptr& server, - const Ice::TimerPtr& timer, + const IceInternal::TimerPtr& timer, chrono::seconds timeout, bool deactivate) : TimedServerCommand(server, timer, timeout), diff --git a/cpp/src/IceGrid/ServerI.h b/cpp/src/IceGrid/ServerI.h index 61a7669be05..e07e138b887 100644 --- a/cpp/src/IceGrid/ServerI.h +++ b/cpp/src/IceGrid/ServerI.h @@ -162,7 +162,7 @@ namespace IceGrid std::set _activatedAdapters; std::optional _failureTime; ServerActivation _previousActivation; - std::shared_ptr _timerTask; + std::shared_ptr _timerTask; bool _waitForReplication; std::string _stdErrFile; std::string _stdOutFile; @@ -197,15 +197,15 @@ namespace IceGrid class TimedServerCommand : public ServerCommand, public std::enable_shared_from_this { public: - TimedServerCommand(const std::shared_ptr&, const Ice::TimerPtr&, std::chrono::seconds); + TimedServerCommand(const std::shared_ptr&, const IceInternal::TimerPtr&, std::chrono::seconds); virtual void timeout() = 0; void startTimer(); void stopTimer(); private: - Ice::TimerPtr _timer; - std::shared_ptr _timerTask; + IceInternal::TimerPtr _timer; + std::shared_ptr _timerTask; std::chrono::seconds _timeout; }; @@ -232,7 +232,7 @@ namespace IceGrid class StopCommand : public TimedServerCommand { public: - StopCommand(const std::shared_ptr&, const Ice::TimerPtr&, std::chrono::seconds, bool = true); + StopCommand(const std::shared_ptr&, const IceInternal::TimerPtr&, std::chrono::seconds, bool = true); static bool isStopped(ServerI::InternalServerState); diff --git a/cpp/src/IceGrid/SessionI.cpp b/cpp/src/IceGrid/SessionI.cpp index 9a396a86dfa..ea58da7e224 100644 --- a/cpp/src/IceGrid/SessionI.cpp +++ b/cpp/src/IceGrid/SessionI.cpp @@ -111,7 +111,7 @@ BaseSessionI::getGlacier2AdapterIdSet() return _servantManager->getGlacier2AdapterIdSet(shared_from_this()); } -SessionI::SessionI(const string& id, const shared_ptr& database, const Ice::TimerPtr& timer) +SessionI::SessionI(const string& id, const shared_ptr& database, const IceInternal::TimerPtr& timer) : BaseSessionI(id, "client", database), _timer(timer), _allocationTimeout(-1) @@ -261,7 +261,7 @@ SessionI::destroyImpl(bool shutdown) ClientSessionFactory::ClientSessionFactory( const shared_ptr& servantManager, const shared_ptr& database, - const Ice::TimerPtr& timer, + const IceInternal::TimerPtr& timer, const shared_ptr& reaper) : _servantManager(servantManager), _database(database), diff --git a/cpp/src/IceGrid/SessionI.h b/cpp/src/IceGrid/SessionI.h index 30ccda16a60..517a06863e7 100644 --- a/cpp/src/IceGrid/SessionI.h +++ b/cpp/src/IceGrid/SessionI.h @@ -53,7 +53,7 @@ namespace IceGrid class SessionI final : public BaseSessionI, public Session { public: - SessionI(const std::string&, const std::shared_ptr&, const Ice::TimerPtr&); + SessionI(const std::string&, const std::shared_ptr&, const IceInternal::TimerPtr&); Ice::ObjectPrx _register(const std::shared_ptr&, const Ice::ConnectionPtr&); @@ -74,7 +74,7 @@ namespace IceGrid void destroy(const Ice::Current&) final; int getAllocationTimeout() const; - const Ice::TimerPtr& getTimer() const { return _timer; } + const IceInternal::TimerPtr& getTimer() const { return _timer; } bool addAllocationRequest(const std::shared_ptr&); void removeAllocationRequest(const std::shared_ptr&); @@ -84,7 +84,7 @@ namespace IceGrid private: void destroyImpl(bool) final; - const Ice::TimerPtr _timer; + const IceInternal::TimerPtr _timer; int _allocationTimeout; std::set> _requests; std::set> _allocations; @@ -96,7 +96,7 @@ namespace IceGrid ClientSessionFactory( const std::shared_ptr&, const std::shared_ptr&, - const Ice::TimerPtr&, + const IceInternal::TimerPtr&, const std::shared_ptr&); Glacier2::SessionPrx @@ -109,7 +109,7 @@ namespace IceGrid private: const std::shared_ptr _servantManager; const std::shared_ptr _database; - const Ice::TimerPtr _timer; + const IceInternal::TimerPtr _timer; const std::shared_ptr _reaper; const bool _filters; }; diff --git a/cpp/src/IceLocatorDiscovery/PluginI.cpp b/cpp/src/IceLocatorDiscovery/PluginI.cpp index 25a4de4cf53..7ba9c67d0bd 100644 --- a/cpp/src/IceLocatorDiscovery/PluginI.cpp +++ b/cpp/src/IceLocatorDiscovery/PluginI.cpp @@ -57,7 +57,7 @@ namespace using RequestPtr = std::shared_ptr; class LocatorI final : public Ice::BlobjectArrayAsync, - public Ice::TimerTask, + public IceInternal::TimerTask, public std::enable_shared_from_this { public: @@ -85,7 +85,7 @@ namespace chrono::milliseconds _timeout; int _retryCount; chrono::milliseconds _retryDelay; - const Ice::TimerPtr _timer; + const IceInternal::TimerPtr _timer; const int _traceLevel; string _instanceName; diff --git a/cpp/src/IceStorm/Instance.cpp b/cpp/src/IceStorm/Instance.cpp index f2ba767e573..3a947bcb3b4 100644 --- a/cpp/src/IceStorm/Instance.cpp +++ b/cpp/src/IceStorm/Instance.cpp @@ -97,7 +97,7 @@ Instance::Instance( } } - _timer = make_shared(); + _timer = make_shared(); string policy = properties->getIceProperty("IceStorm.Send.QueueSizeMaxPolicy"); if (policy == "RemoveSubscriber") @@ -206,7 +206,7 @@ Instance::traceLevels() const return _traceLevels; } -Ice::TimerPtr +IceInternal::TimerPtr Instance::timer() const { return _timer; diff --git a/cpp/src/IceStorm/Instance.h b/cpp/src/IceStorm/Instance.h index 355d3ac1642..6ebe9b36b44 100644 --- a/cpp/src/IceStorm/Instance.h +++ b/cpp/src/IceStorm/Instance.h @@ -67,7 +67,7 @@ namespace IceStorm std::shared_ptr node() const; std::optional nodeProxy() const; std::shared_ptr traceLevels() const; - Ice::TimerPtr timer() const; + IceInternal::TimerPtr timer() const; std::optional topicReplicaProxy() const; std::optional publisherReplicaProxy() const; std::shared_ptr observer() const; @@ -101,7 +101,7 @@ namespace IceStorm const std::shared_ptr _topicReaper; std::shared_ptr _node; std::shared_ptr _observers; - Ice::TimerPtr _timer; + IceInternal::TimerPtr _timer; std::shared_ptr _observer; }; diff --git a/cpp/src/IceStorm/NodeI.cpp b/cpp/src/IceStorm/NodeI.cpp index f00c8f2d177..196d7adfb03 100644 --- a/cpp/src/IceStorm/NodeI.cpp +++ b/cpp/src/IceStorm/NodeI.cpp @@ -13,7 +13,7 @@ using namespace std; namespace { - class CheckTask : public Ice::TimerTask + class CheckTask : public IceInternal::TimerTask { const shared_ptr _node; @@ -22,7 +22,7 @@ namespace virtual void runTimerTask() { _node->check(); } }; - class MergeTask : public Ice::TimerTask + class MergeTask : public IceInternal::TimerTask { const shared_ptr _node; const set _s; @@ -32,7 +32,7 @@ namespace virtual void runTimerTask() { _node->merge(_s); } }; - class MergeContinueTask : public Ice::TimerTask + class MergeContinueTask : public IceInternal::TimerTask { const shared_ptr _node; @@ -41,7 +41,7 @@ namespace virtual void runTimerTask() { _node->mergeContinue(); } }; - class TimeoutTask : public Ice::TimerTask + class TimeoutTask : public IceInternal::TimerTask { const shared_ptr _node; diff --git a/cpp/src/IceStorm/NodeI.h b/cpp/src/IceStorm/NodeI.h index cc4299a61a2..2d735ec0d7f 100644 --- a/cpp/src/IceStorm/NodeI.h +++ b/cpp/src/IceStorm/NodeI.h @@ -60,7 +60,7 @@ namespace IceStormElection private: void setState(NodeState); - const Ice::TimerPtr _timer; + const IceInternal::TimerPtr _timer; const std::shared_ptr _traceLevels; const std::shared_ptr _observers; const std::shared_ptr _replica; // The replica. @@ -90,10 +90,10 @@ namespace IceStormElection std::optional _coordinatorProxy; bool _destroy; - Ice::TimerTaskPtr _mergeTask; - Ice::TimerTaskPtr _timeoutTask; - Ice::TimerTaskPtr _checkTask; - Ice::TimerTaskPtr _mergeContinueTask; + IceInternal::TimerTaskPtr _mergeTask; + IceInternal::TimerTaskPtr _timeoutTask; + IceInternal::TimerTaskPtr _checkTask; + IceInternal::TimerTaskPtr _mergeContinueTask; mutable std::recursive_mutex _mutex; std::condition_variable_any _condVar; diff --git a/cpp/test/Ice/hold/Server.cpp b/cpp/test/Ice/hold/Server.cpp index aa87a52d8c2..da3c5cbca48 100644 --- a/cpp/test/Ice/hold/Server.cpp +++ b/cpp/test/Ice/hold/Server.cpp @@ -22,7 +22,7 @@ Server::run(int argc, char** argv) initData.properties->setProperty("Ice.Connection.Server.IdleTimeout", "1"); Ice::CommunicatorHolder communicator = initialize(argc, argv, initData); - Ice::TimerPtr timer = make_shared(); + IceInternal::TimerPtr timer = make_shared(); auto properties = communicator->getProperties(); properties->setProperty("TestAdapter1.Endpoints", getTestEndpoint()); diff --git a/cpp/test/Ice/hold/TestI.cpp b/cpp/test/Ice/hold/TestI.cpp index 32ef07ca03c..ed109da75d4 100644 --- a/cpp/test/Ice/hold/TestI.cpp +++ b/cpp/test/Ice/hold/TestI.cpp @@ -12,7 +12,7 @@ using namespace std; -HoldI::HoldI(const Ice::TimerPtr& timer, const Ice::ObjectAdapterPtr& adapter) +HoldI::HoldI(const IceInternal::TimerPtr& timer, const Ice::ObjectAdapterPtr& adapter) : _last(0), _timer(timer), _adapter(adapter) @@ -22,7 +22,7 @@ HoldI::HoldI(const Ice::TimerPtr& timer, const Ice::ObjectAdapterPtr& adapter) void HoldI::putOnHold(int32_t milliSeconds, const Ice::Current&) { - class PutOnHold : public Ice::TimerTask + class PutOnHold : public IceInternal::TimerTask { public: PutOnHold(const Ice::ObjectAdapterPtr& adapter) : _adapter(adapter) {} @@ -72,7 +72,7 @@ HoldI::putOnHold(int32_t milliSeconds, const Ice::Current&) void HoldI::waitForHold(const Ice::Current& current) { - class WaitForHold final : public Ice::TimerTask + class WaitForHold final : public IceInternal::TimerTask { public: WaitForHold(const Ice::ObjectAdapterPtr& adapter) : _adapter(adapter) {} diff --git a/cpp/test/Ice/hold/TestI.h b/cpp/test/Ice/hold/TestI.h index c8b7cc97e23..ec674a3e659 100644 --- a/cpp/test/Ice/hold/TestI.h +++ b/cpp/test/Ice/hold/TestI.h @@ -10,7 +10,7 @@ class HoldI final : public Test::Hold { public: - HoldI(const Ice::TimerPtr&, const Ice::ObjectAdapterPtr&); + HoldI(const IceInternal::TimerPtr&, const Ice::ObjectAdapterPtr&); void putOnHold(std::int32_t, const Ice::Current&) final; void waitForHold(const Ice::Current&) final; @@ -20,7 +20,7 @@ class HoldI final : public Test::Hold private: int _last; - const Ice::TimerPtr _timer; + const IceInternal::TimerPtr _timer; const Ice::ObjectAdapterPtr _adapter; std::mutex _mutex; }; diff --git a/cpp/test/IceUtil/timer/Client.cpp b/cpp/test/IceUtil/timer/Client.cpp index 4627e3dfc85..a8cb10b1b16 100644 --- a/cpp/test/IceUtil/timer/Client.cpp +++ b/cpp/test/IceUtil/timer/Client.cpp @@ -11,7 +11,7 @@ #include #include -using namespace Ice; +using namespace IceInternal; using namespace std; template struct TargetLess @@ -29,7 +29,7 @@ template struct TargetLess } }; -class TestTask : public Ice::TimerTask +class TestTask : public TimerTask { public: TestTask() : _count(0) {} @@ -93,10 +93,10 @@ class TestTask : public Ice::TimerTask }; using TestTaskPtr = std::shared_ptr; -class DestroyTask : public Ice::TimerTask +class DestroyTask : public TimerTask { public: - DestroyTask(const Ice::TimerPtr& timer) : _timer(timer), _run(false) {} + DestroyTask(const IceInternal::TimerPtr& timer) : _timer(timer), _run(false) {} virtual void runTimerTask() { @@ -127,7 +127,7 @@ class DestroyTask : public Ice::TimerTask } private: - Ice::TimerPtr _timer; + IceInternal::TimerPtr _timer; bool _run; mutable mutex _mutex; condition_variable _condition; @@ -145,7 +145,7 @@ Client::run(int, char*[]) { cout << "testing timer... " << flush; { - auto timer = make_shared(); + auto timer = make_shared(); { TestTaskPtr task = make_shared(); timer->schedule(task, chrono::seconds::zero()); @@ -229,7 +229,7 @@ Client::run(int, char*[]) cout << "testing timer destroy... " << flush; { { - auto timer = make_shared(); + auto timer = make_shared(); DestroyTaskPtr destroyTask = make_shared(timer); timer->schedule(destroyTask, chrono::seconds::zero()); destroyTask->waitForRun(); @@ -244,7 +244,7 @@ Client::run(int, char*[]) timer->destroy(); } { - auto timer = make_shared(); + auto timer = make_shared(); TestTaskPtr testTask = make_shared(); timer->schedule(testTask, chrono::seconds::zero()); timer->destroy();