You've already forked mariadb-columnstore-engine
mirror of
https://github.com/mariadb-corporation/mariadb-columnstore-engine.git
synced 2025-07-29 08:21:15 +03:00
MCOL-513 Threadpool to unlimited threads when queuesize = 0. Idle down after 10 minutes
This commit is contained in:
@ -74,7 +74,7 @@ JobList::JobList(bool isEM) :
|
||||
JobList::~JobList()
|
||||
{
|
||||
vector<boost::thread *> joiners;
|
||||
boost::thread *tmp;
|
||||
// boost::thread *tmp;
|
||||
try
|
||||
{
|
||||
if (fIsRunning)
|
||||
|
@ -56,7 +56,7 @@ namespace joblist
|
||||
{
|
||||
boost::mutex JobStep::fLogMutex; //=PTHREAD_MUTEX_INITIALIZER;
|
||||
|
||||
ThreadPool JobStep::jobstepThreadPool(100,200);
|
||||
ThreadPool JobStep::jobstepThreadPool(defaultJLThreadPoolSize, 0);
|
||||
|
||||
ostream& operator<<(ostream& os, const JobStep* rhs)
|
||||
{
|
||||
|
@ -60,7 +60,8 @@ namespace joblist
|
||||
const uint64_t defaultHUATotalMem = 8 * 1024 * 1024 * 1024ULL;
|
||||
|
||||
const uint32_t defaultTupleDLMaxSize = 64 * 1024;
|
||||
const uint32_t defaultTupleMaxBuckets = 256;
|
||||
|
||||
const uint32_t defaultJLThreadPoolSize = 100;
|
||||
|
||||
//pcolscan.cpp
|
||||
const uint32_t defaultScanLbidReqLimit = 10000;
|
||||
@ -168,6 +169,10 @@ namespace joblist
|
||||
uint32_t getJlScanLbidReqLimit() const { return getUintVal(fJobListStr, "ScanLbidReqLimit",defaultScanLbidReqLimit); }
|
||||
uint32_t getJlScanLbidReqThreshold() const { return getUintVal(fJobListStr,"ScanLbidReqThreshold", defaultScanLbidReqThreshold); }
|
||||
|
||||
// @MCOL-513 - Added threadpool to JobSteps
|
||||
uint32_t getJLThreadPoolSize() const { return getUintVal(fJobListStr, "ThreadPoolSize", defaultJLThreadPoolSize); }
|
||||
std::string getJlThreadPoolDebug() const { return getStringVal(fJobListStr, "ThreadPoolDebug", "N"); }
|
||||
|
||||
// @bug 1264 - Added LogicalBlocksPerScan configurable which determines the number of blocks contained in each BPS scan request.
|
||||
uint32_t getJlLogicalBlocksPerScan() const { return getUintVal(fJobListStr,"LogicalBlocksPerScan", defaultLogicalBlocksPerScan); }
|
||||
uint32_t getJlProjectBlockReqLimit() const { return getUintVal(fJobListStr, "ProjectBlockReqLimit", defaultProjectBlockReqLimit ); }
|
||||
|
@ -1393,6 +1393,20 @@ int main(int argc, char* argv[])
|
||||
}
|
||||
}
|
||||
|
||||
// It's possible that PM modules use this threadpool. Only ExeMgr creates
|
||||
// massive amounts of threads and needs to be settable. It's also possible that
|
||||
// other process on this UM module use this threadpool. In this case, they share.
|
||||
// We can't call rm functions during the static creation because rm has a isExeMgr
|
||||
// flag that is set upon first creation. For the pool, who has no idea if it is ExeMgr,
|
||||
// to create the singleton rm would be wrong, no matter which way we set the flag.
|
||||
JobStep::jobstepThreadPool.setMaxThreads(rm->getJLThreadPoolSize());
|
||||
JobStep::jobstepThreadPool.setName("ExeMgr");
|
||||
if (rm->getJlThreadPoolDebug() == "Y" || rm->getJlThreadPoolDebug() == "y")
|
||||
{
|
||||
JobStep::jobstepThreadPool.setDebug(true);
|
||||
JobStep::jobstepThreadPool.invoke(ThreadPoolMonitor(&JobStep::jobstepThreadPool));
|
||||
}
|
||||
|
||||
int serverThreads = rm->getEmServerThreads();
|
||||
int serverQueueSize = rm->getEmServerQueueSize();
|
||||
int maxPct = rm->getEmMaxPct();
|
||||
@ -1433,13 +1447,6 @@ int main(int argc, char* argv[])
|
||||
}
|
||||
}
|
||||
|
||||
// if (!JobStep::jobstepThreadPool.debug())
|
||||
// {
|
||||
// JobStep::jobstepThreadPool.setName("ExeMgr");
|
||||
// JobStep::jobstepThreadPool.setDebug(true);
|
||||
// JobStep::jobstepThreadPool.invoke(ThreadPoolMonitor(&JobStep::jobstepThreadPool));
|
||||
// }
|
||||
|
||||
threadpool::ThreadPool exeMgrThreadPool(serverThreads, serverQueueSize);
|
||||
for (;;)
|
||||
{
|
||||
|
@ -503,6 +503,7 @@
|
||||
as many threads are available across all PMs. -->
|
||||
<!-- <ProcessorThreadsPerScan>16</ProcessorThreadsPerScan> -->
|
||||
<MaxOutstandingRequests>20</MaxOutstandingRequests>
|
||||
<ThreadPoolSize>100</ThreadPoolSize>
|
||||
</JobList>
|
||||
<TupleWSDL>
|
||||
<MaxSize>1M</MaxSize> <!-- Max size in bytes per bucket -->
|
||||
@ -515,9 +516,9 @@
|
||||
<!-- <RowAggrRowGroupsPerThread>20</RowAggrRowGroupsPerThread> --> <!-- Default value is 20 -->
|
||||
</RowAggregation>
|
||||
<CrossEngineSupport>
|
||||
<Host>unassigned</Host>
|
||||
<Host>127.0.0.1</Host>
|
||||
<Port>3306</Port>
|
||||
<User>unassigned</User>
|
||||
<User>root</User>
|
||||
<Password></Password>
|
||||
</CrossEngineSupport>
|
||||
<QueryStats>
|
||||
|
@ -497,6 +497,7 @@
|
||||
as many threads are available across all PMs. -->
|
||||
<!-- <ProcessorThreadsPerScan>16</ProcessorThreadsPerScan> -->
|
||||
<MaxOutstandingRequests>20</MaxOutstandingRequests>
|
||||
<ThreadPoolSize>100</ThreadPoolSize>
|
||||
</JobList>
|
||||
<TupleWSDL>
|
||||
<MaxSize>1M</MaxSize> <!-- Max size in bytes per bucket -->
|
||||
@ -509,9 +510,9 @@
|
||||
<!-- <RowAggrRowGroupsPerThread>20</RowAggrRowGroupsPerThread> --> <!-- Default value is 20 -->
|
||||
</RowAggregation>
|
||||
<CrossEngineSupport>
|
||||
<Host>unassigned</Host>
|
||||
<Host>127.0.0.1</Host>
|
||||
<Port>3306</Port>
|
||||
<User>unassigned</User>
|
||||
<User>root</User>
|
||||
<Password></Password>
|
||||
</CrossEngineSupport>
|
||||
<QueryStats>
|
||||
|
@ -30,6 +30,8 @@ using namespace logging;
|
||||
#include "threadpool.h"
|
||||
#include <iomanip>
|
||||
#include <sstream>
|
||||
#include "boost/date_time/posix_time/posix_time_types.hpp"
|
||||
|
||||
namespace threadpool
|
||||
{
|
||||
|
||||
@ -43,22 +45,19 @@ ThreadPool::ThreadPool( size_t maxThreads, size_t queueSize )
|
||||
:fMaxThreads( maxThreads ), fQueueSize( queueSize )
|
||||
{
|
||||
init();
|
||||
|
||||
if (fQueueSize == 0)
|
||||
fQueueSize = fMaxThreads*2;
|
||||
}
|
||||
|
||||
|
||||
|
||||
ThreadPool::~ThreadPool() throw()
|
||||
{
|
||||
// delete fThreadCreated;
|
||||
try
|
||||
{
|
||||
stop();
|
||||
}
|
||||
catch (...)
|
||||
{}
|
||||
{
|
||||
}
|
||||
}
|
||||
|
||||
void ThreadPool::init()
|
||||
@ -67,10 +66,9 @@ void ThreadPool::init()
|
||||
fGeneralErrors = 0;
|
||||
fFunctorErrors = 0;
|
||||
waitingFunctorsSize = 0;
|
||||
issued = 0;
|
||||
fIssued = 0;
|
||||
fDebug = false;
|
||||
fStop = false;
|
||||
// fThreadCreated = new NoOp();
|
||||
fNextFunctor = fWaitingFunctors.end();
|
||||
fNextHandle=1;
|
||||
}
|
||||
@ -88,11 +86,6 @@ void ThreadPool::setMaxThreads(size_t maxThreads)
|
||||
fMaxThreads = maxThreads;
|
||||
}
|
||||
|
||||
void ThreadPool::setThreadCreatedListener(const Functor_T &f)
|
||||
{
|
||||
// fThreadCreated = f;
|
||||
}
|
||||
|
||||
void ThreadPool::stop()
|
||||
{
|
||||
boost::mutex::scoped_lock lock1(fMutex);
|
||||
@ -183,7 +176,6 @@ uint64_t ThreadPool::invoke(const Functor_T &threadfunc)
|
||||
uint64_t thrHandle=0;
|
||||
for (;;)
|
||||
{
|
||||
|
||||
try
|
||||
{
|
||||
if (waitingFunctorsSize < fThreadCount)
|
||||
@ -197,14 +189,15 @@ uint64_t ThreadPool::invoke(const Functor_T &threadfunc)
|
||||
|
||||
bool bAdded = false;
|
||||
|
||||
if ( waitingFunctorsSize < fQueueSize)
|
||||
if (waitingFunctorsSize < fQueueSize || fQueueSize == 0)
|
||||
{
|
||||
// Don't create a thread unless you have to
|
||||
thrHandle = addFunctor(threadfunc);
|
||||
bAdded = true;
|
||||
}
|
||||
|
||||
if ( fThreadCount < fMaxThreads)
|
||||
// fQueueSize = 0 disables the queue and is an indicator to allow any number of threads to actually run.
|
||||
if (fThreadCount < fMaxThreads || fQueueSize == 0)
|
||||
{
|
||||
++fThreadCount;
|
||||
|
||||
@ -271,10 +264,8 @@ void ThreadPool::beginThread() throw()
|
||||
{
|
||||
try
|
||||
{
|
||||
// fThreadCreated();
|
||||
|
||||
boost::mutex::scoped_lock lock1(fMutex);
|
||||
|
||||
boost::system_time timeout = boost::get_system_time()+boost::posix_time::minutes(10);
|
||||
for (;;)
|
||||
{
|
||||
if (fStop)
|
||||
@ -283,41 +274,70 @@ void ThreadPool::beginThread() throw()
|
||||
if (fNextFunctor == fWaitingFunctors.end())
|
||||
{
|
||||
// Wait until someone needs a thread
|
||||
// Add the timed waait for queueSize == 0 so we can idle away threads
|
||||
// over fMaxThreads
|
||||
if (fQueueSize > 0)
|
||||
{
|
||||
fNeedThread.wait(lock1);
|
||||
}
|
||||
else
|
||||
{
|
||||
// Wait no more than 10 minutes
|
||||
if (fNeedThread.timed_wait(lock1, timeout) == boost::cv_status::timeout)
|
||||
{
|
||||
if (fThreadCount > fMaxThreads)
|
||||
{
|
||||
--fThreadCount;
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
/* Need to tune these magic #s */
|
||||
|
||||
vector<Container_T::iterator> todoList;
|
||||
int i, num;
|
||||
Container_T::const_iterator iter;
|
||||
|
||||
/* Use this to control how many jobs are issued to a single thread */
|
||||
num = (waitingFunctorsSize - issued >= 1 ? 1 : 0);
|
||||
/* Use num to control how many jobs are issued to a single thread
|
||||
should you want to batch more than one */
|
||||
num = (waitingFunctorsSize - fIssued >= 1 ? 1 : 0);
|
||||
|
||||
for (i = 0; i < num; i++)
|
||||
todoList.push_back(fNextFunctor++);
|
||||
|
||||
issued += num;
|
||||
fIssued += num;
|
||||
// cerr << "got " << num << " jobs." << endl;
|
||||
// cerr << "got " << num << " jobs. waitingFunctorsSize=" <<
|
||||
// waitingFunctorsSize << " issued=" << issued << " fThreadCount=" <<
|
||||
// waitingFunctorsSize << " fIssued=" << fIssued << " fThreadCount=" <<
|
||||
// fThreadCount << endl;
|
||||
lock1.unlock();
|
||||
|
||||
for (i = 0; i < num; i++) {
|
||||
try {
|
||||
for (i = 0; i < num; i++)
|
||||
{
|
||||
try
|
||||
{
|
||||
(*todoList[i]).functor();
|
||||
}
|
||||
catch(exception &e) {
|
||||
catch (exception &e)
|
||||
{
|
||||
++fFunctorErrors;
|
||||
cerr << e.what() << endl;
|
||||
#ifndef NOLOGGING
|
||||
logging::Message::Args args;
|
||||
logging::Message message(5);
|
||||
args.add("ThreadPool: Caught exception during execution: ");
|
||||
args.add(e.what());
|
||||
message.format( args );
|
||||
logging::LoggingID lid(22);
|
||||
logging::MessageLog ml(lid);
|
||||
ml.logErrorMessage( message );
|
||||
#endif
|
||||
}
|
||||
}
|
||||
lock1.lock();
|
||||
|
||||
issued -= num;
|
||||
fIssued -= num;
|
||||
waitingFunctorsSize -= num;
|
||||
for (i = 0; i < num; i++)
|
||||
fWaitingFunctors.erase(todoList[i]);
|
||||
@ -326,8 +346,8 @@ void ThreadPool::beginThread() throw()
|
||||
cerr << "size mismatch! fake size=" << waitingFunctorsSize <<
|
||||
" real size=" << fWaitingFunctors.size() << endl;
|
||||
*/
|
||||
timeout = boost::get_system_time()+boost::posix_time::minutes(10);
|
||||
fThreadAvailable.notify_all();
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -458,4 +478,5 @@ void ThreadPoolMonitor::operator()()
|
||||
sleep(2);
|
||||
}
|
||||
}
|
||||
|
||||
} // namespace threadpool
|
||||
|
@ -75,6 +75,9 @@ public:
|
||||
* of simultaneuous operations that can go on.
|
||||
* @param queueSize the maximum number of work tasks in the queue. This is the maximum
|
||||
* number of jobs that can queue up in the work list before invoke() blocks.
|
||||
* If 0, then threads never block and total threads may
|
||||
* exceed maxThreads. Nothing waits. Thread count will
|
||||
* idle down to maxThreads when less work is required.
|
||||
*/
|
||||
EXPORT explicit ThreadPool( size_t maxThreads, size_t queueSize );
|
||||
|
||||
@ -108,11 +111,6 @@ public:
|
||||
*/
|
||||
inline size_t getMaxThreads() const { return fMaxThreads; }
|
||||
|
||||
/** @brief register a functor to be called when a new thread
|
||||
* is created
|
||||
*/
|
||||
EXPORT void setThreadCreatedListener(const Functor_T &f) ;
|
||||
|
||||
/** @brief queue size accessor
|
||||
*
|
||||
*/
|
||||
@ -218,9 +216,8 @@ private:
|
||||
typedef std::list<PoolFunction_T> Container_T;
|
||||
Container_T fWaitingFunctors;
|
||||
Container_T::iterator fNextFunctor;
|
||||
// Functor_T * fThreadCreated;
|
||||
|
||||
uint32_t issued;
|
||||
uint32_t fIssued;
|
||||
boost::mutex fMutex;
|
||||
boost::condition fThreadAvailable; // triggered when a thread is available
|
||||
boost::condition fNeedThread; // triggered when a thread is needed
|
||||
|
Reference in New Issue
Block a user