MCOL-5499 Enable ControlFlow for same node communication processing path to avoid DEC queue overloading (#2848)

This commit is contained in:
Roman Nozdrin
2023-06-07 13:41:59 +01:00
committed by GitHub
parent cacbbee1c2
commit 62dc392476
9 changed files with 193 additions and 202 deletions

View File

@ -33,7 +33,7 @@
#include <stdexcept>
#include <unistd.h>
#include <cstring>
//#define NDEBUG
// #define NDEBUG
#include <cassert>
#include <string>
#include <sstream>
@ -629,7 +629,7 @@ void BatchPrimitiveProcessor::addToJoiner(ByteStream& bs)
{
uint64_t key;
uint32_t value;
} * arr;
}* arr;
#pragma pack(pop)
/* skip the header */
@ -931,7 +931,7 @@ void BatchPrimitiveProcessor::initProcessor()
strValues.reset(new utils::NullString[LOGICAL_BLOCK_RIDS]);
outMsgSize = defaultBufferSize;
outputMsg.reset(new(std::align_val_t(MAXCOLUMNWIDTH)) uint8_t[outMsgSize]);
outputMsg.reset(new (std::align_val_t(MAXCOLUMNWIDTH)) uint8_t[outMsgSize]);
if (ot == ROW_GROUP)
{
@ -1479,8 +1479,8 @@ void BatchPrimitiveProcessor::execute()
if (!asyncLoaded[p + 1])
{
loadBlockAsync(col->getLBIDAux(), versionInfo, txnID, 2, &cachedIO, &physIO,
LBIDTrace, sessionID, &counterLock, &busyLoaderCount, sendThread, &vssCache);
loadBlockAsync(col->getLBIDAux(), versionInfo, txnID, 2, &cachedIO, &physIO, LBIDTrace, sessionID,
&counterLock, &busyLoaderCount, sendThread, &vssCache);
asyncLoaded[p + 1] = true;
}
}
@ -2169,8 +2169,18 @@ void BatchPrimitiveProcessor::sendResponse()
// !sock has a 'same host connection' semantics here.
if (initiatedByEM_ && (!sock || exeMgrDecPtr->clientAtTheSameHost(sock)))
{
exeMgrDecPtr->addDataToOutput(serialized);
serialized.reset();
// Flow Control now handles same node connections so the recieving DEC queue
// is limited.
if (sendThread->flowControlEnabled())
{
sendThread->sendResult({serialized, nullptr, nullptr, 0}, false);
}
else
{
exeMgrDecPtr->addDataToOutput(serialized);
serialized.reset();
}
return;
}

View File

