You've already forked mariadb-columnstore-engine
mirror of
https://github.com/mariadb-corporation/mariadb-columnstore-engine.git
synced 2025-08-05 16:15:50 +03:00
Mcol 4841 dev6 Handle large joins without OOM (#2155)
* MCOL-4846 dev-6 Handle large join results Use a loop to shrink the number of results reported per message to something manageable. * MCOL-4841 small changes requested by review * Add EXTRA threads to prioritythreadpool prioritythreadpool is configured at startup with a fixed number of threads available. This is to prevent thread thrashing. Since most of the time, BPP job steps are short lived, and a rescheduling mechanism exist if no threads are available, this works to keep cpu wastage to a minimum. However, if a query or queries consume all the threads in prioritythreadpool and then block (due to the consumer not consuming fast enough) we can run out of threads and no work will be done until some threads unblock. A new mechanism allows for EXTRA threads to be generated for the duration of the blocking action. These threads can act on new queries. When all blocking is completed, these threads will be released when idle. * MCOL-4841 dev6 Reconcile with changes in develop-6 * MCOL-4841 Some format corrections * MCOL-4841 dev clean up some things based on review * MCOL-4841 dev 6 ExeMgr Crashes after large join This commit fixes up memory accounting issues in ExeMgr * MCOL-4841 remove LDI change Opened MCOL-4968 to address the issue * MCOL-4841 Add fMaxBPPSendQueue to ResourceManager This causes the setting to be loaded at run time (requires restart to accept a change) BPPSendthread gets this in it's ctor Also rolled back changes to TupleHashJoinStep::smallRunnerFcn() that used a local variable to count locally allocated memory, then added it into the global counter at function's end. Not counting the memory globally caused conversion to UM only join way later than it should. This resulted in MCOL-4971. * MCOL-4841 make blockedThreads and extraThreads atomic Also restore previous scope of locks in bppsendthread. There is some small chance the new scope could be incorrect, and the performance boost is negligible. Better safe than sorry.
This commit is contained in:
@@ -23,59 +23,56 @@
|
||||
|
||||
#include <unistd.h>
|
||||
#include <stdexcept>
|
||||
#include <mutex>
|
||||
#include "bppsendthread.h"
|
||||
|
||||
using namespace std;
|
||||
using namespace boost;
|
||||
|
||||
#include "atomicops.h"
|
||||
|
||||
namespace primitiveprocessor
|
||||
{
|
||||
|
||||
extern uint32_t connectionsPerUM;
|
||||
extern uint32_t BPPCount;
|
||||
|
||||
BPPSendThread::BPPSendThread() : die(false), gotException(false), mainThreadWaiting(false),
|
||||
sizeThreshold(100), msgsLeft(-1), waiting(false), sawAllConnections(false),
|
||||
fcEnabled(false), currentByteSize(0), maxByteSize(25000000)
|
||||
fcEnabled(false), currentByteSize(0)
|
||||
{
|
||||
maxByteSize = joblist::ResourceManager::instance()->getMaxBPPSendQueue();
|
||||
runner = boost::thread(Runner_t(this));
|
||||
}
|
||||
|
||||
BPPSendThread::BPPSendThread(uint32_t initMsgsLeft) : die(false), gotException(false),
|
||||
mainThreadWaiting(false), sizeThreshold(100), msgsLeft(initMsgsLeft), waiting(false),
|
||||
sawAllConnections(false), fcEnabled(false), currentByteSize(0), maxByteSize(25000000)
|
||||
sawAllConnections(false), fcEnabled(false), currentByteSize(0)
|
||||
{
|
||||
maxByteSize = joblist::ResourceManager::instance()->getMaxBPPSendQueue();
|
||||
runner = boost::thread(Runner_t(this));
|
||||
}
|
||||
|
||||
BPPSendThread::~BPPSendThread()
|
||||
{
|
||||
boost::mutex::scoped_lock sl(msgQueueLock);
|
||||
boost::mutex::scoped_lock sl2(ackLock);
|
||||
die = true;
|
||||
queueNotEmpty.notify_one();
|
||||
okToSend.notify_one();
|
||||
sl.unlock();
|
||||
sl2.unlock();
|
||||
abort();
|
||||
runner.join();
|
||||
}
|
||||
|
||||
bool BPPSendThread::okToProceed()
|
||||
{
|
||||
// keep the queue size below the 100 msg threshold & below the 25MB mark,
|
||||
// but at least 2 msgs so there is always 1 ready to be sent.
|
||||
return ((msgQueue.size() < sizeThreshold && currentByteSize < maxByteSize)
|
||||
|| msgQueue.size() < 3) && !die;
|
||||
}
|
||||
|
||||
void BPPSendThread::sendResult(const Msg_t& msg, bool newConnection)
|
||||
{
|
||||
// Wait for the queue to empty out a bit if it's stuffed full
|
||||
if (sizeTooBig())
|
||||
{
|
||||
std::unique_lock<std::mutex> sl1(respondLock);
|
||||
while (currentByteSize >= maxByteSize && msgQueue.size() > 3 && !die)
|
||||
{
|
||||
respondWait = true;
|
||||
fProcessorPool->incBlockedThreads();
|
||||
okToRespond.wait(sl1);
|
||||
fProcessorPool->decBlockedThreads();
|
||||
respondWait = false;
|
||||
}
|
||||
}
|
||||
if (die)
|
||||
return;
|
||||
|
||||
boost::mutex::scoped_lock sl(msgQueueLock);
|
||||
|
||||
|
||||
std::unique_lock<std::mutex> sl(msgQueueLock);
|
||||
|
||||
if (gotException)
|
||||
throw runtime_error(exceptionString);
|
||||
|
||||
@@ -105,11 +102,24 @@ void BPPSendThread::sendResult(const Msg_t& msg, bool newConnection)
|
||||
|
||||
void BPPSendThread::sendResults(const vector<Msg_t>& msgs, bool newConnection)
|
||||
{
|
||||
// Wait for the queue to empty out a bit if it's stuffed full
|
||||
if (sizeTooBig())
|
||||
{
|
||||
std::unique_lock<std::mutex> sl1(respondLock);
|
||||
while (currentByteSize >= maxByteSize && msgQueue.size() > 3 && !die)
|
||||
{
|
||||
respondWait = true;
|
||||
fProcessorPool->incBlockedThreads();
|
||||
okToRespond.wait(sl1);
|
||||
fProcessorPool->decBlockedThreads();
|
||||
respondWait = false;
|
||||
}
|
||||
}
|
||||
if (die)
|
||||
return;
|
||||
|
||||
boost::mutex::scoped_lock sl(msgQueueLock);
|
||||
|
||||
std::unique_lock<std::mutex> sl(msgQueueLock);
|
||||
|
||||
if (gotException)
|
||||
throw runtime_error(exceptionString);
|
||||
|
||||
@@ -143,7 +153,7 @@ void BPPSendThread::sendResults(const vector<Msg_t>& msgs, bool newConnection)
|
||||
|
||||
void BPPSendThread::sendMore(int num)
|
||||
{
|
||||
boost::mutex::scoped_lock sl(ackLock);
|
||||
std::unique_lock<std::mutex> sl(ackLock);
|
||||
|
||||
// cout << "got an ACK for " << num << " msgsLeft=" << msgsLeft << endl;
|
||||
if (num == -1)
|
||||
@@ -156,6 +166,7 @@ void BPPSendThread::sendMore(int num)
|
||||
else
|
||||
(void)atomicops::atomicAdd(&msgsLeft, num);
|
||||
|
||||
sl.unlock();
|
||||
if (waiting)
|
||||
okToSend.notify_one();
|
||||
}
|
||||
@@ -178,7 +189,7 @@ void BPPSendThread::mainLoop()
|
||||
|
||||
while (!die)
|
||||
{
|
||||
boost::mutex::scoped_lock sl(msgQueueLock);
|
||||
std::unique_lock<std::mutex> sl(msgQueueLock);
|
||||
|
||||
if (msgQueue.empty() && !die)
|
||||
{
|
||||
@@ -209,8 +220,7 @@ void BPPSendThread::mainLoop()
|
||||
|
||||
if (msgsLeft <= 0 && fcEnabled && !die)
|
||||
{
|
||||
boost::mutex::scoped_lock sl2(ackLock);
|
||||
|
||||
std::unique_lock<std::mutex> sl2(ackLock);
|
||||
while (msgsLeft <= 0 && fcEnabled && !die)
|
||||
{
|
||||
waiting = true;
|
||||
@@ -254,19 +264,26 @@ void BPPSendThread::mainLoop()
|
||||
(void)atomicops::atomicSub(¤tByteSize, bsSize);
|
||||
msg[msgsSent].msg.reset();
|
||||
}
|
||||
|
||||
if (respondWait && currentByteSize < maxByteSize)
|
||||
{
|
||||
okToRespond.notify_one();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void BPPSendThread::abort()
|
||||
{
|
||||
boost::mutex::scoped_lock sl(msgQueueLock);
|
||||
boost::mutex::scoped_lock sl2(ackLock);
|
||||
die = true;
|
||||
queueNotEmpty.notify_one();
|
||||
okToSend.notify_one();
|
||||
sl.unlock();
|
||||
sl2.unlock();
|
||||
std::lock_guard<std::mutex> sl(msgQueueLock);
|
||||
std::lock_guard<std::mutex> sl2(ackLock);
|
||||
std::lock_guard<std::mutex> sl3(respondLock);
|
||||
{
|
||||
die = true;
|
||||
}
|
||||
queueNotEmpty.notify_all();
|
||||
okToSend.notify_all();
|
||||
okToRespond.notify_all();
|
||||
}
|
||||
|
||||
}
|
||||
|
Reference in New Issue
Block a user