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

@ -419,8 +419,9 @@ void DiskJoinStep::joinFcn()
while (largeData) while (largeData)
{ {
l_largeRG.setData(largeData.get()); l_largeRG.setData(largeData.get());
thjs->joinOneRG(0, &joinResults, l_largeRG, l_outputRG, l_largeRow, l_joinFERow, l_outputRow, baseRow, thjs->joinOneRG(0, joinResults, l_largeRG, l_outputRG, l_largeRow, l_joinFERow, l_outputRow, baseRow,
joinMatches, smallRowTemplates, &joiners, &colMappings, &fergMappings, &smallNullMem); joinMatches, smallRowTemplates, outputDL.get(), &joiners, &colMappings, &fergMappings,
&smallNullMem);
for (j = 0; j < (int)joinResults.size(); j++) for (j = 0; j < (int)joinResults.size(); j++)
{ {
@ -428,7 +429,7 @@ void DiskJoinStep::joinFcn()
// cout << "got joined output " << l_outputRG.toString() << endl; // cout << "got joined output " << l_outputRG.toString() << endl;
outputDL->insert(joinResults[j]); outputDL->insert(joinResults[j]);
} }
thjs->returnMemory();
joinResults.clear(); joinResults.clear();
largeData = in->jp->getNextLargeRGData(); largeData = in->jp->getNextLargeRGData();
} }
@ -477,6 +478,22 @@ void DiskJoinStep::joinFcn()
{ {
outputDL->insert(rgData); outputDL->insert(rgData);
// cout << "inserting a full RG" << endl; // cout << "inserting a full RG" << endl;
if (thjs)
{
if (!thjs->getMemory(l_outputRG.getMaxDataSize()))
{
// calculate guess of size required for error message
uint64_t memReqd = (unmatched.size() * outputRG.getDataSize(1)) / 1048576;
Message::Args args;
args.add(memReqd);
args.add(thjs->resourceManager->getConfiguredUMMemLimit() / 1048576);
std::cerr << logging::IDBErrorInfo::instance()->errorMsg(logging::ERR_JOIN_RESULT_TOO_BIG,
args)
<< " @" << __FILE__ << ":" << __LINE__;
throw logging::IDBExcept(logging::ERR_JOIN_RESULT_TOO_BIG, args);
}
}
rgData.reinit(l_outputRG); rgData.reinit(l_outputRG);
l_outputRG.setData(&rgData); l_outputRG.setData(&rgData);
l_outputRG.resetRowGroup(0); l_outputRG.resetRowGroup(0);
@ -491,6 +508,10 @@ void DiskJoinStep::joinFcn()
// cout << "inserting an rg with " << l_outputRG.getRowCount() << endl; // cout << "inserting an rg with " << l_outputRG.getRowCount() << endl;
outputDL->insert(rgData); outputDL->insert(rgData);
} }
if (thjs)
{
thjs->returnMemory();
}
} }
} }
} }

View File

@ -297,7 +297,11 @@ void DistributedEngineComm::Setup()
writeToLog(__FILE__, __LINE__, writeToLog(__FILE__, __LINE__,
"Could not connect to PMS" + std::to_string(connectionId) + ": " + ex.what(), "Could not connect to PMS" + std::to_string(connectionId) + ": " + ex.what(),
LOG_TYPE_ERROR); LOG_TYPE_ERROR);
cerr << "Could not connect to PMS" << std::to_string(connectionId) << ": " << ex.what() << endl; if (newPmCount == 0)
{
writeToLog(__FILE__, __LINE__, "No more PMs to try to connect to", LOG_TYPE_ERROR);
break;
}
} }
catch (...) catch (...)
{ {
@ -306,6 +310,11 @@ void DistributedEngineComm::Setup()
writeToLog(__FILE__, __LINE__, "Could not connect to PMS" + std::to_string(connectionId), writeToLog(__FILE__, __LINE__, "Could not connect to PMS" + std::to_string(connectionId),
LOG_TYPE_ERROR); LOG_TYPE_ERROR);
if (newPmCount == 0)
{
writeToLog(__FILE__, __LINE__, "No more PMs to try to connect to", LOG_TYPE_ERROR);
break;
}
} }
} }

View File

