1
0
mirror of https://github.com/mariadb-corporation/mariadb-columnstore-engine.git synced 2025-07-29 08:21:15 +03:00

MCOL4841 dev port run large join without OOM

This commit is contained in:
David Hall
2022-02-09 17:33:55 -06:00
parent d30e140dc3
commit 27dea733c5
34 changed files with 821 additions and 518 deletions

View File

@ -23,16 +23,14 @@
#include <unistd.h>
#include <stdexcept>
#include <mutex>
#include "bppsendthread.h"
using namespace std;
using namespace boost;
#include "atomicops.h"
#include "resourcemanager.h"
namespace primitiveprocessor
{
extern uint32_t connectionsPerUM;
extern uint32_t BPPCount;
BPPSendThread::BPPSendThread()
: die(false)
@ -44,8 +42,8 @@ BPPSendThread::BPPSendThread()
, sawAllConnections(false)
, fcEnabled(false)
, currentByteSize(0)
, maxByteSize(25000000)
{
maxByteSize = joblist::ResourceManager::instance()->getMaxBPPSendQueue();
runner = boost::thread(Runner_t(this));
}
@ -59,36 +57,36 @@ BPPSendThread::BPPSendThread(uint32_t initMsgsLeft)
, sawAllConnections(false)
, fcEnabled(false)
, currentByteSize(0)
, maxByteSize(25000000)
{
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);
@ -119,10 +117,23 @@ 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);
@ -157,7 +168,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)
@ -170,6 +181,7 @@ void BPPSendThread::sendMore(int num)
else
(void)atomicops::atomicAdd(&msgsLeft, num);
sl.unlock();
if (waiting)
okToSend.notify_one();
}
@ -192,7 +204,7 @@ void BPPSendThread::mainLoop()
while (!die)
{
boost::mutex::scoped_lock sl(msgQueueLock);
std::unique_lock<std::mutex> sl(msgQueueLock);
if (msgQueue.empty() && !die)
{
@ -223,8 +235,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;
@ -267,19 +278,26 @@ void BPPSendThread::mainLoop()
(void)atomicops::atomicSub(&currentByteSize, 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);
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_one();
okToSend.notify_one();
sl.unlock();
sl2.unlock();
queueNotEmpty.notify_all();
okToSend.notify_all();
okToRespond.notify_all();
}
} // namespace primitiveprocessor