1
0
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:
drrtuy 2024-12-03 22:17:49 +00:00 committed by GitHub
parent bba2133cd0
commit 6445f4dff3
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
9 changed files with 76 additions and 74 deletions

View File

@ -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
{ {

View File

@ -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;

View File

@ -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
<< " "; << " ";

View File

@ -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();
} }

View File

@ -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();
} }

View File

@ -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 -->

View File

@ -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

View File

@ -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;
} }

View File

@ -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();