You've already forked mariadb-columnstore-engine
mirror of
https://github.com/mariadb-corporation/mariadb-columnstore-engine.git
synced 2025-08-05 16:15:50 +03:00
Mcol 4841 dev6 Handle large joins without OOM (#2155)
* MCOL-4846 dev-6 Handle large join results Use a loop to shrink the number of results reported per message to something manageable. * MCOL-4841 small changes requested by review * Add EXTRA threads to prioritythreadpool prioritythreadpool is configured at startup with a fixed number of threads available. This is to prevent thread thrashing. Since most of the time, BPP job steps are short lived, and a rescheduling mechanism exist if no threads are available, this works to keep cpu wastage to a minimum. However, if a query or queries consume all the threads in prioritythreadpool and then block (due to the consumer not consuming fast enough) we can run out of threads and no work will be done until some threads unblock. A new mechanism allows for EXTRA threads to be generated for the duration of the blocking action. These threads can act on new queries. When all blocking is completed, these threads will be released when idle. * MCOL-4841 dev6 Reconcile with changes in develop-6 * MCOL-4841 Some format corrections * MCOL-4841 dev clean up some things based on review * MCOL-4841 dev 6 ExeMgr Crashes after large join This commit fixes up memory accounting issues in ExeMgr * MCOL-4841 remove LDI change Opened MCOL-4968 to address the issue * MCOL-4841 Add fMaxBPPSendQueue to ResourceManager This causes the setting to be loaded at run time (requires restart to accept a change) BPPSendthread gets this in it's ctor Also rolled back changes to TupleHashJoinStep::smallRunnerFcn() that used a local variable to count locally allocated memory, then added it into the global counter at function's end. Not counting the memory globally caused conversion to UM only join way later than it should. This resulted in MCOL-4971. * MCOL-4841 make blockedThreads and extraThreads atomic Also restore previous scope of locks in bppsendthread. There is some small chance the new scope could be incorrect, and the performance boost is negligible. Better safe than sorry.
This commit is contained in:
@@ -1151,17 +1151,24 @@ void BatchPrimitiveProcessor::initProcessor()
|
||||
}
|
||||
|
||||
/* This version does a join on projected rows */
|
||||
void BatchPrimitiveProcessor::executeTupleJoin()
|
||||
// In order to prevent super size result sets in the case of near cartesian joins on three or more joins,
|
||||
// the startRid start at 0) is used to begin the rid loop and if we cut off processing early because of
|
||||
// the size of the result set, we return the next rid to start with. If we finish ridCount rids, return 0-
|
||||
uint32_t BatchPrimitiveProcessor::executeTupleJoin(uint32_t startRid)
|
||||
{
|
||||
uint32_t newRowCount = 0, i, j;
|
||||
vector<uint32_t> matches;
|
||||
uint64_t largeKey;
|
||||
|
||||
uint64_t resultCount = 0;
|
||||
uint32_t newStartRid = startRid;
|
||||
outputRG.getRow(0, &oldRow);
|
||||
outputRG.getRow(0, &newRow);
|
||||
|
||||
//cout << "before join, RG has " << outputRG.getRowCount() << " BPP ridcount= " << ridCount << endl;
|
||||
for (i = 0; i < ridCount && !sendThread->aborted(); i++, oldRow.nextRow())
|
||||
// ridCount gets modified based on the number of Rids actually processed during this call.
|
||||
// origRidCount is the number of rids for this thread after filter, which are the total
|
||||
// number of rids to be processed from all calls to this function during this thread.
|
||||
for (i = startRid; i < origRidCount && !sendThread->aborted(); i++, oldRow.nextRow())
|
||||
{
|
||||
/* Decide whether this large-side row belongs in the output. The breaks
|
||||
* in the loop mean that it doesn't.
|
||||
@@ -1270,10 +1277,9 @@ void BatchPrimitiveProcessor::executeTupleJoin()
|
||||
|
||||
if (j == joinerCount)
|
||||
{
|
||||
uint32_t matchCount;
|
||||
for (j = 0; j < joinerCount; j++)
|
||||
{
|
||||
uint32_t matchCount;
|
||||
|
||||
/* The result is already known if...
|
||||
* -- anti-join with no fcnexp
|
||||
* -- semi-join with no fcnexp and not scalar
|
||||
@@ -1361,6 +1367,8 @@ void BatchPrimitiveProcessor::executeTupleJoin()
|
||||
tSmallSideMatches[j][newRowCount].push_back(-1);
|
||||
matchCount = 1;
|
||||
}
|
||||
|
||||
resultCount += matchCount;
|
||||
}
|
||||
|
||||
/* Finally, copy the row into the output */
|
||||
@@ -1384,8 +1392,18 @@ void BatchPrimitiveProcessor::executeTupleJoin()
|
||||
//else
|
||||
// cout << "j != joinerCount\n";
|
||||
}
|
||||
// If we've accumulated more than maxResultCount -- 1048576 (2^20)_ of resultCounts, cut off processing. The caller will
|
||||
// restart to continue where we left off.
|
||||
if (resultCount >= maxResultCount)
|
||||
{
|
||||
newStartRid += newRowCount;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (resultCount < maxResultCount)
|
||||
newStartRid = 0;
|
||||
|
||||
ridCount = newRowCount;
|
||||
outputRG.setRowCount(ridCount);
|
||||
|
||||
@@ -1404,6 +1422,7 @@ void BatchPrimitiveProcessor::executeTupleJoin()
|
||||
}
|
||||
}
|
||||
*/
|
||||
return newStartRid;
|
||||
}
|
||||
|
||||
#ifdef PRIMPROC_STOPWATCH
|
||||
@@ -1412,6 +1431,9 @@ void BatchPrimitiveProcessor::execute(StopWatch* stopwatch)
|
||||
void BatchPrimitiveProcessor::execute()
|
||||
#endif
|
||||
{
|
||||
uint8_t sendCount = 0;
|
||||
// bool smoreRGs = false;
|
||||
// uint32_t sStartRid = 0;
|
||||
uint32_t i, j;
|
||||
|
||||
try
|
||||
@@ -1616,9 +1638,9 @@ void BatchPrimitiveProcessor::execute()
|
||||
}
|
||||
|
||||
/* 7/7/09 PL: I Changed the projection alg to reduce block touches when there's
|
||||
a join. The key columns get projected first, the join is executed to further
|
||||
a join. The key columns get projected first, the join is executed to further
|
||||
reduce the ridlist, then the rest of the columns get projected */
|
||||
|
||||
|
||||
if (!doJoin)
|
||||
{
|
||||
for (j = 0; j < projectCount; ++j)
|
||||
@@ -1638,15 +1660,92 @@ void BatchPrimitiveProcessor::execute()
|
||||
// else
|
||||
// cout << " no target found for OID " << projectSteps[j]->getOID() << endl;
|
||||
}
|
||||
if (fe2)
|
||||
{
|
||||
/* functionize this -> processFE2() */
|
||||
fe2Output.resetRowGroup ( baseRid );
|
||||
fe2Output.getRow ( 0, &fe2Out );
|
||||
fe2Input->getRow ( 0, &fe2In );
|
||||
|
||||
//cerr << "input row: " << fe2In.toString() << endl;
|
||||
for ( j = 0; j < outputRG.getRowCount(); j++, fe2In.nextRow() )
|
||||
{
|
||||
if ( fe2->evaluate ( &fe2In ) )
|
||||
{
|
||||
applyMapping ( fe2Mapping, fe2In, &fe2Out );
|
||||
//cerr << " passed. output row: " << fe2Out.toString() << endl;
|
||||
fe2Out.setRid ( fe2In.getRelRid() );
|
||||
fe2Output.incRowCount();
|
||||
fe2Out.nextRow();
|
||||
}
|
||||
}
|
||||
|
||||
if ( !fAggregator )
|
||||
{
|
||||
*serialized << ( uint8_t ) 1; // the "count this msg" var
|
||||
fe2Output.setDBRoot ( dbRoot );
|
||||
fe2Output.serializeRGData ( *serialized );
|
||||
//*serialized << fe2Output.getDataSize();
|
||||
//serialized->append(fe2Output.getData(), fe2Output.getDataSize());
|
||||
}
|
||||
}
|
||||
|
||||
if (fAggregator)
|
||||
{
|
||||
*serialized << ( uint8_t ) 1; // the "count this msg" var
|
||||
|
||||
RowGroup& toAggregate = ( fe2 ? fe2Output : outputRG );
|
||||
//toAggregate.convertToInlineDataInPlace();
|
||||
|
||||
if ( fe2 )
|
||||
fe2Output.setDBRoot ( dbRoot );
|
||||
else
|
||||
outputRG.setDBRoot ( dbRoot );
|
||||
|
||||
fAggregator->addRowGroup ( &toAggregate );
|
||||
|
||||
if ( ( currentBlockOffset + 1 ) == count ) // @bug4507, 8k
|
||||
{
|
||||
fAggregator->loadResult ( *serialized ); // @bug4507, 8k
|
||||
} // @bug4507, 8k
|
||||
else if ( utils::MonitorProcMem::isMemAvailable() ) // @bug4507, 8k
|
||||
{
|
||||
fAggregator->loadEmptySet ( *serialized ); // @bug4507, 8k
|
||||
} // @bug4507, 8k
|
||||
else // @bug4507, 8k
|
||||
{
|
||||
fAggregator->loadResult ( *serialized ); // @bug4507, 8k
|
||||
fAggregator->aggReset(); // @bug4507, 8k
|
||||
} // @bug4507, 8k
|
||||
}
|
||||
|
||||
if (!fAggregator && !fe2)
|
||||
{
|
||||
*serialized << ( uint8_t ) 1; // the "count this msg" var
|
||||
outputRG.setDBRoot ( dbRoot );
|
||||
//cerr << "serializing " << outputRG.toString() << endl;
|
||||
outputRG.serializeRGData ( *serialized );
|
||||
|
||||
//*serialized << outputRG.getDataSize();
|
||||
//serialized->append(outputRG.getData(), outputRG.getDataSize());
|
||||
}
|
||||
|
||||
#ifdef PRIMPROC_STOPWATCH
|
||||
stopwatch->stop ( "- if(ot != ROW_GROUP) else" );
|
||||
#endif
|
||||
}
|
||||
else
|
||||
else // Is doJoin
|
||||
{
|
||||
uint32_t startRid = 0;
|
||||
ByteStream preamble = *serialized;
|
||||
origRidCount = ridCount; // ridCount can get modified by executeTupleJoin(). We need to keep track of the original val.
|
||||
/* project the key columns. If there's the filter IN the join, project everything.
|
||||
Also need to project 'long' strings b/c executeTupleJoin may copy entire rows
|
||||
using copyRow(), which will try to interpret the uninit'd string ptr.
|
||||
Valgrind will legitimately complain about copying uninit'd values for the
|
||||
other types but that is technically safe. */
|
||||
for (j = 0; j < projectCount; j++)
|
||||
{
|
||||
if (keyColumnProj[j] || (projectionMap[j] != -1 && (hasJoinFEFilters ||
|
||||
oldRow.isLongString(projectionMap[j]))))
|
||||
{
|
||||
@@ -1656,218 +1755,174 @@ void BatchPrimitiveProcessor::execute()
|
||||
stopwatch->stop("-- projectIntoRowGroup");
|
||||
#else
|
||||
projectSteps[j]->projectIntoRowGroup(outputRG, projectionMap[j]);
|
||||
#endif
|
||||
}
|
||||
|
||||
|
||||
#ifdef PRIMPROC_STOPWATCH
|
||||
stopwatch->start("-- executeTupleJoin()");
|
||||
executeTupleJoin();
|
||||
stopwatch->stop("-- executeTupleJoin()");
|
||||
#else
|
||||
executeTupleJoin();
|
||||
#endif
|
||||
|
||||
/* project the non-key columns */
|
||||
for (j = 0; j < projectCount; ++j)
|
||||
{
|
||||
if (projectionMap[j] != -1 && !keyColumnProj[j] && !hasJoinFEFilters &&
|
||||
!oldRow.isLongString(projectionMap[j]))
|
||||
{
|
||||
#ifdef PRIMPROC_STOPWATCH
|
||||
stopwatch->start("-- projectIntoRowGroup");
|
||||
projectSteps[j]->projectIntoRowGroup(outputRG, projectionMap[j]);
|
||||
stopwatch->stop("-- projectIntoRowGroup");
|
||||
#else
|
||||
projectSteps[j]->projectIntoRowGroup(outputRG, projectionMap[j]);
|
||||
#endif
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/* The RowGroup is fully joined at this point.
|
||||
Add additional RowGroup processing here.
|
||||
TODO: Try to clean up all of the switching */
|
||||
|
||||
if (doJoin && (fe2 || fAggregator))
|
||||
{
|
||||
bool moreRGs = true;
|
||||
ByteStream preamble = *serialized;
|
||||
initGJRG();
|
||||
|
||||
while (moreRGs && !sendThread->aborted())
|
||||
do //while (startRid > 0)
|
||||
{
|
||||
/*
|
||||
generate 1 rowgroup (8192 rows max) of joined rows
|
||||
if there's an FE2, run it
|
||||
-pack results into a new rowgroup
|
||||
-if there are < 8192 rows in the new RG, continue
|
||||
if there's an agg, run it
|
||||
send the result
|
||||
*/
|
||||
resetGJRG();
|
||||
moreRGs = generateJoinedRowGroup(baseJRow);
|
||||
*serialized << (uint8_t) !moreRGs;
|
||||
|
||||
if (fe2)
|
||||
#ifdef PRIMPROC_STOPWATCH
|
||||
stopwatch->start("-- executeTupleJoin()");
|
||||
startRid = executeTupleJoin(startRid);
|
||||
stopwatch->stop("-- executeTupleJoin()");
|
||||
#else
|
||||
startRid = executeTupleJoin(startRid);
|
||||
// sStartRid = startRid;
|
||||
#endif
|
||||
/* project the non-key columns */
|
||||
for (j = 0; j < projectCount; ++j)
|
||||
{
|
||||
/* functionize this -> processFE2()*/
|
||||
fe2Output.resetRowGroup(baseRid);
|
||||
fe2Output.setDBRoot(dbRoot);
|
||||
fe2Output.getRow(0, &fe2Out);
|
||||
fe2Input->getRow(0, &fe2In);
|
||||
|
||||
for (j = 0; j < joinedRG.getRowCount(); j++, fe2In.nextRow())
|
||||
if (fe2->evaluate(&fe2In))
|
||||
{
|
||||
applyMapping(fe2Mapping, fe2In, &fe2Out);
|
||||
fe2Out.setRid(fe2In.getRelRid());
|
||||
fe2Output.incRowCount();
|
||||
fe2Out.nextRow();
|
||||
}
|
||||
if (projectionMap[j] != -1 && !keyColumnProj[j] && !hasJoinFEFilters &&
|
||||
!oldRow.isLongString(projectionMap[j]))
|
||||
{
|
||||
#ifdef PRIMPROC_STOPWATCH
|
||||
stopwatch->start("-- projectIntoRowGroup");
|
||||
projectSteps[j]->projectIntoRowGroup(outputRG, projectionMap[j]);
|
||||
stopwatch->stop("-- projectIntoRowGroup");
|
||||
#else
|
||||
projectSteps[j]->projectIntoRowGroup(outputRG, projectionMap[j]);
|
||||
#endif
|
||||
}
|
||||
}
|
||||
/* The RowGroup is fully joined at this point.
|
||||
* Add additional RowGroup processing here.
|
||||
* TODO: Try to clean up all of the switching */
|
||||
|
||||
RowGroup& nextRG = (fe2 ? fe2Output : joinedRG);
|
||||
nextRG.setDBRoot(dbRoot);
|
||||
|
||||
if (fAggregator)
|
||||
if (fe2 || fAggregator)
|
||||
{
|
||||
fAggregator->addRowGroup(&nextRG);
|
||||
bool moreRGs = true;
|
||||
initGJRG();
|
||||
|
||||
if ((currentBlockOffset + 1) == count && moreRGs == false) // @bug4507, 8k
|
||||
while (moreRGs && !sendThread->aborted())
|
||||
{
|
||||
fAggregator->loadResult(*serialized); // @bug4507, 8k
|
||||
} // @bug4507, 8k
|
||||
else if (utils::MonitorProcMem::isMemAvailable()) // @bug4507, 8k
|
||||
/*
|
||||
* generate 1 rowgroup (8192 rows max) of joined rows
|
||||
* if there's an FE2, run it
|
||||
* -pack results into a new rowgroup
|
||||
* -if there are < 8192 rows in the new RG, continue
|
||||
* if there's an agg, run it
|
||||
* send the result
|
||||
*/
|
||||
resetGJRG();
|
||||
moreRGs = generateJoinedRowGroup(baseJRow);
|
||||
// smoreRGs = moreRGs;
|
||||
sendCount = (uint8_t)(!moreRGs && !startRid);
|
||||
// *serialized << (uint8_t)(!moreRGs && !startRid); // the "count this msg" var
|
||||
*serialized << sendCount;
|
||||
if (fe2)
|
||||
{
|
||||
/* functionize this -> processFE2()*/
|
||||
fe2Output.resetRowGroup(baseRid);
|
||||
fe2Output.setDBRoot(dbRoot);
|
||||
fe2Output.getRow(0, &fe2Out);
|
||||
fe2Input->getRow(0, &fe2In);
|
||||
|
||||
for (j = 0; j < joinedRG.getRowCount(); j++, fe2In.nextRow())
|
||||
if (fe2->evaluate(&fe2In))
|
||||
{
|
||||
applyMapping(fe2Mapping, fe2In, &fe2Out);
|
||||
fe2Out.setRid(fe2In.getRelRid());
|
||||
fe2Output.incRowCount();
|
||||
fe2Out.nextRow();
|
||||
}
|
||||
}
|
||||
|
||||
RowGroup& nextRG = (fe2 ? fe2Output : joinedRG);
|
||||
nextRG.setDBRoot(dbRoot);
|
||||
|
||||
if (fAggregator)
|
||||
{
|
||||
fAggregator->addRowGroup(&nextRG);
|
||||
|
||||
if ((currentBlockOffset + 1) == count && moreRGs == false && startRid == 0) // @bug4507, 8k
|
||||
{
|
||||
fAggregator->loadResult(*serialized); // @bug4507, 8k
|
||||
} // @bug4507, 8k
|
||||
else if (utils::MonitorProcMem::isMemAvailable()) // @bug4507, 8k
|
||||
{
|
||||
fAggregator->loadEmptySet(*serialized); // @bug4507, 8k
|
||||
} // @bug4507, 8k
|
||||
else // @bug4507, 8k
|
||||
{
|
||||
fAggregator->loadResult(*serialized); // @bug4507, 8k
|
||||
fAggregator->aggReset(); // @bug4507, 8k
|
||||
} // @bug4507, 8k
|
||||
}
|
||||
else
|
||||
{
|
||||
//cerr <<" * serialzing " << nextRG.toString() << endl;
|
||||
nextRG.serializeRGData(*serialized);
|
||||
}
|
||||
|
||||
/* send the msg & reinit the BS */
|
||||
if (moreRGs)
|
||||
{
|
||||
sendResponse();
|
||||
serialized.reset(new ByteStream());
|
||||
*serialized = preamble;
|
||||
}
|
||||
}
|
||||
|
||||
if ( hasSmallOuterJoin )
|
||||
{
|
||||
fAggregator->loadEmptySet(*serialized); // @bug4507, 8k
|
||||
} // @bug4507, 8k
|
||||
else // @bug4507, 8k
|
||||
// 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
|
||||
|
||||
*serialized << ridCount;
|
||||
|
||||
for ( i = 0; i < joinerCount; i++ )
|
||||
{
|
||||
for ( j = 0; j < ridCount; ++j )
|
||||
{
|
||||
serializeInlineVector<uint32_t> ( *serialized,
|
||||
tSmallSideMatches[i][j] );
|
||||
tSmallSideMatches[i][j].clear();
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
fAggregator->loadResult(*serialized); // @bug4507, 8k
|
||||
fAggregator->aggReset(); // @bug4507, 8k
|
||||
} // @bug4507, 8k
|
||||
// We hae no more use for this allocation
|
||||
for ( i = 0; i < joinerCount; i++ )
|
||||
for ( j = 0; j < ridCount; ++j )
|
||||
tSmallSideMatches[i][j].clear();
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
//cerr <<" * serialzing " << nextRG.toString() << endl;
|
||||
nextRG.serializeRGData(*serialized);
|
||||
}
|
||||
*serialized << ( uint8_t )(startRid > 0 ? 0 : 1); // the "count this msg" var
|
||||
outputRG.setDBRoot ( dbRoot );
|
||||
//cerr << "serializing " << outputRG.toString() << endl;
|
||||
outputRG.serializeRGData ( *serialized );
|
||||
|
||||
/* send the msg & reinit the BS */
|
||||
if (moreRGs)
|
||||
{
|
||||
sendResponse();
|
||||
serialized.reset(new ByteStream());
|
||||
*serialized = preamble;
|
||||
}
|
||||
}
|
||||
|
||||
if (hasSmallOuterJoin)
|
||||
{
|
||||
*serialized << ridCount;
|
||||
|
||||
for (i = 0; i < joinerCount; i++)
|
||||
for (j = 0; j < ridCount; ++j)
|
||||
serializeInlineVector<uint32_t>(*serialized,
|
||||
tSmallSideMatches[i][j]);
|
||||
}
|
||||
}
|
||||
|
||||
if (!doJoin && fe2)
|
||||
{
|
||||
/* functionize this -> processFE2() */
|
||||
fe2Output.resetRowGroup(baseRid);
|
||||
fe2Output.getRow(0, &fe2Out);
|
||||
fe2Input->getRow(0, &fe2In);
|
||||
|
||||
//cerr << "input row: " << fe2In.toString() << endl;
|
||||
for (j = 0; j < outputRG.getRowCount(); j++, fe2In.nextRow())
|
||||
{
|
||||
if (fe2->evaluate(&fe2In))
|
||||
{
|
||||
applyMapping(fe2Mapping, fe2In, &fe2Out);
|
||||
//cerr << " passed. output row: " << fe2Out.toString() << endl;
|
||||
fe2Out.setRid (fe2In.getRelRid());
|
||||
fe2Output.incRowCount();
|
||||
fe2Out.nextRow();
|
||||
}
|
||||
}
|
||||
|
||||
if (!fAggregator)
|
||||
{
|
||||
*serialized << (uint8_t) 1; // the "count this msg" var
|
||||
fe2Output.setDBRoot(dbRoot);
|
||||
fe2Output.serializeRGData(*serialized);
|
||||
//*serialized << fe2Output.getDataSize();
|
||||
//serialized->append(fe2Output.getData(), fe2Output.getDataSize());
|
||||
}
|
||||
}
|
||||
|
||||
if (!doJoin && fAggregator)
|
||||
{
|
||||
*serialized << (uint8_t) 1; // the "count this msg" var
|
||||
|
||||
RowGroup& toAggregate = (fe2 ? fe2Output : outputRG);
|
||||
//toAggregate.convertToInlineDataInPlace();
|
||||
|
||||
if (fe2)
|
||||
fe2Output.setDBRoot(dbRoot);
|
||||
else
|
||||
outputRG.setDBRoot(dbRoot);
|
||||
|
||||
fAggregator->addRowGroup(&toAggregate);
|
||||
|
||||
if ((currentBlockOffset + 1) == count) // @bug4507, 8k
|
||||
{
|
||||
fAggregator->loadResult(*serialized); // @bug4507, 8k
|
||||
} // @bug4507, 8k
|
||||
else if (utils::MonitorProcMem::isMemAvailable()) // @bug4507, 8k
|
||||
{
|
||||
fAggregator->loadEmptySet(*serialized); // @bug4507, 8k
|
||||
} // @bug4507, 8k
|
||||
else // @bug4507, 8k
|
||||
{
|
||||
fAggregator->loadResult(*serialized); // @bug4507, 8k
|
||||
fAggregator->aggReset(); // @bug4507, 8k
|
||||
} // @bug4507, 8k
|
||||
}
|
||||
|
||||
if (!fAggregator && !fe2)
|
||||
{
|
||||
*serialized << (uint8_t) 1; // the "count this msg" var
|
||||
outputRG.setDBRoot(dbRoot);
|
||||
//cerr << "serializing " << outputRG.toString() << endl;
|
||||
outputRG.serializeRGData(*serialized);
|
||||
|
||||
//*serialized << outputRG.getDataSize();
|
||||
//serialized->append(outputRG.getData(), outputRG.getDataSize());
|
||||
if (doJoin)
|
||||
{
|
||||
for (i = 0; i < joinerCount; i++)
|
||||
{
|
||||
for (j = 0; j < ridCount; ++j)
|
||||
//*serialized << outputRG.getDataSize();
|
||||
//serialized->append(outputRG.getData(), outputRG.getDataSize());
|
||||
for ( i = 0; i < joinerCount; i++ )
|
||||
{
|
||||
serializeInlineVector<uint32_t>(*serialized,
|
||||
tSmallSideMatches[i][j]);
|
||||
for ( j = 0; j < ridCount; ++j )
|
||||
{
|
||||
serializeInlineVector<uint32_t> ( *serialized,
|
||||
tSmallSideMatches[i][j] );
|
||||
tSmallSideMatches[i][j].clear();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (startRid > 0)
|
||||
{
|
||||
sendResponse();
|
||||
serialized.reset ( new ByteStream() );
|
||||
*serialized = preamble;
|
||||
}
|
||||
}while (startRid > 0);
|
||||
}
|
||||
|
||||
// clear small side match vector
|
||||
if (doJoin)
|
||||
{
|
||||
for (i = 0; i < joinerCount; i++)
|
||||
for (j = 0; j < ridCount; ++j)
|
||||
tSmallSideMatches[i][j].clear();
|
||||
}
|
||||
|
||||
#ifdef PRIMPROC_STOPWATCH
|
||||
stopwatch->stop("- if(ot != ROW_GROUP) else");
|
||||
#endif
|
||||
}
|
||||
|
||||
ridCount = origRidCount; // May not be needed, but just to be safe.
|
||||
// std::cout << "end of send. startRid=" << sStartRid << " moreRG=" << smoreRGs << " sendCount=" << sendCount << std::endl;
|
||||
if (projectCount > 0 || ot == ROW_GROUP)
|
||||
{
|
||||
*serialized << cachedIO;
|
||||
@@ -2215,8 +2270,9 @@ int BatchPrimitiveProcessor::operator()()
|
||||
if (sendThread->aborted())
|
||||
break;
|
||||
|
||||
if (!sendThread->okToProceed())
|
||||
if (sendThread->sizeTooBig())
|
||||
{
|
||||
// The send buffer is full of messages yet to be sent, so this thread would block anyway.
|
||||
freeLargeBuffers();
|
||||
return -1; // the reschedule error code
|
||||
}
|
||||
|
Reference in New Issue
Block a user