mirror of
https://github.com/mariadb-corporation/mariadb-columnstore-engine.git
synced 2025-04-18 21:44:02 +03:00
feat(joblist,runtime): this is the first part of the execution model that produces a workload that can be predicted for a given query.
* feat(joblist,runtime): this is the first part of the execution model that produces a workload that can be predicted for a given query. - forces to UM join converter to use a value from a configuration - replaces a constant used to control a number of outstanding requests with a value depends on column width - modifies related Columnstore.xml values
This commit is contained in:
parent
bba2133cd0
commit
6445f4dff3
@ -68,6 +68,7 @@ using namespace oam;
|
|||||||
using namespace joblist;
|
using namespace joblist;
|
||||||
|
|
||||||
#include "atomicops.h"
|
#include "atomicops.h"
|
||||||
|
#include "threadnaming.h"
|
||||||
|
|
||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
@ -131,6 +132,7 @@ struct EngineCommRunner
|
|||||||
uint32_t connIndex;
|
uint32_t connIndex;
|
||||||
void operator()()
|
void operator()()
|
||||||
{
|
{
|
||||||
|
utils::setThreadName("DECRunner");
|
||||||
// cout << "Listening on client at 0x" << hex << (ptrdiff_t)client << dec << endl;
|
// cout << "Listening on client at 0x" << hex << (ptrdiff_t)client << dec << endl;
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
|
@ -112,12 +112,12 @@ class pColStep : public JobStep
|
|||||||
*
|
*
|
||||||
* Starts processing. Set at least the RID list before calling this.
|
* Starts processing. Set at least the RID list before calling this.
|
||||||
*/
|
*/
|
||||||
virtual void run(){};
|
virtual void run() {};
|
||||||
/** @brief Sync's the caller with the end of execution.
|
/** @brief Sync's the caller with the end of execution.
|
||||||
*
|
*
|
||||||
* Does nothing. Returns when this instance is finished.
|
* Does nothing. Returns when this instance is finished.
|
||||||
*/
|
*/
|
||||||
virtual void join(){};
|
virtual void join() {};
|
||||||
|
|
||||||
virtual const std::string toString() const;
|
virtual const std::string toString() const;
|
||||||
|
|
||||||
@ -1459,6 +1459,8 @@ class TupleBPS : public BatchPrimitive, public TupleDeliveryStep
|
|||||||
void interleaveJobs(std::vector<Job>* jobs) const;
|
void interleaveJobs(std::vector<Job>* jobs) const;
|
||||||
void sendJobs(const std::vector<Job>& jobs);
|
void sendJobs(const std::vector<Job>& jobs);
|
||||||
uint32_t numDBRoots;
|
uint32_t numDBRoots;
|
||||||
|
// presumably there must be not more than 2^32 blocks per job as of 23.02.
|
||||||
|
uint32_t blocksPerJob;
|
||||||
|
|
||||||
/* Pseudo column filter processing. Think about refactoring into a separate class. */
|
/* Pseudo column filter processing. Think about refactoring into a separate class. */
|
||||||
bool processPseudoColFilters(uint32_t extentIndex, boost::shared_ptr<std::map<int, int>> dbRootPMMap) const;
|
bool processPseudoColFilters(uint32_t extentIndex, boost::shared_ptr<std::map<int, int>> dbRootPMMap) const;
|
||||||
|
@ -19,7 +19,7 @@
|
|||||||
// $Id: tuple-bps.cpp 9705 2013-07-17 20:06:07Z pleblanc $
|
// $Id: tuple-bps.cpp 9705 2013-07-17 20:06:07Z pleblanc $
|
||||||
|
|
||||||
#include <unistd.h>
|
#include <unistd.h>
|
||||||
//#define NDEBUG
|
// #define NDEBUG
|
||||||
#include <cassert>
|
#include <cassert>
|
||||||
#include <sstream>
|
#include <sstream>
|
||||||
#include <iomanip>
|
#include <iomanip>
|
||||||
@ -77,7 +77,9 @@ using namespace querytele;
|
|||||||
|
|
||||||
#include "columnwidth.h"
|
#include "columnwidth.h"
|
||||||
#include "pseudocolumn.h"
|
#include "pseudocolumn.h"
|
||||||
//#define DEBUG 1
|
// #define DEBUG 1
|
||||||
|
|
||||||
|
// #include "poormanprofiler.inc"
|
||||||
|
|
||||||
extern boost::mutex fileLock_g;
|
extern boost::mutex fileLock_g;
|
||||||
|
|
||||||
@ -396,15 +398,6 @@ void TupleBPS::initializeConfigParms()
|
|||||||
{
|
{
|
||||||
string strVal;
|
string strVal;
|
||||||
|
|
||||||
//...Get the tuning parameters that throttle msgs sent to primproc
|
|
||||||
//...fFilterRowReqLimit puts a cap on how many rids we will request from
|
|
||||||
//... primproc, before pausing to let the consumer thread catch up.
|
|
||||||
//... Without this limit, there is a chance that PrimProc could flood
|
|
||||||
//... ExeMgr with thousands of messages that will consume massive
|
|
||||||
//... amounts of memory for a 100 gigabyte database.
|
|
||||||
//...fFilterRowReqThreshold is the level at which the number of outstanding
|
|
||||||
//... rids must fall below, before the producer can send more rids.
|
|
||||||
|
|
||||||
// These could go in constructor
|
// These could go in constructor
|
||||||
fRequestSize = fRm->getJlRequestSize();
|
fRequestSize = fRm->getJlRequestSize();
|
||||||
fMaxOutstandingRequests = fRm->getJlMaxOutstandingRequests();
|
fMaxOutstandingRequests = fRm->getJlMaxOutstandingRequests();
|
||||||
@ -556,14 +549,14 @@ TupleBPS::TupleBPS(const pColScanStep& rhs, const JobInfo& jobInfo) : BatchPrimi
|
|||||||
|
|
||||||
throw runtime_error(oss.str());
|
throw runtime_error(oss.str());
|
||||||
}
|
}
|
||||||
catch(std::exception& ex)
|
catch (std::exception& ex)
|
||||||
{
|
{
|
||||||
std::ostringstream oss;
|
std::ostringstream oss;
|
||||||
oss << "Error getting AUX column OID for table " << tableName.toString();
|
oss << "Error getting AUX column OID for table " << tableName.toString();
|
||||||
oss << " due to: " << ex.what();
|
oss << " due to: " << ex.what();
|
||||||
throw runtime_error(oss.str());
|
throw runtime_error(oss.str());
|
||||||
}
|
}
|
||||||
catch(...)
|
catch (...)
|
||||||
{
|
{
|
||||||
std::ostringstream oss;
|
std::ostringstream oss;
|
||||||
oss << "Error getting AUX column OID for table " << tableName.toString();
|
oss << "Error getting AUX column OID for table " << tableName.toString();
|
||||||
@ -1684,7 +1677,8 @@ void TupleBPS::sendJobs(const vector<Job>& jobs)
|
|||||||
if (recvWaiting)
|
if (recvWaiting)
|
||||||
condvar.notify_all();
|
condvar.notify_all();
|
||||||
|
|
||||||
while ((msgsSent - msgsRecvd > fMaxOutstandingRequests << LOGICAL_EXTENT_CONVERTER) && !fDie)
|
// Send not more than fMaxOutstandingRequests jobs out. min(blocksPerJob) = 16
|
||||||
|
while ((msgsSent - msgsRecvd > fMaxOutstandingRequests * (blocksPerJob >> 1)) && !fDie)
|
||||||
{
|
{
|
||||||
sendWaiting = true;
|
sendWaiting = true;
|
||||||
condvarWakeupProducer.wait(tplLock);
|
condvarWakeupProducer.wait(tplLock);
|
||||||
@ -2007,7 +2001,6 @@ void TupleBPS::makeJobs(vector<Job>* jobs)
|
|||||||
uint32_t i;
|
uint32_t i;
|
||||||
uint32_t lbidsToScan;
|
uint32_t lbidsToScan;
|
||||||
uint32_t blocksToScan;
|
uint32_t blocksToScan;
|
||||||
uint32_t blocksPerJob;
|
|
||||||
LBID_t startingLBID;
|
LBID_t startingLBID;
|
||||||
oam::OamCache* oamCache = oam::OamCache::makeOamCache();
|
oam::OamCache* oamCache = oam::OamCache::makeOamCache();
|
||||||
boost::shared_ptr<map<int, int>> dbRootConnectionMap = oamCache->getDBRootToConnectionMap();
|
boost::shared_ptr<map<int, int>> dbRootConnectionMap = oamCache->getDBRootToConnectionMap();
|
||||||
@ -2227,6 +2220,8 @@ void TupleBPS::processByteStreamVector(vector<boost::shared_ptr<messageqcpp::Byt
|
|||||||
// changes made here should also be made there and vice versa.
|
// changes made here should also be made there and vice versa.
|
||||||
if (hasUMJoin || !fBPP->pmSendsFinalResult())
|
if (hasUMJoin || !fBPP->pmSendsFinalResult())
|
||||||
{
|
{
|
||||||
|
utils::setThreadName("BSPJoin");
|
||||||
|
|
||||||
data->joinedData = RGData(data->local_outputRG);
|
data->joinedData = RGData(data->local_outputRG);
|
||||||
data->local_outputRG.setData(&data->joinedData);
|
data->local_outputRG.setData(&data->joinedData);
|
||||||
data->local_outputRG.resetRowGroup(data->local_primRG.getBaseRid());
|
data->local_outputRG.resetRowGroup(data->local_primRG.getBaseRid());
|
||||||
@ -2340,6 +2335,8 @@ void TupleBPS::processByteStreamVector(vector<boost::shared_ptr<messageqcpp::Byt
|
|||||||
{
|
{
|
||||||
rgDatav.push_back(data->joinedData);
|
rgDatav.push_back(data->joinedData);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
utils::setThreadName("ByteStreamProcessor");
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
@ -2351,6 +2348,7 @@ void TupleBPS::processByteStreamVector(vector<boost::shared_ptr<messageqcpp::Byt
|
|||||||
memAmount = 0;
|
memAmount = 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
utils::setThreadName("BSPFE2");
|
||||||
// Execute UM F & E group 2 on rgDatav
|
// Execute UM F & E group 2 on rgDatav
|
||||||
if (fe2 && !bRunFEonPM && rgDatav.size() > 0 && !cancelled())
|
if (fe2 && !bRunFEonPM && rgDatav.size() > 0 && !cancelled())
|
||||||
{
|
{
|
||||||
@ -2358,6 +2356,8 @@ void TupleBPS::processByteStreamVector(vector<boost::shared_ptr<messageqcpp::Byt
|
|||||||
rgDataVecToDl(rgDatav, data->local_fe2Output, dlp);
|
rgDataVecToDl(rgDatav, data->local_fe2Output, dlp);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
utils::setThreadName("ByteStreamProcessor");
|
||||||
|
|
||||||
data->cachedIO_Thread += cachedIO;
|
data->cachedIO_Thread += cachedIO;
|
||||||
data->physIO_Thread += physIO;
|
data->physIO_Thread += physIO;
|
||||||
data->touchedBlocks_Thread += touchedBlocks;
|
data->touchedBlocks_Thread += touchedBlocks;
|
||||||
@ -2777,8 +2777,7 @@ void TupleBPS::receiveMultiPrimitiveMessages()
|
|||||||
<< totalBlockedReadCount << "/" << totalBlockedWriteCount << "; output size-" << ridsReturned
|
<< totalBlockedReadCount << "/" << totalBlockedWriteCount << "; output size-" << ridsReturned
|
||||||
<< endl
|
<< endl
|
||||||
<< "\tPartitionBlocksEliminated-" << fNumBlksSkipped << "; MsgBytesIn-" << msgBytesInKB << "KB"
|
<< "\tPartitionBlocksEliminated-" << fNumBlksSkipped << "; MsgBytesIn-" << msgBytesInKB << "KB"
|
||||||
<< "; MsgBytesOut-" << msgBytesOutKB << "KB"
|
<< "; MsgBytesOut-" << msgBytesOutKB << "KB" << "; TotalMsgs-" << totalMsgs << endl
|
||||||
<< "; TotalMsgs-" << totalMsgs << endl
|
|
||||||
<< "\t1st read " << dlTimes.FirstReadTimeString() << "; EOI " << dlTimes.EndOfInputTimeString()
|
<< "\t1st read " << dlTimes.FirstReadTimeString() << "; EOI " << dlTimes.EndOfInputTimeString()
|
||||||
<< "; runtime-" << JSTimeStamp::tsdiffstr(dlTimes.EndOfInputTime(), dlTimes.FirstReadTime())
|
<< "; runtime-" << JSTimeStamp::tsdiffstr(dlTimes.EndOfInputTime(), dlTimes.FirstReadTime())
|
||||||
<< "s\n\tUUID " << uuids::to_string(fStepUuid) << "\n\tQuery UUID "
|
<< "s\n\tUUID " << uuids::to_string(fStepUuid) << "\n\tQuery UUID "
|
||||||
@ -3179,9 +3178,8 @@ bool TupleBPS::deliverStringTableRowGroup() const
|
|||||||
void TupleBPS::formatMiniStats()
|
void TupleBPS::formatMiniStats()
|
||||||
{
|
{
|
||||||
ostringstream oss;
|
ostringstream oss;
|
||||||
oss << "BPS "
|
oss << "BPS " << "PM " << alias() << " " << fTableOid << " " << fBPP->toMiniString() << " " << fPhysicalIO
|
||||||
<< "PM " << alias() << " " << fTableOid << " " << fBPP->toMiniString() << " " << fPhysicalIO << " "
|
<< " " << fCacheIO << " " << fNumBlksSkipped << " "
|
||||||
<< fCacheIO << " " << fNumBlksSkipped << " "
|
|
||||||
<< JSTimeStamp::tsdiffstr(dlTimes.EndOfInputTime(), dlTimes.FirstReadTime()) << " " << ridsReturned
|
<< JSTimeStamp::tsdiffstr(dlTimes.EndOfInputTime(), dlTimes.FirstReadTime()) << " " << ridsReturned
|
||||||
<< " ";
|
<< " ";
|
||||||
|
|
||||||
|
@ -18,7 +18,7 @@
|
|||||||
|
|
||||||
// $Id: tupleannexstep.cpp 9661 2013-07-01 20:33:05Z pleblanc $
|
// $Id: tupleannexstep.cpp 9661 2013-07-01 20:33:05Z pleblanc $
|
||||||
|
|
||||||
//#define NDEBUG
|
// #define NDEBUG
|
||||||
#include <cassert>
|
#include <cassert>
|
||||||
#include <sstream>
|
#include <sstream>
|
||||||
#include <iomanip>
|
#include <iomanip>
|
||||||
@ -251,10 +251,6 @@ void TupleAnnexStep::run()
|
|||||||
fRunnersList.resize(fMaxThreads);
|
fRunnersList.resize(fMaxThreads);
|
||||||
fInputIteratorsList.resize(fMaxThreads + 1);
|
fInputIteratorsList.resize(fMaxThreads + 1);
|
||||||
|
|
||||||
// Activate stats collecting before CS spawns threads.
|
|
||||||
if (traceOn())
|
|
||||||
dlTimes.setFirstReadTime();
|
|
||||||
|
|
||||||
// *DRRTUY Make this block conditional
|
// *DRRTUY Make this block conditional
|
||||||
StepTeleStats sts;
|
StepTeleStats sts;
|
||||||
sts.query_uuid = fQueryUuid;
|
sts.query_uuid = fQueryUuid;
|
||||||
@ -858,7 +854,7 @@ void TupleAnnexStep::finalizeParallelOrderByDistinct()
|
|||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
} // end of limit bound while loop
|
} // end of limit bound while loop
|
||||||
} // end of if-else
|
} // end of if-else
|
||||||
|
|
||||||
if (fRowGroupOut.getRowCount() > 0)
|
if (fRowGroupOut.getRowCount() > 0)
|
||||||
{
|
{
|
||||||
@ -1045,7 +1041,7 @@ void TupleAnnexStep::finalizeParallelOrderBy()
|
|||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
} // end of limit bound while loop
|
} // end of limit bound while loop
|
||||||
} // end of if-else
|
} // end of if-else
|
||||||
|
|
||||||
if (fRowGroupOut.getRowCount() > 0)
|
if (fRowGroupOut.getRowCount() > 0)
|
||||||
{
|
{
|
||||||
@ -1065,9 +1061,6 @@ void TupleAnnexStep::finalizeParallelOrderBy()
|
|||||||
|
|
||||||
if (traceOn())
|
if (traceOn())
|
||||||
{
|
{
|
||||||
if (dlTimes.FirstReadTime().tv_sec == 0)
|
|
||||||
dlTimes.setFirstReadTime();
|
|
||||||
|
|
||||||
dlTimes.setLastReadTime();
|
dlTimes.setLastReadTime();
|
||||||
dlTimes.setEndOfInputTime();
|
dlTimes.setEndOfInputTime();
|
||||||
printCalTrace();
|
printCalTrace();
|
||||||
@ -1102,6 +1095,13 @@ void TupleAnnexStep::executeParallelOrderBy(uint64_t id)
|
|||||||
try
|
try
|
||||||
{
|
{
|
||||||
more = fInputDL->next(fInputIteratorsList[id], &rgDataIn);
|
more = fInputDL->next(fInputIteratorsList[id], &rgDataIn);
|
||||||
|
|
||||||
|
// Stats collecting.
|
||||||
|
if (more && (id == 1) && traceOn())
|
||||||
|
{
|
||||||
|
dlTimes.setFirstReadTime();
|
||||||
|
}
|
||||||
|
|
||||||
if (more)
|
if (more)
|
||||||
dlOffset++;
|
dlOffset++;
|
||||||
|
|
||||||
@ -1241,14 +1241,9 @@ void TupleAnnexStep::formatMiniStats()
|
|||||||
{
|
{
|
||||||
ostringstream oss;
|
ostringstream oss;
|
||||||
oss << "TNS ";
|
oss << "TNS ";
|
||||||
oss << "UM "
|
oss << "UM " << "- " << "- " << "- " << "- " << "- " << "- "
|
||||||
<< "- "
|
<< JSTimeStamp::tsdiffstr(dlTimes.EndOfInputTime(), dlTimes.FirstReadTime()) << " " << fRowsReturned
|
||||||
<< "- "
|
<< " ";
|
||||||
<< "- "
|
|
||||||
<< "- "
|
|
||||||
<< "- "
|
|
||||||
<< "- " << JSTimeStamp::tsdiffstr(dlTimes.EndOfInputTime(), dlTimes.FirstReadTime()) << " "
|
|
||||||
<< fRowsReturned << " ";
|
|
||||||
fMiniInfo += oss.str();
|
fMiniInfo += oss.str();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -278,12 +278,12 @@ void TupleHashJoinStep::startSmallRunners(uint index)
|
|||||||
if (typelessJoin[index])
|
if (typelessJoin[index])
|
||||||
{
|
{
|
||||||
joiner.reset(new TupleJoiner(smallRGs[index], largeRG, smallSideKeys[index], largeSideKeys[index], jt,
|
joiner.reset(new TupleJoiner(smallRGs[index], largeRG, smallSideKeys[index], largeSideKeys[index], jt,
|
||||||
&jobstepThreadPool));
|
&jobstepThreadPool, numCores));
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
joiner.reset(new TupleJoiner(smallRGs[index], largeRG, smallSideKeys[index][0], largeSideKeys[index][0],
|
joiner.reset(new TupleJoiner(smallRGs[index], largeRG, smallSideKeys[index][0], largeSideKeys[index][0],
|
||||||
jt, &jobstepThreadPool));
|
jt, &jobstepThreadPool, numCores));
|
||||||
}
|
}
|
||||||
|
|
||||||
joiner->setUniqueLimit(uniqueLimit);
|
joiner->setUniqueLimit(uniqueLimit);
|
||||||
@ -1297,15 +1297,11 @@ void TupleHashJoinStep::formatMiniStats(uint32_t index)
|
|||||||
else
|
else
|
||||||
oss << "- ";
|
oss << "- ";
|
||||||
|
|
||||||
oss << " "
|
oss << " " << "- " << "- " << "- "
|
||||||
<< "- "
|
|
||||||
<< "- "
|
|
||||||
<< "- "
|
|
||||||
<< "- "
|
<< "- "
|
||||||
// << JSTimeStamp::tsdiffstr(dlTimes.EndOfInputTime(), dlTimes.FirstReadTime()) << " "
|
// << JSTimeStamp::tsdiffstr(dlTimes.EndOfInputTime(), dlTimes.FirstReadTime()) << " "
|
||||||
// dlTimes are not timed in this step, using '--------' instead.
|
// dlTimes are not timed in this step, using '--------' instead.
|
||||||
<< "-------- "
|
<< "-------- " << "-\n";
|
||||||
<< "-\n";
|
|
||||||
fMiniInfo += oss.str();
|
fMiniInfo += oss.str();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -203,12 +203,11 @@
|
|||||||
is 20 extents worth of work for the PMs to process at any given time.
|
is 20 extents worth of work for the PMs to process at any given time.
|
||||||
ProcessorThreadsPerScan * MaxOutstandingRequests should be at least
|
ProcessorThreadsPerScan * MaxOutstandingRequests should be at least
|
||||||
as many threads are available across all PMs. -->
|
as many threads are available across all PMs. -->
|
||||||
<!-- <ProcessorThreadsPerScan>16</ProcessorThreadsPerScan> -->
|
<!-- <ProcessorThreadsPerScan>16</ProcessorThreadsPerScan> -->
|
||||||
<!-- MaxOutstandingRequests is going to default to the num of cores available
|
<!-- MaxOutstandingRequests is a windows size for outstanding PrimitiveJobs messages per query step.
|
||||||
across all performance modules * 4 divided by the ProcessorThreadsPerScan,
|
A bigger number increases PrimProc workload. -->
|
||||||
but will be lower bounded by 20 -->
|
<MaxOutstandingRequests>20000</MaxOutstandingRequests>
|
||||||
<!-- <MaxOutstandingRequests>20</MaxOutstandingRequests> -->
|
<ThreadPoolSize>1000</ThreadPoolSize>
|
||||||
<ThreadPoolSize>100</ThreadPoolSize>
|
|
||||||
</JobList>
|
</JobList>
|
||||||
<RowAggregation>
|
<RowAggregation>
|
||||||
<!-- <RowAggrThreads>4</RowAggrThreads> --> <!-- Default value is the number of cores -->
|
<!-- <RowAggrThreads>4</RowAggrThreads> --> <!-- Default value is the number of cores -->
|
||||||
|
@ -1387,6 +1387,7 @@ void BatchPrimitiveProcessor::execute()
|
|||||||
#ifdef PRIMPROC_STOPWATCH
|
#ifdef PRIMPROC_STOPWATCH
|
||||||
stopwatch->start("BatchPrimitiveProcessor::execute first part");
|
stopwatch->start("BatchPrimitiveProcessor::execute first part");
|
||||||
#endif
|
#endif
|
||||||
|
utils::setThreadName("BPPFilt&Pr");
|
||||||
|
|
||||||
// if only one scan step which has no predicate, async load all columns
|
// if only one scan step which has no predicate, async load all columns
|
||||||
if (filterCount == 1 && hasScan)
|
if (filterCount == 1 && hasScan)
|
||||||
@ -1550,6 +1551,8 @@ void BatchPrimitiveProcessor::execute()
|
|||||||
#endif
|
#endif
|
||||||
outputRG.resetRowGroup(baseRid);
|
outputRG.resetRowGroup(baseRid);
|
||||||
|
|
||||||
|
utils::setThreadName("BPPFE1_1");
|
||||||
|
|
||||||
if (fe1)
|
if (fe1)
|
||||||
{
|
{
|
||||||
uint32_t newRidCount = 0;
|
uint32_t newRidCount = 0;
|
||||||
@ -1616,6 +1619,8 @@ void BatchPrimitiveProcessor::execute()
|
|||||||
}
|
}
|
||||||
if (fe2)
|
if (fe2)
|
||||||
{
|
{
|
||||||
|
utils::setThreadName("BPPFE2_1");
|
||||||
|
|
||||||
/* functionize this -> processFE2() */
|
/* functionize this -> processFE2() */
|
||||||
fe2Output.resetRowGroup(baseRid);
|
fe2Output.resetRowGroup(baseRid);
|
||||||
fe2Output.getRow(0, &fe2Out);
|
fe2Output.getRow(0, &fe2Out);
|
||||||
@ -1646,6 +1651,8 @@ void BatchPrimitiveProcessor::execute()
|
|||||||
|
|
||||||
if (fAggregator)
|
if (fAggregator)
|
||||||
{
|
{
|
||||||
|
utils::setThreadName("BPPAgg_1");
|
||||||
|
|
||||||
*serialized << (uint8_t)1; // the "count this msg" var
|
*serialized << (uint8_t)1; // the "count this msg" var
|
||||||
|
|
||||||
// see TupleBPS::setFcnExpGroup2() and where it gets called.
|
// see TupleBPS::setFcnExpGroup2() and where it gets called.
|
||||||
@ -1662,17 +1669,17 @@ void BatchPrimitiveProcessor::execute()
|
|||||||
|
|
||||||
if ((currentBlockOffset + 1) == count) // @bug4507, 8k
|
if ((currentBlockOffset + 1) == count) // @bug4507, 8k
|
||||||
{
|
{
|
||||||
fAggregator->loadResult(*serialized); // @bug4507, 8k
|
fAggregator->loadResult(*serialized); // @bug4507, 8k
|
||||||
} // @bug4507, 8k
|
} // @bug4507, 8k
|
||||||
else if (utils::MonitorProcMem::isMemAvailable()) // @bug4507, 8k
|
else if (utils::MonitorProcMem::isMemAvailable()) // @bug4507, 8k
|
||||||
{
|
{
|
||||||
fAggregator->loadEmptySet(*serialized); // @bug4507, 8k
|
fAggregator->loadEmptySet(*serialized); // @bug4507, 8k
|
||||||
} // @bug4507, 8k
|
} // @bug4507, 8k
|
||||||
else // @bug4507, 8k
|
else // @bug4507, 8k
|
||||||
{
|
{
|
||||||
fAggregator->loadResult(*serialized); // @bug4507, 8k
|
fAggregator->loadResult(*serialized); // @bug4507, 8k
|
||||||
fAggregator->aggReset(); // @bug4507, 8k
|
fAggregator->aggReset(); // @bug4507, 8k
|
||||||
} // @bug4507, 8k
|
} // @bug4507, 8k
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!fAggregator && !fe2)
|
if (!fAggregator && !fe2)
|
||||||
@ -1726,6 +1733,8 @@ void BatchPrimitiveProcessor::execute()
|
|||||||
|
|
||||||
do // while (startRid > 0)
|
do // while (startRid > 0)
|
||||||
{
|
{
|
||||||
|
utils::setThreadName("BPPJoin_1");
|
||||||
|
|
||||||
#ifdef PRIMPROC_STOPWATCH
|
#ifdef PRIMPROC_STOPWATCH
|
||||||
stopwatch->start("-- executeTupleJoin()");
|
stopwatch->start("-- executeTupleJoin()");
|
||||||
startRid = executeTupleJoin(startRid, largeSideRowGroup);
|
startRid = executeTupleJoin(startRid, largeSideRowGroup);
|
||||||
@ -1777,6 +1786,8 @@ void BatchPrimitiveProcessor::execute()
|
|||||||
*serialized << sendCount;
|
*serialized << sendCount;
|
||||||
if (fe2)
|
if (fe2)
|
||||||
{
|
{
|
||||||
|
utils::setThreadName("BPPFE2_2");
|
||||||
|
|
||||||
/* functionize this -> processFE2()*/
|
/* functionize this -> processFE2()*/
|
||||||
fe2Output.resetRowGroup(baseRid);
|
fe2Output.resetRowGroup(baseRid);
|
||||||
fe2Output.setDBRoot(dbRoot);
|
fe2Output.setDBRoot(dbRoot);
|
||||||
@ -1800,21 +1811,23 @@ void BatchPrimitiveProcessor::execute()
|
|||||||
|
|
||||||
if (fAggregator)
|
if (fAggregator)
|
||||||
{
|
{
|
||||||
|
utils::setThreadName("BPPAgg_2");
|
||||||
|
|
||||||
fAggregator->addRowGroup(&nextRG);
|
fAggregator->addRowGroup(&nextRG);
|
||||||
|
|
||||||
if ((currentBlockOffset + 1) == count && moreRGs == false && startRid == 0) // @bug4507, 8k
|
if ((currentBlockOffset + 1) == count && moreRGs == false && startRid == 0) // @bug4507, 8k
|
||||||
{
|
{
|
||||||
fAggregator->loadResult(*serialized); // @bug4507, 8k
|
fAggregator->loadResult(*serialized); // @bug4507, 8k
|
||||||
} // @bug4507, 8k
|
} // @bug4507, 8k
|
||||||
else if (utils::MonitorProcMem::isMemAvailable()) // @bug4507, 8k
|
else if (utils::MonitorProcMem::isMemAvailable()) // @bug4507, 8k
|
||||||
{
|
{
|
||||||
fAggregator->loadEmptySet(*serialized); // @bug4507, 8k
|
fAggregator->loadEmptySet(*serialized); // @bug4507, 8k
|
||||||
} // @bug4507, 8k
|
} // @bug4507, 8k
|
||||||
else // @bug4507, 8k
|
else // @bug4507, 8k
|
||||||
{
|
{
|
||||||
fAggregator->loadResult(*serialized); // @bug4507, 8k
|
fAggregator->loadResult(*serialized); // @bug4507, 8k
|
||||||
fAggregator->aggReset(); // @bug4507, 8k
|
fAggregator->aggReset(); // @bug4507, 8k
|
||||||
} // @bug4507, 8k
|
} // @bug4507, 8k
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
@ -1901,6 +1914,7 @@ void BatchPrimitiveProcessor::execute()
|
|||||||
// cout << "sent physIO=" << physIO << " cachedIO=" << cachedIO <<
|
// cout << "sent physIO=" << physIO << " cachedIO=" << cachedIO <<
|
||||||
// " touchedBlocks=" << touchedBlocks << endl;
|
// " touchedBlocks=" << touchedBlocks << endl;
|
||||||
}
|
}
|
||||||
|
utils::setThreadName("BPPExecuteEnd");
|
||||||
|
|
||||||
#ifdef PRIMPROC_STOPWATCH
|
#ifdef PRIMPROC_STOPWATCH
|
||||||
stopwatch->stop("BatchPrimitiveProcessor::execute fourth part");
|
stopwatch->stop("BatchPrimitiveProcessor::execute fourth part");
|
||||||
@ -2751,7 +2765,6 @@ void BatchPrimitiveProcessor::buildVSSCache(uint32_t loopCount)
|
|||||||
if (rc == 0)
|
if (rc == 0)
|
||||||
for (i = 0; i < vssData.size(); i++)
|
for (i = 0; i < vssData.size(); i++)
|
||||||
vssCache.insert(make_pair(lbidList[i], vssData[i]));
|
vssCache.insert(make_pair(lbidList[i], vssData[i]));
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
} // namespace primitiveprocessor
|
} // namespace primitiveprocessor
|
||||||
|
@ -39,7 +39,7 @@ namespace joiner
|
|||||||
// Typed joiner ctor
|
// Typed joiner ctor
|
||||||
TupleJoiner::TupleJoiner(const rowgroup::RowGroup& smallInput, const rowgroup::RowGroup& largeInput,
|
TupleJoiner::TupleJoiner(const rowgroup::RowGroup& smallInput, const rowgroup::RowGroup& largeInput,
|
||||||
uint32_t smallJoinColumn, uint32_t largeJoinColumn, JoinType jt,
|
uint32_t smallJoinColumn, uint32_t largeJoinColumn, JoinType jt,
|
||||||
threadpool::ThreadPool* jsThreadPool)
|
threadpool::ThreadPool* jsThreadPool, const uint64_t numCores)
|
||||||
: smallRG(smallInput)
|
: smallRG(smallInput)
|
||||||
, largeRG(largeInput)
|
, largeRG(largeInput)
|
||||||
, joinAlg(INSERTING)
|
, joinAlg(INSERTING)
|
||||||
@ -49,6 +49,7 @@ TupleJoiner::TupleJoiner(const rowgroup::RowGroup& smallInput, const rowgroup::R
|
|||||||
, bSignedUnsignedJoin(false)
|
, bSignedUnsignedJoin(false)
|
||||||
, uniqueLimit(100)
|
, uniqueLimit(100)
|
||||||
, finished(false)
|
, finished(false)
|
||||||
|
, numCores(numCores)
|
||||||
, jobstepThreadPool(jsThreadPool)
|
, jobstepThreadPool(jsThreadPool)
|
||||||
, _convertToDiskJoin(false)
|
, _convertToDiskJoin(false)
|
||||||
{
|
{
|
||||||
@ -145,7 +146,7 @@ TupleJoiner::TupleJoiner(const rowgroup::RowGroup& smallInput, const rowgroup::R
|
|||||||
// Typeless joiner ctor
|
// Typeless joiner ctor
|
||||||
TupleJoiner::TupleJoiner(const rowgroup::RowGroup& smallInput, const rowgroup::RowGroup& largeInput,
|
TupleJoiner::TupleJoiner(const rowgroup::RowGroup& smallInput, const rowgroup::RowGroup& largeInput,
|
||||||
const vector<uint32_t>& smallJoinColumns, const vector<uint32_t>& largeJoinColumns,
|
const vector<uint32_t>& smallJoinColumns, const vector<uint32_t>& largeJoinColumns,
|
||||||
JoinType jt, threadpool::ThreadPool* jsThreadPool)
|
JoinType jt, threadpool::ThreadPool* jsThreadPool, const uint64_t numCores)
|
||||||
: smallRG(smallInput)
|
: smallRG(smallInput)
|
||||||
, largeRG(largeInput)
|
, largeRG(largeInput)
|
||||||
, joinAlg(INSERTING)
|
, joinAlg(INSERTING)
|
||||||
@ -157,6 +158,7 @@ TupleJoiner::TupleJoiner(const rowgroup::RowGroup& smallInput, const rowgroup::R
|
|||||||
, bSignedUnsignedJoin(false)
|
, bSignedUnsignedJoin(false)
|
||||||
, uniqueLimit(100)
|
, uniqueLimit(100)
|
||||||
, finished(false)
|
, finished(false)
|
||||||
|
, numCores(numCores)
|
||||||
, jobstepThreadPool(jsThreadPool)
|
, jobstepThreadPool(jsThreadPool)
|
||||||
, _convertToDiskJoin(false)
|
, _convertToDiskJoin(false)
|
||||||
{
|
{
|
||||||
@ -254,11 +256,6 @@ bool TupleJoiner::operator<(const TupleJoiner& tj) const
|
|||||||
|
|
||||||
void TupleJoiner::getBucketCount()
|
void TupleJoiner::getBucketCount()
|
||||||
{
|
{
|
||||||
// get the # of cores, round up to nearest power of 2
|
|
||||||
// make the bucket mask
|
|
||||||
numCores = sysconf(_SC_NPROCESSORS_ONLN);
|
|
||||||
if (numCores <= 0)
|
|
||||||
numCores = 8;
|
|
||||||
bucketCount = (numCores == 1 ? 1 : (1 << (32 - __builtin_clz(numCores - 1))));
|
bucketCount = (numCores == 1 ? 1 : (1 << (32 - __builtin_clz(numCores - 1))));
|
||||||
bucketMask = bucketCount - 1;
|
bucketMask = bucketCount - 1;
|
||||||
}
|
}
|
||||||
|
@ -268,12 +268,12 @@ class TupleJoiner
|
|||||||
/* ctor to use for numeric join */
|
/* ctor to use for numeric join */
|
||||||
TupleJoiner(const rowgroup::RowGroup& smallInput, const rowgroup::RowGroup& largeInput,
|
TupleJoiner(const rowgroup::RowGroup& smallInput, const rowgroup::RowGroup& largeInput,
|
||||||
uint32_t smallJoinColumn, uint32_t largeJoinColumn, joblist::JoinType jt,
|
uint32_t smallJoinColumn, uint32_t largeJoinColumn, joblist::JoinType jt,
|
||||||
threadpool::ThreadPool* jsThreadPool);
|
threadpool::ThreadPool* jsThreadPool, const uint64_t numCores);
|
||||||
|
|
||||||
/* ctor to use for string & compound join */
|
/* ctor to use for string & compound join */
|
||||||
TupleJoiner(const rowgroup::RowGroup& smallInput, const rowgroup::RowGroup& largeInput,
|
TupleJoiner(const rowgroup::RowGroup& smallInput, const rowgroup::RowGroup& largeInput,
|
||||||
const std::vector<uint32_t>& smallJoinColumns, const std::vector<uint32_t>& largeJoinColumns,
|
const std::vector<uint32_t>& smallJoinColumns, const std::vector<uint32_t>& largeJoinColumns,
|
||||||
joblist::JoinType jt, threadpool::ThreadPool* jsThreadPool);
|
joblist::JoinType jt, threadpool::ThreadPool* jsThreadPool, const uint64_t numCores);
|
||||||
|
|
||||||
~TupleJoiner();
|
~TupleJoiner();
|
||||||
|
|
||||||
|
Loading…
x
Reference in New Issue
Block a user