diff --git a/dbcon/joblist/resourcemanager.cpp b/dbcon/joblist/resourcemanager.cpp index 9a5d501b3..46e20312a 100644 --- a/dbcon/joblist/resourcemanager.cpp +++ b/dbcon/joblist/resourcemanager.cpp @@ -55,6 +55,7 @@ const string ResourceManager::fExtentMapStr("ExtentMap"); //const string ResourceManager::fDMLProcStr("DMLProc"); //const string ResourceManager::fBatchInsertStr("BatchInsert"); const string ResourceManager::fOrderByLimitStr("OrderByLimit"); +const string ResourceManager::fRowAggregationStr("RowAggregation"); ResourceManager* ResourceManager::fInstance = NULL; boost::mutex mx; @@ -254,6 +255,10 @@ ResourceManager::ResourceManager(bool runningInExeMgr) : fUseHdfs = true; else fUseHdfs = false; + + fAllowedDiskAggregation = getBoolVal(fRowAggregationStr, + "AllowDiskBasedAggregation", + defaultAllowDiskAggregation); } int ResourceManager::getEmPriority() const diff --git a/dbcon/joblist/resourcemanager.h b/dbcon/joblist/resourcemanager.h index 8fd1dc6b2..e28314a0f 100644 --- a/dbcon/joblist/resourcemanager.h +++ b/dbcon/joblist/resourcemanager.h @@ -126,6 +126,8 @@ const uint64_t defaultOrderByLimitMaxMemory = 1 * 1024 * 1024 * 1024ULL; const uint64_t defaultDECThrottleThreshold = 200000000; // ~200 MB const uint8_t defaultUseCpimport = 1; + +const bool defaultAllowDiskAggregation = false; /** @brief ResourceManager * Returns requested values from Config * @@ -149,7 +151,7 @@ public: /** @brief dtor */ - virtual ~ResourceManager() { } + virtual ~ResourceManager() {} typedef std::map MemMap; @@ -177,6 +179,11 @@ public: return getIntVal(fExeMgrStr, "ExecQueueSize", defaultEMExecQueueSize); } + bool getAllowDiskAggregation() const + { + return fAllowedDiskAggregation; + } + int getHjMaxBuckets() const { return getUintVal(fHashJoinStr, "MaxBuckets", defaultHJMaxBuckets); @@ -391,7 +398,7 @@ public: atomicops::atomicAdd(&totalUmMemLimit, amount); atomicops::atomicAdd(sessionLimit.get(), amount); } - inline int64_t availableMemory() + inline int64_t availableMemory() const { return totalUmMemLimit; } @@ -559,6 +566,8 @@ private: template IntType getIntVal(const std::string& section, const std::string& name, IntType defval) const; + bool getBoolVal(const std::string& section, const std::string& name, bool defval) const; + void logMessage(logging::LOG_TYPE logLevel, logging::Message::MessageID mid, uint64_t value = 0, uint32_t sessionId = 0); /*static const*/ std::string fExeMgrStr; @@ -573,6 +582,7 @@ private: /*static const*/ std::string fDMLProcStr; /*static const*/ std::string fBatchInsertStr; static const std::string fOrderByLimitStr; + static const std::string fRowAggregationStr; config::Config* fConfig; static ResourceManager* fInstance; uint32_t fTraceFlags; @@ -604,6 +614,7 @@ private: bool isExeMgr; bool fUseHdfs; + bool fAllowedDiskAggregation{false}; }; @@ -644,7 +655,11 @@ inline IntType ResourceManager::getIntVal(const std::string& section, const std: return ( 0 == retStr.length() ? defval : fConfig->fromText(retStr) ); } - +inline bool ResourceManager::getBoolVal(const std::string& section, const std::string& name, bool defval) const +{ + auto retStr = fConfig->getConfig(section, name); + return ( 0 == retStr.length() ? defval : (retStr == "y" || retStr == "Y") ); +} } diff --git a/dbcon/joblist/tupleaggregatestep.cpp b/dbcon/joblist/tupleaggregatestep.cpp index 74f8b46cb..eefcab950 100644 --- a/dbcon/joblist/tupleaggregatestep.cpp +++ b/dbcon/joblist/tupleaggregatestep.cpp @@ -129,7 +129,7 @@ struct cmpTuple } }; -typedef vector RowBucket; +typedef vector> RowBucket; typedef vector RowBucketVec; // The AGG_MAP type is used to maintain a list of aggregate functions in order to @@ -402,6 +402,17 @@ TupleAggregateStep::TupleAggregateStep( fNumOfThreads = fRm->aggNumThreads(); fNumOfBuckets = fRm->aggNumBuckets(); fNumOfRowGroups = fRm->aggNumRowGroups(); + + auto memLimit = std::min(fRm->availableMemory(), *fSessionMemLimit); + fNumOfBuckets = calcNumberOfBuckets(memLimit, + fNumOfThreads, + fNumOfBuckets, + fNumOfRowGroups, + fRowGroupIn.getRowSize(), + fRowGroupOut.getRowSize(), + fRm->getAllowDiskAggregation()); + fNumOfThreads = std::min(fNumOfThreads, fNumOfBuckets); + fMemUsage.reset(new uint64_t[fNumOfThreads]); memset(fMemUsage.get(), 0, fNumOfThreads * sizeof(uint64_t)); @@ -440,7 +451,7 @@ void TupleAggregateStep::initializeMultiThread() for (i = 0; i < fNumOfBuckets; i++) { - boost::mutex* lock = new boost::mutex(); + boost::mutex* lock = new boost::mutex(); fAgg_mutex.push_back(lock); fRowGroupOuts[i] = fRowGroupOut; rgData.reinit(fRowGroupOut); @@ -481,9 +492,10 @@ void TupleAggregateStep::doThreadedSecondPhaseAggregate(uint32_t threadID) RowAggregationDistinct* aggDist = dynamic_cast(fAggregators[threadID].get()); RowAggregationMultiDistinct* multiDist = dynamic_cast(fAggregators[threadID].get()); Row rowIn; - RowGroup* rowGroupIn = 0; + RowGroup* rowGroupIn = nullptr; rowGroupIn = (aggDist->aggregator()->getOutputRowGroup()); uint32_t bucketID; + std::vector> rgDataVec; if (multiDist) { @@ -503,10 +515,12 @@ void TupleAggregateStep::doThreadedSecondPhaseAggregate(uint32_t threadID) { rowGroupIn = (multiDist->subAggregators()[j]->getOutputRowGroup()); rowGroupIn->initRow(&rowIn); + auto* subDistAgg = dynamic_cast(multiDist->subAggregators()[j].get()); - while (dynamic_cast(multiDist->subAggregators()[j].get())->nextRowGroup()) + while (subDistAgg->nextRowGroup()) { rowGroupIn = (multiDist->subAggregators()[j]->getOutputRowGroup()); + rgDataVec.emplace_back(subDistAgg->moveCurrentRGData()); rowGroupIn->getRow(0, &rowIn); for (uint64_t i = 0; i < rowGroupIn->getRowCount(); ++i) @@ -514,8 +528,9 @@ void TupleAggregateStep::doThreadedSecondPhaseAggregate(uint32_t threadID) // The key is the groupby columns, which are the leading columns. //uint8_t* hashMapKey = rowIn.getData() + 2; //bucketID = hash.operator()(hashMapKey) & fBucketMask; - bucketID = rowIn.hash(hashlen - 1) % fNumOfBuckets; - rowBucketVecs[bucketID][j].push_back(rowIn.getPointer()); + uint64_t hash = rowgroup::hashRow(rowIn, hashlen - 1); + bucketID = hash % fNumOfBuckets; + rowBucketVecs[bucketID][j].emplace_back(rowIn.getPointer(), hash); rowIn.nextRow(); } } @@ -524,10 +539,12 @@ void TupleAggregateStep::doThreadedSecondPhaseAggregate(uint32_t threadID) else { rowGroupIn->initRow(&rowIn); + auto* subAgg = dynamic_cast(aggDist->aggregator().get()); - while (dynamic_cast(aggDist->aggregator().get())->nextRowGroup()) + while (subAgg->nextRowGroup()) { rowGroupIn->setData(aggDist->aggregator()->getOutputRowGroup()->getRGData()); + rgDataVec.emplace_back(subAgg->moveCurrentRGData()); rowGroupIn->getRow(0, &rowIn); for (uint64_t i = 0; i < rowGroupIn->getRowCount(); ++i) @@ -535,8 +552,9 @@ void TupleAggregateStep::doThreadedSecondPhaseAggregate(uint32_t threadID) // The key is the groupby columns, which are the leading columns. //uint8_t* hashMapKey = rowIn.getData() + 2; //bucketID = hash.operator()(hashMapKey) & fBucketMask; - bucketID = rowIn.hash(hashlen - 1) % fNumOfBuckets; - rowBucketVecs[bucketID][0].push_back(rowIn.getPointer()); + uint64_t hash = rowgroup::hashRow(rowIn, hashlen - 1); + bucketID = hash % fNumOfBuckets; + rowBucketVecs[bucketID][0].emplace_back(rowIn.getPointer(), hash); rowIn.nextRow(); } } @@ -971,7 +989,7 @@ SJSTEP TupleAggregateStep::prepAggregate(SJSTEP& step, JobInfo& jobInfo) if (doUMOnly) rgs.push_back(rgs[0]); } - + if (!doUMOnly) { if (distinctAgg == true) @@ -1013,7 +1031,7 @@ SJSTEP TupleAggregateStep::prepAggregate(SJSTEP& step, JobInfo& jobInfo) // Setup the input JobstepAssoctiation -- the mechanism // whereby the previous step feeds data to this step. - // Otherwise, we need to create one and hook to the + // Otherwise, we need to create one and hook to the // previous step as well as this aggregate step. spjs->stepId(step->stepId() + 1); @@ -1299,7 +1317,7 @@ void TupleAggregateStep::prep1PhaseAggregate( if (pUDAFFunc && udafc->getContext().getParamKeys()->size() == 0) { for (uint64_t k = i+1; - k < returnedColVec.size() && returnedColVec[k].second == AggregateColumn::MULTI_PARM; + k < returnedColVec.size() && returnedColVec[k].second == AggregateColumn::MULTI_PARM; ++k) { udafc->getContext().getParamKeys()->push_back(returnedColVec[k].first); @@ -1333,7 +1351,7 @@ void TupleAggregateStep::prep1PhaseAggregate( precisionAgg.push_back(precisionProj[colProj]); typeAgg.push_back(typeProj[colProj]); csNumAgg.push_back(csNumProj[colProj]); - widthAgg.push_back(width[colProj]); + widthAgg.push_back(width[colProj]); } break; @@ -1836,7 +1854,7 @@ void TupleAggregateStep::prep1PhaseDistinctAggregate( if (pUDAFFunc && udafc->getContext().getParamKeys()->size() == 0) { for (uint64_t k = i+1; - k < aggColVec.size() && aggColVec[k].second == AggregateColumn::MULTI_PARM; + k < aggColVec.size() && aggColVec[k].second == AggregateColumn::MULTI_PARM; ++k) { udafc->getContext().getParamKeys()->push_back(aggColVec[k].first); @@ -2140,7 +2158,7 @@ void TupleAggregateStep::prep1PhaseDistinctAggregate( groupByNoDist.push_back(groupby); aggFuncMap.insert(make_pair(boost::make_tuple(keysAgg[i], 0, pUDAFFunc, udafc ? udafc->getContext().getParamKeys() : NULL), i)); } - + // locate the return column position in aggregated rowgroup uint64_t outIdx = 0; for (uint64_t i = 0; i < returnedColVec.size(); i++) @@ -2198,7 +2216,7 @@ void TupleAggregateStep::prep1PhaseDistinctAggregate( if (pUDAFFunc && udafc->getContext().getParamKeys()->size() == 0) { for (uint64_t k = i+1; - k < returnedColVec.size() && returnedColVec[k].second == AggregateColumn::MULTI_PARM; + k < returnedColVec.size() && returnedColVec[k].second == AggregateColumn::MULTI_PARM; ++k) { udafc->getContext().getParamKeys()->push_back(returnedColVec[k].first); @@ -3120,7 +3138,7 @@ void TupleAggregateStep::prep2PhasesAggregate( if (pUDAFFunc && udafc->getContext().getParamKeys()->size() == 0) { for (uint64_t k = i+1; - k < aggColVec.size() && aggColVec[k].second == AggregateColumn::MULTI_PARM; + k < aggColVec.size() && aggColVec[k].second == AggregateColumn::MULTI_PARM; ++k) { udafc->getContext().getParamKeys()->push_back(aggColVec[k].first); @@ -3430,7 +3448,7 @@ void TupleAggregateStep::prep2PhasesAggregate( if (pUDAFFunc && udafc->getContext().getParamKeys()->size() == 0) { for (uint64_t k = i+1; - k < returnedColVec.size() && returnedColVec[k].second == AggregateColumn::MULTI_PARM; + k < returnedColVec.size() && returnedColVec[k].second == AggregateColumn::MULTI_PARM; ++k) { udafc->getContext().getParamKeys()->push_back(returnedColVec[k].first); @@ -3732,7 +3750,7 @@ void TupleAggregateStep::prep2PhasesAggregate( for (uint64_t i = 0; i < oidsAggUm.size(); i++) posAggUm.push_back(posAggUm[i] + widthAggUm[i]); - RowGroup aggRgUm(oidsAggUm.size(), posAggUm, oidsAggUm, keysAggUm, typeAggUm, + RowGroup aggRgUm(oidsAggUm.size(), posAggUm, oidsAggUm, keysAggUm, typeAggUm, csNumAggUm, scaleAggUm, precisionAggUm, jobInfo.stringTableThreshold); SP_ROWAGG_UM_t rowAggUm(new RowAggregationUMP2(groupByUm, functionVecUm, jobInfo.rm, jobInfo.umMemLimit)); rowAggUm->timeZone(jobInfo.timeZone); @@ -3744,7 +3762,7 @@ void TupleAggregateStep::prep2PhasesAggregate( for (uint64_t i = 0; i < oidsAggPm.size(); i++) posAggPm.push_back(posAggPm[i] + widthAggPm[i]); - RowGroup aggRgPm(oidsAggPm.size(), posAggPm, oidsAggPm, keysAggPm, typeAggPm, + RowGroup aggRgPm(oidsAggPm.size(), posAggPm, oidsAggPm, keysAggPm, typeAggPm, csNumAggPm, scaleAggPm, precisionAggPm, jobInfo.stringTableThreshold); SP_ROWAGG_PM_t rowAggPm(new RowAggregation(groupByPm, functionVecPm)); rowAggPm->timeZone(jobInfo.timeZone); @@ -4005,7 +4023,7 @@ void TupleAggregateStep::prep2PhasesDistinctAggregate( if (pUDAFFunc && udafc->getContext().getParamKeys()->size() == 0) { for (uint64_t k = i+1; - k < aggColVec.size() && aggColVec[k].second == AggregateColumn::MULTI_PARM; + k < aggColVec.size() && aggColVec[k].second == AggregateColumn::MULTI_PARM; ++k) { udafc->getContext().getParamKeys()->push_back(aggColVec[k].first); @@ -4401,7 +4419,7 @@ void TupleAggregateStep::prep2PhasesDistinctAggregate( if (pUDAFFunc && udafc->getContext().getParamKeys()->size() == 0) { for (uint64_t k = i+1; - k < returnedColVec.size() && returnedColVec[k].second == AggregateColumn::MULTI_PARM; + k < returnedColVec.size() && returnedColVec[k].second == AggregateColumn::MULTI_PARM; ++k) { udafc->getContext().getParamKeys()->push_back(returnedColVec[k].first); @@ -4808,7 +4826,7 @@ void TupleAggregateStep::prep2PhasesDistinctAggregate( for (uint64_t i = 0; i < oidsAggUm.size(); i++) posAggUm.push_back(posAggUm[i] + widthAggUm[i]); - RowGroup aggRgUm(oidsAggUm.size(), posAggUm, oidsAggUm, keysAggUm, typeAggUm, + RowGroup aggRgUm(oidsAggUm.size(), posAggUm, oidsAggUm, keysAggUm, typeAggUm, csNumAggUm, scaleAggUm, precisionAggUm, jobInfo.stringTableThreshold); SP_ROWAGG_UM_t rowAggUm(new RowAggregationUMP2(groupByUm, functionNoDistVec, jobInfo.rm, jobInfo.umMemLimit)); rowAggUm->timeZone(jobInfo.timeZone); @@ -4818,8 +4836,8 @@ void TupleAggregateStep::prep2PhasesDistinctAggregate( for (uint64_t i = 0; i < oidsAggDist.size(); i++) posAggDist.push_back(posAggDist[i] + widthAggDist[i]); - RowGroup aggRgDist(oidsAggDist.size(), posAggDist, oidsAggDist, keysAggDist, - typeAggDist, csNumAggDist, scaleAggDist, + RowGroup aggRgDist(oidsAggDist.size(), posAggDist, oidsAggDist, keysAggDist, + typeAggDist, csNumAggDist, scaleAggDist, precisionAggDist, jobInfo.stringTableThreshold); SP_ROWAGG_DIST rowAggDist(new RowAggregationDistinct(groupByNoDist, functionVecUm, jobInfo.rm, jobInfo.umMemLimit)); rowAggDist->timeZone(jobInfo.timeZone); @@ -5058,7 +5076,7 @@ void TupleAggregateStep::prep2PhasesDistinctAggregate( for (uint64_t i = 0; i < oidsAggPm.size(); i++) posAggPm.push_back(posAggPm[i] + widthAggPm[i]); - RowGroup aggRgPm(oidsAggPm.size(), posAggPm, oidsAggPm, keysAggPm, typeAggPm, + RowGroup aggRgPm(oidsAggPm.size(), posAggPm, oidsAggPm, keysAggPm, typeAggPm, csNumAggPm, scaleAggPm, precisionAggPm, jobInfo.stringTableThreshold); SP_ROWAGG_PM_t rowAggPm(new RowAggregation(groupByPm, functionVecPm)); rowAggPm->timeZone(jobInfo.timeZone); @@ -5100,7 +5118,7 @@ void TupleAggregateStep::prepExpressionOnAggregate(SP_ROWAGG_UM_t& aggUM, JobInf uint64_t eid = -1; if (((ac = dynamic_cast(it->get())) != NULL) && - (ac->aggColumnList().size() > 0) && + (ac->aggColumnList().size() > 0) && (ac->windowfunctionColumnList().size() == 0)) { const vector& scols = ac->simpleColumnList(); @@ -5256,6 +5274,26 @@ void TupleAggregateStep::aggregateRowGroups() } } +void TupleAggregateStep::threadedAggregateFinalize(uint32_t threadID) +{ + for (uint32_t i = 0; i < fNumOfBuckets; ++i) + { + if (fAgg_mutex[i]->try_lock()) + { + try + { + if (fAggregators[i]) + fAggregators[i]->finalAggregation(); + } + catch (...) + { + fAgg_mutex[i]->unlock(); + throw; + } + fAgg_mutex[i]->unlock(); + } + } +} void TupleAggregateStep::threadedAggregateRowGroups(uint32_t threadID) { @@ -5268,9 +5306,10 @@ void TupleAggregateStep::threadedAggregateRowGroups(uint32_t threadID) vector hashLens; bool locked = false; bool more = true; - RowGroupDL* dlIn = NULL; + RowGroupDL* dlIn = nullptr; + uint32_t rgVecShift = float(fNumOfBuckets) / fNumOfThreads * threadID; - RowAggregationMultiDistinct* multiDist = NULL; + RowAggregationMultiDistinct* multiDist = nullptr; if (!fDoneAggregate) { @@ -5279,7 +5318,7 @@ void TupleAggregateStep::threadedAggregateRowGroups(uint32_t threadID) dlIn = fInputJobStepAssociation.outAt(0)->rowGroupDL(); - if (dlIn == NULL) + if (dlIn == nullptr) throw logic_error("Input is not RowGroup data list in delivery step."); vector rgDatas; @@ -5358,29 +5397,35 @@ void TupleAggregateStep::threadedAggregateRowGroups(uint32_t threadID) if (more) { - fRowGroupIns[threadID].setData(&rgData); - fMemUsage[threadID] += fRowGroupIns[threadID].getSizeWithStrings(); + fRowGroupIns[threadID].setData(&rgData); + fMemUsage[threadID] += + fRowGroupIns[threadID].getSizeWithStrings(); - if (!fRm->getMemory(fRowGroupIns[threadID].getSizeWithStrings(), fSessionMemLimit)) - { - rgDatas.clear(); // to short-cut the rest of processing - abort(); - more = false; - fEndOfResult = true; + bool diskAggAllowed = fRm->getAllowDiskAggregation(); + if (!fRm->getMemory( + fRowGroupIns[threadID].getSizeWithStrings(), + fSessionMemLimit, !diskAggAllowed)) + { + if (!diskAggAllowed) + { + rgDatas.clear(); // to short-cut the rest of processing + more = false; + fEndOfResult = true; - if (status() == 0) - { - errorMessage(IDBErrorInfo::instance()->errorMsg( - ERR_AGGREGATION_TOO_BIG)); - status(ERR_AGGREGATION_TOO_BIG); - } - - break; - } - else - { - rgDatas.push_back(rgData); - } + if (status() == 0) + { + errorMessage(IDBErrorInfo::instance()->errorMsg( + ERR_AGGREGATION_TOO_BIG)); + status(ERR_AGGREGATION_TOO_BIG); + } + } + else + { + rgDatas.push_back(rgData); + } + break; + } + rgDatas.push_back(rgData); } else { @@ -5429,8 +5474,9 @@ void TupleAggregateStep::threadedAggregateRowGroups(uint32_t threadID) // TBD This approach could potentiall // put all values in on bucket. - bucketID = distRow[j].hash(hashLens[j] - 1) % fNumOfBuckets; - rowBucketVecs[bucketID][j].push_back(rowIn.getPointer()); + uint64_t hash = rowgroup::hashRow(distRow[j], hashLens[j] - 1); + bucketID = hash % fNumOfBuckets; + rowBucketVecs[bucketID][j].emplace_back(rowIn.getPointer(), hash); rowIn.nextRow(); } } @@ -5447,10 +5493,11 @@ void TupleAggregateStep::threadedAggregateRowGroups(uint32_t threadID) for (uint64_t i = 0; i < fRowGroupIns[threadID].getRowCount(); ++i) { // The key is the groupby columns, which are the leading columns. - // TBD This approach could potentiall + // TBD This approach could potential // put all values in on bucket. - int bucketID = rowIn.hash(hashLens[0] - 1) % fNumOfBuckets; - rowBucketVecs[bucketID][0].push_back(rowIn.getPointer()); + uint64_t hash = rowgroup::hashRow(rowIn, hashLens[0] - 1); + int bucketID = hash% fNumOfBuckets; + rowBucketVecs[bucketID][0].emplace_back(rowIn.getPointer(), hash); rowIn.nextRow(); } } @@ -5465,8 +5512,11 @@ void TupleAggregateStep::threadedAggregateRowGroups(uint32_t threadID) bool didWork = false; done = true; - for (uint32_t c = 0; c < fNumOfBuckets && !cancelled(); c++) + // each thread starts from its own bucket for better distribution + uint32_t shift = (rgVecShift++) % fNumOfBuckets; + for (uint32_t ci = 0; ci < fNumOfBuckets && !cancelled(); ci++) { + uint32_t c = (ci + shift) % fNumOfBuckets; if (!fEndOfResult && !bucketDone[c] && fAgg_mutex[c]->try_lock()) { try @@ -5484,9 +5534,9 @@ void TupleAggregateStep::threadedAggregateRowGroups(uint32_t threadID) throw; } - fAgg_mutex[c]->unlock(); rowBucketVecs[c][0].clear(); bucketDone[c] = true; + fAgg_mutex[c]->unlock(); } else if (!bucketDone[c]) { @@ -5519,7 +5569,7 @@ void TupleAggregateStep::threadedAggregateRowGroups(uint32_t threadID) handleException(std::current_exception(), logging::tupleAggregateStepErr, logging::ERR_AGGREGATION_TOO_BIG, - "TupleAggregateStep::threadedAggregateRowGroups()"); + "TupleAggregateStep::threadedAggregateRowGroups()[" + std::to_string(threadID) + "]"); fEndOfResult = true; fDoneAggregate = true; } @@ -5527,7 +5577,8 @@ void TupleAggregateStep::threadedAggregateRowGroups(uint32_t threadID) if (!locked) fMutex.lock(); - while (more) more = dlIn->next(fInputIter, &rgData); + while (more) + more = dlIn->next(fInputIter, &rgData); fMutex.unlock(); locked = false; @@ -5639,6 +5690,20 @@ uint64_t TupleAggregateStep::doThreadedAggregate(ByteStream& bs, RowGroupDL* dlp jobstepThreadPool.join(runners); } + if (!cancelled()) + { + vector runners; + // use half of the threads because finalizing requires twice as + // much memory on average + uint32_t threads = std::max(1U, fNumOfThreads / 2); + runners.reserve(threads); + for (i = 0; i < threads; ++i) + { + runners.push_back(jobstepThreadPool.invoke(ThreadedAggregateFinalizer(this, i))); + } + jobstepThreadPool.join(runners); + } + if (dynamic_cast(fAggregator.get()) && fAggregator->aggMapKeyLength() > 0) { // 2nd phase multi-threaded aggregate @@ -5700,7 +5765,7 @@ uint64_t TupleAggregateStep::doThreadedAggregate(ByteStream& bs, RowGroupDL* dlp } else { - RowAggregationDistinct* agg = dynamic_cast(fAggregator.get()); + auto* agg = dynamic_cast(fAggregator.get()); if (!fEndOfResult) { @@ -5713,27 +5778,26 @@ uint64_t TupleAggregateStep::doThreadedAggregate(ByteStream& bs, RowGroupDL* dlp // do the final aggregtion and deliver the results // at least one RowGroup for aggregate results // for "distinct without group by" case - if (agg != NULL) + if (agg != nullptr) { - RowAggregationMultiDistinct* aggMultiDist = + auto* aggMultiDist = dynamic_cast(fAggregators[i].get()); - RowAggregationDistinct* aggDist = + auto* aggDist = dynamic_cast(fAggregators[i].get()); agg->aggregator(aggDist->aggregator()); if (aggMultiDist) + { (dynamic_cast(agg)) - ->subAggregators(aggMultiDist->subAggregators()); + ->subAggregators(aggMultiDist->subAggregators()); + } agg->doDistinctAggregation(); } // for "group by without distinct" case else { - fAggregator->resultDataVec().insert( - fAggregator->resultDataVec().end(), - fAggregators[i]->resultDataVec().begin(), - fAggregators[i]->resultDataVec().end()); + fAggregator->append(fAggregators[i].get()); } } } diff --git a/dbcon/joblist/tupleaggregatestep.h b/dbcon/joblist/tupleaggregatestep.h index f137daf6f..07884dcb3 100644 --- a/dbcon/joblist/tupleaggregatestep.h +++ b/dbcon/joblist/tupleaggregatestep.h @@ -105,6 +105,7 @@ private: uint64_t doThreadedAggregate(messageqcpp::ByteStream& bs, RowGroupDL* dlp); void aggregateRowGroups(); void threadedAggregateRowGroups(uint32_t threadID); + void threadedAggregateFinalize(uint32_t threadID); void doThreadedSecondPhaseAggregate(uint32_t threadID); bool nextDeliveredRowGroup(); void pruneAuxColumns(); @@ -156,7 +157,9 @@ private: {} void operator()() { - utils::setThreadName("TASThrAggr"); + std::string t{"TASThrAggr"}; + t.append(std::to_string(fThreadID)); + utils::setThreadName(t.c_str()); fStep->threadedAggregateRowGroups(fThreadID); } @@ -164,6 +167,26 @@ private: uint32_t fThreadID; }; + class ThreadedAggregateFinalizer + { + public: + ThreadedAggregateFinalizer(TupleAggregateStep* step, uint32_t threadID) : + fStep(step), + fThreadID(threadID) + {} + + void operator()() + { + std::string t{"TASThrFin"}; + t.append(std::to_string(fThreadID)); + utils::setThreadName(t.c_str()); + fStep->threadedAggregateFinalize(fThreadID); + } + + TupleAggregateStep* fStep; + uint32_t fThreadID; + }; + class ThreadedSecondPhaseAggregator { public: diff --git a/exemgr/main.cpp b/exemgr/main.cpp index 52fab7c55..4c518275d 100644 --- a/exemgr/main.cpp +++ b/exemgr/main.cpp @@ -1453,49 +1453,85 @@ int setupResources() void cleanTempDir() { - const auto config = config::Config::makeConfig(); - std::string allowDJS = config->getConfig("HashJoin", "AllowDiskBasedJoin"); - std::string tmpPrefix = config->getConfig("HashJoin", "TempFilePath"); + using TempDirPurpose = config::Config::TempDirPurpose; + struct Dirs + { + std::string section; + std::string allowed; + TempDirPurpose purpose; + }; + std::vector dirs{ + { + "HashJoin", + "AllowDiskBasedJoin", + TempDirPurpose::Joins + }, + { + "RowAggregation", + "AllowDiskBasedAggregation", + TempDirPurpose::Aggregates + } + }; + const auto config = config::Config::makeConfig(); - if (allowDJS == "N" || allowDJS == "n") - return; + for (const auto& dir : dirs) + { + std::string allowStr = config->getConfig(dir.section, dir.allowed); + bool allow = (allowStr == "Y" || allowStr == "y"); - if (tmpPrefix.empty()) - tmpPrefix = "/tmp/cs-diskjoin"; + std::string tmpPrefix = config->getTempFileDir(dir.purpose); + + if (allow && tmpPrefix.empty()) + { + std::cerr << "Empty tmp directory name for " << dir.section << std::endl; + logging::LoggingID logid(16, 0, 0); + logging::Message::Args args; + logging::Message message(8); + args.add("Empty tmp directory name for:"); + args.add(dir.section); + message.format(args); + logging::Logger logger(logid.fSubsysID); + logger.logMessage(logging::LOG_TYPE_CRITICAL, message, logid); + } tmpPrefix += "/"; - assert(tmpPrefix != "/"); + idbassert(tmpPrefix != "/"); /* This is quite scary as ExeMgr usually runs as root */ try { + if (allow) + { boost::filesystem::remove_all(tmpPrefix); - boost::filesystem::create_directories(tmpPrefix); + } + boost::filesystem::create_directories(tmpPrefix); } - catch (const std::exception& ex) + catch (const std::exception &ex) { - std::cerr << ex.what() << std::endl; - logging::LoggingID logid(16, 0, 0); - logging::Message::Args args; - logging::Message message(8); - args.add("Execption whilst cleaning tmpdir: "); - args.add(ex.what()); - message.format( args ); - logging::Logger logger(logid.fSubsysID); - logger.logMessage(logging::LOG_TYPE_WARNING, message, logid); + std::cerr << ex.what() << std::endl; + logging::LoggingID logid(16, 0, 0); + logging::Message::Args args; + logging::Message message(8); + args.add("Exception whilst cleaning tmpdir: "); + args.add(ex.what()); + message.format(args); + logging::Logger logger(logid.fSubsysID); + logger.logMessage(logging::LOG_TYPE_WARNING, message, logid); } catch (...) { - std::cerr << "Caught unknown exception during tmpdir cleanup" << std::endl; - logging::LoggingID logid(16, 0, 0); - logging::Message::Args args; - logging::Message message(8); - args.add("Unknown execption whilst cleaning tmpdir"); - message.format( args ); - logging::Logger logger(logid.fSubsysID); - logger.logMessage(logging::LOG_TYPE_WARNING, message, logid); + std::cerr << "Caught unknown exception during tmpdir cleanup" + << std::endl; + logging::LoggingID logid(16, 0, 0); + logging::Message::Args args; + logging::Message message(8); + args.add("Unknown exception whilst cleaning tmpdir"); + message.format(args); + logging::Logger logger(logid.fSubsysID); + logger.logMessage(logging::LOG_TYPE_WARNING, message, logid); } + } } diff --git a/oam/etc/Columnstore.xml b/oam/etc/Columnstore.xml index 14734d9d4..b7caf0b62 100644 --- a/oam/etc/Columnstore.xml +++ b/oam/etc/Columnstore.xml @@ -264,7 +264,10 @@ --> /rdwrscratch /columnstore_tmp_files - /tmp/columnstore_tmp_files + + /tmp/columnstore_tmp_files dm @@ -488,10 +491,6 @@ 25% 100 N - Y @@ -518,9 +517,10 @@ 512 - + + 127.0.0.1 diff --git a/oam/etc/Columnstore.xml.singleserver b/oam/etc/Columnstore.xml.singleserver index 1dbd870d6..483207323 100644 --- a/oam/etc/Columnstore.xml.singleserver +++ b/oam/etc/Columnstore.xml.singleserver @@ -253,6 +253,10 @@ 8G --> /tmp/rdwrscratch + + /tmp/columnstore_tmp_files dm @@ -482,10 +486,6 @@ 10% 100 N - - /var/lib/columnstore/tmp/cs-diskjoin Y @@ -512,9 +512,10 @@ 512 - + + 127.0.0.1 diff --git a/tools/rgprint/CMakeLists.txt b/tools/rgprint/CMakeLists.txt new file mode 100644 index 000000000..f2f877726 --- /dev/null +++ b/tools/rgprint/CMakeLists.txt @@ -0,0 +1,14 @@ + +include_directories( ${ENGINE_COMMON_INCLUDES} ) + + +########### next target ############### + +set(rgprint_SRCS rgprint.cpp) + +add_executable(rgprint ${rgprint_SRCS}) + +target_link_libraries(rgprint ${ENGINE_LDFLAGS} ${NETSNMP_LIBRARIES} ${MARIADB_CLIENT_LIBS} ${ENGINE_WRITE_LIBS}) + +install(TARGETS rgprint DESTINATION ${ENGINE_BINDIR} COMPONENT columnstore-engine) + diff --git a/tools/rgprint/rgprint.cpp b/tools/rgprint/rgprint.cpp new file mode 100644 index 000000000..7f2f13cf0 --- /dev/null +++ b/tools/rgprint/rgprint.cpp @@ -0,0 +1,94 @@ +/* Copyright (C) 2021 MariaDB Corporation + + This program is free software; you can redistribute it and/or + modify it under the terms of the GNU General Public License + as published by the Free Software Foundation; version 2 of + the License. + + This program is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + GNU General Public License for more details. + + You should have received a copy of the GNU General Public License + along with this program; if not, write to the Free Software + Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, + MA 02110-1301, USA. */ + +#include +#include +#include +#include +#include +#include + +int main(int argc, char* argv[]) +{ + if (argc < 2) + { + std::cerr << "Usage: " << argv[0] << " " << std::endl; + return 0; + } + rowgroup::RowGroup rg; + char* p = strrchr(argv[1], '/'); + int rfd = -1; + if (p == nullptr) + p = argv[1]; + unsigned pid; + void* agg; + auto c = sscanf(p, "Agg-p%u-t%p-", &pid, &agg); + if (c == 2) { + char fname[1024]; + snprintf(fname, sizeof(fname), "META-p%u-t%p", pid, agg); + rfd = open(fname, O_RDONLY); + } + if (rfd < 0) + rfd = open("./META", O_RDONLY); + if (rfd >= 0) { + struct stat rst; + fstat(rfd, &rst); + messageqcpp::ByteStream rbs; + rbs.needAtLeast(rst.st_size); + rbs.restart(); + auto r = read(rfd, rbs.getInputPtr(), rst.st_size); + if (r != rst.st_size) + abort(); + rbs.advanceInputPtr(r); + rg.deserialize(rbs); + close(rfd); + } else { + std::vector pos{2, 6, 22, 30, 46, 54}; // ? + std::vector oids{3011, 3011, 3011, 3011, 3011}; // ? + std::vector keys{1, 1, 1, 1, 1}; // ? + std::vector col_t{ + execplan::CalpontSystemCatalog::INT, + execplan::CalpontSystemCatalog::LONGDOUBLE, + execplan::CalpontSystemCatalog::UBIGINT, + execplan::CalpontSystemCatalog::LONGDOUBLE, + execplan::CalpontSystemCatalog::UBIGINT + }; + std::vector csN{8, 8, 8, 8, 8}; + std::vector scale{0, 0, 0, 0, 0}; + std::vector prec{10, 4294967295, 9999, 4294967295, 19}; + rg = rowgroup::RowGroup(5, pos, oids, keys, col_t, csN, scale, prec, 20, false, std::vector{}); + } + + int fd = open(argv[1], O_RDONLY); + struct stat st; + fstat(fd, &st); + + messageqcpp::ByteStream bs; + bs.needAtLeast(st.st_size); + bs.restart(); + auto r = read(fd, bs.getInputPtr(), st.st_size); + if (r != st.st_size) + abort(); + bs.advanceInputPtr(r); + rowgroup::RGData rst; + rst.deserialize(bs); + + rg.setData(&rst); + close(fd); + std::cout << "RowGroup data:\n" << rg.toString() << std::endl; + return 0; +} \ No newline at end of file diff --git a/utils/common/robin_hood.h b/utils/common/robin_hood.h new file mode 100644 index 000000000..5304ca53c --- /dev/null +++ b/utils/common/robin_hood.h @@ -0,0 +1,2454 @@ +// ______ _____ ______ _________ +// ______________ ___ /_ ___(_)_______ ___ /_ ______ ______ ______ / +// __ ___/_ __ \__ __ \__ / __ __ \ __ __ \_ __ \_ __ \_ __ / +// _ / / /_/ /_ /_/ /_ / _ / / / _ / / // /_/ // /_/ // /_/ / +// /_/ \____/ /_.___/ /_/ /_/ /_/ ________/_/ /_/ \____/ \____/ \__,_/ +// _/_____/ +// +// Fast & memory efficient hashtable based on robin hood hashing for C++11/14/17/20 +// https://github.com/martinus/robin-hood-hashing +// +// Licensed under the MIT License . +// SPDX-License-Identifier: MIT +// Copyright (c) 2018-2020 Martin Ankerl +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in all +// copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +// SOFTWARE. + +#ifndef ROBIN_HOOD_H_INCLUDED +#define ROBIN_HOOD_H_INCLUDED + +// see https://semver.org/ +#define ROBIN_HOOD_VERSION_MAJOR 3 // for incompatible API changes +#define ROBIN_HOOD_VERSION_MINOR 10 // for adding functionality in a backwards-compatible manner +#define ROBIN_HOOD_VERSION_PATCH 0 // for backwards-compatible bug fixes + +#include +#include +#include +#include +#include // only to support hash of smart pointers +#include +#include +#include +#include +#if __cplusplus >= 201703L +# include +#endif + +// #define ROBIN_HOOD_LOG_ENABLED +#ifdef ROBIN_HOOD_LOG_ENABLED +# include +# define ROBIN_HOOD_LOG(...) \ + std::cout << __FUNCTION__ << "@" << __LINE__ << ": " << __VA_ARGS__ << std::endl; +#else +# define ROBIN_HOOD_LOG(x) +#endif + +// #define ROBIN_HOOD_TRACE_ENABLED +#ifdef ROBIN_HOOD_TRACE_ENABLED +# include +# define ROBIN_HOOD_TRACE(...) \ + std::cout << __FUNCTION__ << "@" << __LINE__ << ": " << __VA_ARGS__ << std::endl; +#else +# define ROBIN_HOOD_TRACE(x) +#endif + +// #define ROBIN_HOOD_COUNT_ENABLED +#ifdef ROBIN_HOOD_COUNT_ENABLED +# include +# define ROBIN_HOOD_COUNT(x) ++counts().x; +namespace robin_hood { +struct Counts { + uint64_t shiftUp{}; + uint64_t shiftDown{}; +}; +inline std::ostream& operator<<(std::ostream& os, Counts const& c) { + return os << c.shiftUp << " shiftUp" << std::endl << c.shiftDown << " shiftDown" << std::endl; +} + +static Counts& counts() { + static Counts counts{}; + return counts; +} +} // namespace robin_hood +#else +# define ROBIN_HOOD_COUNT(x) +#endif + +// all non-argument macros should use this facility. See +// https://www.fluentcpp.com/2019/05/28/better-macros-better-flags/ +#define ROBIN_HOOD(x) ROBIN_HOOD_PRIVATE_DEFINITION_##x() + +// mark unused members with this macro +#define ROBIN_HOOD_UNUSED(identifier) + +// bitness +#if SIZE_MAX == UINT32_MAX +# define ROBIN_HOOD_PRIVATE_DEFINITION_BITNESS() 32 +#elif SIZE_MAX == UINT64_MAX +# define ROBIN_HOOD_PRIVATE_DEFINITION_BITNESS() 64 +#else +# error Unsupported bitness +#endif + +// endianess +#ifdef _MSC_VER +# define ROBIN_HOOD_PRIVATE_DEFINITION_LITTLE_ENDIAN() 1 +# define ROBIN_HOOD_PRIVATE_DEFINITION_BIG_ENDIAN() 0 +#else +# define ROBIN_HOOD_PRIVATE_DEFINITION_LITTLE_ENDIAN() \ + (__BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__) +# define ROBIN_HOOD_PRIVATE_DEFINITION_BIG_ENDIAN() (__BYTE_ORDER__ == __ORDER_BIG_ENDIAN__) +#endif + +// inline +#ifdef _MSC_VER +# define ROBIN_HOOD_PRIVATE_DEFINITION_NOINLINE() __declspec(noinline) +#else +# define ROBIN_HOOD_PRIVATE_DEFINITION_NOINLINE() __attribute__((noinline)) +#endif + +// exceptions +#if !defined(__cpp_exceptions) && !defined(__EXCEPTIONS) && !defined(_CPPUNWIND) +# define ROBIN_HOOD_PRIVATE_DEFINITION_HAS_EXCEPTIONS() 0 +#else +# define ROBIN_HOOD_PRIVATE_DEFINITION_HAS_EXCEPTIONS() 1 +#endif + +// count leading/trailing bits +#if !defined(ROBIN_HOOD_DISABLE_INTRINSICS) +# ifdef _MSC_VER +# if ROBIN_HOOD(BITNESS) == 32 +# define ROBIN_HOOD_PRIVATE_DEFINITION_BITSCANFORWARD() _BitScanForward +# else +# define ROBIN_HOOD_PRIVATE_DEFINITION_BITSCANFORWARD() _BitScanForward64 +# endif +# include +# pragma intrinsic(ROBIN_HOOD(BITSCANFORWARD)) +# define ROBIN_HOOD_COUNT_TRAILING_ZEROES(x) \ + [](size_t mask) noexcept -> int { \ + unsigned long index; \ + return ROBIN_HOOD(BITSCANFORWARD)(&index, mask) ? static_cast(index) \ + : ROBIN_HOOD(BITNESS); \ + }(x) +# else +# if ROBIN_HOOD(BITNESS) == 32 +# define ROBIN_HOOD_PRIVATE_DEFINITION_CTZ() __builtin_ctzl +# define ROBIN_HOOD_PRIVATE_DEFINITION_CLZ() __builtin_clzl +# else +# define ROBIN_HOOD_PRIVATE_DEFINITION_CTZ() __builtin_ctzll +# define ROBIN_HOOD_PRIVATE_DEFINITION_CLZ() __builtin_clzll +# endif +# define ROBIN_HOOD_COUNT_LEADING_ZEROES(x) ((x) ? ROBIN_HOOD(CLZ)(x) : ROBIN_HOOD(BITNESS)) +# define ROBIN_HOOD_COUNT_TRAILING_ZEROES(x) ((x) ? ROBIN_HOOD(CTZ)(x) : ROBIN_HOOD(BITNESS)) +# endif +#endif + +// fallthrough +#ifndef __has_cpp_attribute // For backwards compatibility +# define __has_cpp_attribute(x) 0 +#endif +#if __has_cpp_attribute(clang::fallthrough) +# define ROBIN_HOOD_PRIVATE_DEFINITION_FALLTHROUGH() [[clang::fallthrough]] +#elif __has_cpp_attribute(gnu::fallthrough) +# define ROBIN_HOOD_PRIVATE_DEFINITION_FALLTHROUGH() [[gnu::fallthrough]] +#else +# define ROBIN_HOOD_PRIVATE_DEFINITION_FALLTHROUGH() +#endif + +// likely/unlikely +#ifdef _MSC_VER +# define ROBIN_HOOD_LIKELY(condition) condition +# define ROBIN_HOOD_UNLIKELY(condition) condition +#else +# define ROBIN_HOOD_LIKELY(condition) __builtin_expect(condition, 1) +# define ROBIN_HOOD_UNLIKELY(condition) __builtin_expect(condition, 0) +#endif + +// detect if native wchar_t type is availiable in MSVC +#ifdef _MSC_VER +# ifdef _NATIVE_WCHAR_T_DEFINED +# define ROBIN_HOOD_PRIVATE_DEFINITION_HAS_NATIVE_WCHART() 1 +# else +# define ROBIN_HOOD_PRIVATE_DEFINITION_HAS_NATIVE_WCHART() 0 +# endif +#else +# define ROBIN_HOOD_PRIVATE_DEFINITION_HAS_NATIVE_WCHART() 1 +#endif + +// workaround missing "is_trivially_copyable" in g++ < 5.0 +// See https://stackoverflow.com/a/31798726/48181 +#if defined(__GNUC__) && __GNUC__ < 5 +# define ROBIN_HOOD_IS_TRIVIALLY_COPYABLE(...) __has_trivial_copy(__VA_ARGS__) +#else +# define ROBIN_HOOD_IS_TRIVIALLY_COPYABLE(...) std::is_trivially_copyable<__VA_ARGS__>::value +#endif + +// helpers for C++ versions, see https://gcc.gnu.org/onlinedocs/cpp/Standard-Predefined-Macros.html +#define ROBIN_HOOD_PRIVATE_DEFINITION_CXX() __cplusplus +#define ROBIN_HOOD_PRIVATE_DEFINITION_CXX98() 199711L +#define ROBIN_HOOD_PRIVATE_DEFINITION_CXX11() 201103L +#define ROBIN_HOOD_PRIVATE_DEFINITION_CXX14() 201402L +#define ROBIN_HOOD_PRIVATE_DEFINITION_CXX17() 201703L + +#if ROBIN_HOOD(CXX) >= ROBIN_HOOD(CXX17) +# define ROBIN_HOOD_PRIVATE_DEFINITION_NODISCARD() [[nodiscard]] +#else +# define ROBIN_HOOD_PRIVATE_DEFINITION_NODISCARD() +#endif + +namespace robin_hood { + +#if ROBIN_HOOD(CXX) >= ROBIN_HOOD(CXX14) +# define ROBIN_HOOD_STD std +#else + +// c++11 compatibility layer +namespace ROBIN_HOOD_STD { +template +struct alignment_of + : std::integral_constant::type)> {}; + +template +class integer_sequence { +public: + using value_type = T; + static_assert(std::is_integral::value, "not integral type"); + static constexpr std::size_t size() noexcept { + return sizeof...(Ints); + } +}; +template +using index_sequence = integer_sequence; + +namespace detail_ { +template +struct IntSeqImpl { + using TValue = T; + static_assert(std::is_integral::value, "not integral type"); + static_assert(Begin >= 0 && Begin < End, "unexpected argument (Begin<0 || Begin<=End)"); + + template + struct IntSeqCombiner; + + template + struct IntSeqCombiner, integer_sequence> { + using TResult = integer_sequence; + }; + + using TResult = + typename IntSeqCombiner::TResult, + typename IntSeqImpl::TResult>::TResult; +}; + +template +struct IntSeqImpl { + using TValue = T; + static_assert(std::is_integral::value, "not integral type"); + static_assert(Begin >= 0, "unexpected argument (Begin<0)"); + using TResult = integer_sequence; +}; + +template +struct IntSeqImpl { + using TValue = T; + static_assert(std::is_integral::value, "not integral type"); + static_assert(Begin >= 0, "unexpected argument (Begin<0)"); + using TResult = integer_sequence; +}; +} // namespace detail_ + +template +using make_integer_sequence = typename detail_::IntSeqImpl::TResult; + +template +using make_index_sequence = make_integer_sequence; + +template +using index_sequence_for = make_index_sequence; + +} // namespace ROBIN_HOOD_STD + +#endif + +namespace detail { + +// make sure we static_cast to the correct type for hash_int +#if ROBIN_HOOD(BITNESS) == 64 +using SizeT = uint64_t; +#else +using SizeT = uint32_t; +#endif + +template +T rotr(T x, unsigned k) { + return (x >> k) | (x << (8U * sizeof(T) - k)); +} + +// This cast gets rid of warnings like "cast from 'uint8_t*' {aka 'unsigned char*'} to +// 'uint64_t*' {aka 'long unsigned int*'} increases required alignment of target type". Use with +// care! +template +inline T reinterpret_cast_no_cast_align_warning(void* ptr) noexcept { + return reinterpret_cast(ptr); +} + +template +inline T reinterpret_cast_no_cast_align_warning(void const* ptr) noexcept { + return reinterpret_cast(ptr); +} + +// make sure this is not inlined as it is slow and dramatically enlarges code, thus making other +// inlinings more difficult. Throws are also generally the slow path. +template +[[noreturn]] ROBIN_HOOD(NOINLINE) +#if ROBIN_HOOD(HAS_EXCEPTIONS) + void doThrow(Args&&... args) { + // NOLINTNEXTLINE(cppcoreguidelines-pro-bounds-array-to-pointer-decay) + throw E(std::forward(args)...); +} +#else + void doThrow(Args&&... ROBIN_HOOD_UNUSED(args) /*unused*/) { + abort(); +} +#endif + +template +T* assertNotNull(T* t, Args&&... args) { + if (ROBIN_HOOD_UNLIKELY(nullptr == t)) { + doThrow(std::forward(args)...); + } + return t; +} + +template +inline T unaligned_load(void const* ptr) noexcept { + // using memcpy so we don't get into unaligned load problems. + // compiler should optimize this very well anyways. + T t; + std::memcpy(&t, ptr, sizeof(T)); + return t; +} + +// Allocates bulks of memory for objects of type T. This deallocates the memory in the destructor, +// and keeps a linked list of the allocated memory around. Overhead per allocation is the size of a +// pointer. +template +class BulkPoolAllocator { +public: + BulkPoolAllocator() noexcept = default; + + // does not copy anything, just creates a new allocator. + BulkPoolAllocator(const BulkPoolAllocator& ROBIN_HOOD_UNUSED(o) /*unused*/) noexcept + : mHead(nullptr) + , mListForFree(nullptr) {} + + BulkPoolAllocator(BulkPoolAllocator&& o) noexcept + : mHead(o.mHead) + , mListForFree(o.mListForFree) { + o.mListForFree = nullptr; + o.mHead = nullptr; + } + + BulkPoolAllocator& operator=(BulkPoolAllocator&& o) noexcept { + reset(); + mHead = o.mHead; + mListForFree = o.mListForFree; + o.mListForFree = nullptr; + o.mHead = nullptr; + return *this; + } + + BulkPoolAllocator& + // NOLINTNEXTLINE(bugprone-unhandled-self-assignment,cert-oop54-cpp) + operator=(const BulkPoolAllocator& ROBIN_HOOD_UNUSED(o) /*unused*/) noexcept { + // does not do anything + return *this; + } + + ~BulkPoolAllocator() noexcept { + reset(); + } + + // Deallocates all allocated memory. + void reset() noexcept { + while (mListForFree) { + T* tmp = *mListForFree; + ROBIN_HOOD_LOG("std::free") + std::free(mListForFree); + mListForFree = reinterpret_cast_no_cast_align_warning(tmp); + } + mHead = nullptr; + } + + // allocates, but does NOT initialize. Use in-place new constructor, e.g. + // T* obj = pool.allocate(); + // ::new (static_cast(obj)) T(); + T* allocate() { + T* tmp = mHead; + if (!tmp) { + tmp = performAllocation(); + } + + mHead = *reinterpret_cast_no_cast_align_warning(tmp); + return tmp; + } + + // does not actually deallocate but puts it in store. + // make sure you have already called the destructor! e.g. with + // obj->~T(); + // pool.deallocate(obj); + void deallocate(T* obj) noexcept { + *reinterpret_cast_no_cast_align_warning(obj) = mHead; + mHead = obj; + } + + // Adds an already allocated block of memory to the allocator. This allocator is from now on + // responsible for freeing the data (with free()). If the provided data is not large enough to + // make use of, it is immediately freed. Otherwise it is reused and freed in the destructor. + void addOrFree(void* ptr, const size_t numBytes) noexcept { + // calculate number of available elements in ptr + if (numBytes < ALIGNMENT + ALIGNED_SIZE) { + // not enough data for at least one element. Free and return. + ROBIN_HOOD_LOG("std::free") + std::free(ptr); + } else { + ROBIN_HOOD_LOG("add to buffer") + add(ptr, numBytes); + } + } + + void swap(BulkPoolAllocator& other) noexcept { + using std::swap; + swap(mHead, other.mHead); + swap(mListForFree, other.mListForFree); + } + +private: + // iterates the list of allocated memory to calculate how many to alloc next. + // Recalculating this each time saves us a size_t member. + // This ignores the fact that memory blocks might have been added manually with addOrFree. In + // practice, this should not matter much. + ROBIN_HOOD(NODISCARD) size_t calcNumElementsToAlloc() const noexcept { + auto tmp = mListForFree; + size_t numAllocs = MinNumAllocs; + + while (numAllocs * 2 <= MaxNumAllocs && tmp) { + auto x = reinterpret_cast(tmp); + tmp = *x; + numAllocs *= 2; + } + + return numAllocs; + } + + // WARNING: Underflow if numBytes < ALIGNMENT! This is guarded in addOrFree(). + void add(void* ptr, const size_t numBytes) noexcept { + const size_t numElements = (numBytes - ALIGNMENT) / ALIGNED_SIZE; + + auto data = reinterpret_cast(ptr); + + // link free list + auto x = reinterpret_cast(data); + *x = mListForFree; + mListForFree = data; + + // create linked list for newly allocated data + auto* const headT = + reinterpret_cast_no_cast_align_warning(reinterpret_cast(ptr) + ALIGNMENT); + + auto* const head = reinterpret_cast(headT); + + // Visual Studio compiler automatically unrolls this loop, which is pretty cool + for (size_t i = 0; i < numElements; ++i) { + *reinterpret_cast_no_cast_align_warning(head + i * ALIGNED_SIZE) = + head + (i + 1) * ALIGNED_SIZE; + } + + // last one points to 0 + *reinterpret_cast_no_cast_align_warning(head + (numElements - 1) * ALIGNED_SIZE) = + mHead; + mHead = headT; + } + + // Called when no memory is available (mHead == 0). + // Don't inline this slow path. + ROBIN_HOOD(NOINLINE) T* performAllocation() { + size_t const numElementsToAlloc = calcNumElementsToAlloc(); + + // alloc new memory: [prev |T, T, ... T] + size_t const bytes = ALIGNMENT + ALIGNED_SIZE * numElementsToAlloc; + ROBIN_HOOD_LOG("std::malloc " << bytes << " = " << ALIGNMENT << " + " << ALIGNED_SIZE + << " * " << numElementsToAlloc) + add(assertNotNull(std::malloc(bytes)), bytes); + return mHead; + } + + // enforce byte alignment of the T's +#if ROBIN_HOOD(CXX) >= ROBIN_HOOD(CXX14) + static constexpr size_t ALIGNMENT = + (std::max)(std::alignment_of::value, std::alignment_of::value); +#else + static const size_t ALIGNMENT = + (ROBIN_HOOD_STD::alignment_of::value > ROBIN_HOOD_STD::alignment_of::value) + ? ROBIN_HOOD_STD::alignment_of::value + : +ROBIN_HOOD_STD::alignment_of::value; // the + is for walkarround +#endif + + static constexpr size_t ALIGNED_SIZE = ((sizeof(T) - 1) / ALIGNMENT + 1) * ALIGNMENT; + + static_assert(MinNumAllocs >= 1, "MinNumAllocs"); + static_assert(MaxNumAllocs >= MinNumAllocs, "MaxNumAllocs"); + static_assert(ALIGNED_SIZE >= sizeof(T*), "ALIGNED_SIZE"); + static_assert(0 == (ALIGNED_SIZE % sizeof(T*)), "ALIGNED_SIZE mod"); + static_assert(ALIGNMENT >= sizeof(T*), "ALIGNMENT"); + + T* mHead{nullptr}; + T** mListForFree{nullptr}; +}; + +template +struct NodeAllocator; + +// dummy allocator that does nothing +template +struct NodeAllocator { + + // we are not using the data, so just free it. + void addOrFree(void* ptr, size_t ROBIN_HOOD_UNUSED(numBytes) /*unused*/) noexcept { + ROBIN_HOOD_LOG("std::free") + std::free(ptr); + } +}; + +template +struct NodeAllocator : public BulkPoolAllocator {}; + +// dummy hash, unsed as mixer when robin_hood::hash is already used +template +struct identity_hash { + constexpr size_t operator()(T const& obj) const noexcept { + return static_cast(obj); + } +}; + +// c++14 doesn't have is_nothrow_swappable, and clang++ 6.0.1 doesn't like it either, so I'm making +// my own here. +namespace swappable { +#if ROBIN_HOOD(CXX) < ROBIN_HOOD(CXX17) +using std::swap; +template +struct nothrow { + static const bool value = noexcept(swap(std::declval(), std::declval())); +}; +#else +template +struct nothrow { + static const bool value = std::is_nothrow_swappable::value; +}; +#endif +} // namespace swappable + +} // namespace detail + +struct is_transparent_tag {}; + +// A custom pair implementation is used in the map because std::pair is not is_trivially_copyable, +// which means it would not be allowed to be used in std::memcpy. This struct is copyable, which is +// also tested. +template +struct pair { + using first_type = T1; + using second_type = T2; + + template ::value && + std::is_default_constructible::value>::type> + constexpr pair() noexcept(noexcept(U1()) && noexcept(U2())) + : first() + , second() {} + + // pair constructors are explicit so we don't accidentally call this ctor when we don't have to. + explicit constexpr pair(std::pair const& o) noexcept( + noexcept(T1(std::declval())) && noexcept(T2(std::declval()))) + : first(o.first) + , second(o.second) {} + + // pair constructors are explicit so we don't accidentally call this ctor when we don't have to. + explicit constexpr pair(std::pair&& o) noexcept(noexcept( + T1(std::move(std::declval()))) && noexcept(T2(std::move(std::declval())))) + : first(std::move(o.first)) + , second(std::move(o.second)) {} + + constexpr pair(T1&& a, T2&& b) noexcept(noexcept( + T1(std::move(std::declval()))) && noexcept(T2(std::move(std::declval())))) + : first(std::move(a)) + , second(std::move(b)) {} + + template + constexpr pair(U1&& a, U2&& b) noexcept(noexcept(T1(std::forward( + std::declval()))) && noexcept(T2(std::forward(std::declval())))) + : first(std::forward(a)) + , second(std::forward(b)) {} + + template + constexpr pair( + std::piecewise_construct_t /*unused*/, std::tuple a, + std::tuple b) noexcept(noexcept(pair(std::declval&>(), + std::declval&>(), + ROBIN_HOOD_STD::index_sequence_for(), + ROBIN_HOOD_STD::index_sequence_for()))) + : pair(a, b, ROBIN_HOOD_STD::index_sequence_for(), + ROBIN_HOOD_STD::index_sequence_for()) {} + + // constructor called from the std::piecewise_construct_t ctor + template + pair(std::tuple& a, std::tuple& b, ROBIN_HOOD_STD::index_sequence /*unused*/, ROBIN_HOOD_STD::index_sequence /*unused*/) noexcept( + noexcept(T1(std::forward(std::get( + std::declval&>()))...)) && noexcept(T2(std:: + forward(std::get( + std::declval&>()))...))) + : first(std::forward(std::get(a))...) + , second(std::forward(std::get(b))...) { + // make visual studio compiler happy about warning about unused a & b. + // Visual studio's pair implementation disables warning 4100. + (void)a; + (void)b; + } + + void swap(pair& o) noexcept((detail::swappable::nothrow::value) && + (detail::swappable::nothrow::value)) { + using std::swap; + swap(first, o.first); + swap(second, o.second); + } + + T1 first; // NOLINT(misc-non-private-member-variables-in-classes) + T2 second; // NOLINT(misc-non-private-member-variables-in-classes) +}; + +template +inline void swap(pair& a, pair& b) noexcept( + noexcept(std::declval&>().swap(std::declval&>()))) { + a.swap(b); +} + +template +inline constexpr bool operator==(pair const& x, pair const& y) { + return (x.first == y.first) && (x.second == y.second); +} +template +inline constexpr bool operator!=(pair const& x, pair const& y) { + return !(x == y); +} +template +inline constexpr bool operator<(pair const& x, pair const& y) noexcept(noexcept( + std::declval() < std::declval()) && noexcept(std::declval() < + std::declval())) { + return x.first < y.first || (!(y.first < x.first) && x.second < y.second); +} +template +inline constexpr bool operator>(pair const& x, pair const& y) { + return y < x; +} +template +inline constexpr bool operator<=(pair const& x, pair const& y) { + return !(x > y); +} +template +inline constexpr bool operator>=(pair const& x, pair const& y) { + return !(x < y); +} + +inline size_t hash_bytes(void const* ptr, size_t len) noexcept { + static constexpr uint64_t m = UINT64_C(0xc6a4a7935bd1e995); + static constexpr uint64_t seed = UINT64_C(0xe17a1465); + static constexpr unsigned int r = 47; + + auto const* const data64 = static_cast(ptr); + uint64_t h = seed ^ (len * m); + + size_t const n_blocks = len / 8; + for (size_t i = 0; i < n_blocks; ++i) { + auto k = detail::unaligned_load(data64 + i); + + k *= m; + k ^= k >> r; + k *= m; + + h ^= k; + h *= m; + } + + auto const* const data8 = reinterpret_cast(data64 + n_blocks); + switch (len & 7U) { + case 7: + h ^= static_cast(data8[6]) << 48U; + ROBIN_HOOD(FALLTHROUGH); // FALLTHROUGH + case 6: + h ^= static_cast(data8[5]) << 40U; + ROBIN_HOOD(FALLTHROUGH); // FALLTHROUGH + case 5: + h ^= static_cast(data8[4]) << 32U; + ROBIN_HOOD(FALLTHROUGH); // FALLTHROUGH + case 4: + h ^= static_cast(data8[3]) << 24U; + ROBIN_HOOD(FALLTHROUGH); // FALLTHROUGH + case 3: + h ^= static_cast(data8[2]) << 16U; + ROBIN_HOOD(FALLTHROUGH); // FALLTHROUGH + case 2: + h ^= static_cast(data8[1]) << 8U; + ROBIN_HOOD(FALLTHROUGH); // FALLTHROUGH + case 1: + h ^= static_cast(data8[0]); + h *= m; + ROBIN_HOOD(FALLTHROUGH); // FALLTHROUGH + default: + break; + } + + h ^= h >> r; + h *= m; + h ^= h >> r; + return static_cast(h); +} + +inline size_t hash_int(uint64_t x) noexcept { + // inspired by lemire's strongly universal hashing + // https://lemire.me/blog/2018/08/15/fast-strongly-universal-64-bit-hashing-everywhere/ + // + // Instead of shifts, we use rotations so we don't lose any bits. + // + // Added a final multiplcation with a constant for more mixing. It is most important that + // the lower bits are well mixed. + auto h1 = x * UINT64_C(0xA24BAED4963EE407); + auto h2 = detail::rotr(x, 32U) * UINT64_C(0x9FB21C651E98DF25); + auto h = detail::rotr(h1 + h2, 32U); + return static_cast(h); +} + +// A thin wrapper around std::hash, performing an additional simple mixing step of the result. +template +struct hash : public std::hash { + size_t operator()(T const& obj) const + noexcept(noexcept(std::declval>().operator()(std::declval()))) { + // call base hash + auto result = std::hash::operator()(obj); + // return mixed of that, to be save against identity has + return hash_int(static_cast(result)); + } +}; + +template +struct hash> { + size_t operator()(std::basic_string const& str) const noexcept { + return hash_bytes(str.data(), sizeof(CharT) * str.size()); + } +}; + +#if ROBIN_HOOD(CXX) >= ROBIN_HOOD(CXX17) +template +struct hash> { + size_t operator()(std::basic_string_view const& sv) const noexcept { + return hash_bytes(sv.data(), sizeof(CharT) * sv.size()); + } +}; +#endif + +template +struct hash { + size_t operator()(T* ptr) const noexcept { + return hash_int(reinterpret_cast(ptr)); + } +}; + +template +struct hash> { + size_t operator()(std::unique_ptr const& ptr) const noexcept { + return hash_int(reinterpret_cast(ptr.get())); + } +}; + +template +struct hash> { + size_t operator()(std::shared_ptr const& ptr) const noexcept { + return hash_int(reinterpret_cast(ptr.get())); + } +}; + +template +struct hash::value>::type> { + size_t operator()(Enum e) const noexcept { + using Underlying = typename std::underlying_type::type; + return hash{}(static_cast(e)); + } +}; + +#define ROBIN_HOOD_HASH_INT(T) \ + template <> \ + struct hash { \ + size_t operator()(T const& obj) const noexcept { \ + return hash_int(static_cast(obj)); \ + } \ + } + +#if defined(__GNUC__) && !defined(__clang__) +# pragma GCC diagnostic push +# pragma GCC diagnostic ignored "-Wuseless-cast" +#endif +// see https://en.cppreference.com/w/cpp/utility/hash +ROBIN_HOOD_HASH_INT(bool); +ROBIN_HOOD_HASH_INT(char); +ROBIN_HOOD_HASH_INT(signed char); +ROBIN_HOOD_HASH_INT(unsigned char); +ROBIN_HOOD_HASH_INT(char16_t); +ROBIN_HOOD_HASH_INT(char32_t); +#if ROBIN_HOOD(HAS_NATIVE_WCHART) +ROBIN_HOOD_HASH_INT(wchar_t); +#endif +ROBIN_HOOD_HASH_INT(short); +ROBIN_HOOD_HASH_INT(unsigned short); +ROBIN_HOOD_HASH_INT(int); +ROBIN_HOOD_HASH_INT(unsigned int); +ROBIN_HOOD_HASH_INT(long); +ROBIN_HOOD_HASH_INT(long long); +ROBIN_HOOD_HASH_INT(unsigned long); +ROBIN_HOOD_HASH_INT(unsigned long long); +#if defined(__GNUC__) && !defined(__clang__) +# pragma GCC diagnostic pop +#endif +namespace detail { + +template +struct void_type { + using type = void; +}; + +template +struct has_is_transparent : public std::false_type {}; + +template +struct has_is_transparent::type> + : public std::true_type {}; + +// using wrapper classes for hash and key_equal prevents the diamond problem when the same type +// is used. see https://stackoverflow.com/a/28771920/48181 +template +struct WrapHash : public T { + WrapHash() = default; + explicit WrapHash(T const& o) noexcept(noexcept(T(std::declval()))) + : T(o) {} +}; + +template +struct WrapKeyEqual : public T { + WrapKeyEqual() = default; + explicit WrapKeyEqual(T const& o) noexcept(noexcept(T(std::declval()))) + : T(o) {} +}; + +// A highly optimized hashmap implementation, using the Robin Hood algorithm. +// +// In most cases, this map should be usable as a drop-in replacement for std::unordered_map, but +// be about 2x faster in most cases and require much less allocations. +// +// This implementation uses the following memory layout: +// +// [Node, Node, ... Node | info, info, ... infoSentinel ] +// +// * Node: either a DataNode that directly has the std::pair as member, +// or a DataNode with a pointer to std::pair. Which DataNode representation to use +// depends on how fast the swap() operation is. Heuristically, this is automatically choosen +// based on sizeof(). there are always 2^n Nodes. +// +// * info: Each Node in the map has a corresponding info byte, so there are 2^n info bytes. +// Each byte is initialized to 0, meaning the corresponding Node is empty. Set to 1 means the +// corresponding node contains data. Set to 2 means the corresponding Node is filled, but it +// actually belongs to the previous position and was pushed out because that place is already +// taken. +// +// * infoSentinel: Sentinel byte set to 1, so that iterator's ++ can stop at end() without the +// need for a idx variable. +// +// According to STL, order of templates has effect on throughput. That's why I've moved the +// boolean to the front. +// https://www.reddit.com/r/cpp/comments/ahp6iu/compile_time_binary_size_reductions_and_cs_future/eeguck4/ +template +class Table + : public WrapHash, + public WrapKeyEqual, + detail::NodeAllocator< + typename std::conditional< + std::is_void::value, Key, + robin_hood::pair::type, T>>::type, + 4, 16384, IsFlat> { +public: + static constexpr bool is_flat = IsFlat; + static constexpr bool is_map = !std::is_void::value; + static constexpr bool is_set = !is_map; + static constexpr bool is_transparent = + has_is_transparent::value && has_is_transparent::value; + + using key_type = Key; + using mapped_type = T; + using value_type = typename std::conditional< + is_set, Key, + robin_hood::pair::type, T>>::type; + using size_type = size_t; + using hasher = Hash; + using key_equal = KeyEqual; + using Self = Table; + +private: + static_assert(MaxLoadFactor100 > 10 && MaxLoadFactor100 < 100, + "MaxLoadFactor100 needs to be >10 && < 100"); + + using WHash = WrapHash; + using WKeyEqual = WrapKeyEqual; + + // configuration defaults + + // make sure we have 8 elements, needed to quickly rehash mInfo + static constexpr size_t InitialNumElements = sizeof(uint64_t); + static constexpr uint32_t InitialInfoNumBits = 5; + static constexpr uint8_t InitialInfoInc = 1U << InitialInfoNumBits; + static constexpr size_t InfoMask = InitialInfoInc - 1U; + static constexpr uint8_t InitialInfoHashShift = 0; + using DataPool = detail::NodeAllocator; + + // type needs to be wider than uint8_t. + using InfoType = uint32_t; + + // DataNode //////////////////////////////////////////////////////// + + // Primary template for the data node. We have special implementations for small and big + // objects. For large objects it is assumed that swap() is fairly slow, so we allocate these + // on the heap so swap merely swaps a pointer. + template + class DataNode {}; + + // Small: just allocate on the stack. + template + class DataNode final { + public: + template + explicit DataNode(M& ROBIN_HOOD_UNUSED(map) /*unused*/, Args&&... args) noexcept( + noexcept(value_type(std::forward(args)...))) + : mData(std::forward(args)...) {} + + DataNode(M& ROBIN_HOOD_UNUSED(map) /*unused*/, DataNode&& n) noexcept( + std::is_nothrow_move_constructible::value) + : mData(std::move(n.mData)) {} + + // doesn't do anything + void destroy(M& ROBIN_HOOD_UNUSED(map) /*unused*/) noexcept {} + void destroyDoNotDeallocate() noexcept {} + + value_type const* operator->() const noexcept { + return &mData; + } + value_type* operator->() noexcept { + return &mData; + } + + const value_type& operator*() const noexcept { + return mData; + } + + value_type& operator*() noexcept { + return mData; + } + + template + ROBIN_HOOD(NODISCARD) + typename std::enable_if::type getFirst() noexcept { + return mData.first; + } + template + ROBIN_HOOD(NODISCARD) + typename std::enable_if::type getFirst() noexcept { + return mData; + } + + template + ROBIN_HOOD(NODISCARD) + typename std::enable_if::type + getFirst() const noexcept { + return mData.first; + } + template + ROBIN_HOOD(NODISCARD) + typename std::enable_if::type getFirst() const noexcept { + return mData; + } + + template + ROBIN_HOOD(NODISCARD) + typename std::enable_if::type getSecond() noexcept { + return mData.second; + } + + template + ROBIN_HOOD(NODISCARD) + typename std::enable_if::type getSecond() const noexcept { + return mData.second; + } + + void swap(DataNode& o) noexcept( + noexcept(std::declval().swap(std::declval()))) { + mData.swap(o.mData); + } + + private: + value_type mData; + }; + + // big object: allocate on heap. + template + class DataNode { + public: + template + explicit DataNode(M& map, Args&&... args) + : mData(map.allocate()) { + ::new (static_cast(mData)) value_type(std::forward(args)...); + } + + DataNode(M& ROBIN_HOOD_UNUSED(map) /*unused*/, DataNode&& n) noexcept + : mData(std::move(n.mData)) {} + + void destroy(M& map) noexcept { + // don't deallocate, just put it into list of datapool. + mData->~value_type(); + map.deallocate(mData); + } + + void destroyDoNotDeallocate() noexcept { + mData->~value_type(); + } + + value_type const* operator->() const noexcept { + return mData; + } + + value_type* operator->() noexcept { + return mData; + } + + const value_type& operator*() const { + return *mData; + } + + value_type& operator*() { + return *mData; + } + + template + ROBIN_HOOD(NODISCARD) + typename std::enable_if::type getFirst() noexcept { + return mData->first; + } + template + ROBIN_HOOD(NODISCARD) + typename std::enable_if::type getFirst() noexcept { + return *mData; + } + + template + ROBIN_HOOD(NODISCARD) + typename std::enable_if::type + getFirst() const noexcept { + return mData->first; + } + template + ROBIN_HOOD(NODISCARD) + typename std::enable_if::type getFirst() const noexcept { + return *mData; + } + + template + ROBIN_HOOD(NODISCARD) + typename std::enable_if::type getSecond() noexcept { + return mData->second; + } + + template + ROBIN_HOOD(NODISCARD) + typename std::enable_if::type getSecond() const noexcept { + return mData->second; + } + + void swap(DataNode& o) noexcept { + using std::swap; + swap(mData, o.mData); + } + + private: + value_type* mData; + }; + + using Node = DataNode; + + // helpers for doInsert: extract first entry (only const required) + ROBIN_HOOD(NODISCARD) key_type const& getFirstConst(Node const& n) const noexcept { + return n.getFirst(); + } + + // in case we have void mapped_type, we are not using a pair, thus we just route k through. + // No need to disable this because it's just not used if not applicable. + ROBIN_HOOD(NODISCARD) key_type const& getFirstConst(key_type const& k) const noexcept { + return k; + } + + // in case we have non-void mapped_type, we have a standard robin_hood::pair + template + ROBIN_HOOD(NODISCARD) + typename std::enable_if::value, key_type const&>::type + getFirstConst(value_type const& vt) const noexcept { + return vt.first; + } + + // Cloner ////////////////////////////////////////////////////////// + + template + struct Cloner; + + // fast path: Just copy data, without allocating anything. + template + struct Cloner { + void operator()(M const& source, M& target) const { + auto const* const src = reinterpret_cast(source.mKeyVals); + auto* tgt = reinterpret_cast(target.mKeyVals); + auto const numElementsWithBuffer = target.calcNumElementsWithBuffer(target.mMask + 1); + std::copy(src, src + target.calcNumBytesTotal(numElementsWithBuffer), tgt); + } + }; + + template + struct Cloner { + void operator()(M const& s, M& t) const { + auto const numElementsWithBuffer = t.calcNumElementsWithBuffer(t.mMask + 1); + std::copy(s.mInfo, s.mInfo + t.calcNumBytesInfo(numElementsWithBuffer), t.mInfo); + + for (size_t i = 0; i < numElementsWithBuffer; ++i) { + if (t.mInfo[i]) { + ::new (static_cast(t.mKeyVals + i)) Node(t, *s.mKeyVals[i]); + } + } + } + }; + + // Destroyer /////////////////////////////////////////////////////// + + template + struct Destroyer {}; + + template + struct Destroyer { + void nodes(M& m) const noexcept { + m.mNumElements = 0; + } + + void nodesDoNotDeallocate(M& m) const noexcept { + m.mNumElements = 0; + } + }; + + template + struct Destroyer { + void nodes(M& m) const noexcept { + m.mNumElements = 0; + // clear also resets mInfo to 0, that's sometimes not necessary. + auto const numElementsWithBuffer = m.calcNumElementsWithBuffer(m.mMask + 1); + + for (size_t idx = 0; idx < numElementsWithBuffer; ++idx) { + if (0 != m.mInfo[idx]) { + Node& n = m.mKeyVals[idx]; + n.destroy(m); + n.~Node(); + } + } + } + + void nodesDoNotDeallocate(M& m) const noexcept { + m.mNumElements = 0; + // clear also resets mInfo to 0, that's sometimes not necessary. + auto const numElementsWithBuffer = m.calcNumElementsWithBuffer(m.mMask + 1); + for (size_t idx = 0; idx < numElementsWithBuffer; ++idx) { + if (0 != m.mInfo[idx]) { + Node& n = m.mKeyVals[idx]; + n.destroyDoNotDeallocate(); + n.~Node(); + } + } + } + }; + + // Iter //////////////////////////////////////////////////////////// + + struct fast_forward_tag {}; + + // generic iterator for both const_iterator and iterator. + template + // NOLINTNEXTLINE(hicpp-special-member-functions,cppcoreguidelines-special-member-functions) + class Iter { + private: + using NodePtr = typename std::conditional::type; + + public: + using difference_type = std::ptrdiff_t; + using value_type = typename Self::value_type; + using reference = typename std::conditional::type; + using pointer = typename std::conditional::type; + using iterator_category = std::forward_iterator_tag; + + // default constructed iterator can be compared to itself, but WON'T return true when + // compared to end(). + Iter() = default; + + // Rule of zero: nothing specified. The conversion constructor is only enabled for + // iterator to const_iterator, so it doesn't accidentally work as a copy ctor. + + // Conversion constructor from iterator to const_iterator. + template ::type> + // NOLINTNEXTLINE(hicpp-explicit-conversions) + Iter(Iter const& other) noexcept + : mKeyVals(other.mKeyVals) + , mInfo(other.mInfo) {} + + Iter(NodePtr valPtr, uint8_t const* infoPtr) noexcept + : mKeyVals(valPtr) + , mInfo(infoPtr) {} + + Iter(NodePtr valPtr, uint8_t const* infoPtr, + fast_forward_tag ROBIN_HOOD_UNUSED(tag) /*unused*/) noexcept + : mKeyVals(valPtr) + , mInfo(infoPtr) { + fastForward(); + } + + template ::type> + Iter& operator=(Iter const& other) noexcept { + mKeyVals = other.mKeyVals; + mInfo = other.mInfo; + return *this; + } + + // prefix increment. Undefined behavior if we are at end()! + Iter& operator++() noexcept { + mInfo++; + mKeyVals++; + fastForward(); + return *this; + } + + Iter operator++(int) noexcept { + Iter tmp = *this; + ++(*this); + return tmp; + } + + reference operator*() const { + return **mKeyVals; + } + + pointer operator->() const { + return &**mKeyVals; + } + + template + bool operator==(Iter const& o) const noexcept { + return mKeyVals == o.mKeyVals; + } + + template + bool operator!=(Iter const& o) const noexcept { + return mKeyVals != o.mKeyVals; + } + + private: + // fast forward to the next non-free info byte + // I've tried a few variants that don't depend on intrinsics, but unfortunately they are + // quite a bit slower than this one. So I've reverted that change again. See map_benchmark. + void fastForward() noexcept { + size_t n = 0; + while (0U == (n = detail::unaligned_load(mInfo))) { + mInfo += sizeof(size_t); + mKeyVals += sizeof(size_t); + } +#if defined(ROBIN_HOOD_DISABLE_INTRINSICS) + // we know for certain that within the next 8 bytes we'll find a non-zero one. + if (ROBIN_HOOD_UNLIKELY(0U == detail::unaligned_load(mInfo))) { + mInfo += 4; + mKeyVals += 4; + } + if (ROBIN_HOOD_UNLIKELY(0U == detail::unaligned_load(mInfo))) { + mInfo += 2; + mKeyVals += 2; + } + if (ROBIN_HOOD_UNLIKELY(0U == *mInfo)) { + mInfo += 1; + mKeyVals += 1; + } +#else +# if ROBIN_HOOD(LITTLE_ENDIAN) + auto inc = ROBIN_HOOD_COUNT_TRAILING_ZEROES(n) / 8; +# else + auto inc = ROBIN_HOOD_COUNT_LEADING_ZEROES(n) / 8; +# endif + mInfo += inc; + mKeyVals += inc; +#endif + } + + friend class Table; + NodePtr mKeyVals{nullptr}; + uint8_t const* mInfo{nullptr}; + }; + + //////////////////////////////////////////////////////////////////// + + // highly performance relevant code. + // Lower bits are used for indexing into the array (2^n size) + // The upper 1-5 bits need to be a reasonable good hash, to save comparisons. + template + void keyToIdx(HashKey&& key, size_t* idx, InfoType* info) const { + // for a user-specified hash that is *not* robin_hood::hash, apply robin_hood::hash as + // an additional mixing step. This serves as a bad hash prevention, if the given data is + // badly mixed. + using Mix = + typename std::conditional, hasher>::value, + ::robin_hood::detail::identity_hash, + ::robin_hood::hash>::type; + + // the lower InitialInfoNumBits are reserved for info. + auto h = Mix{}(WHash::operator()(key)); + *info = mInfoInc + static_cast((h & InfoMask) >> mInfoHashShift); + *idx = (h >> InitialInfoNumBits) & mMask; + } + + // forwards the index by one, wrapping around at the end + void next(InfoType* info, size_t* idx) const noexcept { + *idx = *idx + 1; + *info += mInfoInc; + } + + void nextWhileLess(InfoType* info, size_t* idx) const noexcept { + // unrolling this by hand did not bring any speedups. + while (*info < mInfo[*idx]) { + next(info, idx); + } + } + + // Shift everything up by one element. Tries to move stuff around. + void + shiftUp(size_t startIdx, + size_t const insertion_idx) noexcept(std::is_nothrow_move_assignable::value) { + auto idx = startIdx; + ::new (static_cast(mKeyVals + idx)) Node(std::move(mKeyVals[idx - 1])); + while (--idx != insertion_idx) { + mKeyVals[idx] = std::move(mKeyVals[idx - 1]); + } + + idx = startIdx; + while (idx != insertion_idx) { + ROBIN_HOOD_COUNT(shiftUp) + mInfo[idx] = static_cast(mInfo[idx - 1] + mInfoInc); + if (ROBIN_HOOD_UNLIKELY(mInfo[idx] + mInfoInc > 0xFF)) { + mMaxNumElementsAllowed = 0; + } + --idx; + } + } + + void shiftDown(size_t idx) noexcept(std::is_nothrow_move_assignable::value) { + // until we find one that is either empty or has zero offset. + // TODO(martinus) we don't need to move everything, just the last one for the same + // bucket. + mKeyVals[idx].destroy(*this); + + // until we find one that is either empty or has zero offset. + while (mInfo[idx + 1] >= 2 * mInfoInc) { + ROBIN_HOOD_COUNT(shiftDown) + mInfo[idx] = static_cast(mInfo[idx + 1] - mInfoInc); + mKeyVals[idx] = std::move(mKeyVals[idx + 1]); + ++idx; + } + + mInfo[idx] = 0; + // don't destroy, we've moved it + // mKeyVals[idx].destroy(*this); + mKeyVals[idx].~Node(); + } + + // copy of find(), except that it returns iterator instead of const_iterator. + template + ROBIN_HOOD(NODISCARD) + size_t findIdx(Other const& key) const { + size_t idx{}; + InfoType info{}; + keyToIdx(key, &idx, &info); + + do { + // unrolling this twice gives a bit of a speedup. More unrolling did not help. + if (info == mInfo[idx] && + ROBIN_HOOD_LIKELY(WKeyEqual::operator()(key, mKeyVals[idx].getFirst()))) { + return idx; + } + next(&info, &idx); + if (info == mInfo[idx] && + ROBIN_HOOD_LIKELY(WKeyEqual::operator()(key, mKeyVals[idx].getFirst()))) { + return idx; + } + next(&info, &idx); + } while (info <= mInfo[idx]); + + // nothing found! + return mMask == 0 ? 0 + : static_cast(std::distance( + mKeyVals, reinterpret_cast_no_cast_align_warning(mInfo))); + } + + void cloneData(const Table& o) { + Cloner()(o, *this); + } + + // inserts a keyval that is guaranteed to be new, e.g. when the hashmap is resized. + // @return index where the element was created + size_t insert_move(Node&& keyval) { + // we don't retry, fail if overflowing + // don't need to check max num elements + if (0 == mMaxNumElementsAllowed && !try_increase_info()) { + throwOverflowError(); // impossible to reach LCOV_EXCL_LINE + } + + size_t idx{}; + InfoType info{}; + keyToIdx(keyval.getFirst(), &idx, &info); + + // skip forward. Use <= because we are certain that the element is not there. + while (info <= mInfo[idx]) { + idx = idx + 1; + info += mInfoInc; + } + + // key not found, so we are now exactly where we want to insert it. + auto const insertion_idx = idx; + auto const insertion_info = static_cast(info); + if (ROBIN_HOOD_UNLIKELY(insertion_info + mInfoInc > 0xFF)) { + mMaxNumElementsAllowed = 0; + } + + // find an empty spot + while (0 != mInfo[idx]) { + next(&info, &idx); + } + + auto& l = mKeyVals[insertion_idx]; + if (idx == insertion_idx) { + ::new (static_cast(&l)) Node(std::move(keyval)); + } else { + shiftUp(idx, insertion_idx); + l = std::move(keyval); + } + + // put at empty spot + mInfo[insertion_idx] = insertion_info; + + ++mNumElements; + return insertion_idx; + } + +public: + using iterator = Iter; + using const_iterator = Iter; + + Table() noexcept(noexcept(Hash()) && noexcept(KeyEqual())) + : WHash() + , WKeyEqual() { + ROBIN_HOOD_TRACE(this) + } + + // Creates an empty hash map. Nothing is allocated yet, this happens at the first insert. + // This tremendously speeds up ctor & dtor of a map that never receives an element. The + // penalty is payed at the first insert, and not before. Lookup of this empty map works + // because everybody points to DummyInfoByte::b. parameter bucket_count is dictated by the + // standard, but we can ignore it. + explicit Table( + size_t ROBIN_HOOD_UNUSED(bucket_count) /*unused*/, const Hash& h = Hash{}, + const KeyEqual& equal = KeyEqual{}) noexcept(noexcept(Hash(h)) && noexcept(KeyEqual(equal))) + : WHash(h) + , WKeyEqual(equal) { + ROBIN_HOOD_TRACE(this) + } + + template + Table(Iter first, Iter last, size_t ROBIN_HOOD_UNUSED(bucket_count) /*unused*/ = 0, + const Hash& h = Hash{}, const KeyEqual& equal = KeyEqual{}) + : WHash(h) + , WKeyEqual(equal) { + ROBIN_HOOD_TRACE(this) + insert(first, last); + } + + Table(std::initializer_list initlist, + size_t ROBIN_HOOD_UNUSED(bucket_count) /*unused*/ = 0, const Hash& h = Hash{}, + const KeyEqual& equal = KeyEqual{}) + : WHash(h) + , WKeyEqual(equal) { + ROBIN_HOOD_TRACE(this) + insert(initlist.begin(), initlist.end()); + } + + Table(Table&& o) noexcept + : WHash(std::move(static_cast(o))) + , WKeyEqual(std::move(static_cast(o))) + , DataPool(std::move(static_cast(o))) { + ROBIN_HOOD_TRACE(this) + if (o.mMask) { + mKeyVals = std::move(o.mKeyVals); + mInfo = std::move(o.mInfo); + mNumElements = std::move(o.mNumElements); + mMask = std::move(o.mMask); + mMaxNumElementsAllowed = std::move(o.mMaxNumElementsAllowed); + mInfoInc = std::move(o.mInfoInc); + mInfoHashShift = std::move(o.mInfoHashShift); + // set other's mask to 0 so its destructor won't do anything + o.init(); + } + } + + Table& operator=(Table&& o) noexcept { + ROBIN_HOOD_TRACE(this) + if (&o != this) { + if (o.mMask) { + // only move stuff if the other map actually has some data + destroy(); + mKeyVals = std::move(o.mKeyVals); + mInfo = std::move(o.mInfo); + mNumElements = std::move(o.mNumElements); + mMask = std::move(o.mMask); + mMaxNumElementsAllowed = std::move(o.mMaxNumElementsAllowed); + mInfoInc = std::move(o.mInfoInc); + mInfoHashShift = std::move(o.mInfoHashShift); + WHash::operator=(std::move(static_cast(o))); + WKeyEqual::operator=(std::move(static_cast(o))); + DataPool::operator=(std::move(static_cast(o))); + + o.init(); + + } else { + // nothing in the other map => just clear us. + clear(); + } + } + return *this; + } + + Table(const Table& o) + : WHash(static_cast(o)) + , WKeyEqual(static_cast(o)) + , DataPool(static_cast(o)) { + ROBIN_HOOD_TRACE(this) + if (!o.empty()) { + // not empty: create an exact copy. it is also possible to just iterate through all + // elements and insert them, but copying is probably faster. + + auto const numElementsWithBuffer = calcNumElementsWithBuffer(o.mMask + 1); + auto const numBytesTotal = calcNumBytesTotal(numElementsWithBuffer); + + ROBIN_HOOD_LOG("std::malloc " << numBytesTotal << " = calcNumBytesTotal(" + << numElementsWithBuffer << ")") + mKeyVals = static_cast( + detail::assertNotNull(std::malloc(numBytesTotal))); + // no need for calloc because clonData does memcpy + mInfo = reinterpret_cast(mKeyVals + numElementsWithBuffer); + mNumElements = o.mNumElements; + mMask = o.mMask; + mMaxNumElementsAllowed = o.mMaxNumElementsAllowed; + mInfoInc = o.mInfoInc; + mInfoHashShift = o.mInfoHashShift; + cloneData(o); + } + } + + // Creates a copy of the given map. Copy constructor of each entry is used. + // Not sure why clang-tidy thinks this doesn't handle self assignment, it does + // NOLINTNEXTLINE(bugprone-unhandled-self-assignment,cert-oop54-cpp) + Table& operator=(Table const& o) { + ROBIN_HOOD_TRACE(this) + if (&o == this) { + // prevent assigning of itself + return *this; + } + + // we keep using the old allocator and not assign the new one, because we want to keep + // the memory available. when it is the same size. + if (o.empty()) { + if (0 == mMask) { + // nothing to do, we are empty too + return *this; + } + + // not empty: destroy what we have there + // clear also resets mInfo to 0, that's sometimes not necessary. + destroy(); + init(); + WHash::operator=(static_cast(o)); + WKeyEqual::operator=(static_cast(o)); + DataPool::operator=(static_cast(o)); + + return *this; + } + + // clean up old stuff + Destroyer::value>{}.nodes(*this); + + if (mMask != o.mMask) { + // no luck: we don't have the same array size allocated, so we need to realloc. + if (0 != mMask) { + // only deallocate if we actually have data! + ROBIN_HOOD_LOG("std::free") + std::free(mKeyVals); + } + + auto const numElementsWithBuffer = calcNumElementsWithBuffer(o.mMask + 1); + auto const numBytesTotal = calcNumBytesTotal(numElementsWithBuffer); + ROBIN_HOOD_LOG("std::malloc " << numBytesTotal << " = calcNumBytesTotal(" + << numElementsWithBuffer << ")") + mKeyVals = static_cast( + detail::assertNotNull(std::malloc(numBytesTotal))); + + // no need for calloc here because cloneData performs a memcpy. + mInfo = reinterpret_cast(mKeyVals + numElementsWithBuffer); + // sentinel is set in cloneData + } + WHash::operator=(static_cast(o)); + WKeyEqual::operator=(static_cast(o)); + DataPool::operator=(static_cast(o)); + mNumElements = o.mNumElements; + mMask = o.mMask; + mMaxNumElementsAllowed = o.mMaxNumElementsAllowed; + mInfoInc = o.mInfoInc; + mInfoHashShift = o.mInfoHashShift; + cloneData(o); + + return *this; + } + + // Swaps everything between the two maps. + void swap(Table& o) { + ROBIN_HOOD_TRACE(this) + using std::swap; + swap(o, *this); + } + + // Clears all data, without resizing. + void clear() { + ROBIN_HOOD_TRACE(this) + if (empty()) { + // don't do anything! also important because we don't want to write to + // DummyInfoByte::b, even though we would just write 0 to it. + return; + } + + Destroyer::value>{}.nodes(*this); + + auto const numElementsWithBuffer = calcNumElementsWithBuffer(mMask + 1); + // clear everything, then set the sentinel again + uint8_t const z = 0; + std::fill(mInfo, mInfo + calcNumBytesInfo(numElementsWithBuffer), z); + mInfo[numElementsWithBuffer] = 1; + + mInfoInc = InitialInfoInc; + mInfoHashShift = InitialInfoHashShift; + } + + // Destroys the map and all it's contents. + ~Table() { + ROBIN_HOOD_TRACE(this) + destroy(); + } + + // Checks if both tables contain the same entries. Order is irrelevant. + bool operator==(const Table& other) const { + ROBIN_HOOD_TRACE(this) + if (other.size() != size()) { + return false; + } + for (auto const& otherEntry : other) { + if (!has(otherEntry)) { + return false; + } + } + + return true; + } + + bool operator!=(const Table& other) const { + ROBIN_HOOD_TRACE(this) + return !operator==(other); + } + + template + typename std::enable_if::value, Q&>::type operator[](const key_type& key) { + ROBIN_HOOD_TRACE(this) + return doCreateByKey(key); + } + + template + typename std::enable_if::value, Q&>::type operator[](key_type&& key) { + ROBIN_HOOD_TRACE(this) + return doCreateByKey(std::move(key)); + } + + template + void insert(Iter first, Iter last) { + for (; first != last; ++first) { + // value_type ctor needed because this might be called with std::pair's + insert(value_type(*first)); + } + } + + template + std::pair emplace(Args&&... args) { + ROBIN_HOOD_TRACE(this) + Node n{*this, std::forward(args)...}; + auto r = doInsert(std::move(n)); + if (!r.second) { + // insertion not possible: destroy node + // NOLINTNEXTLINE(bugprone-use-after-move) + n.destroy(*this); + } + return r; + } + + template + std::pair try_emplace(const key_type& key, Args&&... args) { + return try_emplace_impl(key, std::forward(args)...); + } + + template + std::pair try_emplace(key_type&& key, Args&&... args) { + return try_emplace_impl(std::move(key), std::forward(args)...); + } + + template + std::pair try_emplace(const_iterator hint, const key_type& key, + Args&&... args) { + (void)hint; + return try_emplace_impl(key, std::forward(args)...); + } + + template + std::pair try_emplace(const_iterator hint, key_type&& key, Args&&... args) { + (void)hint; + return try_emplace_impl(std::move(key), std::forward(args)...); + } + + template + std::pair insert_or_assign(const key_type& key, Mapped&& obj) { + return insert_or_assign_impl(key, std::forward(obj)); + } + + template + std::pair insert_or_assign(key_type&& key, Mapped&& obj) { + return insert_or_assign_impl(std::move(key), std::forward(obj)); + } + + template + std::pair insert_or_assign(const_iterator hint, const key_type& key, + Mapped&& obj) { + (void)hint; + return insert_or_assign_impl(key, std::forward(obj)); + } + + template + std::pair insert_or_assign(const_iterator hint, key_type&& key, Mapped&& obj) { + (void)hint; + return insert_or_assign_impl(std::move(key), std::forward(obj)); + } + + std::pair insert(const value_type& keyval) { + ROBIN_HOOD_TRACE(this) + return doInsert(keyval); + } + + std::pair insert(value_type&& keyval) { + return doInsert(std::move(keyval)); + } + + // Returns 1 if key is found, 0 otherwise. + size_t count(const key_type& key) const { // NOLINT(modernize-use-nodiscard) + ROBIN_HOOD_TRACE(this) + auto kv = mKeyVals + findIdx(key); + if (kv != reinterpret_cast_no_cast_align_warning(mInfo)) { + return 1; + } + return 0; + } + + template + // NOLINTNEXTLINE(modernize-use-nodiscard) + typename std::enable_if::type count(const OtherKey& key) const { + ROBIN_HOOD_TRACE(this) + auto kv = mKeyVals + findIdx(key); + if (kv != reinterpret_cast_no_cast_align_warning(mInfo)) { + return 1; + } + return 0; + } + + bool contains(const key_type& key) const { // NOLINT(modernize-use-nodiscard) + return 1U == count(key); + } + + template + // NOLINTNEXTLINE(modernize-use-nodiscard) + typename std::enable_if::type contains(const OtherKey& key) const { + return 1U == count(key); + } + + // Returns a reference to the value found for key. + // Throws std::out_of_range if element cannot be found + template + // NOLINTNEXTLINE(modernize-use-nodiscard) + typename std::enable_if::value, Q&>::type at(key_type const& key) { + ROBIN_HOOD_TRACE(this) + auto kv = mKeyVals + findIdx(key); + if (kv == reinterpret_cast_no_cast_align_warning(mInfo)) { + doThrow("key not found"); + } + return kv->getSecond(); + } + + // Returns a reference to the value found for key. + // Throws std::out_of_range if element cannot be found + template + // NOLINTNEXTLINE(modernize-use-nodiscard) + typename std::enable_if::value, Q const&>::type at(key_type const& key) const { + ROBIN_HOOD_TRACE(this) + auto kv = mKeyVals + findIdx(key); + if (kv == reinterpret_cast_no_cast_align_warning(mInfo)) { + doThrow("key not found"); + } + return kv->getSecond(); + } + + const_iterator find(const key_type& key) const { // NOLINT(modernize-use-nodiscard) + ROBIN_HOOD_TRACE(this) + const size_t idx = findIdx(key); + return const_iterator{mKeyVals + idx, mInfo + idx}; + } + + template + const_iterator find(const OtherKey& key, is_transparent_tag /*unused*/) const { + ROBIN_HOOD_TRACE(this) + const size_t idx = findIdx(key); + return const_iterator{mKeyVals + idx, mInfo + idx}; + } + + template + typename std::enable_if::type // NOLINT(modernize-use-nodiscard) + find(const OtherKey& key) const { // NOLINT(modernize-use-nodiscard) + ROBIN_HOOD_TRACE(this) + const size_t idx = findIdx(key); + return const_iterator{mKeyVals + idx, mInfo + idx}; + } + + iterator find(const key_type& key) { + ROBIN_HOOD_TRACE(this) + const size_t idx = findIdx(key); + return iterator{mKeyVals + idx, mInfo + idx}; + } + + template + iterator find(const OtherKey& key, is_transparent_tag /*unused*/) { + ROBIN_HOOD_TRACE(this) + const size_t idx = findIdx(key); + return iterator{mKeyVals + idx, mInfo + idx}; + } + + template + typename std::enable_if::type find(const OtherKey& key) { + ROBIN_HOOD_TRACE(this) + const size_t idx = findIdx(key); + return iterator{mKeyVals + idx, mInfo + idx}; + } + + iterator begin() { + ROBIN_HOOD_TRACE(this) + if (empty()) { + return end(); + } + return iterator(mKeyVals, mInfo, fast_forward_tag{}); + } + const_iterator begin() const { // NOLINT(modernize-use-nodiscard) + ROBIN_HOOD_TRACE(this) + return cbegin(); + } + const_iterator cbegin() const { // NOLINT(modernize-use-nodiscard) + ROBIN_HOOD_TRACE(this) + if (empty()) { + return cend(); + } + return const_iterator(mKeyVals, mInfo, fast_forward_tag{}); + } + + iterator end() { + ROBIN_HOOD_TRACE(this) + // no need to supply valid info pointer: end() must not be dereferenced, and only node + // pointer is compared. + return iterator{reinterpret_cast_no_cast_align_warning(mInfo), nullptr}; + } + const_iterator end() const { // NOLINT(modernize-use-nodiscard) + ROBIN_HOOD_TRACE(this) + return cend(); + } + const_iterator cend() const { // NOLINT(modernize-use-nodiscard) + ROBIN_HOOD_TRACE(this) + return const_iterator{reinterpret_cast_no_cast_align_warning(mInfo), nullptr}; + } + + iterator erase(const_iterator pos) { + ROBIN_HOOD_TRACE(this) + // its safe to perform const cast here + // NOLINTNEXTLINE(cppcoreguidelines-pro-type-const-cast) + return erase(iterator{const_cast(pos.mKeyVals), const_cast(pos.mInfo)}); + } + + // Erases element at pos, returns iterator to the next element. + iterator erase(iterator pos) { + ROBIN_HOOD_TRACE(this) + // we assume that pos always points to a valid entry, and not end(). + auto const idx = static_cast(pos.mKeyVals - mKeyVals); + + shiftDown(idx); + --mNumElements; + + if (*pos.mInfo) { + // we've backward shifted, return this again + return pos; + } + + // no backward shift, return next element + return ++pos; + } + + size_t erase(const key_type& key) { + ROBIN_HOOD_TRACE(this) + size_t idx{}; + InfoType info{}; + keyToIdx(key, &idx, &info); + + // check while info matches with the source idx + do { + if (info == mInfo[idx] && WKeyEqual::operator()(key, mKeyVals[idx].getFirst())) { + shiftDown(idx); + --mNumElements; + return 1; + } + next(&info, &idx); + } while (info <= mInfo[idx]); + + // nothing found to delete + return 0; + } + + // reserves space for the specified number of elements. Makes sure the old data fits. + // exactly the same as reserve(c). + void rehash(size_t c) { + // forces a reserve + reserve(c, true); + } + + // reserves space for the specified number of elements. Makes sure the old data fits. + // Exactly the same as rehash(c). Use rehash(0) to shrink to fit. + void reserve(size_t c) { + // reserve, but don't force rehash + reserve(c, false); + } + + // If possible reallocates the map to a smaller one. This frees the underlying table. + // Does not do anything if load_factor is too large for decreasing the table's size. + void compact() { + ROBIN_HOOD_TRACE(this) + auto newSize = InitialNumElements; + while (calcMaxNumElementsAllowed(newSize) < mNumElements && newSize != 0) { + newSize *= 2; + } + if (ROBIN_HOOD_UNLIKELY(newSize == 0)) { + throwOverflowError(); + } + + ROBIN_HOOD_LOG("newSize > mMask + 1: " << newSize << " > " << mMask << " + 1") + + // only actually do anything when the new size is bigger than the old one. This prevents to + // continuously allocate for each reserve() call. + if (newSize < mMask + 1) { + rehashPowerOfTwo(newSize, true); + } + } + + size_type size() const noexcept { // NOLINT(modernize-use-nodiscard) + ROBIN_HOOD_TRACE(this) + return mNumElements; + } + + size_type max_size() const noexcept { // NOLINT(modernize-use-nodiscard) + ROBIN_HOOD_TRACE(this) + return static_cast(-1); + } + + ROBIN_HOOD(NODISCARD) bool empty() const noexcept { + ROBIN_HOOD_TRACE(this) + return 0 == mNumElements; + } + + float max_load_factor() const noexcept { // NOLINT(modernize-use-nodiscard) + ROBIN_HOOD_TRACE(this) + return MaxLoadFactor100 / 100.0F; + } + + // Average number of elements per bucket. Since we allow only 1 per bucket + float load_factor() const noexcept { // NOLINT(modernize-use-nodiscard) + ROBIN_HOOD_TRACE(this) + return static_cast(size()) / static_cast(mMask + 1); + } + + ROBIN_HOOD(NODISCARD) size_t mask() const noexcept { + ROBIN_HOOD_TRACE(this) + return mMask; + } + + ROBIN_HOOD(NODISCARD) size_t calcMaxNumElementsAllowed(size_t maxElements) const noexcept { + if (ROBIN_HOOD_LIKELY(maxElements <= (std::numeric_limits::max)() / 100)) { + return maxElements * MaxLoadFactor100 / 100; + } + + // we might be a bit inprecise, but since maxElements is quite large that doesn't matter + return (maxElements / 100) * MaxLoadFactor100; + } + + ROBIN_HOOD(NODISCARD) size_t calcNumBytesInfo(size_t numElements) const noexcept { + // we add a uint64_t, which houses the sentinel (first byte) and padding so we can load + // 64bit types. + return numElements + sizeof(uint64_t); + } + + ROBIN_HOOD(NODISCARD) + size_t calcNumElementsWithBuffer(size_t numElements) const noexcept { + auto maxNumElementsAllowed = calcMaxNumElementsAllowed(numElements); + return numElements + (std::min)(maxNumElementsAllowed, (static_cast(0xFF))); + } + + // calculation only allowed for 2^n values + ROBIN_HOOD(NODISCARD) size_t calcNumBytesTotal(size_t numElements) const { +#if ROBIN_HOOD(BITNESS) == 64 + return numElements * sizeof(Node) + calcNumBytesInfo(numElements); +#else + // make sure we're doing 64bit operations, so we are at least safe against 32bit overflows. + auto const ne = static_cast(numElements); + auto const s = static_cast(sizeof(Node)); + auto const infos = static_cast(calcNumBytesInfo(numElements)); + + auto const total64 = ne * s + infos; + auto const total = static_cast(total64); + + if (ROBIN_HOOD_UNLIKELY(static_cast(total) != total64)) { + throwOverflowError(); + } + return total; +#endif + } + +private: + template + ROBIN_HOOD(NODISCARD) + typename std::enable_if::value, bool>::type has(const value_type& e) const { + ROBIN_HOOD_TRACE(this) + auto it = find(e.first); + return it != end() && it->second == e.second; + } + + template + ROBIN_HOOD(NODISCARD) + typename std::enable_if::value, bool>::type has(const value_type& e) const { + ROBIN_HOOD_TRACE(this) + return find(e) != end(); + } + + void reserve(size_t c, bool forceRehash) { + ROBIN_HOOD_TRACE(this) + auto const minElementsAllowed = (std::max)(c, mNumElements); + auto newSize = InitialNumElements; + while (calcMaxNumElementsAllowed(newSize) < minElementsAllowed && newSize != 0) { + newSize *= 2; + } + if (ROBIN_HOOD_UNLIKELY(newSize == 0)) { + throwOverflowError(); + } + + ROBIN_HOOD_LOG("newSize > mMask + 1: " << newSize << " > " << mMask << " + 1") + + // only actually do anything when the new size is bigger than the old one. This prevents to + // continuously allocate for each reserve() call. + if (forceRehash || newSize > mMask + 1) { + rehashPowerOfTwo(newSize, false); + } + } + + // reserves space for at least the specified number of elements. + // only works if numBuckets if power of two + void rehashPowerOfTwo(size_t numBuckets, bool forceFree) { + ROBIN_HOOD_TRACE(this) + + Node* const oldKeyVals = mKeyVals; + uint8_t const* const oldInfo = mInfo; + + const size_t oldMaxElementsWithBuffer = calcNumElementsWithBuffer(mMask + 1); + + // resize operation: move stuff + init_data(numBuckets); + if (oldMaxElementsWithBuffer > 1) { + for (size_t i = 0; i < oldMaxElementsWithBuffer; ++i) { + if (oldInfo[i] != 0) { + insert_move(std::move(oldKeyVals[i])); + // destroy the node but DON'T destroy the data. + oldKeyVals[i].~Node(); + } + } + + // this check is not necessary as it's guarded by the previous if, but it helps silence + // g++'s overeager "attempt to free a non-heap object 'map' + // [-Werror=free-nonheap-object]" warning. + if (oldKeyVals != reinterpret_cast_no_cast_align_warning(&mMask)) { + // don't destroy old data: put it into the pool instead + if (forceFree) { + std::free(oldKeyVals); + } else { + DataPool::addOrFree(oldKeyVals, calcNumBytesTotal(oldMaxElementsWithBuffer)); + } + } + } + } + + ROBIN_HOOD(NOINLINE) void throwOverflowError() const { +#if ROBIN_HOOD(HAS_EXCEPTIONS) + throw std::overflow_error("robin_hood::map overflow"); +#else + abort(); +#endif + } + + template + std::pair try_emplace_impl(OtherKey&& key, Args&&... args) { + ROBIN_HOOD_TRACE(this) + auto it = find(key); + if (it == end()) { + return emplace(std::piecewise_construct, + std::forward_as_tuple(std::forward(key)), + std::forward_as_tuple(std::forward(args)...)); + } + return {it, false}; + } + + template + std::pair insert_or_assign_impl(OtherKey&& key, Mapped&& obj) { + ROBIN_HOOD_TRACE(this) + auto it = find(key); + if (it == end()) { + return emplace(std::forward(key), std::forward(obj)); + } + it->second = std::forward(obj); + return {it, false}; + } + + void init_data(size_t max_elements) { + mNumElements = 0; + mMask = max_elements - 1; + mMaxNumElementsAllowed = calcMaxNumElementsAllowed(max_elements); + + auto const numElementsWithBuffer = calcNumElementsWithBuffer(max_elements); + + // calloc also zeroes everything + auto const numBytesTotal = calcNumBytesTotal(numElementsWithBuffer); + ROBIN_HOOD_LOG("std::calloc " << numBytesTotal << " = calcNumBytesTotal(" + << numElementsWithBuffer << ")") + mKeyVals = reinterpret_cast( + detail::assertNotNull(std::calloc(1, numBytesTotal))); + mInfo = reinterpret_cast(mKeyVals + numElementsWithBuffer); + + // set sentinel + mInfo[numElementsWithBuffer] = 1; + + mInfoInc = InitialInfoInc; + mInfoHashShift = InitialInfoHashShift; + } + + template + typename std::enable_if::value, Q&>::type doCreateByKey(Arg&& key) { + while (true) { + size_t idx{}; + InfoType info{}; + keyToIdx(key, &idx, &info); + nextWhileLess(&info, &idx); + + // while we potentially have a match. Can't do a do-while here because when mInfo is + // 0 we don't want to skip forward + while (info == mInfo[idx]) { + if (WKeyEqual::operator()(key, mKeyVals[idx].getFirst())) { + // key already exists, do not insert. + return mKeyVals[idx].getSecond(); + } + next(&info, &idx); + } + + // unlikely that this evaluates to true + if (ROBIN_HOOD_UNLIKELY(mNumElements >= mMaxNumElementsAllowed)) { + increase_size(); + continue; + } + + // key not found, so we are now exactly where we want to insert it. + auto const insertion_idx = idx; + auto const insertion_info = info; + if (ROBIN_HOOD_UNLIKELY(insertion_info + mInfoInc > 0xFF)) { + mMaxNumElementsAllowed = 0; + } + + // find an empty spot + while (0 != mInfo[idx]) { + next(&info, &idx); + } + + auto& l = mKeyVals[insertion_idx]; + if (idx == insertion_idx) { + // put at empty spot. This forwards all arguments into the node where the object + // is constructed exactly where it is needed. + ::new (static_cast(&l)) + Node(*this, std::piecewise_construct, + std::forward_as_tuple(std::forward(key)), std::forward_as_tuple()); + } else { + shiftUp(idx, insertion_idx); + l = Node(*this, std::piecewise_construct, + std::forward_as_tuple(std::forward(key)), std::forward_as_tuple()); + } + + // mKeyVals[idx].getFirst() = std::move(key); + mInfo[insertion_idx] = static_cast(insertion_info); + + ++mNumElements; + return mKeyVals[insertion_idx].getSecond(); + } + } + + // This is exactly the same code as operator[], except for the return values + template + std::pair doInsert(Arg&& keyval) { + while (true) { + size_t idx{}; + InfoType info{}; + keyToIdx(getFirstConst(keyval), &idx, &info); + nextWhileLess(&info, &idx); + + // while we potentially have a match + while (info == mInfo[idx]) { + if (WKeyEqual::operator()(getFirstConst(keyval), mKeyVals[idx].getFirst())) { + // key already exists, do NOT insert. + // see http://en.cppreference.com/w/cpp/container/unordered_map/insert + return std::make_pair(iterator(mKeyVals + idx, mInfo + idx), + false); + } + next(&info, &idx); + } + + // unlikely that this evaluates to true + if (ROBIN_HOOD_UNLIKELY(mNumElements >= mMaxNumElementsAllowed)) { + increase_size(); + continue; + } + + // key not found, so we are now exactly where we want to insert it. + auto const insertion_idx = idx; + auto const insertion_info = info; + if (ROBIN_HOOD_UNLIKELY(insertion_info + mInfoInc > 0xFF)) { + mMaxNumElementsAllowed = 0; + } + + // find an empty spot + while (0 != mInfo[idx]) { + next(&info, &idx); + } + + auto& l = mKeyVals[insertion_idx]; + if (idx == insertion_idx) { + ::new (static_cast(&l)) Node(*this, std::forward(keyval)); + } else { + shiftUp(idx, insertion_idx); + l = Node(*this, std::forward(keyval)); + } + + // put at empty spot + mInfo[insertion_idx] = static_cast(insertion_info); + + ++mNumElements; + return std::make_pair(iterator(mKeyVals + insertion_idx, mInfo + insertion_idx), true); + } + } + + bool try_increase_info() { + ROBIN_HOOD_LOG("mInfoInc=" << mInfoInc << ", numElements=" << mNumElements + << ", maxNumElementsAllowed=" + << calcMaxNumElementsAllowed(mMask + 1)) + if (mInfoInc <= 2) { + // need to be > 2 so that shift works (otherwise undefined behavior!) + return false; + } + // we got space left, try to make info smaller + mInfoInc = static_cast(mInfoInc >> 1U); + + // remove one bit of the hash, leaving more space for the distance info. + // This is extremely fast because we can operate on 8 bytes at once. + ++mInfoHashShift; + auto const numElementsWithBuffer = calcNumElementsWithBuffer(mMask + 1); + + for (size_t i = 0; i < numElementsWithBuffer; i += 8) { + auto val = unaligned_load(mInfo + i); + val = (val >> 1U) & UINT64_C(0x7f7f7f7f7f7f7f7f); + std::memcpy(mInfo + i, &val, sizeof(val)); + } + // update sentinel, which might have been cleared out! + mInfo[numElementsWithBuffer] = 1; + + mMaxNumElementsAllowed = calcMaxNumElementsAllowed(mMask + 1); + return true; + } + + void increase_size() { + // nothing allocated yet? just allocate InitialNumElements + if (0 == mMask) { + init_data(InitialNumElements); + return; + } + + auto const maxNumElementsAllowed = calcMaxNumElementsAllowed(mMask + 1); + if (mNumElements < maxNumElementsAllowed && try_increase_info()) { + return; + } + + ROBIN_HOOD_LOG("mNumElements=" << mNumElements << ", maxNumElementsAllowed=" + << maxNumElementsAllowed << ", load=" + << (static_cast(mNumElements) * 100.0 / + (static_cast(mMask) + 1))) + // it seems we have a really bad hash function! don't try to resize again + if (mNumElements * 2 < calcMaxNumElementsAllowed(mMask + 1)) { + throwOverflowError(); + } + + rehashPowerOfTwo((mMask + 1) * 2, false); + } + + void destroy() { + if (0 == mMask) { + // don't deallocate! + return; + } + + Destroyer::value>{} + .nodesDoNotDeallocate(*this); + + // This protection against not deleting mMask shouldn't be needed as it's sufficiently + // protected with the 0==mMask check, but I have this anyways because g++ 7 otherwise + // reports a compile error: attempt to free a non-heap object 'fm' + // [-Werror=free-nonheap-object] + if (mKeyVals != reinterpret_cast_no_cast_align_warning(&mMask)) { + ROBIN_HOOD_LOG("std::free") + std::free(mKeyVals); + } + } + + void init() noexcept { + mKeyVals = reinterpret_cast_no_cast_align_warning(&mMask); + mInfo = reinterpret_cast(&mMask); + mNumElements = 0; + mMask = 0; + mMaxNumElementsAllowed = 0; + mInfoInc = InitialInfoInc; + mInfoHashShift = InitialInfoHashShift; + } + + // members are sorted so no padding occurs + Node* mKeyVals = reinterpret_cast_no_cast_align_warning(&mMask); // 8 byte 8 + uint8_t* mInfo = reinterpret_cast(&mMask); // 8 byte 16 + size_t mNumElements = 0; // 8 byte 24 + size_t mMask = 0; // 8 byte 32 + size_t mMaxNumElementsAllowed = 0; // 8 byte 40 + InfoType mInfoInc = InitialInfoInc; // 4 byte 44 + InfoType mInfoHashShift = InitialInfoHashShift; // 4 byte 48 + // 16 byte 56 if NodeAllocator +}; + +} // namespace detail + +// map + +template , + typename KeyEqual = std::equal_to, size_t MaxLoadFactor100 = 80> +using unordered_flat_map = detail::Table; + +template , + typename KeyEqual = std::equal_to, size_t MaxLoadFactor100 = 80> +using unordered_node_map = detail::Table; + +template , + typename KeyEqual = std::equal_to, size_t MaxLoadFactor100 = 80> +using unordered_map = + detail::Table) <= sizeof(size_t) * 6 && + std::is_nothrow_move_constructible>::value && + std::is_nothrow_move_assignable>::value, + MaxLoadFactor100, Key, T, Hash, KeyEqual>; + +// set + +template , typename KeyEqual = std::equal_to, + size_t MaxLoadFactor100 = 80> +using unordered_flat_set = detail::Table; + +template , typename KeyEqual = std::equal_to, + size_t MaxLoadFactor100 = 80> +using unordered_node_set = detail::Table; + +template , typename KeyEqual = std::equal_to, + size_t MaxLoadFactor100 = 80> +using unordered_set = detail::Table::value && + std::is_nothrow_move_assignable::value, + MaxLoadFactor100, Key, void, Hash, KeyEqual>; + +} // namespace robin_hood + +#endif diff --git a/utils/common/threadnaming.cpp b/utils/common/threadnaming.cpp index 65498ab61..204b84474 100644 --- a/utils/common/threadnaming.cpp +++ b/utils/common/threadnaming.cpp @@ -16,6 +16,7 @@ MA 02110-1301, USA. */ #include +#include "threadnaming.h" namespace utils { @@ -23,4 +24,11 @@ namespace utils { prctl(PR_SET_NAME, threadName, 0, 0, 0); } + + std::string getThreadName() + { + char buf[32]; + prctl(PR_GET_NAME, buf, 0, 0, 0); + return std::string(buf); + } } // end of namespace diff --git a/utils/common/threadnaming.h b/utils/common/threadnaming.h index 4cdf28d20..486cd168a 100644 --- a/utils/common/threadnaming.h +++ b/utils/common/threadnaming.h @@ -17,8 +17,11 @@ #ifndef H_SETTHREADNAME #define H_SETTHREADNAME +#include + namespace utils { void setThreadName(const char *threadName); + std::string getThreadName(); } // end of namespace #endif diff --git a/utils/configcpp/configcpp.cpp b/utils/configcpp/configcpp.cpp index c1171910b..93a777dfe 100644 --- a/utils/configcpp/configcpp.cpp +++ b/utils/configcpp/configcpp.cpp @@ -59,6 +59,9 @@ namespace fs = boost::filesystem; #include "installdir.h" #ifdef _MSC_VER #include "idbregistry.h" +#include +#else +#include #endif #include "bytestream.h" @@ -673,6 +676,24 @@ const vector Config::enumSection(const string& section) return fParser.enumSection(fDoc, section); } +std::string Config::getTempFileDir(Config::TempDirPurpose what) +{ + std::string prefix = getConfig("SystemConfig", "SystemTempFileDir"); + if (prefix.empty()) + { + prefix.assign("/tmp/columnstore_tmp_files"); + } + prefix.append("/"); + switch (what) + { + case TempDirPurpose::Joins: + return prefix.append("joins/"); + case TempDirPurpose::Aggregates: + return prefix.append("aggregates/"); + } + // NOTREACHED + return {}; +} } //namespace config // vim:ts=4 sw=4: diff --git a/utils/configcpp/configcpp.h b/utils/configcpp/configcpp.h index 2dfe8be5b..57e90bc20 100644 --- a/utils/configcpp/configcpp.h +++ b/utils/configcpp/configcpp.h @@ -203,6 +203,14 @@ public: */ EXPORT const std::vector enumSection(const std::string& section); + enum class TempDirPurpose + { + Joins, ///< disk joins + Aggregates ///< disk-based aggregation + }; + /** @brief Return temporaru directory path for the specified purpose */ + EXPORT std::string getTempFileDir(TempDirPurpose what); + protected: /** @brief parse the XML file * diff --git a/utils/joiner/joinpartition.cpp b/utils/joiner/joinpartition.cpp index e3b763120..0b102cd70 100644 --- a/utils/joiner/joinpartition.cpp +++ b/utils/joiner/joinpartition.cpp @@ -129,7 +129,8 @@ JoinPartition::JoinPartition(const JoinPartition& jp, bool splitMode) : // Instead, each will double in size, giving a capacity of 8GB -> 16 -> 32, and so on. // bucketCount = jp.bucketCount; bucketCount = 2; - filenamePrefix = startup::StartUp::tmpDir(); + config::Config* config = config::Config::makeConfig(); + filenamePrefix = config->getTempFileDir(config::Config::TempDirPurpose::Joins); filenamePrefix += "/Columnstore-join-data-"; diff --git a/utils/loggingcpp/ErrorMessage.txt b/utils/loggingcpp/ErrorMessage.txt index e98a96f98..4a899fc4c 100755 --- a/utils/loggingcpp/ErrorMessage.txt +++ b/utils/loggingcpp/ErrorMessage.txt @@ -100,6 +100,10 @@ 2053 ERR_FUNC_OUT_OF_RANGE_RESULT The result is out of range for function %1% using value(s): %2% %3% +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. +2056 ERR_DISKAGG_FILEIO_ERROR There was an IO error during a disk-based aggregation: %1% + # Sub-query errors 3001 ERR_NON_SUPPORT_SUB_QUERY_TYPE This subquery type is not supported yet. 3002 ERR_MORE_THAN_1_ROW Subquery returns more than 1 row. diff --git a/utils/rowgroup/CMakeLists.txt b/utils/rowgroup/CMakeLists.txt index 876d620cf..6f07e1fb9 100644 --- a/utils/rowgroup/CMakeLists.txt +++ b/utils/rowgroup/CMakeLists.txt @@ -4,7 +4,7 @@ include_directories( ${ENGINE_COMMON_INCLUDES} ) ########### next target ############### -set(rowgroup_LIB_SRCS rowaggregation.cpp rowgroup.cpp) +set(rowgroup_LIB_SRCS rowaggregation.cpp rowgroup.cpp rowstorage.cpp) #librowgroup_la_CXXFLAGS = $(march_flags) $(AM_CXXFLAGS) diff --git a/utils/rowgroup/rowaggregation.cpp b/utils/rowgroup/rowaggregation.cpp index 25f89b305..558ad2a6e 100755 --- a/utils/rowgroup/rowaggregation.cpp +++ b/utils/rowgroup/rowaggregation.cpp @@ -52,6 +52,9 @@ #include "vlarray.h" +#include "threadnaming.h" +#include "rowstorage.h" + //..comment out NDEBUG to enable assertions, uncomment NDEBUG to disable //#define NDEBUG #include "mcs_decimal.h" @@ -64,9 +67,6 @@ using namespace dataconvert; namespace { -// @bug3522, use smaller rowgroup size to conserve memory. -const int64_t AGG_ROWGROUP_SIZE = 256; - template inline bool minMax(T d1, T d2, int type) { @@ -203,7 +203,7 @@ inline uint64_t getUintNullValue(int colType, int colWidth = 0) inline double getDoubleNullValue() { uint64_t x = joblist::DOUBLENULL; - double* y = (double*)&x; + auto* y = (double*)&x; return *y; } @@ -211,7 +211,7 @@ inline double getDoubleNullValue() inline float getFloatNullValue() { uint32_t x = joblist::FLOATNULL; - float* y = (float*)&x; + auto* y = (float*)&x; return *y; } @@ -229,7 +229,7 @@ inline string getStringNullValue() namespace rowgroup { -const std::string typeStr(""); +const std::string typeStr; const static_any::any& RowAggregation::charTypeId((char)1); const static_any::any& RowAggregation::scharTypeId((signed char)1); const static_any::any& RowAggregation::shortTypeId((short)1); @@ -247,96 +247,6 @@ const static_any::any& RowAggregation::doubleTypeId((double)1); const static_any::any& RowAggregation::longdoubleTypeId((long double)1); const static_any::any& RowAggregation::strTypeId(typeStr); -using Dec = datatypes::Decimal; - -KeyStorage::KeyStorage(const RowGroup& keys, Row** tRow) : tmpRow(tRow), rg(keys) -{ - RGData data(rg); - - rg.setData(&data); - rg.resetRowGroup(0); - rg.initRow(&row); - rg.getRow(0, &row); - storage.push_back(data); - memUsage = 0; -} - -inline RowPosition KeyStorage::addKey() -{ - RowPosition pos; - - if (rg.getRowCount() == 8192) - { - RGData data(rg); - rg.setData(&data); - rg.resetRowGroup(0); - rg.getRow(0, &row); - storage.push_back(data); - } - - copyRow(**tmpRow, &row); - memUsage += row.getRealSize(); - pos.group = storage.size() - 1; - pos.row = rg.getRowCount(); - rg.incRowCount(); - row.nextRow(); - return pos; -} - -inline uint64_t KeyStorage::getMemUsage() -{ - return memUsage; -} - - -ExternalKeyHasher::ExternalKeyHasher(const RowGroup& r, KeyStorage* k, uint32_t keyColCount, Row** tRow) : - tmpRow(tRow), lastKeyCol(keyColCount - 1), ks(k) -{ - r.initRow(&row); -} - -inline uint64_t ExternalKeyHasher::operator()(const RowPosition& pos) const -{ - if (pos.group == RowPosition::MSB) - return (*tmpRow)->hash(lastKeyCol); - - RGData& rgData = ks->storage[pos.group]; - rgData.getRow(pos.row, &row); - return row.hash(lastKeyCol); -} - - -ExternalKeyEq::ExternalKeyEq(const RowGroup& r, KeyStorage* k, uint32_t keyColCount, Row** tRow) : - tmpRow(tRow), lastKeyCol(keyColCount - 1), ks(k) -{ - r.initRow(&row1); - r.initRow(&row2); -} - -inline bool ExternalKeyEq::operator()(const RowPosition& pos1, const RowPosition& pos2) const -{ - Row* r1, *r2; - - if (pos1.group == RowPosition::MSB) - r1 = *tmpRow; - else - { - ks->storage[pos1.group].getRow(pos1.row, &row1); - r1 = &row1; - } - - if (pos2.group == RowPosition::MSB) - r2 = *tmpRow; - else - { - ks->storage[pos2.group].getRow(pos2.row, &row2); - r2 = &row2; - } - - return r1->equals(*r2, lastKeyCol); -} - - static const string overflowMsg("Aggregation overflow."); inline void RowAggregation::updateIntMinMax(int128_t* val1, int128_t* val2, int64_t col, int func) @@ -592,32 +502,43 @@ inline bool RowAggregation::isNull(const RowGroup* pRowGroup, const Row& row, in //------------------------------------------------------------------------------ // Row Aggregation default constructor //------------------------------------------------------------------------------ -RowAggregation::RowAggregation() : - fAggMapPtr(NULL), fRowGroupOut(NULL), - fTotalRowCount(0), fMaxTotalRowCount(AGG_ROWGROUP_SIZE), - fSmallSideRGs(NULL), fLargeSideRG(NULL), fSmallSideCount(0), - fOrigFunctionCols(NULL) +RowAggregation::RowAggregation() + : fRowGroupOut(nullptr) + , fSmallSideRGs(nullptr) + , fLargeSideRG(nullptr) + , fSmallSideCount(0) + , fOrigFunctionCols(nullptr) { } RowAggregation::RowAggregation(const vector& rowAggGroupByCols, - const vector& rowAggFunctionCols) : - fAggMapPtr(NULL), fRowGroupOut(NULL), - fTotalRowCount(0), fMaxTotalRowCount(AGG_ROWGROUP_SIZE), - fSmallSideRGs(NULL), fLargeSideRG(NULL), fSmallSideCount(0), - fOrigFunctionCols(NULL) + const vector& rowAggFunctionCols, + joblist::ResourceManager* rm, + boost::shared_ptr sl) + : fRowGroupOut(nullptr) + , fSmallSideRGs(nullptr) + , fLargeSideRG(nullptr) + , fSmallSideCount(0) + , fOrigFunctionCols(nullptr) + , fRm(rm) + , fSessionMemLimit(std::move(sl)) { fGroupByCols.assign(rowAggGroupByCols.begin(), rowAggGroupByCols.end()); fFunctionCols.assign(rowAggFunctionCols.begin(), rowAggFunctionCols.end()); } -RowAggregation::RowAggregation(const RowAggregation& rhs): - fAggMapPtr(NULL), fRowGroupOut(NULL), - fTotalRowCount(0), fMaxTotalRowCount(AGG_ROWGROUP_SIZE), - fSmallSideRGs(NULL), fLargeSideRG(NULL), fSmallSideCount(0), - fRGContext(rhs.fRGContext), fOrigFunctionCols(NULL) +RowAggregation::RowAggregation(const RowAggregation& rhs) + : fRowGroupOut(nullptr) + , fSmallSideRGs(nullptr) + , fLargeSideRG(nullptr) + , fSmallSideCount(0) + , fKeyOnHeap(rhs.fKeyOnHeap) + , fRGContext(rhs.fRGContext) + , fOrigFunctionCols(nullptr) + , fRm(rhs.fRm) + , fSessionMemLimit(rhs.fSessionMemLimit) { fGroupByCols.assign(rhs.fGroupByCols.begin(), rhs.fGroupByCols.end()); fFunctionCols.assign(rhs.fFunctionCols.begin(), rhs.fFunctionCols.end()); @@ -629,11 +550,6 @@ RowAggregation::RowAggregation(const RowAggregation& rhs): //------------------------------------------------------------------------------ RowAggregation::~RowAggregation() { - if (fAggMapPtr) - { - delete fAggMapPtr; - fAggMapPtr = NULL; - } } @@ -672,21 +588,23 @@ void RowAggregation::addRowGroup(const RowGroup* pRows) aggregateRow(rowIn); rowIn.nextRow(); } + fRowAggStorage->dump(); } -void RowAggregation::addRowGroup(const RowGroup* pRows, vector& inRows) +void RowAggregation::addRowGroup(const RowGroup* pRows, vector>& inRows) { // this function is for threaded aggregation, which is for group by and distinct. // if (countSpecial(pRows)) Row rowIn; pRows->initRow(&rowIn); - for (uint32_t i = 0; i < inRows.size(); i++) + for (const auto& inRow : inRows) { - rowIn.setData(inRows[i]); - aggregateRow(rowIn); + rowIn.setData(inRow.first); + aggregateRow(rowIn, &inRow.second); } + fRowAggStorage->dump(); } @@ -754,9 +672,43 @@ void RowAggregation::initialize() { // Calculate the length of the hashmap key. fAggMapKeyCount = fGroupByCols.size(); + bool disk_agg = fRm ? fRm->getAllowDiskAggregation() : false; + bool allow_gen = true; + for (auto& fun : fFunctionCols) + { + if (fun->fAggFunction == ROWAGG_UDAF || fun->fAggFunction == ROWAGG_GROUP_CONCAT) + { + allow_gen = false; + break; + } + } + + config::Config* config = config::Config::makeConfig(); + string tmpDir = config->getTempFileDir(config::Config::TempDirPurpose::Aggregates); + + if (fKeyOnHeap) + { + fRowAggStorage.reset(new RowAggStorage(tmpDir, + fRowGroupOut, + &fKeyRG, + fAggMapKeyCount, + fRm, + fSessionMemLimit, + disk_agg, + allow_gen)); + } + else + { + fRowAggStorage.reset(new RowAggStorage(tmpDir, + fRowGroupOut, + fAggMapKeyCount, + fRm, + fSessionMemLimit, + disk_agg, + allow_gen)); + } // Initialize the work row. - fRowGroupOut->resetRowGroup(0); fRowGroupOut->initRow(&fRow); fRowGroupOut->getRow(0, &fRow); makeAggFieldsNull(fRow); @@ -767,21 +719,11 @@ void RowAggregation::initialize() fNullRow.setData(fNullRowData.get()); copyRow(fRow, &fNullRow); - // save the original output rowgroup data as primary row data - fPrimaryRowData = fRowGroupOut->getRGData(); - // Lazy approach w/o a mapping b/w fFunctionCols idx and fRGContextColl idx fRGContextColl.resize(fFunctionCols.size()); // Need map only if groupby list is not empty. - if (!fGroupByCols.empty()) - { - fHasher.reset(new AggHasher(fRow, &tmpRow, fGroupByCols.size(), this)); - fEq.reset(new AggComparator(fRow, &tmpRow, fGroupByCols.size(), this)); - fAlloc.reset(new utils::STLPoolAllocator()); - fAggMapPtr = new RowAggMap_t(10, *fHasher, *fEq, *fAlloc); - } - else + if (fGroupByCols.empty()) { fRowGroupOut->setRowCount(1); attachGroupConcatAg(); @@ -796,9 +738,6 @@ void RowAggregation::initialize() } } - // Save the RowGroup data pointer - fResultDataVec.push_back(fRowGroupOut->getRGData()); - // for 8k poc: an empty output row group to match message count fEmptyRowGroup = *fRowGroupOut; fEmptyRowData.reinit(*fRowGroupOut, 1); @@ -818,26 +757,45 @@ void RowAggregation::initialize() //------------------------------------------------------------------------------ void RowAggregation::aggReset() { - fTotalRowCount = 0; - fMaxTotalRowCount = AGG_ROWGROUP_SIZE; - fRowGroupOut->setData(fPrimaryRowData); - fRowGroupOut->resetRowGroup(0); + bool disk_agg = fRm ? fRm->getAllowDiskAggregation() : false; + bool allow_gen = true; + for (auto& fun : fFunctionCols) + { + if (fun->fAggFunction == ROWAGG_UDAF || fun->fAggFunction == ROWAGG_GROUP_CONCAT) + { + allow_gen = false; + break; + } + } + + config::Config* config = config::Config::makeConfig(); + string tmpDir = config->getTempFileDir(config::Config::TempDirPurpose::Aggregates); + + if (fKeyOnHeap) + { + fRowAggStorage.reset(new RowAggStorage(tmpDir, + fRowGroupOut, + &fKeyRG, + fAggMapKeyCount, + fRm, + fSessionMemLimit, + disk_agg, + allow_gen)); + } + else + { + fRowAggStorage.reset(new RowAggStorage(tmpDir, + fRowGroupOut, + fAggMapKeyCount, + fRm, + fSessionMemLimit, + disk_agg, + allow_gen)); + } fRowGroupOut->getRow(0, &fRow); copyNullRow(fRow); attachGroupConcatAg(); - if (!fGroupByCols.empty()) - { - fHasher.reset(new AggHasher(fRow, &tmpRow, fGroupByCols.size(), this)); - fEq.reset(new AggComparator(fRow, &tmpRow, fGroupByCols.size(), this)); - fAlloc.reset(new utils::STLPoolAllocator()); - delete fAggMapPtr; - fAggMapPtr = new RowAggMap_t(10, *fHasher, *fEq, *fAlloc); - } - - fResultDataVec.clear(); - fResultDataVec.push_back(fRowGroupOut->getRGData()); - // For UDAF, reset the data for (uint64_t i = 0; i < fFunctionCols.size(); i++) { @@ -849,51 +807,42 @@ void RowAggregation::aggReset() } } - -void RowAggregationUM::aggReset() +void RowAggregation::append(RowAggregation* other) { - RowAggregation::aggReset(); - - if (fKeyOnHeap) - { - fKeyRG = fRowGroupIn.truncate(fGroupByCols.size()); - fKeyStore.reset(new KeyStorage(fKeyRG, &tmpRow)); - fExtEq.reset(new ExternalKeyEq(fKeyRG, fKeyStore.get(), fKeyRG.getColumnCount(), &tmpRow)); - fExtHash.reset(new ExternalKeyHasher(fKeyRG, fKeyStore.get(), fKeyRG.getColumnCount(), &tmpRow)); - fExtKeyMapAlloc.reset(new utils::STLPoolAllocator >()); - fExtKeyMap.reset(new ExtKeyMap_t(10, *fExtHash, *fExtEq, *fExtKeyMapAlloc)); - } + fRowAggStorage->append(*other->fRowAggStorage); } -void RowAggregationUM::aggregateRowWithRemap(Row& row, - std::vector* rgContextColl) +void RowAggregationUM::aggReset() { - pair inserted; - RowPosition pos(RowPosition::MSB, 0); - - tmpRow = &row; - inserted = fExtKeyMap->insert(pair(pos, pos)); - - if (inserted.second) + if (fKeyOnHeap) { - // if it was successfully inserted, fix the inserted values - if (++fTotalRowCount > fMaxTotalRowCount && !newRowGroup()) - { - throw logging::IDBExcept(logging::IDBErrorInfo::instance()-> - errorMsg(logging::ERR_AGGREGATION_TOO_BIG), logging::ERR_AGGREGATION_TOO_BIG); - } + fKeyRG = fRowGroupIn.truncate(fGroupByCols.size()); + } + RowAggregation::aggReset(); +} - pos = fKeyStore->addKey(); - fRowGroupOut->getRow(fRowGroupOut->getRowCount(), &fRow); - fRowGroupOut->incRowCount(); - initMapData(row); //seems heavy-handed - attachGroupConcatAg(); - inserted.first->second = RowPosition(fResultDataVec.size() - 1, fRowGroupOut->getRowCount() - 1); +void RowAggregation::aggregateRow(Row& row, + const uint64_t* hash, + std::vector* rgContextColl) +{ + // groupby column list is not empty, find the entry. + if (!fGroupByCols.empty()) + { + bool is_new_row; + if (hash != nullptr) + is_new_row = fRowAggStorage->getTargetRow(row, *hash, fRow); + else + is_new_row = fRowAggStorage->getTargetRow(row, fRow); - // If there's UDAF involved, reset the user data. - if (fOrigFunctionCols) + if (is_new_row) { + initMapData(row); + attachGroupConcatAg(); + + // If there's UDAF involved, reset the user data. + if (fOrigFunctionCols) + { // This is a multi-distinct query and fFunctionCols may not // contain all the UDAF we need to reset for (uint64_t i = 0; i < fOrigFunctionCols->size(); i++) @@ -904,103 +853,18 @@ void RowAggregationUM::aggregateRowWithRemap(Row& row, resetUDAF(rowUDAFColumnPtr, i); } } - } - else - { - for (uint64_t i = 0; i < fFunctionCols.size(); i++) - { - if (fFunctionCols[i]->fAggFunction == ROWAGG_UDAF) - { - auto rowUDAFColumnPtr = dynamic_cast(fFunctionCols[i].get()); - resetUDAF(rowUDAFColumnPtr, i); - } - } - } - // replace the key value with an equivalent copy, yes this is OK - const_cast((inserted.first->first)) = pos; - } - else - { - pos = inserted.first->second; - fResultDataVec[pos.group]->getRow(pos.row, &fRow); - } - - updateEntry(row, rgContextColl); -} - - - -void RowAggregationUM::aggregateRow(Row& row, - std::vector* rgContextColl) -{ - if (UNLIKELY(fKeyOnHeap)) - aggregateRowWithRemap(row, rgContextColl); - else - RowAggregation::aggregateRow(row, rgContextColl); -} - -void RowAggregation::aggregateRow(Row& row, - std::vector* rgContextColl) -{ - // groupby column list is not empty, find the entry. - if (!fGroupByCols.empty()) - { - pair inserted; - - // do a speculative insert - tmpRow = &row; - inserted = fAggMapPtr->insert(RowPosition(RowPosition::MSB, 0)); - - if (inserted.second) - { - // if it was successfully inserted, fix the inserted values - if (++fTotalRowCount > fMaxTotalRowCount && !newRowGroup()) - { - throw logging::IDBExcept(logging::IDBErrorInfo::instance()-> - errorMsg(logging::ERR_AGGREGATION_TOO_BIG), logging::ERR_AGGREGATION_TOO_BIG); - } - - fRowGroupOut->getRow(fRowGroupOut->getRowCount(), &fRow); - fRowGroupOut->incRowCount(); - initMapData(row); //seems heavy-handed - - attachGroupConcatAg(); - - // replace the key value with an equivalent copy, yes this is OK - const_cast(*(inserted.first)) = - RowPosition(fResultDataVec.size() - 1, fRowGroupOut->getRowCount() - 1); - - // If there's UDAF involved, reset the user data. - if (fOrigFunctionCols) - { - // This is a multi-distinct query and fFunctionCols may not - // contain all the UDAF we need to reset - for (uint64_t i = 0; i < fOrigFunctionCols->size(); i++) - { - if ((*fOrigFunctionCols)[i]->fAggFunction == ROWAGG_UDAF) - { - auto rowUDAFColumnPtr = dynamic_cast((*fOrigFunctionCols)[i].get()); - resetUDAF(rowUDAFColumnPtr, i); - } - } - } - else - { - for (uint64_t i = 0; i < fFunctionCols.size(); i++) - { - if (fFunctionCols[i]->fAggFunction == ROWAGG_UDAF) - { - auto rowUDAFColumnPtr = dynamic_cast(fFunctionCols[i].get()); - resetUDAF(rowUDAFColumnPtr, i); - } - } - } - } - else - { - //fRow.setData(*(inserted.first)); - const RowPosition& pos = *(inserted.first); - fResultDataVec[pos.group]->getRow(pos.row, &fRow); + } + else + { + for (uint64_t i = 0; i < fFunctionCols.size(); i++) + { + if (fFunctionCols[i]->fAggFunction == ROWAGG_UDAF) + { + auto rowUDAFColumnPtr = dynamic_cast(fFunctionCols[i].get()); + resetUDAF(rowUDAFColumnPtr, i); + } + } + } } } @@ -1017,14 +881,14 @@ void RowAggregation::initMapData(const Row& rowIn) copyNullRow(fRow); // Then, populate the groupby cols. - for (uint64_t i = 0; i < fGroupByCols.size(); i++) + for (auto& fGroupByCol : fGroupByCols) { - int64_t colOut = fGroupByCols[i]->fOutputColumnIndex; + int64_t colOut = fGroupByCol->fOutputColumnIndex; if (colOut == numeric_limits::max()) continue; - int64_t colIn = fGroupByCols[i]->fInputColumnIndex; + int64_t colIn = fGroupByCol->fInputColumnIndex; int colDataType = ((fRowGroupIn.getColTypes())[colIn]); switch (colDataType) @@ -1148,29 +1012,29 @@ void RowAggregation::makeAggFieldsNull(Row& row) memset(row.getData(), 0, row.getSize()); //row.initToNull(); - for (uint64_t i = 0; i < fFunctionCols.size(); i++) + for (auto& fFunctionCol : fFunctionCols) { // Initial count fields to 0. - int64_t colOut = fFunctionCols[i]->fOutputColumnIndex; + int64_t colOut = fFunctionCol->fOutputColumnIndex; - if (fFunctionCols[i]->fAggFunction == ROWAGG_COUNT_ASTERISK || - fFunctionCols[i]->fAggFunction == ROWAGG_COUNT_COL_NAME || - fFunctionCols[i]->fAggFunction == ROWAGG_COUNT_DISTINCT_COL_NAME || - fFunctionCols[i]->fAggFunction == ROWAGG_COUNT_NO_OP || - fFunctionCols[i]->fAggFunction == ROWAGG_GROUP_CONCAT || - fFunctionCols[i]->fAggFunction == ROWAGG_STATS) + if (fFunctionCol->fAggFunction == ROWAGG_COUNT_ASTERISK || + fFunctionCol->fAggFunction == ROWAGG_COUNT_COL_NAME || + fFunctionCol->fAggFunction == ROWAGG_COUNT_DISTINCT_COL_NAME || + fFunctionCol->fAggFunction == ROWAGG_COUNT_NO_OP || + fFunctionCol->fAggFunction == ROWAGG_GROUP_CONCAT || + fFunctionCol->fAggFunction == ROWAGG_STATS) { continue; } // ROWAGG_BIT_AND : 0xFFFFFFFFFFFFFFFFULL; // ROWAGG_BIT_OR/ROWAGG_BIT_XOR : 0 (already set). - if (fFunctionCols[i]->fAggFunction == ROWAGG_BIT_OR || - fFunctionCols[i]->fAggFunction == ROWAGG_BIT_XOR) + if (fFunctionCol->fAggFunction == ROWAGG_BIT_OR || + fFunctionCol->fAggFunction == ROWAGG_BIT_XOR) { continue; } - else if (fFunctionCols[i]->fAggFunction == ROWAGG_BIT_AND) + else if (fFunctionCol->fAggFunction == ROWAGG_BIT_AND) { row.setUintField(0xFFFFFFFFFFFFFFFFULL, colOut); continue; @@ -1414,12 +1278,13 @@ void RowAggregation::doMinMax(const Row& rowIn, int64_t colIn, int64_t colOut, i //------------------------------------------------------------------------------ void RowAggregation::doSum(const Row& rowIn, int64_t colIn, int64_t colOut, int funcType) { - int colDataType = (fRowGroupIn.getColTypes())[colIn]; + int colDataType = rowIn.getColType(colIn); long double valIn = 0; bool isWideDataType = false; void *wideValInPtr = nullptr; - if (isNull(&fRowGroupIn, rowIn, colIn) == true) + long double valOut = fRow.getLongDoubleField(colOut); + if (rowIn.isNullValue(colIn)) return; switch (colDataType) @@ -1616,7 +1481,7 @@ void RowAggregation::doBitOp(const Row& rowIn, int64_t colIn, int64_t colOut, in case execplan::CalpontSystemCatalog::TEXT: { string str = rowIn.getStringField(colIn); - valIn = strtoll(str.c_str(), NULL, 10); + valIn = strtoll(str.c_str(), nullptr, 10); break; } @@ -1679,7 +1544,7 @@ void RowAggregation::doBitOp(const Row& rowIn, int64_t colIn, int64_t colOut, in string str = DataConvert::timestampToString1(timestamp, fTimeZone); // strip off micro seconds str = str.substr(0, 14); - valIn = strtoll(str.c_str(), NULL, 10); + valIn = strtoll(str.c_str(), nullptr, 10); break; } @@ -1879,6 +1744,73 @@ void RowAggregation::updateEntry(const Row& rowIn, } } +//------------------------------------------------------------------------------ +// Merge the aggregation subtotals in the internal hashmap for the specified row. +// NULL values are recognized and ignored for all agg functions except for +// COUNT(*), which counts all rows regardless of value. +// rowIn(in) - Row to be included in aggregation. +//------------------------------------------------------------------------------ +void RowAggregation::mergeEntries(const Row& rowIn) +{ + for (uint64_t i = 0; i < fFunctionCols.size(); i++) + { + int64_t colOut = fFunctionCols[i]->fOutputColumnIndex; + + switch (fFunctionCols[i]->fAggFunction) + { + case ROWAGG_COUNT_COL_NAME: + case ROWAGG_COUNT_ASTERISK: + fRow.setUintField<8>(fRow.getUintField<8>(colOut) + rowIn.getUintField<8>(colOut), colOut); + break; + + case ROWAGG_MIN: + case ROWAGG_MAX: + doMinMax(rowIn, colOut, colOut, fFunctionCols[i]->fAggFunction); + break; + + case ROWAGG_SUM: + doSum(rowIn, colOut, colOut, fFunctionCols[i]->fAggFunction); + break; + + case ROWAGG_AVG: + // count(column) for average is inserted after the sum, + // colOut+1 is the position of the count column. + doAvg(rowIn, colOut, colOut, colOut + 1, true); + break; + + case ROWAGG_STATS: + mergeStatistics(rowIn, colOut, colOut + 1); + break; + + case ROWAGG_BIT_AND: + case ROWAGG_BIT_OR: + case ROWAGG_BIT_XOR: + doBitOp(rowIn, colOut, colOut, fFunctionCols[i]->fAggFunction); + break; + + case ROWAGG_COUNT_NO_OP: + case ROWAGG_DUP_FUNCT: + case ROWAGG_DUP_AVG: + case ROWAGG_DUP_STATS: + case ROWAGG_DUP_UDAF: + case ROWAGG_CONSTANT: + case ROWAGG_GROUP_CONCAT: + break; + + case ROWAGG_UDAF: + doUDAF(rowIn, colOut, colOut, colOut + 1, i); + break; + + default: + std::ostringstream errmsg; + errmsg << "RowAggregation: function (id = " << + (uint64_t) fFunctionCols[i]->fAggFunction << ") is not supported."; + cerr << errmsg.str() << endl; + throw logging::QueryDataExcept(errmsg.str(), logging::aggregateFuncErr); + break; + } + } +} //------------------------------------------------------------------------------ // Update the sum and count fields for average if input is not null. @@ -1887,12 +1819,12 @@ void RowAggregation::updateEntry(const Row& rowIn, // colOut(in) - column in the output row group stores the sum // colAux(in) - column in the output row group stores the count //------------------------------------------------------------------------------ -void RowAggregation::doAvg(const Row& rowIn, int64_t colIn, int64_t colOut, int64_t colAux) +void RowAggregation::doAvg(const Row& rowIn, int64_t colIn, int64_t colOut, int64_t colAux, bool merge) { - if (isNull(&fRowGroupIn, rowIn, colIn) == true) + if (rowIn.isNullValue(colIn)) return; - int colDataType = (fRowGroupIn.getColTypes())[colIn]; + int colDataType = rowIn.getColType(colIn); long double valIn = 0; long double valOut = fRow.getLongDoubleField(colOut); bool isWideDataType = false; @@ -1975,10 +1907,20 @@ void RowAggregation::doAvg(const Row& rowIn, int64_t colIn, int64_t colOut, int6 } // min(count) = 0 - uint64_t count = fRow.getUintField(colAux) + 1; - fRow.setUintField<8>(count, colAux); - bool notFirstValue = count > 1; + uint64_t count = fRow.getUintField(colAux); + bool notFirstValue = count > 0; + // Set count column + if (merge) + { + fRow.setUintField<8>(count + rowIn.getUintField<8>(colAux), colAux); + } + else + { + fRow.setUintField<8>(count + 1, colAux); + } + + // Set sum column if (LIKELY(!isWideDataType)) { if (LIKELY(notFirstValue)) @@ -2084,6 +2026,13 @@ void RowAggregation::doStatistics(const Row& rowIn, int64_t colIn, int64_t colOu fRow.setLongDoubleField(fRow.getLongDoubleField(colAux + 1) + valIn * valIn, colAux + 1); } +void RowAggregation::mergeStatistics(const Row& rowIn, uint64_t colOut, uint64_t colAux) +{ + fRow.setDoubleField(fRow.getDoubleField(colOut) + rowIn.getDoubleField(colOut), colOut); + fRow.setLongDoubleField(fRow.getLongDoubleField(colAux) + rowIn.getLongDoubleField(colAux), colAux); + fRow.setLongDoubleField(fRow.getLongDoubleField(colAux + 1) + rowIn.getLongDoubleField(colAux + 1), colAux + 1); +} + void RowAggregation::doUDAF(const Row& rowIn, int64_t colIn, int64_t colOut, @@ -2116,8 +2065,8 @@ void RowAggregation::doUDAF(const Row& rowIn, dataFlags[i] = 0; // If this particular parameter is a constant, then we need - // to acces the constant value rather than a row value. - cc = NULL; + // to access the constant value rather than a row value. + cc = nullptr; if (fFunctionCols[funcColsIdx]->fpConstCol) { @@ -2129,7 +2078,7 @@ void RowAggregation::doUDAF(const Row& rowIn, { if (udafContextsColl[origFuncColsIdx].getRunFlag(mcsv1sdk::UDAF_IGNORE_NULLS)) { - // When Ignore nulls, if there are multiple parameters and any + // When Ignore nulls, if there are multiple parameters and any // one of them is NULL, we ignore the entry. We need to increment // funcColsIdx the number of extra parameters. funcColsIdx += paramCount - i - 1; @@ -2412,29 +2361,6 @@ void RowAggregation::doUDAF(const Row& rowIn, } } -//------------------------------------------------------------------------------ -// Allocate a new data array for the output RowGroup -// return - true if successfully allocated -//------------------------------------------------------------------------------ -bool RowAggregation::newRowGroup() -{ - // For now, n*n relation is not supported, no memory limit. - // May apply a restriction when more resarch is done -- bug 1604 - boost::shared_ptr data(new RGData(*fRowGroupOut, AGG_ROWGROUP_SIZE)); - - if (data.get() != NULL) - { - fRowGroupOut->setData(data.get()); - fRowGroupOut->resetRowGroup(0); - fSecondaryRowDataVec.push_back(data); - fResultDataVec.push_back(data.get()); - fMaxTotalRowCount += AGG_ROWGROUP_SIZE; - } - - return (data.get() != NULL); -} - - //------------------------------------------------------------------------------ // Concatenate multiple RowGroup data into one byte stream. This is for matching // the message counts of request and response. @@ -2444,17 +2370,25 @@ bool RowAggregation::newRowGroup() //------------------------------------------------------------------------------ void RowAggregation::loadResult(messageqcpp::ByteStream& bs) { - uint32_t size = fResultDataVec.size(); - bs << size; + uint32_t sz = 0; + messageqcpp::ByteStream rgdbs; + while (auto rgd = fRowAggStorage->getNextRGData()) + { + ++sz; + fRowGroupOut->setData(rgd.get()); + fRowGroupOut->serializeRGData(rgdbs); + } - for (uint32_t i = 0; i < size; i++) - { - fRowGroupOut->setData(fResultDataVec[i]); - fRowGroupOut->serializeRGData(bs); - } - - fResultDataVec.clear(); - fSecondaryRowDataVec.clear(); + if (sz == 0) + { + sz = 1; + RGData rgd(*fRowGroupOut, 1); + fRowGroupOut->setData(&rgd); + fRowGroupOut->resetRowGroup(0); + fRowGroupOut->serializeRGData(rgdbs); + } + bs << sz; + bs.append(rgdbs.buf(), rgdbs.length()); } @@ -2470,10 +2404,13 @@ void RowAggregation::loadEmptySet(messageqcpp::ByteStream& bs) //------------------------------------------------------------------------------ RowAggregationUM::RowAggregationUM(const vector& rowAggGroupByCols, const vector& rowAggFunctionCols, - joblist::ResourceManager* r, boost::shared_ptr sessionLimit) : - RowAggregation(rowAggGroupByCols, rowAggFunctionCols), fHasAvg(false), fKeyOnHeap(false), - fHasStatsFunc(false), fHasUDAF(false), fTotalMemUsage(0), fRm(r), - fSessionMemLimit(sessionLimit), fLastMemUsage(0), fNextRGIndex(0) + joblist::ResourceManager* r, boost::shared_ptr sessionLimit) + : RowAggregation(rowAggGroupByCols, rowAggFunctionCols, r, sessionLimit) + , fHasAvg(false) + , fHasStatsFunc(false) + , fHasUDAF(false) + , fTotalMemUsage(0) + , fLastMemUsage(0) { // Check if there are any avg, stats or UDAF functions. // These flags are used in finalize. @@ -2503,30 +2440,19 @@ RowAggregationUM::RowAggregationUM(const vector& rowAggGroupB RowAggregationUM::RowAggregationUM(const RowAggregationUM& rhs) : RowAggregation(rhs), fHasAvg(rhs.fHasAvg), - fKeyOnHeap(rhs.fKeyOnHeap), fHasStatsFunc(rhs.fHasStatsFunc), fHasUDAF(rhs.fHasUDAF), fExpression(rhs.fExpression), fTotalMemUsage(rhs.fTotalMemUsage), - fRm(rhs.fRm), fConstantAggregate(rhs.fConstantAggregate), fGroupConcat(rhs.fGroupConcat), - fSessionMemLimit(rhs.fSessionMemLimit), - fLastMemUsage(rhs.fLastMemUsage), - fNextRGIndex(0) + fLastMemUsage(rhs.fLastMemUsage) { - } RowAggregationUM::~RowAggregationUM() { - // on UM, a groupby column may be not a projected column, key is separated from output - // and is stored on heap, need to return the space to heap at the end. - clearAggMap(); - - // fAggMapPtr deleted by base destructor. - fRm->returnMemory(fTotalMemUsage, fSessionMemLimit); } @@ -2549,17 +2475,12 @@ void RowAggregationUM::initialize() if (fGroupConcat.size() > 0) fFunctionColGc = fFunctionCols; - RowAggregation::initialize(); - if (fKeyOnHeap) { fKeyRG = fRowGroupIn.truncate(fGroupByCols.size()); - fKeyStore.reset(new KeyStorage(fKeyRG, &tmpRow)); - fExtEq.reset(new ExternalKeyEq(fKeyRG, fKeyStore.get(), fKeyRG.getColumnCount(), &tmpRow)); - fExtHash.reset(new ExternalKeyHasher(fKeyRG, fKeyStore.get(), fKeyRG.getColumnCount(), &tmpRow)); - fExtKeyMapAlloc.reset(new utils::STLPoolAllocator >()); - fExtKeyMap.reset(new ExtKeyMap_t(10, *fExtHash, *fExtEq, *fExtKeyMapAlloc)); } + + RowAggregation::initialize(); } @@ -2983,7 +2904,7 @@ void RowAggregationUM::SetUDAFValue(static_any::any& valOut, int64_t colOut) case execplan::CalpontSystemCatalog::TEXT: if (valOut.compatible(strTypeId)) { - std::string strOut = valOut.cast(); + strOut = valOut.cast(); fRow.setStringField(strOut, colOut); bSetSuccess = true; } @@ -2995,7 +2916,7 @@ void RowAggregationUM::SetUDAFValue(static_any::any& valOut, int64_t colOut) case execplan::CalpontSystemCatalog::BLOB: if (valOut.compatible(strTypeId)) { - std::string strOut = valOut.cast(); + strOut = valOut.cast(); fRow.setVarBinaryField(strOut, colOut); bSetSuccess = true; } @@ -3142,12 +3063,12 @@ void RowAggregationUM::SetUDAFAnyValue(static_any::any& valOut, int64_t colOut) if (valOut.compatible(strTypeId)) { - std::string strOut = valOut.cast(); + strOut = valOut.cast(); // Convert the string to numeric type, just in case. intOut = atol(strOut.c_str()); - uintOut = strtoul(strOut.c_str(), NULL, 10); - doubleOut = strtod(strOut.c_str(), NULL); - longdoubleOut = strtold(strOut.c_str(), NULL); + uintOut = strtoul(strOut.c_str(), nullptr, 10); + doubleOut = strtod(strOut.c_str(), nullptr); + longdoubleOut = strtold(strOut.c_str(), nullptr); int128Out = longdoubleOut; } else @@ -3260,7 +3181,7 @@ void RowAggregationUM::SetUDAFAnyValue(static_any::any& valOut, int64_t colOut) //------------------------------------------------------------------------------ void RowAggregationUM::calculateUDAFColumns() { - RowUDAFFunctionCol* rowUDAF = NULL; + RowUDAFFunctionCol* rowUDAF = nullptr; static_any::any valOut; for (uint64_t i = 0; i < fFunctionCols.size(); i++) @@ -3281,14 +3202,14 @@ void RowAggregationUM::calculateUDAFColumns() fRowGroupOut->getRow(j, &fRow); // Turn the NULL flag off. We can't know NULL at this point - fRGContext.setDataFlags(NULL); + fRGContext.setDataFlags(nullptr); // The intermediate values are stored in colAux. fRGContext.setUserData(fRow.getUserData(colAux)); // Call the UDAF evaluate function mcsv1sdk::mcsv1_UDAF::ReturnCode rc; rc = fRGContext.getFunction()->evaluate(&fRGContext, valOut); - fRGContext.setUserData(NULL); + fRGContext.setUserData(nullptr); if (rc == mcsv1sdk::mcsv1_UDAF::ERROR) { @@ -3300,7 +3221,7 @@ void RowAggregationUM::calculateUDAFColumns() SetUDAFValue(valOut, colOut); } - fRGContext.setUserData(NULL); + fRGContext.setUserData(nullptr); } } @@ -3643,7 +3564,7 @@ void RowAggregationUM::doNullConstantAggregate(const ConstantAggData& aggData, u static_any::any valOut; rc = fRGContext.getFunction()->evaluate(&fRGContext, valOut); - fRGContext.setUserData(NULL); + fRGContext.setUserData(nullptr); if (rc == mcsv1sdk::mcsv1_UDAF::ERROR) { @@ -3653,7 +3574,7 @@ void RowAggregationUM::doNullConstantAggregate(const ConstantAggData& aggData, u // Set the returned value into the output row SetUDAFValue(valOut, colOut); - fRGContext.setDataFlags(NULL); + fRGContext.setDataFlags(nullptr); } break; @@ -3692,7 +3613,7 @@ void RowAggregationUM::doNotNullConstantAggregate(const ConstantAggData& aggData case execplan::CalpontSystemCatalog::INT: case execplan::CalpontSystemCatalog::BIGINT: { - fRow.setIntField(strtol(aggData.fConstValue.c_str(), 0, 10), colOut); + fRow.setIntField(strtol(aggData.fConstValue.c_str(), nullptr, 10), colOut); } break; @@ -3703,7 +3624,7 @@ void RowAggregationUM::doNotNullConstantAggregate(const ConstantAggData& aggData case execplan::CalpontSystemCatalog::UINT: case execplan::CalpontSystemCatalog::UBIGINT: { - fRow.setUintField(strtoul(aggData.fConstValue.c_str(), 0, 10), colOut); + fRow.setUintField(strtoul(aggData.fConstValue.c_str(), nullptr, 10), colOut); } break; @@ -3738,13 +3659,13 @@ void RowAggregationUM::doNotNullConstantAggregate(const ConstantAggData& aggData case execplan::CalpontSystemCatalog::DOUBLE: case execplan::CalpontSystemCatalog::UDOUBLE: { - fRow.setDoubleField(strtod(aggData.fConstValue.c_str(), 0), colOut); + fRow.setDoubleField(strtod(aggData.fConstValue.c_str(), nullptr), colOut); } break; case execplan::CalpontSystemCatalog::LONGDOUBLE: { - fRow.setLongDoubleField(strtold(aggData.fConstValue.c_str(), 0), colOut); + fRow.setLongDoubleField(strtold(aggData.fConstValue.c_str(), nullptr), colOut); } break; @@ -3754,7 +3675,7 @@ void RowAggregationUM::doNotNullConstantAggregate(const ConstantAggData& aggData #ifdef _MSC_VER fRow.setFloatField(strtod(aggData.fConstValue.c_str(), 0), colOut); #else - fRow.setFloatField(strtof(aggData.fConstValue.c_str(), 0), colOut); + fRow.setFloatField(strtof(aggData.fConstValue.c_str(), nullptr), colOut); #endif } break; @@ -3805,7 +3726,7 @@ void RowAggregationUM::doNotNullConstantAggregate(const ConstantAggData& aggData case execplan::CalpontSystemCatalog::INT: case execplan::CalpontSystemCatalog::BIGINT: { - int64_t constVal = strtol(aggData.fConstValue.c_str(), 0, 10); + int64_t constVal = strtol(aggData.fConstValue.c_str(), nullptr, 10); if (constVal != 0) { @@ -3828,7 +3749,7 @@ void RowAggregationUM::doNotNullConstantAggregate(const ConstantAggData& aggData case execplan::CalpontSystemCatalog::UINT: case execplan::CalpontSystemCatalog::UBIGINT: { - uint64_t constVal = strtoul(aggData.fConstValue.c_str(), 0, 10); + uint64_t constVal = strtoul(aggData.fConstValue.c_str(), nullptr, 10); fRow.setUintField(constVal * rowCnt, colOut); } break; @@ -3868,21 +3789,20 @@ void RowAggregationUM::doNotNullConstantAggregate(const ConstantAggData& aggData idbassert(0); throw std::logic_error("RowAggregationUM::doNotNullConstantAggregate(): sum() DECIMAL bad length."); } - } break; case execplan::CalpontSystemCatalog::DOUBLE: case execplan::CalpontSystemCatalog::UDOUBLE: { - double dbl = strtod(aggData.fConstValue.c_str(), 0) * rowCnt; + double dbl = strtod(aggData.fConstValue.c_str(), nullptr) * rowCnt; fRow.setDoubleField(dbl, colOut); } break; case execplan::CalpontSystemCatalog::LONGDOUBLE: { - long double dbl = strtold(aggData.fConstValue.c_str(), 0) * rowCnt; + long double dbl = strtold(aggData.fConstValue.c_str(), nullptr) * rowCnt; fRow.setLongDoubleField(dbl, colOut); } break; @@ -3894,7 +3814,7 @@ void RowAggregationUM::doNotNullConstantAggregate(const ConstantAggData& aggData #ifdef _MSC_VER flt = strtod(aggData.fConstValue.c_str(), 0) * rowCnt; #else - flt = strtof(aggData.fConstValue.c_str(), 0) * rowCnt; + flt = strtof(aggData.fConstValue.c_str(), nullptr) * rowCnt; #endif fRow.setFloatField(flt, colOut); } @@ -3982,7 +3902,7 @@ void RowAggregationUM::doNotNullConstantAggregate(const ConstantAggData& aggData case execplan::CalpontSystemCatalog::TEXT: default: { - fRow.setStringField(0, colOut); + fRow.setStringField(nullptr, colOut); } break; } @@ -4004,7 +3924,7 @@ void RowAggregationUM::doNotNullConstantAggregate(const ConstantAggData& aggData case ROWAGG_BIT_AND: case ROWAGG_BIT_OR: { - double dbl = strtod(aggData.fConstValue.c_str(), 0); + double dbl = strtod(aggData.fConstValue.c_str(), nullptr); dbl += (dbl > 0) ? 0.5 : -0.5; int64_t intVal = (int64_t) dbl; fRow.setUintField(intVal, colOut); @@ -4052,7 +3972,7 @@ void RowAggregationUM::doNotNullConstantAggregate(const ConstantAggData& aggData case execplan::CalpontSystemCatalog::INT: case execplan::CalpontSystemCatalog::BIGINT: { - datum.columnData = strtol(aggData.fConstValue.c_str(), 0, 10); + datum.columnData = strtol(aggData.fConstValue.c_str(), nullptr, 10); } break; @@ -4062,7 +3982,7 @@ void RowAggregationUM::doNotNullConstantAggregate(const ConstantAggData& aggData case execplan::CalpontSystemCatalog::UINT: case execplan::CalpontSystemCatalog::UBIGINT: { - datum.columnData = strtoul(aggData.fConstValue.c_str(), 0, 10); + datum.columnData = strtoul(aggData.fConstValue.c_str(), nullptr, 10); } break; @@ -4080,13 +4000,13 @@ void RowAggregationUM::doNotNullConstantAggregate(const ConstantAggData& aggData case execplan::CalpontSystemCatalog::DOUBLE: case execplan::CalpontSystemCatalog::UDOUBLE: { - datum.columnData = strtod(aggData.fConstValue.c_str(), 0); + datum.columnData = strtod(aggData.fConstValue.c_str(), nullptr); } break; case execplan::CalpontSystemCatalog::LONGDOUBLE: { - datum.columnData = strtold(aggData.fConstValue.c_str(), 0); + datum.columnData = strtold(aggData.fConstValue.c_str(), nullptr); } break; @@ -4096,7 +4016,7 @@ void RowAggregationUM::doNotNullConstantAggregate(const ConstantAggData& aggData #ifdef _MSC_VER datum.columnData = strtod(aggData.fConstValue.c_str(), 0); #else - datum.columnData = strtof(aggData.fConstValue.c_str(), 0); + datum.columnData = strtof(aggData.fConstValue.c_str(), nullptr); #endif } break; @@ -4147,7 +4067,7 @@ void RowAggregationUM::doNotNullConstantAggregate(const ConstantAggData& aggData static_any::any valOut; rc = fRGContext.getFunction()->evaluate(&fRGContext, valOut); - fRGContext.setUserData(NULL); + fRGContext.setUserData(nullptr); if (rc == mcsv1sdk::mcsv1_UDAF::ERROR) { @@ -4157,7 +4077,7 @@ void RowAggregationUM::doNotNullConstantAggregate(const ConstantAggData& aggData // Set the returned value into the output row SetUDAFValue(valOut, colOut); - fRGContext.setDataFlags(NULL); + fRGContext.setDataFlags(nullptr); } break; @@ -4197,47 +4117,6 @@ void RowAggregationUM::setGroupConcatString() } } - -//------------------------------------------------------------------------------ -// Allocate a new data array for the output RowGroup -// return - true if successfully allocated -//------------------------------------------------------------------------------ -bool RowAggregationUM::newRowGroup() -{ - uint64_t allocSize = 0; - uint64_t memDiff = 0; - bool ret = false; - - allocSize = fRowGroupOut->getSizeWithStrings(); - - if (fKeyOnHeap) - memDiff = fKeyStore->getMemUsage() + fExtKeyMapAlloc->getMemUsage() - fLastMemUsage; - else - memDiff = fAlloc->getMemUsage() - fLastMemUsage; - - fLastMemUsage += memDiff; - - fTotalMemUsage += allocSize + memDiff; - - if (fRm->getMemory(allocSize + memDiff, fSessionMemLimit)) - { - boost::shared_ptr data(new RGData(*fRowGroupOut, AGG_ROWGROUP_SIZE)); - - if (data.get() != NULL) - { - fMaxTotalRowCount += AGG_ROWGROUP_SIZE; - fSecondaryRowDataVec.push_back(data); - fRowGroupOut->setData(data.get()); - fResultDataVec.push_back(data.get()); - fRowGroupOut->resetRowGroup(0); - - ret = true; - } - } - - return ret; -} - void RowAggregationUM::setInputOutput(const RowGroup& pRowGroupIn, RowGroup* pRowGroupOut) { RowAggregation::setInputOutput(pRowGroupIn, pRowGroupOut); @@ -4245,11 +4124,6 @@ void RowAggregationUM::setInputOutput(const RowGroup& pRowGroupIn, RowGroup* pRo if (fKeyOnHeap) { fKeyRG = fRowGroupIn.truncate(fGroupByCols.size()); - fKeyStore.reset(new KeyStorage(fKeyRG, &tmpRow)); - fExtEq.reset(new ExternalKeyEq(fKeyRG, fKeyStore.get(), fKeyRG.getColumnCount(), &tmpRow)); - fExtHash.reset(new ExternalKeyHasher(fKeyRG, fKeyStore.get(), fKeyRG.getColumnCount(), &tmpRow)); - fExtKeyMapAlloc.reset(new utils::STLPoolAllocator >()); - fExtKeyMap.reset(new ExtKeyMap_t(10, *fExtHash, *fExtEq, *fExtKeyMapAlloc)); } } @@ -4267,16 +4141,16 @@ void RowAggregationUM::setInputOutput(const RowGroup& pRowGroupIn, RowGroup* pRo //------------------------------------------------------------------------------ bool RowAggregationUM::nextRowGroup() { - bool more = (fResultDataVec.size() > 0); + fCurRGData = fRowAggStorage->getNextRGData(); + bool more = static_cast(fCurRGData); - if (more) - { - // load the top result set - fRowGroupOut->setData(fResultDataVec.back()); - fResultDataVec.pop_back(); - } + if (more) + { + // load the top result set + fRowGroupOut->setData(fCurRGData.get()); + } - return more; + return more; } @@ -4402,12 +4276,12 @@ void RowAggregationUMP2::updateEntry(const Row& rowIn, // colOut(in) - column in the output row group stores the sum // colAux(in) - column in the output row group stores the count //------------------------------------------------------------------------------ -void RowAggregationUMP2::doAvg(const Row& rowIn, int64_t colIn, int64_t colOut, int64_t colAux) +void RowAggregationUMP2::doAvg(const Row& rowIn, int64_t colIn, int64_t colOut, int64_t colAux, bool) { - if (isNull(&fRowGroupIn, rowIn, colIn) == true) + if (rowIn.isNullValue(colIn)) return; - int colDataType = (fRowGroupIn.getColTypes())[colIn]; + int colDataType = rowIn.getColType(colIn); long double valIn = 0; long double valOut = fRow.getLongDoubleField(colOut); bool isWideDataType = false; @@ -4635,7 +4509,7 @@ void RowAggregationUMP2::doUDAF(const Row& rowIn, if (rc == mcsv1sdk::mcsv1_UDAF::ERROR) { - RowUDAFFunctionCol* rowUDAF = dynamic_cast(fFunctionCols[funcColsIdx].get()); + auto* rowUDAF = dynamic_cast(fFunctionCols[funcColsIdx].get()); rowUDAF->bInterrupted = true; throw logging::IDBExcept(udafContextsColl[funcColsIdx].getErrorMessage(), logging::aggregateFuncErr); @@ -4677,7 +4551,8 @@ void RowAggregationDistinct::setInputOutput(const RowGroup& pRowGroupIn, RowGrou fRowGroupIn = fRowGroupDist; fRowGroupOut = pRowGroupOut; initialize(); - fDataForDist.reinit(fRowGroupDist, AGG_ROWGROUP_SIZE); + fDataForDist.reinit(fRowGroupDist, + RowAggStorage::getMaxRows(fRm ? fRm->getAllowDiskAggregation() : false)); fRowGroupDist.setData(&fDataForDist); fAggregator->setInputOutput(pRowGroupIn, &fRowGroupDist); } @@ -4705,7 +4580,7 @@ void RowAggregationDistinct::addRowGroup(const RowGroup* pRows) } -void RowAggregationDistinct::addRowGroup(const RowGroup* pRows, vector& inRows) +void RowAggregationDistinct::addRowGroup(const RowGroup* pRows, vector>& inRows) { fAggregator->addRowGroup(pRows, inRows); } @@ -4719,7 +4594,7 @@ void RowAggregationDistinct::doDistinctAggregation() { while (dynamic_cast(fAggregator.get())->nextRowGroup()) { - fRowGroupIn.setData(fAggregator.get()->getOutputRowGroup()->getRGData()); + fRowGroupIn.setData(fAggregator->getOutputRowGroup()->getRGData()); Row rowIn; fRowGroupIn.initRow(&rowIn); @@ -4733,15 +4608,15 @@ void RowAggregationDistinct::doDistinctAggregation() } -void RowAggregationDistinct::doDistinctAggregation_rowVec(vector& inRows) +void RowAggregationDistinct::doDistinctAggregation_rowVec(vector>& inRows) { Row rowIn; fRowGroupIn.initRow(&rowIn); for (uint64_t i = 0; i < inRows.size(); ++i) { - rowIn.setData(inRows[i]); - aggregateRow(rowIn); + rowIn.setData(inRows[i].first); + aggregateRow(rowIn, &inRows[i].second); } } @@ -4863,6 +4738,7 @@ RowAggregationSubDistinct::RowAggregationSubDistinct( boost::shared_ptr sessionLimit) : RowAggregationUM(rowAggGroupByCols, rowAggFunctionCols, r, sessionLimit) { + fKeyOnHeap = false; } @@ -4901,7 +4777,6 @@ void RowAggregationSubDistinct::setInputOutput(const RowGroup& pRowGroupIn, RowG void RowAggregationSubDistinct::addRowGroup(const RowGroup* pRows) { Row rowIn; - pair inserted; uint32_t i, j; pRows->initRow(&rowIn); @@ -4917,39 +4792,23 @@ void RowAggregationSubDistinct::addRowGroup(const RowGroup* pRows) } tmpRow = &fDistRow; - inserted = fAggMapPtr->insert(RowPosition(RowPosition::MSB, 0)); - - if (inserted.second) + if (fRowAggStorage->getTargetRow(fDistRow, fRow)) { - // if it was successfully inserted, fix the inserted values - if (++fTotalRowCount > fMaxTotalRowCount && !newRowGroup()) - { - throw logging::IDBExcept(logging::IDBErrorInfo::instance()-> - errorMsg(logging::ERR_AGGREGATION_TOO_BIG), logging::ERR_AGGREGATION_TOO_BIG); - } - - fRowGroupOut->getRow(fRowGroupOut->getRowCount(), &fRow); - fRowGroupOut->incRowCount(); copyRow(fDistRow, &fRow); - - // replace the key value with an equivalent copy, yes this is OK - const_cast(*(inserted.first)) = - RowPosition(fResultDataVec.size() - 1, fRowGroupOut->getRowCount() - 1); } } } -void RowAggregationSubDistinct::addRowGroup(const RowGroup* pRows, std::vector& inRows) +void RowAggregationSubDistinct::addRowGroup(const RowGroup* pRows, std::vector>& inRows) { Row rowIn; - pair inserted; uint32_t i, j; pRows->initRow(&rowIn); for (i = 0; i < inRows.size(); ++i, rowIn.nextRow()) { - rowIn.setData(inRows[i]); + rowIn.setData(inRows[i].first); /* TODO: We can make the functors a little smarter and avoid doing this copy before the * tentative insert */ @@ -4957,24 +4816,9 @@ void RowAggregationSubDistinct::addRowGroup(const RowGroup* pRows, std::vectorfInputColumnIndex); tmpRow = &fDistRow; - inserted = fAggMapPtr->insert(RowPosition(RowPosition::MSB, 0)); - - if (inserted.second) + if (fRowAggStorage->getTargetRow(fDistRow, fRow)) { - // if it was successfully inserted, fix the inserted values - if (++fTotalRowCount > fMaxTotalRowCount && !newRowGroup()) - { - throw logging::IDBExcept(logging::IDBErrorInfo::instance()-> - errorMsg(logging::ERR_AGGREGATION_TOO_BIG), logging::ERR_AGGREGATION_TOO_BIG); - } - - fRowGroupOut->getRow(fRowGroupOut->getRowCount(), &fRow); - fRowGroupOut->incRowCount(); copyRow(fDistRow, &fRow); - - // replace the key value with an equivalent copy, yes this is OK - const_cast(*(inserted.first)) = - RowPosition(fResultDataVec.size() - 1, fRowGroupOut->getRowCount() - 1); } } } @@ -5028,7 +4872,7 @@ RowAggregationMultiDistinct::RowAggregationMultiDistinct(const RowAggregationMul errorMsg(logging::ERR_AGGREGATION_TOO_BIG), logging::ERR_AGGREGATION_TOO_BIG); #endif - data.reset(new RGData(fSubRowGroups[i], AGG_ROWGROUP_SIZE)); + data.reset(new RGData(fSubRowGroups[i], RowAggStorage::getMaxRows(fRm ? fRm->getAllowDiskAggregation() : false))); fSubRowData.push_back(data); fSubRowGroups[i].setData(data.get()); agg.reset(rhs.fSubAggregators[i]->clone()); @@ -5074,7 +4918,7 @@ void RowAggregationMultiDistinct::addSubAggregator(const boost::shared_ptrgetAllowDiskAggregation() : false))); fSubRowData.push_back(data); //assert (agg->aggMapKeyLength() > 0); @@ -5099,7 +4943,7 @@ void RowAggregationMultiDistinct::addRowGroup(const RowGroup* pRows) // //------------------------------------------------------------------------------ void RowAggregationMultiDistinct::addRowGroup(const RowGroup* pRowGroupIn, - vector >& inRows) + vector> >& inRows) { for (uint64_t i = 0; i < fSubAggregators.size(); ++i) { @@ -5139,7 +4983,7 @@ void RowAggregationMultiDistinct::doDistinctAggregation() for (uint64_t j = 0; j < fRowGroupIn.getRowCount(); ++j, rowIn.nextRow()) { - aggregateRow(rowIn, rgContextColl); + aggregateRow(rowIn, nullptr, rgContextColl); } } } @@ -5150,12 +4994,12 @@ void RowAggregationMultiDistinct::doDistinctAggregation() } -void RowAggregationMultiDistinct::doDistinctAggregation_rowVec(vector >& inRows) +void RowAggregationMultiDistinct::doDistinctAggregation_rowVec(vector> >& inRows) { // backup the function column vector for finalize(). vector origFunctionCols = fFunctionCols; fOrigFunctionCols = &origFunctionCols; - + // aggregate data from each sub-aggregator to distinct aggregator for (uint64_t i = 0; i < fSubAggregators.size(); ++i) { @@ -5167,8 +5011,8 @@ void RowAggregationMultiDistinct::doDistinctAggregation_rowVec(vectorfResultDataVec[data.group]->getRow(data.row, &r); - row = &r; - } - - ret = row->hash(lastKeyCol); - //cout << "hash=" << ret << " keys=" << keyColCount << " row=" << r.toString() << endl; - return ret; -} - - -AggComparator::AggComparator(const Row& row, Row** tRow, uint32_t keyCount, RowAggregation* ra) - : agg(ra), tmpRow(tRow), r1(row), r2(row), lastKeyCol(keyCount - 1) -{ -} - -inline bool AggComparator::operator()(const RowPosition& d1, const RowPosition& d2) const -{ - bool ret; - Row* pr1, *pr2; - - if (d1.group == RowPosition::MSB) - pr1 = *tmpRow; - else - { - agg->fResultDataVec[d1.group]->getRow(d1.row, &r1); - pr1 = &r1; - } - - if (d2.group == RowPosition::MSB) - pr2 = *tmpRow; - else - { - agg->fResultDataVec[d2.group]->getRow(d2.row, &r2); - pr2 = &r2; - } - - ret = pr1->equals(*pr2, lastKeyCol); - //cout << "eq=" << (int) ret << " keys=" << keyColCount << ": r1=" << r1.toString() << - //"\n r2=" << r2.toString() << endl; - return ret; -} - - } // end of rowgroup namespace diff --git a/utils/rowgroup/rowaggregation.h b/utils/rowgroup/rowaggregation.h index 0e7bd3cb1..9537460e2 100644 --- a/utils/rowgroup/rowaggregation.h +++ b/utils/rowgroup/rowaggregation.h @@ -30,7 +30,8 @@ */ #include -#include +#include +#include #include #ifdef _MSC_VER #include @@ -54,6 +55,9 @@ #include "constantcolumn.h" +#include "resourcemanager.h" +#include "rowstorage.h" + // To do: move code that depends on joblist to a proper subsystem. namespace joblist { @@ -63,17 +67,6 @@ class ResourceManager; namespace rowgroup { - -struct RowPosition -{ - uint64_t group: 48; - uint64_t row: 16; - - static const uint64_t MSB = 0x800000000000ULL; //48th bit is set - inline RowPosition(uint64_t g, uint64_t r) : group(g), row(r) { } - inline RowPosition() { } -}; - /** @brief Enumerates aggregate functions supported by RowAggregation */ enum RowAggFunctionType @@ -143,9 +136,9 @@ struct RowAggGroupByCol * outputColIndex argument should be omitted if this GroupBy * column is not to be included in the output. */ - RowAggGroupByCol(int32_t inputColIndex, int32_t outputColIndex = -1) : + explicit RowAggGroupByCol(int32_t inputColIndex, int32_t outputColIndex = -1) : fInputColumnIndex(inputColIndex), fOutputColumnIndex(outputColIndex) {} - ~RowAggGroupByCol() {} + ~RowAggGroupByCol() = default; uint32_t fInputColumnIndex; uint32_t fOutputColumnIndex; @@ -184,7 +177,7 @@ struct RowAggFunctionCol int32_t inputColIndex, int32_t outputColIndex, int32_t auxColIndex = -1) : fAggFunction(aggFunction), fStatsFunction(stats), fInputColumnIndex(inputColIndex), fOutputColumnIndex(outputColIndex), fAuxColumnIndex(auxColIndex) {} - virtual ~RowAggFunctionCol() {} + virtual ~RowAggFunctionCol() = default; virtual void serialize(messageqcpp::ByteStream& bs) const; virtual void deserialize(messageqcpp::ByteStream& bs); @@ -237,10 +230,10 @@ struct RowUDAFFunctionCol : public RowAggFunctionCol bInterrupted(false) {} - virtual ~RowUDAFFunctionCol() {} + ~RowUDAFFunctionCol() override = default; - virtual void serialize(messageqcpp::ByteStream& bs) const; - virtual void deserialize(messageqcpp::ByteStream& bs); + void serialize(messageqcpp::ByteStream& bs) const override; + void deserialize(messageqcpp::ByteStream& bs) override; mcsv1sdk::mcsv1Context fUDAFContext; // The UDAF context bool bInterrupted; // Shared by all the threads @@ -312,104 +305,18 @@ struct ConstantAggData ConstantAggData() : fOp(ROWAGG_FUNCT_UNDEFINE), fIsNull(false) {} - ConstantAggData(const std::string& v, RowAggFunctionType f, bool n) : - fConstValue(v), fOp(f), fIsNull(n) + ConstantAggData(std::string v, RowAggFunctionType f, bool n) : + fConstValue(std::move(v)), fOp(f), fIsNull(n) {} - ConstantAggData(const std::string& v, const std::string u, RowAggFunctionType f, bool n) : - fConstValue(v), fUDAFName(u), fOp(f), fIsNull(n) + ConstantAggData(std::string v, std::string u, RowAggFunctionType f, bool n) : + fConstValue(std::move(v)), fUDAFName(std::move(u)), fOp(f), fIsNull(n) {} }; typedef boost::shared_ptr SP_ROWAGG_GRPBY_t; typedef boost::shared_ptr SP_ROWAGG_FUNC_t; -class RowAggregation; - -class AggHasher -{ -public: - AggHasher(const Row& row, Row** tRow, uint32_t keyCount, RowAggregation* ra); - inline uint64_t operator()(const RowPosition& p) const; - -private: - explicit AggHasher(); - RowAggregation* agg; - Row** tmpRow; - mutable Row r; - uint32_t lastKeyCol; -}; - -class AggComparator -{ -public: - AggComparator(const Row& row, Row** tRow, uint32_t keyCount, RowAggregation* ra); - inline bool operator()(const RowPosition&, const RowPosition&) const; - -private: - explicit AggComparator(); - RowAggregation* agg; - Row** tmpRow; - mutable Row r1, r2; - uint32_t lastKeyCol; -}; - -class KeyStorage -{ -public: - KeyStorage(const RowGroup& keyRG, Row** tRow); - - inline RowPosition addKey(); - inline uint64_t getMemUsage(); - -private: - Row row; - Row** tmpRow; - RowGroup rg; - std::vector storage; - uint64_t memUsage; - - friend class ExternalKeyEq; - friend class ExternalKeyHasher; -}; - -class ExternalKeyHasher -{ -public: - ExternalKeyHasher(const RowGroup& keyRG, KeyStorage* ks, uint32_t keyColCount, Row** tRow); - inline uint64_t operator()(const RowPosition& pos) const; - -private: - mutable Row row; - mutable Row** tmpRow; - uint32_t lastKeyCol; - KeyStorage* ks; -}; - -class ExternalKeyEq -{ -public: - ExternalKeyEq(const RowGroup& keyRG, KeyStorage* ks, uint32_t keyColCount, Row** tRow); - inline bool operator()(const RowPosition& pos1, const RowPosition& pos2) const; - -private: - mutable Row row1, row2; - mutable Row** tmpRow; - uint32_t lastKeyCol; - KeyStorage* ks; -}; - -typedef std::tr1::unordered_set > -RowAggMap_t; - -#if defined(__GNUC__) && (__GNUC__ == 4 && __GNUC_MINOR__ < 5) -typedef std::tr1::unordered_map > > ExtKeyMap_t; -#else -typedef std::tr1::unordered_map > > ExtKeyMap_t; -#endif - struct GroupConcat { // GROUP_CONCAT(DISTINCT col1, 'const', col2 ORDER BY col3 desc SEPARATOR 'sep') @@ -427,7 +334,7 @@ struct GroupConcat boost::shared_ptr fSessionMemLimit; std::string fTimeZone; - GroupConcat() : fRm(NULL) {} + GroupConcat() : fRm(nullptr) {} }; typedef boost::shared_ptr SP_GroupConcat; @@ -436,7 +343,7 @@ typedef boost::shared_ptr SP_GroupConcat; class GroupConcatAg { public: - GroupConcatAg(SP_GroupConcat&); + explicit GroupConcatAg(SP_GroupConcat&); virtual ~GroupConcatAg(); virtual void initialize() {}; @@ -446,7 +353,7 @@ public: void getResult(uint8_t*) {}; uint8_t* getResult() { - return NULL; + return nullptr; } protected: @@ -478,12 +385,14 @@ public: */ RowAggregation(); RowAggregation(const std::vector& rowAggGroupByCols, - const std::vector& rowAggFunctionCols); + const std::vector& rowAggFunctionCols, + joblist::ResourceManager* rm = nullptr, + boost::shared_ptr sessMemLimit = {}); RowAggregation(const RowAggregation& rhs); /** @brief RowAggregation default destructor */ - virtual ~RowAggregation(); + ~RowAggregation() override; /** @brief clone this object for multi-thread use */ @@ -551,28 +460,19 @@ public: * @parm pRowGroupIn(in) RowGroup to be added to aggregation. */ virtual void addRowGroup(const RowGroup* pRowGroupIn); - virtual void addRowGroup(const RowGroup* pRowGroupIn, std::vector& inRows); + virtual void addRowGroup(const RowGroup* pRowGroupIn, std::vector>& inRows); /** @brief Serialize RowAggregation object into a ByteStream. * * @parm bs(out) BytesStream that is to be written to. */ - void serialize(messageqcpp::ByteStream& bs) const; + void serialize(messageqcpp::ByteStream& bs) const override; /** @brief Unserialize RowAggregation object from a ByteStream. * * @parm bs(in) BytesStream that is to be read from. */ - void deserialize(messageqcpp::ByteStream& bs); - - /** @brief set the memory limit for RowAggregation - * - * @parm limit(in) memory limit for both Map and secondary RowGroups - */ - void setMaxMemory(uint64_t limit) - { - fMaxMemory = limit; - } + void deserialize(messageqcpp::ByteStream& bs) override; /** @brief load result set into byte stream * @@ -594,18 +494,12 @@ public: return fRowGroupOut; } - RowAggMap_t* mapPtr() - { - return fAggMapPtr; - } - std::vector& resultDataVec() - { - return fResultDataVec; - } + void append(RowAggregation* other); virtual void aggregateRow(Row& row, + const uint64_t* hash = nullptr, std::vector* rgContextColl = nullptr); - inline uint32_t aggMapKeyLength() + inline uint32_t aggMapKeyLength() const { return fAggMapKeyCount; } @@ -623,6 +517,16 @@ public: return &fRGContextColl; } + void finalAggregation() + { + return fRowAggStorage->finalize([this](Row& row) { mergeEntries(row);}, fRow); + } + + std::unique_ptr moveCurrentRGData() + { + return std::move(fCurRGData); + } + protected: virtual void initialize(); virtual void initMapData(const Row& row); @@ -630,10 +534,12 @@ protected: virtual void updateEntry(const Row& row, std::vector* rgContextColl = nullptr); + void mergeEntries(const Row& row); virtual void doMinMax(const Row&, int64_t, int64_t, int); virtual void doSum(const Row&, int64_t, int64_t, int); - virtual void doAvg(const Row&, int64_t, int64_t, int64_t); + virtual void doAvg(const Row&, int64_t, int64_t, int64_t, bool merge = false); virtual void doStatistics(const Row&, int64_t, int64_t, int64_t); + void mergeStatistics(const Row&, uint64_t colOut, uint64_t colAux); virtual void doBitOp(const Row&, int64_t, int64_t, int); virtual void doUDAF(const Row&, int64_t, @@ -647,12 +553,6 @@ protected: return true; } - virtual bool newRowGroup(); - virtual void clearAggMap() - { - if (fAggMapPtr) fAggMapPtr->clear(); - } - void resetUDAF(RowUDAFFunctionCol* rowUDAF); void resetUDAF(RowUDAFFunctionCol* rowUDAF, uint64_t funcColIdx); @@ -673,24 +573,19 @@ protected: inline void updateStringMinMax(std::string val1, std::string val2, int64_t col, int func); std::vector fGroupByCols; std::vector fFunctionCols; - RowAggMap_t* fAggMapPtr; uint32_t fAggMapKeyCount; // the number of columns that make up the key RowGroup fRowGroupIn; RowGroup* fRowGroupOut; + // for when the group by & distinct keys are not stored in the output rows + rowgroup::RowGroup fKeyRG; + Row fRow; Row fNullRow; Row* tmpRow; // used by the hashers & eq functors boost::scoped_array fNullRowData; - std::vector fResultDataVec; - uint64_t fTotalRowCount; - uint64_t fMaxTotalRowCount; - uint64_t fMaxMemory; - - RGData* fPrimaryRowData; - - std::vector > fSecondaryRowDataVec; + std::unique_ptr fRowAggStorage; // for support PM aggregation after PM hashjoin std::vector* fSmallSideRGs; @@ -700,28 +595,19 @@ protected: uint32_t fSmallSideCount; boost::scoped_array rowSmalls; - // for hashmap - boost::shared_ptr > fAlloc; - // for 8k poc RowGroup fEmptyRowGroup; RGData fEmptyRowData; Row fEmptyRow; - boost::scoped_ptr fHasher; - boost::scoped_ptr fEq; + bool fKeyOnHeap = false; std::string fTimeZone; - //TODO: try to get rid of these friend decl's. AggHasher & Comparator - //need access to rowgroup storage holding the rows to hash & ==. - friend class AggHasher; - friend class AggComparator; - // We need a separate copy for each thread. mcsv1sdk::mcsv1Context fRGContext; std::vector fRGContextColl; - + // These are handy for testing the actual type of static_any for UDAF static const static_any::any& charTypeId; static const static_any::any& scharTypeId; @@ -742,6 +628,10 @@ protected: // For UDAF along with with multiple distinct columns std::vector* fOrigFunctionCols; + + joblist::ResourceManager* fRm = nullptr; + boost::shared_ptr fSessionMemLimit; + std::unique_ptr fCurRGData; }; //------------------------------------------------------------------------------ @@ -764,11 +654,11 @@ public: /** @brief RowAggregationUM default destructor */ - ~RowAggregationUM(); + ~RowAggregationUM() override; /** @brief Denotes end of data insertion following multiple calls to addRowGroup(). */ - void endOfInput(); + void endOfInput() override; /** @brief Finializes the result set before sending back to the front end. */ @@ -805,7 +695,7 @@ public: { return fRm; } - inline virtual RowAggregationUM* clone() const + inline RowAggregationUM* clone() const override { return new RowAggregationUM (*this); } @@ -832,22 +722,18 @@ public: return fGroupConcat; } - void aggregateRow(Row&, - std::vector* rgContextColl = nullptr) override; - virtual void aggReset(); + void aggReset() override; - void setInputOutput(const RowGroup& pRowGroupIn, RowGroup* pRowGroupOut); + void setInputOutput(const RowGroup& pRowGroupIn, RowGroup* pRowGroupOut) override; protected: + // virtual methods from base void initialize() override; + + void attachGroupConcatAg() override; void updateEntry(const Row& row, std::vector* rgContextColl = nullptr) override; - - void aggregateRowWithRemap(Row&, - std::vector* rgContextColl = nullptr); - - void attachGroupConcatAg(); - bool countSpecial(const RowGroup* pRG) + bool countSpecial(const RowGroup* pRG) override { fRow.setIntField<8>( fRow.getIntField<8>( @@ -856,8 +742,6 @@ protected: return true; } - bool newRowGroup(); - // calculate the average after all rows received. UM only function. void calculateAvgColumns(); @@ -889,7 +773,6 @@ protected: virtual void setGroupConcatString(); bool fHasAvg; - bool fKeyOnHeap; bool fHasStatsFunc; bool fHasUDAF; @@ -902,8 +785,6 @@ protected: * the memory from rm in that order. */ uint64_t fTotalMemUsage; - joblist::ResourceManager* fRm; - // @bug3475, aggregate(constant), sum(0), count(null), etc std::vector fConstantAggregate; @@ -912,18 +793,8 @@ protected: std::vector fGroupConcatAg; std::vector fFunctionColGc; - // for when the group by & distinct keys are not stored in the output rows - rowgroup::RowGroup fKeyRG; - boost::scoped_ptr fExtEq; - boost::scoped_ptr fExtHash; - boost::scoped_ptr fKeyStore; - boost::scoped_ptr > > fExtKeyMapAlloc; - boost::scoped_ptr fExtKeyMap; - - boost::shared_ptr fSessionMemLimit; private: uint64_t fLastMemUsage; - uint32_t fNextRGIndex; }; @@ -951,8 +822,8 @@ public: /** @brief RowAggregationUMP2 default destructor */ - ~RowAggregationUMP2(); - inline virtual RowAggregationUMP2* clone() const + ~RowAggregationUMP2() override; + inline RowAggregationUMP2* clone() const override { return new RowAggregationUMP2 (*this); } @@ -961,17 +832,17 @@ protected: // virtual methods from base void updateEntry(const Row& row, std::vector* rgContextColl = nullptr) override; - void doAvg(const Row&, int64_t, int64_t, int64_t); - void doStatistics(const Row&, int64_t, int64_t, int64_t); - void doGroupConcat(const Row&, int64_t, int64_t); - void doBitOp(const Row&, int64_t, int64_t, int); + void doAvg(const Row&, int64_t, int64_t, int64_t, bool merge = false) override; + void doStatistics(const Row&, int64_t, int64_t, int64_t) override; + void doGroupConcat(const Row&, int64_t, int64_t) override; + void doBitOp(const Row&, int64_t, int64_t, int) override; void doUDAF(const Row&, int64_t, int64_t, int64_t, uint64_t& funcColsIdx, std::vector* rgContextColl = nullptr) override; - bool countSpecial(const RowGroup* pRG) + bool countSpecial(const RowGroup* pRG) override { return false; } @@ -1002,18 +873,18 @@ public: /** @brief RowAggregationDistinct default destructor */ - ~RowAggregationDistinct(); + ~RowAggregationDistinct() override; /** @brief Add an aggregator for pre-DISTINCT aggregation */ void addAggregator(const boost::shared_ptr& agg, const RowGroup& rg); - void setInputOutput(const RowGroup& pRowGroupIn, RowGroup* pRowGroupOut); + void setInputOutput(const RowGroup& pRowGroupIn, RowGroup* pRowGroupOut) override; virtual void doDistinctAggregation(); - virtual void doDistinctAggregation_rowVec(std::vector& inRows); - void addRowGroup(const RowGroup* pRowGroupIn); - void addRowGroup(const RowGroup* pRowGroupIn, std::vector& inRows); + virtual void doDistinctAggregation_rowVec(std::vector>& inRows); + void addRowGroup(const RowGroup* pRowGroupIn) override; + void addRowGroup(const RowGroup* pRowGroupIn, std::vector>& inRows) override; // multi-threade debug boost::shared_ptr& aggregator() @@ -1022,7 +893,7 @@ public: } void aggregator(boost::shared_ptr aggregator) { - fAggregator = aggregator; + fAggregator = std::move(aggregator); } RowGroup& rowGroupDist() { @@ -1032,7 +903,7 @@ public: { fRowGroupDist = rowGroupDist; } - inline virtual RowAggregationDistinct* clone() const + inline RowAggregationDistinct* clone() const override { return new RowAggregationDistinct (*this); } @@ -1067,20 +938,20 @@ public: /** @brief RowAggregationSubDistinct default destructor */ - ~RowAggregationSubDistinct(); + ~RowAggregationSubDistinct() override; - void setInputOutput(const RowGroup& pRowGroupIn, RowGroup* pRowGroupOut); - void addRowGroup(const RowGroup* pRowGroupIn); - inline virtual RowAggregationSubDistinct* clone() const + void setInputOutput(const RowGroup& pRowGroupIn, RowGroup* pRowGroupOut) override; + void addRowGroup(const RowGroup* pRowGroupIn) override; + inline RowAggregationSubDistinct* clone() const override { return new RowAggregationSubDistinct (*this); } - void addRowGroup(const RowGroup* pRowGroupIn, std::vector& inRow); + void addRowGroup(const RowGroup* pRowGroupIn, std::vector>& inRow) override; protected: // virtual methods from RowAggregationUM - void doGroupConcat(const Row&, int64_t, int64_t); + void doGroupConcat(const Row&, int64_t, int64_t) override; // for groupby columns and the aggregated distinct column Row fDistRow; @@ -1108,7 +979,7 @@ public: /** @brief RowAggregationMultiDistinct default destructor */ - ~RowAggregationMultiDistinct(); + ~RowAggregationMultiDistinct() override; /** @brief Add sub aggregators */ @@ -1116,21 +987,21 @@ public: const RowGroup& rg, const std::vector& funct); - void setInputOutput(const RowGroup& pRowGroupIn, RowGroup* pRowGroupOut); + void setInputOutput(const RowGroup& pRowGroupIn, RowGroup* pRowGroupOut) override; using RowAggregationDistinct::addRowGroup; - void addRowGroup(const RowGroup* pRowGroupIn); + void addRowGroup(const RowGroup* pRowGroupIn) override; using RowAggregationDistinct::doDistinctAggregation; - virtual void doDistinctAggregation(); + void doDistinctAggregation() override; using RowAggregationDistinct::doDistinctAggregation_rowVec; - virtual void doDistinctAggregation_rowVec(std::vector >& inRows); + virtual void doDistinctAggregation_rowVec(std::vector> >& inRows); - inline virtual RowAggregationMultiDistinct* clone() const + inline RowAggregationMultiDistinct* clone() const override { return new RowAggregationMultiDistinct (*this); } - void addRowGroup(const RowGroup* pRowGroupIn, std::vector >& inRows); + void addRowGroup(const RowGroup* pRowGroupIn, std::vector>>& inRows); std::vector >& subAggregators() { diff --git a/utils/rowgroup/rowgroup.cpp b/utils/rowgroup/rowgroup.cpp index be46b2929..c35116ccc 100644 --- a/utils/rowgroup/rowgroup.cpp +++ b/utils/rowgroup/rowgroup.cpp @@ -32,6 +32,7 @@ using namespace std; #include +#include using namespace boost; #include "bytestream.h" @@ -405,6 +406,7 @@ RGData::RGData(const RowGroup& rg, uint32_t rowCount) */ memset(rowData.get(), 0, rg.getDataSize(rowCount)); // XXXPAT: make valgrind happy temporarily #endif + memset(rowData.get(), 0, rg.getDataSize(rowCount)); // XXXPAT: make valgrind happy temporarily } RGData::RGData(const RowGroup& rg) @@ -481,7 +483,7 @@ void RGData::serialize(ByteStream& bs, uint32_t amount) const bs << (uint8_t) 0; } -void RGData::deserialize(ByteStream& bs, bool hasLenField) +void RGData::deserialize(ByteStream& bs, uint32_t defAmount) { uint32_t amount, sig; uint8_t* buf; @@ -493,7 +495,7 @@ void RGData::deserialize(ByteStream& bs, bool hasLenField) { bs >> sig; bs >> amount; - rowData.reset(new uint8_t[amount]); + rowData.reset(new uint8_t[std::max(amount, defAmount)]); buf = bs.buf(); memcpy(rowData.get(), buf, amount); bs.advance(amount); @@ -577,12 +579,13 @@ Row& Row::operator=(const Row& r) return *this; } -string Row::toString() const +string Row::toString(uint32_t rownum) const { ostringstream os; uint32_t i; //os << getRid() << ": "; + os << "[" << std::setw(5) << rownum << std::setw(0) << "]: "; os << (int) useStringTable << ": "; for (i = 0; i < columnCount; i++) @@ -1447,7 +1450,7 @@ uint32_t RowGroup::getColumnCount() const return columnCount; } -string RowGroup::toString() const +string RowGroup::toString(const std::vector& used) const { ostringstream os; ostream_iterator oIter1(os, "\t"); @@ -1479,6 +1482,8 @@ string RowGroup::toString() const os << "uses a string table\n"; else os << "doesn't use a string table\n"; + if (!used.empty()) + os << "sparse\n"; //os << "strings = " << hex << (int64_t) strings << "\n"; //os << "data = " << (int64_t) data << "\n" << dec; @@ -1488,14 +1493,25 @@ string RowGroup::toString() const initRow(&r); getRow(0, &r); os << "rowcount = " << getRowCount() << endl; + if (!used.empty()) + { + uint64_t cnt = std::accumulate(used.begin(), used.end(), 0ULL, + [](uint64_t a, uint64_t bits) { + return a + __builtin_popcountll(bits); + }); + os << "sparse row count = " << cnt << endl; + } os << "base rid = " << getBaseRid() << endl; os << "status = " << getStatus() << endl; os << "dbroot = " << getDBRoot() << endl; os << "row data...\n"; - for (uint32_t i = 0; i < getRowCount(); i++) + uint32_t max_cnt = used.empty() ? getRowCount() : (used.size() * 64); + for (uint32_t i = 0; i < max_cnt; i++) { - os << r.toString() << endl; + if (!used.empty() && !(used[i/64] & (1ULL << (i%64)))) + continue; + os << r.toString(i) << endl; r.nextRow(); } } diff --git a/utils/rowgroup/rowgroup.h b/utils/rowgroup/rowgroup.h index 341d6feb7..6ba25df3c 100644 --- a/utils/rowgroup/rowgroup.h +++ b/utils/rowgroup/rowgroup.h @@ -270,7 +270,7 @@ public: // the 'hasLengthField' is there b/c PM aggregation (and possibly others) currently sends // inline data with a length field. Once that's converted to string table format, that // option can go away. - void deserialize(messageqcpp::ByteStream&, bool hasLengthField = false); // returns the # of bytes read + void deserialize(messageqcpp::ByteStream&, uint32_t amount = 0); // returns the # of bytes read inline uint64_t getStringTableMemUsage(); void clear(); @@ -531,7 +531,7 @@ public: template inline void copyBinaryField(Row& dest, uint32_t destIndex, uint32_t srcIndex) const; - std::string toString() const; + std::string toString(uint32_t rownum = 0) const; std::string toCSV() const; /* These fcns are used only in joins. The RID doesn't matter on the side that @@ -1537,7 +1537,7 @@ public: RGData duplicate(); // returns a copy of the attached RGData - std::string toString() const; + std::string toString(const std::vector& used = {}) const; /** operator+= * diff --git a/utils/rowgroup/rowstorage.cpp b/utils/rowgroup/rowstorage.cpp new file mode 100644 index 000000000..b7a64e3f8 --- /dev/null +++ b/utils/rowgroup/rowstorage.cpp @@ -0,0 +1,2237 @@ +/* Copyright (C) 2021 MariaDB Corporation + + This program is free software; you can redistribute it and/or + modify it under the terms of the GNU General Public License + as published by the Free Software Foundation; version 2 of + the License. + + This program is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + GNU General Public License for more details. + + You should have received a copy of the GNU General Public License + along with this program; if not, write to the Free Software + Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, + MA 02110-1301, USA. */ + +#include +#include +#include +#include "rowgroup.h" +#include +#include +#include "rowstorage.h" +#include "robin_hood.h" + +namespace +{ + +int writeData(int fd, const char* buf, size_t sz) +{ + if (sz == 0) + return 0; + + auto to_write = sz; + while (to_write > 0) + { + auto r = write(fd, buf + sz - to_write, to_write); + if (UNLIKELY(r < 0)) + { + if (errno == EAGAIN) + continue; + + return errno; + } + assert(size_t(r) <= to_write); + to_write -= r; + } + + return 0; +} + +int readData(int fd, char* buf, size_t sz) +{ + if (sz == 0) + return 0; + + auto to_read = sz; + while (to_read > 0) + { + auto r = read(fd, buf + sz - to_read, to_read); + if (UNLIKELY(r < 0)) + { + if (errno == EAGAIN) + continue; + + return errno; + } + + assert(size_t(r) <= to_read); + to_read -= r; + } + + return 0; +} + +std::string errorString(int errNo) +{ + char tmp[1024]; + auto* buf = strerror_r(errNo, tmp, sizeof(tmp)); + return {buf}; +} + +inline uint64_t hashData(const void* ptr, uint32_t len, uint64_t x = 0ULL) +{ + static constexpr uint64_t m = 0xc6a4a7935bd1e995ULL; + static constexpr uint64_t seed = 0xe17a1465ULL; + static constexpr unsigned int r = 47; + + auto const* const data64 = static_cast(ptr); + uint64_t h = seed ^ (len * m); + + std::size_t const n_blocks = len / 8; + if (x) + { + x *= m; + x ^= x >> r; + x *= m; + h ^= x; + h *= m; + } + for (std::size_t i = 0; i < n_blocks; ++i) { + uint64_t k; + memcpy(&k, data64 + i, sizeof(k)); + + k *= m; + k ^= k >> r; + k *= m; + + h ^= k; + h *= m; + } + + auto const* const data8 = reinterpret_cast(data64 + n_blocks); + switch (len & 7U) { + case 7: + h ^= static_cast(data8[6]) << 48U; + // FALLTHROUGH + case 6: + h ^= static_cast(data8[5]) << 40U; + // FALLTHROUGH + case 5: + h ^= static_cast(data8[4]) << 32U; + // FALLTHROUGH + case 4: + h ^= static_cast(data8[3]) << 24U; + // FALLTHROUGH + case 3: + h ^= static_cast(data8[2]) << 16U; + // FALLTHROUGH + case 2: + h ^= static_cast(data8[1]) << 8U; + // FALLTHROUGH + case 1: + h ^= static_cast(data8[0]); + h *= m; + // FALLTHROUGH + default: + break; + } + + h ^= h >> r; + h *= m; + h ^= h >> r; + + return h; +} + +} // anonymous namespace + +namespace rowgroup +{ + +uint64_t hashRow(const rowgroup::Row& r, std::size_t lastCol) +{ + uint64_t ret = 0; + if (lastCol >= r.getColumnCount()) + return 0; + + datatypes::MariaDBHasher h; + bool strHashUsed = false; + for (uint32_t i = 0; i <= lastCol; ++i) + { + switch (r.getColType(i)) + { + case execplan::CalpontSystemCatalog::CHAR: + case execplan::CalpontSystemCatalog::VARCHAR: + case execplan::CalpontSystemCatalog::BLOB: + case execplan::CalpontSystemCatalog::TEXT: + h.add(r.getCharset(i), r.getConstString(i)); + strHashUsed = true; + break; + + default: + ret = hashData(r.getData() + r.getOffset(i), r.getColumnWidth(i), ret); + break; + } + } + + if (strHashUsed) + { + uint64_t strhash = h.finalize(); + ret = hashData(&strhash, sizeof(strhash), ret); + } + + return ret; +} + +/** @brief NoOP interface to LRU-cache used by RowGroupStorage & HashStorage + */ +struct LRUIface +{ + using List = std::list; + + virtual ~LRUIface() = default; + /** @brief Put an ID to cache or set it as last used */ + virtual void add(uint64_t) {} + /** @brief Remove an ID from cache */ + virtual void remove(uint64_t) {} + /** @brief Get iterator of the most recently used ID */ + virtual List::const_reverse_iterator begin() const { return List::const_reverse_iterator(); } + /** @brief Get iterator after the latest ID */ + virtual List::const_reverse_iterator end() const { return List::const_reverse_iterator(); } + /** @brief Get iterator of the latest ID */ + virtual List::const_iterator rbegin() const { return {}; } + /** @brief Get iterator after the most recently used ID */ + virtual List::const_iterator rend() const { return {}; } + + virtual void clear() {} + virtual std::size_t size() const { return 0; } + virtual bool empty() const { return true; } + virtual LRUIface* clone() const { return new LRUIface(); } +}; + +struct LRU : public LRUIface +{ + ~LRU() override { + fMap.clear(); + fList.clear(); + } + inline void add(uint64_t rgid) final { + auto it = fMap.find(rgid); + if (it != fMap.end()) { + fList.erase(it->second); + } + fMap[rgid] = fList.insert(fList.end(), rgid); + } + + inline void remove(uint64_t rgid) final { + auto it = fMap.find(rgid); + if (UNLIKELY(it != fMap.end())) { + fList.erase(it->second); + fMap.erase(it); + } + } + + inline List::const_reverse_iterator begin() const final { return fList.crbegin(); } + inline List::const_reverse_iterator end() const final { return fList.crend(); } + inline List::const_iterator rbegin() const final { return fList.cbegin(); } + inline List::const_iterator rend() const final { return fList.cend(); } + inline void clear() final { + fMap.clear(); + fList.clear(); + } + + size_t size() const final { return fMap.size(); } + bool empty() const final { return fList.empty(); } + + LRUIface* clone() const final { return new LRU(); } + + robin_hood::unordered_flat_map fMap; + List fList; +}; + +/** @brief Some service wrapping around ResourceManager (or NoOP) */ +class MemManager +{ +public: + MemManager() {} + virtual ~MemManager() { + release(fMemUsed); + } + + bool acquire(std::size_t amount) + { + return acquireImpl(amount); + } + void release(ssize_t amount = 0) + { + // in some cases it tries to release more memory than acquired, ie create + // new rowgroup, acquire maximum size (w/o strings), add some rows with + // strings and finally release the actual size of RG with strings + if (amount == 0 || amount > fMemUsed) + amount = fMemUsed; + releaseImpl(amount); + } + + ssize_t getUsed() const { return fMemUsed; } + virtual int64_t getFree() const + { + return std::numeric_limits::max(); + } + + virtual bool isStrict() const { return false; } + + virtual MemManager* clone() const { return new MemManager(); } + + virtual joblist::ResourceManager* getResourceManaged() { return nullptr; } + virtual boost::shared_ptr getSessionLimit() { return {}; } + +protected: + virtual bool acquireImpl(std::size_t amount) { fMemUsed += amount; return true; } + virtual void releaseImpl(std::size_t amount) { fMemUsed -= amount; } + ssize_t fMemUsed = 0; +}; + +class RMMemManager : public MemManager +{ +public: + RMMemManager(joblist::ResourceManager* rm, boost::shared_ptr sl, bool wait = true, bool strict = true) + : fRm(rm), fSessLimit(std::move(sl)), fWait(wait), fStrict(strict) + {} + + ~RMMemManager() override + { + release(fMemUsed); + fMemUsed = 0; + } + + int64_t getFree() const final + { + return std::min(fRm->availableMemory(), *fSessLimit); + } + + bool isStrict() const final { return fStrict; } + + MemManager* clone() const final + { + return new RMMemManager(fRm, fSessLimit, fWait, fStrict); + } + + joblist::ResourceManager* getResourceManaged() override { return fRm; } + boost::shared_ptr getSessionLimit() override { return fSessLimit; } + +protected: + bool acquireImpl(size_t amount) final + { + MemManager::acquireImpl(amount); + if (!fRm->getMemory(amount, fSessLimit, fWait) && fStrict) + { + return false; + } + + return true; + } + void releaseImpl(size_t amount) override { + MemManager::releaseImpl(amount); + fRm->returnMemory(amount, fSessLimit); + } + +private: + joblist::ResourceManager* fRm = nullptr; + boost::shared_ptr fSessLimit; + const bool fWait; + const bool fStrict; +}; + +/** @brief Storage for RGData with LRU-cache & memory management + */ +class RowGroupStorage +{ +public: + using RGDataStorage = std::vector>; + +public: + /** @brief Default constructor + * + * @param tmpDir(in) directory for tmp data + * @param rowGroupOut(in,out) RowGroup metadata + * @param maxRows(in) number of rows per rowgroup + * @param rm ResourceManager to use or nullptr if we don't + * need memory accounting + * @param sessLimit session memory limit + * @param wait shall we wait a bit if we haven't enough memory + * right now? + * @param strict true -> throw an exception if not enough memory + * false -> deal with it later + */ + RowGroupStorage(const std::string& tmpDir, + RowGroup* rowGroupOut, + size_t maxRows, + joblist::ResourceManager* rm = nullptr, + boost::shared_ptr sessLimit = {}, + bool wait = false, + bool strict = false) + : fRowGroupOut(rowGroupOut) + , fMaxRows(maxRows) + , fRGDatas() + , fUniqId(this) + , fTmpDir(tmpDir) + { + if (rm) + { + fMM.reset(new RMMemManager(rm, sessLimit, wait, strict)); + if (!wait && !strict) + { + fLRU = std::unique_ptr(new LRU()); + } + else + { + fLRU = std::unique_ptr(new LRUIface()); + } + } + else + { + fMM.reset(new MemManager()); + fLRU = std::unique_ptr(new LRUIface()); + } + auto* curRG = new RGData(*fRowGroupOut, fMaxRows); + fRowGroupOut->setData(curRG); + fRowGroupOut->resetRowGroup(0); + fRGDatas.emplace_back(curRG); + fMM->acquire(fRowGroupOut->getSizeWithStrings(fMaxRows)); + } + + ~RowGroupStorage() = default; + + ssize_t getAproxRGSize() const + { + return fRowGroupOut->getSizeWithStrings(fMaxRows); + } + + /** @brief Take away RGDatas from another RowGroupStorage + * + * If some of the RGDatas is not in the memory do not load them, + * just rename dump file to match new RowGroupStorage pattern + * + * @param o RowGroupStorage to take from + */ + void append(std::unique_ptr o) { return append(o.get()); } + void append(RowGroupStorage* o) + { + std::unique_ptr rgd; + std::string ofname; + while (o->getNextRGData(rgd, ofname)) + { + fRGDatas.push_back(std::move(rgd)); + uint64_t rgid = fRGDatas.size() - 1; + if (fRGDatas[rgid]) + { + fRowGroupOut->setData(fRGDatas[rgid].get()); + int64_t memSz = fRowGroupOut->getSizeWithStrings(fMaxRows); + if (!fMM->acquire(memSz)) + { + throw logging::IDBExcept(logging::IDBErrorInfo::instance()->errorMsg( + logging::ERR_AGGREGATION_TOO_BIG), + logging::ERR_AGGREGATION_TOO_BIG); + } + + fLRU->add(rgid); + } + else + { + auto r = rename(ofname.c_str(), makeRGFilename(rgid).c_str()); + if (UNLIKELY(r < 0)) + { + throw logging::IDBExcept(logging::IDBErrorInfo::instance()->errorMsg( + logging::ERR_DISKAGG_FILEIO_ERROR, errorString(errno)), + logging::ERR_DISKAGG_FILEIO_ERROR); + } + } + rgd.reset(); + ofname.clear(); + } + } + + /** @brief Returns next RGData, load it from disk if necessary. + * + * @returns pointer to the next RGData or empty pointer if there is nothing + */ + std::unique_ptr getNextRGData() + { + while (!fRGDatas.empty()) + { + uint64_t rgid = fRGDatas.size() - 1; + if (!fRGDatas[rgid]) + loadRG(rgid, fRGDatas[rgid], true); + unlink(makeRGFilename(rgid).c_str()); + + auto rgdata = std::move(fRGDatas[rgid]); + fRGDatas.pop_back(); + + fRowGroupOut->setData(rgdata.get()); + int64_t memSz = fRowGroupOut->getSizeWithStrings(fMaxRows); + fMM->release(memSz); + fLRU->remove(rgid); + if (fRowGroupOut->getRowCount() == 0) + continue; + return rgdata; + } + + return {}; + } + + void initRow(Row& row) const + { + fRowGroupOut->initRow(&row); + } + + /** @brief Get the row at the specified position, loading corresponding RGData if needed. + * + * @param idx(in) index (from 0) of the row + * @param row(out) resulting row + */ + void getRow(uint64_t idx, Row& row) + { + uint64_t rgid = idx / fMaxRows; + uint64_t rid = idx % fMaxRows; + if (UNLIKELY(!fRGDatas[rgid])) + { + loadRG(rgid); + } + fRGDatas[rgid]->getRow(rid, &row); + fLRU->add(rgid); + } + + /** @brief Return a row and an index at the first free position. + * + * @param idx(out) index of the row + * @param row(out) the row itself + */ + void putRow(uint64_t& idx, Row& row) + { + bool need_new = false; + if (UNLIKELY(fRGDatas.empty())) + { + need_new = true; + } + else if (UNLIKELY(!fRGDatas[fCurRgid])) + { + need_new = true; + } + else + { + fRowGroupOut->setData(fRGDatas[fCurRgid].get()); + if (UNLIKELY(fRowGroupOut->getRowCount() >= fMaxRows)) + need_new = true; + } + + if (UNLIKELY(need_new)) + { + for (auto rgid : *fLRU) + { + if (LIKELY(static_cast(fRGDatas[rgid]))) + { + fRowGroupOut->setData(fRGDatas[rgid].get()); + if (fRowGroupOut->getRowCount() < fMaxRows) + { + fCurRgid = rgid; + need_new = false; + break; + } + } + } + } + + if (UNLIKELY(need_new)) + { + auto memSz = fRowGroupOut->getSizeWithStrings(fMaxRows); + if (!fMM->acquire(memSz)) + { + throw logging::IDBExcept(logging::IDBErrorInfo::instance()->errorMsg( + logging::ERR_AGGREGATION_TOO_BIG), + logging::ERR_AGGREGATION_TOO_BIG); + } + auto* curRG = new RGData(*fRowGroupOut, fMaxRows); + fRowGroupOut->setData(curRG); + fRowGroupOut->resetRowGroup(0); + fRGDatas.emplace_back(curRG); + fCurRgid = fRGDatas.size() - 1; + } + + fLRU->add(fCurRgid); + idx = fCurRgid * fMaxRows + fRowGroupOut->getRowCount(); + fRowGroupOut->getRow(fRowGroupOut->getRowCount(), &row); + fRowGroupOut->incRowCount(); + } + + /** @brief Create a row at the specified position. + * + * Used only for key rows in case of external keys. Indexes of data row and + * corresponding key row are always the same. + * + * @param idx(in) index to create row + * @param row(out) row itself + */ + void putKeyRow(uint64_t idx, Row& row) + { + uint64_t rgid = idx / fMaxRows; + uint64_t rid = idx % fMaxRows; + + while (rgid >= fRGDatas.size()) + { + int64_t memSz = fRowGroupOut->getSizeWithStrings(fMaxRows); + if (!fMM->acquire(memSz)) + { + throw logging::IDBExcept(logging::IDBErrorInfo::instance()->errorMsg( + logging::ERR_AGGREGATION_TOO_BIG), + logging::ERR_AGGREGATION_TOO_BIG); + } + auto* curRG = new RGData(*fRowGroupOut, fMaxRows); + fRowGroupOut->setData(curRG); + fRowGroupOut->resetRowGroup(0); + fRGDatas.emplace_back(curRG); + fCurRgid = fRGDatas.size() - 1; + fLRU->add(fCurRgid); + } + + if (UNLIKELY(!fRGDatas[rgid])) + { + loadRG(rgid); + } + else + { + fRowGroupOut->setData(fRGDatas[rgid].get()); + } + + fLRU->add(rgid); + + assert(rid == fRowGroupOut->getRowCount()); + fRowGroupOut->getRow(fRowGroupOut->getRowCount(), &row); + fRowGroupOut->incRowCount(); + } + + /** @brief Dump the oldest RGData to disk, freeing memory + * + * @returns true if any RGData was dumped + */ + bool dump() + { + // Always leave at least 2 RG as this is the minimum size of the hashmap + constexpr size_t MIN_INMEMORY = 2; + if (fLRU->size() <= MIN_INMEMORY) + { + return false; + } + size_t moved = 0; + auto it = fLRU->rbegin(); + while (LIKELY(it != fLRU->rend())) + { + if (fLRU->size() <= MIN_INMEMORY) + return false; + uint64_t rgid = *it; + if (UNLIKELY(!fRGDatas[rgid])) + { + ++it; + fLRU->remove(rgid); + continue; + } + fRowGroupOut->setData(fRGDatas[rgid].get()); + if (moved <= MIN_INMEMORY && fRowGroupOut->getRowCount() < fMaxRows) + { + ++it; + ++moved; + fLRU->add(rgid); + continue; + } + saveRG(rgid); + fLRU->remove(rgid); + fRGDatas[rgid].reset(); + return true; + } + return false; + } + + /** @brief Dump all data, clear state and start over */ + void startNewGeneration() + { + dumpAll(); + fLRU->clear(); + fMM->release(); + fRGDatas.clear(); + ++fGeneration; + } + + /** @brief Save "finalized" bitmap to disk for future use */ + void dumpFinalizedInfo() const + { + auto fname = makeFinalizedFilename(); + int fd = open(fname.c_str(), + O_WRONLY | O_CREAT | O_TRUNC, 0644); + if (UNLIKELY(fd < 0)) + { + throw logging::IDBExcept(logging::IDBErrorInfo::instance()->errorMsg( + logging::ERR_DISKAGG_FILEIO_ERROR, errorString(errno)), + logging::ERR_DISKAGG_FILEIO_ERROR); + } + uint64_t sz = fRGDatas.size(); + uint64_t finsz = fFinalizedRows.size(); + + int errNo; + if ((errNo = writeData(fd, (const char*)&sz, sizeof(sz))) != 0 || + (errNo = writeData(fd, (const char*)&finsz, sizeof(finsz)) != 0) || + (errNo = writeData(fd, (const char*)fFinalizedRows.data(), finsz * sizeof(uint64_t)) != 0)) + { + close(fd); + unlink(fname.c_str()); + throw logging::IDBExcept(logging::IDBErrorInfo::instance()->errorMsg( + logging::ERR_DISKAGG_FILEIO_ERROR, errorString(errNo)), + logging::ERR_DISKAGG_FILEIO_ERROR); + } + close(fd); + } + + /** @brief Load "finalized" bitmap */ + void loadFinalizedInfo() + { + auto fname = makeFinalizedFilename(); + int fd = open(fname.c_str(), O_RDONLY); + if (fd < 0) + { + throw logging::IDBExcept(logging::IDBErrorInfo::instance()->errorMsg( + logging::ERR_DISKAGG_FILEIO_ERROR, errorString(errno)), + logging::ERR_DISKAGG_FILEIO_ERROR); + } + uint64_t sz; + uint64_t finsz; + int errNo; + if ((errNo = readData(fd, (char*)&sz, sizeof(sz)) != 0) || + (errNo = readData(fd, (char*)&finsz, sizeof(finsz)) != 0)) + { + close(fd); + unlink(fname.c_str()); + throw logging::IDBExcept(logging::IDBErrorInfo::instance()->errorMsg( + logging::ERR_DISKAGG_FILEIO_ERROR, errorString(errNo)), + logging::ERR_DISKAGG_FILEIO_ERROR); + } + fRGDatas.resize(sz); + fFinalizedRows.resize(finsz); + if ((errNo = readData(fd, (char*)fFinalizedRows.data(), finsz * sizeof(uint64_t))) != 0) + { + close(fd); + unlink(fname.c_str()); + throw logging::IDBExcept(logging::IDBErrorInfo::instance()->errorMsg( + logging::ERR_DISKAGG_FILEIO_ERROR, errorString(errNo)), + logging::ERR_DISKAGG_FILEIO_ERROR); + } + close(fd); + } + + /** @brief Save all RGData to disk */ + void dumpAll(bool dumpFin = true) const + { +#ifdef DISK_AGG_DEBUG + dumpMeta(); +#endif + for (uint64_t i = 0; i < fRGDatas.size(); ++i) + { + if (fRGDatas[i]) + saveRG(i, fRGDatas[i].get()); + else + { + auto fname = makeRGFilename(i); + if (access(fname.c_str(), F_OK) != 0) + ::abort(); + } + } + if (dumpFin) + dumpFinalizedInfo(); + } + + /** @brief Create new RowGroupStorage with the save LRU, MemManager & uniq ID */ + RowGroupStorage* clone(uint16_t gen) const + { + auto* ret = new RowGroupStorage(fTmpDir, fRowGroupOut, fMaxRows); + ret->fRGDatas.clear(); + ret->fLRU.reset(fLRU->clone()); + ret->fMM.reset(fMM->clone()); + ret->fUniqId = fUniqId; + ret->fGeneration = gen; + ret->loadFinalizedInfo(); + return ret; + } + + /** @brief Mark row at specified index as finalized so it should be skipped + */ + void markFinalized(uint64_t idx) + { + uint64_t gid = idx / 64; + uint64_t rid = idx % 64; + if (LIKELY(fFinalizedRows.size() <= gid)) + fFinalizedRows.resize(gid + 1, 0ULL); + + fFinalizedRows[gid] |= 1ULL << rid; + } + + /** @brief Check if row at specified index was finalized earlier */ + bool isFinalized(uint64_t idx) const + { + uint64_t gid = idx / 64; + uint64_t rid = idx % 64; + if (LIKELY(fFinalizedRows.size() <= gid)) + return false; + + return fFinalizedRows[gid] & (1ULL << rid); + } + + void getTmpFilePrefixes(std::vector& prefixes) const + { + char buf[PATH_MAX]; + snprintf(buf, sizeof(buf), "Agg-p%u-t%p-rg", getpid(), fUniqId); + prefixes.emplace_back(buf); + + snprintf(buf, sizeof(buf), "AggFin-p%u-t%p-g", getpid(), fUniqId); + prefixes.emplace_back(buf); + } + +private: + + /** @brief Get next available RGData and fill filename of the dump if it's + * not in the memory. + * + * It skips finalized rows, shifting data within rowgroups + * + * @param rgdata(out) RGData to return + * @param fname(out) Filename of the dump if it's not in the memory + * @returns true if there is available RGData + */ + bool getNextRGData(std::unique_ptr& rgdata, std::string& fname) + { + if (UNLIKELY(fRGDatas.empty())) + { + fMM->release(); + return false; + } + while (!fRGDatas.empty()) + { + uint64_t rgid = fRGDatas.size() - 1; + rgdata = std::move(fRGDatas[rgid]); + fRGDatas.pop_back(); + + uint64_t fgid = rgid * fMaxRows / 64; + uint64_t tgid = fgid + fMaxRows / 64; + if (fFinalizedRows.size() > fgid) + { + if (tgid >= fFinalizedRows.size()) + fFinalizedRows.resize(tgid + 1, 0ULL); + + if (!rgdata) + { + for (auto i = fgid; i < tgid; ++i) + { + if (fFinalizedRows[i] != ~0ULL) + { + loadRG(rgid, rgdata, true); + break; + } + } + } + + if (!rgdata) + { + unlink(makeRGFilename(rgid).c_str()); + continue; + } + + uint64_t pos = 0; + uint64_t opos = 0; + fRowGroupOut->setData(rgdata.get()); + for (auto i = fgid; i < tgid; ++i) + { + if ((i - fgid) * 64 >= fRowGroupOut->getRowCount()) + break; + uint64_t mask = ~fFinalizedRows[i]; + if ((i - fgid + 1) * 64 > fRowGroupOut->getRowCount()) + { + mask &= (~0ULL) >> ((i - fgid + 1) * 64 - fRowGroupOut->getRowCount()); + } + opos = (i - fgid) * 64; + if (mask == ~0ULL) + { + if (LIKELY(pos != opos)) + moveRows(rgdata.get(), pos, opos, 64); + pos += 64; + continue; + } + + if (mask == 0) + continue; + + while (mask != 0) + { + size_t b = __builtin_ffsll(mask); + size_t e = __builtin_ffsll(~(mask >> b)) + b; + if (UNLIKELY(e >= 64)) + mask = 0; + else + mask >>= e; + if (LIKELY(pos != opos + b - 1)) + moveRows(rgdata.get(), pos, opos + b - 1, e - b); + pos += e - b; + opos += e; + } + --opos; + } + + if (pos == 0) + { + fLRU->remove(rgid); + unlink(makeRGFilename(rgid).c_str()); + continue; + } + + fRowGroupOut->setData(rgdata.get()); + fRowGroupOut->setRowCount(pos); + } + + if (rgdata) + { + fRowGroupOut->setData(rgdata.get()); + int64_t memSz = fRowGroupOut->getSizeWithStrings(fMaxRows); + fMM->release(memSz); + unlink(makeRGFilename(rgid).c_str()); + } + else + { + fname = makeRGFilename(rgid); + } + fLRU->remove(rgid); + return true; + } + return false; + } + + /** @brief Compact rowgroup with finalized rows + * + * Move raw data from the next non-finalized row to replace finalized rows + * It is safe because pointers to long string also stored inside data + * + * @param rgdata(in,out) RGData to work with + * @param to(in) row num inside rgdata of the first finalized row + * @param from(in) row num of the first actual row + * @param numRows(in) how many rows should be moved + */ + void moveRows(RGData* rgdata, uint64_t to, uint64_t from, size_t numRows) + { + const size_t rowsz = fRowGroupOut->getRowSize(); + const size_t hdrsz = RowGroup::getHeaderSize(); + uint8_t* data = rgdata->rowData.get() + hdrsz; + memmove(data + to * rowsz, + data + from * rowsz, + numRows * rowsz); + } + + /** @brief Load RGData from disk dump. + * + * @param rgid(in) RGData ID + */ + void loadRG(uint64_t rgid) + { + if (UNLIKELY(static_cast(fRGDatas[rgid]))) + { + fRowGroupOut->setData(fRGDatas[rgid].get()); + return; + } + + loadRG(rgid, fRGDatas[rgid]); + } + + void loadRG(uint64_t rgid, std::unique_ptr& rgdata, bool unlinkDump = false) + { + auto fname = makeRGFilename(rgid); + int fd = open(fname.c_str(), O_RDONLY); + if (UNLIKELY(fd < 0)) + { + throw logging::IDBExcept(logging::IDBErrorInfo::instance()->errorMsg( + logging::ERR_DISKAGG_FILEIO_ERROR, errorString(errno)), + logging::ERR_DISKAGG_FILEIO_ERROR); + } + messageqcpp::ByteStream bs; + + try + { + struct stat st + { + }; + fstat(fd, &st); + + bs.needAtLeast(st.st_size); + bs.restart(); + int errNo; + if ((errNo = readData(fd, (char*)bs.getInputPtr(), st.st_size)) != 0) + { + close(fd); + unlink(fname.c_str()); + throw logging::IDBExcept(logging::IDBErrorInfo::instance()->errorMsg( + logging::ERR_DISKAGG_FILEIO_ERROR, errorString(errNo)), + logging::ERR_DISKAGG_FILEIO_ERROR); + } + bs.advanceInputPtr(st.st_size); + close(fd); + } + catch (...) + { + close(fd); + throw; + } + + if (unlinkDump) + unlink(fname.c_str()); + rgdata.reset(new RGData()); + rgdata->deserialize(bs, fRowGroupOut->getDataSize(fMaxRows)); + + fRowGroupOut->setData(rgdata.get()); + auto memSz = fRowGroupOut->getSizeWithStrings(fMaxRows); + + if (!fMM->acquire(memSz)) + { + throw logging::IDBExcept(logging::IDBErrorInfo::instance()->errorMsg( + logging::ERR_AGGREGATION_TOO_BIG), + logging::ERR_AGGREGATION_TOO_BIG); + } + } + + /** @brief Dump RGData to disk. + * + * @param rgid(in) RGData ID + */ + void saveRG(uint64_t rgid) + { + auto rgdata = std::move(fRGDatas[rgid]); + if (!rgdata) + return; + + fLRU->remove(rgid); + fRowGroupOut->setData(rgdata.get()); + fMM->release(fRowGroupOut->getSizeWithStrings(fMaxRows)); + + saveRG(rgid, rgdata.get()); + } + + /** @brief Dump RGData to disk. + * + * @param rgid(in) RGData ID + * @param rgdata(in) pointer to RGData itself + */ + void saveRG(uint64_t rgid, RGData* rgdata) const + { + messageqcpp::ByteStream bs; + fRowGroupOut->setData(rgdata); + rgdata->serialize(bs, fRowGroupOut->getDataSize()); + + int fd = open(makeRGFilename(rgid).c_str(), O_WRONLY | O_CREAT | O_TRUNC, 0644); + if (UNLIKELY(fd < 0)) + { + throw logging::IDBExcept(logging::IDBErrorInfo::instance()->errorMsg( + logging::ERR_DISKAGG_FILEIO_ERROR, errorString(errno)), + logging::ERR_DISKAGG_FILEIO_ERROR); + } + + int errNo; + if ((errNo = writeData(fd, (char*)bs.buf(), bs.length())) != 0) + { + close(fd); + throw logging::IDBExcept(logging::IDBErrorInfo::instance()->errorMsg( + logging::ERR_DISKAGG_FILEIO_ERROR, errorString(errNo)), + logging::ERR_DISKAGG_FILEIO_ERROR); + } + close(fd); + } + +#ifdef DISK_AGG_DEBUG + void dumpMeta() const + { + messageqcpp::ByteStream bs; + fRowGroupOut->serialize(bs); + + char buf[1024]; + snprintf(buf, sizeof(buf), "/tmp/kemm/META-p%u-t%p", getpid(), fUniqPtr); + int fd = open(buf, O_WRONLY | O_TRUNC | O_CREAT, 0644); + assert(fd >= 0); + + auto r = write(fd, bs.buf(), bs.length()); + assert(r == bs.length()); + close(fd); + } +#endif + + /** @brief Create dump filename. + */ + std::string makeRGFilename(uint64_t rgid) const + { + char buf[PATH_MAX]; + snprintf(buf, sizeof(buf), "%s/Agg-p%u-t%p-rg%lu-g%u", + fTmpDir.c_str(), getpid(), fUniqId, rgid, fGeneration); + return buf; + } + + std::string makeFinalizedFilename() const + { + char fname[PATH_MAX]; + snprintf(fname, sizeof(fname), "%s/AggFin-p%u-t%p-g%u", + fTmpDir.c_str(), getpid(), fUniqId, fGeneration); + return fname; + } + +private: + friend class RowAggStorage; + RowGroup* fRowGroupOut{nullptr}; + const size_t fMaxRows; + std::unique_ptr fMM; + std::unique_ptr fLRU; + RGDataStorage fRGDatas; + const void* fUniqId; + + uint64_t fCurRgid{0}; + uint16_t fGeneration{0}; + std::vector fFinalizedRows; + std::string fTmpDir; +}; + +/** @brief Internal data for the hashmap */ +struct RowPosHash +{ + uint64_t hash; ///< Row hash + uint64_t idx; ///< index in the RowGroupStorage +}; + +/*** + * @ brief Storage for row positions and hashes memory management + * and the ability to save on disk + */ +class RowPosHashStorage +{ +public: + /*** + * @brief Default constructor + * + * The internal data is stored as a plain vector of row pos & hash + * + * @param tmpDir(in) directory for tmp data + * @param size(in) maximum size of storage (NB: without the padding) + * @param rm ResourceManager to use + * @param sessLimit session memory limit + * @param enableDiskAgg is disk aggregation enabled? + */ + RowPosHashStorage(const std::string& tmpDir, + size_t size, + joblist::ResourceManager* rm, + boost::shared_ptr sessLimit, + bool enableDiskAgg) + : fUniqId(this) + , fTmpDir(tmpDir) + { + if (rm) + fMM.reset(new RMMemManager(rm, sessLimit, !enableDiskAgg, !enableDiskAgg)); + else + fMM.reset(new MemManager()); + + if (size != 0) + init(size); + } + + /*** + * @brief Get the row position and hash at the idx + * + * @param idx(in) index (from 0) of the row + */ + RowPosHash& get(uint64_t idx) + { + return fPosHashes[idx]; + } + + /*** + * @brief Store the row position and hash at the idx + * + * @param idx(in) index of the row + * @param rowData(in) position and hash of the row + */ + void set(uint64_t idx, const RowPosHash& pos) + { + memcpy(&fPosHashes[idx], &pos, sizeof(pos)); + } + + /*** @return Size of data */ + ssize_t memUsage() const + { + return fMM->getUsed(); + } + + /*** @brief Unregister used memory */ + void releaseMemory() + { + fMM->release(); + } + + /*** + * @brief Move row positions & hashes inside [insIdx, startIdx] up by 1 + * + * @param startIdx(in) last index to move + * @param insIdx(in) first index to move + */ + void shiftUp(uint64_t startIdx, uint64_t insIdx) + { + memmove(&fPosHashes[insIdx + 1], + &fPosHashes[insIdx], + (startIdx - insIdx) * sizeof(decltype(fPosHashes)::value_type)); + } + + /*** + * @brief Create new storage with the same MemManager & uniq ID + * @param size(in) maximum size of storage without padding + * @param gen(in) new generation + */ + RowPosHashStoragePtr clone(size_t size, uint16_t gen, bool loadDump = false) const + { + RowPosHashStoragePtr cloned; + + cloned.reset(new RowPosHashStorage()); + cloned->fMM.reset(fMM->clone()); + cloned->fTmpDir = fTmpDir; + cloned->init(size); + cloned->fUniqId = fUniqId; + cloned->fGeneration = gen; + if (loadDump) + cloned->load(); + return cloned; + } + + /*** @brief Remove dump file */ + void cleanup() const + { + unlink(makeDumpName().c_str()); + } + + /*** + * @brief Dump all data, clear state, and start over with the new generation + */ + void startNewGeneration() + { + dump(); + ++fGeneration; + fPosHashes.clear(); + fMM->release(); + } + + void dump() + { + int fd = open(makeDumpName().c_str(), O_WRONLY | O_CREAT | O_TRUNC, 0644); + if (fd < 0) + { + throw logging::IDBExcept( + logging::IDBErrorInfo::instance()->errorMsg(logging::ERR_DISKAGG_FILEIO_ERROR, + errorString(errno)), + logging::ERR_DISKAGG_FILEIO_ERROR); + } + + int errNo; + size_t sz = fPosHashes.size() * sizeof(decltype(fPosHashes)::value_type); + if ((errNo = writeData(fd, (char*)fPosHashes.data(), sz)) != 0) + { + close(fd); + throw logging::IDBExcept( + logging::IDBErrorInfo::instance()->errorMsg(logging::ERR_DISKAGG_FILEIO_ERROR, + errorString(errNo)), + logging::ERR_DISKAGG_FILEIO_ERROR); + } + + close(fd); + } + +private: + RowPosHashStorage() = default; + + void init(size_t size) + { + auto bkts = size + 0xFFUL; + if (!fMM->acquire(bkts * sizeof(decltype(fPosHashes)::value_type))) + { + throw logging::IDBExcept(logging::IDBErrorInfo::instance()->errorMsg( + logging::ERR_AGGREGATION_TOO_BIG), + logging::ERR_AGGREGATION_TOO_BIG); + } + fPosHashes.resize(bkts); + } + + std::string makeDumpName() const + { + char fname[PATH_MAX]; + snprintf(fname, sizeof(fname), "%s/Agg-PosHash-p%u-t%p-g%u", + fTmpDir.c_str(), getpid(), fUniqId, fGeneration); + return fname; + } + + void load() + { + int fd = open(makeDumpName().c_str(), O_RDONLY); + if (fd < 0) + { + throw logging::IDBExcept( + logging::IDBErrorInfo::instance()->errorMsg(logging::ERR_DISKAGG_FILEIO_ERROR, + errorString(errno)), + logging::ERR_DISKAGG_FILEIO_ERROR); + } + + struct stat st; + fstat(fd, &st); + fPosHashes.resize(st.st_size / sizeof(decltype(fPosHashes)::value_type)); + int errNo; + if ((errNo = readData(fd, (char*)fPosHashes.data(), st.st_size)) != 0) + { + close(fd); + throw logging::IDBExcept( + logging::IDBErrorInfo::instance()->errorMsg(logging::ERR_DISKAGG_FILEIO_ERROR, + errorString(errNo)), + logging::ERR_DISKAGG_FILEIO_ERROR); + } + + close(fd); + } + +private: + std::unique_ptr fMM; + std::vector fPosHashes; + uint16_t fGeneration{0}; ///< current aggregation generation + void* fUniqId; ///< uniq ID to make an uniq dump filename + std::string fTmpDir; +}; + +/*--------------------------------------------------------------------------- + * Based on the Robin Hood hashmap implementation by Martin Ankerl: + * https://github.com/martinus/robin-hood-hashing + * + * But store actual row data within RowGroupStorage, that stores a vector + * of RGData with possibility to dump unused to disk. Also store some service + * information (index and hash) in the vector of portions of the same size in + * elements, that can be dumped on disk either. + ----------------------------------------------------------------------------*/ +RowAggStorage::RowAggStorage(const std::string& tmpDir, + RowGroup *rowGroupOut, RowGroup *keysRowGroup, + uint32_t keyCount, + joblist::ResourceManager *rm, + boost::shared_ptr sessLimit, + bool enabledDiskAgg, + bool allowGenerations) + : fMaxRows(getMaxRows(enabledDiskAgg)) + , fExtKeys(rowGroupOut != keysRowGroup) + , fLastKeyCol(keyCount - 1) + , fUniqId(this) + , fAllowGenerations(allowGenerations) + , fEnabledDiskAggregation(enabledDiskAgg) + , fTmpDir(tmpDir) + , fRowGroupOut(rowGroupOut) + , fKeysRowGroup(keysRowGroup) +{ + char suffix[PATH_MAX]; + snprintf(suffix, sizeof(suffix), "/p%u-t%p/", getpid(), this); + fTmpDir.append(suffix); + if (enabledDiskAgg) + boost::filesystem::create_directories(fTmpDir); + + if (rm) + { + fMM.reset(new RMMemManager(rm, sessLimit, !enabledDiskAgg, !enabledDiskAgg)); + fNumOfInputRGPerThread = std::max(1, rm->aggNumRowGroups()); + } + else + { + fMM.reset(new MemManager()); + fNumOfInputRGPerThread = 1; + } + fStorage.reset(new RowGroupStorage(fTmpDir, rowGroupOut, 1, rm, sessLimit, !enabledDiskAgg, !enabledDiskAgg)); + if (fExtKeys) + { + fKeysStorage = new RowGroupStorage(fTmpDir, keysRowGroup, 1, rm, sessLimit, !enabledDiskAgg, !enabledDiskAgg); + } + else + { + fKeysStorage = fStorage.get(); + } + fKeysStorage->initRow(fKeyRow); + fGens.emplace_back(new Data); + fCurData = fGens.back().get(); + fCurData->fHashes.reset(new RowPosHashStorage(fTmpDir, 0, rm, sessLimit, fEnabledDiskAggregation)); +} + +RowAggStorage::~RowAggStorage() +{ + cleanupAll(); + + if (fExtKeys) + delete fKeysStorage; + for (auto& data : fGens) + { + if (data->fInfo != nullptr) + free(data->fInfo); + } +} + +bool RowAggStorage::getTargetRow(const Row &row, Row &rowOut) +{ + uint64_t hash = hashRow(row, fLastKeyCol); + return getTargetRow(row, hash, rowOut); +} + +bool RowAggStorage::getTargetRow(const Row &row, uint64_t hash, Row &rowOut) +{ + if (UNLIKELY(!fInitialized)) + { + fInitialized = true; + fStorage.reset(new RowGroupStorage(fTmpDir, + fRowGroupOut, + fMaxRows, + fMM->getResourceManaged(), + fMM->getSessionLimit(), + !fEnabledDiskAggregation, + !fEnabledDiskAggregation)); + if (fExtKeys) + { + fKeysStorage = new RowGroupStorage(fTmpDir, + fKeysRowGroup, + fMaxRows, + fMM->getResourceManaged(), + fMM->getSessionLimit(), + !fEnabledDiskAggregation, + !fEnabledDiskAggregation); + } + else + { + fKeysStorage = fStorage.get(); + } + fKeysStorage->initRow(fKeyRow); + reserve(fMaxRows); + } + else if (UNLIKELY(fCurData->fSize >= fCurData->fMaxSize)) + { + increaseSize(); + } + size_t idx{}; + uint32_t info{}; + + rowHashToIdx(hash, info, idx); + nextWhileLess(info, idx); + + while (info == fCurData->fInfo[idx]) + { + auto& pos = fCurData->fHashes->get(idx); + if (pos.hash == hash) + { + auto& keyRow = fExtKeys ? fKeyRow : rowOut; + fKeysStorage->getRow(pos.idx, keyRow); + if (row.equals(keyRow, fLastKeyCol)) + { + if (!fExtKeys) + return false; + + fStorage->getRow(pos.idx, rowOut); + return false; + } + } + + next(info, idx); + } + + if (!fEnabledDiskAggregation && fGeneration != 0) + { + // there are several generations here, so let's try to find suitable row in them + uint16_t gen = fGeneration - 1; + do + { + auto *genData = fGens[gen].get(); + size_t gidx{}; + uint32_t ginfo{}; + rowHashToIdx(hash, ginfo, gidx, genData); + nextWhileLess(ginfo, gidx, genData); + + while (ginfo == genData->fInfo[gidx]) + { + auto& pos = genData->fHashes->get(idx); + if (pos.hash == hash) + { + auto& keyRow = fExtKeys ? fKeyRow : rowOut; + + fKeysStorage->getRow(pos.idx, keyRow); + if (row.equals(keyRow, fLastKeyCol)) + { + if (!fExtKeys) + return false; + + fStorage->getRow(pos.idx, rowOut); + return false; + } + } + next(ginfo, gidx, genData); + } + } while (gen-- != 0); + } + + const auto ins_idx = idx; + const auto ins_info = info; + if (UNLIKELY(ins_info + fCurData->fInfoInc > 0xFF)) + { + fCurData->fMaxSize = 0; + } + + while (fCurData->fInfo[idx] != 0) + { + next(info, idx); + } + + if (idx != ins_idx) + { + shiftUp(idx, ins_idx); + } + RowPosHash pos; + pos.hash = hash; + fStorage->putRow(pos.idx, rowOut); + if (fExtKeys) + { + fKeysStorage->putKeyRow(pos.idx, fKeyRow); + copyRow(row, &fKeyRow); + } + fCurData->fHashes->set(ins_idx, pos); + fCurData->fInfo[ins_idx] = static_cast(ins_info); + ++fCurData->fSize; + return true; +} + +void RowAggStorage::dump() +{ + if (!fEnabledDiskAggregation) + return; + + const int64_t leaveFree = fNumOfInputRGPerThread * fRowGroupOut->getRowSize() * getBucketSize(); + uint64_t freeAttempts{0}; + int64_t freeMem = 0; + while (true) + { + ++freeAttempts; + freeMem = fMM->getFree(); + if (freeMem > leaveFree) + break; + + bool success = fStorage->dump(); + if (fExtKeys) + success |= fKeysStorage->dump(); + + if (!success) + break; + } + + // If the generations are allowed and there are less than half of + // rowgroups in memory, then we start a new generation + if (fAllowGenerations && + fStorage->fLRU->size() < fStorage->fRGDatas.size() / 2 && + fStorage->fRGDatas.size() > 10) + { + startNewGeneration(); + } + else if (!fAllowGenerations && freeMem < 0 && freeAttempts == 1) + { + // safety guard so aggregation couldn't eat all available memory + throw logging::IDBExcept(logging::IDBErrorInfo::instance()->errorMsg( + logging::ERR_DISKAGG_TOO_BIG), + logging::ERR_DISKAGG_TOO_BIG); + } +} + +void RowAggStorage::append(RowAggStorage& other) +{ + // we don't need neither key rows storage nor any internal data anymore + // neither in this RowAggStorage nor in the other + cleanup(); + freeData(); + if (other.fGeneration == 0 || !fEnabledDiskAggregation) + { + // even if there is several generations in the other RowAggStorage + // in case of in-memory-only aggregation they all share the same RowStorage + other.cleanup(); + other.freeData(); + fStorage->append(std::move(other.fStorage)); + return; + } + + // iff other RowAggStorage has several generations, sequential load and append + // them all + // the only needed data is the aggregated RowStorage itself + auto gen = other.fGeneration; + while (true) + { + fStorage->append(other.fStorage.get()); + other.cleanup(); + if (gen == 0) + break; + --gen; + other.fGeneration = gen; + other.fStorage.reset(other.fStorage->clone(gen)); + } +} + +std::unique_ptr RowAggStorage::getNextRGData() +{ + if (!fStorage) + { + return {}; + } + cleanup(); + freeData(); + return fStorage->getNextRGData(); +} + +void RowAggStorage::freeData() +{ + if (fExtKeys && fKeysStorage) + { + delete fKeysStorage; + fKeysStorage = nullptr; + } + for (auto& data : fGens) + { + data->fHashes.reset(); + if (data->fInfo) + { + const size_t memSz = calcSizeWithBuffer(data->fMask + 1); + fMM->release(memSz); + free(data->fInfo); + data->fInfo = nullptr; + } + } + fGens.clear(); + fCurData = nullptr; +} + +void RowAggStorage::shiftUp(size_t startIdx, size_t insIdx) +{ + auto idx = startIdx; + while (idx != insIdx) { + fCurData->fInfo[idx] = static_cast(fCurData->fInfo[idx - 1] + fCurData->fInfoInc); + if (UNLIKELY(fCurData->fInfo[idx] + fCurData->fInfoInc > 0xFF)) { + fCurData->fMaxSize = 0; + } + --idx; + } + fCurData->fHashes->shiftUp(startIdx, insIdx); +} + +void RowAggStorage::rowToIdx(const Row &row, + uint32_t& info, + size_t& idx, + uint64_t& hash, + const Data* curData) const +{ + hash = hashRow(row, fLastKeyCol); + return rowHashToIdx(hash, info, idx, curData); +} + +void RowAggStorage::rowToIdx(const Row &row, + uint32_t& info, + size_t& idx, + uint64_t& hash) const +{ + return rowToIdx(row, info, idx, hash, fCurData); +} + +void RowAggStorage::increaseSize() +{ + if (fCurData->fMask == 0) + { + initData(INIT_SIZE, fCurData->fHashes.get()); + } + + const auto maxSize = calcMaxSize(fCurData->fMask + 1); + if (fCurData->fSize < maxSize && tryIncreaseInfo()) + return; + + if (fCurData->fSize * 2 < calcMaxSize(fCurData->fMask + 1)) + { + // something strange happens... + throw logging::IDBExcept(logging::IDBErrorInfo::instance()->errorMsg( + logging::ERR_DISKAGG_ERROR), + logging::ERR_DISKAGG_ERROR); + } + + auto freeMem = fMM->getFree(); + if (fEnabledDiskAggregation || + freeMem > (fMM->getUsed() + fCurData->fHashes->memUsage() + fStorage->getAproxRGSize()) * 2) + { + rehashPowerOfTwo((fCurData->fMask + 1) * 2); + } + else if (fGeneration < MAX_INMEMORY_GENS - 1) + { + startNewGeneration(); + } + else + { + throw logging::IDBExcept(logging::IDBErrorInfo::instance()->errorMsg( + logging::ERR_AGGREGATION_TOO_BIG), + logging::ERR_AGGREGATION_TOO_BIG); + } +} + +bool RowAggStorage::tryIncreaseInfo() +{ + if (fCurData->fInfoInc <= 2) + return false; + + fCurData->fInfoInc = static_cast(fCurData->fInfoInc >> 1U); + ++fCurData->fInfoHashShift; + const auto elems = calcSizeWithBuffer(fCurData->fMask + 1); + for (size_t i = 0; i < elems; i += 8) + { + uint64_t val; + memcpy(&val, fCurData->fInfo + i, sizeof(val)); + val = (val >> 1U) & 0x7f7f7f7f7f7f7f7fULL; + memcpy(fCurData->fInfo + i, &val, sizeof(val)); + } + + fCurData->fInfo[elems] = 1; + fCurData->fMaxSize = calcMaxSize(fCurData->fMask + 1); + return true; +} + +void RowAggStorage::rehashPowerOfTwo(size_t elems) +{ + const size_t oldSz = calcSizeWithBuffer(fCurData->fMask + 1); + const uint8_t* const oldInfo = fCurData->fInfo; + auto oldHashes = std::move(fCurData->fHashes); + fMM->release(calcBytes(oldSz)); + + try + { + initData(elems, oldHashes.get()); + oldHashes->releaseMemory(); + + if (oldSz > 1) + { + for (size_t i = 0; i < oldSz; ++i) + { + if (UNLIKELY(oldInfo[i] != 0)) + { + insertSwap(i, oldHashes.get()); + } + } + } + } + catch (...) + { + if (oldInfo != nullptr && oldInfo != fCurData->fInfo) + { + free((void *)oldInfo); + } + throw; + } + if (oldInfo != nullptr && oldInfo != fCurData->fInfo) + { + free((void *)oldInfo); + } +} + +void RowAggStorage::insertSwap(size_t oldIdx, RowPosHashStorage* oldHashes) +{ + if (fCurData->fMaxSize == 0 && !tryIncreaseInfo()) + { + throw logging::IDBExcept(logging::IDBErrorInfo::instance()->errorMsg( + logging::ERR_DISKAGG_ERROR), + logging::ERR_DISKAGG_ERROR); + } + + size_t idx{}; + uint32_t info{}; + auto pos = oldHashes->get(oldIdx); + rowHashToIdx(pos.hash, info, idx); + + while (info <= fCurData->fInfo[idx]) + { + ++idx; + info += fCurData->fInfoInc; + } + + // don't need to compare rows here - they differ by definition + const auto ins_idx = idx; + const auto ins_info = static_cast(info); + if (UNLIKELY(ins_info + fCurData->fInfoInc > 0xFF)) + fCurData->fMaxSize = 0; + + while (fCurData->fInfo[idx] != 0) + { + next(info, idx); + } + + if (idx != ins_idx) + shiftUp(idx, ins_idx); + + fCurData->fHashes->set(ins_idx, pos); + fCurData->fInfo[ins_idx] = ins_info; + ++fCurData->fSize; +} + +void RowAggStorage::initData(size_t elems, const RowPosHashStorage* oldHashes) +{ + fCurData->fSize = 0; + fCurData->fMask = elems - 1; + fCurData->fMaxSize = calcMaxSize(elems); + + const auto sizeWithBuffer = calcSizeWithBuffer(elems, fCurData->fMaxSize); + const auto bytes = calcBytes(sizeWithBuffer); + + if (!fMM->acquire(bytes)) + { + throw logging::IDBExcept(logging::IDBErrorInfo::instance()->errorMsg( + logging::ERR_AGGREGATION_TOO_BIG), + logging::ERR_AGGREGATION_TOO_BIG); + } + fCurData->fHashes = oldHashes->clone(elems, fGeneration); + fCurData->fInfo = reinterpret_cast(calloc(1, bytes)); + fCurData->fInfo[sizeWithBuffer] = 1; + fCurData->fInfoInc = INIT_INFO_INC; + fCurData->fInfoHashShift = INIT_INFO_HASH_SHIFT; +} + +void RowAggStorage::reserve(size_t c) +{ + auto const minElementsAllowed = (std::max)(c, fCurData->fSize); + auto newSize = INIT_SIZE; + while (calcMaxSize(newSize) < minElementsAllowed && newSize != 0) { + newSize *= 2; + } + if (UNLIKELY(newSize == 0)) { + throw logging::IDBExcept( + logging::IDBErrorInfo::instance()->errorMsg(logging::ERR_DISKAGG_ERROR), + logging::ERR_DISKAGG_ERROR); + } + + // only actually do anything when the new size is bigger than the old one. This prevents to + // continuously allocate for each reserve() call. + if (newSize > fCurData->fMask + 1) { + rehashPowerOfTwo(newSize); + } +} + +void RowAggStorage::startNewGeneration() +{ + if (!fEnabledDiskAggregation) + { + ++fGeneration; + fGens.emplace_back(new Data); + auto* newData = fGens.back().get(); + newData->fHashes = fCurData->fHashes->clone(0, fGeneration); + fCurData = newData; + reserve(fMaxRows); + return; + } + + if (fCurData->fSize == 0) + return; + // save all data and free storages' memory + dumpInternalData(); + fCurData->fHashes->startNewGeneration(); + fStorage->startNewGeneration(); + if (fExtKeys) + fKeysStorage->startNewGeneration(); + + ++fGeneration; + fMM->release(); + // reinitialize internal structures + if (fCurData->fInfo) + { + free(fCurData->fInfo); + fCurData->fInfo = nullptr; + } + fCurData->fSize = 0; + fCurData->fMask = 0; + fCurData->fMaxSize = 0; + fCurData->fInfoInc = INIT_INFO_INC; + fCurData->fInfoHashShift = INIT_INFO_HASH_SHIFT; + reserve(fMaxRows); + fAggregated = false; +} + +std::string RowAggStorage::makeDumpFilename(int32_t gen) const +{ + char fname[PATH_MAX]; + uint16_t rgen = gen < 0 ? fGeneration : gen; + snprintf(fname, sizeof(fname), "%s/AggMap-p%u-t%p-g%u", + fTmpDir.c_str(), getpid(), fUniqId, rgen); + return fname; +} + +void RowAggStorage::dumpInternalData() const +{ + if (!fCurData->fInfo) + return; + + messageqcpp::ByteStream bs; + bs << fCurData->fSize; + bs << fCurData->fMask; + bs << fCurData->fMaxSize; + bs << fCurData->fInfoInc; + bs << fCurData->fInfoHashShift; + bs.append(fCurData->fInfo, calcBytes(calcSizeWithBuffer(fCurData->fMask + 1, fCurData->fMaxSize))); + int fd = open(makeDumpFilename().c_str(), O_WRONLY | O_CREAT | O_TRUNC, 0644); + if (fd < 0) + { + throw logging::IDBExcept(logging::IDBErrorInfo::instance()->errorMsg( + logging::ERR_DISKAGG_FILEIO_ERROR, errorString(errno)), + logging::ERR_DISKAGG_FILEIO_ERROR); + } + + int errNo; + if ((errNo = writeData(fd, (const char*)bs.buf(), bs.length())) != 0) + { + close(fd); + throw logging::IDBExcept(logging::IDBErrorInfo::instance()->errorMsg( + logging::ERR_DISKAGG_FILEIO_ERROR, errorString(errNo)), + logging::ERR_DISKAGG_FILEIO_ERROR); + } + close(fd); +} + +void RowAggStorage::finalize(std::function mergeFunc, Row& rowOut) +{ + if (fAggregated || fGeneration == 0 || !fEnabledDiskAggregation) + { + cleanup(); + return; + } + + Row tmpKeyRow; + fKeysStorage->initRow(tmpKeyRow); + Row tmpRow; + fStorage->initRow(tmpRow); + dumpInternalData(); + fCurData->fHashes->dump(); + fStorage->dumpAll(); + if (fExtKeys) + fKeysStorage->dumpAll(); + + uint16_t curGen = fGeneration + 1; + while (--curGen > 0) + { + bool genUpdated = false; + if (fCurData->fSize == 0) + { + fStorage->dumpFinalizedInfo(); + if (fExtKeys) + fKeysStorage->dumpFinalizedInfo(); + cleanup(curGen); + loadGeneration(curGen - 1); + auto oh = std::move(fCurData->fHashes); + fCurData->fHashes = oh->clone(0, curGen - 1, true); + fStorage.reset(fStorage->clone(curGen - 1)); + if (fExtKeys) + { + auto* oks = fKeysStorage; + fKeysStorage = oks->clone(curGen - 1); + delete oks; + } + else + fKeysStorage = fStorage.get(); + continue; + } + std::unique_ptr prevHashes; + size_t prevSize; + size_t prevMask; + size_t prevMaxSize; + uint32_t prevInfoInc; + uint32_t prevInfoHashShift; + uint8_t *prevInfo{nullptr}; + + std::unique_ptr prevRowStorage; + RowGroupStorage *prevKeyRowStorage{nullptr}; + + auto elems = calcSizeWithBuffer(fCurData->fMask + 1); + + for (uint16_t prevGen = 0; prevGen < curGen; ++prevGen) + { + prevRowStorage.reset(fStorage->clone(prevGen)); + if (fExtKeys) + { + delete prevKeyRowStorage; + prevKeyRowStorage = fKeysStorage->clone(prevGen); + } + else + prevKeyRowStorage = prevRowStorage.get(); + + loadGeneration(prevGen, prevSize, prevMask, prevMaxSize, prevInfoInc, + prevInfoHashShift, prevInfo); + prevHashes = fCurData->fHashes->clone(prevMask + 1, prevGen, true); + + // iterate over current generation rows + uint64_t idx{}; + uint32_t info{}; + for (;; next(info, idx)) + { + nextExisting(info, idx); + + if (idx >= elems) + { + // done finalizing generation + break; + } + + const auto& pos = fCurData->fHashes->get(idx); + + if (fKeysStorage->isFinalized(pos.idx)) + { + // this row was already merged into newer generation, skip it + continue; + } + + // now try to find row in the previous generation + uint32_t pinfo = + prevInfoInc + static_cast((pos.hash & INFO_MASK) >> prevInfoHashShift); + uint64_t pidx = (pos.hash >> INIT_INFO_BITS) & prevMask; + while (pinfo < prevInfo[pidx]) + { + ++pidx; + pinfo += prevInfoInc; + } + if (prevInfo[pidx] != pinfo) + { + // there is no such row + continue; + } + + auto ppos = prevHashes->get(pidx); + while (prevInfo[pidx] == pinfo && pos.hash != ppos.hash) + { + // hashes are not equal => collision, try all matched rows + ++pidx; + pinfo += prevInfoInc; + ppos = prevHashes->get(pidx); + } + if (prevInfo[pidx] != pinfo || pos.hash != ppos.hash) + { + // no matches + continue; + } + + bool found = false; + auto &keyRow = fExtKeys ? fKeyRow : rowOut; + fKeysStorage->getRow(pos.idx, keyRow); + while (!found && prevInfo[pidx] == pinfo) + { + // try to find exactly match in case of hash collision + if (UNLIKELY(pos.hash != ppos.hash)) + { + // hashes are not equal => no such row + ++pidx; + pinfo += prevInfoInc; + ppos = prevHashes->get(pidx); + continue; + } + + prevKeyRowStorage->getRow(ppos.idx, fExtKeys ? tmpKeyRow : tmpRow); + if (!keyRow.equals(fExtKeys ? tmpKeyRow : tmpRow, fLastKeyCol)) + { + ++pidx; + pinfo += prevInfoInc; + ppos = prevHashes->get(pidx); + continue; + } + found = true; + } + + if (!found) + { + // nothing was found, go to the next row + continue; + } + + if (UNLIKELY(prevKeyRowStorage->isFinalized(ppos.idx))) + { + // just to be sure, it can NEVER happen + continue; + } + + // here it is! So: + // 1 Get actual row datas + // 2 Merge it into the current generation + // 3 Mark generations as updated + // 4 Mark the prev generation row as finalized + if (fExtKeys) + { + prevRowStorage->getRow(ppos.idx, tmpRow); + fStorage->getRow(pos.idx, rowOut); + } + mergeFunc(tmpRow); + genUpdated = true; + prevKeyRowStorage->markFinalized(ppos.idx); + if (fExtKeys) + { + prevRowStorage->markFinalized(ppos.idx); + } + } + + prevRowStorage->dumpFinalizedInfo(); + if (fExtKeys) + { + prevKeyRowStorage->dumpFinalizedInfo(); + } + } + + fStorage->dumpFinalizedInfo(); + if (fExtKeys) + fKeysStorage->dumpFinalizedInfo(); + if (genUpdated) + { + // we have to save current generation data to disk 'cause we update some rows + // TODO: to reduce disk IO we can dump only affected rowgroups + fStorage->dumpAll(false); + if (fExtKeys) + fKeysStorage->dumpAll(false); + } + + // swap current generation N with the prev generation N-1 + fCurData->fSize = prevSize; + fCurData->fMask = prevMask; + fCurData->fMaxSize = prevMaxSize; + fCurData->fInfoInc = prevInfoInc; + fCurData->fInfoHashShift = prevInfoHashShift; + if (fCurData->fInfo) + free(fCurData->fInfo); + fCurData->fInfo = prevInfo; + fCurData->fHashes = std::move(prevHashes); + fStorage = std::move(prevRowStorage); + if (fExtKeys) + delete fKeysStorage; + fKeysStorage = prevKeyRowStorage; + } + + fStorage->dumpFinalizedInfo(); + if (fExtKeys) + fKeysStorage->dumpFinalizedInfo(); + auto oh = std::move(fCurData->fHashes); + fCurData->fHashes = oh->clone(fCurData->fMask + 1, fGeneration, true); + fStorage.reset(fStorage->clone(fGeneration)); + if (fExtKeys) + { + auto* oks = fKeysStorage; + fKeysStorage = oks->clone(fGeneration); + delete oks; + } + else + fKeysStorage = fStorage.get(); + + fAggregated = true; +} + +void RowAggStorage::loadGeneration(uint16_t gen) +{ + loadGeneration(gen, fCurData->fSize, fCurData->fMask, fCurData->fMaxSize, fCurData->fInfoInc, fCurData->fInfoHashShift, fCurData->fInfo); +} + +void RowAggStorage::loadGeneration(uint16_t gen, size_t &size, size_t &mask, size_t &maxSize, uint32_t &infoInc, uint32_t &infoHashShift, uint8_t *&info) +{ + messageqcpp::ByteStream bs; + int fd = open(makeDumpFilename(gen).c_str(), O_RDONLY); + if (fd < 0) + { + throw logging::IDBExcept(logging::IDBErrorInfo::instance()->errorMsg( + logging::ERR_DISKAGG_FILEIO_ERROR, errorString(errno)), + logging::ERR_DISKAGG_FILEIO_ERROR); + } + struct stat st{}; + fstat(fd, &st); + bs.needAtLeast(st.st_size); + bs.restart(); + int errNo; + if ((errNo = readData(fd, (char*)bs.getInputPtr(), st.st_size)) != 0) + { + close(fd); + throw logging::IDBExcept(logging::IDBErrorInfo::instance()->errorMsg( + logging::ERR_DISKAGG_FILEIO_ERROR, errorString(errNo)), + logging::ERR_DISKAGG_FILEIO_ERROR); + } + close(fd); + bs.advanceInputPtr(st.st_size); + + bs >> size; + bs >> mask; + bs >> maxSize; + bs >> infoInc; + bs >> infoHashShift; + size_t infoSz = calcBytes(calcSizeWithBuffer(mask + 1, maxSize)); + if (info) + free(info); + info = (uint8_t*)calloc(1, infoSz); + bs >> info; +} + +void RowAggStorage::cleanupAll() noexcept +{ + try + { + boost::filesystem::remove_all(fTmpDir); + } + catch (...) + { + } +} + +void RowAggStorage::cleanup() +{ + cleanup(fGeneration); +} + +void RowAggStorage::cleanup(uint16_t gen) +{ + if (!fInitialized) + return; + unlink(makeDumpFilename(gen).c_str()); +} + +size_t RowAggStorage::getBucketSize() +{ + return 1 /* info byte */ + sizeof(RowPosHash); +} + +uint32_t calcNumberOfBuckets(ssize_t availMem, + uint32_t numOfThreads, + uint32_t numOfBuckets, + uint32_t groupsPerThread, + uint32_t inRowSize, + uint32_t outRowSize, + bool enabledDiskAggr) +{ + if (availMem < 0) + { + // Most likely, nothing can be processed, but we will still try + return 1; + } + uint32_t ret = numOfBuckets; + + ssize_t minNeededMemPerThread = groupsPerThread * inRowSize * RowAggStorage::getMaxRows(false); + auto rowGroupSize = RowAggStorage::getMaxRows(enabledDiskAggr); + ssize_t minNeededMemPerBucket = outRowSize * rowGroupSize * 2 + + RowAggStorage::calcSizeWithBuffer(rowGroupSize, rowGroupSize) * RowAggStorage::getBucketSize(); + if ((availMem - minNeededMemPerThread * numOfThreads) / numOfBuckets < minNeededMemPerBucket) + { + ret = 0; + if (availMem > minNeededMemPerThread * numOfThreads) + { + ret = (availMem - minNeededMemPerThread * numOfThreads) / + minNeededMemPerBucket; + } + + if (ret < numOfThreads) + { + ret = availMem / (minNeededMemPerBucket + minNeededMemPerThread); + } + } + + return ret == 0 ? 1 : ret; +} + +} // namespace rowgroup diff --git a/utils/rowgroup/rowstorage.h b/utils/rowgroup/rowstorage.h new file mode 100644 index 000000000..a7cdfff7d --- /dev/null +++ b/utils/rowgroup/rowstorage.h @@ -0,0 +1,366 @@ +/* Copyright (C) 2021 MariaDB Corporation + + This program is free software; you can redistribute it and/or + modify it under the terms of the GNU General Public License + as published by the Free Software Foundation; version 2 of + the License. + + This program is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + GNU General Public License for more details. + + You should have received a copy of the GNU General Public License + along with this program; if not, write to the Free Software + Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, + MA 02110-1301, USA. */ + +#ifndef ROWSTORAGE_H +#define ROWSTORAGE_H + +#include "rowgroup.h" +#include +#include + +namespace rowgroup +{ + +uint32_t calcNumberOfBuckets(ssize_t availMem, + uint32_t numOfThreads, + uint32_t numOfBuckets, + uint32_t groupsPerThread, + uint32_t inRowSize, + uint32_t outRowSize, + bool enabledDiskAggr); + +class MemManager; +class RowPosHashStorage; +using RowPosHashStoragePtr = std::unique_ptr; +class RowGroupStorage; + +uint64_t hashRow(const rowgroup::Row& r, std::size_t lastCol); + +class RowAggStorage +{ +public: + RowAggStorage(const std::string& tmpDir, + RowGroup* rowGroupOut, + RowGroup* keysRowGroup, + uint32_t keyCount, + joblist::ResourceManager* rm = nullptr, + boost::shared_ptr sessLimit = {}, + bool enabledDiskAgg = false, + bool allowGenerations = false); + + RowAggStorage(const std::string& tmpDir, + RowGroup* rowGroupOut, + uint32_t keyCount, + joblist::ResourceManager* rm = nullptr, + boost::shared_ptr sessLimit = {}, + bool enabledDiskAgg = false, + bool allowGenerations = false) + : RowAggStorage(tmpDir, rowGroupOut, rowGroupOut, keyCount, + rm, std::move(sessLimit), + enabledDiskAgg, allowGenerations) + {} + + ~RowAggStorage(); + + static uint16_t getMaxRows(bool enabledDiskAgg) + { + return (enabledDiskAgg ? 8192 : 256); + } + + static size_t getBucketSize(); + + /** @brief Find or create resulting row. + * + * Create "aggregation key" row if necessary. + * NB! Using getTargetRow() after append() is UB! + * + * @param row(in) input row + * @param rowOut() row to aggregate data from input row + * + * @returns true if new row created, false otherwise + */ + bool getTargetRow(const Row& row, Row& rowOut); + bool getTargetRow(const Row& row, uint64_t row_hash, Row& rowOut); + + /** @brief Dump some RGDatas to disk and release memory for further use. + */ + void dump(); + + /** @brief Append RGData from other RowAggStorage and clear it. + * + * NB! Any operation except getNextRGData() or append() is UB! + * + * @param other(in) donor storage + */ + void append(RowAggStorage& other); + + /** @brief Remove last RGData from internal RGData storage and return it. + * + * @returns pointer to the next RGData or nullptr if empty + */ + std::unique_ptr getNextRGData(); + + /** @brief TODO + * + * @param mergeFunc + * @param rowOut + */ + void finalize(std::function mergeFunc, Row &rowOut); + + /** @brief Calculate maximum size of hash assuming 80% fullness. + * + * @param elems(in) number of elements + * @returns calculated size + */ + inline static size_t calcMaxSize(size_t elems) noexcept + { + if (LIKELY(elems <= std::numeric_limits::max() / 100)) + return elems * 80 / 100; + + return (elems / 100) * 80; + } + + inline static size_t calcSizeWithBuffer(size_t elems, size_t maxSize) noexcept + { + return elems + std::min(maxSize, 0xFFUL); + } + + inline static size_t calcSizeWithBuffer(size_t elems) noexcept + { + return calcSizeWithBuffer(elems, calcMaxSize(elems)); + } + +private: + struct Data; + /** @brief Create new RowAggStorage with the same params and load dumped data + * + * @param gen(in) generation number + * @return pointer to a new RowAggStorage + */ + RowAggStorage* clone(uint16_t gen) const; + + /** @brief Free any internal data + */ + void freeData(); + + /** @brief Move internal data & row position inside [insIdx, startIdx] up by 1. + * + * @param startIdx(in) last element's index to move + * @param insIdx(in) first element's index to move + */ + void shiftUp(size_t startIdx, size_t insIdx); + + /** @brief Find best position of row and save it's hash. + * + * @param row(in) input row + * @param info(out) info data + * @param idx(out) index computed from row hash + * @param hash(out) row hash value + */ + void rowToIdx(const Row& row, uint32_t& info, size_t& idx, uint64_t& hash) const; + void rowToIdx(const Row& row, uint32_t& info, size_t& idx, uint64_t& hash, const Data* curData) const; + + /** @brief Find best position using precomputed hash + * + * @param h(in) row hash + * @param info(out) info data + * @param idx(out) index + */ + inline void rowHashToIdx(uint64_t h, uint32_t& info, size_t& idx, const Data* curData) const + { + info = curData->fInfoInc + static_cast((h & INFO_MASK) >> curData->fInfoHashShift); + idx = (h >> INIT_INFO_BITS) & curData->fMask; + } + + inline void rowHashToIdx(uint64_t h, uint32_t& info, size_t& idx) const + { + return rowHashToIdx(h, info, idx, fCurData); + } + + /** @brief Iterate over internal info until info with less-or-equal distance + * from the best position was found. + * + * @param info(in,out) info data + * @param idx(in,out) index + */ + inline void nextWhileLess(uint32_t& info, size_t& idx, const Data* curData) const noexcept + { + while (info < curData->fInfo[idx]) + { + next(info, idx, curData); + } + } + + inline void nextWhileLess(uint32_t& info, size_t& idx) const noexcept + { + return nextWhileLess(info, idx, fCurData); + } + + /** @brief Get next index and corresponding info + */ + inline void next(uint32_t& info, size_t& idx, const Data* curData) const noexcept + { + ++(idx); + info += curData->fInfoInc; + } + + inline void next(uint32_t& info, size_t& idx) const noexcept + { + return next(info, idx, fCurData); + } + + /** @brief Get index and info of the next non-empty entry + */ + inline void nextExisting(uint32_t& info, size_t& idx) const noexcept + { + uint64_t n = 0; + uint64_t data; + while (true) + { + memcpy(&data, fCurData->fInfo + idx, sizeof(data)); + if (data == 0) + { + idx += sizeof(n); + } + else + { + break; + } + } + +#if BYTE_ORDER == BIG_ENDIAN + n = __builtin_clzll(data) / sizeof(data); +#else + n = __builtin_ctzll(data) / sizeof(data); +#endif + idx += n; + info = fCurData->fInfo[idx]; + } + + /** @brief Increase internal data size if needed + */ + void increaseSize(); + + /** @brief Increase distance capacity of info removing 1 bit of the hash. + * + * @returns success + */ + bool tryIncreaseInfo(); + + /** @brief Reserve space for number of elements (power of two) + * + * This function performs re-insert all data + * + * @param elems(in) new size + */ + void rehashPowerOfTwo(size_t elems); + + /** @brief Move elements from old one into rehashed data. + * + * It's mostly the same algo as in getTargetRow(), but returns nothing + * and skips some checks because it's guaranteed that there is no dups. + * + * @param oldIdx(in) index of "old" data + * @param oldHashes(in) old storage of row positions and hashes + */ + void insertSwap(size_t oldIdx, RowPosHashStorage* oldHashes); + + /** @brief (Re)Initialize internal data of specified size. + * + * @param elems(in) number of elements + */ + void initData(size_t elems, const RowPosHashStorage* oldHashes); + + /** @brief Calculate memory size of info data + * + * @param elems(in) number of elements + * @returns size in bytes + */ + inline static size_t calcBytes(size_t elems) noexcept + { + return elems + sizeof(uint64_t); + } + + /** @brief Reserve place sufficient for elems + * + * @param elems(in) number of elements + */ + void reserve(size_t elems); + + /** @brief Start new aggregation generation + * + * Dump all the data on disk, including internal info data, positions & row + * hashes, and the rowgroups itself. + */ + void startNewGeneration(); + + /** @brief Save internal info data on disk */ + void dumpInternalData() const; + + /** @brief Load previously dumped data from disk + * + * @param gen(in) generation number + */ + void loadGeneration(uint16_t gen); + /** @brief Load previously dumped data into the tmp storage */ + void loadGeneration(uint16_t gen, size_t& size, size_t& mask, size_t& maxSize, uint32_t& infoInc, uint32_t& infoHashShift, uint8_t*& info); + + /** @brief Remove temporary data files */ + void cleanup(); + void cleanup(uint16_t gen); + + /** @brief Remove all temporary data files */ + void cleanupAll() noexcept; + + std::string makeDumpFilename(int32_t gen = -1) const; + +private: + static constexpr size_t INIT_SIZE{sizeof(uint64_t)}; + static constexpr uint32_t INIT_INFO_BITS{5}; + static constexpr uint8_t INIT_INFO_INC{1U << INIT_INFO_BITS}; + static constexpr size_t INFO_MASK{INIT_INFO_INC - 1U}; + static constexpr uint8_t INIT_INFO_HASH_SHIFT{0}; + static constexpr uint16_t MAX_INMEMORY_GENS{4}; + + struct Data + { + RowPosHashStoragePtr fHashes; + uint8_t *fInfo{nullptr}; + size_t fSize{0}; + size_t fMask{0}; + size_t fMaxSize{0}; + uint32_t fInfoInc{INIT_INFO_INC}; + uint32_t fInfoHashShift{INIT_INFO_HASH_SHIFT}; + }; + std::vector> fGens; + Data* fCurData; + uint32_t fMaxRows; + const bool fExtKeys; + + std::unique_ptr fStorage; + RowGroupStorage* fKeysStorage; + uint32_t fLastKeyCol; + + uint16_t fGeneration{0}; + void* fUniqId; + + Row fKeyRow; + + std::unique_ptr fMM; + uint32_t fNumOfInputRGPerThread; + bool fAggregated = true; + bool fAllowGenerations; + bool fEnabledDiskAggregation; + std::string fTmpDir; + bool fInitialized{false}; + rowgroup::RowGroup* fRowGroupOut; + rowgroup::RowGroup* fKeysRowGroup; +}; + +} // namespace rowgroup + +#endif // MYSQL_ROWSTORAGE_H diff --git a/writeengine/shared/we_rbmetawriter.cpp b/writeengine/shared/we_rbmetawriter.cpp index f832e5287..ee9ff7c46 100644 --- a/writeengine/shared/we_rbmetawriter.cpp +++ b/writeengine/shared/we_rbmetawriter.cpp @@ -452,7 +452,7 @@ std::string RBMetaWriter::openMetaFile ( uint16_t dbRoot ) throw WeException( oss.str(), ERR_FILE_OPEN ); } - { + { std::ostringstream ossChown; idbdatafile::IDBFileSystem& fs = IDBPolicy::getFs(tmpMetaFileName.c_str()); if (chownPath(ossChown, tmpMetaFileName, fs) @@ -1338,7 +1338,7 @@ int RBMetaWriter::writeHWMChunk( return ERR_METADATABKUP_COMP_RENAME; } - { + { std::ostringstream ossChown; idbdatafile::IDBFileSystem& fs = IDBPolicy::getFs(fileName.c_str()); if (chownPath(ossChown, fileName, fs)