@ -26,6 +26,7 @@
#include <mutex>
#include "bppsendthread.h"
#include "resourcemanager.h"
#include "serviceexemgr.h"
namespace primitiveprocessor
{
@ -33,32 +34,9 @@ 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 = 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 = joblist::ResourceManager::instance()->getMaxBPPSendQueue();
queueBytesThresh = joblist::ResourceManager::instance()->getBPPSendThreadBytesThresh();
queueMsgThresh = joblist::ResourceManager::instance()->getBPPSendThreadMsgThresh();
runner = boost::thread(Runner_t(this));
}
@ -74,7 +52,7 @@ void BPPSendThread::sendResult(const Msg_t& msg, bool newConnection)
if (sizeTooBig())
{
std::unique_lock<std::mutex> sl1(respondLock);
while (currentByteSize >= maxByteSize && msgQueue.size() > 3 && !die)
while (currentByteSize >= queueBytesThresh && msgQueue.size() > 3 && !die)
{
fProcessorPool->incBlockedThreads();
okToRespond.wait(sl1);
@ -119,7 +97,7 @@ void BPPSendThread::sendResults(const vector<Msg_t>& msgs, bool newConnection)
if (sizeTooBig())
{
std::unique_lock<std::mutex> sl1(respondLock);
while (currentByteSize >= maxByteSize && msgQueue.size() > 3 && !die)
while (currentByteSize >= queueBytesThresh && msgQueue.size() > 3 && !die)
{
fProcessorPool->incBlockedThreads();
okToRespond.wait(sl1);
@ -166,7 +144,6 @@ void BPPSendThread::sendMore(int num)
{
std::unique_lock<std::mutex> sl(ackLock);
// cout << "got an ACK for " << num << " msgsLeft=" << msgsLeft << endl;
if (num == -1)
fcEnabled = false;
else if (num == 0)
@ -256,18 +233,27 @@ void BPPSendThread::mainLoop()
bsSize = msg[msgsSent].msg->lengthWithHdrOverhead();
try
// Same node processing path
if (!sock)
{
boost::mutex::scoped_lock sl2(*lock);
sock->write(*msg[msgsSent].msg);
// cout << "sent 1 msg\n";
auto* exeMgrDecPtr = exemgr::globServiceExeMgr->getDec();
assert(exeMgrDecPtr);
exeMgrDecPtr->addDataToOutput(msg[msgsSent].msg);
}
catch (std::exception& e)
else
{
sl.lock();
exceptionString = e.what();
gotException = true;
return;
try
{
boost::mutex::scoped_lock sl2(*lock);
sock->write(*msg[msgsSent].msg);
}
catch (std::exception& e)
{
sl.lock();
exceptionString = e.what();
gotException = true;
return;
}
}
(void)atomicops::atomicDec(&msgsLeft);
@ -275,7 +261,7 @@ void BPPSendThread::mainLoop()
msg[msgsSent].msg.reset();
}
if (fProcessorPool->blockedThreadCount() > 0 && currentByteSize < maxByteSize)
if (fProcessorPool->blockedThreadCount() > 0 && currentByteSize < queueBytesThresh)
{
okToRespond.notify_one();
}

View File

@ -37,8 +37,7 @@ namespace primitiveprocessor
class BPPSendThread
{
public:
BPPSendThread(); // starts unthrottled
BPPSendThread(uint32_t initMsgsLeft); // starts throttled
BPPSendThread(); // starts unthrottled
virtual ~BPPSendThread();
struct Msg_t
@ -71,7 +70,8 @@ class BPPSendThread
{
// keep the queue size below the 100 msg threshold & below the 250MB mark,
// but at least 3 msgs so there is always 1 ready to be sent.
return ((msgQueue.size() > sizeThreshold) || (currentByteSize >= maxByteSize && msgQueue.size() > 3)) &&
return ((msgQueue.size() > queueMsgThresh) ||
(currentByteSize >= queueBytesThresh && msgQueue.size() > 3)) &&
!die;
}
@ -111,11 +111,13 @@ class BPPSendThread
std::queue<Msg_t> msgQueue;
std::mutex msgQueueLock;
std::condition_variable queueNotEmpty;
volatile bool die, gotException, mainThreadWaiting;
volatile bool die = false;
volatile bool gotException = false;
volatile bool mainThreadWaiting = false;
std::string exceptionString;
uint32_t sizeThreshold;
volatile int32_t msgsLeft;
bool waiting;
uint32_t queueMsgThresh = 0;
volatile int32_t msgsLeft = -1;
bool waiting = false;
std::mutex ackLock;
std::condition_variable okToSend;
// Condition to prevent run away queue
@ -141,12 +143,12 @@ class BPPSendThread
};
std::set<Connection_t> connections_s;
std::vector<Connection_t> connections_v;
bool sawAllConnections;
volatile bool fcEnabled;
bool sawAllConnections = false;
volatile bool fcEnabled = false;
/* secondary queue size restriction based on byte size */
volatile uint64_t currentByteSize;
uint64_t maxByteSize;
volatile uint64_t currentByteSize = 0;
uint64_t queueBytesThresh;
// Used to tell the ThreadPool It should consider additional threads because a
// queue full event has happened and a thread has been blocked.
boost::shared_ptr<threadpool::FairThreadPool> fProcessorPool;