2023-04-09 12:50:26 +00:00
# pragma once
# include <condition_variable>
# include <exception>
# include <memory>
# include <map>
# include <mutex>
2023-04-22 17:28:53 +00:00
# include <vector>
2023-04-09 12:50:26 +00:00
# include <unordered_set>
# include <unordered_map>
# include <boost/noncopyable.hpp>
# include <base/types.h>
# include <Common/CurrentMetrics.h>
2023-04-27 14:49:35 +00:00
# include <Common/Stopwatch.h>
2023-04-09 12:50:26 +00:00
# include <Common/ThreadPool.h>
2023-04-27 14:49:35 +00:00
namespace Poco { class Logger ; }
2023-04-09 12:50:26 +00:00
namespace DB
{
class LoadJob ;
using LoadJobPtr = std : : shared_ptr < LoadJob > ;
using LoadJobSet = std : : unordered_set < LoadJobPtr > ;
class LoadTask ;
2023-04-22 17:28:53 +00:00
using LoadTaskPtr = std : : shared_ptr < LoadTask > ;
2023-04-09 12:50:26 +00:00
class AsyncLoader ;
2023-04-27 14:49:35 +00:00
void logAboutProgress ( Poco : : Logger * log , size_t processed , size_t total , AtomicStopwatch & watch ) ;
2023-04-16 18:17:11 +00:00
// Execution status of a load job.
2023-04-09 12:50:26 +00:00
enum class LoadStatus
{
2023-04-16 18:17:11 +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
} ;
2023-04-16 18:17:11 +00:00
// Smallest indivisible part of a loading process. Load job can have multiple dependencies, thus jobs constitute a direct acyclic graph (DAG).
// Job encapsulates a function to be executed by `AsyncLoader` as soon as job functions of all dependencies are successfully executed.
// Job can be waited for by an arbitrary number of threads. See `AsyncLoader` class description for more details.
2023-04-09 12:50:26 +00:00
class LoadJob : private boost : : noncopyable
{
public :
template < class Func >
2023-04-22 17:28:53 +00:00
LoadJob ( LoadJobSet & & dependencies_ , String name_ , ssize_t priority_ , Func & & func_ )
2023-04-09 12:50:26 +00:00
: dependencies ( std : : move ( dependencies_ ) )
2023-04-19 11:35:58 +00:00
, name ( std : : move ( name_ ) )
2023-04-09 12:50:26 +00:00
, func ( std : : forward < Func > ( func_ ) )
2023-04-22 17:28:53 +00:00
, load_priority ( priority_ )
{ }
template < class Func >
LoadJob ( LoadJobSet & & dependencies_ , String name_ , Func & & func_ )
: LoadJob ( std : : move ( dependencies_ ) , std : : move ( name_ ) , 0 , std : : forward < Func > ( func_ ) )
2023-04-09 12:50:26 +00:00
{ }
2023-04-16 18:17:11 +00:00
// Current job status.
LoadStatus status ( ) const ;
std : : exception_ptr exception ( ) const ;
2023-04-09 12:50:26 +00:00
2023-04-22 17:28:53 +00:00
// Returns current value of a priority of the job. May differ from initial priority.
2023-04-16 18:17:11 +00:00
ssize_t priority ( ) const ;
2023-04-14 17:16:30 +00:00
2023-04-16 18:17:11 +00:00
// Sync wait for a pending job to be finished: OK, FAILED or CANCELED status.
// Throws if job is FAILED or CANCELED. Returns or throws immediately on non-pending job.
void wait ( ) const ;
2023-04-14 00:04:31 +00:00
2023-04-16 18:17:11 +00:00
// Wait for a job to reach any non PENDING status.
void waitNoThrow ( ) const noexcept ;
// Returns number of threads blocked by `wait()` or `waitNoThrow()` calls.
2023-04-19 11:46:56 +00:00
size_t waitersCount ( ) const ;
2023-04-14 00:04:31 +00:00
2023-04-16 18:17:11 +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 ;
private :
friend class AsyncLoader ;
2023-04-16 18:17:11 +00:00
void ok ( ) ;
void failed ( const std : : exception_ptr & ptr ) ;
void canceled ( const std : : exception_ptr & ptr ) ;
2023-04-17 10:11:59 +00:00
void finish ( ) ;
2023-04-09 12:50:26 +00:00
2023-04-27 14:49:35 +00:00
void scheduled ( ) ;
void execute ( const LoadJobPtr & self ) ;
2023-04-14 13:14:31 +00:00
std : : function < void ( const LoadJobPtr & self ) > func ;
2023-04-22 17:28:53 +00:00
std : : atomic < ssize_t > load_priority ;
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-27 14:49:35 +00:00
UInt64 scheduled_ns = 0 ;
UInt64 started_ns = 0 ;
UInt64 finished_ns = 0 ;
2023-04-09 12:50:26 +00:00
} ;
template < class Func >
2023-04-19 11:48:38 +00:00
LoadJobPtr makeLoadJob ( LoadJobSet & & dependencies , String name , Func & & func )
2023-04-09 12:50:26 +00:00
{
2023-04-19 11:48:38 +00:00
return std : : make_shared < LoadJob > ( std : : move ( dependencies ) , std : : move ( name ) , std : : forward < Func > ( func ) ) ;
2023-04-09 12:50:26 +00:00
}
2023-04-22 17:28:53 +00:00
template < class Func >
LoadJobPtr makeLoadJob ( LoadJobSet & & dependencies , String name , ssize_t priority , Func & & func )
{
return std : : make_shared < LoadJob > ( std : : move ( dependencies ) , std : : move ( name ) , priority , std : : forward < Func > ( func ) ) ;
}
// Represents a logically connected set of LoadJobs required to achieve some goals (final LoadJob in the set).
class LoadTask : private boost : : noncopyable
{
public :
2023-04-27 11:55:26 +00:00
LoadTask ( AsyncLoader & loader_ , LoadJobSet & & jobs_ , LoadJobSet & & goal_jobs_ = { } ) ;
2023-04-22 17:28:53 +00:00
~ LoadTask ( ) ;
// Merge all jobs from other task into this task.
2023-04-26 18:38:28 +00:00
void merge ( const LoadTaskPtr & task ) ;
2023-04-22 17:28:53 +00:00
// Schedule all jobs with AsyncLoader.
void schedule ( ) ;
// Remove all jobs of this task from AsyncLoader.
void remove ( ) ;
// Do not track jobs in this task.
// WARNING: Jobs will never be removed() and are going to be stored as finished jobs until ~AsyncLoader().
void detach ( ) ;
// Return the final jobs in this tasks. This jobs subset should be used as `dependencies` for dependent jobs or tasks.
const LoadJobSet & goals ( ) const { return goal_jobs . empty ( ) ? jobs : goal_jobs ; }
private :
friend class AsyncLoader ;
AsyncLoader & loader ;
LoadJobSet jobs ;
LoadJobSet goal_jobs ;
} ;
inline LoadTaskPtr makeLoadTask ( AsyncLoader & loader , LoadJobSet & & jobs , LoadJobSet & & goals = { } )
{
return std : : make_shared < LoadTask > ( loader , std : : move ( jobs ) , std : : move ( goals ) ) ;
}
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) {
2023-04-16 18:18:50 +00:00
// LOG_TRACE(log, "Executing load job '{}' with priority '{}'", self->name, self->priority());
2023-04-16 12:07:46 +00:00
// };
// auto job1 = makeLoadJob({}, "job1", job_func);
// auto job2 = makeLoadJob({ job1 }, "job2", job_func);
// auto job3 = makeLoadJob({ job1 }, "job3", job_func);
2023-04-22 17:28:53 +00:00
// auto task = makeLoadTask(async_loader, { job1, job2, job3 });
// task.schedule();
2023-04-17 09:47:16 +00:00
// Here we have created and scheduled a task consisting of three jobs. Job1 has no dependencies and is run first.
// Job2 and job3 depend on job1 and are run only after job1 completion. Another thread may prioritize a job and wait for it:
2023-04-22 17:28:53 +00:00
// async_loader->prioritize(job3, /* priority = */ 1); // higher priority jobs are run first, default priority is zero.
2023-04-16 13:09:55 +00:00
// job3->wait(); // blocks until job completion or cancellation and rethrow an exception (if any)
2023-04-16 12:07:46 +00:00
//
// AsyncLoader tracks state of all scheduled jobs. Job lifecycle is the following:
2023-04-22 17:28:53 +00:00
// 1) Job is constructed with PENDING status and initial priority. The job is placed into a task.
// 2) The task is scheduled with all its jobs. A scheduled job may be ready (i.e. have all its dependencies finished) or blocked.
// 3a) When all dependencies are successfully executed, the job became ready. A ready job is enqueued into the ready queue.
// 3b) If at least one of the job dependencies is failed or canceled, then this job is canceled (with all it's dependent jobs as well).
// On cancellation an ASYNC_LOAD_CANCELED exception is generated and saved inside LoadJob object. The job status is changed to CANCELED.
// Exception is rethrown by any existing or new `wait()` call. The job is moved to the set of the finished jobs.
// 4) The scheduled pending ready job starts execution by a worker. The job is dequeuedCallback `job_func` is called.
// Status of an executing job is PENDING. And it is still considered as a scheduled job by AsyncLoader.
2023-04-16 12:07:46 +00:00
// Note that `job_func` of a CANCELED job is never executed.
2023-04-22 17:28:53 +00:00
// 5a) On successful execution the 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 an ASYNC_LOAD_FAILED exception and save inside LoadJob.
// The job status is changed to FAILED. All the dependent jobs are canceled. The exception is rethrown from all existing and new `wait()` calls.
// 6) The job is no longer considered as scheduled and is instead moved to the finished jobs set. This is just for introspection of the finished jobs.
// 7) The task containing this job is destructed or `remove()` is explicitly called. The job is removed from the finished job set.
// 8) The job is destructed.
2023-04-16 12:07:46 +00:00
//
// 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).
2023-04-16 18:18:50 +00:00
// Note that to avoid priority inversion `job_func` should use `self->priority()` to schedule new jobs in AsyncLoader or any other pool.
2023-04-16 12:07:46 +00:00
// 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-22 17:28:53 +00:00
// The only forbidden thing is a dependency on job, that was not scheduled in AsyncLoader yet: all the dependent jobs are immediately canceled.
2023-04-09 12:50:26 +00:00
class AsyncLoader : private boost : : noncopyable
{
private :
2023-04-16 18:17:11 +00:00
// Key of a pending job in the 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
}
} ;
2023-04-16 18:17:11 +00:00
// Scheduling information for a pending job.
2023-04-11 00:08:55 +00:00
struct Info
{
2023-04-16 18:17:11 +00:00
ssize_t initial_priority = 0 ; // Initial priority passed into schedule().
ssize_t priority = 0 ; // Elevated priority, due to priority inheritance or prioritize().
size_t dependencies_left = 0 ; // Current number of dependencies on pending jobs.
UInt64 ready_seqno = 0 ; // Zero means that job is not in ready queue.
LoadJobSet dependent_jobs ; // Set of jobs dependent on this job.
2023-04-09 12:50:26 +00:00
2023-04-17 10:03:54 +00:00
// Three independent states of a non-finished job.
2023-04-14 14:00:54 +00:00
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 ;
2023-04-16 18:17:11 +00:00
AsyncLoader ( Metric metric_threads , Metric metric_active_threads , size_t max_threads_ , bool log_failures_ ) ;
2023-04-09 12:50:26 +00:00
2023-04-22 17:28:53 +00:00
// WARNING: all LoadTask instances returned by `schedule()` should be destructed before AsyncLoader.
2023-04-16 18:17:11 +00:00
~ AsyncLoader ( ) ;
2023-04-13 22:48:46 +00:00
2023-04-16 18:17:11 +00:00
// Start workers to execute scheduled load jobs.
void start ( ) ;
2023-04-09 12:50:26 +00:00
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.
2023-04-16 18:17:11 +00:00
void wait ( ) ;
2023-04-10 23:14:20 +00:00
// Wait for currently executing jobs to finish, but do not run any other pending jobs.
// Not finished jobs are left in pending state:
2023-04-17 10:03:54 +00:00
// - they can be executed by calling start() again;
2023-04-10 23:14:20 +00:00
// - or canceled using ~Task() or remove() later.
2023-04-16 18:17:11 +00:00
void stop ( ) ;
2023-04-09 12:50:26 +00:00
2023-04-22 17:28:53 +00:00
// Schedule all jobs of given task.
2023-04-16 18:17:11 +00:00
// Higher priority jobs (with greater `priority` value) are executed earlier.
// All dependencies of a scheduled job inherit its priority if it is higher. This way higher priority job
// never wait for (blocked by) lower priority jobs (no priority inversion).
2023-04-22 17:28:53 +00:00
// Task destructor ensures that all the `jobs` are finished (OK, FAILED or CANCELED)
2023-04-17 10:03:54 +00:00
// and are removed from AsyncLoader, so it is thread-safe to destroy them.
2023-04-22 17:28:53 +00:00
void schedule ( LoadTask & task ) ;
void schedule ( const LoadTaskPtr & task ) ;
// Schedule all tasks atomically. To ensure only highest priority jobs among all tasks are run first.
void schedule ( const std : : vector < LoadTaskPtr > & tasks ) ;
2023-04-09 12:50:26 +00:00
2023-04-17 10:03:54 +00:00
// Increase priority of a job and all its dependencies recursively.
2023-04-16 18:17:11 +00:00
void prioritize ( const LoadJobPtr & job , ssize_t new_priority ) ;
2023-04-09 12:50:26 +00:00
2023-04-16 18:17:11 +00:00
// Remove finished jobs, cancel scheduled jobs, wait for executing jobs to finish and remove them.
void remove ( const LoadJobSet & jobs ) ;
2023-04-09 12:50:26 +00:00
2023-04-16 18:17:11 +00:00
// Increase or decrease maximum number of simultaneously executing jobs.
void setMaxThreads ( size_t value ) ;
2023-04-14 12:57:51 +00:00
2023-04-16 18:17:11 +00:00
size_t getMaxThreads ( ) const ;
size_t getScheduledJobCount ( ) const ;
2023-04-14 20:49:56 +00:00
2023-04-09 12:50:26 +00:00
private :
2023-04-16 18:17:11 +00:00
void checkCycle ( const LoadJobSet & jobs , std : : unique_lock < std : : mutex > & lock ) ;
String checkCycleImpl ( const LoadJobPtr & job , LoadJobSet & left , LoadJobSet & visited , std : : unique_lock < std : : mutex > & lock ) ;
void finish ( std : : unique_lock < std : : mutex > & lock , const LoadJobPtr & job , LoadStatus status , std : : exception_ptr exception_from_job = { } ) ;
2023-04-22 17:28:53 +00:00
void scheduleImpl ( const LoadJobSet & jobs ) ;
2023-04-16 18:17:11 +00:00
void prioritize ( const LoadJobPtr & job , ssize_t new_priority , std : : unique_lock < std : : mutex > & lock ) ;
void enqueue ( Info & info , const LoadJobPtr & job , std : : unique_lock < std : : mutex > & lock ) ;
void spawn ( std : : unique_lock < std : : mutex > & ) ;
void worker ( ) ;
2023-04-09 12:50:26 +00:00
2023-04-27 14:49:35 +00:00
// Logging
2023-04-16 18:17:11 +00:00
const bool log_failures ; // Worker should log all exceptions caught from job functions.
2023-04-27 14:49:35 +00:00
Poco : : Logger * log ;
UInt64 busy_period_started_ns ;
AtomicStopwatch stopwatch ;
size_t old_jobs = 0 ; // Number of jobs that were finished in previous busy period (for correct progress indication)
2023-04-09 12:50:26 +00:00
2023-04-16 18:17:11 +00:00
mutable std : : mutex mutex ; // Guards all the fields below.
2023-04-09 12:50:26 +00:00
bool is_running = false ;
2023-04-16 18:17:11 +00:00
// Full set of scheduled pending jobs along with scheduling info.
2023-04-09 12:50:26 +00:00
std : : unordered_map < LoadJobPtr , Info > scheduled_jobs ;
2023-04-16 18:17:11 +00:00
// Subset of scheduled pending non-blocked jobs (waiting for a worker to be executed).
// Represent a queue of jobs in order of decreasing priority and FIFO for jobs with equal priorities.
2023-04-09 12:50:26 +00:00
std : : map < ReadyKey , LoadJobPtr > ready_queue ;
2023-04-16 18:17:11 +00:00
// Set of finished jobs (for introspection only, until jobs are removed).
2023-04-09 12:50:26 +00:00
LoadJobSet finished_jobs ;
2023-04-16 18:17:11 +00:00
// Increasing counter for `ReadyKey` assignment (to preserve FIFO order of the jobs with equal priorities).
2023-04-09 12:50:26 +00:00
UInt64 last_ready_seqno = 0 ;
2023-04-16 18:17:11 +00:00
// For executing jobs. Note that we avoid using an internal queue of the pool to be able to prioritize jobs.
2023-04-09 12:50:26 +00:00
size_t max_threads ;
size_t workers = 0 ;
ThreadPool pool ;
} ;
}