@ -763,13 +763,13 @@ void GroupConcatOrderBy::processRow(const rowgroup::Row& row)
fDataQueue.push(fData); fDataQueue.push(fData);
uint64_t newSize = fRowsPerRG * fRowGroup.getRowSize(); uint64_t newSize = fRowsPerRG * fRowGroup.getRowSize();
fMemSize += newSize;
if (!fRm->getMemory(newSize, fSessionMemLimit)) if (!fRm->getMemory(newSize, fSessionMemLimit))
{ {
cerr << IDBErrorInfo::instance()->errorMsg(fErrorCode) << " @" << __FILE__ << ":" << __LINE__; cerr << IDBErrorInfo::instance()->errorMsg(fErrorCode) << " @" << __FILE__ << ":" << __LINE__;
throw IDBExcept(fErrorCode); throw IDBExcept(fErrorCode);
} }
fMemSize += newSize;
fData.reinit(fRowGroup, fRowsPerRG); fData.reinit(fRowGroup, fRowsPerRG);
fRowGroup.setData(&fData); fRowGroup.setData(&fData);
@ -945,13 +945,13 @@ void GroupConcatNoOrder::initialize(const rowgroup::SP_GroupConcat& gcc)
fConcatColumns.push_back((*(i++)).second); fConcatColumns.push_back((*(i++)).second);
uint64_t newSize = fRowsPerRG * fRowGroup.getRowSize(); uint64_t newSize = fRowsPerRG * fRowGroup.getRowSize();
fMemSize += newSize;
if (!fRm->getMemory(newSize, fSessionMemLimit)) if (!fRm->getMemory(newSize, fSessionMemLimit))
{ {
cerr << IDBErrorInfo::instance()->errorMsg(fErrorCode) << " @" << __FILE__ << ":" << __LINE__; cerr << IDBErrorInfo::instance()->errorMsg(fErrorCode) << " @" << __FILE__ << ":" << __LINE__;
throw IDBExcept(fErrorCode); throw IDBExcept(fErrorCode);
} }
fMemSize += newSize;
fData.reinit(fRowGroup, fRowsPerRG); fData.reinit(fRowGroup, fRowsPerRG);
fRowGroup.setData(&fData); fRowGroup.setData(&fData);
@ -978,13 +978,12 @@ void GroupConcatNoOrder::processRow(const rowgroup::Row& row)
{ {
uint64_t newSize = fRowsPerRG * fRowGroup.getRowSize(); uint64_t newSize = fRowsPerRG * fRowGroup.getRowSize();
fMemSize += newSize;
if (!fRm->getMemory(newSize, fSessionMemLimit)) if (!fRm->getMemory(newSize, fSessionMemLimit))
{ {
cerr << IDBErrorInfo::instance()->errorMsg(fErrorCode) << " @" << __FILE__ << ":" << __LINE__; cerr << IDBErrorInfo::instance()->errorMsg(fErrorCode) << " @" << __FILE__ << ":" << __LINE__;
throw IDBExcept(fErrorCode); throw IDBExcept(fErrorCode);
} }
fMemSize += newSize;
fDataQueue.push(fData); fDataQueue.push(fData);
fData.reinit(fRowGroup, fRowsPerRG); fData.reinit(fRowGroup, fRowsPerRG);

View File

@ -123,12 +123,12 @@ void LimitedOrderBy::processRow(const rowgroup::Row& row)
fUncommitedMemory += memSizeInc; fUncommitedMemory += memSizeInc;
if (fUncommitedMemory >= fMaxUncommited) if (fUncommitedMemory >= fMaxUncommited)
{ {
fMemSize += fUncommitedMemory;
if (!fRm->getMemory(fUncommitedMemory, fSessionMemLimit)) if (!fRm->getMemory(fUncommitedMemory, fSessionMemLimit))
{ {
cerr << IDBErrorInfo::instance()->errorMsg(fErrorCode) << " @" << __FILE__ << ":" << __LINE__; cerr << IDBErrorInfo::instance()->errorMsg(fErrorCode) << " @" << __FILE__ << ":" << __LINE__;
throw IDBExcept(fErrorCode); throw IDBExcept(fErrorCode);
} }
fMemSize += fUncommitedMemory;
fUncommitedMemory = 0; fUncommitedMemory = 0;
} }
@ -143,13 +143,13 @@ void LimitedOrderBy::processRow(const rowgroup::Row& row)
{ {
fDataQueue.push(fData); fDataQueue.push(fData);
uint64_t newSize = fRowGroup.getSizeWithStrings() - fRowGroup.getHeaderSize(); uint64_t newSize = fRowGroup.getSizeWithStrings() - fRowGroup.getHeaderSize();
fMemSize += newSize;
if (!fRm->getMemory(newSize, fSessionMemLimit)) if (!fRm->getMemory(newSize, fSessionMemLimit))
{ {
cerr << IDBErrorInfo::instance()->errorMsg(fErrorCode) << " @" << __FILE__ << ":" << __LINE__; cerr << IDBErrorInfo::instance()->errorMsg(fErrorCode) << " @" << __FILE__ << ":" << __LINE__;
throw IDBExcept(fErrorCode); throw IDBExcept(fErrorCode);
} }
fMemSize += newSize;
fData.reinit(fRowGroup, fRowsPerRG); fData.reinit(fRowGroup, fRowsPerRG);
fRowGroup.setData(&fData); fRowGroup.setData(&fData);
@ -184,12 +184,12 @@ void LimitedOrderBy::finalize()
{ {
if (fUncommitedMemory > 0) if (fUncommitedMemory > 0)
{ {
fMemSize += fUncommitedMemory;
if (!fRm->getMemory(fUncommitedMemory, fSessionMemLimit)) if (!fRm->getMemory(fUncommitedMemory, fSessionMemLimit))
{ {
cerr << IDBErrorInfo::instance()->errorMsg(fErrorCode) << " @" << __FILE__ << ":" << __LINE__; cerr << IDBErrorInfo::instance()->errorMsg(fErrorCode) << " @" << __FILE__ << ":" << __LINE__;
throw IDBExcept(fErrorCode); throw IDBExcept(fErrorCode);
} }
fMemSize += fUncommitedMemory;
fUncommitedMemory = 0; fUncommitedMemory = 0;
} }
@ -202,13 +202,13 @@ void LimitedOrderBy::finalize()
// *DRRTUY Very memory intensive. CS needs to account active // *DRRTUY Very memory intensive. CS needs to account active
// memory only and release memory if needed. // memory only and release memory if needed.
uint64_t memSizeInc = fRowGroup.getSizeWithStrings() - fRowGroup.getHeaderSize(); uint64_t memSizeInc = fRowGroup.getSizeWithStrings() - fRowGroup.getHeaderSize();
fMemSize += memSizeInc;
if (!fRm->getMemory(memSizeInc, fSessionMemLimit)) if (!fRm->getMemory(memSizeInc, fSessionMemLimit))
{ {
cerr << IDBErrorInfo::instance()->errorMsg(fErrorCode) << " @" << __FILE__ << ":" << __LINE__; cerr << IDBErrorInfo::instance()->errorMsg(fErrorCode) << " @" << __FILE__ << ":" << __LINE__;
throw IDBExcept(fErrorCode); throw IDBExcept(fErrorCode);
} }
fMemSize += memSizeInc;
uint64_t offset = 0; uint64_t offset = 0;
uint64_t i = 0; uint64_t i = 0;
@ -256,13 +256,13 @@ void LimitedOrderBy::finalize()
if (offset == (uint64_t)-1) if (offset == (uint64_t)-1)
{ {
tempRGDataList.push_front(fData); tempRGDataList.push_front(fData);
fMemSize += memSizeInc;
if (!fRm->getMemory(memSizeInc, fSessionMemLimit)) if (!fRm->getMemory(memSizeInc, fSessionMemLimit))
{ {
cerr << IDBErrorInfo::instance()->errorMsg(fErrorCode) << " @" << __FILE__ << ":" << __LINE__; cerr << IDBErrorInfo::instance()->errorMsg(fErrorCode) << " @" << __FILE__ << ":" << __LINE__;
throw IDBExcept(fErrorCode); throw IDBExcept(fErrorCode);
} }
fMemSize += memSizeInc;
fData.reinit(fRowGroup, fRowsPerRG); fData.reinit(fRowGroup, fRowsPerRG);
fRowGroup.setData(&fData); fRowGroup.setData(&fData);

View File

@ -1349,6 +1349,14 @@ class TupleBPS : public BatchPrimitive, public TupleDeliveryStep
{ {
return true; return true;
} }
ResourceManager* resourceManager() const
{
return fRm;
}
bool runFEonPM() const
{
return bRunFEonPM;
}
protected: protected:
void sendError(uint16_t status); void sendError(uint16_t status);
@ -1438,12 +1446,6 @@ class TupleBPS : public BatchPrimitive, public TupleDeliveryStep
void serializeJoiner(); void serializeJoiner();
void serializeJoiner(uint32_t connectionNumber); void serializeJoiner(uint32_t connectionNumber);
void generateJoinResultSet(const std::vector<std::vector<rowgroup::Row::Pointer>>& joinerOutput,
rowgroup::Row& baseRow, const std::vector<boost::shared_array<int>>& mappings,
const uint32_t depth, rowgroup::RowGroup& outputRG, rowgroup::RGData& rgData,
std::vector<rowgroup::RGData>* outputData,
const boost::scoped_array<rowgroup::Row>& smallRows, rowgroup::Row& joinedRow);
std::vector<boost::shared_ptr<joiner::TupleJoiner>> tjoiners; std::vector<boost::shared_ptr<joiner::TupleJoiner>> tjoiners;
bool doJoin, hasPMJoin, hasUMJoin; bool doJoin, hasPMJoin, hasUMJoin;
std::vector<rowgroup::RowGroup> joinerMatchesRGs; // parses the small-side matches from joiner std::vector<rowgroup::RowGroup> joinerMatchesRGs; // parses the small-side matches from joiner
@ -1475,18 +1477,12 @@ class TupleBPS : public BatchPrimitive, public TupleDeliveryStep
boost::shared_ptr<funcexp::FuncExpWrapper> fe1, fe2; boost::shared_ptr<funcexp::FuncExpWrapper> fe1, fe2;
rowgroup::RowGroup fe1Input, fe2Output; rowgroup::RowGroup fe1Input, fe2Output;
boost::shared_array<int> fe2Mapping; boost::shared_array<int> fe2Mapping;
bool runFEonPM; bool bRunFEonPM;
/* for UM F & E 2 processing */ /* for UM F & E 2 processing */
rowgroup::RGData fe2Data; rowgroup::RGData fe2Data;
rowgroup::Row fe2InRow, fe2OutRow; rowgroup::Row fe2InRow, fe2OutRow;
void processFE2(rowgroup::RowGroup& input, rowgroup::RowGroup& output, rowgroup::Row& inRow,
rowgroup::Row& outRow, std::vector<rowgroup::RGData>* rgData,
funcexp::FuncExpWrapper* localFE2);
void processFE2_oneRG(rowgroup::RowGroup& input, rowgroup::RowGroup& output, rowgroup::Row& inRow,
rowgroup::Row& outRow, funcexp::FuncExpWrapper* localFE2);
/* Runtime Casual Partitioning adjustments. The CP code is needlessly complicated; /* Runtime Casual Partitioning adjustments. The CP code is needlessly complicated;
* to avoid making it worse, decided to designate 'scanFlags' as the static * to avoid making it worse, decided to designate 'scanFlags' as the static
* component and this new array as the runtime component. The final CP decision * component and this new array as the runtime component. The final CP decision
@ -1500,8 +1496,9 @@ class TupleBPS : public BatchPrimitive, public TupleDeliveryStep
boost::shared_ptr<RowGroupDL> deliveryDL; boost::shared_ptr<RowGroupDL> deliveryDL;
uint32_t deliveryIt; uint32_t deliveryIt;
struct JoinLocalData class JoinLocalData
{ {
public:
JoinLocalData() = delete; JoinLocalData() = delete;
JoinLocalData(const JoinLocalData&) = delete; JoinLocalData(const JoinLocalData&) = delete;
JoinLocalData(JoinLocalData&&) = delete; JoinLocalData(JoinLocalData&&) = delete;
@ -1509,12 +1506,20 @@ class TupleBPS : public BatchPrimitive, public TupleDeliveryStep
JoinLocalData& operator=(JoinLocalData&&) = delete; JoinLocalData& operator=(JoinLocalData&&) = delete;
~JoinLocalData() = default; ~JoinLocalData() = default;
JoinLocalData(rowgroup::RowGroup& primRowGroup, rowgroup::RowGroup& outputRowGroup, JoinLocalData(TupleBPS* pTupleBPS, rowgroup::RowGroup& primRowGroup, rowgroup::RowGroup& outputRowGroup,
boost::shared_ptr<funcexp::FuncExpWrapper>& fe2, rowgroup::RowGroup& fe2Output, boost::shared_ptr<funcexp::FuncExpWrapper>& fe2, rowgroup::RowGroup& fe2Output,
std::vector<rowgroup::RowGroup>& joinerMatchesRGs, rowgroup::RowGroup& joinFERG, std::vector<rowgroup::RowGroup>& joinerMatchesRGs, rowgroup::RowGroup& joinFERG,
std::vector<boost::shared_ptr<joiner::TupleJoiner>>& tjoiners, uint32_t smallSideCount, std::vector<boost::shared_ptr<joiner::TupleJoiner>>& tjoiners, uint32_t smallSideCount,
bool doJoin); bool doJoin);
friend class TupleBPS;
private:
uint64_t generateJoinResultSet(const uint32_t depth, std::vector<rowgroup::RGData>& outputData,
RowGroupDL* dlp);
void processFE2(vector<rowgroup::RGData>& rgData);
TupleBPS* tbps; // Parent
rowgroup::RowGroup local_primRG; rowgroup::RowGroup local_primRG;
rowgroup::RowGroup local_outputRG; rowgroup::RowGroup local_outputRG;
@ -1576,7 +1581,7 @@ class TupleBPS : public BatchPrimitive, public TupleDeliveryStep
for (uint32_t i = 0; i < numThreads; ++i) for (uint32_t i = 0; i < numThreads; ++i)
{ {
joinLocalDataPool.push_back(std::shared_ptr<JoinLocalData>( joinLocalDataPool.push_back(std::shared_ptr<JoinLocalData>(
new JoinLocalData(primRowGroup, outputRowGroup, fe2, fe2Output, joinerMatchesRGs, joinFERG, new JoinLocalData(this, primRowGroup, outputRowGroup, fe2, fe2Output, joinerMatchesRGs, joinFERG,
tjoiners, smallSideCount, doJoin))); tjoiners, smallSideCount, doJoin)));
} }

View File

@ -261,6 +261,9 @@ ResourceManager::ResourceManager(bool runningInExeMgr)
fAllowedDiskAggregation = fAllowedDiskAggregation =
getBoolVal(fRowAggregationStr, "AllowDiskBasedAggregation", defaultAllowDiskAggregation); getBoolVal(fRowAggregationStr, "AllowDiskBasedAggregation", defaultAllowDiskAggregation);
fMaxBPPSendQueue = getUintVal(fPrimitiveServersStr, "MaxBPPSendQueue", defaultMaxBPPSendQueue);
if (!load_encryption_keys()) if (!load_encryption_keys())
{ {
Logger log; Logger log;
@ -463,23 +466,51 @@ bool ResourceManager::userPriorityEnabled() const
return "Y" == val; return "Y" == val;
} }
bool ResourceManager::getMemory(int64_t amount, boost::shared_ptr<int64_t> sessionLimit, bool patience) // Counts memory. This funtion doesn't actually malloc, just counts against two limits
// totalUmMemLimit for overall UM counting and (optional) sessionLimit for a single session.
// If both have space, return true.
bool ResourceManager::getMemory(int64_t amount, boost::shared_ptr<int64_t>& sessionLimit, bool patience)
{ {
bool ret1 = (atomicops::atomicSub(&totalUmMemLimit, amount) >= 0); bool ret1 = (atomicops::atomicSub(&totalUmMemLimit, amount) >= 0);
bool ret2 = (atomicops::atomicSub(sessionLimit.get(), amount) >= 0); bool ret2 = sessionLimit ? (atomicops::atomicSub(sessionLimit.get(), amount) >= 0) : ret1;
uint32_t retryCounter = 0, maxRetries = 20; // 10s delay uint32_t retryCounter = 0, maxRetries = 20; // 10s delay
while (patience && !(ret1 && ret2) && retryCounter++ < maxRetries) while (patience && !(ret1 && ret2) && retryCounter++ < maxRetries)
{ {
atomicops::atomicAdd(&totalUmMemLimit, amount); atomicops::atomicAdd(&totalUmMemLimit, amount);
atomicops::atomicAdd(sessionLimit.get(), amount); sessionLimit ? atomicops::atomicAdd(sessionLimit.get(), amount) : 0;
usleep(500000); usleep(500000);
ret1 = (atomicops::atomicSub(&totalUmMemLimit, amount) >= 0); ret1 = (atomicops::atomicSub(&totalUmMemLimit, amount) >= 0);
ret2 = (atomicops::atomicSub(sessionLimit.get(), amount) >= 0); ret2 = sessionLimit ? (atomicops::atomicSub(sessionLimit.get(), amount) >= 0) : ret1;
}
if (!(ret1 && ret2))
{
// If we didn't get any memory, restore the counters.
atomicops::atomicAdd(&totalUmMemLimit, amount);
sessionLimit ? atomicops::atomicAdd(sessionLimit.get(), amount) : 0;
} }
return (ret1 && ret2); return (ret1 && ret2);
} }
// Don't care about session memory
bool ResourceManager::getMemory(int64_t amount, bool patience)
{
bool ret1 = (atomicops::atomicSub(&totalUmMemLimit, amount) >= 0);
uint32_t retryCounter = 0, maxRetries = 20; // 10s delay
while (patience && !ret1 && retryCounter++ < maxRetries)
{
atomicops::atomicAdd(&totalUmMemLimit, amount);
usleep(500000);
ret1 = (atomicops::atomicSub(&totalUmMemLimit, amount) >= 0);
}
if (!ret1)
{
// If we didn't get any memory, restore the counters.
atomicops::atomicAdd(&totalUmMemLimit, amount);
}
return ret1;
}
} // namespace joblist } // namespace joblist

View File

@ -82,6 +82,17 @@ const uint32_t defaultMaxOutstandingRequests = 20;
const uint32_t defaultProcessorThreadsPerScan = 16; const uint32_t defaultProcessorThreadsPerScan = 16;
const uint32_t defaultJoinerChunkSize = 16 * 1024 * 1024; const uint32_t defaultJoinerChunkSize = 16 * 1024 * 1024;
// I estimate that the average non-cloud columnstore node has 64GB. I've seen from 16GB to 256GB. Cloud can be
// as low as 4GB However, ExeMgr has a targetRecvQueueSize hardcoded to 50,000,000, so some number greater
// than this makes sense. Seriously greater doesn't make sense, so I went with 5x. If there are a number of
// simultaneous queries that return giant result sets, then 0.25 GB each seems reasonable. This is only for
// the return queue. We still need room for all the processing, and if a single node system, for ExeMgr as
// well. On small systems, I recommend we use a smaller value. I believe a larger value will not improve
// anything since at this point, we're just filling a queue much faster than it can be emptied. Even if we
// make this default larger, giant results will still eventually block. Just with less memory available for
// other processing.
const uint64_t defaultMaxBPPSendQueue = 250000000; // ~250MB
// bucketreuse // bucketreuse
const std::string defaultTempDiskPath = "/tmp"; const std::string defaultTempDiskPath = "/tmp";
const std::string defaultWorkingDir = "."; //"/tmp"; const std::string defaultWorkingDir = "."; //"/tmp";
@ -382,6 +393,11 @@ class ResourceManager
return getUintVal(fJobListStr, "DECThrottleThreshold", defaultDECThrottleThreshold); return getUintVal(fJobListStr, "DECThrottleThreshold", defaultDECThrottleThreshold);
} }
uint64_t getMaxBPPSendQueue() const
{
return fMaxBPPSendQueue;
}
EXPORT void emServerThreads(); EXPORT void emServerThreads();
EXPORT void emServerQueueSize(); EXPORT void emServerQueueSize();
EXPORT void emSecondsBetweenMemChecks(); EXPORT void emSecondsBetweenMemChecks();
@ -399,11 +415,16 @@ class ResourceManager
/* sessionLimit is a pointer to the var holding the session-scope limit, should be JobInfo.umMemLimit /* sessionLimit is a pointer to the var holding the session-scope limit, should be JobInfo.umMemLimit
for the query. */ for the query. */
/* Temporary parameter 'patience', will wait for up to 10s to get the memory. */ /* Temporary parameter 'patience', will wait for up to 10s to get the memory. */
EXPORT bool getMemory(int64_t amount, boost::shared_ptr<int64_t> sessionLimit, bool patience = true); EXPORT bool getMemory(int64_t amount, boost::shared_ptr<int64_t>& sessionLimit, bool patience = true);
inline void returnMemory(int64_t amount, boost::shared_ptr<int64_t> sessionLimit) EXPORT bool getMemory(int64_t amount, bool patience = true);
inline void returnMemory(int64_t amount)
{ {
atomicops::atomicAdd(&totalUmMemLimit, amount); atomicops::atomicAdd(&totalUmMemLimit, amount);
atomicops::atomicAdd(sessionLimit.get(), amount); }
inline void returnMemory(int64_t amount, boost::shared_ptr<int64_t>& sessionLimit)
{
atomicops::atomicAdd(&totalUmMemLimit, amount);
sessionLimit ? atomicops::atomicAdd(sessionLimit.get(), amount) : 0;
} }
inline int64_t availableMemory() const inline int64_t availableMemory() const
{ {
@ -607,7 +628,7 @@ class ResourceManager
/* new HJ/Union/Aggregation support */ /* new HJ/Union/Aggregation support */
volatile int64_t totalUmMemLimit; // mem limit for join, union, and aggregation on the UM volatile int64_t totalUmMemLimit; // mem limit for join, union, and aggregation on the UM
uint64_t configuredUmMemLimit; int64_t configuredUmMemLimit;
uint64_t pmJoinMemLimit; // mem limit on individual PM joins uint64_t pmJoinMemLimit; // mem limit on individual PM joins
/* multi-thread aggregate */ /* multi-thread aggregate */
@ -622,6 +643,7 @@ class ResourceManager
bool fUseHdfs; bool fUseHdfs;
bool fAllowedDiskAggregation{false}; bool fAllowedDiskAggregation{false};
uint64_t fDECConnectionsPerQuery; uint64_t fDECConnectionsPerQuery;
uint64_t fMaxBPPSendQueue = 250000000;
}; };
inline std::string ResourceManager::getStringVal(const std::string& section, const std::string& name, inline std::string ResourceManager::getStringVal(const std::string& section, const std::string& name,

View File

@ -155,14 +155,15 @@ struct TupleBPSAggregators
} }
}; };
TupleBPS::JoinLocalData::JoinLocalData(RowGroup& primRowGroup, RowGroup& outputRowGroup, TupleBPS::JoinLocalData::JoinLocalData(TupleBPS* pTupleBPS, RowGroup& primRowGroup, RowGroup& outputRowGroup,
boost::shared_ptr<funcexp::FuncExpWrapper>& fe2, boost::shared_ptr<funcexp::FuncExpWrapper>& fe2,
rowgroup::RowGroup& fe2Output, rowgroup::RowGroup& fe2Output,
std::vector<rowgroup::RowGroup>& joinerMatchesRGs, std::vector<rowgroup::RowGroup>& joinerMatchesRGs,
rowgroup::RowGroup& joinFERG, rowgroup::RowGroup& joinFERG,
std::vector<boost::shared_ptr<joiner::TupleJoiner>>& tjoiners, std::vector<boost::shared_ptr<joiner::TupleJoiner>>& tjoiners,
uint32_t smallSideCount, bool doJoin) uint32_t smallSideCount, bool doJoin)
: local_primRG(primRowGroup) : tbps(pTupleBPS)
, local_primRG(primRowGroup)
, local_outputRG(outputRowGroup) , local_outputRG(outputRowGroup)
, fe2(fe2) , fe2(fe2)
, fe2Output(fe2Output) , fe2Output(fe2Output)
@ -239,6 +240,130 @@ TupleBPS::JoinLocalData::JoinLocalData(RowGroup& primRowGroup, RowGroup& outputR
} }
} }
uint64_t TupleBPS::JoinLocalData::generateJoinResultSet(const uint32_t depth,
std::vector<rowgroup::RGData>& outputData,
RowGroupDL* dlp)
{
uint32_t i;
Row& smallRow = smallSideRows[depth];
uint64_t memSizeForOutputRG = 0;
if (depth < smallSideCount - 1)
{
for (i = 0; i < joinerOutput[depth].size() && !tbps->cancelled(); i++)
{
smallRow.setPointer(joinerOutput[depth][i]);
applyMapping(smallMappings[depth], smallRow, &joinedBaseRow);
memSizeForOutputRG += generateJoinResultSet(depth + 1, outputData, dlp);
}
}
else
{
local_outputRG.getRow(local_outputRG.getRowCount(), &postJoinRow);
for (i = 0; i < joinerOutput[depth].size() && !tbps->cancelled();
i++, postJoinRow.nextRow(), local_outputRG.incRowCount())
{
smallRow.setPointer(joinerOutput[depth][i]);
if (UNLIKELY(local_outputRG.getRowCount() == 8192))
{
uint32_t dbRoot = local_outputRG.getDBRoot();
uint64_t baseRid = local_outputRG.getBaseRid();
outputData.push_back(joinedData);
// Don't let the join results buffer get out of control.
if (tbps->resourceManager()->getMemory(local_outputRG.getMaxDataSize(), false))
{
memSizeForOutputRG += local_outputRG.getMaxDataSize();
}
else
{
// Don't wait for memory, just send the data on to DL.
RowGroup out(local_outputRG);
if (fe2 && tbps->runFEonPM())
{
processFE2(outputData);
tbps->rgDataVecToDl(outputData, local_fe2Output, dlp);
}
else
{
tbps->rgDataVecToDl(outputData, out, dlp);
}
tbps->resourceManager()->returnMemory(memSizeForOutputRG);
memSizeForOutputRG = 0;
}
joinedData.reinit(local_outputRG);
local_outputRG.setData(&joinedData);
local_outputRG.resetRowGroup(baseRid);
local_outputRG.setDBRoot(dbRoot);
local_outputRG.getRow(0, &postJoinRow);
}
applyMapping(smallMappings[depth], smallRow, &joinedBaseRow);
copyRow(joinedBaseRow, &postJoinRow);
}
}
return memSizeForOutputRG;
}
void TupleBPS::JoinLocalData::processFE2(vector<rowgroup::RGData>& rgData)
{
vector<RGData> results;
RGData result;
uint32_t i, j;
bool ret;
result = RGData(local_fe2Output);
local_fe2Output.setData(&result);
local_fe2Output.resetRowGroup(-1);
local_fe2Output.getRow(0, &local_fe2OutRow);
for (i = 0; i < rgData.size(); i++)
{
local_outputRG.setData(&(rgData)[i]);
if (local_fe2Output.getRowCount() == 0)
{
local_fe2Output.resetRowGroup(local_outputRG.getBaseRid());
local_fe2Output.setDBRoot(local_outputRG.getDBRoot());
}
local_outputRG.getRow(0, &postJoinRow);
for (j = 0; j < local_outputRG.getRowCount(); j++, postJoinRow.nextRow())
{
ret = local_fe2.evaluate(&postJoinRow);
if (ret)
{
applyMapping(tbps->fe2Mapping, postJoinRow, &local_fe2OutRow);
local_fe2OutRow.setRid(postJoinRow.getRelRid());
local_fe2Output.incRowCount();
local_fe2OutRow.nextRow();
if (local_fe2Output.getRowCount() == 8192 ||
local_fe2Output.getDBRoot() != local_outputRG.getDBRoot() ||
local_fe2Output.getBaseRid() != local_outputRG.getBaseRid())
{
results.push_back(result);
result = RGData(local_fe2Output);
local_fe2Output.setData(&result);
local_fe2Output.resetRowGroup(local_outputRG.getBaseRid());
local_fe2Output.setDBRoot(local_outputRG.getDBRoot());
local_fe2Output.getRow(0, &local_fe2OutRow);
}
}
}
}
if (local_fe2Output.getRowCount() > 0)
{
results.push_back(result);
}
rgData.swap(results);
}
struct ByteStreamProcessor struct ByteStreamProcessor
{ {
ByteStreamProcessor(TupleBPS* tbps, vector<boost::shared_ptr<messageqcpp::ByteStream>>& bsv, ByteStreamProcessor(TupleBPS* tbps, vector<boost::shared_ptr<messageqcpp::ByteStream>>& bsv,
@ -1265,7 +1390,7 @@ void TupleBPS::run()
if (fe1) if (fe1)
fBPP->setFEGroup1(fe1, fe1Input); fBPP->setFEGroup1(fe1, fe1Input);
if (fe2 && runFEonPM) if (fe2 && bRunFEonPM)
fBPP->setFEGroup2(fe2, fe2Output); fBPP->setFEGroup2(fe2, fe2Output);
if (fe2) if (fe2)
@ -1970,6 +2095,7 @@ void TupleBPS::processByteStreamVector(vector<boost::shared_ptr<messageqcpp::Byt
uint32_t cachedIO; uint32_t cachedIO;
uint32_t physIO; uint32_t physIO;
uint32_t touchedBlocks; uint32_t touchedBlocks;
int32_t memAmount = 0;
for (uint32_t i = begin; i < end; ++i) for (uint32_t i = begin; i < end; ++i)
{ {
@ -2131,26 +2257,7 @@ void TupleBPS::processByteStreamVector(vector<boost::shared_ptr<messageqcpp::Byt
{ {
applyMapping(data->largeMapping, data->largeSideRow, &data->joinedBaseRow); applyMapping(data->largeMapping, data->largeSideRow, &data->joinedBaseRow);
data->joinedBaseRow.setRid(data->largeSideRow.getRelRid()); data->joinedBaseRow.setRid(data->largeSideRow.getRelRid());
generateJoinResultSet(data->joinerOutput, data->joinedBaseRow, data->smallMappings, 0, memAmount += data->generateJoinResultSet(0, rgDatav, dlp);
data->local_outputRG, data->joinedData, &rgDatav, data->smallSideRows,
data->postJoinRow);
// Bug 3510: Don't let the join results buffer get out of control. Need
// to refactor this. All post-join processing needs to go here AND below
// for now.
if (rgDatav.size() * data->local_outputRG.getMaxDataSize() > 50000000)
{
RowGroup out(data->local_outputRG);
if (fe2 && !runFEonPM)
{
processFE2(out, data->local_fe2Output, data->postJoinRow, data->local_fe2OutRow, &rgDatav,
&data->local_fe2);
rgDataVecToDl(rgDatav, data->local_fe2Output, dlp);
}
else
rgDataVecToDl(rgDatav, out, dlp);
}
} }
} // end of the for-loop in the join code } // end of the for-loop in the join code
@ -2163,12 +2270,16 @@ void TupleBPS::processByteStreamVector(vector<boost::shared_ptr<messageqcpp::Byt
{ {
rgDatav.push_back(rgData); rgDatav.push_back(rgData);
} }
if (memAmount)
{
resourceManager()->returnMemory(memAmount);
memAmount = 0;
}
// Execute UM F & E group 2 on rgDatav // Execute UM F & E group 2 on rgDatav
if (fe2 && !runFEonPM && rgDatav.size() > 0 && !cancelled()) if (fe2 && !bRunFEonPM && rgDatav.size() > 0 && !cancelled())
{ {
processFE2(data->local_outputRG, data->local_fe2Output, data->postJoinRow, data->local_fe2OutRow, data->processFE2(rgDatav);
&rgDatav, &data->local_fe2);
rgDataVecToDl(rgDatav, data->local_fe2Output, dlp); rgDataVecToDl(rgDatav, data->local_fe2Output, dlp);
} }
@ -2192,7 +2303,7 @@ void TupleBPS::processByteStreamVector(vector<boost::shared_ptr<messageqcpp::Byt
// insert the resulting rowgroup data from a single bytestream into dlp // insert the resulting rowgroup data from a single bytestream into dlp
if (rgDatav.size() > 0) if (rgDatav.size() > 0)
{ {
if (fe2 && runFEonPM) if (fe2 && bRunFEonPM)
rgDataVecToDl(rgDatav, data->local_fe2Output, dlp); rgDataVecToDl(rgDatav, data->local_fe2Output, dlp);
else else
rgDataVecToDl(rgDatav, data->local_outputRG, dlp); rgDataVecToDl(rgDatav, data->local_outputRG, dlp);
@ -2339,10 +2450,7 @@ void TupleBPS::receiveMultiPrimitiveMessages()
start = end; start = end;
} }
// Join threads. jobstepThreadPool.join(fProcessorThreads);
for (uint32_t i = 0, e = fProcessorThreads.size(); i < e; ++i)
jobstepThreadPool.join(fProcessorThreads[i]);
// Clear all. // Clear all.
fProcessorThreads.clear(); fProcessorThreads.clear();
bsv.clear(); bsv.clear();
@ -2399,7 +2507,7 @@ void TupleBPS::receiveMultiPrimitiveMessages()
abort_nolock(); abort_nolock();
} }
// We have on thread here and do not need to notify any waiting producer threads, because we are done of // We have one thread here and do not need to notify any waiting producer threads, because we are done with
// consuming messages from queue. // consuming messages from queue.
tplLock.unlock(); tplLock.unlock();
@ -2447,8 +2555,7 @@ void TupleBPS::receiveMultiPrimitiveMessages()
if (fe2) if (fe2)
{ {
rgDatav.push_back(data->joinedData); rgDatav.push_back(data->joinedData);
processFE2(data->local_outputRG, data->local_fe2Output, data->postJoinRow, data->local_fe2OutRow, data->processFE2(rgDatav);
&rgDatav, &data->local_fe2);
if (rgDatav.size() > 0) if (rgDatav.size() > 0)
rgDataToDl(rgDatav[0], data->local_fe2Output, dlp); rgDataToDl(rgDatav[0], data->local_fe2Output, dlp);
@ -2470,8 +2577,7 @@ void TupleBPS::receiveMultiPrimitiveMessages()
if (fe2) if (fe2)
{ {
rgDatav.push_back(data->joinedData); rgDatav.push_back(data->joinedData);
processFE2(data->local_outputRG, data->local_fe2Output, data->postJoinRow, data->local_fe2OutRow, data->processFE2(rgDatav);
&rgDatav, &data->local_fe2);
if (rgDatav.size() > 0) if (rgDatav.size() > 0)
rgDataToDl(rgDatav[0], data->local_fe2Output, dlp); rgDataToDl(rgDatav[0], data->local_fe2Output, dlp);
@ -2811,51 +2917,6 @@ void TupleBPS::setJoinedResultRG(const rowgroup::RowGroup& rg)
fe2Mapping = makeMapping(outputRowGroup, fe2Output); fe2Mapping = makeMapping(outputRowGroup, fe2Output);
} }
/* probably worthwhile to make some of these class vars */
void TupleBPS::generateJoinResultSet(const vector<vector<Row::Pointer>>& joinerOutput, Row& baseRow,
const vector<shared_array<int>>& mappings, const uint32_t depth,
RowGroup& outputRG, RGData& rgData, vector<RGData>* outputData,
const scoped_array<Row>& smallRows, Row& joinedRow)
{
uint32_t i;
Row& smallRow = smallRows[depth];
if (depth < smallSideCount - 1)
{
for (i = 0; i < joinerOutput[depth].size(); i++)
{
smallRow.setPointer(joinerOutput[depth][i]);
applyMapping(mappings[depth], smallRow, &baseRow);
generateJoinResultSet(joinerOutput, baseRow, mappings, depth + 1, outputRG, rgData, outputData,
smallRows, joinedRow);
}
}
else
{
outputRG.getRow(outputRG.getRowCount(), &joinedRow);
for (i = 0; i < joinerOutput[depth].size(); i++, joinedRow.nextRow(), outputRG.incRowCount())
{
smallRow.setPointer(joinerOutput[depth][i]);
if (UNLIKELY(outputRG.getRowCount() == 8192))
{
uint32_t dbRoot = outputRG.getDBRoot();
uint64_t baseRid = outputRG.getBaseRid();
outputData->push_back(rgData);
rgData = RGData(outputRG);
outputRG.setData(&rgData);
outputRG.resetRowGroup(baseRid);
outputRG.setDBRoot(dbRoot);
outputRG.getRow(0, &joinedRow);
}
applyMapping(mappings[depth], smallRow, &baseRow);
copyRow(baseRow, &joinedRow);
}
}
}
const rowgroup::RowGroup& TupleBPS::getOutputRowGroup() const const rowgroup::RowGroup& TupleBPS::getOutputRowGroup() const
{ {
return outputRowGroup; return outputRowGroup;
@ -2970,9 +3031,9 @@ void TupleBPS::setFcnExpGroup2(const boost::shared_ptr<funcexp::FuncExpWrapper>&
fe2Output = rg; fe2Output = rg;
checkDupOutputColumns(rg); checkDupOutputColumns(rg);
fe2Mapping = makeMapping(outputRowGroup, fe2Output); fe2Mapping = makeMapping(outputRowGroup, fe2Output);
runFEonPM = runFE2onPM; bRunFEonPM = runFE2onPM;
if (runFEonPM) if (bRunFEonPM)
fBPP->setFEGroup2(fe2, fe2Output); fBPP->setFEGroup2(fe2, fe2Output);
} }
@ -2985,7 +3046,7 @@ void TupleBPS::setFcnExpGroup3(const vector<execplan::SRCP>& fe)
fe2->addReturnedColumn(fe[i]); fe2->addReturnedColumn(fe[i]);
// if this is called, there's no join, so it can always run on the PM // if this is called, there's no join, so it can always run on the PM
runFEonPM = true; bRunFEonPM = true;
fBPP->setFEGroup2(fe2, fe2Output); fBPP->setFEGroup2(fe2, fe2Output);
} }
@ -2995,93 +3056,10 @@ void TupleBPS::setFE23Output(const rowgroup::RowGroup& feOutput)
checkDupOutputColumns(feOutput); checkDupOutputColumns(feOutput);
fe2Mapping = makeMapping(outputRowGroup, fe2Output); fe2Mapping = makeMapping(outputRowGroup, fe2Output);
if (fe2 && runFEonPM) if (fe2 && bRunFEonPM)
fBPP->setFEGroup2(fe2, fe2Output); fBPP->setFEGroup2(fe2, fe2Output);
} }
void TupleBPS::processFE2_oneRG(RowGroup& input, RowGroup& output, Row& inRow, Row& outRow,
funcexp::FuncExpWrapper* local_fe)
{
bool ret;
uint32_t i;
output.resetRowGroup(input.getBaseRid());
output.setDBRoot(input.getDBRoot());
output.getRow(0, &outRow);
input.getRow(0, &inRow);
for (i = 0; i < input.getRowCount(); i++, inRow.nextRow())
{
ret = local_fe->evaluate(&inRow);
if (ret)
{
applyMapping(fe2Mapping, inRow, &outRow);
outRow.setRid(inRow.getRelRid());
output.incRowCount();
outRow.nextRow();
}
}
}
void TupleBPS::processFE2(RowGroup& input, RowGroup& output, Row& inRow, Row& outRow, vector<RGData>* rgData,
funcexp::FuncExpWrapper* local_fe)
{
vector<RGData> results;
RGData result;
uint32_t i, j;
bool ret;
result = RGData(output);
output.setData(&result);
output.resetRowGroup(-1);
output.getRow(0, &outRow);
for (i = 0; i < rgData->size(); i++)
{
input.setData(&(*rgData)[i]);
if (output.getRowCount() == 0)
{
output.resetRowGroup(input.getBaseRid());
output.setDBRoot(input.getDBRoot());
}
input.getRow(0, &inRow);
for (j = 0; j < input.getRowCount(); j++, inRow.nextRow())
{
ret = local_fe->evaluate(&inRow);
if (ret)
{
applyMapping(fe2Mapping, inRow, &outRow);
outRow.setRid(inRow.getRelRid());
output.incRowCount();
outRow.nextRow();
if (output.getRowCount() == 8192 || output.getDBRoot() != input.getDBRoot() ||
output.getBaseRid() != input.getBaseRid())
{
results.push_back(result);
result = RGData(output);
output.setData(&result);
output.resetRowGroup(input.getBaseRid());
output.setDBRoot(input.getDBRoot());
output.getRow(0, &outRow);
}
}
}
}
if (output.getRowCount() > 0)
{
results.push_back(result);
}
rgData->swap(results);
}
const rowgroup::RowGroup& TupleBPS::getDeliveredRowGroup() const const rowgroup::RowGroup& TupleBPS::getDeliveredRowGroup() const
{ {
if (fe2) if (fe2)

View File

@ -5432,11 +5432,10 @@ void TupleAggregateStep::threadedAggregateRowGroups(uint32_t threadID)
if (more) if (more)
{ {
fRowGroupIns[threadID].setData(&rgData); fRowGroupIns[threadID].setData(&rgData);
fMemUsage[threadID] += fRowGroupIns[threadID].getSizeWithStrings();
bool diskAggAllowed = fRm->getAllowDiskAggregation(); bool diskAggAllowed = fRm->getAllowDiskAggregation();
if (!fRm->getMemory(fRowGroupIns[threadID].getSizeWithStrings(), fSessionMemLimit, int64_t memSize = fRowGroupIns[threadID].getSizeWithStrings();
!diskAggAllowed)) if (!fRm->getMemory(memSize, fSessionMemLimit, !diskAggAllowed))
{ {
if (!diskAggAllowed) if (!diskAggAllowed)
{ {
@ -5456,6 +5455,7 @@ void TupleAggregateStep::threadedAggregateRowGroups(uint32_t threadID)
} }
break; break;
} }
fMemUsage[threadID] += memSize;
rgDatas.push_back(rgData); rgDatas.push_back(rgData);
} }
else else

View File

@ -39,7 +39,6 @@ using namespace std;
#include "tuplehashjoin.h" #include "tuplehashjoin.h"
#include "calpontsystemcatalog.h" #include "calpontsystemcatalog.h"
#include "elementcompression.h" #include "elementcompression.h"
#include "resourcemanager.h"
#include "tupleaggregatestep.h" #include "tupleaggregatestep.h"
#include "errorids.h" #include "errorids.h"
#include "diskjoinstep.h" #include "diskjoinstep.h"
@ -74,6 +73,7 @@ TupleHashJoinStep::TupleHashJoinStep(const JobInfo& jobInfo)
, fTupleId2(-1) , fTupleId2(-1)
, fCorrelatedSide(0) , fCorrelatedSide(0)
, resourceManager(jobInfo.rm) , resourceManager(jobInfo.rm)
, fMemSizeForOutputRG(0)
, runRan(false) , runRan(false)
, joinRan(false) , joinRan(false)
, largeSideIndex(1) , largeSideIndex(1)
@ -135,9 +135,12 @@ TupleHashJoinStep::~TupleHashJoinStep()
if (memUsedByEachJoin) if (memUsedByEachJoin)
{ {
for (uint i = 0; i < smallDLs.size(); i++) for (uint i = 0; i < smallDLs.size(); i++)
{
if (memUsedByEachJoin[i])
resourceManager->returnMemory(memUsedByEachJoin[i], sessionMemLimit); resourceManager->returnMemory(memUsedByEachJoin[i], sessionMemLimit);
} }
}
returnMemory();
// cout << "deallocated THJS, UM memory available: " << resourceManager.availableMemory() << endl; // cout << "deallocated THJS, UM memory available: " << resourceManager.availableMemory() << endl;
} }
@ -221,11 +224,13 @@ void TupleHashJoinStep::trackMem(uint index)
memAfter = joiner->getMemUsage(); memAfter = joiner->getMemUsage();
if (memAfter != memBefore) if (memAfter != memBefore)
{ {
gotMem = resourceManager->getMemory(memAfter - memBefore, sessionMemLimit, false); gotMem = resourceManager->getMemory(memAfter - memBefore, sessionMemLimit, true);
if (gotMem)
atomicops::atomicAdd(&memUsedByEachJoin[index], memAfter - memBefore); atomicops::atomicAdd(&memUsedByEachJoin[index], memAfter - memBefore);
memBefore = memAfter; else
if (!gotMem)
return; return;
memBefore = memAfter;
} }
memTrackDone.timed_wait(scoped, boost::posix_time::seconds(1)); memTrackDone.timed_wait(scoped, boost::posix_time::seconds(1));
} }
@ -235,16 +240,22 @@ void TupleHashJoinStep::trackMem(uint index)
memAfter = joiner->getMemUsage(); memAfter = joiner->getMemUsage();
if (memAfter == memBefore) if (memAfter == memBefore)
return; return;
gotMem = resourceManager->getMemory(memAfter - memBefore, sessionMemLimit, false); gotMem = resourceManager->getMemory(memAfter - memBefore, sessionMemLimit, true);
if (gotMem)
{
atomicops::atomicAdd(&memUsedByEachJoin[index], memAfter - memBefore); atomicops::atomicAdd(&memUsedByEachJoin[index], memAfter - memBefore);
if (!gotMem) }
else
{ {
if (!joinIsTooBig && if (!joinIsTooBig &&
(isDML || !allowDJS || (fSessionId & 0x80000000) || (tableOid() < 3000 && tableOid() >= 1000))) (isDML || !allowDJS || (fSessionId & 0x80000000) || (tableOid() < 3000 && tableOid() >= 1000)))
{ {
joinIsTooBig = true; joinIsTooBig = true;
fLogger->logMessage(logging::LOG_TYPE_INFO, logging::ERR_JOIN_TOO_BIG); ostringstream oss;
errorMessage(logging::IDBErrorInfo::instance()->errorMsg(logging::ERR_JOIN_TOO_BIG)); oss << "(" << __LINE__ << ") "
<< logging::IDBErrorInfo::instance()->errorMsg(logging::ERR_JOIN_TOO_BIG);
fLogger->logMessage(logging::LOG_TYPE_INFO, oss.str());
errorMessage(oss.str());
status(logging::ERR_JOIN_TOO_BIG); status(logging::ERR_JOIN_TOO_BIG);
cout << "Join is too big, raise the UM join limit for now (monitor thread)" << endl; cout << "Join is too big, raise the UM join limit for now (monitor thread)" << endl;
abort(); abort();
@ -387,7 +398,6 @@ void TupleHashJoinStep::smallRunnerFcn(uint32_t index, uint threadID, uint64_t*
smallRG = smallRGs[index]; smallRG = smallRGs[index];
smallRG.initRow(&r); smallRG.initRow(&r);
try try
{ {
ssize_t rgSize; ssize_t rgSize;
@ -405,9 +415,12 @@ void TupleHashJoinStep::smallRunnerFcn(uint32_t index, uint threadID, uint64_t*
utils::releaseSpinlock(rgdLock); utils::releaseSpinlock(rgdLock);
rgSize = smallRG.getSizeWithStrings(); rgSize = smallRG.getSizeWithStrings();
gotMem = resourceManager->getMemory(rgSize, sessionMemLimit, true);
if (gotMem)
{
atomicops::atomicAdd(&memUsedByEachJoin[index], rgSize); atomicops::atomicAdd(&memUsedByEachJoin[index], rgSize);
gotMem = resourceManager->getMemory(rgSize, sessionMemLimit, false); }
if (!gotMem) else
{ {
/* Mem went over the limit. /* Mem went over the limit.
If DML or a syscat query, abort. If DML or a syscat query, abort.
@ -420,19 +433,21 @@ void TupleHashJoinStep::smallRunnerFcn(uint32_t index, uint threadID, uint64_t*
if (!allowDJS || isDML || (fSessionId & 0x80000000) || (tableOid() < 3000 && tableOid() >= 1000)) if (!allowDJS || isDML || (fSessionId & 0x80000000) || (tableOid() < 3000 && tableOid() >= 1000))
{ {
joinIsTooBig = true; joinIsTooBig = true;
fLogger->logMessage(logging::LOG_TYPE_INFO, logging::ERR_JOIN_TOO_BIG); ostringstream oss;
errorMessage(logging::IDBErrorInfo::instance()->errorMsg(logging::ERR_JOIN_TOO_BIG)); oss << "(" << __LINE__ << ") "
<< logging::IDBErrorInfo::instance()->errorMsg(logging::ERR_JOIN_TOO_BIG);
fLogger->logMessage(logging::LOG_TYPE_INFO, oss.str());
errorMessage(oss.str());
status(logging::ERR_JOIN_TOO_BIG); status(logging::ERR_JOIN_TOO_BIG);
cout << "Join is too big, raise the UM join limit for now (small runner)" << endl; cout << "Join is too big, raise the UM join limit for now (small runner)" << endl;
abort(); abort();
} }
else if (allowDJS) else if (allowDJS)
joiner->setConvertToDiskJoin(); joiner->setConvertToDiskJoin();
return; return;
} }
joiner->insertRGData(smallRG, threadID); joiner->insertRGData(smallRG, threadID);
if (!joiner->inUM() && (memUsedByEachJoin[index] > pmMemLimit)) if (!joiner->inUM() && (memUsedByEachJoin[index] > pmMemLimit))
{ {
joiner->setInUM(rgData[index]); joiner->setInUM(rgData[index]);
@ -452,7 +467,6 @@ void TupleHashJoinStep::smallRunnerFcn(uint32_t index, uint threadID, uint64_t*
"TupleHashJoinStep::smallRunnerFcn()"); "TupleHashJoinStep::smallRunnerFcn()");
status(logging::ERR_EXEMGR_MALFUNCTION); status(logging::ERR_EXEMGR_MALFUNCTION);
} }
if (!joiner->inUM()) if (!joiner->inUM())
joiner->setInPM(); joiner->setInPM();
} }
@ -644,7 +658,7 @@ void TupleHashJoinStep::hjRunner()
memUsedByEachJoin.reset(new ssize_t[smallDLs.size()]); memUsedByEachJoin.reset(new ssize_t[smallDLs.size()]);
for (i = 0; i < smallDLs.size(); i++) for (i = 0; i < smallDLs.size(); i++)
memUsedByEachJoin[i] = 0; atomicops::atomicZero(&memUsedByEachJoin[i]);
try try
{ {
@ -742,7 +756,7 @@ void TupleHashJoinStep::hjRunner()
{ {
vector<RGData> empty; vector<RGData> empty;
resourceManager->returnMemory(memUsedByEachJoin[djsJoinerMap[i]], sessionMemLimit); resourceManager->returnMemory(memUsedByEachJoin[djsJoinerMap[i]], sessionMemLimit);
memUsedByEachJoin[djsJoinerMap[i]] = 0; atomicops::atomicZero(&memUsedByEachJoin[i]);
djs[i].loadExistingData(rgData[djsJoinerMap[i]]); djs[i].loadExistingData(rgData[djsJoinerMap[i]]);
rgData[djsJoinerMap[i]].swap(empty); rgData[djsJoinerMap[i]].swap(empty);
} }
@ -828,8 +842,11 @@ void TupleHashJoinStep::hjRunner()
{ {
if (joinIsTooBig && !status()) if (joinIsTooBig && !status())
{ {
fLogger->logMessage(logging::LOG_TYPE_INFO, logging::ERR_JOIN_TOO_BIG); ostringstream oss;
errorMessage(logging::IDBErrorInfo::instance()->errorMsg(logging::ERR_JOIN_TOO_BIG)); oss << "(" << __LINE__ << ") "
<< logging::IDBErrorInfo::instance()->errorMsg(logging::ERR_JOIN_TOO_BIG);
fLogger->logMessage(logging::LOG_TYPE_INFO, oss.str());
errorMessage(oss.str());
status(logging::ERR_JOIN_TOO_BIG); status(logging::ERR_JOIN_TOO_BIG);
cout << "Join is too big, raise the UM join limit for now" << endl; cout << "Join is too big, raise the UM join limit for now" << endl;
@ -842,7 +859,7 @@ void TupleHashJoinStep::hjRunner()
for (uint i = 0; i < smallDLs.size(); i++) for (uint i = 0; i < smallDLs.size(); i++)
{ {
resourceManager->returnMemory(memUsedByEachJoin[i], sessionMemLimit); resourceManager->returnMemory(memUsedByEachJoin[i], sessionMemLimit);
memUsedByEachJoin[i] = 0; atomicops::atomicZero(&memUsedByEachJoin[i]);
} }
} }
} }
@ -1022,7 +1039,7 @@ uint32_t TupleHashJoinStep::nextBand(messageqcpp::ByteStream& bs)
for (uint i = 0; i < smallDLs.size(); i++) for (uint i = 0; i < smallDLs.size(); i++)
{ {
resourceManager->returnMemory(memUsedByEachJoin[i], sessionMemLimit); resourceManager->returnMemory(memUsedByEachJoin[i], sessionMemLimit);
memUsedByEachJoin[i] = 0; atomicops::atomicZero(&memUsedByEachJoin[i]);
} }
return 0; return 0;
} }
@ -1046,7 +1063,7 @@ uint32_t TupleHashJoinStep::nextBand(messageqcpp::ByteStream& bs)
for (uint i = 0; i < smallDLs.size(); i++) for (uint i = 0; i < smallDLs.size(); i++)
{ {
resourceManager->returnMemory(memUsedByEachJoin[i], sessionMemLimit); resourceManager->returnMemory(memUsedByEachJoin[i], sessionMemLimit);
memUsedByEachJoin[i] = 0; atomicops::atomicZero(&memUsedByEachJoin[i]);
} }
return 0; return 0;
} }
@ -1544,8 +1561,8 @@ void TupleHashJoinStep::joinRunnerFcn(uint32_t threadID)
if (local_inputRG.getRowCount() == 0) if (local_inputRG.getRowCount() == 0)
continue; continue;
joinOneRG(threadID, &joinedRowData, local_inputRG, local_outputRG, largeRow, joinFERow, joinedRow, joinOneRG(threadID, joinedRowData, local_inputRG, local_outputRG, largeRow, joinFERow, joinedRow,
baseRow, joinMatches, smallRowTemplates); baseRow, joinMatches, smallRowTemplates, outputDL);
} }
if (fe2) if (fe2)
@ -1553,6 +1570,7 @@ void TupleHashJoinStep::joinRunnerFcn(uint32_t threadID)
processDupList(threadID, (fe2 ? local_fe2RG : local_outputRG), &joinedRowData); processDupList(threadID, (fe2 ? local_fe2RG : local_outputRG), &joinedRowData);
sendResult(joinedRowData); sendResult(joinedRowData);
returnMemory();
joinedRowData.clear(); joinedRowData.clear();
grabSomeWork(&inputData); grabSomeWork(&inputData);
} }
@ -1684,9 +1702,9 @@ void TupleHashJoinStep::grabSomeWork(vector<RGData>* work)
/* This function is a port of the main join loop in TupleBPS::receiveMultiPrimitiveMessages(). Any /* This function is a port of the main join loop in TupleBPS::receiveMultiPrimitiveMessages(). Any
* changes made here should also be made there and vice versa. */ * changes made here should also be made there and vice versa. */
void TupleHashJoinStep::joinOneRG( void TupleHashJoinStep::joinOneRG(
uint32_t threadID, vector<RGData>* out, RowGroup& inputRG, RowGroup& joinOutput, Row& largeSideRow, uint32_t threadID, vector<RGData>& out, RowGroup& inputRG, RowGroup& joinOutput, Row& largeSideRow,
Row& joinFERow, Row& joinedRow, Row& baseRow, vector<vector<Row::Pointer> >& joinMatches, Row& joinFERow, Row& joinedRow, Row& baseRow, vector<vector<Row::Pointer> >& joinMatches,
shared_array<Row>& smallRowTemplates, shared_array<Row>& smallRowTemplates, RowGroupDL* outputDL,
// disk-join support vars. This param list is insane; refactor attempt would be nice at some point. // disk-join support vars. This param list is insane; refactor attempt would be nice at some point.
vector<boost::shared_ptr<joiner::TupleJoiner> >* tjoiners, vector<boost::shared_ptr<joiner::TupleJoiner> >* tjoiners,
boost::shared_array<boost::shared_array<int> >* rgMappings, boost::shared_array<boost::shared_array<int> >* rgMappings,
@ -1812,19 +1830,19 @@ void TupleHashJoinStep::joinOneRG(
applyMapping((*rgMappings)[smallSideCount], largeSideRow, &baseRow); applyMapping((*rgMappings)[smallSideCount], largeSideRow, &baseRow);
baseRow.setRid(largeSideRow.getRelRid()); baseRow.setRid(largeSideRow.getRelRid());
generateJoinResultSet(joinMatches, baseRow, *rgMappings, 0, joinOutput, joinedData, out, generateJoinResultSet(joinMatches, baseRow, *rgMappings, 0, joinOutput, joinedData, out,
smallRowTemplates, joinedRow); smallRowTemplates, joinedRow, outputDL);
} }
} }
if (joinOutput.getRowCount() > 0) if (joinOutput.getRowCount() > 0)
out->push_back(joinedData); out.push_back(joinedData);
} }
void TupleHashJoinStep::generateJoinResultSet(const vector<vector<Row::Pointer> >& joinerOutput, Row& baseRow, void TupleHashJoinStep::generateJoinResultSet(const vector<vector<Row::Pointer> >& joinerOutput, Row& baseRow,
const shared_array<shared_array<int> >& mappings, const shared_array<shared_array<int> >& mappings,
const uint32_t depth, RowGroup& l_outputRG, RGData& rgData, const uint32_t depth, RowGroup& l_outputRG, RGData& rgData,
vector<RGData>* outputData, const shared_array<Row>& smallRows, vector<RGData>& outputData, const shared_array<Row>& smallRows,
Row& joinedRow) Row& joinedRow, RowGroupDL* dlp)
{ {
uint32_t i; uint32_t i;
Row& smallRow = smallRows[depth]; Row& smallRow = smallRows[depth];
@ -1836,10 +1854,8 @@ void TupleHashJoinStep::generateJoinResultSet(const vector<vector<Row::Pointer>
{ {
smallRow.setPointer(joinerOutput[depth][i]); smallRow.setPointer(joinerOutput[depth][i]);
applyMapping(mappings[depth], smallRow, &baseRow); applyMapping(mappings[depth], smallRow, &baseRow);
// cout << "depth " << depth << ", size " << joinerOutput[depth].size() << ", row "
// << i << ": " << smallRow.toString() << endl;
generateJoinResultSet(joinerOutput, baseRow, mappings, depth + 1, l_outputRG, rgData, outputData, generateJoinResultSet(joinerOutput, baseRow, mappings, depth + 1, l_outputRG, rgData, outputData,
smallRows, joinedRow); smallRows, joinedRow, dlp);
} }
} }
else else
@ -1854,7 +1870,15 @@ void TupleHashJoinStep::generateJoinResultSet(const vector<vector<Row::Pointer>
{ {
uint32_t dbRoot = l_outputRG.getDBRoot(); uint32_t dbRoot = l_outputRG.getDBRoot();
uint64_t baseRid = l_outputRG.getBaseRid(); uint64_t baseRid = l_outputRG.getBaseRid();
outputData->push_back(rgData); outputData.push_back(rgData);
// Count the memory
if (UNLIKELY(!getMemory(l_outputRG.getMaxDataSize())))
{
// Don't let the join results buffer get out of control.
sendResult(outputData);
outputData.clear();
returnMemory();
}
rgData.reinit(l_outputRG); rgData.reinit(l_outputRG);
l_outputRG.setData(&rgData); l_outputRG.setData(&rgData);
l_outputRG.resetRowGroup(baseRid); l_outputRG.resetRowGroup(baseRid);
@ -1862,12 +1886,8 @@ void TupleHashJoinStep::generateJoinResultSet(const vector<vector<Row::Pointer>
l_outputRG.getRow(0, &joinedRow); l_outputRG.getRow(0, &joinedRow);
} }
// cout << "depth " << depth << ", size " << joinerOutput[depth].size() << ", row "
// << i << ": " << smallRow.toString() << endl;
applyMapping(mappings[depth], smallRow, &baseRow); applyMapping(mappings[depth], smallRow, &baseRow);
copyRow(baseRow, &joinedRow); copyRow(baseRow, &joinedRow);
// memcpy(joinedRow.getData(), baseRow.getData(), joinedRow.getSize());
// cout << "(step " << stepID << ") fully joined row is: " << joinedRow.toString() << endl;
} }
} }
} }

View File

@ -30,11 +30,12 @@
#include <string> #include <string>
#include <vector> #include <vector>
#include <utility> #include <utility>
#include "resourcemanager.h"
#include "exceptclasses.h"
namespace joblist namespace joblist
{ {
class BatchPrimitive; class BatchPrimitive;
class ResourceManager;
class TupleBPS; class TupleBPS;
struct FunctionJoinInfo; struct FunctionJoinInfo;
class DiskJoinStep; class DiskJoinStep;
@ -365,6 +366,21 @@ class TupleHashJoinStep : public JobStep, public TupleDeliveryStep
} }
void abort(); void abort();
void returnMemory()
{
if (fMemSizeForOutputRG > 0)
{
resourceManager->returnMemory(fMemSizeForOutputRG);
fMemSizeForOutputRG = 0;
}
}
bool getMemory(uint64_t memSize)
{
bool gotMem = resourceManager->getMemory(memSize);
if (gotMem)
fMemSizeForOutputRG += memSize;
return gotMem;
}
private: private:
TupleHashJoinStep(); TupleHashJoinStep();
@ -422,6 +438,7 @@ class TupleHashJoinStep : public JobStep, public TupleDeliveryStep
std::vector<std::vector<uint32_t> > smallSideKeys; std::vector<std::vector<uint32_t> > smallSideKeys;
ResourceManager* resourceManager; ResourceManager* resourceManager;
uint64_t fMemSizeForOutputRG;
struct JoinerSorter struct JoinerSorter
{ {
@ -521,18 +538,19 @@ class TupleHashJoinStep : public JobStep, public TupleDeliveryStep
rowgroup::Row& baseRow, rowgroup::Row& baseRow,
const boost::shared_array<boost::shared_array<int> >& mappings, const boost::shared_array<boost::shared_array<int> >& mappings,
const uint32_t depth, rowgroup::RowGroup& outputRG, rowgroup::RGData& rgData, const uint32_t depth, rowgroup::RowGroup& outputRG, rowgroup::RGData& rgData,
std::vector<rowgroup::RGData>* outputData, std::vector<rowgroup::RGData>& outputData,
const boost::shared_array<rowgroup::Row>& smallRows, rowgroup::Row& joinedRow); const boost::shared_array<rowgroup::Row>& smallRows, rowgroup::Row& joinedRow,
RowGroupDL* outputDL);
void grabSomeWork(std::vector<rowgroup::RGData>* work); void grabSomeWork(std::vector<rowgroup::RGData>* work);
void sendResult(const std::vector<rowgroup::RGData>& res); void sendResult(const std::vector<rowgroup::RGData>& res);
void processFE2(rowgroup::RowGroup& input, rowgroup::RowGroup& output, rowgroup::Row& inRow, void processFE2(rowgroup::RowGroup& input, rowgroup::RowGroup& output, rowgroup::Row& inRow,
rowgroup::Row& outRow, std::vector<rowgroup::RGData>* rgData, rowgroup::Row& outRow, std::vector<rowgroup::RGData>* rgData,
funcexp::FuncExpWrapper* local_fe); funcexp::FuncExpWrapper* local_fe);
void joinOneRG(uint32_t threadID, std::vector<rowgroup::RGData>* out, rowgroup::RowGroup& inputRG, void joinOneRG(uint32_t threadID, std::vector<rowgroup::RGData>& out, rowgroup::RowGroup& inputRG,
rowgroup::RowGroup& joinOutput, rowgroup::Row& largeSideRow, rowgroup::Row& joinFERow, rowgroup::RowGroup& joinOutput, rowgroup::Row& largeSideRow, rowgroup::Row& joinFERow,
rowgroup::Row& joinedRow, rowgroup::Row& baseRow, rowgroup::Row& joinedRow, rowgroup::Row& baseRow,
std::vector<std::vector<rowgroup::Row::Pointer> >& joinMatches, std::vector<std::vector<rowgroup::Row::Pointer> >& joinMatches,
boost::shared_array<rowgroup::Row>& smallRowTemplates, boost::shared_array<rowgroup::Row>& smallRowTemplates, RowGroupDL* outputDL,
std::vector<boost::shared_ptr<joiner::TupleJoiner> >* joiners = NULL, std::vector<boost::shared_ptr<joiner::TupleJoiner> >* joiners = NULL,
boost::shared_array<boost::shared_array<int> >* rgMappings = NULL, boost::shared_array<boost::shared_array<int> >* rgMappings = NULL,
boost::shared_array<boost::shared_array<int> >* feMappings = NULL, boost::shared_array<boost::shared_array<int> >* feMappings = NULL,
@ -577,6 +595,7 @@ class TupleHashJoinStep : public JobStep, public TupleDeliveryStep
} }
void operator()() void operator()()
{ {
utils::setThreadName("DJSReader");
HJ->djsReaderFcn(index); HJ->djsReaderFcn(index);
} }
TupleHashJoinStep* HJ; TupleHashJoinStep* HJ;

View File

@ -244,10 +244,13 @@ void TupleUnion::readInput(uint32_t which)
memUsageAfter = allocator.getMemUsage(); memUsageAfter = allocator.getMemUsage();
memDiff += (memUsageAfter - memUsageBefore); memDiff += (memUsageAfter - memUsageBefore);
memUsage += memDiff;
} }
if (!rm->getMemory(memDiff, sessionMemLimit)) if (rm->getMemory(memDiff, sessionMemLimit))
{
memUsage += memDiff;
}
else
{ {
fLogger->logMessage(logging::LOG_TYPE_INFO, logging::ERR_UNION_TOO_BIG); fLogger->logMessage(logging::LOG_TYPE_INFO, logging::ERR_UNION_TOO_BIG);

View File

@ -884,11 +884,12 @@ void WindowFunctionStep::execute()
{ {
fInRowGroupData.push_back(rgData); fInRowGroupData.push_back(rgData);
uint64_t memAdd = fRowGroupIn.getSizeWithStrings() + rowCnt * sizeof(RowPosition); uint64_t memAdd = fRowGroupIn.getSizeWithStrings() + rowCnt * sizeof(RowPosition);
fMemUsage += memAdd;
if (fRm->getMemory(memAdd, fSessionMemLimit) == false) if (fRm->getMemory(memAdd, fSessionMemLimit) == false)
throw IDBExcept(ERR_WF_DATA_SET_TOO_BIG); throw IDBExcept(ERR_WF_DATA_SET_TOO_BIG);
fMemUsage += memAdd;
for (uint64_t j = 0; j < rowCnt; ++j) for (uint64_t j = 0; j < rowCnt; ++j)
{ {
if (i > 0x0000FFFFFFFFFFFFULL || j > 0x000000000000FFFFULL) if (i > 0x0000FFFFFFFFFFFFULL || j > 0x000000000000FFFFULL)
@ -1012,11 +1013,12 @@ void WindowFunctionStep::doFunction()
while (((i = nextFunctionIndex()) < fFunctionCount) && !cancelled()) while (((i = nextFunctionIndex()) < fFunctionCount) && !cancelled())
{ {
uint64_t memAdd = fRows.size() * sizeof(RowPosition); uint64_t memAdd = fRows.size() * sizeof(RowPosition);
fMemUsage += memAdd;
if (fRm->getMemory(memAdd, fSessionMemLimit) == false) if (fRm->getMemory(memAdd, fSessionMemLimit) == false)
throw IDBExcept(ERR_WF_DATA_SET_TOO_BIG); throw IDBExcept(ERR_WF_DATA_SET_TOO_BIG);
fMemUsage += memAdd;
fFunctions[i]->setCallback(this, i); fFunctions[i]->setCallback(this, i);
(*fFunctions[i].get())(); (*fFunctions[i].get())();
} }

View File

@ -754,7 +754,7 @@ int ha_mcs_impl_write_batch_row_(const uchar* buf, TABLE* table, cal_impl_if::ca
} }
} }
rc = fprintf(ci.filePtr, "\n"); //@bug 6077 check whether thhe pipe is still open rc = fprintf(ci.filePtr, "\n"); //@bug 6077 check whether the pipe is still open
if (rc < 0) if (rc < 0)
rc = -1; rc = -1;

View File

@ -19,6 +19,7 @@
#include "iosocket.h" #include "iosocket.h"
#include "femsghandler.h" #include "femsghandler.h"
#include "threadnaming.h"
using namespace std; using namespace std;
using namespace joblist; using namespace joblist;
@ -36,6 +37,7 @@ class Runner
} }
void operator()() void operator()()
{ {
utils::setThreadName("FEMsgHandler");
target->threadFcn(); target->threadFcn();
} }
FEMsgHandler* target; FEMsgHandler* target;

View File

@ -79,6 +79,7 @@
#include "mariadb_my_sys.h" #include "mariadb_my_sys.h"
#include "statistics.h" #include "statistics.h"
#include "threadnaming.h"
class Opt class Opt
{ {
@ -617,8 +618,6 @@ class SessionThread
if (jl->status() == 0) if (jl->status() == 0)
{ {
std::string emsg;
if (jl->putEngineComm(fEc) != 0) if (jl->putEngineComm(fEc) != 0)
throw std::runtime_error(jl->errMsg()); throw std::runtime_error(jl->errMsg());
} }
@ -711,6 +710,7 @@ class SessionThread
public: public:
void operator()() void operator()()
{ {
utils::setThreadName("SessionThread");
messageqcpp::ByteStream bs, inbs; messageqcpp::ByteStream bs, inbs;
execplan::CalpontSelectExecutionPlan csep; execplan::CalpontSelectExecutionPlan csep;
csep.sessionID(0); csep.sessionID(0);

View File

@ -104,6 +104,7 @@ using namespace compress;
using namespace idbdatafile; using namespace idbdatafile;
#include "mcsconfig.h" #include "mcsconfig.h"
#include "threadnaming.h"
typedef tr1::unordered_set<BRM::OID_t> USOID; typedef tr1::unordered_set<BRM::OID_t> USOID;
@ -384,6 +385,7 @@ static int updateptrs(char* ptr, FdCacheType_t::iterator fdit)
void* thr_popper(ioManager* arg) void* thr_popper(ioManager* arg)
{ {
utils::setThreadName("thr_popper");
ioManager* iom = arg; ioManager* iom = arg;
FileBufferMgr* fbm; FileBufferMgr* fbm;
int totalRqst = 0; int totalRqst = 0;

View File

@ -1148,17 +1148,24 @@ void BatchPrimitiveProcessor::initProcessor()
} }
/* This version does a join on projected rows */ /* This version does a join on projected rows */
void BatchPrimitiveProcessor::executeTupleJoin() // In order to prevent super size result sets in the case of near cartesian joins on three or more joins,
// the startRid start at 0) is used to begin the rid loop and if we cut off processing early because of
// the size of the result set, we return the next rid to start with. If we finish ridCount rids, return 0-
uint32_t BatchPrimitiveProcessor::executeTupleJoin(uint32_t startRid)
{ {
uint32_t newRowCount = 0, i, j; uint32_t newRowCount = 0, i, j;
vector<uint32_t> matches; vector<uint32_t> matches;
uint64_t largeKey; uint64_t largeKey;
uint64_t resultCount = 0;
uint32_t newStartRid = startRid;
outputRG.getRow(0, &oldRow); outputRG.getRow(0, &oldRow);
outputRG.getRow(0, &newRow); outputRG.getRow(0, &newRow);
// cout << "before join, RG has " << outputRG.getRowCount() << " BPP ridcount= " << ridCount << endl; // cout << "before join, RG has " << outputRG.getRowCount() << " BPP ridcount= " << ridCount << endl;
for (i = 0; i < ridCount && !sendThread->aborted(); i++, oldRow.nextRow()) // ridCount gets modified based on the number of Rids actually processed during this call.
// origRidCount is the number of rids for this thread after filter, which are the total
// number of rids to be processed from all calls to this function during this thread.
for (i = startRid; i < origRidCount && !sendThread->aborted(); i++, oldRow.nextRow())
{ {
/* Decide whether this large-side row belongs in the output. The breaks /* Decide whether this large-side row belongs in the output. The breaks
* in the loop mean that it doesn't. * in the loop mean that it doesn't.
@ -1265,10 +1272,9 @@ void BatchPrimitiveProcessor::executeTupleJoin()
if (j == joinerCount) if (j == joinerCount)
{ {
uint32_t matchCount;
for (j = 0; j < joinerCount; j++) for (j = 0; j < joinerCount; j++)
{ {
uint32_t matchCount;
/* The result is already known if... /* The result is already known if...
* -- anti-join with no fcnexp * -- anti-join with no fcnexp
* -- semi-join with no fcnexp and not scalar * -- semi-join with no fcnexp and not scalar
@ -1356,6 +1362,8 @@ void BatchPrimitiveProcessor::executeTupleJoin()
tSmallSideMatches[j][newRowCount].push_back(-1); tSmallSideMatches[j][newRowCount].push_back(-1);
matchCount = 1; matchCount = 1;
} }
resultCount += matchCount;
} }
/* Finally, copy the row into the output */ /* Finally, copy the row into the output */
@ -1379,8 +1387,18 @@ void BatchPrimitiveProcessor::executeTupleJoin()
// else // else
// cout << "j != joinerCount\n"; // cout << "j != joinerCount\n";
} }
// If we've accumulated more than maxResultCount -- 1048576 (2^20)_ of resultCounts, cut off processing.
// The caller will restart to continue where we left off.
if (resultCount >= maxResultCount)
{
newStartRid += newRowCount;
break;
}
} }
if (resultCount < maxResultCount)
newStartRid = 0;
ridCount = newRowCount; ridCount = newRowCount;
outputRG.setRowCount(ridCount); outputRG.setRowCount(ridCount);
@ -1397,6 +1415,7 @@ void BatchPrimitiveProcessor::executeTupleJoin()
} }
} }
*/ */
return newStartRid;
} }
#ifdef PRIMPROC_STOPWATCH #ifdef PRIMPROC_STOPWATCH
@ -1405,6 +1424,9 @@ void BatchPrimitiveProcessor::execute(StopWatch* stopwatch)
void BatchPrimitiveProcessor::execute() void BatchPrimitiveProcessor::execute()
#endif #endif
{ {
uint8_t sendCount = 0;
// bool smoreRGs = false;
// uint32_t sStartRid = 0;
uint32_t i, j; uint32_t i, j;
try try
@ -1443,6 +1465,7 @@ void BatchPrimitiveProcessor::execute()
// filters use relrids and values for intermediate results. // filters use relrids and values for intermediate results.
if (bop == BOP_AND) if (bop == BOP_AND)
{
for (j = 0; j < filterCount; ++j) for (j = 0; j < filterCount; ++j)
{ {
#ifdef PRIMPROC_STOPWATCH #ifdef PRIMPROC_STOPWATCH
@ -1453,6 +1476,7 @@ void BatchPrimitiveProcessor::execute()
filterSteps[j]->execute(); filterSteps[j]->execute();
#endif #endif
} }
}
else // BOP_OR else // BOP_OR
{ {
/* XXXPAT: This is a hacky impl of OR logic. Each filter is configured to /* XXXPAT: This is a hacky impl of OR logic. Each filter is configured to
@ -1542,10 +1566,12 @@ void BatchPrimitiveProcessor::execute()
// projection commands read relrids and write output directly to a rowgroup // projection commands read relrids and write output directly to a rowgroup
// or the serialized bytestream // or the serialized bytestream
if (ot != ROW_GROUP) if (ot != ROW_GROUP)
{
for (j = 0; j < projectCount; ++j) for (j = 0; j < projectCount; ++j)
{ {
projectSteps[j]->project(); projectSteps[j]->project();
} }
}
else else
{ {
/* Function & Expression group 1 processing /* Function & Expression group 1 processing
@ -1621,15 +1647,93 @@ void BatchPrimitiveProcessor::execute()
// cout << " no target found for OID " << projectSteps[j]->getOID() << // cout << " no target found for OID " << projectSteps[j]->getOID() <<
//endl; //endl;
} }
if (fe2)
{
/* functionize this -> processFE2() */
fe2Output.resetRowGroup(baseRid);
fe2Output.getRow(0, &fe2Out);
fe2Input->getRow(0, &fe2In);
// cerr << "input row: " << fe2In.toString() << endl;
for (j = 0; j < outputRG.getRowCount(); j++, fe2In.nextRow())
{
if (fe2->evaluate(&fe2In))
{
applyMapping(fe2Mapping, fe2In, &fe2Out);
// cerr << " passed. output row: " << fe2Out.toString() << endl;
fe2Out.setRid(fe2In.getRelRid());
fe2Output.incRowCount();
fe2Out.nextRow();
}
}
if (!fAggregator)
{
*serialized << (uint8_t)1; // the "count this msg" var
fe2Output.setDBRoot(dbRoot);
fe2Output.serializeRGData(*serialized);
//*serialized << fe2Output.getDataSize();
// serialized->append(fe2Output.getData(), fe2Output.getDataSize());
}
}
if (fAggregator)
{
*serialized << (uint8_t)1; // the "count this msg" var
RowGroup& toAggregate = (fe2 ? fe2Output : outputRG);
// toAggregate.convertToInlineDataInPlace();
if (fe2)
fe2Output.setDBRoot(dbRoot);
else
outputRG.setDBRoot(dbRoot);
fAggregator->addRowGroup(&toAggregate);
if ((currentBlockOffset + 1) == count) // @bug4507, 8k
{
fAggregator->loadResult(*serialized); // @bug4507, 8k
} // @bug4507, 8k
else if (utils::MonitorProcMem::isMemAvailable()) // @bug4507, 8k
{
fAggregator->loadEmptySet(*serialized); // @bug4507, 8k
} // @bug4507, 8k
else // @bug4507, 8k
{
fAggregator->loadResult(*serialized); // @bug4507, 8k
fAggregator->aggReset(); // @bug4507, 8k
} // @bug4507, 8k
}
if (!fAggregator && !fe2)
{
*serialized << (uint8_t)1; // the "count this msg" var
outputRG.setDBRoot(dbRoot);
// cerr << "serializing " << outputRG.toString() << endl;
outputRG.serializeRGData(*serialized);
//*serialized << outputRG.getDataSize();
// serialized->append(outputRG.getData(), outputRG.getDataSize());
}
#ifdef PRIMPROC_STOPWATCH
stopwatch->stop("- if(ot != ROW_GROUP) else");
#endif
} }
else else // Is doJoin
{ {
uint32_t startRid = 0;
ByteStream preamble = *serialized;
origRidCount = ridCount; // ridCount can get modified by executeTupleJoin(). We need to keep track of
// the original val.
/* project the key columns. If there's the filter IN the join, project everything. /* project the key columns. If there's the filter IN the join, project everything.
Also need to project 'long' strings b/c executeTupleJoin may copy entire rows Also need to project 'long' strings b/c executeTupleJoin may copy entire rows
using copyRow(), which will try to interpret the uninit'd string ptr. using copyRow(), which will try to interpret the uninit'd string ptr.
Valgrind will legitimately complain about copying uninit'd values for the Valgrind will legitimately complain about copying uninit'd values for the
other types but that is technically safe. */ other types but that is technically safe. */
for (j = 0; j < projectCount; j++) for (j = 0; j < projectCount; j++)
{
if (keyColumnProj[j] || if (keyColumnProj[j] ||
(projectionMap[j] != -1 && (hasJoinFEFilters || oldRow.isLongString(projectionMap[j])))) (projectionMap[j] != -1 && (hasJoinFEFilters || oldRow.isLongString(projectionMap[j]))))
{ {
@ -1639,215 +1743,176 @@ void BatchPrimitiveProcessor::execute()
stopwatch->stop("-- projectIntoRowGroup"); stopwatch->stop("-- projectIntoRowGroup");
#else #else
projectSteps[j]->projectIntoRowGroup(outputRG, projectionMap[j]); projectSteps[j]->projectIntoRowGroup(outputRG, projectionMap[j]);
#endif
}
#ifdef PRIMPROC_STOPWATCH
stopwatch->start("-- executeTupleJoin()");
executeTupleJoin();
stopwatch->stop("-- executeTupleJoin()");
#else
executeTupleJoin();
#endif
/* project the non-key columns */
for (j = 0; j < projectCount; ++j)
{
if (projectionMap[j] != -1 && !keyColumnProj[j] && !hasJoinFEFilters &&
!oldRow.isLongString(projectionMap[j]))
{
#ifdef PRIMPROC_STOPWATCH
stopwatch->start("-- projectIntoRowGroup");
projectSteps[j]->projectIntoRowGroup(outputRG, projectionMap[j]);
stopwatch->stop("-- projectIntoRowGroup");
#else
projectSteps[j]->projectIntoRowGroup(outputRG, projectionMap[j]);
#endif #endif
} }
} }
}
/* The RowGroup is fully joined at this point. do // while (startRid > 0)
Add additional RowGroup processing here.
TODO: Try to clean up all of the switching */
if (doJoin && (fe2 || fAggregator))
{
bool moreRGs = true;
ByteStream preamble = *serialized;
initGJRG();
while (moreRGs && !sendThread->aborted())
{ {
/* #ifdef PRIMPROC_STOPWATCH
generate 1 rowgroup (8192 rows max) of joined rows stopwatch->start("-- executeTupleJoin()");
if there's an FE2, run it startRid = executeTupleJoin(startRid);
-pack results into a new rowgroup stopwatch->stop("-- executeTupleJoin()");
-if there are < 8192 rows in the new RG, continue #else
if there's an agg, run it startRid = executeTupleJoin(startRid);
send the result // sStartRid = startRid;
*/ #endif
resetGJRG(); /* project the non-key columns */
moreRGs = generateJoinedRowGroup(baseJRow); for (j = 0; j < projectCount; ++j)
*serialized << (uint8_t)!moreRGs;
if (fe2)
{ {
/* functionize this -> processFE2()*/ if (projectionMap[j] != -1 && !keyColumnProj[j] && !hasJoinFEFilters &&
fe2Output.resetRowGroup(baseRid); !oldRow.isLongString(projectionMap[j]))
fe2Output.setDBRoot(dbRoot); {
fe2Output.getRow(0, &fe2Out); #ifdef PRIMPROC_STOPWATCH
fe2Input->getRow(0, &fe2In); stopwatch->start("-- projectIntoRowGroup");
projectSteps[j]->projectIntoRowGroup(outputRG, projectionMap[j]);
for (j = 0; j < joinedRG.getRowCount(); j++, fe2In.nextRow()) stopwatch->stop("-- projectIntoRowGroup");
if (fe2->evaluate(&fe2In)) #else
{ projectSteps[j]->projectIntoRowGroup(outputRG, projectionMap[j]);
applyMapping(fe2Mapping, fe2In, &fe2Out); #endif
fe2Out.setRid(fe2In.getRelRid()); }
fe2Output.incRowCount();
fe2Out.nextRow();
}
} }
/* The RowGroup is fully joined at this point.
* Add additional RowGroup processing here.
* TODO: Try to clean up all of the switching */
RowGroup& nextRG = (fe2 ? fe2Output : joinedRG); if (fe2 || fAggregator)
nextRG.setDBRoot(dbRoot);
if (fAggregator)
{ {
fAggregator->addRowGroup(&nextRG); bool moreRGs = true;
initGJRG();
if ((currentBlockOffset + 1) == count && moreRGs == false) // @bug4507, 8k while (moreRGs && !sendThread->aborted())
{ {
fAggregator->loadResult(*serialized); // @bug4507, 8k /*
} // @bug4507, 8k * generate 1 rowgroup (8192 rows max) of joined rows
else if (utils::MonitorProcMem::isMemAvailable()) // @bug4507, 8k * if there's an FE2, run it
* -pack results into a new rowgroup
* -if there are < 8192 rows in the new RG, continue
* if there's an agg, run it
* send the result
*/
resetGJRG();
moreRGs = generateJoinedRowGroup(baseJRow);
// smoreRGs = moreRGs;
sendCount = (uint8_t)(!moreRGs && !startRid);
// *serialized << (uint8_t)(!moreRGs && !startRid); // the "count
// this msg" var
*serialized << sendCount;
if (fe2)
{
/* functionize this -> processFE2()*/
fe2Output.resetRowGroup(baseRid);
fe2Output.setDBRoot(dbRoot);
fe2Output.getRow(0, &fe2Out);
fe2Input->getRow(0, &fe2In);
for (j = 0; j < joinedRG.getRowCount(); j++, fe2In.nextRow())
{
if (fe2->evaluate(&fe2In))
{
applyMapping(fe2Mapping, fe2In, &fe2Out);
fe2Out.setRid(fe2In.getRelRid());
fe2Output.incRowCount();
fe2Out.nextRow();
}
}
}
RowGroup& nextRG = (fe2 ? fe2Output : joinedRG);
nextRG.setDBRoot(dbRoot);
if (fAggregator)
{
fAggregator->addRowGroup(&nextRG);
if ((currentBlockOffset + 1) == count && moreRGs == false && startRid == 0) // @bug4507, 8k
{
fAggregator->loadResult(*serialized); // @bug4507, 8k
} // @bug4507, 8k
else if (utils::MonitorProcMem::isMemAvailable()) // @bug4507, 8k
{
fAggregator->loadEmptySet(*serialized); // @bug4507, 8k
} // @bug4507, 8k
else // @bug4507, 8k
{
fAggregator->loadResult(*serialized); // @bug4507, 8k
fAggregator->aggReset(); // @bug4507, 8k
} // @bug4507, 8k
}
else
{
// cerr <<" * serialzing " << nextRG.toString() << endl;
nextRG.serializeRGData(*serialized);
}
/* send the msg & reinit the BS */
if (moreRGs)
{
sendResponse();
serialized.reset(new ByteStream());
*serialized = preamble;
}
}
if (hasSmallOuterJoin)
{ {
fAggregator->loadEmptySet(*serialized); // @bug4507, 8k // Should we happen to finish sending data rows right on the boundary of when moreRGs flips off,
} // @bug4507, 8k // then we need to start a new buffer. I.e., it needs the count this message byte pushed.
else // @bug4507, 8k if (serialized->length() == preamble.length())
*serialized << (uint8_t)(startRid > 0 ? 0 : 1); // the "count this msg" var
*serialized << ridCount;
for (i = 0; i < joinerCount; i++)
{
for (j = 0; j < ridCount; ++j)
{
serializeInlineVector<uint32_t>(*serialized, tSmallSideMatches[i][j]);
tSmallSideMatches[i][j].clear();
}
}
}
else
{ {
fAggregator->loadResult(*serialized); // @bug4507, 8k // We hae no more use for this allocation
fAggregator->aggReset(); // @bug4507, 8k for (i = 0; i < joinerCount; i++)
} // @bug4507, 8k for (j = 0; j < ridCount; ++j)
tSmallSideMatches[i][j].clear();
}
} }
else else
{ {
// cerr <<" * serialzing " << nextRG.toString() << endl; *serialized << (uint8_t)(startRid > 0 ? 0 : 1); // the "count this msg" var
nextRG.serializeRGData(*serialized); outputRG.setDBRoot(dbRoot);
} // cerr << "serializing " << outputRG.toString() << endl;
outputRG.serializeRGData(*serialized);
/* send the msg & reinit the BS */ //*serialized << outputRG.getDataSize();
if (moreRGs) // serialized->append(outputRG.getData(), outputRG.getDataSize());
for (i = 0; i < joinerCount; i++)
{
for (j = 0; j < ridCount; ++j)
{
serializeInlineVector<uint32_t>(*serialized, tSmallSideMatches[i][j]);
tSmallSideMatches[i][j].clear();
}
}
}
if (startRid > 0)
{ {
sendResponse(); sendResponse();
serialized.reset(new ByteStream()); serialized.reset(new ByteStream());
*serialized = preamble; *serialized = preamble;
} }
} } while (startRid > 0);
if (hasSmallOuterJoin)
{
*serialized << ridCount;
for (i = 0; i < joinerCount; i++)
for (j = 0; j < ridCount; ++j)
serializeInlineVector<uint32_t>(*serialized, tSmallSideMatches[i][j]);
}
} }
if (!doJoin && fe2)
{
/* functionize this -> processFE2() */
fe2Output.resetRowGroup(baseRid);
fe2Output.getRow(0, &fe2Out);
fe2Input->getRow(0, &fe2In);
// cerr << "input row: " << fe2In.toString() << endl;
for (j = 0; j < outputRG.getRowCount(); j++, fe2In.nextRow())
{
if (fe2->evaluate(&fe2In))
{
applyMapping(fe2Mapping, fe2In, &fe2Out);
// cerr << " passed. output row: " << fe2Out.toString() << endl;
fe2Out.setRid(fe2In.getRelRid());
fe2Output.incRowCount();
fe2Out.nextRow();
}
}
if (!fAggregator)
{
*serialized << (uint8_t)1; // the "count this msg" var
fe2Output.setDBRoot(dbRoot);
fe2Output.serializeRGData(*serialized);
//*serialized << fe2Output.getDataSize();
// serialized->append(fe2Output.getData(), fe2Output.getDataSize());
}
}
if (!doJoin && fAggregator)
{
*serialized << (uint8_t)1; // the "count this msg" var
RowGroup& toAggregate = (fe2 ? fe2Output : outputRG);
// toAggregate.convertToInlineDataInPlace();
if (fe2)
fe2Output.setDBRoot(dbRoot);
else
outputRG.setDBRoot(dbRoot);
fAggregator->addRowGroup(&toAggregate);
if ((currentBlockOffset + 1) == count) // @bug4507, 8k
{
fAggregator->loadResult(*serialized); // @bug4507, 8k
} // @bug4507, 8k
else if (utils::MonitorProcMem::isMemAvailable()) // @bug4507, 8k
{
fAggregator->loadEmptySet(*serialized); // @bug4507, 8k
} // @bug4507, 8k
else // @bug4507, 8k
{
fAggregator->loadResult(*serialized); // @bug4507, 8k
fAggregator->aggReset(); // @bug4507, 8k
} // @bug4507, 8k
}
if (!fAggregator && !fe2)
{
*serialized << (uint8_t)1; // the "count this msg" var
outputRG.setDBRoot(dbRoot);
// cerr << "serializing " << outputRG.toString() << endl;
outputRG.serializeRGData(*serialized);
//*serialized << outputRG.getDataSize();
// serialized->append(outputRG.getData(), outputRG.getDataSize());
if (doJoin)
{
for (i = 0; i < joinerCount; i++)
{
for (j = 0; j < ridCount; ++j)
{
serializeInlineVector<uint32_t>(*serialized, tSmallSideMatches[i][j]);
}
}
}
}
// clear small side match vector
if (doJoin)
{
for (i = 0; i < joinerCount; i++)
for (j = 0; j < ridCount; ++j)
tSmallSideMatches[i][j].clear();
}
#ifdef PRIMPROC_STOPWATCH #ifdef PRIMPROC_STOPWATCH
stopwatch->stop("- if(ot != ROW_GROUP) else"); stopwatch->stop("- if(ot != ROW_GROUP) else");
#endif #endif
} }
ridCount = origRidCount; // May not be needed, but just to be safe.
// std::cout << "end of send. startRid=" << sStartRid << " moreRG=" << smoreRGs << " sendCount=" <<
// sendCount << std::endl;
if (projectCount > 0 || ot == ROW_GROUP) if (projectCount > 0 || ot == ROW_GROUP)
{ {
*serialized << cachedIO; *serialized << cachedIO;
@ -2187,8 +2252,9 @@ int BatchPrimitiveProcessor::operator()()
if (sendThread->aborted()) if (sendThread->aborted())
break; break;
if (!sendThread->okToProceed()) if (sendThread->sizeTooBig())
{ {
// The send buffer is full of messages yet to be sent, so this thread would block anyway.
freeLargeBuffers(); freeLargeBuffers();
return -1; // the reschedule error code return -1; // the reschedule error code
} }

View File

@ -224,6 +224,7 @@ class BatchPrimitiveProcessor
int128_t wide128Values[LOGICAL_BLOCK_RIDS]; int128_t wide128Values[LOGICAL_BLOCK_RIDS];
boost::scoped_array<uint64_t> absRids; boost::scoped_array<uint64_t> absRids;
boost::scoped_array<std::string> strValues; boost::scoped_array<std::string> strValues;
uint16_t origRidCount;
uint16_t ridCount; uint16_t ridCount;
bool needStrValues; bool needStrValues;
uint16_t wideColumnsWidths; uint16_t wideColumnsWidths;
@ -333,7 +334,7 @@ class BatchPrimitiveProcessor
boost::shared_array<boost::shared_array<boost::shared_ptr<TJoiner>>> tJoiners; boost::shared_array<boost::shared_array<boost::shared_ptr<TJoiner>>> tJoiners;
typedef std::vector<uint32_t> MatchedData[LOGICAL_BLOCK_RIDS]; typedef std::vector<uint32_t> MatchedData[LOGICAL_BLOCK_RIDS];
boost::shared_array<MatchedData> tSmallSideMatches; boost::shared_array<MatchedData> tSmallSideMatches;
void executeTupleJoin(); uint32_t executeTupleJoin(uint32_t startRid);
bool getTupleJoinRowGroupData; bool getTupleJoinRowGroupData;
std::vector<rowgroup::RowGroup> smallSideRGs; std::vector<rowgroup::RowGroup> smallSideRGs;
rowgroup::RowGroup largeSideRG; rowgroup::RowGroup largeSideRG;
@ -432,6 +433,8 @@ class BatchPrimitiveProcessor
uint ptMask; uint ptMask;
bool firstInstance; bool firstInstance;
static const uint64_t maxResultCount = 1048576; // 2^20
friend class Command; friend class Command;
friend class ColumnCommand; friend class ColumnCommand;
friend class DictStep; friend class DictStep;

View File

@ -23,16 +23,14 @@
#include <unistd.h> #include <unistd.h>
#include <stdexcept> #include <stdexcept>
#include <mutex>
#include "bppsendthread.h" #include "bppsendthread.h"
#include "resourcemanager.h"
using namespace std;
using namespace boost;
#include "atomicops.h"
namespace primitiveprocessor namespace primitiveprocessor
{ {
extern uint32_t connectionsPerUM; extern uint32_t connectionsPerUM;
extern uint32_t BPPCount;
BPPSendThread::BPPSendThread() BPPSendThread::BPPSendThread()
: die(false) : die(false)
@ -44,8 +42,8 @@ BPPSendThread::BPPSendThread()
, sawAllConnections(false) , sawAllConnections(false)
, fcEnabled(false) , fcEnabled(false)
, currentByteSize(0) , currentByteSize(0)
, maxByteSize(25000000)
{ {
maxByteSize = joblist::ResourceManager::instance()->getMaxBPPSendQueue();
runner = boost::thread(Runner_t(this)); runner = boost::thread(Runner_t(this));
} }
@ -59,36 +57,36 @@ BPPSendThread::BPPSendThread(uint32_t initMsgsLeft)
, sawAllConnections(false) , sawAllConnections(false)
, fcEnabled(false) , fcEnabled(false)
, currentByteSize(0) , currentByteSize(0)
, maxByteSize(25000000)
{ {
maxByteSize = joblist::ResourceManager::instance()->getMaxBPPSendQueue();
runner = boost::thread(Runner_t(this)); runner = boost::thread(Runner_t(this));
} }
BPPSendThread::~BPPSendThread() BPPSendThread::~BPPSendThread()
{ {
boost::mutex::scoped_lock sl(msgQueueLock); abort();
boost::mutex::scoped_lock sl2(ackLock);
die = true;
queueNotEmpty.notify_one();
okToSend.notify_one();
sl.unlock();
sl2.unlock();
runner.join(); 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) 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) if (die)
return; return;
boost::mutex::scoped_lock sl(msgQueueLock); std::unique_lock<std::mutex> sl(msgQueueLock);
if (gotException) if (gotException)
throw runtime_error(exceptionString); 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) 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) if (die)
return; return;
boost::mutex::scoped_lock sl(msgQueueLock); std::unique_lock<std::mutex> sl(msgQueueLock);
if (gotException) if (gotException)
throw runtime_error(exceptionString); throw runtime_error(exceptionString);
@ -157,7 +168,7 @@ void BPPSendThread::sendResults(const vector<Msg_t>& msgs, bool newConnection)
void BPPSendThread::sendMore(int num) 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; // cout << "got an ACK for " << num << " msgsLeft=" << msgsLeft << endl;
if (num == -1) if (num == -1)
@ -170,6 +181,7 @@ void BPPSendThread::sendMore(int num)
else else
(void)atomicops::atomicAdd(&msgsLeft, num); (void)atomicops::atomicAdd(&msgsLeft, num);
sl.unlock();
if (waiting) if (waiting)
okToSend.notify_one(); okToSend.notify_one();
} }
@ -192,7 +204,7 @@ void BPPSendThread::mainLoop()
while (!die) while (!die)
{ {
boost::mutex::scoped_lock sl(msgQueueLock); std::unique_lock<std::mutex> sl(msgQueueLock);
if (msgQueue.empty() && !die) if (msgQueue.empty() && !die)
{ {
@ -223,8 +235,7 @@ void BPPSendThread::mainLoop()
if (msgsLeft <= 0 && fcEnabled && !die) if (msgsLeft <= 0 && fcEnabled && !die)
{ {
boost::mutex::scoped_lock sl2(ackLock); std::unique_lock<std::mutex> sl2(ackLock);
while (msgsLeft <= 0 && fcEnabled && !die) while (msgsLeft <= 0 && fcEnabled && !die)
{ {
waiting = true; waiting = true;
@ -267,19 +278,26 @@ void BPPSendThread::mainLoop()
(void)atomicops::atomicSub(&currentByteSize, bsSize); (void)atomicops::atomicSub(&currentByteSize, bsSize);
msg[msgsSent].msg.reset(); msg[msgsSent].msg.reset();
} }
if (respondWait && currentByteSize < maxByteSize)
{
okToRespond.notify_one();
}
} }
} }
} }
void BPPSendThread::abort() void BPPSendThread::abort()
{ {
boost::mutex::scoped_lock sl(msgQueueLock); std::lock_guard<std::mutex> sl(msgQueueLock);
boost::mutex::scoped_lock sl2(ackLock); std::lock_guard<std::mutex> sl2(ackLock);
std::lock_guard<std::mutex> sl3(respondLock);
die = true; die = true;
queueNotEmpty.notify_one();
okToSend.notify_one(); queueNotEmpty.notify_all();
sl.unlock(); okToSend.notify_all();
sl2.unlock(); okToRespond.notify_all();
} }
} // namespace primitiveprocessor } // namespace primitiveprocessor

View File

@ -27,8 +27,9 @@
#include "umsocketselector.h" #include "umsocketselector.h"
#include <queue> #include <queue>
#include <set> #include <set>
#include <boost/thread/thread.hpp> #include <condition_variable>
#include <boost/thread/condition.hpp> #include "threadnaming.h"
#include "prioritythreadpool.h"
namespace primitiveprocessor namespace primitiveprocessor
{ {
@ -65,7 +66,14 @@ class BPPSendThread
} }
}; };
bool okToProceed(); bool sizeTooBig()
{
// 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)) &&
!die;
}
void sendMore(int num); void sendMore(int num);
void sendResults(const std::vector<Msg_t>& msgs, bool newConnection); void sendResults(const std::vector<Msg_t>& msgs, bool newConnection);
void sendResult(const Msg_t& msg, bool newConnection); void sendResult(const Msg_t& msg, bool newConnection);
@ -76,6 +84,10 @@ class BPPSendThread
{ {
return die; return die;
} }
void setProcessorPool(threadpool::PriorityThreadPool* processorPool)
{
fProcessorPool = processorPool;
}
private: private:
BPPSendThread(const BPPSendThread&); BPPSendThread(const BPPSendThread&);
@ -89,21 +101,26 @@ class BPPSendThread
} }
void operator()() void operator()()
{ {
utils::setThreadName("BPPSendThread");
bppst->mainLoop(); bppst->mainLoop();
} }
}; };
boost::thread runner; boost::thread runner;
std::queue<Msg_t> msgQueue; std::queue<Msg_t> msgQueue;
boost::mutex msgQueueLock; std::mutex msgQueueLock;
boost::condition queueNotEmpty; std::condition_variable queueNotEmpty;
volatile bool die, gotException, mainThreadWaiting; volatile bool die, gotException, mainThreadWaiting;
std::string exceptionString; std::string exceptionString;
uint32_t sizeThreshold; uint32_t sizeThreshold;
volatile int32_t msgsLeft; volatile int32_t msgsLeft;
bool waiting; bool waiting;
boost::mutex ackLock; std::mutex ackLock;
boost::condition okToSend; std::condition_variable okToSend;
// Condition to prevent run away queue
bool respondWait;
std::mutex respondLock;
std::condition_variable okToRespond;
/* Load balancing structures */ /* Load balancing structures */
struct Connection_t struct Connection_t
@ -130,6 +147,9 @@ class BPPSendThread
/* secondary queue size restriction based on byte size */ /* secondary queue size restriction based on byte size */
volatile uint64_t currentByteSize; volatile uint64_t currentByteSize;
uint64_t maxByteSize; uint64_t maxByteSize;
// Used to tell the PriorityThreadPool It should consider additional threads because a
// queue full event has happened and a thread has been blocked.
threadpool::PriorityThreadPool* fProcessorPool;
}; };
} // namespace primitiveprocessor } // namespace primitiveprocessor

View File

@ -1395,7 +1395,7 @@ struct BPPHandler
SBPPV bppv; SBPPV bppv;
// make the new BPP object // make the new BPP object
bppv.reset(new BPPV()); bppv.reset(new BPPV(fPrimitiveServerPtr));
bpp.reset(new BatchPrimitiveProcessor(bs, fPrimitiveServerPtr->prefetchThreshold(), bppv->getSendThread(), bpp.reset(new BatchPrimitiveProcessor(bs, fPrimitiveServerPtr->prefetchThreshold(), bppv->getSendThread(),
fPrimitiveServerPtr->ProcessorThreads())); fPrimitiveServerPtr->ProcessorThreads()));
@ -1857,7 +1857,7 @@ struct ReadThread
/* Message format: /* Message format:
* ISMPacketHeader * ISMPacketHeader
* Partition count - 32 bits * Partition count - 32 bits
* Partition set - sizeof(LogicalPartition) * count * Partition set - sizeof(LogicalPartition) boost::shared_ptr* count
* OID count - 32 bits * OID count - 32 bits
* OID array - 32 bits * count * OID array - 32 bits * count
*/ */
@ -1948,8 +1948,7 @@ struct ReadThread
void operator()() void operator()()
{ {
utils::setThreadName("PPReadThread"); utils::setThreadName("PPReadThread");
boost::shared_ptr<threadpool::PriorityThreadPool> procPoolPtr = threadpool::PriorityThreadPool* procPoolPtr = fPrimitiveServerPtr->getProcessorThreadPool();
fPrimitiveServerPtr->getProcessorThreadPool();
SBS bs; SBS bs;
UmSocketSelector* pUmSocketSelector = UmSocketSelector::instance(); UmSocketSelector* pUmSocketSelector = UmSocketSelector::instance();
@ -2407,8 +2406,8 @@ PrimitiveServer::PrimitiveServer(int serverThreads, int serverQueueSize, int pro
fServerpool.setQueueSize(fServerQueueSize); fServerpool.setQueueSize(fServerQueueSize);
fServerpool.setName("PrimitiveServer"); fServerpool.setName("PrimitiveServer");
fProcessorPool.reset(new threadpool::PriorityThreadPool(fProcessorWeight, highPriorityThreads, fProcessorPool = new threadpool::PriorityThreadPool(fProcessorWeight, highPriorityThreads,
medPriorityThreads, lowPriorityThreads, 0)); medPriorityThreads, lowPriorityThreads, 0);
// We're not using either the priority or the job-clustering features, just need a threadpool // We're not using either the priority or the job-clustering features, just need a threadpool
// that can reschedule jobs, and an unlimited non-blocking queue // that can reschedule jobs, and an unlimited non-blocking queue
@ -2460,9 +2459,10 @@ void PrimitiveServer::start(Service* service)
cerr << "PrimitiveServer::start() exiting!" << endl; cerr << "PrimitiveServer::start() exiting!" << endl;
} }
BPPV::BPPV() BPPV::BPPV(PrimitiveServer* ps)
{ {
sendThread.reset(new BPPSendThread()); sendThread.reset(new BPPSendThread());
sendThread->setProcessorPool(ps->getProcessorThreadPool());
v.reserve(BPPCount); v.reserve(BPPCount);
pos = 0; pos = 0;
joinDataReceived = false; joinDataReceived = false;
@ -2503,7 +2503,7 @@ const vector<boost::shared_ptr<BatchPrimitiveProcessor> >& BPPV::get()
boost::shared_ptr<BatchPrimitiveProcessor> BPPV::next() boost::shared_ptr<BatchPrimitiveProcessor> BPPV::next()
{ {
uint32_t size = v.size(); uint32_t size = v.size();
uint32_t i; uint32_t i = 0;
#if 0 #if 0

View File

@ -56,10 +56,12 @@ extern uint32_t highPriorityThreads, medPriorityThreads, lowPriorityThreads;
class BPPSendThread; class BPPSendThread;
class PrimitiveServer;
class BPPV class BPPV
{ {
public: public:
BPPV(); BPPV(PrimitiveServer* ps);
~BPPV(); ~BPPV();
boost::shared_ptr<BatchPrimitiveProcessor> next(); boost::shared_ptr<BatchPrimitiveProcessor> next();
void add(boost::shared_ptr<BatchPrimitiveProcessor> a); void add(boost::shared_ptr<BatchPrimitiveProcessor> a);
@ -128,7 +130,7 @@ class PrimitiveServer
/** @brief get a pointer the shared processor thread pool /** @brief get a pointer the shared processor thread pool
*/ */
inline boost::shared_ptr<threadpool::PriorityThreadPool> getProcessorThreadPool() const inline threadpool::PriorityThreadPool* getProcessorThreadPool() const
{ {
return fProcessorPool; return fProcessorPool;
} }
@ -165,7 +167,7 @@ class PrimitiveServer
/** @brief the thread pool used to process /** @brief the thread pool used to process
* primitive commands * primitive commands
*/ */
boost::shared_ptr<threadpool::PriorityThreadPool> fProcessorPool; threadpool::PriorityThreadPool* fProcessorPool;
int fServerThreads; int fServerThreads;
int fServerQueueSize; int fServerQueueSize;

View File

@ -75,6 +75,7 @@ using namespace idbdatafile;
#include "mariadb_my_sys.h" #include "mariadb_my_sys.h"
#include "service.h" #include "service.h"
#include "threadnaming.h"
class Opt class Opt
{ {
@ -246,6 +247,7 @@ class QszMonThd
void operator()() void operator()()
{ {
utils::setThreadName("QszMonThd");
for (;;) for (;;)
{ {
uint32_t qd = fPsp->getProcessorThreadPool()->getWaiting(); uint32_t qd = fPsp->getProcessorThreadPool()->getWaiting();
@ -287,6 +289,7 @@ class QszMonThd
#ifdef DUMP_CACHE_CONTENTS #ifdef DUMP_CACHE_CONTENTS
void* waitForSIGUSR1(void* p) void* waitForSIGUSR1(void* p)
{ {
utils::setThreadName("waitForSIGUSR1");
#if defined(__LP64__) || defined(_MSC_VER) #if defined(__LP64__) || defined(_MSC_VER)
ptrdiff_t tmp = reinterpret_cast<ptrdiff_t>(p); ptrdiff_t tmp = reinterpret_cast<ptrdiff_t>(p);
int cacheCount = static_cast<int>(tmp); int cacheCount = static_cast<int>(tmp);

View File

@ -42,6 +42,7 @@ using namespace std;
using namespace logging; using namespace logging;
#include "MonitorProcMem.h" #include "MonitorProcMem.h"
#include "threadnaming.h"
namespace utils namespace utils
{ {
@ -56,6 +57,7 @@ int MonitorProcMem::fMemPctCheck = 0;
//------------------------------------------------------------------------------ //------------------------------------------------------------------------------
void MonitorProcMem::operator()() const void MonitorProcMem::operator()() const
{ {
utils::setThreadName("MonitorProcMem");
while (1) while (1)
{ {
if (fMaxPct > 0) if (fMaxPct > 0)

View File

@ -154,6 +154,24 @@ inline bool atomicCAS(volatile T* mem, T comp, T swap)
#endif #endif
} }
// implements a zero out of a variable
template <typename T>
inline void atomicZero(volatile T* mem)
{
#ifdef _MSC_VER
switch (sizeof(T))
{
case 4:
default: InterlockedXor(reinterpret_cast<volatile LONG*>(mem), (static_cast<LONG>(*mem))); break;
case 8: InterlockedXor64(reinterpret_cast<volatile LONG*>(mem), (static_cast<LONG>(*mem))); break;
}
#else
__sync_xor_and_fetch(mem, *mem);
#endif
}
// Implements a scheduler yield // Implements a scheduler yield
inline void atomicYield() inline void atomicYield()
{ {

View File

@ -108,9 +108,6 @@ JoinPartition::JoinPartition(const RowGroup& lRG, const RowGroup& sRG, const vec
buckets.reserve(bucketCount); buckets.reserve(bucketCount);
for (int i = 0; i < (int)bucketCount; i++)
buckets.push_back(boost::shared_ptr<JoinPartition>(new JoinPartition(*this, false)));
string compressionType; string compressionType;
try try
{ {
@ -128,6 +125,9 @@ JoinPartition::JoinPartition(const RowGroup& lRG, const RowGroup& sRG, const vec
{ {
compressor.reset(new compress::CompressInterfaceSnappy()); compressor.reset(new compress::CompressInterfaceSnappy());
} }
for (uint32_t i = 0; i < bucketCount; i++)
buckets.push_back(boost::shared_ptr<JoinPartition>(new JoinPartition(*this, false)));
} }
/* Ctor used by JoinPartition on expansion, creates JP's in filemode */ /* Ctor used by JoinPartition on expansion, creates JP's in filemode */

View File

@ -103,6 +103,7 @@
2054 ERR_DISKAGG_ERROR Unknown error while aggregation. 2054 ERR_DISKAGG_ERROR Unknown error while aggregation.
2055 ERR_DISKAGG_TOO_BIG Not enough memory to make disk-based aggregation. Raise TotalUmMemory if possible. 2055 ERR_DISKAGG_TOO_BIG Not enough memory to make disk-based aggregation. Raise TotalUmMemory if possible.
2056 ERR_DISKAGG_FILEIO_ERROR There was an IO error during a disk-based aggregation: %1% 2056 ERR_DISKAGG_FILEIO_ERROR There was an IO error during a disk-based aggregation: %1%
2057 ERR_JOIN_RESULT_TOO_BIG Not enough memory to consolidate join results. Estimated %1% MB needed. TotalUmMemory is %2% MB.
# Sub-query errors # Sub-query errors
3001 ERR_NON_SUPPORT_SUB_QUERY_TYPE This subquery type is not supported yet. 3001 ERR_NON_SUPPORT_SUB_QUERY_TYPE This subquery type is not supported yet.

View File

@ -4705,12 +4705,12 @@ RowAggregationMultiDistinct::RowAggregationMultiDistinct(const RowAggregationMul
for (uint32_t i = 0; i < rhs.fSubAggregators.size(); i++) for (uint32_t i = 0; i < rhs.fSubAggregators.size(); i++)
{ {
#if 0 #if 0
fTotalMemUsage += fSubRowGroups[i].getDataSize(AGG_ROWGROUP_SIZE);
if (!fRm->getMemory(fSubRowGroups[i].getDataSize(AGG_ROWGROUP_SIZE, fSessionMemLimit))) if (!fRm->getMemory(fSubRowGroups[i].getDataSize(AGG_ROWGROUP_SIZE, fSessionMemLimit)))
throw logging::IDBExcept(logging::IDBErrorInfo::instance()-> throw logging::IDBExcept(logging::IDBErrorInfo::instance()->
errorMsg(logging::ERR_AGGREGATION_TOO_BIG), logging::ERR_AGGREGATION_TOO_BIG); errorMsg(logging::ERR_AGGREGATION_TOO_BIG), logging::ERR_AGGREGATION_TOO_BIG);
fTotalMemUsage += fSubRowGroups[i].getDataSize(AGG_ROWGROUP_SIZE);
#endif #endif
data.reset(new RGData(fSubRowGroups[i], data.reset(new RGData(fSubRowGroups[i],
RowAggStorage::getMaxRows(fRm ? fRm->getAllowDiskAggregation() : false))); RowAggStorage::getMaxRows(fRm ? fRm->getAllowDiskAggregation() : false)));
@ -4748,12 +4748,11 @@ void RowAggregationMultiDistinct::addSubAggregator(const boost::shared_ptr<RowAg
{ {
boost::shared_ptr<RGData> data; boost::shared_ptr<RGData> data;
#if 0 #if 0
fTotalMemUsage += rg.getDataSize(AGG_ROWGROUP_SIZE);
if (!fRm->getMemory(rg.getDataSize(AGG_ROWGROUP_SIZE), fSessionMemLimit)) if (!fRm->getMemory(rg.getDataSize(AGG_ROWGROUP_SIZE), fSessionMemLimit))
throw logging::IDBExcept(logging::IDBErrorInfo::instance()-> throw logging::IDBExcept(logging::IDBErrorInfo::instance()->
errorMsg(logging::ERR_AGGREGATION_TOO_BIG), logging::ERR_AGGREGATION_TOO_BIG); errorMsg(logging::ERR_AGGREGATION_TOO_BIG), logging::ERR_AGGREGATION_TOO_BIG);
fTotalMemUsage += rg.getDataSize(AGG_ROWGROUP_SIZE);
#endif #endif
data.reset(new RGData(rg, RowAggStorage::getMaxRows(fRm ? fRm->getAllowDiskAggregation() : false))); data.reset(new RGData(rg, RowAggStorage::getMaxRows(fRm ? fRm->getAllowDiskAggregation() : false)));
fSubRowData.push_back(data); fSubRowData.push_back(data);

View File

@ -412,20 +412,25 @@ class RMMemManager : public MemManager
protected: protected:
bool acquireImpl(size_t amount) final bool acquireImpl(size_t amount) final
{ {
MemManager::acquireImpl(amount); if (amount)
{
if (!fRm->getMemory(amount, fSessLimit, fWait) && fStrict) if (!fRm->getMemory(amount, fSessLimit, fWait) && fStrict)
{ {
return false; return false;
} }
MemManager::acquireImpl(amount);
}
return true; return true;
} }
void releaseImpl(size_t amount) override void releaseImpl(size_t amount) override
{ {
if (amount)
{
MemManager::releaseImpl(amount); MemManager::releaseImpl(amount);
fRm->returnMemory(amount, fSessLimit); fRm->returnMemory(amount, fSessLimit);
} }
}
private: private:
joblist::ResourceManager* fRm = nullptr; joblist::ResourceManager* fRm = nullptr;

View File

@ -28,6 +28,7 @@ using namespace std;
#include "messageobj.h" #include "messageobj.h"
#include "messagelog.h" #include "messagelog.h"
#include "threadnaming.h"
using namespace logging; using namespace logging;
#include "prioritythreadpool.h" #include "prioritythreadpool.h"
@ -39,7 +40,7 @@ namespace threadpool
{ {
PriorityThreadPool::PriorityThreadPool(uint targetWeightPerRun, uint highThreads, uint midThreads, PriorityThreadPool::PriorityThreadPool(uint targetWeightPerRun, uint highThreads, uint midThreads,
uint lowThreads, uint ID) uint lowThreads, uint ID)
: _stop(false), weightPerRun(targetWeightPerRun), id(ID) : _stop(false), weightPerRun(targetWeightPerRun), id(ID), blockedThreads(0), extraThreads(0), stopExtra(true)
{ {
boost::thread* newThread; boost::thread* newThread;
for (uint32_t i = 0; i < highThreads; i++) for (uint32_t i = 0; i < highThreads; i++)
@ -98,6 +99,21 @@ void PriorityThreadPool::addJob(const Job& job, bool useLock)
threadCounts[LOW]++; threadCounts[LOW]++;
} }
// If some threads have blocked (because of output queue full)
// Temporarily add some extra worker threads to make up for the blocked threads.
if (blockedThreads > extraThreads)
{
stopExtra = false;
newThread = threads.create_thread(ThreadHelper(this, EXTRA));
newThread->detach();
extraThreads++;
}
else if (blockedThreads == 0)
{
// Release the temporary threads -- some threads have become unblocked.
stopExtra = true;
}
if (job.priority > 66) if (job.priority > 66)
jobQueues[HIGH].push_back(job); jobQueues[HIGH].push_back(job);
else if (job.priority > 33) else if (job.priority > 33)
@ -125,7 +141,7 @@ void PriorityThreadPool::removeJobs(uint32_t id)
PriorityThreadPool::Priority PriorityThreadPool::pickAQueue(Priority preference) PriorityThreadPool::Priority PriorityThreadPool::pickAQueue(Priority preference)
{ {
if (!jobQueues[preference].empty()) if (preference != EXTRA && !jobQueues[preference].empty())
return preference; return preference;
else if (!jobQueues[HIGH].empty()) else if (!jobQueues[HIGH].empty())
return HIGH; return HIGH;
@ -137,6 +153,10 @@ PriorityThreadPool::Priority PriorityThreadPool::pickAQueue(Priority preference)
void PriorityThreadPool::threadFcn(const Priority preferredQueue) throw() void PriorityThreadPool::threadFcn(const Priority preferredQueue) throw()
{ {
if (preferredQueue == EXTRA)
utils::setThreadName("Extra");
else
utils::setThreadName("Idle");
Priority queue = LOW; Priority queue = LOW;
uint32_t weight, i = 0; uint32_t weight, i = 0;
vector<Job> runList; vector<Job> runList;
@ -155,6 +175,14 @@ void PriorityThreadPool::threadFcn(const Priority preferredQueue) throw()
if (jobQueues[queue].empty()) if (jobQueues[queue].empty())
{ {
// If this is an EXTRA thread due toother threads blocking, and all blockers are unblocked,
// we don't want this one any more.
if (preferredQueue == EXTRA && stopExtra)
{
extraThreads--;
return;
}
newJob.wait(lk); newJob.wait(lk);
continue; continue;
} }
@ -190,6 +218,10 @@ void PriorityThreadPool::threadFcn(const Priority preferredQueue) throw()
if (reschedule[i]) if (reschedule[i])
rescheduleCount++; rescheduleCount++;
} }
if (preferredQueue == EXTRA)
utils::setThreadName("Extra (used)");
else
utils::setThreadName("Idle");
// no real work was done, prevent intensive busy waiting // no real work was done, prevent intensive busy waiting
if (rescheduleCount == runList.size()) if (rescheduleCount == runList.size())

View File

@ -34,8 +34,10 @@
#include <boost/thread/condition.hpp> #include <boost/thread/condition.hpp>
#include <boost/shared_ptr.hpp> #include <boost/shared_ptr.hpp>
#include <boost/function.hpp> #include <boost/function.hpp>
#include <atomic>
#include "../winport/winport.h" #include "../winport/winport.h"
#include "primitives/primproc/umsocketselector.h" #include "primitives/primproc/umsocketselector.h"
#include "atomicops.h"
namespace threadpool namespace threadpool
{ {
@ -72,7 +74,9 @@ class PriorityThreadPool
LOW, LOW,
MEDIUM, MEDIUM,
HIGH, HIGH,
_COUNT _COUNT,
EXTRA // After _COUNT because _COUNT is for jobQueue size and EXTRA isn't a jobQueue. But we need EXTRA
// in places where Priority is used.
}; };
/********************************************* /*********************************************
@ -95,6 +99,20 @@ class PriorityThreadPool
*/ */
void dump(); void dump();
// If a job is blocked, we want to temporarily increase the number of threads managed by the pool
// A problem can occur if all threads are running long or blocked for a single query. Other
// queries won't get serviced, even though there are cpu cycles available.
// These calls are currently protected by respondLock in sendThread(). If you call from other
// places, you need to consider atomicity.
void incBlockedThreads()
{
blockedThreads++;
}
void decBlockedThreads()
{
blockedThreads--;
}
protected: protected:
private: private:
struct ThreadHelper struct ThreadHelper
@ -127,6 +145,10 @@ class PriorityThreadPool
bool _stop; bool _stop;
uint32_t weightPerRun; uint32_t weightPerRun;
volatile uint id; // prevent it from being optimized out volatile uint id; // prevent it from being optimized out
std::atomic<uint32_t> blockedThreads;
std::atomic<uint32_t> extraThreads;
bool stopExtra;
}; };
} // namespace threadpool } // namespace threadpool

View File

@ -83,6 +83,7 @@ void ThreadPool::setQueueSize(size_t queueSize)
void ThreadPool::pruneThread() void ThreadPool::pruneThread()
{ {
utils::setThreadName("pruneThread");
boost::unique_lock<boost::mutex> lock2(fPruneMutex); boost::unique_lock<boost::mutex> lock2(fPruneMutex);
while (true) while (true)

View File

@ -754,14 +754,12 @@ void IdbOrderBy::initialize(const RowGroup& rg)
IdbCompare::initialize(rg); IdbCompare::initialize(rg);
uint64_t newSize = rg.getSizeWithStrings(fRowsPerRG); uint64_t newSize = rg.getSizeWithStrings(fRowsPerRG);
fMemSize += newSize; if (fRm && !fRm->getMemory(newSize, fSessionMemLimit))
if (!fRm->getMemory(newSize, fSessionMemLimit))
{ {
cerr << IDBErrorInfo::instance()->errorMsg(fErrorCode) << " @" << __FILE__ << ":" << __LINE__; cerr << IDBErrorInfo::instance()->errorMsg(fErrorCode) << " @" << __FILE__ << ":" << __LINE__;
throw IDBExcept(fErrorCode); throw IDBExcept(fErrorCode);
} }
fMemSize += newSize;
fData.reinit(fRowGroup, fRowsPerRG); fData.reinit(fRowGroup, fRowsPerRG);
fRowGroup.setData(&fData); fRowGroup.setData(&fData);
fRowGroup.resetRowGroup(0); fRowGroup.resetRowGroup(0);