Commit d9773eb7 authored by neverlord's avatar neverlord

use std::thread instead of boost::thread on linux

parent fd091b93
...@@ -215,3 +215,4 @@ src/abstract_tuple.cpp ...@@ -215,3 +215,4 @@ src/abstract_tuple.cpp
cppa/util/duration.hpp cppa/util/duration.hpp
src/duration.cpp src/duration.cpp
cppa/receive.hpp cppa/receive.hpp
cppa/detail/thread.hpp
#ifndef ACTOR_COUNT_HPP #ifndef ACTOR_COUNT_HPP
#define ACTOR_COUNT_HPP #define ACTOR_COUNT_HPP
#include <atomic> //#include "cppa/attachable.hpp"
#include <boost/thread.hpp>
#include "cppa/attachable.hpp"
namespace cppa { namespace detail { namespace cppa { namespace detail {
class actor_count { void inc_actor_count();
void dec_actor_count();
public:
actor_count();
void inc();
void dec();
// @pre expected <= 1 /*
void wait_until(size_t expected); * @pre @p expected <= 1
*/
static actor_count& get(); void actor_count_wait_until(size_t expected);
private:
static actor_count* s_instance;
boost::mutex m_ra_mtx;
boost::condition_variable m_ra_cv;
std::atomic<size_t> m_running_actors;
};
/*
struct exit_observer : cppa::attachable struct exit_observer : cppa::attachable
{ {
virtual ~exit_observer(); virtual ~exit_observer();
}; };
*/
} } // namespace cppa::detail } } // namespace cppa::detail
......
#ifndef TASK_SCHEDULER_HPP #ifndef TASK_SCHEDULER_HPP
#define TASK_SCHEDULER_HPP #define TASK_SCHEDULER_HPP
#include <boost/thread.hpp>
#include "cppa/scheduler.hpp" #include "cppa/scheduler.hpp"
#include "cppa/detail/thread.hpp"
#include "cppa/detail/scheduled_actor.hpp" #include "cppa/detail/scheduled_actor.hpp"
#include "cppa/util/single_reader_queue.hpp" #include "cppa/util/single_reader_queue.hpp"
...@@ -16,7 +15,7 @@ class task_scheduler : public scheduler ...@@ -16,7 +15,7 @@ class task_scheduler : public scheduler
job_queue m_queue; job_queue m_queue;
scheduled_actor m_dummy; scheduled_actor m_dummy;
boost::thread m_worker; thread m_worker;
static void worker_loop(job_queue*, scheduled_actor* dummy); static void worker_loop(job_queue*, scheduled_actor* dummy);
......
#ifndef THREAD_HPP
#define THREAD_HPP
#ifdef __APPLE__
#include <boost/thread.hpp>
namespace cppa { namespace detail {
using boost::mutex;
using boost::thread;
using boost::condition_variable;
} } // namespace cppa::detail
#else
#include <mutex>
#include <thread>
#include <condition_variable>
namespace cppa { namespace detail {
using std::mutex;
using std::thread;
using std::condition_variable;
using std::unique_lock;
namespace this_thread { using namespace std::this_thread; }
// returns false if a timeout occured
template<class Lock, class Condition, typename TimePoint>
inline bool wait_until(Lock& lock, Condition& cond, const TimePoint& timeout)
{
return cond.wait_until(lock, timeout) != std::cv_status::timeout;
}
inline auto now() -> decltype(std::chrono::high_resolution_clock::now())
{
return std::chrono::high_resolution_clock::now();
}
} } // namespace cppa::detail
#endif // __APPLE__
#endif // THREAD_HPP
#ifndef THREAD_POOL_SCHEDULER_HPP #ifndef THREAD_POOL_SCHEDULER_HPP
#define THREAD_POOL_SCHEDULER_HPP #define THREAD_POOL_SCHEDULER_HPP
#include <boost/thread.hpp>
#include "cppa/scheduler.hpp" #include "cppa/scheduler.hpp"
#include "cppa/detail/thread.hpp"
#include "cppa/detail/scheduled_actor.hpp" #include "cppa/detail/scheduled_actor.hpp"
namespace cppa { namespace detail { namespace cppa { namespace detail {
...@@ -16,7 +15,7 @@ class thread_pool_scheduler : public scheduler ...@@ -16,7 +15,7 @@ class thread_pool_scheduler : public scheduler
job_queue m_queue; job_queue m_queue;
scheduled_actor m_dummy; scheduled_actor m_dummy;
boost::thread m_supervisor; thread m_supervisor;
public: public:
......
...@@ -4,8 +4,6 @@ ...@@ -4,8 +4,6 @@
#include <chrono> #include <chrono>
#include <cstdint> #include <cstdint>
#include <boost/thread/thread_time.hpp>
namespace cppa { namespace util { namespace cppa { namespace util {
enum class time_unit : std::uint32_t enum class time_unit : std::uint32_t
...@@ -66,10 +64,28 @@ inline bool operator!=(const duration& lhs, const duration& rhs) ...@@ -66,10 +64,28 @@ inline bool operator!=(const duration& lhs, const duration& rhs)
} } // namespace cppa::util } } // namespace cppa::util
namespace boost { template<class Clock, class Duration>
std::chrono::time_point<Clock, Duration>&
operator+=(std::chrono::time_point<Clock, Duration>& lhs,
const cppa::util::duration& rhs)
{
switch (rhs.unit)
{
case cppa::util::time_unit::seconds:
lhs += std::chrono::seconds(rhs.count);
break;
case cppa::util::time_unit::milliseconds:
lhs += std::chrono::milliseconds(rhs.count);
break;
system_time& operator+=(system_time& lhs, const cppa::util::duration& d); case cppa::util::time_unit::microseconds:
lhs += std::chrono::microseconds(rhs.count);
break;
} // namespace boost default: break;
}
return lhs;
}
#endif // DURATION_HPP #endif // DURATION_HPP
...@@ -2,7 +2,8 @@ ...@@ -2,7 +2,8 @@
#define SINGLE_READER_QUEUE_HPP #define SINGLE_READER_QUEUE_HPP
#include <atomic> #include <atomic>
#include <boost/thread.hpp>
#include "cppa/detail/thread.hpp"
namespace cppa { namespace util { namespace cppa { namespace util {
...@@ -13,7 +14,7 @@ template<typename T> ...@@ -13,7 +14,7 @@ template<typename T>
class single_reader_queue class single_reader_queue
{ {
typedef boost::unique_lock<boost::mutex> lock_type; typedef detail::unique_lock<detail::mutex> lock_type;
public: public:
...@@ -37,24 +38,12 @@ class single_reader_queue ...@@ -37,24 +38,12 @@ class single_reader_queue
return take_head(); return take_head();
} }
element_type* try_pop(boost::system_time timeout) template<typename TimePoint>
element_type* try_pop(const TimePoint& abs_time)
{ {
return (timed_wait_for_data(timeout)) ? take_head() : nullptr; return (timed_wait_for_data(abs_time)) ? take_head() : nullptr;
} }
/*
element_type* try_pop(unsigned long ms_timeout)
{
boost::system_time st = boost::get_system_time();
st += boost::posix_time::milliseconds(ms_timeout);
if (timed_wait_for_data(st))
{
return try_pop();
}
return nullptr;
}
*/
/** /**
* @warning call only from the reader (owner) * @warning call only from the reader (owner)
*/ */
...@@ -164,17 +153,18 @@ class single_reader_queue ...@@ -164,17 +153,18 @@ class single_reader_queue
element_type* m_head; element_type* m_head;
// locked on enqueue/dequeue operations to/from an empty list // locked on enqueue/dequeue operations to/from an empty list
boost::mutex m_mtx; detail::mutex m_mtx;
boost::condition_variable m_cv; detail::condition_variable m_cv;
bool timed_wait_for_data(const boost::system_time& timeout) template<typename TimePoint>
bool timed_wait_for_data(const TimePoint& timeout)
{ {
if (!m_head && !(m_tail.load())) if (!m_head && !(m_tail.load()))
{ {
lock_type guard(m_mtx); lock_type guard(m_mtx);
while (!(m_tail.load())) while (!(m_tail.load()))
{ {
if (!m_cv.timed_wait(guard, timeout)) if (detail::wait_until(guard, m_cv, timeout) == false)
{ {
return false; return false;
} }
......
#include <atomic>
#include <limits> #include <limits>
#include <stdexcept> #include <stdexcept>
#include "cppa/detail/thread.hpp"
#include "cppa/detail/actor_count.hpp" #include "cppa/detail/actor_count.hpp"
namespace { namespace {
typedef boost::unique_lock<boost::mutex> guard_type; typedef cppa::detail::unique_lock<cppa::detail::mutex> guard_type;
} // namespace <anonymous> class actor_count
{
namespace cppa { namespace detail {
// TODO: free cppa::detail::mutex m_ra_mtx;
actor_count* actor_count::s_instance = new actor_count(); cppa::detail::condition_variable m_ra_cv;
std::atomic<size_t> m_running_actors;
actor_count& actor_count::get() public:
{
return *s_instance;
}
actor_count::actor_count() : m_running_actors(0) actor_count() : m_running_actors(0) { }
{
}
void actor_count::inc() void inc()
{ {
++m_running_actors; ++m_running_actors;
} }
void actor_count::dec() void dec()
{ {
size_t new_val = --m_running_actors; size_t new_val = --m_running_actors;
if (new_val == std::numeric_limits<size_t>::max()) if (new_val == std::numeric_limits<size_t>::max())
{ {
...@@ -40,20 +37,39 @@ void actor_count::dec() ...@@ -40,20 +37,39 @@ void actor_count::dec()
guard_type guard(m_ra_mtx); guard_type guard(m_ra_mtx);
m_ra_cv.notify_all(); m_ra_cv.notify_all();
} }
} }
void actor_count::wait_until(size_t expected) void wait_until(size_t expected)
{ {
guard_type lock(m_ra_mtx); guard_type lock(m_ra_mtx);
while (m_running_actors.load() != expected) while (m_running_actors.load() != expected)
{ {
m_ra_cv.wait(lock); m_ra_cv.wait(lock);
} }
}
};
// TODO: free
actor_count* s_actor_count = new actor_count;
} // namespace <anonymous>
namespace cppa { namespace detail {
void inc_actor_count()
{
s_actor_count->inc();
}
void dec_actor_count()
{
s_actor_count->dec();
} }
exit_observer::~exit_observer() void actor_count_wait_until(size_t expected)
{ {
actor_count::get().dec(); s_actor_count->wait_until(expected);
} }
} } // namespace cppa::detail } } // namespace cppa::detail
#include <boost/thread.hpp>
#include "cppa/atom.hpp" #include "cppa/atom.hpp"
#include "cppa/message.hpp" #include "cppa/message.hpp"
#include "cppa/detail/thread.hpp"
#include "cppa/detail/actor_proxy_cache.hpp" #include "cppa/detail/actor_proxy_cache.hpp"
// thread_specific_ptr
#include <boost/thread.hpp>
namespace cppa { namespace detail { namespace cppa { namespace detail {
class actor_proxy_caches { class actor_proxy_caches {
......
#include <vector> #include <vector>
#include <memory> #include <memory>
#include <boost/thread.hpp>
#include "cppa/atom.hpp" #include "cppa/atom.hpp"
#include "cppa/match.hpp" #include "cppa/match.hpp"
#include "cppa/context.hpp" #include "cppa/context.hpp"
...@@ -102,7 +100,7 @@ bool blocking_message_queue_impl::dequeue_impl(timed_invoke_rules& rules, ...@@ -102,7 +100,7 @@ bool blocking_message_queue_impl::dequeue_impl(timed_invoke_rules& rules,
std::unique_ptr<queue_node> node(m_queue.try_pop()); std::unique_ptr<queue_node> node(m_queue.try_pop());
if (!node) if (!node)
{ {
boost::system_time timeout = boost::get_system_time(); auto timeout = now();
timeout += rules.timeout(); timeout += rules.timeout();
node.reset(m_queue.try_pop(timeout)); node.reset(m_queue.try_pop(timeout));
if (!node) if (!node)
......
...@@ -18,28 +18,3 @@ bool operator==(const duration& lhs, const duration& rhs) ...@@ -18,28 +18,3 @@ bool operator==(const duration& lhs, const duration& rhs)
} }
} } // namespace cppa::util } } // namespace cppa::util
namespace boost {
system_time& operator+=(system_time& lhs, const cppa::util::duration& d)
{
switch (d.unit)
{
case cppa::util::time_unit::seconds:
lhs += posix_time::seconds(d.count);
break;
case cppa::util::time_unit::milliseconds:
lhs += posix_time::milliseconds(d.count);
break;
case cppa::util::time_unit::microseconds:
lhs += posix_time::microseconds(d.count);
break;
default: break;
}
return lhs;
}
} // namespace boost
#include <atomic> #include <atomic>
#include <iostream> #include <iostream>
#include <boost/thread.hpp> // boost::barrier
#include "cppa/to_string.hpp" #include "cppa/to_string.hpp"
#include "cppa/detail/mailman.hpp" #include "cppa/detail/mailman.hpp"
......
...@@ -3,8 +3,6 @@ ...@@ -3,8 +3,6 @@
#include <atomic> #include <atomic>
#include <iostream> #include <iostream>
#include <boost/thread/mutex.hpp>
#include <boost/thread/condition_variable.hpp>
#include "cppa/message.hpp" #include "cppa/message.hpp"
#include "cppa/context.hpp" #include "cppa/context.hpp"
...@@ -12,6 +10,8 @@ ...@@ -12,6 +10,8 @@
#include "cppa/attachable.hpp" #include "cppa/attachable.hpp"
#include "cppa/invoke_rules.hpp" #include "cppa/invoke_rules.hpp"
#include "cppa/actor_behavior.hpp" #include "cppa/actor_behavior.hpp"
#include "cppa/detail/thread.hpp"
#include "cppa/detail/actor_count.hpp" #include "cppa/detail/actor_count.hpp"
#include "cppa/detail/mock_scheduler.hpp" #include "cppa/detail/mock_scheduler.hpp"
#include "cppa/detail/to_uniform_name.hpp" #include "cppa/detail/to_uniform_name.hpp"
...@@ -34,7 +34,7 @@ void run_actor(cppa::intrusive_ptr<cppa::context> m_self, ...@@ -34,7 +34,7 @@ void run_actor(cppa::intrusive_ptr<cppa::context> m_self,
catch (...) { } catch (...) { }
delete behavior; delete behavior;
} }
cppa::detail::actor_count::get().dec(); cppa::detail::dec_actor_count();
} }
} // namespace <anonymous> } // namespace <anonymous>
...@@ -43,10 +43,10 @@ namespace cppa { namespace detail { ...@@ -43,10 +43,10 @@ namespace cppa { namespace detail {
actor_ptr mock_scheduler::spawn(actor_behavior* behavior) actor_ptr mock_scheduler::spawn(actor_behavior* behavior)
{ {
actor_count::get().inc(); inc_actor_count();
CPPA_MEMORY_BARRIER(); CPPA_MEMORY_BARRIER();
intrusive_ptr<context> ctx(new detail::converted_thread_context); intrusive_ptr<context> ctx(new detail::converted_thread_context);
boost::thread(run_actor, ctx, behavior).detach(); thread(run_actor, ctx, behavior).detach();
return ctx; return ctx;
} }
......
...@@ -8,6 +8,7 @@ ...@@ -8,6 +8,7 @@
#include <exception> // std::logic_error #include <exception> // std::logic_error
#include <algorithm> // std::find_if #include <algorithm> // std::find_if
#include <stdexcept> // std::underflow_error #include <stdexcept> // std::underflow_error
#include <sstream>
#include <cstdio> #include <cstdio>
#include <fcntl.h> #include <fcntl.h>
...@@ -24,6 +25,7 @@ ...@@ -24,6 +25,7 @@
#include "cppa/util/single_reader_queue.hpp" #include "cppa/util/single_reader_queue.hpp"
// used cppa details // used cppa details
#include "cppa/detail/thread.hpp"
#include "cppa/detail/buffer.hpp" #include "cppa/detail/buffer.hpp"
#include "cppa/detail/mailman.hpp" #include "cppa/detail/mailman.hpp"
#include "cppa/detail/post_office.hpp" #include "cppa/detail/post_office.hpp"
...@@ -171,7 +173,7 @@ struct post_office_manager ...@@ -171,7 +173,7 @@ struct post_office_manager
int m_pipe[2]; // m_pipe[0]: read; m_pipe[1]: write int m_pipe[2]; // m_pipe[0]: read; m_pipe[1]: write
queue_t m_queue; // post office queue queue_t m_queue; // post office queue
boost::thread* m_loop; // post office thread thread* m_loop; // post office thread
mailman_queue_t m_mailman_queue; // mailman queue mailman_queue_t m_mailman_queue; // mailman queue
post_office_manager() post_office_manager()
...@@ -184,7 +186,7 @@ struct post_office_manager ...@@ -184,7 +186,7 @@ struct post_office_manager
free(error_cstr); free(error_cstr);
throw std::logic_error(error_str); throw std::logic_error(error_str);
} }
m_loop = new boost::thread(post_office_loop, m_pipe[0]); m_loop = new thread(post_office_loop, m_pipe[0]);
} }
int write_handle() int write_handle()
...@@ -572,7 +574,7 @@ class po_doorman : public post_office_worker ...@@ -572,7 +574,7 @@ class po_doorman : public post_office_worker
// starts and stops mailman_loop // starts and stops mailman_loop
struct mailman_worker struct mailman_worker
{ {
boost::thread m_thread; thread m_thread;
mailman_worker() : m_thread(mailman_loop) mailman_worker() : m_thread(mailman_loop)
{ {
} }
...@@ -583,54 +585,6 @@ struct mailman_worker ...@@ -583,54 +585,6 @@ struct mailman_worker
} }
}; };
/*
// complexity: O(n)
// pred only evaluates values
template<typename K, typename V, typename Predicate>
void remove_if_value(std::map<K,V>& haystack, Predicate&& pred)
{
auto i = haystack.begin();
auto end = haystack.end();
while (i != end)
{
if (pred(i->second))
{
haystack.erase(i++);
}
else
{
++i;
}
}
}
template<typename K, typename V, typename Predicate>
typename std::map<K,V>::iterator
find_if_value(std::map<K,V>& haystack, Predicate&& pred)
{
auto end = haystack.end();
for (auto i = haystack.begin(); i != end; ++i)
{
if (pred(i->second)) return i;
}
return end;
}
template<typename K, typename V, typename Predicate>
typename std::map<K,V>::const_iterator
find_if_value(const std::map<K,V>& haystack, Predicate&& pred)
{
auto end = haystack.end();
for (auto i = haystack.begin(); i != end; ++i)
{
if (pred(i->second)) return i;
}
return end;
}
typedef std::map<process_information, po_peer> peer_map;
*/
void post_office_loop(int pipe_read_handle) void post_office_loop(int pipe_read_handle)
{ {
mailman_worker mworker; mailman_worker mworker;
...@@ -649,7 +603,7 @@ void post_office_loop(int pipe_read_handle) ...@@ -649,7 +603,7 @@ void post_office_loop(int pipe_read_handle)
// keeps track about what peer we are iterating at this time // keeps track about what peer we are iterating at this time
po_peer* selected_peer = nullptr; po_peer* selected_peer = nullptr;
// thread id of post_office // thread id of post_office
auto thread_id = boost::this_thread::get_id(); auto thread_id = this_thread::get_id();
// if an actor calls its quit() handler in this thread, // if an actor calls its quit() handler in this thread,
// we 'catch' the released socket here // we 'catch' the released socket here
std::vector<native_socket_t> released_socks; std::vector<native_socket_t> released_socks;
...@@ -677,7 +631,7 @@ void post_office_loop(int pipe_read_handle) ...@@ -677,7 +631,7 @@ void post_office_loop(int pipe_read_handle)
auto msock = selected_peer->get_socket(); auto msock = selected_peer->get_socket();
pptr->attach_functor([msock, thread_id, &released_socks](std::uint32_t) pptr->attach_functor([msock, thread_id, &released_socks](std::uint32_t)
{ {
if (boost::this_thread::get_id() == thread_id) if (this_thread::get_id() == thread_id)
{ {
released_socks.push_back(msock); released_socks.push_back(msock);
} }
......
#include <atomic> #include <atomic>
#ifndef _GLIBCXX_HAS_GTHREADS
#define _GLIBCXX_HAS_GTHREADS
#endif
#include <mutex>
#include <iostream> #include <iostream>
#include "cppa/on.hpp" #include "cppa/on.hpp"
...@@ -12,6 +6,7 @@ ...@@ -12,6 +6,7 @@
#include "cppa/scheduler.hpp" #include "cppa/scheduler.hpp"
#include "cppa/to_string.hpp" #include "cppa/to_string.hpp"
#include "cppa/detail/thread.hpp"
#include "cppa/detail/actor_count.hpp" #include "cppa/detail/actor_count.hpp"
#include "cppa/detail/mock_scheduler.hpp" #include "cppa/detail/mock_scheduler.hpp"
#include "cppa/detail/thread_pool_scheduler.hpp" #include "cppa/detail/thread_pool_scheduler.hpp"
...@@ -42,6 +37,14 @@ struct static_cleanup_helper ...@@ -42,6 +37,14 @@ struct static_cleanup_helper
s_cleanup_helper; s_cleanup_helper;
*/ */
struct exit_observer : cppa::attachable
{
~exit_observer()
{
cppa::detail::dec_actor_count();
}
};
} // namespace <anonymous> } // namespace <anonymous>
namespace cppa { namespace cppa {
...@@ -54,7 +57,7 @@ struct scheduler_helper ...@@ -54,7 +57,7 @@ struct scheduler_helper
scheduler_helper() : m_worker(new detail::converted_thread_context) scheduler_helper() : m_worker(new detail::converted_thread_context)
{ {
// do NOT increase actor count; worker is "invisible" // do NOT increase actor count; worker is "invisible"
boost::thread(&scheduler_helper::time_emitter, m_worker).detach(); detail::thread(&scheduler_helper::time_emitter, m_worker).detach();
} }
~scheduler_helper() ~scheduler_helper()
...@@ -62,8 +65,6 @@ struct scheduler_helper ...@@ -62,8 +65,6 @@ struct scheduler_helper
m_worker->enqueue(message(m_worker, m_worker, atom(":_DIE"))); m_worker->enqueue(message(m_worker, m_worker, atom(":_DIE")));
} }
//std::multimap<boost::system_time, cppa::any_tuple> m_messages;
ptr_type m_worker; ptr_type m_worker;
private: private:
...@@ -78,9 +79,9 @@ void scheduler_helper::time_emitter(scheduler_helper::ptr_type m_self) ...@@ -78,9 +79,9 @@ void scheduler_helper::time_emitter(scheduler_helper::ptr_type m_self)
set_self(m_self.get()); set_self(m_self.get());
auto& queue = m_self->m_mailbox.queue(); auto& queue = m_self->m_mailbox.queue();
typedef std::pair<cppa::actor_ptr, cppa::any_tuple> future_msg; typedef std::pair<cppa::actor_ptr, cppa::any_tuple> future_msg;
std::multimap<boost::system_time, future_msg> messages; std::multimap<decltype(detail::now()), future_msg> messages;
decltype(queue.pop()) msg_ptr = nullptr; decltype(queue.pop()) msg_ptr = nullptr;
boost::system_time now; decltype(detail::now()) now;
bool done = false; bool done = false;
// message handling rules // message handling rules
auto rules = auto rules =
...@@ -91,25 +92,8 @@ void scheduler_helper::time_emitter(scheduler_helper::ptr_type m_self) ...@@ -91,25 +92,8 @@ void scheduler_helper::time_emitter(scheduler_helper::ptr_type m_self)
if (!tup.empty()) if (!tup.empty())
{ {
// calculate timeout // calculate timeout
boost::system_time timeout = boost::get_system_time(); auto timeout = detail::now();
switch (d.unit) timeout += d;
{
case util::time_unit::seconds:
timeout += boost::posix_time::seconds(d.count);
break;
case util::time_unit::milliseconds:
timeout += boost::posix_time::milliseconds(d.count);
break;
case util::time_unit::microseconds:
timeout += boost::posix_time::microseconds(d.count);
break;
default:
// unsupported duration type
return;
}
future_msg fmsg(msg_ptr->msg.sender(), tup); future_msg fmsg(msg_ptr->msg.sender(), tup);
messages.insert(std::make_pair(std::move(timeout), messages.insert(std::make_pair(std::move(timeout),
std::move(fmsg))); std::move(fmsg)));
...@@ -131,7 +115,7 @@ void scheduler_helper::time_emitter(scheduler_helper::ptr_type m_self) ...@@ -131,7 +115,7 @@ void scheduler_helper::time_emitter(scheduler_helper::ptr_type m_self)
} }
else else
{ {
now = boost::get_system_time(); now = detail::now();
// handle timeouts (send messages) // handle timeouts (send messages)
auto it = messages.begin(); auto it = messages.begin();
while (it != messages.end() && (it->first) <= now) while (it != messages.end() && (it->first) <= now)
...@@ -171,22 +155,22 @@ channel* scheduler::future_send_helper() ...@@ -171,22 +155,22 @@ channel* scheduler::future_send_helper()
void scheduler::await_others_done() void scheduler::await_others_done()
{ {
detail::actor_count::get().wait_until((unchecked_self() == nullptr) ? 0 : 1); detail::actor_count_wait_until((unchecked_self() == nullptr) ? 0 : 1);
} }
void scheduler::register_converted_context(context* what) void scheduler::register_converted_context(context* what)
{ {
if (what) if (what)
{ {
detail::actor_count::get().inc(); detail::inc_actor_count();
what->attach(new detail::exit_observer); what->attach(new exit_observer);
} }
} }
attachable* scheduler::register_hidden_context() attachable* scheduler::register_hidden_context()
{ {
detail::actor_count::get().inc(); detail::inc_actor_count();
return new detail::exit_observer; return new exit_observer;
} }
void scheduler::exit_context(context* ctx, std::uint32_t reason) void scheduler::exit_context(context* ctx, std::uint32_t reason)
......
#include <iostream> #include <iostream>
#include <boost/thread.hpp>
#include "cppa/config.hpp" #include "cppa/config.hpp"
#include "cppa/context.hpp" #include "cppa/context.hpp"
...@@ -40,7 +39,7 @@ void task_scheduler::worker_loop(job_queue* jq, scheduled_actor* dummy) ...@@ -40,7 +39,7 @@ void task_scheduler::worker_loop(job_queue* jq, scheduled_actor* dummy)
{ {
if (!job->deref()) delete job; if (!job->deref()) delete job;
CPPA_MEMORY_BARRIER(); CPPA_MEMORY_BARRIER();
actor_count::get().dec(); dec_actor_count();
}); });
} }
} }
...@@ -49,7 +48,7 @@ task_scheduler::task_scheduler() ...@@ -49,7 +48,7 @@ task_scheduler::task_scheduler()
: m_queue() : m_queue()
, m_dummy() , m_dummy()
{ {
m_worker = boost::thread(worker_loop, &m_queue, &m_dummy); m_worker = std::thread(worker_loop, &m_queue, &m_dummy);
} }
task_scheduler::~task_scheduler() task_scheduler::~task_scheduler()
...@@ -62,7 +61,7 @@ void task_scheduler::schedule(scheduled_actor* what) ...@@ -62,7 +61,7 @@ void task_scheduler::schedule(scheduled_actor* what)
{ {
if (what) if (what)
{ {
if (boost::this_thread::get_id() == m_worker.get_id()) if (std::this_thread::get_id() == m_worker.get_id())
{ {
m_queue._push_back(what); m_queue._push_back(what);
} }
...@@ -75,7 +74,7 @@ void task_scheduler::schedule(scheduled_actor* what) ...@@ -75,7 +74,7 @@ void task_scheduler::schedule(scheduled_actor* what)
actor_ptr task_scheduler::spawn(actor_behavior* behavior, scheduling_hint) actor_ptr task_scheduler::spawn(actor_behavior* behavior, scheduling_hint)
{ {
actor_count::get().inc(); inc_actor_count();
intrusive_ptr<scheduled_actor> ctx(new scheduled_actor(behavior, intrusive_ptr<scheduled_actor> ctx(new scheduled_actor(behavior,
enqueue_fun, enqueue_fun,
this)); this));
......
#include <cstdint> #include <cstdint>
#include <cstddef> #include <cstddef>
#include "cppa/detail/actor_count.hpp" #include "cppa/detail/actor_count.hpp"
#include "cppa/detail/mock_scheduler.hpp" #include "cppa/detail/mock_scheduler.hpp"
#include "cppa/detail/thread_pool_scheduler.hpp" #include "cppa/detail/thread_pool_scheduler.hpp"
...@@ -14,7 +15,7 @@ void enqueue_fun(cppa::detail::thread_pool_scheduler* where, ...@@ -14,7 +15,7 @@ void enqueue_fun(cppa::detail::thread_pool_scheduler* where,
where->schedule(what); where->schedule(what);
} }
typedef boost::unique_lock<boost::mutex> guard_type; typedef std::unique_lock<std::mutex> guard_type;
typedef std::unique_ptr<thread_pool_scheduler::worker> worker_ptr; typedef std::unique_ptr<thread_pool_scheduler::worker> worker_ptr;
typedef util::single_reader_queue<thread_pool_scheduler::worker> worker_queue; typedef util::single_reader_queue<thread_pool_scheduler::worker> worker_queue;
...@@ -28,9 +29,9 @@ struct thread_pool_scheduler::worker ...@@ -28,9 +29,9 @@ struct thread_pool_scheduler::worker
job_queue* m_job_queue; job_queue* m_job_queue;
scheduled_actor* m_job; scheduled_actor* m_job;
worker_queue* m_supervisor_queue; worker_queue* m_supervisor_queue;
boost::thread m_thread; std::thread m_thread;
boost::mutex m_mtx; std::mutex m_mtx;
boost::condition_variable m_cv; std::condition_variable m_cv;
worker(worker_queue* supervisor_queue, job_queue* jq) worker(worker_queue* supervisor_queue, job_queue* jq)
: next(nullptr), m_done(false), m_job_queue(jq), m_job(nullptr) : next(nullptr), m_done(false), m_job_queue(jq), m_job(nullptr)
...@@ -61,13 +62,13 @@ struct thread_pool_scheduler::worker ...@@ -61,13 +62,13 @@ struct thread_pool_scheduler::worker
} }
// run actor up to 300ms // run actor up to 300ms
bool reschedule = false; bool reschedule = false;
boost::system_time tout = boost::get_system_time(); auto tout = now();
tout += boost::posix_time::milliseconds(300); tout += std::chrono::milliseconds(300);
scheduled_actor::execute(m_job, scheduled_actor::execute(m_job,
fself, fself,
[&]() -> bool [&]() -> bool
{ {
if (tout >= boost::get_system_time()) if (tout >= now())
{ {
reschedule = true; reschedule = true;
return false; return false;
...@@ -78,7 +79,7 @@ struct thread_pool_scheduler::worker ...@@ -78,7 +79,7 @@ struct thread_pool_scheduler::worker
{ {
if (!m_job->deref()) delete m_job; if (!m_job->deref()) delete m_job;
CPPA_MEMORY_BARRIER(); CPPA_MEMORY_BARRIER();
actor_count::get().dec(); dec_actor_count();
}); });
if (reschedule) if (reschedule)
{ {
...@@ -103,13 +104,12 @@ void thread_pool_scheduler::supervisor_loop(job_queue* jqueue, ...@@ -103,13 +104,12 @@ void thread_pool_scheduler::supervisor_loop(job_queue* jqueue,
worker_queue wqueue; worker_queue wqueue;
std::vector<worker_ptr> workers; std::vector<worker_ptr> workers;
// init // init
size_t num_workers = std::max(boost::thread::hardware_concurrency(), size_t num_workers = std::max(std::thread::hardware_concurrency(),
static_cast<unsigned>(1)); static_cast<unsigned>(1));
for (size_t i = 0; i < num_workers; ++i) for (size_t i = 0; i < num_workers; ++i)
{ {
workers.push_back(worker_ptr(new worker(&wqueue, jqueue))); workers.push_back(worker_ptr(new worker(&wqueue, jqueue)));
} }
boost::system_time timeout;
bool done = false; bool done = false;
// loop // loop
do do
...@@ -124,8 +124,8 @@ void thread_pool_scheduler::supervisor_loop(job_queue* jqueue, ...@@ -124,8 +124,8 @@ void thread_pool_scheduler::supervisor_loop(job_queue* jqueue,
{ {
// fetch waiting worker (wait up to 500ms) // fetch waiting worker (wait up to 500ms)
worker* w = nullptr; worker* w = nullptr;
timeout = boost::get_system_time(); auto timeout = now();
timeout += boost::posix_time::milliseconds(500); timeout += std::chrono::milliseconds(500);
while (!w) while (!w)
{ {
w = wqueue.try_pop(timeout); w = wqueue.try_pop(timeout);
...@@ -187,7 +187,7 @@ actor_ptr thread_pool_scheduler::spawn(actor_behavior* behavior, ...@@ -187,7 +187,7 @@ actor_ptr thread_pool_scheduler::spawn(actor_behavior* behavior,
} }
else else
{ {
actor_count::get().inc(); inc_actor_count();
CPPA_MEMORY_BARRIER(); CPPA_MEMORY_BARRIER();
intrusive_ptr<scheduled_actor> ctx(new scheduled_actor(behavior, intrusive_ptr<scheduled_actor> ctx(new scheduled_actor(behavior,
enqueue_fun, enqueue_fun,
......
...@@ -3,11 +3,11 @@ ...@@ -3,11 +3,11 @@
#include <ios> // ios_base::failure #include <ios> // ios_base::failure
#include <list> #include <list>
#include <memory> #include <memory>
#include <cstring> // memset
#include <iostream> #include <iostream>
#include <stdexcept> #include <stdexcept>
#include <fcntl.h> #include <fcntl.h>
//#include <boost/thread.hpp>
#include "cppa/cppa.hpp" #include "cppa/cppa.hpp"
#include "cppa/atom.hpp" #include "cppa/atom.hpp"
...@@ -115,9 +115,6 @@ void publish(actor_ptr& whom, std::uint16_t port) ...@@ -115,9 +115,6 @@ void publish(actor_ptr& whom, std::uint16_t port)
// ok, no exceptions // ok, no exceptions
sguard.release(); sguard.release();
detail::post_office_publish(sockfd, whom); detail::post_office_publish(sockfd, whom);
//intrusive_ptr<shared_barrier> barrier_ptr(new shared_barrier);
//boost::thread(middle_man_loop, sockfd, whom, barrier_ptr).detach();
//whom->attach(new mm_handle(sockfd, barrier_ptr));
} }
void publish(actor_ptr&& whom, std::uint16_t port) void publish(actor_ptr&& whom, std::uint16_t port)
...@@ -167,8 +164,6 @@ actor_ptr remote_actor(const char* host, std::uint16_t port) ...@@ -167,8 +164,6 @@ actor_ptr remote_actor(const char* host, std::uint16_t port)
detail::post_office_add_peer(sockfd, pinfptr, result, detail::post_office_add_peer(sockfd, pinfptr, result,
std::unique_ptr<attachable>()); std::unique_ptr<attachable>());
//auto ptr = get_scheduler()->register_hidden_context(); //auto ptr = get_scheduler()->register_hidden_context();
//boost::thread(post_office_loop, sockfd,
// peer_pinf, result, ptr.release()).detach();
return result; return result;
} }
......
#include <boost/thread.hpp> #include <memory>
#include "cppa/detail/yield_interface.hpp" #include "cppa/detail/yield_interface.hpp"
namespace { namespace {
void cleanup_fun(cppa::util::fiber*) { } using namespace cppa;
boost::thread_specific_ptr<cppa::util::fiber> t_caller(cleanup_fun); __thread util::fiber* t_caller = nullptr;
boost::thread_specific_ptr<cppa::util::fiber> t_callee(cleanup_fun); __thread util::fiber* t_callee = nullptr;
boost::thread_specific_ptr<cppa::detail::yield_state> t_ystate; __thread detail::yield_state t_ystate = detail::yield_state::invalid;
} // namespace <anonymous> } // namespace <anonymous>
...@@ -16,27 +16,21 @@ namespace cppa { namespace detail { ...@@ -16,27 +16,21 @@ namespace cppa { namespace detail {
void yield(yield_state ystate) void yield(yield_state ystate)
{ {
if (t_ystate.get() == nullptr) t_ystate = ystate;
{
t_ystate.reset(new yield_state(ystate));
}
else
{
*t_ystate = ystate;
}
util::fiber::swap(*t_callee, *t_caller); util::fiber::swap(*t_callee, *t_caller);
} }
yield_state yielded_state() yield_state yielded_state()
{ {
return (t_ystate.get() == nullptr) ? yield_state::invalid : *t_ystate; return t_ystate;
} }
void call(util::fiber* what, util::fiber* from) void call(util::fiber* what, util::fiber* from)
{ {
t_ystate.reset(); t_ystate = yield_state::invalid;
t_caller.reset(from); t_caller = from;
t_callee.reset(what); //t_caller.reset(from);
t_callee = what;
util::fiber::swap(*from, *what); util::fiber::swap(*from, *what);
} }
......
#include <string> #include <string>
#include <sstream>
#include <iostream> #include <iostream>
#include <boost/thread.hpp>
#include "test.hpp" #include "test.hpp"
#include "ping_pong.hpp" #include "ping_pong.hpp"
#include "cppa/cppa.hpp" #include "cppa/cppa.hpp"
#include "cppa/exception.hpp" #include "cppa/exception.hpp"
#include "cppa/detail/thread.hpp"
using std::cout; using std::cout;
using std::endl; using std::endl;
...@@ -73,7 +74,7 @@ size_t test__remote_actor(const char* app_path, bool is_client, ...@@ -73,7 +74,7 @@ size_t test__remote_actor(const char* app_path, bool is_client,
} }
// execute client_part() in a separate process, // execute client_part() in a separate process,
// connected via localhost socket // connected via localhost socket
boost::thread child([&cmd]() { system(cmd.c_str()); }); detail::thread child([&cmd]() { system(cmd.c_str()); });
await_all_others_done(); await_all_others_done();
CPPA_CHECK_EQUAL(pongs(), 5); CPPA_CHECK_EQUAL(pongs(), 5);
// wait until separate process (in sep. thread) finished execution // wait until separate process (in sep. thread) finished execution
......
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment