1
0
mirror of https://github.com/mariadb-corporation/mariadb-columnstore-engine.git synced 2025-07-30 19:23:07 +03:00

feat(): Replacing STLPoolAllocator with CountingAllocator for in-memory joins

This commit is contained in:
drrtuy
2025-02-14 12:08:28 +00:00
parent 101a07d90b
commit 4c1d9bceb7
6 changed files with 86 additions and 240 deletions

View File

@ -1397,7 +1397,7 @@ bool BatchPrimitiveProcessorJL::pickNextJoinerNum()
for (i = 0; i < PMJoinerCount; i++) for (i = 0; i < PMJoinerCount; i++)
{ {
joinerNum = (joinerNum + 1) % PMJoinerCount; joinerNum = (joinerNum + 1) % PMJoinerCount;
if (posByJoinerNum[joinerNum] != tJoiners[joinerNum]->getSmallSide()->size()) if (posByJoinerNum[joinerNum] != tJoiners[joinerNum]->getSmallSide().size())
break; break;
} }
if (i == PMJoinerCount) if (i == PMJoinerCount)
@ -1410,10 +1410,9 @@ bool BatchPrimitiveProcessorJL::pickNextJoinerNum()
/* XXXPAT: Going to interleave across joiners to take advantage of the new locking env in PrimProc */ /* XXXPAT: Going to interleave across joiners to take advantage of the new locking env in PrimProc */
bool BatchPrimitiveProcessorJL::nextTupleJoinerMsg(ByteStream& bs) bool BatchPrimitiveProcessorJL::nextTupleJoinerMsg(ByteStream& bs)
{ {
uint32_t size = 0, toSend, i, j; uint32_t toSend, i, j;
ISMPacketHeader ism; ISMPacketHeader ism;
Row r; Row r;
vector<Row::Pointer>* tSmallSide;
joiner::TypelessData tlData; joiner::TypelessData tlData;
uint32_t smallKeyCol; uint32_t smallKeyCol;
uint32_t largeKeyCol; uint32_t largeKeyCol;
@ -1436,8 +1435,8 @@ bool BatchPrimitiveProcessorJL::nextTupleJoinerMsg(ByteStream& bs)
} }
memset((void*)&ism, 0, sizeof(ism)); memset((void*)&ism, 0, sizeof(ism));
tSmallSide = tJoiners[joinerNum]->getSmallSide(); auto& tSmallSide = tJoiners[joinerNum]->getSmallSide();
size = tSmallSide->size(); auto size = tSmallSide.size();
#if 0 #if 0
if (joinerNum == PMJoinerCount - 1 && pos == size) if (joinerNum == PMJoinerCount - 1 && pos == size)
@ -1487,7 +1486,7 @@ bool BatchPrimitiveProcessorJL::nextTupleJoinerMsg(ByteStream& bs)
for (i = pos; i < pos + toSend; i++) for (i = pos; i < pos + toSend; i++)
{ {
r.setPointer((*tSmallSide)[i]); r.setPointer(tSmallSide[i]);
isNull = false; isNull = false;
bSignedUnsigned = tJoiners[joinerNum]->isSignedUnsignedJoin(); bSignedUnsigned = tJoiners[joinerNum]->isSignedUnsignedJoin();
@ -1554,7 +1553,7 @@ bool BatchPrimitiveProcessorJL::nextTupleJoinerMsg(ByteStream& bs)
for (i = pos, j = 0; i < pos + toSend; ++i, ++j) for (i = pos, j = 0; i < pos + toSend; ++i, ++j)
{ {
r.setPointer((*tSmallSide)[i]); r.setPointer(tSmallSide[i]);
if (r.getColType(smallKeyCol) == CalpontSystemCatalog::LONGDOUBLE) if (r.getColType(smallKeyCol) == CalpontSystemCatalog::LONGDOUBLE)
{ {
@ -1627,7 +1626,7 @@ bool BatchPrimitiveProcessorJL::nextTupleJoinerMsg(ByteStream& bs)
for (i = pos; i < pos + toSend; i++, tmpRow.nextRow()) for (i = pos; i < pos + toSend; i++, tmpRow.nextRow())
{ {
r.setPointer((*tSmallSide)[i]); r.setPointer(tSmallSide[i]);
copyRow(r, &tmpRow); copyRow(r, &tmpRow);
} }

View File

@ -207,60 +207,6 @@ void TupleHashJoinStep::join()
} }
} }
// simple sol'n. Poll mem usage of Joiner once per second. Request mem
// increase after the fact. Failure to get mem will be detected and handled by
// the threads inserting into Joiner.
void TupleHashJoinStep::trackMem(uint index)
{
auto joiner = joiners[index];
ssize_t memBefore = 0, memAfter = 0;
bool gotMem;
boost::unique_lock<boost::mutex> scoped(memTrackMutex);
while (!stopMemTracking)
{
memAfter = joiner->getMemUsage();
if (memAfter != memBefore)
{
gotMem = resourceManager->getMemory(memAfter - memBefore, sessionMemLimit, true);
if (gotMem)
atomicops::atomicAdd(&memUsedByEachJoin[index], memAfter - memBefore);
else
return;
memBefore = memAfter;
}
memTrackDone.timed_wait(scoped, boost::posix_time::seconds(1));
}
// one more iteration to capture mem usage since last poll, for this one
// raise an error if mem went over the limit
memAfter = joiner->getMemUsage();
if (memAfter == memBefore)
return;
gotMem = resourceManager->getMemory(memAfter - memBefore, sessionMemLimit, true);
if (gotMem)
{
atomicops::atomicAdd(&memUsedByEachJoin[index], memAfter - memBefore);
}
else
{
if (!joinIsTooBig &&
(isDML || !allowDJS || (fSessionId & 0x80000000) || (tableOid() < 3000 && tableOid() >= 1000)))
{
joinIsTooBig = true;
ostringstream oss;
oss << "(" << __LINE__ << ") "
<< logging::IDBErrorInfo::instance()->errorMsg(logging::ERR_JOIN_TOO_BIG);
fLogger->logMessage(logging::LOG_TYPE_INFO, oss.str());
errorMessage(oss.str());
status(logging::ERR_JOIN_TOO_BIG);
cout << "Join is too big, raise the UM join limit for now (monitor thread)" << endl;
abort();
}
}
}
void TupleHashJoinStep::startSmallRunners(uint index) void TupleHashJoinStep::startSmallRunners(uint index)
{ {
utils::setThreadName("HJSStartSmall"); utils::setThreadName("HJSStartSmall");
@ -302,7 +248,6 @@ void TupleHashJoinStep::startSmallRunners(uint index)
stopMemTracking = false; stopMemTracking = false;
utils::VLArray<uint64_t> jobs(numCores); utils::VLArray<uint64_t> jobs(numCores);
// uint64_t memMonitor = jobstepThreadPool.invoke([this, index] { this->trackMem(index); });
// starting 1 thread when in PM mode, since it's only inserting into a // starting 1 thread when in PM mode, since it's only inserting into a
// vector of rows. The rest will be started when converted to UM mode. // vector of rows. The rest will be started when converted to UM mode.
if (joiners[index]->inUM()) if (joiners[index]->inUM())

View File

@ -42,6 +42,7 @@ FixedAllocator::FixedAllocator(const FixedAllocator& f)
currentlyStored = 0; currentlyStored = 0;
useLock = f.useLock; useLock = f.useLock;
lock = false; lock = false;
alloc = f.alloc;
} }
FixedAllocator& FixedAllocator::operator=(const FixedAllocator& f) FixedAllocator& FixedAllocator::operator=(const FixedAllocator& f)
@ -51,6 +52,7 @@ FixedAllocator& FixedAllocator::operator=(const FixedAllocator& f)
tmpSpace = f.tmpSpace; tmpSpace = f.tmpSpace;
useLock = f.useLock; useLock = f.useLock;
lock = false; lock = false;
alloc = f.alloc;
deallocateAll(); deallocateAll();
return *this; return *this;
} }

