2023-04-09 12:50:26 +00:00
# pragma once
# include <condition_variable>
# include <exception>
# include <memory>
# include <map>
# include <mutex>
# include <unordered_set>
# include <unordered_map>
# include <boost/noncopyable.hpp>
# include <base/defines.h>
# include <base/types.h>
# include <Common/CurrentMetrics.h>
# include <Common/ThreadPool.h>
# include <Common/ErrorCodes.h>
# include <Common/Exception.h>
# include <Common/noexcept_scope.h>
# include <Common/setThreadName.h>
namespace DB
{
class LoadJob ;
using LoadJobPtr = std : : shared_ptr < LoadJob > ;
using LoadJobSet = std : : unordered_set < LoadJobPtr > ;
class LoadTask ;
class AsyncLoader ;
namespace ErrorCodes
{
2023-04-13 18:43:58 +00:00
extern const int ASYNC_LOAD_SCHEDULE_FAILED ;
2023-04-09 12:50:26 +00:00
extern const int ASYNC_LOAD_FAILED ;
extern const int ASYNC_LOAD_CANCELED ;
}
enum class LoadStatus
{
2023-04-14 14:00:54 +00:00
PENDING , // Load job is not started yet
OK , // Load job executed and was successful
FAILED , // Load job executed and failed
CANCELED // Load job is not going to be executed due to removal or dependency failure
2023-04-09 12:50:26 +00:00
} ;
class LoadJob : private boost : : noncopyable
{
public :
template < class Func >
LoadJob ( LoadJobSet & & dependencies_ , const String & name_ , Func & & func_ )
: dependencies ( std : : move ( dependencies_ ) )
, name ( name_ )
, func ( std : : forward < Func > ( func_ ) )
{ }
LoadStatus status ( ) const
{
std : : unique_lock lock { mutex } ;
2023-04-14 14:00:54 +00:00
return load_status ;
2023-04-09 12:50:26 +00:00
}
2023-04-14 17:16:30 +00:00
std : : exception_ptr exception ( ) const
{
std : : unique_lock lock { mutex } ;
return load_exception ;
}
2023-04-14 00:04:31 +00:00
void wait ( ) const
2023-04-09 12:50:26 +00:00
{
std : : unique_lock lock { mutex } ;
2023-04-14 00:04:31 +00:00
waiters + + ;
2023-04-14 14:00:54 +00:00
finished . wait ( lock , [ this ] { return load_status ! = LoadStatus : : PENDING ; } ) ;
2023-04-14 00:04:31 +00:00
waiters - - ;
2023-04-14 17:16:30 +00:00
if ( load_exception )
std : : rethrow_exception ( load_exception ) ;
2023-04-09 12:50:26 +00:00
}
2023-04-14 00:04:31 +00:00
void waitNoThrow ( ) const
{
std : : unique_lock lock { mutex } ;
waiters + + ;
2023-04-14 14:00:54 +00:00
finished . wait ( lock , [ this ] { return load_status ! = LoadStatus : : PENDING ; } ) ;
2023-04-14 00:04:31 +00:00
waiters - - ;
}
size_t waiters_count ( ) const
{
std : : unique_lock lock { mutex } ;
return waiters ;
}
2023-04-13 18:43:58 +00:00
const LoadJobSet dependencies ; // jobs to be done before this one (with ownership), it is `const` to make creation of cycles hard
2023-04-09 12:50:26 +00:00
const String name ;
std : : atomic < ssize_t > priority { 0 } ;
private :
friend class AsyncLoader ;
2023-04-14 14:00:54 +00:00
void ok ( )
2023-04-09 12:50:26 +00:00
{
std : : unique_lock lock { mutex } ;
2023-04-14 14:00:54 +00:00
load_status = LoadStatus : : OK ;
2023-04-14 00:04:31 +00:00
if ( waiters > 0 )
finished . notify_all ( ) ;
2023-04-09 12:50:26 +00:00
}
2023-04-14 14:00:54 +00:00
void failed ( const std : : exception_ptr & ptr )
2023-04-09 12:50:26 +00:00
{
std : : unique_lock lock { mutex } ;
2023-04-14 14:00:54 +00:00
load_status = LoadStatus : : FAILED ;
2023-04-14 17:16:30 +00:00
load_exception = ptr ;
2023-04-14 14:00:54 +00:00
if ( waiters > 0 )
finished . notify_all ( ) ;
}
void canceled ( const std : : exception_ptr & ptr )
{
std : : unique_lock lock { mutex } ;
load_status = LoadStatus : : CANCELED ;
2023-04-14 17:16:30 +00:00
load_exception = ptr ;
2023-04-14 00:04:31 +00:00
if ( waiters > 0 )
finished . notify_all ( ) ;
2023-04-09 12:50:26 +00:00
}
2023-04-14 13:14:31 +00:00
std : : function < void ( const LoadJobPtr & self ) > func ;
2023-04-09 12:50:26 +00:00
2023-04-10 23:14:20 +00:00
mutable std : : mutex mutex ;
2023-04-14 00:04:31 +00:00
mutable std : : condition_variable finished ;
mutable size_t waiters = 0 ;
2023-04-14 14:00:54 +00:00
LoadStatus load_status { LoadStatus : : PENDING } ;
2023-04-14 17:16:30 +00:00
std : : exception_ptr load_exception ;
2023-04-09 12:50:26 +00:00
} ;
template < class Func >
LoadJobPtr makeLoadJob ( LoadJobSet & & dependencies , const String & name , Func & & func )
{
return std : : make_shared < LoadJob > ( std : : move ( dependencies ) , name , std : : forward < Func > ( func ) ) ;
}
2023-04-16 12:07:46 +00:00
// `AsyncLoader` is a scheduler for DAG of `LoadJob`s. It tracks dependencies and priorities of jobs.
// Basic usage example:
// auto job_func = [&] (const LoadJobPtr & self) {
// LOG_TRACE(log, "Executing load job '{}' with priority '{}'", self->name, self->priority);
// };
// auto job1 = makeLoadJob({}, "job1", job_func);
// auto job2 = makeLoadJob({ job1 }, "job2", job_func);
// auto job3 = makeLoadJob({ job1 }, "job3", job_func);
// auto task = async_loader.schedule({ job1, job2, job3 }, /* priority = */ 0);
// Here we have created and scheduled a task consisting of two jobs. Job1 has no dependencies and is run first.
// Job2 and job3 depends on job1 and are run only after job1 completion. Another thread may prioritize a job and wait for it:
// async_loader->prioritize(job3, 1); // higher priority jobs are run first
// job3->wait(); // blocks until job completion or cancelation and rethrow an exception (if any)
//
// AsyncLoader tracks state of all scheduled jobs. Job lifecycle is the following:
// 1) Job is constructed with PENDING status.
// 2) Job is scheduled and placed into a task. Scheduled job may be ready (i.e. have all its dependencies finished) or blocked.
// 3a) When all dependencies are successfully executed, job became ready. Ready job is enqueued into the ready queue.
// 3b) If at least one of job dependencies is failed or canceled, then this job is canceled (with all it's dependent jobs as well).
2023-04-16 13:09:44 +00:00
// On cancellation an ASYNC_LOAD_CANCELED exception is generated and saved inside LoadJob object. Job status is changed to CANCELED.
2023-04-16 12:07:46 +00:00
// Exception is rethrown by any existing or new `wait()` call. Job is moved to the set of finished jobs.
// 4) Scheduled pending ready job starts execution by a worker. Job is dequeuedCallback `job_func` is called.
// Status of an executing job is PENDING. And it is still considered as scheduled job by AsyncLoader.
// Note that `job_func` of a CANCELED job is never executed.
// 5a) On successful execution job status is changed to OK and all existing and new `wait()` calls finish w/o exceptions.
// 5b) Any exception thrown out of `job_func` is wrapped into ASYNC_LOAD_FAILED exception and save inside LoadJob.
// Job status is changed to FAILED. All dependent jobs are canceled. The exception is rethrown from all existing and new `wait()` calls.
// 6) Job is no longer considered as scheduled and is instead moved to finished jobs set. This is required for introspection of finished jobs.
// 7) Task object containing this job is destructed or `remove()` is explicitly called. Job is removed from the finished job set.
// 8) Job is destructed.
//
// Every job has a priority associated with it. AsyncLoader runs higher priority (greater `priority` value) jobs first. Job priority can be elevated
// (a) if either it has a dependent job with higher priority (in this case priority of a dependent job is inherited);
// (b) or job was explicitly prioritized by `prioritize(job, higher_priority)` call (this also leads to a priority inheritance for all the dependencies).
// Note that to avoid priority inversion `job_func` should use `self->priority` to schedule new jobs in AsyncLoader or any other pool.
// Value stored in load job priority field is atomic and can be increased even during job execution.
//
2023-04-16 12:13:23 +00:00
// When task is scheduled it can contain dependencies on previously scheduled jobs. These jobs can have any status.
2023-04-16 12:07:46 +00:00
// The only forbidden thing is a dependency on job, that was not scheduled in AsyncLoader yet: all dependent jobs are immediately canceled.
2023-04-09 12:50:26 +00:00
class AsyncLoader : private boost : : noncopyable
{
private :
// Key of a pending job in ready queue
2023-04-11 00:08:55 +00:00
struct ReadyKey
{
2023-04-15 19:48:10 +00:00
ssize_t priority ; // Ascending order
ssize_t initial_priority ; // Ascending order
UInt64 ready_seqno ; // Descending order
2023-04-09 12:50:26 +00:00
bool operator < ( const ReadyKey & rhs ) const
{
2023-04-15 19:48:10 +00:00
if ( priority > rhs . priority )
return true ;
if ( priority < rhs . priority )
return false ;
if ( initial_priority > rhs . initial_priority )
return true ;
if ( initial_priority < rhs . initial_priority )
return false ;
return ready_seqno < rhs . ready_seqno ;
2023-04-09 12:50:26 +00:00
}
} ;
// Scheduling information for a pending job
2023-04-11 00:08:55 +00:00
struct Info
{
2023-04-15 19:48:10 +00:00
ssize_t initial_priority = 0 ; // Initial priority passed into schedule()
ssize_t priority = 0 ; // Elevated priority, due to priority inheritance or prioritize()
2023-04-09 12:50:26 +00:00
size_t dependencies_left = 0 ;
2023-04-15 19:48:10 +00:00
UInt64 ready_seqno = 0 ; // Zero means that job is not in ready queue
2023-04-10 23:14:20 +00:00
LoadJobSet dependent_jobs ;
2023-04-09 12:50:26 +00:00
2023-04-14 14:00:54 +00:00
// Three independent states of a non-finished jobs
bool is_blocked ( ) const { return dependencies_left > 0 ; }
bool is_ready ( ) const { return dependencies_left = = 0 & & ready_seqno > 0 ; }
bool is_executing ( ) const { return dependencies_left = = 0 & & ready_seqno = = 0 ; }
// Get key of a ready job
2023-04-09 12:50:26 +00:00
ReadyKey key ( ) const
{
2023-04-15 19:48:10 +00:00
return { . priority = priority , . initial_priority = initial_priority , . ready_seqno = ready_seqno } ;
2023-04-09 12:50:26 +00:00
}
} ;
public :
using Metric = CurrentMetrics : : Metric ;
// Helper class that removes all not started jobs in destructor and wait all executing jobs to finish
class Task
{
public :
Task ( )
: loader ( nullptr )
{ }
Task ( AsyncLoader * loader_ , LoadJobSet & & jobs_ )
: loader ( loader_ )
, jobs ( std : : move ( jobs_ ) )
{ }
Task ( const Task & o ) = delete ;
Task & operator = ( const Task & o ) = delete ;
Task ( Task & & o ) noexcept
: loader ( std : : exchange ( o . loader , nullptr ) )
, jobs ( std : : move ( o . jobs ) )
{ }
Task & operator = ( Task & & o ) noexcept
{
loader = std : : exchange ( o . loader , nullptr ) ;
jobs = std : : move ( o . jobs ) ;
return * this ;
}
void merge ( Task & & o )
{
if ( ! loader )
{
* this = std : : move ( o ) ;
}
else
{
chassert ( loader = = o . loader ) ;
jobs . merge ( o . jobs ) ;
o . loader = nullptr ;
}
}
~ Task ( )
{
2023-04-13 23:18:38 +00:00
remove ( ) ;
}
void remove ( )
{
2023-04-09 12:50:26 +00:00
if ( loader )
2023-04-13 23:18:38 +00:00
{
2023-04-09 12:50:26 +00:00
loader - > remove ( jobs ) ;
2023-04-13 23:18:38 +00:00
detach ( ) ;
}
2023-04-09 12:50:26 +00:00
}
// Do not track jobs in this task
2023-04-16 11:04:59 +00:00
// WARNING: Jobs will never be removed() and are going to be stored as finished_jobs until ~AsyncLoader()
2023-04-09 12:50:26 +00:00
void detach ( )
{
loader = nullptr ;
jobs . clear ( ) ;
}
private :
AsyncLoader * loader ;
LoadJobSet jobs ;
} ;
2023-04-14 17:16:30 +00:00
AsyncLoader ( Metric metric_threads , Metric metric_active_threads , size_t max_threads_ , bool log_failures_ )
: log_failures ( log_failures_ )
, max_threads ( max_threads_ )
2023-04-09 12:50:26 +00:00
, pool ( metric_threads , metric_active_threads , max_threads )
{ }
2023-04-13 22:48:46 +00:00
// WARNING: all Task instances returned by `schedule()` should be destructed before AsyncLoader
~ AsyncLoader ( )
{
stop ( ) ;
}
2023-04-10 23:14:20 +00:00
// Start workers to execute scheduled load jobs
2023-04-09 12:50:26 +00:00
void start ( )
{
std : : unique_lock lock { mutex } ;
is_running = true ;
for ( size_t i = 0 ; workers < max_threads & & i < ready_queue . size ( ) ; i + + )
spawn ( lock ) ;
}
2023-04-10 23:14:20 +00:00
// Wait for all load jobs to finish, including all new jobs. So at first take care to stop adding new jobs.
void wait ( )
{
pool . wait ( ) ;
}
// Wait for currently executing jobs to finish, but do not run any other pending jobs.
// Not finished jobs are left in pending state:
// - they can be resumed by calling start() again;
// - or canceled using ~Task() or remove() later.
2023-04-09 12:50:26 +00:00
void stop ( )
{
2023-04-10 23:14:20 +00:00
{
std : : unique_lock lock { mutex } ;
is_running = false ;
// NOTE: there is no need to notify because workers never wait
}
pool . wait ( ) ;
2023-04-09 12:50:26 +00:00
}
[[nodiscard]] Task schedule ( LoadJobSet & & jobs , ssize_t priority = 0 )
{
std : : unique_lock lock { mutex } ;
// Sanity checks
for ( const auto & job : jobs )
{
if ( job - > status ( ) ! = LoadStatus : : PENDING )
2023-04-13 18:43:58 +00:00
throw Exception ( ErrorCodes : : ASYNC_LOAD_SCHEDULE_FAILED , " Trying to schedule already finished load job '{}' " , job - > name ) ;
2023-04-09 12:50:26 +00:00
if ( scheduled_jobs . contains ( job ) )
2023-04-13 18:43:58 +00:00
throw Exception ( ErrorCodes : : ASYNC_LOAD_SCHEDULE_FAILED , " Load job '{}' has been already scheduled " , job - > name ) ;
2023-04-09 12:50:26 +00:00
}
2023-04-13 18:43:58 +00:00
// Ensure scheduled_jobs graph will have no cycles. The only way to get a cycle is to add a cycle, assuming old jobs cannot reference new ones.
checkCycle ( jobs , lock ) ;
2023-04-09 12:50:26 +00:00
// We do not want any exception to be throws after this point, because the following code is not exception-safe
DENY_ALLOCATIONS_IN_SCOPE ;
// Schedule all incoming jobs
for ( const auto & job : jobs )
{
NOEXCEPT_SCOPE ( {
ALLOW_ALLOCATIONS_IN_SCOPE ;
2023-04-15 19:48:10 +00:00
scheduled_jobs . emplace ( job , Info { . initial_priority = priority , . priority = priority } ) ;
2023-04-10 23:14:20 +00:00
} ) ;
2023-04-09 12:50:26 +00:00
job - > priority . store ( priority ) ; // Set user-facing priority
}
2023-04-14 17:16:30 +00:00
// Process dependencies on scheduled pending jobs
2023-04-09 12:50:26 +00:00
for ( const auto & job : jobs )
{
Info & info = scheduled_jobs . find ( job ) - > second ;
for ( const auto & dep : job - > dependencies )
{
// Register every dependency on scheduled job with back-link to dependent job
if ( auto dep_info = scheduled_jobs . find ( dep ) ; dep_info ! = scheduled_jobs . end ( ) )
{
NOEXCEPT_SCOPE ( {
ALLOW_ALLOCATIONS_IN_SCOPE ;
dep_info - > second . dependent_jobs . insert ( job ) ;
} ) ;
info . dependencies_left + + ;
2023-04-14 17:16:30 +00:00
// Priority inheritance: prioritize deps to have at least given `priority` to avoid priority inversion
prioritize ( dep , priority , lock ) ;
}
2023-04-09 12:50:26 +00:00
}
2023-04-14 14:00:54 +00:00
// Enqueue non-blocked jobs (w/o dependencies) to ready queue
if ( ! info . is_blocked ( ) )
2023-04-09 12:50:26 +00:00
enqueue ( info , job , lock ) ;
}
2023-04-14 17:16:30 +00:00
// Process dependencies on other jobs. It is done in a separate pass to facilitate propagation of cancel signals (if any).
for ( const auto & job : jobs )
{
if ( auto info = scheduled_jobs . find ( job ) ; info ! = scheduled_jobs . end ( ) )
{
for ( const auto & dep : job - > dependencies )
{
if ( scheduled_jobs . contains ( dep ) )
continue ; // Skip dependencies on scheduled pending jobs (already processed)
LoadStatus dep_status = dep - > status ( ) ;
if ( dep_status = = LoadStatus : : OK )
continue ; // Dependency on already successfully finished job -- it's okay.
if ( dep_status = = LoadStatus : : PENDING )
{
// Dependency on not scheduled pending job -- it's bad.
// Probably, there is an error in `jobs` set: not all jobs were passed to `schedule()` call.
// We are not going to run any dependent job, so cancel them all.
std : : exception_ptr e ;
NOEXCEPT_SCOPE ( {
ALLOW_ALLOCATIONS_IN_SCOPE ;
e = std : : make_exception_ptr ( Exception ( ErrorCodes : : ASYNC_LOAD_CANCELED ,
" Load job '{}' -> Load job '{}': not scheduled pending load job (it must be also scheduled), all dependent load jobs are canceled " ,
job - > name ,
dep - > name ) ) ;
} ) ;
finish ( lock , job , LoadStatus : : CANCELED , e ) ;
break ; // This job is now finished, stop its dependencies processing
}
if ( dep_status = = LoadStatus : : FAILED | | dep_status = = LoadStatus : : CANCELED )
{
// Dependency on already failed or canceled job -- it's okay. Cancel all dependent jobs.
std : : exception_ptr e ;
NOEXCEPT_SCOPE ( {
ALLOW_ALLOCATIONS_IN_SCOPE ;
e = std : : make_exception_ptr ( Exception ( ErrorCodes : : ASYNC_LOAD_CANCELED ,
" Load job '{}' -> {} " ,
job - > name ,
getExceptionMessage ( dep - > exception ( ) , /* with_stack_trace = */ false ) ) ) ;
} ) ;
finish ( lock , job , LoadStatus : : CANCELED , e ) ;
break ; // This job is now finished, stop its dependencies processing
}
}
}
else
{
// Job was already canceled on previous iteration of this cycle -- skip
}
}
2023-04-09 12:50:26 +00:00
return Task ( this , std : : move ( jobs ) ) ;
}
// Increase priority of a job and all its dependencies recursively
void prioritize ( const LoadJobPtr & job , ssize_t new_priority )
{
DENY_ALLOCATIONS_IN_SCOPE ;
std : : unique_lock lock { mutex } ;
prioritize ( job , new_priority , lock ) ;
}
// Remove finished jobs, cancel scheduled jobs, wait for executing jobs to finish and remove them
void remove ( const LoadJobSet & jobs )
{
DENY_ALLOCATIONS_IN_SCOPE ;
std : : unique_lock lock { mutex } ;
2023-04-14 01:21:25 +00:00
// On the first pass:
// - cancel all not executing jobs to avoid races
// - do not wait executing jobs (otherwise, on unlock a worker could start executing a dependent job, that should be canceled)
2023-04-09 12:50:26 +00:00
for ( const auto & job : jobs )
{
2023-04-14 01:21:25 +00:00
if ( auto info = scheduled_jobs . find ( job ) ; info ! = scheduled_jobs . end ( ) )
2023-04-09 12:50:26 +00:00
{
2023-04-14 14:00:54 +00:00
if ( info - > second . is_executing ( ) )
continue ; // Skip executing jobs on the first pass
std : : exception_ptr e ;
NOEXCEPT_SCOPE ( {
ALLOW_ALLOCATIONS_IN_SCOPE ;
2023-04-14 17:16:30 +00:00
e = std : : make_exception_ptr ( Exception ( ErrorCodes : : ASYNC_LOAD_CANCELED , " Load job '{}' canceled " , job - > name ) ) ;
2023-04-14 14:00:54 +00:00
} ) ;
2023-04-14 17:16:30 +00:00
finish ( lock , job , LoadStatus : : CANCELED , e ) ;
2023-04-09 12:50:26 +00:00
}
}
2023-04-14 01:21:25 +00:00
// On the second pass wait for executing jobs to finish
for ( const auto & job : jobs )
{
if ( auto info = scheduled_jobs . find ( job ) ; info ! = scheduled_jobs . end ( ) )
{
// Job is currently executing
2023-04-14 14:00:54 +00:00
chassert ( info - > second . is_executing ( ) ) ;
2023-04-14 01:21:25 +00:00
lock . unlock ( ) ;
job - > waitNoThrow ( ) ; // Wait for job to finish
lock . lock ( ) ;
}
}
// On the third pass all jobs are finished - remove them all
// It is better to do it under one lock to avoid exposing intermediate states
for ( const auto & job : jobs )
finished_jobs . erase ( job ) ;
2023-04-09 12:50:26 +00:00
}
2023-04-16 11:04:59 +00:00
void setMaxThreads ( size_t value )
{
std : : unique_lock lock { mutex } ;
pool . setMaxThreads ( value ) ;
pool . setMaxFreeThreads ( value ) ;
pool . setQueueSize ( value ) ;
max_threads = value ;
if ( ! is_running )
return ;
for ( size_t i = 0 ; workers < max_threads & & i < ready_queue . size ( ) ; i + + )
spawn ( lock ) ;
}
2023-04-14 12:57:51 +00:00
size_t getMaxThreads ( ) const
{
std : : unique_lock lock { mutex } ;
2023-04-16 11:04:59 +00:00
return max_threads ;
2023-04-14 12:57:51 +00:00
}
2023-04-14 20:49:56 +00:00
size_t getScheduledJobCount ( ) const
{
std : : unique_lock lock { mutex } ;
return scheduled_jobs . size ( ) ;
}
2023-04-09 12:50:26 +00:00
private :
2023-04-13 18:43:58 +00:00
void checkCycle ( const LoadJobSet & jobs , std : : unique_lock < std : : mutex > & lock )
{
LoadJobSet left = jobs ;
LoadJobSet visited ;
visited . reserve ( left . size ( ) ) ;
while ( ! left . empty ( ) )
{
LoadJobPtr job = * left . begin ( ) ;
checkCycleImpl ( job , left , visited , lock ) ;
}
}
String checkCycleImpl ( const LoadJobPtr & job , LoadJobSet & left , LoadJobSet & visited , std : : unique_lock < std : : mutex > & lock )
{
if ( ! left . contains ( job ) )
return { } ; // Do not consider external dependencies and already processed jobs
if ( auto [ _ , inserted ] = visited . insert ( job ) ; ! inserted )
{
visited . erase ( job ) ; // Mark where cycle ends
return job - > name ;
}
for ( const auto & dep : job - > dependencies )
{
if ( auto chain = checkCycleImpl ( dep , left , visited , lock ) ; ! chain . empty ( ) )
{
if ( ! visited . contains ( job ) ) // Check for cycle end
throw Exception ( ErrorCodes : : ASYNC_LOAD_SCHEDULE_FAILED , " Load job dependency cycle detected: {} -> {} " , job - > name , chain ) ;
else
return fmt : : format ( " {} -> {} " , job - > name , chain ) ; // chain is not a cycle yet -- continue building
}
}
left . erase ( job ) ;
return { } ;
}
2023-04-14 17:16:30 +00:00
void finish ( std : : unique_lock < std : : mutex > & lock , const LoadJobPtr & job , LoadStatus status , std : : exception_ptr exception_from_job = { } )
2023-04-09 12:50:26 +00:00
{
2023-04-14 17:16:30 +00:00
if ( status = = LoadStatus : : OK )
2023-04-09 12:50:26 +00:00
{
2023-04-14 17:16:30 +00:00
// Notify waiters
job - > ok ( ) ;
// Update dependent jobs and enqueue if ready
chassert ( scheduled_jobs . contains ( job ) ) ; // Job was pending
for ( const auto & dep : scheduled_jobs [ job ] . dependent_jobs )
{
chassert ( scheduled_jobs . contains ( dep ) ) ; // All depended jobs must be pending
Info & dep_info = scheduled_jobs [ dep ] ;
dep_info . dependencies_left - - ;
if ( ! dep_info . is_blocked ( ) )
enqueue ( dep_info , dep , lock ) ;
}
2023-04-09 12:50:26 +00:00
}
2023-04-14 17:16:30 +00:00
else
{
// Notify waiters
if ( status = = LoadStatus : : FAILED )
job - > failed ( exception_from_job ) ;
else if ( status = = LoadStatus : : CANCELED )
job - > canceled ( exception_from_job ) ;
chassert ( scheduled_jobs . contains ( job ) ) ; // Job was pending
Info & info = scheduled_jobs [ job ] ;
if ( info . is_ready ( ) )
{
ready_queue . erase ( info . key ( ) ) ;
info . ready_seqno = 0 ;
}
2023-04-09 12:50:26 +00:00
2023-04-14 17:16:30 +00:00
// Recurse into all dependent jobs
LoadJobSet dependent ;
dependent . swap ( info . dependent_jobs ) ; // To avoid container modification during recursion
for ( const auto & dep : dependent )
{
std : : exception_ptr e ;
NOEXCEPT_SCOPE ( {
ALLOW_ALLOCATIONS_IN_SCOPE ;
e = std : : make_exception_ptr (
Exception ( ErrorCodes : : ASYNC_LOAD_CANCELED ,
" Load job '{}' -> {} " ,
dep - > name ,
getExceptionMessage ( exception_from_job , /* with_stack_trace = */ false ) ) ) ;
} ) ;
finish ( lock , dep , LoadStatus : : CANCELED , e ) ;
}
2023-04-09 12:50:26 +00:00
2023-04-14 17:16:30 +00:00
// Clean dependency graph edges
for ( const auto & dep : job - > dependencies )
if ( auto dep_info = scheduled_jobs . find ( dep ) ; dep_info ! = scheduled_jobs . end ( ) )
dep_info - > second . dependent_jobs . erase ( job ) ;
2023-04-09 12:50:26 +00:00
}
// Job became finished
scheduled_jobs . erase ( job ) ;
NOEXCEPT_SCOPE ( {
ALLOW_ALLOCATIONS_IN_SCOPE ;
finished_jobs . insert ( job ) ;
2023-04-10 23:14:20 +00:00
} ) ;
2023-04-09 12:50:26 +00:00
}
void prioritize ( const LoadJobPtr & job , ssize_t new_priority , std : : unique_lock < std : : mutex > & lock )
{
if ( auto info = scheduled_jobs . find ( job ) ; info ! = scheduled_jobs . end ( ) )
{
if ( info - > second . priority > = new_priority )
return ; // Never lower priority
// Update priority and push job forward through ready queue if needed
if ( info - > second . ready_seqno )
ready_queue . erase ( info - > second . key ( ) ) ;
info - > second . priority = new_priority ;
job - > priority . store ( new_priority ) ; // Set user-facing priority (may affect executing jobs)
if ( info - > second . ready_seqno )
{
NOEXCEPT_SCOPE ( {
ALLOW_ALLOCATIONS_IN_SCOPE ;
ready_queue . emplace ( info - > second . key ( ) , job ) ;
} ) ;
}
// Recurse into dependencies
for ( const auto & dep : job - > dependencies )
prioritize ( dep , new_priority , lock ) ;
}
}
void enqueue ( Info & info , const LoadJobPtr & job , std : : unique_lock < std : : mutex > & lock )
{
2023-04-14 14:00:54 +00:00
chassert ( ! info . is_blocked ( ) ) ;
2023-04-09 12:50:26 +00:00
chassert ( info . ready_seqno = = 0 ) ;
info . ready_seqno = + + last_ready_seqno ;
NOEXCEPT_SCOPE ( {
ALLOW_ALLOCATIONS_IN_SCOPE ;
ready_queue . emplace ( info . key ( ) , job ) ;
} ) ;
2023-04-16 11:04:59 +00:00
if ( is_running & & workers < max_threads )
2023-04-09 12:50:26 +00:00
spawn ( lock ) ;
}
void spawn ( std : : unique_lock < std : : mutex > & )
{
workers + + ;
NOEXCEPT_SCOPE ( {
ALLOW_ALLOCATIONS_IN_SCOPE ;
pool . scheduleOrThrowOnError ( [ this ] { worker ( ) ; } ) ;
} ) ;
}
void worker ( )
{
DENY_ALLOCATIONS_IN_SCOPE ;
LoadJobPtr job ;
std : : exception_ptr exception_from_job ;
2023-04-11 00:08:55 +00:00
while ( true )
{
2023-04-09 12:50:26 +00:00
/// This is inside the loop to also reset previous thread names set inside the jobs
setThreadName ( " AsyncLoader " ) ;
{
std : : unique_lock lock { mutex } ;
// Handle just executed job
if ( exception_from_job )
2023-04-14 17:16:30 +00:00
finish ( lock , job , LoadStatus : : FAILED , exception_from_job ) ;
2023-04-09 12:50:26 +00:00
else if ( job )
2023-04-14 17:16:30 +00:00
finish ( lock , job , LoadStatus : : OK ) ;
2023-04-09 12:50:26 +00:00
2023-04-16 11:04:59 +00:00
if ( ! is_running | | ready_queue . empty ( ) | | workers > max_threads )
2023-04-09 12:50:26 +00:00
{
workers - - ;
return ;
}
// Take next job to be executed from the ready queue
auto it = ready_queue . begin ( ) ;
job = it - > second ;
ready_queue . erase ( it ) ;
scheduled_jobs . find ( job ) - > second . ready_seqno = 0 ; // This job is no longer in the ready queue
}
2023-04-14 17:16:30 +00:00
ALLOW_ALLOCATIONS_IN_SCOPE ;
2023-04-09 12:50:26 +00:00
try
{
2023-04-14 13:14:31 +00:00
job - > func ( job ) ;
2023-04-09 12:50:26 +00:00
exception_from_job = { } ;
}
catch ( . . . )
{
NOEXCEPT_SCOPE ( {
2023-04-14 17:16:30 +00:00
if ( log_failures )
tryLogCurrentException ( __PRETTY_FUNCTION__ ) ;
2023-04-09 12:50:26 +00:00
exception_from_job = std : : make_exception_ptr (
2023-04-10 23:14:20 +00:00
Exception ( ErrorCodes : : ASYNC_LOAD_FAILED ,
2023-04-09 12:50:26 +00:00
" Load job '{}' failed: {} " ,
job - > name ,
getCurrentExceptionMessage ( /* with_stacktrace = */ true ) ) ) ;
} ) ;
}
}
}
2023-04-14 17:16:30 +00:00
const bool log_failures ;
2023-04-14 12:57:51 +00:00
mutable std : : mutex mutex ;
2023-04-09 12:50:26 +00:00
bool is_running = false ;
// Full set of scheduled pending jobs along with scheduling info
std : : unordered_map < LoadJobPtr , Info > scheduled_jobs ;
// Subset of scheduled pending jobs with resolved dependencies (waiting for a thread to be executed)
// Represent a queue of jobs in order of decreasing priority and FIFO for jobs with equal priorities
std : : map < ReadyKey , LoadJobPtr > ready_queue ;
// Set of finished jobs (for introspection only, until job is removed)
LoadJobSet finished_jobs ;
// Increasing counter for ReadyKey assignment (to preserve FIFO order of jobs with equal priority)
UInt64 last_ready_seqno = 0 ;
// For executing jobs. Note that we avoid using an internal queue of the pool to be able to prioritize jobs
size_t max_threads ;
size_t workers = 0 ;
ThreadPool pool ;
} ;
}