MXS-2008 Move Worker and MessageQueue to maxbase

This commit is contained in:
Johan Wikman
2018-08-20 13:03:39 +03:00
parent 02ed338afa
commit 88f1795412
29 changed files with 196 additions and 199 deletions

View File

@ -16,14 +16,14 @@
#include <atomic>
#include <maxbase/semaphore.hh>
#include <maxbase/worker.hh>
#include <maxscale/monitor.h>
#include <maxscale/worker.hh>
namespace maxscale
{
class MonitorInstance : public MXS_MONITOR_INSTANCE
, protected maxscale::Worker
, protected maxbase::Worker
{
public:
MonitorInstance(const MonitorInstance&) = delete;

View File

@ -12,10 +12,10 @@
* Public License.
*/
#include <maxscale/ccdefs.hh>
#include <maxbase/ccdefs.hh>
#include <maxbase/poll.hh>
namespace maxscale
namespace maxbase
{
class MessageQueue;

View File

@ -12,7 +12,7 @@
* Public License.
*/
#include <maxscale/ccdefs.hh>
#include <maxbase/ccdefs.hh>
#include <cstring>
#include <functional>
@ -21,14 +21,14 @@
#include <thread>
#include <unordered_map>
#include <maxbase/assert.h>
#include <maxbase/atomic.h>
#include <maxbase/messagequeue.hh>
#include <maxbase/semaphore.hh>
#include <maxbase/worker.h>
#include <maxscale/debug.h>
#include <maxscale/messagequeue.hh>
#include <maxscale/workertask.hh>
#include <maxbase/workertask.hh>
namespace maxscale
namespace maxbase
{
struct WORKER_STATISTICS
@ -155,7 +155,7 @@ public:
return m_load_1_hour.value();
default:
ss_dassert(!true);
mxb_assert(!true);
return 0;
};
}
@ -371,8 +371,8 @@ private:
private:
uint8_t* prev(uint8_t* p)
{
ss_dassert(p >= m_begin);
ss_dassert(p < m_end);
mxb_assert(p >= m_begin);
mxb_assert(p < m_end);
if (p > m_begin)
{
@ -380,20 +380,20 @@ private:
}
else
{
ss_dassert(p == m_begin);
mxb_assert(p == m_begin);
p = m_end - 1;
}
ss_dassert(p >= m_begin);
ss_dassert(p < m_end);
mxb_assert(p >= m_begin);
mxb_assert(p < m_end);
return p;
}
uint8_t* next(uint8_t* p)
{
ss_dassert(p >= m_begin);
ss_dassert(p < m_end);
mxb_assert(p >= m_begin);
mxb_assert(p < m_end);
++p;
@ -402,8 +402,8 @@ private:
p = m_begin;
}
ss_dassert(p >= m_begin);
ss_dassert(p < m_end);
mxb_assert(p >= m_begin);
mxb_assert(p < m_end);
return p;
}
@ -1044,7 +1044,7 @@ private:
, m_delay(delay)
, m_at(get_at(delay))
{
ss_dassert(delay > 0);
mxb_assert(delay > 0);
}
virtual bool do_call(Worker::Call::action_t action) = 0;
@ -1052,11 +1052,11 @@ private:
private:
static int64_t get_at(int32_t delay)
{
ss_dassert(delay > 0);
mxb_assert(delay > 0);
struct timespec ts;
ss_debug(int rv =) clock_gettime(CLOCK_MONOTONIC, &ts);
ss_dassert(rv == 0);
MXB_AT_DEBUG(int rv =) clock_gettime(CLOCK_MONOTONIC, &ts);
mxb_assert(rv == 0);
return delay + (ts.tv_sec * 1000 + ts.tv_nsec / 1000000);
}

View File

@ -12,9 +12,9 @@
* Public License.
*/
#include <maxscale/ccdefs.hh>
#include <maxbase/ccdefs.hh>
namespace maxscale
namespace maxbase
{
class Worker;

View File

@ -3,10 +3,13 @@ add_library(maxbase STATIC
eventcount.cc
log.cc
logger.cc
messagequeue.cc
semaphore.cc
stopwatch.cc
string.cc
stacktrace.cc
worker.cc
workertask.cc
)
set_target_properties(maxbase PROPERTIES VERSION "1.0.0" LINK_FLAGS -Wl,-z,defs)

View File

@ -11,15 +11,16 @@
* Public License.
*/
#include <maxscale/messagequeue.hh>
#include <maxbase/messagequeue.hh>
#include <errno.h>
#include <fcntl.h>
#include <string.h>
#include <unistd.h>
#include <fstream>
#include <maxscale/debug.h>
#include <maxscale/log.h>
#include "internal/routingworker.hh"
#include <maxbase/assert.h>
#include <maxbase/log.h>
#include <maxbase/string.h>
#include <maxbase/worker.hh>
namespace
{
@ -48,7 +49,7 @@ int get_pipe_max_size()
}
namespace maxscale
namespace maxbase
{
MessageQueue::MessageQueue(Handler* pHandler, int read_fd, int write_fd)
@ -58,9 +59,9 @@ MessageQueue::MessageQueue(Handler* pHandler, int read_fd, int write_fd)
, m_write_fd(write_fd)
, m_pWorker(NULL)
{
ss_dassert(pHandler);
ss_dassert(read_fd);
ss_dassert(write_fd);
mxb_assert(pHandler);
mxb_assert(read_fd);
mxb_assert(write_fd);
}
MessageQueue::~MessageQueue()
@ -77,7 +78,7 @@ MessageQueue::~MessageQueue()
//static
bool MessageQueue::init()
{
ss_dassert(!this_unit.initialized);
mxb_assert(!this_unit.initialized);
this_unit.initialized = true;
this_unit.pipe_max_size = get_pipe_max_size();
@ -88,14 +89,14 @@ bool MessageQueue::init()
//static
void MessageQueue::finish()
{
ss_dassert(this_unit.initialized);
mxb_assert(this_unit.initialized);
this_unit.initialized = false;
}
//static
MessageQueue* MessageQueue::create(Handler* pHandler)
{
ss_dassert(this_unit.initialized);
mxb_assert(this_unit.initialized);
/* From "man 7 pipe"
* ----
@ -130,7 +131,7 @@ MessageQueue* MessageQueue::create(Handler* pHandler)
if (rv == 0)
{
// Succeeded, so apparently it was the missing support for O_DIRECT.
MXS_WARNING("Platform does not support O_DIRECT in conjunction with pipes, "
MXB_WARNING("Platform does not support O_DIRECT in conjunction with pipes, "
"using without.");
}
}
@ -146,22 +147,22 @@ MessageQueue* MessageQueue::create(Handler* pHandler)
*/
if (fcntl(fds[0], F_SETPIPE_SZ, this_unit.pipe_max_size) == -1)
{
MXS_WARNING("Failed to increase pipe buffer size to '%d': %d, %s",
this_unit.pipe_max_size, errno, mxs_strerror(errno));
MXB_WARNING("Failed to increase pipe buffer size to '%d': %d, %s",
this_unit.pipe_max_size, errno, mxb_strerror(errno));
}
#endif
pThis = new (std::nothrow) MessageQueue(pHandler, read_fd, write_fd);
if (!pThis)
{
MXS_OOM();
MXB_OOM();
close(read_fd);
close(write_fd);
}
}
else
{
MXS_ERROR("Could not create pipe for worker: %s", mxs_strerror(errno));
MXB_ERROR("Could not create pipe for worker: %s", mxb_strerror(errno));
}
return pThis;
@ -172,7 +173,7 @@ bool MessageQueue::post(const Message& message) const
// NOTE: No logging here, this function must be signal safe.
bool rv = false;
ss_dassert(m_pWorker);
mxb_assert(m_pWorker);
if (m_pWorker)
{
/**
@ -217,20 +218,20 @@ bool MessageQueue::post(const Message& message) const
if (n == -1)
{
MXS_ERROR("Failed to write message: %d, %s", errno, mxs_strerror(errno));
MXB_ERROR("Failed to write message: %d, %s", errno, mxb_strerror(errno));
static bool warn_pipe_buffer_size = true;
if ((errno == EAGAIN || errno == EWOULDBLOCK) && warn_pipe_buffer_size)
{
MXS_ERROR("Consider increasing pipe buffer size (sysctl fs.pipe-max-size)");
MXB_ERROR("Consider increasing pipe buffer size (sysctl fs.pipe-max-size)");
warn_pipe_buffer_size = false;
}
}
}
else
{
MXS_ERROR("Attempt to post using a message queue that is not added to a worker.");
MXB_ERROR("Attempt to post using a message queue that is not added to a worker.");
}
return rv;
@ -269,10 +270,10 @@ uint32_t MessageQueue::handle_poll_events(Worker* pWorker, uint32_t events)
{
uint32_t rc = MXB_POLL_NOP;
ss_dassert(pWorker == m_pWorker);
mxb_assert(pWorker == m_pWorker);
// We only expect EPOLLIN events.
ss_dassert(((events & EPOLLIN) != 0) && ((events & ~EPOLLIN) == 0));
mxb_assert(((events & EPOLLIN) != 0) && ((events & ~EPOLLIN) == 0));
if (events & EPOLLIN)
{
@ -292,7 +293,7 @@ uint32_t MessageQueue::handle_poll_events(Worker* pWorker, uint32_t events)
{
if (errno != EWOULDBLOCK)
{
MXS_ERROR("Worker could not read from pipe: %s", mxs_strerror(errno));
MXB_ERROR("Worker could not read from pipe: %s", mxb_strerror(errno));
}
}
else if (n != 0)
@ -301,9 +302,9 @@ uint32_t MessageQueue::handle_poll_events(Worker* pWorker, uint32_t events)
// should either get a message, nothing at all or an error. In non-debug
// mode we continue reading in order to empty the pipe as otherwise the
// thread may hang.
MXS_ERROR("MessageQueue could only read %ld bytes from pipe, although "
MXB_ERROR("MessageQueue could only read %ld bytes from pipe, although "
"expected %lu bytes.", n, sizeof(message));
ss_dassert(!true);
mxb_assert(!true);
}
}
while ((n != 0) && (n != -1));

View File

@ -5,3 +5,7 @@ add_test(test_mxb_log test_mxb_log)
add_executable(test_semaphore test_semaphore.cc)
target_link_libraries(test_semaphore maxbase pthread)
add_test(test_semaphore test_semaphore)
add_executable(test_worker test_worker.cc)
target_link_libraries(test_worker maxbase pthread)
add_test(test_worker test_worker)

View File

@ -12,10 +12,11 @@
*/
#include <iostream>
#include <maxscale/worker.hh>
#include "../internal/poll.hh"
#include <maxbase/assert.h>
#include <maxbase/log.hh>
#include <maxbase/worker.hh>
using namespace maxscale;
using namespace maxbase;
using namespace std;
namespace
@ -25,8 +26,8 @@ namespace
int64_t get_monotonic_time_ms()
{
struct timespec ts;
ss_debug(int rv =) clock_gettime(CLOCK_MONOTONIC, &ts);
ss_dassert(rv == 0);
MXB_AT_DEBUG(int rv =) clock_gettime(CLOCK_MONOTONIC, &ts);
mxb_assert(rv == 0);
return ts.tv_sec * 1000 + ts.tv_nsec / 1000000;
}
@ -50,11 +51,11 @@ public:
return m_delay;
}
bool tick(mxs::Worker::Call::action_t action)
bool tick(Worker::Call::action_t action)
{
bool rv = false;
if (action == mxs::Worker::Call::EXECUTE)
if (action == Worker::Call::EXECUTE)
{
int64_t now = get_monotonic_time_ms();
int64_t diff = abs(now - m_at);
@ -122,18 +123,10 @@ int run()
int main()
{
int rv = EXIT_FAILURE;
mxb::Log log(MXB_LOG_TARGET_STDOUT);
if (mxs_log_init(NULL, NULL, MXS_LOG_TARGET_STDOUT))
{
poll_init();
maxscale::MessageQueue::init();
maxscale::Worker::init();
maxbase::MessageQueue::init();
maxbase::Worker::init();
rv = run();
mxs_log_finish();
}
return rv;
return run();
}

View File

@ -11,7 +11,7 @@
* Public License.
*/
#include <maxscale/worker.hh>
#include <maxbase/worker.hh>
#include <errno.h>
#include <fcntl.h>
@ -22,7 +22,9 @@
#include <sstream>
#include <sys/timerfd.h>
#include <maxscale/log.h>
#include <maxbase/assert.h>
#include <maxbase/log.h>
#include <maxbase/string.h>
#define WORKER_ABSENT_ID -1
@ -33,7 +35,7 @@ using std::stringstream;
namespace
{
using maxscale::Worker;
using maxbase::Worker;
const int MXB_WORKER_MSG_TASK = -1;
const int MXB_WORKER_MSG_DISPOSABLE_TASK = -2;
@ -72,7 +74,7 @@ typedef struct worker_message
static bool modules_thread_init();
static void modules_thread_finish();
namespace maxscale
namespace maxbase
{
WorkerLoad::WorkerLoad()
@ -112,8 +114,8 @@ uint64_t WorkerLoad::get_time()
timespec t;
ss_debug(int rv = )clock_gettime(CLOCK_MONOTONIC, &t);
ss_dassert(rv == 0);
MXB_AT_DEBUG(int rv = )clock_gettime(CLOCK_MONOTONIC, &t);
mxb_assert(rv == 0);
return t.tv_sec * 1000 + (t.tv_nsec / 1000000);
}
@ -140,34 +142,34 @@ int create_timerfd()
flags |= O_NONBLOCK;
if (fcntl(fd, F_SETFL, flags) == -1)
{
MXS_ALERT("Could not make timer fd non-blocking, MaxScale will not work: %s",
mxs_strerror(errno));
MXB_ALERT("Could not make timer fd non-blocking, system will not work: %s",
mxb_strerror(errno));
close(fd);
fd = -1;
ss_dassert(!true);
mxb_assert(!true);
}
}
else
{
MXS_ALERT("Could not get timer fd flags, MaxScale will not work: %s",
mxs_strerror(errno));
MXB_ALERT("Could not get timer fd flags, system will not work: %s",
mxb_strerror(errno));
close(fd);
fd = -1;
ss_dassert(!true);
mxb_assert(!true);
}
}
else
{
MXS_ALERT("Could not create timer file descriptor even with no flags, MaxScale "
"will not work: %s", mxs_strerror(errno));
ss_dassert(!true);
MXB_ALERT("Could not create timer file descriptor even with no flags, system "
"will not work: %s", mxb_strerror(errno));
mxb_assert(!true);
}
}
else
{
MXS_ALERT("Could not create timer file descriptor, MaxScale will not work: %s",
mxs_strerror(errno));
ss_dassert(!true);
MXB_ALERT("Could not create timer file descriptor, system will not work: %s",
mxb_strerror(errno));
mxb_assert(!true);
}
}
@ -187,10 +189,10 @@ WorkerTimer::WorkerTimer(Worker* pWorker)
{
if (!m_pWorker->add_fd(m_fd, EPOLLIN, this))
{
MXS_ALERT("Could not add timer descriptor to worker, MaxScale will not work.");
MXB_ALERT("Could not add timer descriptor to worker, system will not work.");
::close(m_fd);
m_fd = -1;
ss_dassert(!true);
mxb_assert(!true);
}
}
}
@ -201,7 +203,7 @@ WorkerTimer::~WorkerTimer()
{
if (!m_pWorker->remove_fd(m_fd))
{
MXS_ERROR("Could not remove timer fd from worker.");
MXB_ERROR("Could not remove timer fd from worker.");
}
::close(m_fd);
@ -210,7 +212,7 @@ WorkerTimer::~WorkerTimer()
void WorkerTimer::start(int32_t interval)
{
ss_dassert(interval >= 0);
mxb_assert(interval >= 0);
// TODO: Add possibility to set initial delay and interval.
time_t initial_sec = interval / 1000;
@ -228,7 +230,7 @@ void WorkerTimer::start(int32_t interval)
if (timerfd_settime(m_fd, 0, &time, NULL) != 0)
{
MXS_ERROR("Could not set timer settings.");
MXB_ERROR("Could not set timer settings.");
}
}
@ -239,9 +241,9 @@ void WorkerTimer::cancel()
uint32_t WorkerTimer::handle(Worker* pWorker, uint32_t events)
{
ss_dassert(pWorker == m_pWorker);
ss_dassert(events & EPOLLIN);
ss_dassert((events & ~EPOLLIN) == 0);
mxb_assert(pWorker == m_pWorker);
mxb_assert(events & EPOLLIN);
mxb_assert((events & ~EPOLLIN) == 0);
// Read all events
uint64_t expirations;
@ -271,9 +273,9 @@ int create_epoll_instance()
if (fd == -1)
{
MXS_ALERT("Could not create epoll-instance for worker, MaxScale will not work: %s",
mxs_strerror(errno));
ss_dassert(!true);
MXB_ALERT("Could not create epoll-instance for worker, system will not work: %s",
mxb_strerror(errno));
mxb_assert(!true);
}
return fd;
@ -296,7 +298,7 @@ Worker::Worker(int max_events)
, m_nTotal_descriptors(0)
, m_pTimer(new PrivateTimer(this, this, &Worker::tick))
{
ss_dassert(max_events > 0);
mxb_assert(max_events > 0);
if (m_epoll_fd != -1)
{
@ -306,21 +308,21 @@ Worker::Worker(int max_events)
{
if (!m_pQueue->add_to_worker(this))
{
MXS_ALERT("Could not add message queue to worker, MaxScale will not work.");
ss_dassert(!true);
MXB_ALERT("Could not add message queue to worker, system will not work.");
mxb_assert(!true);
}
}
else
{
MXS_ALERT("Could not create message queue for worker, MaxScale will not work.");
ss_dassert(!true);
MXB_ALERT("Could not create message queue for worker, system will not work.");
mxb_assert(!true);
}
}
}
Worker::~Worker()
{
ss_dassert(!m_started);
mxb_assert(!m_started);
delete m_pTimer;
delete m_pQueue;
@ -337,7 +339,7 @@ Worker::~Worker()
// static
bool Worker::init()
{
ss_dassert(!this_unit.initialized);
mxb_assert(!this_unit.initialized);
this_unit.initialized = true;
@ -346,7 +348,7 @@ bool Worker::init()
void Worker::finish()
{
ss_dassert(this_unit.initialized);
mxb_assert(this_unit.initialized);
this_unit.initialized = false;
}
@ -470,7 +472,7 @@ bool Worker::post_disposable(DisposableTask* pTask, enum execute_mode_t mode)
bool Worker::execute(function<void ()> func, mxb::Semaphore* pSem, execute_mode_t mode)
{
class CustomTask : public maxscale::WorkerTask
class CustomTask : public Task
{
public:
@ -482,7 +484,7 @@ bool Worker::execute(function<void ()> func, mxb::Semaphore* pSem, execute_mode_
private:
function<void ()> m_func;
void execute(maxscale::Worker& worker)
void execute(maxbase::Worker& worker)
{
m_func();
@ -528,7 +530,7 @@ bool Worker::post_message(uint32_t msg_id, intptr_t arg1, intptr_t arg2)
void Worker::run(mxb::Semaphore* pSem)
{
ss_dassert(m_state == STOPPED);
mxb_assert(m_state == STOPPED);
this_thread.pCurrent_worker = this;
if (pre_run())
@ -545,7 +547,7 @@ void Worker::run(mxb::Semaphore* pSem)
m_state = STOPPED;
post_run();
MXS_INFO("Worker %p has shut down.", this);
MXB_INFO("Worker %p has shut down.", this);
}
else if (pSem)
{
@ -557,8 +559,8 @@ void Worker::run(mxb::Semaphore* pSem)
bool Worker::start()
{
ss_dassert(!m_started);
ss_dassert(m_thread.get_id() == std::thread::id());
mxb_assert(!m_started);
mxb_assert(m_thread.get_id() == std::thread::id());
mxb::Semaphore sem;
m_started = true;
@ -572,7 +574,7 @@ bool Worker::start()
}
catch (const std::exception& x)
{
MXS_ERROR("Could not start worker thread: %s", x.what());
MXB_ERROR("Could not start worker thread: %s", x.what());
m_started = false;
}
@ -581,13 +583,13 @@ bool Worker::start()
void Worker::join()
{
ss_dassert(m_thread.get_id() != std::thread::id());
mxb_assert(m_thread.get_id() != std::thread::id());
if (m_started)
{
MXS_INFO("Waiting for worker %p.", this);
MXB_INFO("Waiting for worker %p.", this);
m_thread.join();
MXS_INFO("Waited for worker %p.", this);
MXB_INFO("Waited for worker %p.", this);
m_started = false;
}
}
@ -618,7 +620,7 @@ void Worker::handle_message(MessageQueue& queue, const MessageQueue::Message& ms
{
case MXB_WORKER_MSG_SHUTDOWN:
{
MXS_INFO("Worker %p received shutdown message.", this);
MXB_INFO("Worker %p received shutdown message.", this);
m_should_shutdown = true;
}
break;
@ -654,7 +656,7 @@ void Worker::handle_message(MessageQueue& queue, const MessageQueue::Message& ms
break;
default:
MXS_ERROR("Worker received unknown message %d.", msg.id());
MXB_ERROR("Worker received unknown message %d.", msg.id());
}
}
@ -689,30 +691,30 @@ void Worker::resolve_poll_error(int fd, int errornum, int op)
{
if (EEXIST == errornum)
{
MXS_ERROR("File descriptor %d already present in an epoll instance.", fd);
MXB_ERROR("File descriptor %d already present in an epoll instance.", fd);
return;
}
if (ENOSPC == errornum)
{
MXS_ERROR("The limit imposed by /proc/sys/fs/epoll/max_user_watches was "
MXB_ERROR("The limit imposed by /proc/sys/fs/epoll/max_user_watches was "
"reached when trying to add file descriptor %d to an epoll instance.", fd);
return;
}
}
else
{
ss_dassert(op == EPOLL_CTL_DEL);
mxb_assert(op == EPOLL_CTL_DEL);
/* Must be removing */
if (ENOENT == errornum)
{
MXS_ERROR("File descriptor %d was not found in epoll instance.", fd);
MXB_ERROR("File descriptor %d was not found in epoll instance.", fd);
return;
}
}
/* Common checks for add or remove - crash MaxScale */
/* Common checks for add or remove - crash system */
if (EBADF == errornum)
{
raise(SIGABRT);
@ -787,7 +789,7 @@ void Worker::poll_waitevents()
{
int eno = errno;
errno = 0;
MXS_ERROR("%lu [poll_waitevents] epoll_wait returned "
MXB_ERROR("%lu [poll_waitevents] epoll_wait returned "
"%d, errno %d",
pthread_self(),
nfds,
@ -813,7 +815,7 @@ void Worker::poll_waitevents()
m_statistics.evq_max = nfds;
}
MXS_DEBUG("%lu [poll_waitevents] epoll_wait found %d fds",
MXB_DEBUG("%lu [poll_waitevents] epoll_wait found %d fds",
pthread_self(),
nfds);
atomic_add_int64(&m_statistics.n_pollev, 1);
@ -840,7 +842,7 @@ void Worker::poll_waitevents()
m_statistics.qtimes[qtime]++;
}
m_statistics.maxqtime = MXS_MAX(m_statistics.maxqtime, qtime);
m_statistics.maxqtime = std::max(m_statistics.maxqtime, qtime);
MXB_POLL_DATA *data = (MXB_POLL_DATA*)events[i].data.ptr;
@ -883,7 +885,7 @@ void Worker::poll_waitevents()
m_statistics.exectimes[qtime]++;
}
m_statistics.maxexectime = MXS_MAX(m_statistics.maxexectime, qtime);
m_statistics.maxexectime = std::max(m_statistics.maxexectime, qtime);
}
epoll_tick();
@ -898,8 +900,8 @@ namespace
int64_t get_current_time_ms()
{
struct timespec ts;
ss_debug(int rv = ) clock_gettime(CLOCK_MONOTONIC, &ts);
ss_dassert(rv == 0);
MXB_AT_DEBUG(int rv = ) clock_gettime(CLOCK_MONOTONIC, &ts);
mxb_assert(rv == 0);
return ts.tv_sec * 1000 + ts.tv_nsec / 1000000;
}
@ -920,7 +922,7 @@ void Worker::tick()
DelayedCall* pCall = i->second;
auto j = m_calls.find(pCall->id());
ss_dassert(j != m_calls.end());
mxb_assert(j != m_calls.end());
m_sorted_calls.erase(i);
m_calls.erase(j);
@ -971,7 +973,7 @@ uint32_t Worker::add_delayed_call(DelayedCall* pCall)
m_sorted_calls.insert(std::make_pair(pCall->at(), pCall));
// Insert the delayed call into the map indexed by id.
ss_dassert(m_calls.find(pCall->id()) == m_calls.end());
mxb_assert(m_calls.find(pCall->id()) == m_calls.end());
m_calls.insert(std::make_pair(pCall->id(), pCall));
if (adjust)
@ -1020,7 +1022,7 @@ bool Worker::cancel_delayed_call(uint32_t id)
auto range = m_sorted_calls.equal_range(pCall->at());
auto k = range.first;
ss_dassert(k != range.second);
mxb_assert(k != range.second);
while (k != range.second)
{
@ -1039,12 +1041,12 @@ bool Worker::cancel_delayed_call(uint32_t id)
}
}
ss_dassert(found);
mxb_assert(found);
}
else
{
ss_dassert(!true);
MXS_WARNING("Attempt to remove a delayed call, associated with non-existing id.");
mxb_assert(!true);
MXB_WARNING("Attempt to remove a delayed call, associated with non-existing id.");
}
return found;

View File

@ -11,11 +11,11 @@
* Public License.
*/
#include <maxscale/workertask.hh>
#include <maxbase/workertask.hh>
#include <maxbase/atomic.h>
#include <maxscale/debug.h>
#include <maxbase/assert.h>
namespace maxscale
namespace maxbase
{
//
@ -44,7 +44,7 @@ void WorkerDisposableTask::inc_ref()
void WorkerDisposableTask::dec_ref()
{
ss_dassert(atomic_load_int32(&m_count) > 0);
mxb_assert(atomic_load_int32(&m_count) > 0);
if (atomic_add(&m_count, -1) == 1)
{

View File

@ -24,7 +24,6 @@ add_library(maxscale-common SHARED
mariadb.cc
maxscale_pcre2.cc
misc.cc
messagequeue.cc
modulecmd.cc
modutil.cc
monitor.cc
@ -49,8 +48,6 @@ add_library(maxscale-common SHARED
statistics.cc
users.cc
utils.cc
worker.cc
workertask.cc
)
if(WITH_JEMALLOC)

View File

@ -51,15 +51,13 @@
#include <maxscale/service.h>
#include <maxscale/spinlock.h>
#include <maxscale/utils.h>
#include <maxscale/workertask.hh>
#include "internal/modules.h"
#include "internal/routingworker.hh"
#include "internal/session.h"
using maxscale::RoutingWorker;
using maxscale::Worker;
using maxscale::WorkerTask;
using maxbase::Worker;
//#define DCB_LOG_EVENT_HANDLING
#if defined(DCB_LOG_EVENT_HANDLING)
@ -2892,7 +2890,7 @@ void dcb_process_idle_sessions(int thr)
}
/** Helper class for serial iteration over all DCBs */
class SerialDcbTask : public WorkerTask
class SerialDcbTask : public Worker::Task
{
public:
@ -3228,7 +3226,7 @@ static bool dcb_is_still_valid(DCB* target, int id)
return rval;
}
class FakeEventTask: public mxs::WorkerDisposableTask
class FakeEventTask: public Worker::DisposableTask
{
FakeEventTask(const FakeEventTask&);
FakeEventTask& operator=(const FakeEventTask&);
@ -3290,7 +3288,7 @@ static void poll_add_event_to_dcb(DCB* dcb, GWBUF* buf, uint32_t ev)
if (task)
{
RoutingWorker* worker = static_cast<RoutingWorker*>(dcb->poll.owner);
worker->execute(std::unique_ptr<FakeEventTask>(task), mxs::Worker::EXECUTE_QUEUED);
worker->execute(std::unique_ptr<FakeEventTask>(task), Worker::EXECUTE_QUEUED);
}
else
{
@ -3372,7 +3370,7 @@ static inline void dcb_sanity_check(DCB* dcb)
namespace
{
class AddDcbToWorker: public mxs::WorkerDisposableTask
class AddDcbToWorker: public Worker::DisposableTask
{
public:
AddDcbToWorker(const AddDcbToWorker&) = delete;
@ -3502,7 +3500,7 @@ static bool dcb_add_to_worker(Worker* worker, DCB* dcb, uint32_t events)
Worker* worker = static_cast<RoutingWorker*>(dcb->poll.owner);
ss_dassert(worker);
if (worker->execute(std::unique_ptr<AddDcbToWorker>(task), mxs::Worker::EXECUTE_QUEUED))
if (worker->execute(std::unique_ptr<AddDcbToWorker>(task), Worker::EXECUTE_QUEUED))
{
rv = true;
}

View File

@ -39,6 +39,7 @@
#include <sys/types.h>
#include <sys/wait.h>
#include <maxbase/messagequeue.hh>
#include <maxbase/stacktrace.hh>
#include <maxscale/alloc.h>
#include <maxscale/adminusers.h>
@ -46,7 +47,6 @@
#include <maxscale/housekeeper.h>
#include <maxscale/log.h>
#include <maxscale/maxscale.h>
#include <maxscale/messagequeue.hh>
#include <maxscale/mysql_utils.h>
#include <maxscale/paths.h>
#include <maxscale/query_classifier.h>
@ -1337,7 +1337,7 @@ int main(int argc, char **argv)
bool to_stdout = false;
int numlocks = 0;
bool pid_file_created = false;
Worker* worker;
mxb::Worker* worker;
const char* specified_user = NULL;
char export_cnf[PATH_MAX + 1] = "";
@ -1941,14 +1941,14 @@ int main(int argc, char **argv)
MXS_NOTICE("Module directory: %s", get_libdir());
MXS_NOTICE("Service cache: %s", get_cachedir());
if (!MessageQueue::init())
if (!mxb::MessageQueue::init())
{
MXS_ERROR("Failed to initialize message queue.");
rc = MAXSCALE_INTERNALERROR;
goto return_main;
}
if (!Worker::init())
if (!mxb::Worker::init())
{
MXS_ERROR("Failed to initialize workers.");
rc = MAXSCALE_INTERNALERROR;
@ -2193,8 +2193,8 @@ int main(int argc, char **argv)
service_destroy_instances();
RoutingWorker::finish();
Worker::finish();
MessageQueue::finish();
mxb::Worker::finish();
mxb::MessageQueue::finish();
/*< Call finish on all modules. */
modules_process_finish();

View File

@ -17,17 +17,17 @@
#include <unordered_map>
#include <maxbase/semaphore.hh>
#include <maxbase/worker.hh>
#include <maxscale/poll.h>
#include <maxscale/query_classifier.h>
#include <maxscale/routingworker.h>
#include <maxscale/worker.hh>
#include "session.hh"
namespace maxscale
{
class RoutingWorker : public Worker
class RoutingWorker : public mxb::Worker
, private MXB_POLL_DATA
{
RoutingWorker(const RoutingWorker&) = delete;

View File

@ -37,7 +37,7 @@
#include "internal/poll.hh"
#include "internal/routingworker.hh"
using maxscale::Worker;
using maxbase::Worker;
using maxscale::RoutingWorker;
static int n_threads; /*< Number of threads */

View File

@ -1275,13 +1275,13 @@ static HttpResponse handle_request(const HttpRequest& request)
HttpResponse resource_handle_request(const HttpRequest& request)
{
mxs::Worker* worker = mxs::RoutingWorker::get(mxs::RoutingWorker::MAIN);
mxb::Worker* worker = mxs::RoutingWorker::get(mxs::RoutingWorker::MAIN);
HttpResponse response;
worker->call([&request, &response]()
{
response = handle_request(request);
}, mxs::Worker::EXECUTE_AUTO);
}, mxb::Worker::EXECUTE_AUTO);
return response;
}

View File

@ -39,8 +39,9 @@
#define WORKER_ABSENT_ID -1
using maxbase::Semaphore;
using maxbase::Worker;
using maxbase::WorkerLoad;
using maxscale::RoutingWorker;
using maxscale::WorkerLoad;
using maxscale::Closer;
using std::vector;
using std::stringstream;
@ -866,7 +867,7 @@ int64_t RoutingWorker::get_one_statistic(POLL_STAT what)
//static
bool RoutingWorker::get_qc_stats(int id, QC_CACHE_STATS* pStats)
{
class Task : public mxs::Worker::Task
class Task : public Worker::Task
{
public:
Task(QC_CACHE_STATS* pStats)
@ -874,7 +875,7 @@ bool RoutingWorker::get_qc_stats(int id, QC_CACHE_STATS* pStats)
{
}
void execute(mxs::Worker&)
void execute(Worker&)
{
qc_get_cache_stats(&m_stats);
}
@ -899,7 +900,7 @@ bool RoutingWorker::get_qc_stats(int id, QC_CACHE_STATS* pStats)
//static
void RoutingWorker::get_qc_stats(std::vector<QC_CACHE_STATS>& all_stats)
{
class Task : public mxs::Worker::Task
class Task : public Worker::Task
{
public:
Task(std::vector<QC_CACHE_STATS>* pAll_stats)
@ -908,7 +909,7 @@ void RoutingWorker::get_qc_stats(std::vector<QC_CACHE_STATS>& all_stats)
m_all_stats.resize(config_threadcount());
}
void execute(mxs::Worker& worker)
void execute(Worker& worker)
{
int id = mxs::RoutingWorker::get_current_id();
ss_dassert(id >= 0);
@ -1051,7 +1052,7 @@ namespace
using namespace maxscale;
class WorkerInfoTask: public maxscale::WorkerTask
class WorkerInfoTask: public Worker::Task
{
public:
WorkerInfoTask(const char* zHost, uint32_t nThreads)
@ -1139,7 +1140,7 @@ private:
const char* m_zHost;
};
class FunctionTask: public maxscale::WorkerDisposableTask
class FunctionTask: public Worker::DisposableTask
{
public:
FunctionTask(std::function<void ()> cb):
@ -1200,7 +1201,7 @@ json_t* mxs_rworker_to_json(const char* zHost, int id)
WorkerInfoTask task(zHost, id + 1);
Semaphore sem;
target->execute(&task, &sem, mxs::Worker::EXECUTE_AUTO);
target->execute(&task, &sem, Worker::EXECUTE_AUTO);
sem.wait();
return task.resource(id);

View File

@ -54,9 +54,8 @@
#include "internal/modules.h"
using maxbase::Worker;
using maxscale::RoutingWorker;
using maxscale::Worker;
using maxscale::WorkerTask;
using std::string;
using Guard = std::lock_guard<std::mutex>;
@ -459,7 +458,7 @@ dprintAllServersJson(DCB *dcb)
/**
* A class for cleaning up persistent connections
*/
class CleanupTask : public WorkerTask
class CleanupTask : public Worker::Task
{
public:
CleanupTask(const SERVER* server):

View File

@ -48,6 +48,7 @@
using std::string;
using std::stringstream;
using maxbase::Worker;
using namespace maxscale;
/** Global session id counter. Must be updated atomically. Value 0 is reserved for
@ -323,7 +324,7 @@ void session_close(MXS_SESSION *session)
}
}
class ServiceDestroyTask: public mxs::WorkerDisposableTask
class ServiceDestroyTask: public Worker::DisposableTask
{
public:
ServiceDestroyTask(Service* service):
@ -358,7 +359,7 @@ static void session_free(MXS_SESSION *session)
// Destroy the service in the main routing worker thread
mxs::RoutingWorker* main_worker = mxs::RoutingWorker::get(mxs::RoutingWorker::MAIN);
main_worker->execute(std::unique_ptr<ServiceDestroyTask>(new ServiceDestroyTask(service)),
mxs::Worker::EXECUTE_AUTO);
Worker::EXECUTE_AUTO);
}
}

View File

@ -26,7 +26,6 @@ add_executable(test_trxtracking test_trxtracking.cc)
add_executable(test_users test_users.cc)
add_executable(test_utils test_utils.cc)
add_executable(test_session_track test_session_track.cc)
add_executable(test_worker test_worker.cc)
target_link_libraries(profile_trxboundaryparser maxscale-common)
target_link_libraries(test_adminusers maxscale-common)
@ -56,7 +55,6 @@ target_link_libraries(test_trxtracking maxscale-common)
target_link_libraries(test_users maxscale-common)
target_link_libraries(test_utils maxscale-common)
target_link_libraries(test_session_track mysqlcommon)
target_link_libraries(test_worker maxscale-common)
add_test(test_adminusers test_adminusers)
add_test(test_atomic test_atomic)
@ -92,7 +90,6 @@ add_test(test_trxtracking test_trxtracking)
add_test(test_users test_users)
add_test(test_utils test_utils)
add_test(test_session_track test_session_track)
add_test(test_worker test_worker)
add_subdirectory(rest-api)
add_subdirectory(canonical_tests)

View File

@ -36,7 +36,6 @@
#include <maxscale/config.h>
#include <maxscale/listener.h>
#include <maxscale/messagequeue.hh>
#include "../internal/routingworker.hh"
#include "../dcb.cc"

View File

@ -15,9 +15,9 @@
* Test modulecmd.h functionality
*/
#include <maxbase/messagequeue.hh>
#include <maxscale/alloc.h>
#include <maxscale/dcb.h>
#include <maxscale/messagequeue.hh>
#include <maxscale/paths.h>
#include <maxscale/modulecmd.h>
#include <maxscale/session.h>
@ -466,7 +466,7 @@ int main(int argc, char **argv)
int rc = 0;
mxs_log_init(NULL, NULL, MXS_LOG_TARGET_STDOUT);
mxs::MessageQueue::init();
mxb::MessageQueue::init();
rc += test_arguments();
rc += test_optional_arguments();

View File

@ -15,6 +15,8 @@
*/
#include <maxscale/cdefs.h>
#include <maxbase/messagequeue.hh>
#include <maxbase/worker.hh>
#include <maxscale/dcb.h>
#include <maxscale/housekeeper.h>
#include <maxscale/maxscale_test.h>
@ -46,8 +48,8 @@ void init_test_env(char *path)
qc_setup(NULL, QC_SQL_MODE_DEFAULT, NULL, NULL);
qc_process_init(QC_INIT_BOTH);
poll_init();
maxscale::MessageQueue::init();
maxscale::Worker::init();
mxb::MessageQueue::init();
mxb::Worker::init();
maxscale::RoutingWorker::init();
hkinit();
}

View File

@ -41,7 +41,7 @@ ThrottleSession::~ThrottleSession()
{
if (m_delayed_call_id)
{
maxscale::Worker* worker = maxscale::Worker::get_current();
maxbase::Worker* worker = maxbase::Worker::get_current();
ss_dassert(worker);
worker->cancel_delayed_call(m_delayed_call_id);
}
@ -61,7 +61,7 @@ int ThrottleSession::real_routeQuery(GWBUF *buffer, bool is_delayed)
{
// delay the current routeQuery for at least one cycle at stated max speed.
int32_t delay = 1 + std::ceil(1000.0 / m_filter.config().max_qps);
maxscale::Worker* worker = maxscale::Worker::get_current();
maxbase::Worker* worker = maxbase::Worker::get_current();
ss_dassert(worker);
m_delayed_call_id = worker->delayed_call(delay, &ThrottleSession::delayed_routeQuery,
this, buffer);
@ -100,19 +100,19 @@ int ThrottleSession::real_routeQuery(GWBUF *buffer, bool is_delayed)
return mxs::FilterSession::routeQuery(buffer);
}
bool ThrottleSession::delayed_routeQuery(maxscale::Worker::Call::action_t action, GWBUF *buffer)
bool ThrottleSession::delayed_routeQuery(maxbase::Worker::Call::action_t action, GWBUF *buffer)
{
m_delayed_call_id = 0;
switch (action)
{
case maxscale::Worker::Call::EXECUTE:
case maxbase::Worker::Call::EXECUTE:
if (!real_routeQuery(buffer, true))
{
poll_fake_hangup_event(m_pSession->client_dcb);
}
break;
case maxscale::Worker::Call::CANCEL:
case maxbase::Worker::Call::CANCEL:
gwbuf_free(buffer);
break;
}

View File

@ -12,8 +12,8 @@
*/
#pragma once
#include <maxbase/worker.hh>
#include <maxscale/filter.hh>
#include <maxscale/worker.hh>
#include <maxbase/eventcount.hh>
namespace throttle
@ -31,7 +31,7 @@ public:
int routeQuery(GWBUF* buffer);
private:
bool delayed_routeQuery(maxscale::Worker::Call::action_t action,
bool delayed_routeQuery(maxbase::Worker::Call::action_t action,
GWBUF* buffer);
int real_routeQuery(GWBUF* buffer, bool is_delayed);
ThrottleFilter& m_filter;

View File

@ -18,8 +18,8 @@
#include <string>
#include <sstream>
#include <vector>
#include <maxbase/messagequeue.hh>
#include <maxscale/alloc.h>
#include <maxscale/messagequeue.hh>
#include <set>
using std::string;
@ -70,7 +70,7 @@ private:
int main()
{
mxs::MessageQueue::init();
mxb::MessageQueue::init();
MariaDBMonitor::Test tester;
return tester.run_tests();

View File

@ -71,7 +71,7 @@ void LocalClient::self_destruct()
void LocalClient::close()
{
mxs::Worker* worker = mxs::Worker::get_current();
mxb::Worker* worker = mxb::Worker::get_current();
ss_dassert(worker);
worker->remove_fd(m_sock);
::close(m_sock);
@ -244,7 +244,7 @@ LocalClient* LocalClient::create(MYSQL_session* session, MySQLProtocol* proto, c
if (relay)
{
mxs::Worker* worker = mxs::Worker::get_current();
mxb::Worker* worker = mxb::Worker::get_current();
if (worker->add_fd(fd, poll_events, (MXB_POLL_DATA*)relay))
{

View File

@ -28,8 +28,9 @@
#include <glob.h>
#include <ini.h>
#include <avro/errors.h>
#include <maxscale/alloc.h>
#include <maxbase/atomic.h>
#include <maxbase/worker.hh>
#include <maxscale/alloc.h>
#include <maxscale/dcb.h>
#include <maxscale/log.h>
#include <maxscale/modulecmd.h>
@ -42,7 +43,6 @@
#include <maxscale/spinlock.h>
#include <maxscale/utils.hh>
#include <maxscale/routingworker.h>
#include <maxscale/worker.hh>
#include <binlog_common.h>
#include "avro_converter.hh"

View File

@ -23,8 +23,9 @@
#include <glob.h>
#include <ini.h>
#include <sys/stat.h>
#include <maxscale/alloc.h>
#include <maxbase/atomic.h>
#include <maxbase/worker.hh>
#include <maxscale/alloc.h>
#include <maxscale/dcb.h>
#include <maxscale/log.h>
#include <maxscale/modulecmd.h>
@ -37,12 +38,11 @@
#include <maxscale/spinlock.h>
#include <maxscale/utils.h>
#include <maxscale/routingworker.h>
#include <maxscale/worker.hh>
#include <binlog_common.h>
#include "avro_converter.hh"
using namespace mxs;
using namespace maxbase;
static bool conversion_task_ctl(Avro *inst, bool start);
@ -296,7 +296,7 @@ bool converter_func(Worker::Call::action_t action, Avro* router)
return true;
}
class ConversionCtlTask: public mxs::WorkerDisposableTask
class ConversionCtlTask: public Worker::DisposableTask
{
public:
ConversionCtlTask(Avro* instance, bool start):