View File

@ -116,12 +116,6 @@ STLPoolAllocator<T>::STLPoolAllocator(const STLPoolAllocator<T>& s) throw()
pa = s.pa; pa = s.pa;
} }
template <class T>
STLPoolAllocator<T>::STLPoolAllocator(uint32_t capacity) throw()
{
pa.reset(new PoolAllocator(capacity));
}
template <class T> template <class T>
template <class U> template <class U>
STLPoolAllocator<T>::STLPoolAllocator(const STLPoolAllocator<U>& s) throw() STLPoolAllocator<T>::STLPoolAllocator(const STLPoolAllocator<U>& s) throw()
@ -134,17 +128,6 @@ STLPoolAllocator<T>::~STLPoolAllocator()
{ {
} }
template <class T>
void STLPoolAllocator<T>::usePoolAllocator(boost::shared_ptr<PoolAllocator> p)
{
pa = p;
}
template <class T>
boost::shared_ptr<utils::PoolAllocator> STLPoolAllocator<T>::getPoolAllocator()
{
return pa;
}
template <class T> template <class T>
typename STLPoolAllocator<T>::pointer STLPoolAllocator<T>::allocate( typename STLPoolAllocator<T>::pointer STLPoolAllocator<T>::allocate(
typename STLPoolAllocator<T>::size_type s, typename STLPoolAllocator<T>::const_pointer hint) typename STLPoolAllocator<T>::size_type s, typename STLPoolAllocator<T>::const_pointer hint)

View File

