You've already forked mariadb-columnstore-engine
mirror of
https://github.com/mariadb-corporation/mariadb-columnstore-engine.git
synced 2025-08-01 06:46:55 +03:00
MCOL-5166 This patch adds support for in-memory communication b/w EM to PP via a shared queue in DEC class
JobList low-level code relateod to primitive jobs now uses shared pointers instead of ByteStream refs talking to DEC b/c same-node EM-PP communication now goes over a queue in DEC instead of a network hop. PP now has a separate thread that processes the primitive job messages from that DEC queue.
This commit is contained in:
committed by
Leonid Fedorov
parent
9ef16c6ded
commit
a9d8924683
@ -737,12 +737,12 @@ TupleBPS::~TupleBPS()
|
||||
|
||||
if (BPPIsAllocated)
|
||||
{
|
||||
ByteStream bs;
|
||||
fBPP->destroyBPP(bs);
|
||||
SBS sbs{new ByteStream()};
|
||||
fBPP->destroyBPP(*sbs);
|
||||
|
||||
try
|
||||
{
|
||||
fDec->write(uniqueID, bs);
|
||||
fDec->write(uniqueID, sbs);
|
||||
}
|
||||
catch (const std::exception& e)
|
||||
{
|
||||
@ -1093,8 +1093,8 @@ void TupleBPS::startProcessingThread(TupleBPS* tbps, vector<boost::shared_ptr<me
|
||||
|
||||
void TupleBPS::serializeJoiner()
|
||||
{
|
||||
ByteStream bs;
|
||||
bool more = true;
|
||||
SBS sbs(new ByteStream());
|
||||
|
||||
/* false from nextJoinerMsg means it's the last msg,
|
||||
it's not exactly the exit condition*/
|
||||
@ -1103,16 +1103,17 @@ void TupleBPS::serializeJoiner()
|
||||
{
|
||||
// code block to release the lock immediatly
|
||||
boost::mutex::scoped_lock lk(serializeJoinerMutex);
|
||||
more = fBPP->nextTupleJoinerMsg(bs);
|
||||
more = fBPP->nextTupleJoinerMsg(*sbs);
|
||||
}
|
||||
#ifdef JLF_DEBUG
|
||||
cout << "serializing joiner into " << bs.length() << " bytes" << endl;
|
||||
#endif
|
||||
fDec->write(uniqueID, bs);
|
||||
bs.restart();
|
||||
fDec->write(uniqueID, sbs);
|
||||
sbs.reset(new ByteStream());
|
||||
}
|
||||
}
|
||||
|
||||
// Outdated method
|
||||
void TupleBPS::serializeJoiner(uint32_t conn)
|
||||
{
|
||||
// We need this lock for TupleBPS::serializeJoiner()
|
||||
@ -1376,7 +1377,7 @@ void TupleBPS::run()
|
||||
std::string("TupleBPS")); // step name
|
||||
}
|
||||
|
||||
ByteStream bs;
|
||||
SBS sbs{new ByteStream()};
|
||||
|
||||
if (fDelivery)
|
||||
{
|
||||
@ -1406,8 +1407,8 @@ void TupleBPS::run()
|
||||
{
|
||||
fDec->addDECEventListener(this);
|
||||
fBPP->priority(priority());
|
||||
fBPP->createBPP(bs);
|
||||
fDec->write(uniqueID, bs);
|
||||
fBPP->createBPP(*sbs);
|
||||
fDec->write(uniqueID, sbs);
|
||||
BPPIsAllocated = true;
|
||||
|
||||
if (doJoin && tjoiners[0]->inPM())
|
||||
@ -1453,13 +1454,13 @@ void TupleBPS::join()
|
||||
|
||||
if (BPPIsAllocated)
|
||||
{
|
||||
ByteStream bs;
|
||||
SBS sbs{new ByteStream()};
|
||||
fDec->removeDECEventListener(this);
|
||||
fBPP->destroyBPP(bs);
|
||||
fBPP->destroyBPP(*sbs);
|
||||
|
||||
try
|
||||
{
|
||||
fDec->write(uniqueID, bs);
|
||||
fDec->write(uniqueID, sbs);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
@ -1476,10 +1477,10 @@ void TupleBPS::join()
|
||||
|
||||
void TupleBPS::sendError(uint16_t status)
|
||||
{
|
||||
ByteStream msgBpp;
|
||||
SBS msgBpp;
|
||||
fBPP->setCount(1);
|
||||
fBPP->setStatus(status);
|
||||
fBPP->runErrorBPP(msgBpp);
|
||||
fBPP->runErrorBPP(*msgBpp);
|
||||
|
||||
try
|
||||
{
|
||||
@ -1602,7 +1603,7 @@ void TupleBPS::sendJobs(const vector<Job>& jobs)
|
||||
|
||||
for (i = 0; i < jobs.size() && !cancelled(); i++)
|
||||
{
|
||||
fDec->write(uniqueID, *(jobs[i].msg));
|
||||
fDec->write(uniqueID, jobs[i].msg);
|
||||
tplLock.lock();
|
||||
msgsSent += jobs[i].expectedResponses;
|
||||
|
||||
@ -2613,15 +2614,15 @@ void TupleBPS::receiveMultiPrimitiveMessages()
|
||||
dlTimes.setEndOfInputTime();
|
||||
}
|
||||
|
||||
ByteStream bs;
|
||||
SBS sbs{new ByteStream()};
|
||||
|
||||
try
|
||||
{
|
||||
if (BPPIsAllocated)
|
||||
{
|
||||
fDec->removeDECEventListener(this);
|
||||
fBPP->destroyBPP(bs);
|
||||
fDec->write(uniqueID, bs);
|
||||
fBPP->destroyBPP(*sbs);
|
||||
fDec->write(uniqueID, sbs);
|
||||
BPPIsAllocated = false;
|
||||
}
|
||||
}
|
||||
@ -3302,12 +3303,12 @@ void TupleBPS::abort_nolock()
|
||||
|
||||
if (fDec && BPPIsAllocated)
|
||||
{
|
||||
ByteStream bs;
|
||||
fBPP->abortProcessing(&bs);
|
||||
SBS sbs{new ByteStream()};
|
||||
fBPP->abortProcessing(sbs.get());
|
||||
|
||||
try
|
||||
{
|
||||
fDec->write(uniqueID, bs);
|
||||
fDec->write(uniqueID, sbs);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
|
Reference in New Issue
Block a user