You've already forked mariadb-columnstore-engine
mirror of
https://github.com/mariadb-corporation/mariadb-columnstore-engine.git
synced 2025-06-12 05:01:56 +03:00
feat(PP,ByteStream): new counting memory allocator
This commit is contained in:
@ -213,13 +213,6 @@ BatchPrimitiveProcessor::BatchPrimitiveProcessor(ByteStream& b, double prefetch,
|
||||
initBPP(b);
|
||||
}
|
||||
|
||||
#if 0
|
||||
BatchPrimitiveProcessor::BatchPrimitiveProcessor(const BatchPrimitiveProcessor& bpp)
|
||||
{
|
||||
throw logic_error("copy BPP deprecated");
|
||||
}
|
||||
#endif
|
||||
|
||||
BatchPrimitiveProcessor::~BatchPrimitiveProcessor()
|
||||
{
|
||||
// FIXME: just do a sync fetch
|
||||
@ -247,6 +240,8 @@ void BatchPrimitiveProcessor::initBPP(ByteStream& bs)
|
||||
uint8_t tmp8;
|
||||
uint16_t tmp16;
|
||||
Command::CommandType type;
|
||||
auto cnt = exemgr::globServiceExeMgr->getRm().availableMemory();
|
||||
std::cout << "initBPP availableMemory: " << cnt << std::endl;
|
||||
|
||||
bs.advance(sizeof(ISMPacketHeader)); // skip the header
|
||||
bs >> tmp8;
|
||||
@ -365,13 +360,17 @@ void BatchPrimitiveProcessor::initBPP(ByteStream& bs)
|
||||
|
||||
if (!typelessJoin[i])
|
||||
{
|
||||
auto alloc = exemgr::globServiceExeMgr->getRm().getAllocator<TJoiner::value_type>();
|
||||
|
||||
bs >> joinNullValues[i];
|
||||
bs >> largeSideKeyColumns[i];
|
||||
for (uint j = 0; j < processorThreads; ++j)
|
||||
tJoiners[i][j].reset(new TJoiner(10, TupleJoiner::hasher()));
|
||||
tJoiners[i][j].reset(new TJoiner(10, TupleJoiner::hasher(), alloc));
|
||||
}
|
||||
else
|
||||
{
|
||||
auto alloc = exemgr::globServiceExeMgr->getRm().getAllocator<TLJoiner::value_type>();
|
||||
|
||||
deserializeVector<uint32_t>(bs, tlLargeSideKeyColumns[i]);
|
||||
bs >> tlSmallSideKeyLengths[i];
|
||||
bs >> tmp8;
|
||||
@ -393,7 +392,7 @@ void BatchPrimitiveProcessor::initBPP(ByteStream& bs)
|
||||
mSmallSideKeyColumnsPtr, mSmallSideRGPtr);
|
||||
auto tlComparator = TupleJoiner::TypelessDataComparator(&outputRG, &tlLargeSideKeyColumns[i],
|
||||
mSmallSideKeyColumnsPtr, mSmallSideRGPtr);
|
||||
tlJoiners[i][j].reset(new TLJoiner(10, tlHasher, tlComparator));
|
||||
tlJoiners[i][j].reset(new TLJoiner(10, tlHasher, tlComparator, alloc));
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -497,7 +496,7 @@ void BatchPrimitiveProcessor::initBPP(ByteStream& bs)
|
||||
bs >> *(fAggregator.get());
|
||||
|
||||
// If there's UDAF involved, set up for PM processing
|
||||
for (const auto & pcol : fAggregator->getAggFunctions())
|
||||
for (const auto& pcol : fAggregator->getAggFunctions())
|
||||
{
|
||||
auto* rowUDAF = dynamic_cast<RowUDAFFunctionCol*>(pcol.get());
|
||||
|
||||
@ -843,6 +842,8 @@ int BatchPrimitiveProcessor::endOfJoiner()
|
||||
{
|
||||
endOfJoinerRan = true;
|
||||
pthread_mutex_unlock(&objLock);
|
||||
auto cnt = exemgr::globServiceExeMgr->getRm().availableMemory();
|
||||
std::cout << "endOfJoiner availableMemory: " << cnt << std::endl;
|
||||
return 0;
|
||||
}
|
||||
|
||||
@ -885,6 +886,8 @@ int BatchPrimitiveProcessor::endOfJoiner()
|
||||
endOfJoinerRan = true;
|
||||
|
||||
pthread_mutex_unlock(&objLock);
|
||||
auto cnt = exemgr::globServiceExeMgr->getRm().availableMemory();
|
||||
std::cout << "endOfJoiner availableMemory: " << cnt << std::endl;
|
||||
return 0;
|
||||
}
|
||||
|
||||
@ -1218,7 +1221,7 @@ uint32_t BatchPrimitiveProcessor::executeTupleJoin(uint32_t startRid, RowGroup&
|
||||
{
|
||||
bool hasNull = false;
|
||||
|
||||
for (unsigned int column: tlLargeSideKeyColumns[j])
|
||||
for (unsigned int column : tlLargeSideKeyColumns[j])
|
||||
if (oldRow.isNullValue(column))
|
||||
{
|
||||
hasNull = true;
|
||||
@ -1374,7 +1377,7 @@ uint32_t BatchPrimitiveProcessor::executeTupleJoin(uint32_t startRid, RowGroup&
|
||||
#ifdef PRIMPROC_STOPWATCH
|
||||
void BatchPrimitiveProcessor::execute(StopWatch* stopwatch)
|
||||
#else
|
||||
void BatchPrimitiveProcessor::execute()
|
||||
void BatchPrimitiveProcessor::execute(messageqcpp::SBS& bs)
|
||||
#endif
|
||||
{
|
||||
uint8_t sendCount = 0;
|
||||
@ -1509,7 +1512,7 @@ void BatchPrimitiveProcessor::execute()
|
||||
writeProjectionPreamble();
|
||||
stopwatch->stop("- writeProjectionPreamble");
|
||||
#else
|
||||
writeProjectionPreamble();
|
||||
writeProjectionPreamble(bs);
|
||||
#endif
|
||||
}
|
||||
|
||||
@ -1536,7 +1539,7 @@ void BatchPrimitiveProcessor::execute()
|
||||
{
|
||||
for (j = 0; j < projectCount; ++j)
|
||||
{
|
||||
projectSteps[j]->project();
|
||||
projectSteps[j]->project(bs);
|
||||
}
|
||||
}
|
||||
else
|
||||
@ -1641,9 +1644,9 @@ void BatchPrimitiveProcessor::execute()
|
||||
|
||||
if (!fAggregator)
|
||||
{
|
||||
*serialized << (uint8_t)1; // the "count this msg" var
|
||||
*bs << (uint8_t)1; // the "count this msg" var
|
||||
fe2Output.setDBRoot(dbRoot);
|
||||
fe2Output.serializeRGData(*serialized);
|
||||
fe2Output.serializeRGData(*bs);
|
||||
//*serialized << fe2Output.getDataSize();
|
||||
// serialized->append(fe2Output.getData(), fe2Output.getDataSize());
|
||||
}
|
||||
@ -1653,7 +1656,7 @@ void BatchPrimitiveProcessor::execute()
|
||||
{
|
||||
utils::setThreadName("BPPAgg_1");
|
||||
|
||||
*serialized << (uint8_t)1; // the "count this msg" var
|
||||
*bs << (uint8_t)1; // the "count this msg" var
|
||||
|
||||
// see TupleBPS::setFcnExpGroup2() and where it gets called.
|
||||
// it sets fe2 there, on the other side of communication.
|
||||
@ -1669,25 +1672,25 @@ void BatchPrimitiveProcessor::execute()
|
||||
|
||||
if ((currentBlockOffset + 1) == count) // @bug4507, 8k
|
||||
{
|
||||
fAggregator->loadResult(*serialized); // @bug4507, 8k
|
||||
fAggregator->loadResult(*bs); // @bug4507, 8k
|
||||
} // @bug4507, 8k
|
||||
else if (utils::MonitorProcMem::isMemAvailable()) // @bug4507, 8k
|
||||
{
|
||||
fAggregator->loadEmptySet(*serialized); // @bug4507, 8k
|
||||
fAggregator->loadEmptySet(*bs); // @bug4507, 8k
|
||||
} // @bug4507, 8k
|
||||
else // @bug4507, 8k
|
||||
{
|
||||
fAggregator->loadResult(*serialized); // @bug4507, 8k
|
||||
fAggregator->aggReset(); // @bug4507, 8k
|
||||
fAggregator->loadResult(*bs); // @bug4507, 8k
|
||||
fAggregator->aggReset(); // @bug4507, 8k
|
||||
} // @bug4507, 8k
|
||||
}
|
||||
|
||||
if (!fAggregator && !fe2)
|
||||
{
|
||||
*serialized << (uint8_t)1; // the "count this msg" var
|
||||
*bs << (uint8_t)1; // the "count this msg" var
|
||||
outputRG.setDBRoot(dbRoot);
|
||||
// cerr << "serializing " << outputRG.toString() << endl;
|
||||
outputRG.serializeRGData(*serialized);
|
||||
outputRG.serializeRGData(*bs);
|
||||
|
||||
//*serialized << outputRG.getDataSize();
|
||||
// serialized->append(outputRG.getData(), outputRG.getDataSize());
|
||||
@ -1700,7 +1703,7 @@ void BatchPrimitiveProcessor::execute()
|
||||
else // Is doJoin
|
||||
{
|
||||
uint32_t startRid = 0;
|
||||
ByteStream preamble = *serialized;
|
||||
ByteStream preamble = *bs;
|
||||
origRidCount = ridCount; // ridCount can get modified by executeTupleJoin(). We need to keep track of
|
||||
// the original val.
|
||||
/* project the key columns. If there's the filter IN the join, project everything.
|
||||
@ -1783,7 +1786,7 @@ void BatchPrimitiveProcessor::execute()
|
||||
sendCount = (uint8_t)(!moreRGs && !startRid);
|
||||
// *serialized << (uint8_t)(!moreRGs && !startRid); // the "count
|
||||
// this msg" var
|
||||
*serialized << sendCount;
|
||||
*bs << sendCount;
|
||||
if (fe2)
|
||||
{
|
||||
utils::setThreadName("BPPFE2_2");
|
||||
@ -1817,30 +1820,30 @@ void BatchPrimitiveProcessor::execute()
|
||||
|
||||
if ((currentBlockOffset + 1) == count && moreRGs == false && startRid == 0) // @bug4507, 8k
|
||||
{
|
||||
fAggregator->loadResult(*serialized); // @bug4507, 8k
|
||||
fAggregator->loadResult(*bs); // @bug4507, 8k
|
||||
} // @bug4507, 8k
|
||||
else if (utils::MonitorProcMem::isMemAvailable()) // @bug4507, 8k
|
||||
{
|
||||
fAggregator->loadEmptySet(*serialized); // @bug4507, 8k
|
||||
fAggregator->loadEmptySet(*bs); // @bug4507, 8k
|
||||
} // @bug4507, 8k
|
||||
else // @bug4507, 8k
|
||||
{
|
||||
fAggregator->loadResult(*serialized); // @bug4507, 8k
|
||||
fAggregator->aggReset(); // @bug4507, 8k
|
||||
fAggregator->loadResult(*bs); // @bug4507, 8k
|
||||
fAggregator->aggReset(); // @bug4507, 8k
|
||||
} // @bug4507, 8k
|
||||
}
|
||||
else
|
||||
{
|
||||
// cerr <<" * serialzing " << nextRG.toString() << endl;
|
||||
nextRG.serializeRGData(*serialized);
|
||||
nextRG.serializeRGData(*bs);
|
||||
}
|
||||
|
||||
/* send the msg & reinit the BS */
|
||||
if (moreRGs)
|
||||
{
|
||||
sendResponse();
|
||||
serialized.reset(new ByteStream());
|
||||
*serialized = preamble;
|
||||
sendResponse(bs);
|
||||
bs.reset(new ByteStream());
|
||||
*bs = preamble;
|
||||
}
|
||||
}
|
||||
|
||||
@ -1848,16 +1851,16 @@ void BatchPrimitiveProcessor::execute()
|
||||
{
|
||||
// Should we happen to finish sending data rows right on the boundary of when moreRGs flips off,
|
||||
// then we need to start a new buffer. I.e., it needs the count this message byte pushed.
|
||||
if (serialized->length() == preamble.length())
|
||||
*serialized << (uint8_t)(startRid > 0 ? 0 : 1); // the "count this msg" var
|
||||
if (bs->length() == preamble.length())
|
||||
*bs << (uint8_t)(startRid > 0 ? 0 : 1); // the "count this msg" var
|
||||
|
||||
*serialized << ridCount;
|
||||
*bs << ridCount;
|
||||
|
||||
for (i = 0; i < joinerCount; i++)
|
||||
{
|
||||
for (j = 0; j < ridCount; ++j)
|
||||
{
|
||||
serializeInlineVector<uint32_t>(*serialized, tSmallSideMatches[i][j]);
|
||||
serializeInlineVector<uint32_t>(*bs, tSmallSideMatches[i][j]);
|
||||
tSmallSideMatches[i][j].clear();
|
||||
}
|
||||
}
|
||||
@ -1872,10 +1875,10 @@ void BatchPrimitiveProcessor::execute()
|
||||
}
|
||||
else
|
||||
{
|
||||
*serialized << (uint8_t)(startRid > 0 ? 0 : 1); // the "count this msg" var
|
||||
*bs << (uint8_t)(startRid > 0 ? 0 : 1); // the "count this msg" var
|
||||
outputRG.setDBRoot(dbRoot);
|
||||
// cerr << "serializing " << outputRG.toString() << endl;
|
||||
outputRG.serializeRGData(*serialized);
|
||||
outputRG.serializeRGData(*bs);
|
||||
|
||||
//*serialized << outputRG.getDataSize();
|
||||
// serialized->append(outputRG.getData(), outputRG.getDataSize());
|
||||
@ -1883,16 +1886,16 @@ void BatchPrimitiveProcessor::execute()
|
||||
{
|
||||
for (j = 0; j < ridCount; ++j)
|
||||
{
|
||||
serializeInlineVector<uint32_t>(*serialized, tSmallSideMatches[i][j]);
|
||||
serializeInlineVector<uint32_t>(*bs, tSmallSideMatches[i][j]);
|
||||
tSmallSideMatches[i][j].clear();
|
||||
}
|
||||
}
|
||||
}
|
||||
if (startRid > 0)
|
||||
{
|
||||
sendResponse();
|
||||
serialized.reset(new ByteStream());
|
||||
*serialized = preamble;
|
||||
sendResponse(bs);
|
||||
bs.reset(new ByteStream());
|
||||
*bs = preamble;
|
||||
}
|
||||
} while (startRid > 0);
|
||||
}
|
||||
@ -1905,11 +1908,11 @@ void BatchPrimitiveProcessor::execute()
|
||||
// sendCount << std::endl;
|
||||
if (projectCount > 0 || ot == ROW_GROUP)
|
||||
{
|
||||
*serialized << cachedIO;
|
||||
*bs << cachedIO;
|
||||
cachedIO = 0;
|
||||
*serialized << physIO;
|
||||
*bs << physIO;
|
||||
physIO = 0;
|
||||
*serialized << touchedBlocks;
|
||||
*bs << touchedBlocks;
|
||||
touchedBlocks = 0;
|
||||
// cout << "sent physIO=" << physIO << " cachedIO=" << cachedIO <<
|
||||
// " touchedBlocks=" << touchedBlocks << endl;
|
||||
@ -1922,15 +1925,15 @@ void BatchPrimitiveProcessor::execute()
|
||||
}
|
||||
catch (logging::QueryDataExcept& qex)
|
||||
{
|
||||
writeErrorMsg(qex.what(), qex.errorCode());
|
||||
writeErrorMsg(bs, qex.what(), qex.errorCode());
|
||||
}
|
||||
catch (logging::DictionaryBufferOverflow& db)
|
||||
{
|
||||
writeErrorMsg(db.what(), db.errorCode());
|
||||
writeErrorMsg(bs, db.what(), db.errorCode());
|
||||
}
|
||||
catch (scalar_exception& se)
|
||||
{
|
||||
writeErrorMsg(IDBErrorInfo::instance()->errorMsg(ERR_MORE_THAN_1_ROW), ERR_MORE_THAN_1_ROW, false);
|
||||
writeErrorMsg(bs, IDBErrorInfo::instance()->errorMsg(ERR_MORE_THAN_1_ROW), ERR_MORE_THAN_1_ROW, false);
|
||||
}
|
||||
catch (NeedToRestartJob& n)
|
||||
{
|
||||
@ -1941,20 +1944,21 @@ void BatchPrimitiveProcessor::execute()
|
||||
}
|
||||
catch (IDBExcept& iex)
|
||||
{
|
||||
writeErrorMsg(iex.what(), iex.errorCode(), true, false);
|
||||
writeErrorMsg(bs, iex.what(), iex.errorCode(), true, false);
|
||||
}
|
||||
catch (const std::exception& ex)
|
||||
{
|
||||
writeErrorMsg(ex.what(), logging::batchPrimitiveProcessorErr);
|
||||
writeErrorMsg(bs, ex.what(), logging::batchPrimitiveProcessorErr);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
string msg("BatchPrimitiveProcessor caught an unknown exception");
|
||||
writeErrorMsg(msg, logging::batchPrimitiveProcessorErr);
|
||||
writeErrorMsg(bs, msg, logging::batchPrimitiveProcessorErr);
|
||||
}
|
||||
}
|
||||
|
||||
void BatchPrimitiveProcessor::writeErrorMsg(const string& error, uint16_t errCode, bool logIt, bool critical)
|
||||
void BatchPrimitiveProcessor::writeErrorMsg(messageqcpp::SBS& bs, const string& error, uint16_t errCode,
|
||||
bool logIt, bool critical)
|
||||
{
|
||||
ISMPacketHeader ism;
|
||||
PrimitiveHeader ph;
|
||||
@ -1970,10 +1974,10 @@ void BatchPrimitiveProcessor::writeErrorMsg(const string& error, uint16_t errCod
|
||||
ph.UniqueID = uniqueID;
|
||||
ism.Status = errCode;
|
||||
|
||||
serialized.reset(new ByteStream());
|
||||
serialized->append((uint8_t*)&ism, sizeof(ism));
|
||||
serialized->append((uint8_t*)&ph, sizeof(ph));
|
||||
*serialized << error;
|
||||
bs.reset(new ByteStream());
|
||||
bs->append((uint8_t*)&ism, sizeof(ism));
|
||||
bs->append((uint8_t*)&ph, sizeof(ph));
|
||||
*bs << error;
|
||||
|
||||
if (logIt)
|
||||
{
|
||||
@ -1982,7 +1986,7 @@ void BatchPrimitiveProcessor::writeErrorMsg(const string& error, uint16_t errCod
|
||||
}
|
||||
}
|
||||
|
||||
void BatchPrimitiveProcessor::writeProjectionPreamble()
|
||||
void BatchPrimitiveProcessor::writeProjectionPreamble(SBS& bs)
|
||||
{
|
||||
ISMPacketHeader ism;
|
||||
PrimitiveHeader ph;
|
||||
@ -1997,36 +2001,36 @@ void BatchPrimitiveProcessor::writeProjectionPreamble()
|
||||
ph.StepID = stepID;
|
||||
ph.UniqueID = uniqueID;
|
||||
|
||||
serialized.reset(new ByteStream());
|
||||
serialized->append((uint8_t*)&ism, sizeof(ism));
|
||||
serialized->append((uint8_t*)&ph, sizeof(ph));
|
||||
bs.reset(new ByteStream());
|
||||
bs->append((uint8_t*)&ism, sizeof(ism));
|
||||
bs->append((uint8_t*)&ph, sizeof(ph));
|
||||
|
||||
/* add-ons */
|
||||
if (hasScan)
|
||||
{
|
||||
if (validCPData)
|
||||
{
|
||||
*serialized << (uint8_t)1;
|
||||
*serialized << lbidForCP;
|
||||
*serialized << ((uint8_t)cpDataFromDictScan);
|
||||
*bs << (uint8_t)1;
|
||||
*bs << lbidForCP;
|
||||
*bs << ((uint8_t)cpDataFromDictScan);
|
||||
if (UNLIKELY(hasWideColumnOut))
|
||||
{
|
||||
// PSA width
|
||||
*serialized << (uint8_t)wideColumnWidthOut;
|
||||
*serialized << min128Val;
|
||||
*serialized << max128Val;
|
||||
*bs << (uint8_t)wideColumnWidthOut;
|
||||
*bs << min128Val;
|
||||
*bs << max128Val;
|
||||
}
|
||||
else
|
||||
{
|
||||
*serialized << (uint8_t)utils::MAXLEGACYWIDTH; // width of min/max value
|
||||
*serialized << (uint64_t)minVal;
|
||||
*serialized << (uint64_t)maxVal;
|
||||
*bs << (uint8_t)utils::MAXLEGACYWIDTH; // width of min/max value
|
||||
*bs << (uint64_t)minVal;
|
||||
*bs << (uint64_t)maxVal;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
*serialized << (uint8_t)0;
|
||||
*serialized << lbidForCP;
|
||||
*bs << (uint8_t)0;
|
||||
*bs << lbidForCP;
|
||||
}
|
||||
}
|
||||
|
||||
@ -2035,34 +2039,34 @@ void BatchPrimitiveProcessor::writeProjectionPreamble()
|
||||
|
||||
if (ot != ROW_GROUP)
|
||||
{
|
||||
*serialized << ridCount;
|
||||
*bs << ridCount;
|
||||
|
||||
if (sendRidsAtDelivery)
|
||||
{
|
||||
*serialized << baseRid;
|
||||
serialized->append((uint8_t*)relRids, ridCount << 1);
|
||||
*bs << baseRid;
|
||||
bs->append((uint8_t*)relRids, ridCount << 1);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void BatchPrimitiveProcessor::serializeElementTypes()
|
||||
void BatchPrimitiveProcessor::serializeElementTypes(messageqcpp::SBS& bs)
|
||||
{
|
||||
*serialized << baseRid;
|
||||
*serialized << ridCount;
|
||||
serialized->append((uint8_t*)relRids, ridCount << 1);
|
||||
serialized->append((uint8_t*)values, ridCount << 3);
|
||||
*bs << baseRid;
|
||||
*bs << ridCount;
|
||||
bs->append((uint8_t*)relRids, ridCount << 1);
|
||||
bs->append((uint8_t*)values, ridCount << 3);
|
||||
}
|
||||
|
||||
void BatchPrimitiveProcessor::serializeStrings()
|
||||
void BatchPrimitiveProcessor::serializeStrings(messageqcpp::SBS& bs)
|
||||
{
|
||||
*serialized << ridCount;
|
||||
serialized->append((uint8_t*)absRids.get(), ridCount << 3);
|
||||
*bs << ridCount;
|
||||
bs->append((uint8_t*)absRids.get(), ridCount << 3);
|
||||
|
||||
for (uint32_t i = 0; i < ridCount; ++i)
|
||||
*serialized << strValues[i];
|
||||
*bs << strValues[i];
|
||||
}
|
||||
|
||||
void BatchPrimitiveProcessor::sendResponse()
|
||||
void BatchPrimitiveProcessor::sendResponse(messageqcpp::SBS& bs)
|
||||
{
|
||||
// Here is the fast path for local EM to PM interaction. PM puts into the
|
||||
// input EM DEC queue directly.
|
||||
@ -2073,12 +2077,12 @@ void BatchPrimitiveProcessor::sendResponse()
|
||||
// is limited.
|
||||
if (sendThread->flowControlEnabled())
|
||||
{
|
||||
sendThread->sendResult({serialized, sock, writelock, 0}, false);
|
||||
sendThread->sendResult({bs, sock, writelock, 0}, false);
|
||||
}
|
||||
else
|
||||
{
|
||||
sock->write(serialized);
|
||||
serialized.reset();
|
||||
sock->write(bs);
|
||||
bs.reset();
|
||||
}
|
||||
|
||||
return;
|
||||
@ -2088,20 +2092,20 @@ void BatchPrimitiveProcessor::sendResponse()
|
||||
{
|
||||
// newConnection should be set only for the first result of a batch job
|
||||
// it tells sendthread it should consider it for the connection array
|
||||
sendThread->sendResult(BPPSendThread::Msg_t(serialized, sock, writelock, sockIndex), newConnection);
|
||||
sendThread->sendResult(BPPSendThread::Msg_t(bs, sock, writelock, sockIndex), newConnection);
|
||||
newConnection = false;
|
||||
}
|
||||
else
|
||||
{
|
||||
boost::mutex::scoped_lock lk(*writelock);
|
||||
sock->write(*serialized);
|
||||
sock->write(*bs);
|
||||
}
|
||||
|
||||
serialized.reset();
|
||||
bs.reset();
|
||||
}
|
||||
|
||||
/* The output of a filter chain is either ELEMENT_TYPE or STRING_ELEMENT_TYPE */
|
||||
void BatchPrimitiveProcessor::makeResponse()
|
||||
void BatchPrimitiveProcessor::makeResponse(messageqcpp::SBS& bs)
|
||||
{
|
||||
ISMPacketHeader ism;
|
||||
PrimitiveHeader ph;
|
||||
@ -2116,39 +2120,39 @@ void BatchPrimitiveProcessor::makeResponse()
|
||||
ph.StepID = stepID;
|
||||
ph.UniqueID = uniqueID;
|
||||
|
||||
serialized.reset(new ByteStream());
|
||||
serialized->append((uint8_t*)&ism, sizeof(ism));
|
||||
serialized->append((uint8_t*)&ph, sizeof(ph));
|
||||
bs.reset(new ByteStream());
|
||||
bs->append((uint8_t*)&ism, sizeof(ism));
|
||||
bs->append((uint8_t*)&ph, sizeof(ph));
|
||||
|
||||
/* add-ons */
|
||||
if (hasScan)
|
||||
{
|
||||
if (validCPData)
|
||||
{
|
||||
*serialized << (uint8_t)1;
|
||||
*serialized << lbidForCP;
|
||||
*serialized << ((uint8_t)cpDataFromDictScan);
|
||||
*bs << (uint8_t)1;
|
||||
*bs << lbidForCP;
|
||||
*bs << ((uint8_t)cpDataFromDictScan);
|
||||
|
||||
if (UNLIKELY(hasWideColumnOut))
|
||||
{
|
||||
// PSA width
|
||||
// Remove the assert for >16 bytes DTs.
|
||||
assert(wideColumnWidthOut == datatypes::MAXDECIMALWIDTH);
|
||||
*serialized << (uint8_t)wideColumnWidthOut;
|
||||
*serialized << min128Val;
|
||||
*serialized << max128Val;
|
||||
*bs << (uint8_t)wideColumnWidthOut;
|
||||
*bs << min128Val;
|
||||
*bs << max128Val;
|
||||
}
|
||||
else
|
||||
{
|
||||
*serialized << (uint8_t)utils::MAXLEGACYWIDTH; // width of min/max value
|
||||
*serialized << (uint64_t)minVal;
|
||||
*serialized << (uint64_t)maxVal;
|
||||
*bs << (uint8_t)utils::MAXLEGACYWIDTH; // width of min/max value
|
||||
*bs << (uint64_t)minVal;
|
||||
*bs << (uint64_t)maxVal;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
*serialized << (uint8_t)0;
|
||||
*serialized << lbidForCP;
|
||||
*bs << (uint8_t)0;
|
||||
*bs << lbidForCP;
|
||||
}
|
||||
}
|
||||
|
||||
@ -2156,9 +2160,9 @@ void BatchPrimitiveProcessor::makeResponse()
|
||||
/* Take the rid and value arrays, munge into OutputType ot */
|
||||
switch (ot)
|
||||
{
|
||||
case BPS_ELEMENT_TYPE: serializeElementTypes(); break;
|
||||
case BPS_ELEMENT_TYPE: serializeElementTypes(bs); break;
|
||||
|
||||
case STRING_ELEMENT_TYPE: serializeStrings(); break;
|
||||
case STRING_ELEMENT_TYPE: serializeStrings(bs); break;
|
||||
|
||||
default:
|
||||
{
|
||||
@ -2166,15 +2170,13 @@ void BatchPrimitiveProcessor::makeResponse()
|
||||
oss << "BPP: makeResponse(): Bad output type: " << ot;
|
||||
throw logic_error(oss.str());
|
||||
}
|
||||
|
||||
// throw logic_error("BPP: makeResponse(): Bad output type");
|
||||
}
|
||||
|
||||
*serialized << cachedIO;
|
||||
*bs << cachedIO;
|
||||
cachedIO = 0;
|
||||
*serialized << physIO;
|
||||
*bs << physIO;
|
||||
physIO = 0;
|
||||
*serialized << touchedBlocks;
|
||||
*bs << touchedBlocks;
|
||||
touchedBlocks = 0;
|
||||
|
||||
// cout << "sent physIO=" << physIO << " cachedIO=" << cachedIO <<
|
||||
@ -2230,20 +2232,24 @@ int BatchPrimitiveProcessor::operator()()
|
||||
|
||||
validCPData = false;
|
||||
cpDataFromDictScan = false;
|
||||
|
||||
auto alloc = exemgr::globServiceExeMgr->getRm().getAllocator<messageqcpp::BSBufType>();
|
||||
messageqcpp::SBS bs(new ByteStream(&alloc));
|
||||
|
||||
#ifdef PRIMPROC_STOPWATCH
|
||||
stopwatch->start("BPP() execute");
|
||||
execute(stopwatch);
|
||||
stopwatch->stop("BPP() execute");
|
||||
#else
|
||||
execute();
|
||||
execute(bs);
|
||||
#endif
|
||||
|
||||
if (projectCount == 0 && ot != ROW_GROUP)
|
||||
makeResponse();
|
||||
makeResponse(bs);
|
||||
|
||||
try
|
||||
{
|
||||
sendResponse();
|
||||
sendResponse(bs);
|
||||
}
|
||||
catch (std::exception& e)
|
||||
{
|
||||
@ -2717,7 +2723,7 @@ inline void BatchPrimitiveProcessor::getJoinResults(const Row& r, uint32_t jInde
|
||||
{
|
||||
bool hasNullValue = false;
|
||||
|
||||
for (unsigned int column: tlLargeSideKeyColumns[jIndex])
|
||||
for (unsigned int column : tlLargeSideKeyColumns[jIndex])
|
||||
{
|
||||
if (r.isNullValue(column))
|
||||
{
|
||||
|
Reference in New Issue
Block a user