@ -58,43 +58,34 @@ TupleJoiner::TupleJoiner(const rowgroup::RowGroup& smallInput, const rowgroup::R
{ {
uint i; uint i;
auto alloc = resourceManager_->getAllocator<rowgroup::Row::Pointer>();
rows.reset(new RowPointersVec(alloc));
getBucketCount(); getBucketCount();
m_bucketLocks.reset(new boost::mutex[bucketCount]); m_bucketLocks.reset(new boost::mutex[bucketCount]);
if (smallRG.getColTypes()[smallJoinColumn] == CalpontSystemCatalog::LONGDOUBLE) if (smallRG.getColTypes()[smallJoinColumn] == CalpontSystemCatalog::LONGDOUBLE)
{ {
ld.reset(new boost::scoped_ptr<ldhash_t>[bucketCount]);
// _pool.reset(new boost::shared_ptr<PoolAllocator>[bucketCount]);
for (i = 0; i < bucketCount; i++) for (i = 0; i < bucketCount; i++)
{ {
// STLPoolAllocator<pair<const long double, Row::Pointer>> alloc(resourceManager_);
// _pool[i] = alloc.getPoolAllocator();
auto alloc = resourceManager_->getAllocator<pair<const long double, Row::Pointer>>(); auto alloc = resourceManager_->getAllocator<pair<const long double, Row::Pointer>>();
ld[i].reset(new ldhash_t(10, hasher(), ldhash_t::key_equal(), alloc)); ld.emplace_back(std::unique_ptr<ldhash_t>(new ldhash_t(10, hasher(), ldhash_t::key_equal(), alloc)));
} }
} }
else if (smallRG.usesStringTable()) else if (smallRG.usesStringTable())
{ {
sth.reset(new boost::scoped_ptr<sthash_t>[bucketCount]);
_pool.reset(new boost::shared_ptr<PoolAllocator>[bucketCount]);
for (i = 0; i < bucketCount; i++) for (i = 0; i < bucketCount; i++)
{ {
// STLPoolAllocator<pair<const int64_t, Row::Pointer>> alloc(resourceManager_);
// _pool[i] = alloc.getPoolAllocator();
auto alloc = resourceManager_->getAllocator<pair<const int64_t, Row::Pointer>>(); auto alloc = resourceManager_->getAllocator<pair<const int64_t, Row::Pointer>>();
sth[i].reset(new sthash_t(10, hasher(), sthash_t::key_equal(), alloc)); sth.emplace_back(std::unique_ptr<sthash_t>(new sthash_t(10, hasher(), sthash_t::key_equal(), alloc)));
} }
} }
else else
{ {
h.reset(new boost::scoped_ptr<hash_t>[bucketCount]);
_pool.reset(new boost::shared_ptr<PoolAllocator>[bucketCount]);
for (i = 0; i < bucketCount; i++) for (i = 0; i < bucketCount; i++)
{ {
// STLPoolAllocator<pair<const int64_t, uint8_t*>> alloc(resourceManager_);
// _pool[i] = alloc.getPoolAllocator();
auto alloc = resourceManager_->getAllocator<pair<const int64_t, uint8_t*>>(); auto alloc = resourceManager_->getAllocator<pair<const int64_t, uint8_t*>>();
h[i].reset(new hash_t(10, hasher(), hash_t::key_equal(), alloc)); h.emplace_back(std::unique_ptr<hash_t>(new hash_t(10, hasher(), hash_t::key_equal(), alloc)));
} }
} }
@ -149,13 +140,6 @@ TupleJoiner::TupleJoiner(const rowgroup::RowGroup& smallInput, const rowgroup::R
nullValueForJoinColumn = smallNullRow.getSignedNullValue(smallJoinColumn); nullValueForJoinColumn = smallNullRow.getSignedNullValue(smallJoinColumn);
} }
// TupleJoiner::TupleJoiner(const rowgroup::RowGroup& smallInput, const rowgroup::RowGroup& largeInput,
// const vector<uint32_t>& smallJoinColumns, const vector<uint32_t>& largeJoinColumns,
// JoinType jt, threadpool::ThreadPool* jsThreadPool)
// : TupleJoiner(smallInput, largeInput, smallJoinColumns, largeJoinColumns, jt, jsThreadPool, nullptr)
// {
// }
// Typeless joiner ctor // Typeless joiner ctor
TupleJoiner::TupleJoiner(const rowgroup::RowGroup& smallInput, const rowgroup::RowGroup& largeInput, TupleJoiner::TupleJoiner(const rowgroup::RowGroup& smallInput, const rowgroup::RowGroup& largeInput,
const vector<uint32_t>& smallJoinColumns, const vector<uint32_t>& largeJoinColumns, const vector<uint32_t>& smallJoinColumns, const vector<uint32_t>& largeJoinColumns,
@ -180,14 +164,11 @@ TupleJoiner::TupleJoiner(const rowgroup::RowGroup& smallInput, const rowgroup::R
getBucketCount(); getBucketCount();
_pool.reset(new boost::shared_ptr<PoolAllocator>[bucketCount]);
ht.reset(new boost::scoped_ptr<typelesshash_t>[bucketCount]);
for (i = 0; i < bucketCount; i++) for (i = 0; i < bucketCount; i++)
{ {
// STLPoolAllocator<pair<const TypelessData, Row::Pointer>> alloc(resourceManager_);
// _pool[i] = alloc.getPoolAllocator();
auto alloc = resourceManager_->getAllocator<pair<const TypelessData, Row::Pointer>>(); auto alloc = resourceManager_->getAllocator<pair<const TypelessData, Row::Pointer>>();
ht[i].reset(new typelesshash_t(10, hasher(), typelesshash_t::key_equal(), alloc)); ht.emplace_back(std::unique_ptr<typelesshash_t>(
new typelesshash_t(10, hasher(), typelesshash_t::key_equal(), alloc)));
} }
m_bucketLocks.reset(new boost::mutex[bucketCount]); m_bucketLocks.reset(new boost::mutex[bucketCount]);
@ -239,11 +220,10 @@ TupleJoiner::TupleJoiner(const rowgroup::RowGroup& smallInput, const rowgroup::R
// note, 'numcores' is implied by tuplehashjoin on calls to insertRGData(). // note, 'numcores' is implied by tuplehashjoin on calls to insertRGData().
// TODO: make it explicit to avoid future confusion. // TODO: make it explicit to avoid future confusion.
storedKeyAlloc.reset(new FixedAllocator[numCores]);
for (i = 0; i < (uint)numCores; i++) for (i = 0; i < (uint)numCores; i++)
{ {
auto alloc = resourceManager_->getAllocator<utils::FixedAllocatorBufType>(); auto alloc = resourceManager_->getAllocator<utils::FixedAllocatorBufType>();
storedKeyAlloc[i] = FixedAllocator(alloc, keyLength); storedKeyAlloc.emplace_back(FixedAllocator(alloc, keyLength));
} }
} }
@ -279,7 +259,7 @@ void TupleJoiner::getBucketCount()
} }
template <typename buckets_t, typename hash_table_t> template <typename buckets_t, typename hash_table_t>
void TupleJoiner::bucketsToTables(buckets_t* buckets, hash_table_t* tables) void TupleJoiner::bucketsToTables(buckets_t* buckets, hash_table_t& tables)
{ {
uint i; uint i;
@ -301,7 +281,7 @@ void TupleJoiner::bucketsToTables(buckets_t* buckets, hash_table_t* tables)
} }
tables[i]->insert(buckets[i].begin(), buckets[i].end()); tables[i]->insert(buckets[i].begin(), buckets[i].end());
} }
wasProductive = true; wasProductive = true;
buckets[i].clear(); buckets[i].clear();
} }
@ -326,7 +306,7 @@ void TupleJoiner::um_insertTypeless(uint threadID, uint rowCount, Row& r)
uint bucket = bucketPicker((char*)td[i].data, td[i].len, bpSeed) & bucketMask; uint bucket = bucketPicker((char*)td[i].data, td[i].len, bpSeed) & bucketMask;
v[bucket].emplace_back(pair<TypelessData, Row::Pointer>(td[i], r.getPointer())); v[bucket].emplace_back(pair<TypelessData, Row::Pointer>(td[i], r.getPointer()));
} }
bucketsToTables(&v[0], ht.get()); bucketsToTables(&v[0], ht);
} }
void TupleJoiner::um_insertLongDouble(uint rowCount, Row& r) void TupleJoiner::um_insertLongDouble(uint rowCount, Row& r)
@ -345,7 +325,7 @@ void TupleJoiner::um_insertLongDouble(uint rowCount, Row& r)
else else
v[bucket].emplace_back(pair<long double, Row::Pointer>(smallKey, r.getPointer())); v[bucket].emplace_back(pair<long double, Row::Pointer>(smallKey, r.getPointer()));
} }
bucketsToTables(&v[0], ld.get()); bucketsToTables(&v[0], ld);
} }
void TupleJoiner::um_insertInlineRows(uint rowCount, Row& r) void TupleJoiner::um_insertInlineRows(uint rowCount, Row& r)
@ -367,7 +347,7 @@ void TupleJoiner::um_insertInlineRows(uint rowCount, Row& r)
else else
v[bucket].emplace_back(pair<int64_t, uint8_t*>(smallKey, r.getData())); v[bucket].emplace_back(pair<int64_t, uint8_t*>(smallKey, r.getData()));
} }
bucketsToTables(&v[0], h.get()); bucketsToTables(&v[0], h);
} }
void TupleJoiner::um_insertStringTable(uint rowCount, Row& r) void TupleJoiner::um_insertStringTable(uint rowCount, Row& r)
@ -389,7 +369,7 @@ void TupleJoiner::um_insertStringTable(uint rowCount, Row& r)
else else
v[bucket].emplace_back(pair<int64_t, Row::Pointer>(smallKey, r.getPointer())); v[bucket].emplace_back(pair<int64_t, Row::Pointer>(smallKey, r.getPointer()));
} }
bucketsToTables(&v[0], sth.get()); bucketsToTables(&v[0], sth);
} }
void TupleJoiner::insertRGData(RowGroup& rg, uint threadID) void TupleJoiner::insertRGData(RowGroup& rg, uint threadID)
@ -409,7 +389,7 @@ void TupleJoiner::insertRGData(RowGroup& rg, uint threadID)
r.zeroRid(); r.zeroRid();
} }
} }
rg.getRow(0, &r); rg.getRow(0, &r);
if (joinAlg == UM) if (joinAlg == UM)
@ -427,7 +407,7 @@ void TupleJoiner::insertRGData(RowGroup& rg, uint threadID)
{ {
// while in PM-join mode, inserting is single-threaded // while in PM-join mode, inserting is single-threaded
for (i = 0; i < rowCount; i++, r.nextRow()) for (i = 0; i < rowCount; i++, r.nextRow())
rows.push_back(r.getPointer()); rows->push_back(r.getPointer());
} }
} }
@ -492,7 +472,7 @@ void TupleJoiner::insert(Row& r, bool zeroTheRid)
} }
} }
else else
rows.push_back(r.getPointer()); rows->push_back(r.getPointer());
} }
void TupleJoiner::match(rowgroup::Row& largeSideRow, uint32_t largeRowIndex, uint32_t threadID, void TupleJoiner::match(rowgroup::Row& largeSideRow, uint32_t largeRowIndex, uint32_t threadID,
@ -508,8 +488,8 @@ void TupleJoiner::match(rowgroup::Row& largeSideRow, uint32_t largeRowIndex, uin
uint32_t size = v.size(); uint32_t size = v.size();
for (i = 0; i < size; i++) for (i = 0; i < size; i++)
if (v[i] < rows.size()) if (v[i] < rows->size())
matches->push_back(rows[v[i]]); matches->push_back((*rows)[v[i]]);
if (UNLIKELY((semiJoin() || antiJoin()) && matches->size() == 0)) if (UNLIKELY((semiJoin() || antiJoin()) && matches->size() == 0))
matches->push_back(smallNullRow.getPointer()); matches->push_back(smallNullRow.getPointer());
@ -536,7 +516,7 @@ void TupleJoiner::match(rowgroup::Row& largeSideRow, uint32_t largeRowIndex, uin
for (; range.first != range.second; ++range.first) for (; range.first != range.second; ++range.first)
matches->push_back(range.first->second); matches->push_back(range.first->second);
} }
else if (largeSideRow.getColType(largeKeyColumns[0]) == CalpontSystemCatalog::LONGDOUBLE && ld) else if (largeSideRow.getColType(largeKeyColumns[0]) == CalpontSystemCatalog::LONGDOUBLE && !ld.empty())
{ {
// This is a compare of two long double // This is a compare of two long double
long double largeKey; long double largeKey;
@ -572,7 +552,7 @@ void TupleJoiner::match(rowgroup::Row& largeSideRow, uint32_t largeRowIndex, uin
largeKey = largeSideRow.getIntField(largeKeyColumns[0]); largeKey = largeSideRow.getIntField(largeKeyColumns[0]);
} }
if (ld) if (!ld.empty())
{ {
// Compare against long double // Compare against long double
long double ldKey = largeKey; long double ldKey = largeKey;
@ -619,7 +599,7 @@ void TupleJoiner::match(rowgroup::Row& largeSideRow, uint32_t largeRowIndex, uin
if (UNLIKELY(inUM() && (joinType & MATCHNULLS) && !isNull && !typelessJoin)) if (UNLIKELY(inUM() && (joinType & MATCHNULLS) && !isNull && !typelessJoin))
{ {
if (largeRG.getColType(largeKeyColumns[0]) == CalpontSystemCatalog::LONGDOUBLE && ld) if (largeRG.getColType(largeKeyColumns[0]) == CalpontSystemCatalog::LONGDOUBLE && !ld.empty())
{ {
uint bucket = bucketPicker((char*)&(joblist::LONGDOUBLENULL), sizeof(joblist::LONGDOUBLENULL), bpSeed) & uint bucket = bucketPicker((char*)&(joblist::LONGDOUBLENULL), sizeof(joblist::LONGDOUBLENULL), bpSeed) &
bucketMask; bucketMask;
@ -749,7 +729,7 @@ void TupleJoiner::doneInserting()
for (i = 0; i < rowCount; i++) for (i = 0; i < rowCount; i++)
{ {
if (joinAlg == PM) if (joinAlg == PM)
smallRow.setPointer(rows[pmpos++]); smallRow.setPointer((*rows)[pmpos++]);
else if (typelessJoin) else if (typelessJoin)
{ {
while (thit == ht[bucket]->end()) while (thit == ht[bucket]->end())
@ -839,14 +819,13 @@ void TupleJoiner::umJoinConvert(size_t begin, size_t end)
while (begin < end) while (begin < end)
{ {
smallRow.setPointer(rows[begin++]); smallRow.setPointer((*rows)[begin++]);
insert(smallRow); insert(smallRow);
} }
} }
void TupleJoiner::setInUM() void TupleJoiner::setInUM()
{ {
vector<Row::Pointer> empty;
Row smallRow; Row smallRow;
uint32_t i, size; uint32_t i, size;
@ -854,7 +833,7 @@ void TupleJoiner::setInUM()
return; return;
joinAlg = UM; joinAlg = UM;
size = rows.size(); size = rows->size();
size_t chunkSize = size_t chunkSize =
((size / numCores) + 1 < 50000 ? 50000 ((size / numCores) + 1 < 50000 ? 50000
: (size / numCores) + 1); // don't start a thread to process < 50k rows : (size / numCores) + 1); // don't start a thread to process < 50k rows
@ -872,17 +851,15 @@ void TupleJoiner::setInUM()
#ifdef TJ_DEBUG #ifdef TJ_DEBUG
cout << "done\n"; cout << "done\n";
#endif #endif
rows.swap(empty); auto alloc = resourceManager_->getAllocator<rowgroup::Row::Pointer>();
rows.reset(new RowPointersVec(alloc));
if (typelessJoin) if (typelessJoin)
{ {
tmpKeyAlloc.reset(new FixedAllocator[threadCount]);
for (i = 0; i < threadCount; i++) for (i = 0; i < threadCount; i++)
{ {
auto alloc = resourceManager_->getAllocator<utils::FixedAllocatorBufType>(); auto alloc = resourceManager_->getAllocator<utils::FixedAllocatorBufType>();
tmpKeyAlloc[i] = FixedAllocator(alloc, keyLength, true); tmpKeyAlloc.emplace_back(FixedAllocator(alloc, keyLength, true));
} }
} }
} }
@ -909,8 +886,8 @@ void TupleJoiner::setInUM(vector<RGData>& rgs)
return; return;
{ // don't need rows anymore, free the mem { // don't need rows anymore, free the mem
vector<Row::Pointer> empty; auto alloc = resourceManager_->getAllocator<rowgroup::Row::Pointer>();
rows.swap(empty); rows.reset(new RowPointersVec(alloc));
} }
joinAlg = UM; joinAlg = UM;
@ -935,12 +912,10 @@ void TupleJoiner::setInUM(vector<RGData>& rgs)
if (typelessJoin) if (typelessJoin)
{ {
tmpKeyAlloc.reset(new FixedAllocator[threadCount]);
for (i = 0; i < threadCount; i++) for (i = 0; i < threadCount; i++)
{ {
auto alloc = resourceManager_->getAllocator<utils::FixedAllocatorBufType>(); auto alloc = resourceManager_->getAllocator<utils::FixedAllocatorBufType>();
tmpKeyAlloc[i] = FixedAllocator(alloc, keyLength, true); tmpKeyAlloc.emplace_back(FixedAllocator(alloc, keyLength, true));
} }
} }
} }
@ -958,9 +933,9 @@ void TupleJoiner::markMatches(uint32_t threadID, uint32_t rowCount)
for (i = 0; i < rowCount; i++) for (i = 0; i < rowCount; i++)
for (j = 0; j < matches[i].size(); j++) for (j = 0; j < matches[i].size(); j++)
{ {
if (matches[i][j] < rows.size()) if (matches[i][j] < rows->size())
{ {
smallRow[threadID].setPointer(rows[matches[i][j]]); smallRow[threadID].setPointer((*rows)[matches[i][j]]);
smallRow[threadID].markRow(); smallRow[threadID].markRow();
} }
} }
@ -994,12 +969,10 @@ void TupleJoiner::setThreadCount(uint32_t cnt)
if (typelessJoin) if (typelessJoin)
{ {
tmpKeyAlloc.reset(new FixedAllocator[threadCount]);
for (uint32_t i = 0; i < threadCount; i++) for (uint32_t i = 0; i < threadCount; i++)
{ {
auto alloc = resourceManager_->getAllocator<utils::FixedAllocatorBufType>(); auto alloc = resourceManager_->getAllocator<utils::FixedAllocatorBufType>();
tmpKeyAlloc[i] = FixedAllocator(alloc, keyLength, true); tmpKeyAlloc.emplace_back(FixedAllocator(alloc, keyLength, true));
} }
} }
@ -1023,14 +996,14 @@ void TupleJoiner::getUnmarkedRows(vector<Row::Pointer>* out)
{ {
uint32_t i, size; uint32_t i, size;
size = rows.size(); size = rows->size();
for (i = 0; i < size; i++) for (i = 0; i < size; i++)
{ {
smallR.setPointer(rows[i]); smallR.setPointer((*rows)[i]);
if (!smallR.isMarked()) if (!smallR.isMarked())
out->push_back(rows[i]); out->push_back((*rows)[i]);
} }
} }
else else
@ -1090,28 +1063,6 @@ void TupleJoiner::getUnmarkedRows(vector<Row::Pointer>* out)
} }
} }
uint64_t TupleJoiner::getMemUsage() const
{
if (inUM() && typelessJoin)
{
size_t ret = 0;
for (uint i = 0; i < bucketCount; i++)
ret += _pool[i]->getMemUsage();
for (int i = 0; i < numCores; i++)
ret += storedKeyAlloc[i].getMemUsage();
return ret;
}
else if (inUM())
{
size_t ret = 0;
for (uint i = 0; i < bucketCount; i++)
ret += _pool[i]->getMemUsage();
return ret;
}
else
return (rows.size() * sizeof(Row::Pointer));
}
void TupleJoiner::setFcnExpFilter(boost::shared_ptr<funcexp::FuncExpWrapper> pt) void TupleJoiner::setFcnExpFilter(boost::shared_ptr<funcexp::FuncExpWrapper> pt)
{ {
fe = pt; fe = pt;
@ -1250,7 +1201,7 @@ size_t TupleJoiner::size() const
return ret; return ret;
} }
return rows.size(); return rows->size();
} }
class TypelessDataStringEncoder class TypelessDataStringEncoder
@ -1821,20 +1772,9 @@ void TupleJoiner::setTableName(const string& tname)
void TupleJoiner::clearData() void TupleJoiner::clearData()
{ {
_pool.reset(new boost::shared_ptr<utils::PoolAllocator>[bucketCount]); // This loop calls dtors and deallocates mem.
if (typelessJoin)
ht.reset(new boost::scoped_ptr<typelesshash_t>[bucketCount]);
else if (smallRG.getColTypes()[smallKeyColumns[0]] == CalpontSystemCatalog::LONGDOUBLE)
ld.reset(new boost::scoped_ptr<ldhash_t>[bucketCount]);
else if (smallRG.usesStringTable())
sth.reset(new boost::scoped_ptr<sthash_t>[bucketCount]);
else
h.reset(new boost::scoped_ptr<hash_t>[bucketCount]);
for (uint i = 0; i < bucketCount; i++) for (uint i = 0; i < bucketCount; i++)
{ {
// STLPoolAllocator<pair<const TypelessData, Row::Pointer>> alloc(resourceManager_);
// _pool[i] = alloc.getPoolAllocator();
auto alloc = resourceManager_->getAllocator<pair<const TypelessData, Row::Pointer>>(); auto alloc = resourceManager_->getAllocator<pair<const TypelessData, Row::Pointer>>();
if (typelessJoin) if (typelessJoin)
ht[i].reset(new typelesshash_t(10, hasher(), typelesshash_t::key_equal(), alloc)); ht[i].reset(new typelesshash_t(10, hasher(), typelesshash_t::key_equal(), alloc));
@ -1843,11 +1783,13 @@ void TupleJoiner::clearData()
else if (smallRG.usesStringTable()) else if (smallRG.usesStringTable())
sth[i].reset(new sthash_t(10, hasher(), sthash_t::key_equal(), alloc)); sth[i].reset(new sthash_t(10, hasher(), sthash_t::key_equal(), alloc));
else else
{
h[i].reset(new hash_t(10, hasher(), hash_t::key_equal(), alloc)); h[i].reset(new hash_t(10, hasher(), hash_t::key_equal(), alloc));
}
} }
std::vector<rowgroup::Row::Pointer> empty; auto alloc = resourceManager_->getAllocator<rowgroup::Row::Pointer>();
rows.swap(empty); rows.reset(new RowPointersVec(alloc));
finished = false; finished = false;
} }
@ -1910,11 +1852,10 @@ std::shared_ptr<TupleJoiner> TupleJoiner::copyForDiskJoin()
if (typelessJoin) if (typelessJoin)
{ {
ret->storedKeyAlloc.reset(new FixedAllocator[numCores]);
for (int i = 0; i < numCores; i++) for (int i = 0; i < numCores; i++)
{ {
auto alloc = resourceManager_->getAllocator<utils::FixedAllocatorBufType>(); auto alloc = resourceManager_->getAllocator<utils::FixedAllocatorBufType>();
storedKeyAlloc[i] = FixedAllocator(alloc, keyLength); storedKeyAlloc.emplace_back(FixedAllocator(alloc, keyLength));
} }
} }

View File

@ -204,6 +204,9 @@ class TypelessDataStructure
} }
}; };
using RowPointersVec =
std::vector<rowgroup::Row::Pointer, allocators::CountingAllocator<rowgroup::Row::Pointer>>;
using RowPointersVecUP = std::unique_ptr<RowPointersVec>;
class TupleJoiner class TupleJoiner
{ {
public: public:
@ -268,20 +271,12 @@ class TupleJoiner
}; };
/* ctor to use for numeric join */ /* ctor to use for numeric join */
// TupleJoiner(const rowgroup::RowGroup& smallInput, const rowgroup::RowGroup& largeInput,
// uint32_t smallJoinColumn, uint32_t largeJoinColumn, joblist::JoinType jt,
// threadpool::ThreadPool* jsThreadPool);
TupleJoiner(const rowgroup::RowGroup& smallInput, const rowgroup::RowGroup& largeInput, TupleJoiner(const rowgroup::RowGroup& smallInput, const rowgroup::RowGroup& largeInput,
uint32_t smallJoinColumn, uint32_t largeJoinColumn, joblist::JoinType jt, uint32_t smallJoinColumn, uint32_t largeJoinColumn, joblist::JoinType jt,
threadpool::ThreadPool* jsThreadPool, joblist::ResourceManager* rm, const uint64_t numCores); threadpool::ThreadPool* jsThreadPool, joblist::ResourceManager* rm, const uint64_t numCores);
/* ctor to use for string & compound join */ /* ctor to use for string & compound join */
// TupleJoiner(const rowgroup::RowGroup& smallInput, const rowgroup::RowGroup& largeInput, TupleJoiner(const rowgroup::RowGroup& smallInput, const rowgroup::RowGroup& largeInput,
// const std::vector<uint32_t>& smallJoinColumns, const std::vector<uint32_t>& largeJoinColumns,
// joblist::JoinType jt, threadpool::ThreadPool* jsThreadPool);
TupleJoiner(const rowgroup::RowGroup& smallInput, const rowgroup::RowGroup& largeInput,
const std::vector<uint32_t>& smallJoinColumns, const std::vector<uint32_t>& largeJoinColumns, const std::vector<uint32_t>& smallJoinColumns, const std::vector<uint32_t>& largeJoinColumns,
joblist::JoinType jt, threadpool::ThreadPool* jsThreadPool, joblist::ResourceManager* rm, const uint64_t numCores); joblist::JoinType jt, threadpool::ThreadPool* jsThreadPool, joblist::ResourceManager* rm, const uint64_t numCores);
@ -333,9 +328,9 @@ class TupleJoiner
void setThreadCount(uint32_t cnt); void setThreadCount(uint32_t cnt);
void setPMJoinResults(std::shared_ptr<std::vector<uint32_t>[]>, uint32_t threadID); void setPMJoinResults(std::shared_ptr<std::vector<uint32_t>[]>, uint32_t threadID);
std::shared_ptr<std::vector<uint32_t>[]> getPMJoinArrays(uint32_t threadID); std::shared_ptr<std::vector<uint32_t>[]> getPMJoinArrays(uint32_t threadID);
std::vector<rowgroup::Row::Pointer>* getSmallSide() RowPointersVec& getSmallSide()
{ {
return &rows; return *rows;
} }
inline bool smallOuterJoin() inline bool smallOuterJoin()
{ {
@ -381,8 +376,6 @@ class TupleJoiner
/* To allow sorting */ /* To allow sorting */
bool operator<(const TupleJoiner&) const; bool operator<(const TupleJoiner&) const;
uint64_t getMemUsage() const;
/* Typeless join interface */ /* Typeless join interface */
inline bool isTypelessJoin() inline bool isTypelessJoin()
{ {
@ -410,7 +403,7 @@ class TupleJoiner
{ {
return discreteValues; return discreteValues;
} }
inline const boost::scoped_array<std::vector<int128_t> >& getCPData() inline const boost::scoped_array<std::vector<int128_t>>& getCPData()
{ {
return cpValues; return cpValues;
} }
@ -478,37 +471,22 @@ class TupleJoiner
} }
private: private:
// typedef std::unordered_multimap<int64_t, uint8_t*, hasher, std::equal_to<int64_t>, typedef std::unordered_multimap<int64_t, uint8_t*, hasher, std::equal_to<int64_t>,
// utils::STLPoolAllocator<std::pair<const int64_t, uint8_t*> > > allocators::CountingAllocator<std::pair<const int64_t, uint8_t*>>>
// hash_t; hash_t;
// typedef std::unordered_multimap<int64_t, rowgroup::Row::Pointer, hasher, std::equal_to<int64_t>, typedef std::unordered_multimap<
// utils::STLPoolAllocator<std::pair<const int64_t, rowgroup::Row::Pointer> > > int64_t, rowgroup::Row::Pointer, hasher, std::equal_to<int64_t>,
// sthash_t; allocators::CountingAllocator<std::pair<const int64_t, rowgroup::Row::Pointer>>>
// typedef std::unordered_multimap< sthash_t;
// TypelessData, rowgroup::Row::Pointer, hasher, std::equal_to<TypelessData>, typedef std::unordered_multimap<
// utils::STLPoolAllocator<std::pair<const TypelessData, rowgroup::Row::Pointer> > > TypelessData, rowgroup::Row::Pointer, hasher, std::equal_to<TypelessData>,
// typelesshash_t; allocators::CountingAllocator<std::pair<const TypelessData, rowgroup::Row::Pointer>>>
// // MCOL-1822 Add support for Long Double AVG/SUM small side typelesshash_t;
// typedef std::unordered_multimap< // MCOL-1822 Add support for Long Double AVG/SUM small side
// long double, rowgroup::Row::Pointer, hasher, LongDoubleEq, typedef std::unordered_multimap<
// utils::STLPoolAllocator<std::pair<const long double, rowgroup::Row::Pointer> > > long double, rowgroup::Row::Pointer, hasher, LongDoubleEq,
// ldhash_t; allocators::CountingAllocator<std::pair<const long double, rowgroup::Row::Pointer>>>
ldhash_t;
typedef std::unordered_multimap<int64_t, uint8_t*, hasher, std::equal_to<int64_t>,
allocators::CountingAllocator<std::pair<const int64_t, uint8_t*> > >
hash_t;
typedef std::unordered_multimap<int64_t, rowgroup::Row::Pointer, hasher, std::equal_to<int64_t>,
allocators::CountingAllocator<std::pair<const int64_t, rowgroup::Row::Pointer> > >
sthash_t;
typedef std::unordered_multimap<
TypelessData, rowgroup::Row::Pointer, hasher, std::equal_to<TypelessData>,
allocators::CountingAllocator<std::pair<const TypelessData, rowgroup::Row::Pointer> > >
typelesshash_t;
// MCOL-1822 Add support for Long Double AVG/SUM small side
typedef std::unordered_multimap<
long double, rowgroup::Row::Pointer, hasher, LongDoubleEq,
allocators::CountingAllocator<std::pair<const long double, rowgroup::Row::Pointer> > >
ldhash_t;
typedef hash_t::iterator iterator; typedef hash_t::iterator iterator;
typedef typelesshash_t::iterator thIterator; typedef typelesshash_t::iterator thIterator;
@ -521,11 +499,11 @@ class TupleJoiner
rowgroup::RGData smallNullMemory; rowgroup::RGData smallNullMemory;
boost::scoped_array<boost::scoped_ptr<hash_t> > h; // used for UM joins on ints std::vector<std::unique_ptr<hash_t>> h; // used for UM joins on ints
boost::scoped_array<boost::scoped_ptr<sthash_t> > std::vector<std::unique_ptr<sthash_t>>
sth; // used for UM join on ints where the backing table uses a string table sth; // used for UM join on ints where the backing table uses a string table
boost::scoped_array<boost::scoped_ptr<ldhash_t> > ld; // used for UM join on long double std::vector<std::unique_ptr<ldhash_t>> ld; // used for UM join on long double
std::vector<rowgroup::Row::Pointer> rows; // used for PM join RowPointersVecUP rows; // used for PM join
/* This struct is rough. The BPP-JL stores the parsed results for /* This struct is rough. The BPP-JL stores the parsed results for
the logical block being processed. There are X threads at once, so the logical block being processed. There are X threads at once, so
@ -546,18 +524,16 @@ class TupleJoiner
}; };
JoinAlg joinAlg; JoinAlg joinAlg;
joblist::JoinType joinType; joblist::JoinType joinType;
// WIP
std::shared_ptr<boost::shared_ptr<utils::PoolAllocator>[]> _pool; // pools for the table and nodes
uint32_t threadCount; uint32_t threadCount;
std::string tableName; std::string tableName;
/* vars, & fcns for typeless join */ /* vars, & fcns for typeless join */
bool typelessJoin; bool typelessJoin;
std::vector<uint32_t> smallKeyColumns, largeKeyColumns; std::vector<uint32_t> smallKeyColumns, largeKeyColumns;
boost::scoped_array<boost::scoped_ptr<typelesshash_t> > ht; // used for UM join on strings std::vector<std::unique_ptr<typelesshash_t>> ht; // used for UM join on strings
uint32_t keyLength; uint32_t keyLength;
boost::scoped_array<utils::FixedAllocator> storedKeyAlloc; std::vector<utils::FixedAllocator> storedKeyAlloc;
boost::scoped_array<utils::FixedAllocator> tmpKeyAlloc; std::vector<utils::FixedAllocator> tmpKeyAlloc;
bool bSignedUnsignedJoin; // Set if we have a signed vs unsigned compare in a join. When not set, we can bool bSignedUnsignedJoin; // Set if we have a signed vs unsigned compare in a join. When not set, we can
// save checking for the signed bit. // save checking for the signed bit.
@ -571,7 +547,7 @@ class TupleJoiner
/* Runtime casual partitioning support */ /* Runtime casual partitioning support */
void updateCPData(const rowgroup::Row& r); void updateCPData(const rowgroup::Row& r);
boost::scoped_array<bool> discreteValues; boost::scoped_array<bool> discreteValues;
boost::scoped_array<std::vector<int128_t> > cpValues; // if !discreteValues, [0] has min, [1] has max boost::scoped_array<std::vector<int128_t>> cpValues; // if !discreteValues, [0] has min, [1] has max
uint32_t uniqueLimit; uint32_t uniqueLimit;
bool finished; bool finished;
@ -590,7 +566,7 @@ class TupleJoiner
void um_insertStringTable(uint rowcount, rowgroup::Row& r); void um_insertStringTable(uint rowcount, rowgroup::Row& r);
template <typename buckets_t, typename hash_table_t> template <typename buckets_t, typename hash_table_t>
void bucketsToTables(buckets_t*, hash_table_t*); void bucketsToTables(buckets_t*, hash_table_t&);
bool _convertToDiskJoin; bool _convertToDiskJoin;
joblist::ResourceManager* resourceManager_ = nullptr; joblist::ResourceManager* resourceManager_ = nullptr;