2014-11-30 18:22:57 +00:00
# pragma once
# include <list>
2015-04-23 00:20:41 +00:00
# include <queue>
2014-11-30 18:22:57 +00:00
# include <atomic>
# include <thread>
# include <mutex>
2015-09-29 19:19:54 +00:00
# include <common/logger_useful.h>
2014-11-30 18:22:57 +00:00
2019-05-17 14:34:25 +00:00
# include <DataStreams/IBlockStream_fwd.h>
2017-04-01 09:19:00 +00:00
# include <Common/setThreadName.h>
# include <Common/CurrentMetrics.h>
2018-05-29 18:14:31 +00:00
# include <Common/CurrentThread.h>
2019-01-14 19:22:09 +00:00
# include <Common/ThreadPool.h>
2014-11-30 18:22:57 +00:00
2016-10-24 04:06:27 +00:00
/** Allows to process multiple block input streams (sources) in parallel, using specified number of threads.
* Reads ( pulls ) blocks from any available source and passes it to specified handler .
2014-11-30 18:22:57 +00:00
*
2017-12-04 20:29:13 +00:00
* Before any reading , calls " readPrefix " method of sources in parallel .
*
* ( As an example , " readPrefix " can prepare connections to remote servers ,
* and we want this work to be executed in parallel for different sources )
*
2016-10-24 04:06:27 +00:00
* Implemented in following way :
* - there are multiple input sources to read blocks from ;
* - there are multiple threads , that could simultaneously read blocks from different sources ;
* - " available " sources ( that are not read in any thread right now ) are put in queue of sources ;
* - when thread take a source to read from , it removes source from queue of sources ,
* then read block from source and then put source back to queue of available sources .
2014-11-30 18:22:57 +00:00
*/
2016-10-24 04:06:27 +00:00
namespace CurrentMetrics
{
2017-04-01 07:20:54 +00:00
extern const Metric QueryThread ;
2016-10-24 04:06:27 +00:00
}
2014-11-30 18:22:57 +00:00
namespace DB
{
2017-05-13 22:19:04 +00:00
/// Example of the handler.
2014-11-30 18:22:57 +00:00
struct ParallelInputsHandler
{
2017-05-13 22:19:04 +00:00
/// Processing the data block.
2017-12-01 19:34:51 +00:00
void onBlock ( Block & /*block*/ , size_t /*thread_num*/ ) { }
2014-11-30 18:22:57 +00:00
2017-05-13 22:19:04 +00:00
/// Called for each thread, when the thread has nothing else to do.
/// Due to the fact that part of the sources has run out, and now there are fewer sources left than streams.
/// Called if the `onException` method does not throw an exception; is called before the `onFinish` method.
2017-12-01 19:34:51 +00:00
void onFinishThread ( size_t /*thread_num*/ ) { }
2015-12-01 21:20:14 +00:00
2017-05-13 22:19:04 +00:00
/// Blocks are over. Due to the fact that all sources ran out or because of the cancellation of work.
/// This method is always called exactly once, at the end of the work, if the `onException` method does not throw an exception.
2017-04-01 07:20:54 +00:00
void onFinish ( ) { }
2014-11-30 18:22:57 +00:00
2017-05-13 22:19:04 +00:00
/// Exception handling. It is reasonable to call the ParallelInputsProcessor::cancel method in this method, and also pass the exception to the main thread.
2017-12-01 19:34:51 +00:00
void onException ( std : : exception_ptr & /*exception*/ , size_t /*thread_num*/ ) { }
2014-11-30 18:22:57 +00:00
} ;
2018-11-28 14:33:40 +00:00
template < typename Handler >
2014-11-30 18:22:57 +00:00
class ParallelInputsProcessor
{
public :
2017-05-13 22:19:04 +00:00
/** additional_input_at_end - if not nullptr,
* then the blocks from this source will start to be processed only after all other sources are processed .
* This is done in the main thread .
2017-04-01 07:20:54 +00:00
*
2017-05-13 22:19:04 +00:00
* Intended for implementation of FULL and RIGHT JOIN
* - where you must first make JOIN in parallel , while noting which keys are not found ,
* and only after the completion of this work , create blocks of keys that are not found .
2017-04-01 07:20:54 +00:00
*/
2017-09-08 02:29:47 +00:00
ParallelInputsProcessor ( const BlockInputStreams & inputs_ , const BlockInputStreamPtr & additional_input_at_end_ , size_t max_threads_ , Handler & handler_ )
2017-04-01 07:20:54 +00:00
: inputs ( inputs_ ) , additional_input_at_end ( additional_input_at_end_ ) , max_threads ( std : : min ( inputs_ . size ( ) , max_threads_ ) ) , handler ( handler_ )
{
for ( size_t i = 0 ; i < inputs_ . size ( ) ; + + i )
2017-11-28 08:52:00 +00:00
unprepared_inputs . emplace ( inputs_ [ i ] , i ) ;
2017-04-01 07:20:54 +00:00
}
~ ParallelInputsProcessor ( )
{
try
{
wait ( ) ;
}
catch ( . . . )
{
tryLogCurrentException ( __PRETTY_FUNCTION__ ) ;
}
}
2017-05-13 22:19:04 +00:00
/// Start background threads, start work.
2017-04-01 07:20:54 +00:00
void process ( )
{
active_threads = max_threads ;
threads . reserve ( max_threads ) ;
2018-11-22 17:29:49 +00:00
try
{
for ( size_t i = 0 ; i < max_threads ; + + i )
2019-07-01 00:15:14 +00:00
threads . emplace_back ( & ParallelInputsProcessor : : thread , this , CurrentThread : : getGroup ( ) , i ) ;
2018-11-22 17:29:49 +00:00
}
catch ( . . . )
{
cancel ( false ) ;
wait ( ) ;
if ( active_threads )
{
active_threads = 0 ;
/// handler.onFinish() is supposed to be called from one of the threads when the number of
/// finished threads reaches max_threads. But since we weren't able to launch all threads,
/// we have to call onFinish() manually here.
handler . onFinish ( ) ;
}
throw ;
}
2017-04-01 07:20:54 +00:00
}
2017-05-13 22:19:04 +00:00
/// Ask all sources to stop earlier than they run out.
2018-03-05 21:09:39 +00:00
void cancel ( bool kill )
2017-04-01 07:20:54 +00:00
{
finish = true ;
for ( auto & input : inputs )
{
2019-01-23 14:48:50 +00:00
try
2017-04-01 07:20:54 +00:00
{
2019-01-23 14:48:50 +00:00
input - > cancel ( kill ) ;
}
catch ( . . . )
{
/** If you can not ask one or more sources to stop.
* ( for example , the connection is broken for distributed query processing )
* - then do not care .
*/
LOG_ERROR ( log , " Exception while cancelling " < < input - > getName ( ) ) ;
2017-04-01 07:20:54 +00:00
}
}
}
2017-05-13 22:19:04 +00:00
/// Wait until all threads are finished, before the destructor.
2017-04-01 07:20:54 +00:00
void wait ( )
{
if ( joined_threads )
return ;
for ( auto & thread : threads )
thread . join ( ) ;
threads . clear ( ) ;
joined_threads = true ;
}
size_t getNumActiveThreads ( ) const
{
return active_threads ;
}
2014-11-30 18:22:57 +00:00
private :
2017-05-13 22:19:04 +00:00
/// Single source data
2017-04-01 07:20:54 +00:00
struct InputData
{
BlockInputStreamPtr in ;
2019-04-22 16:07:09 +00:00
size_t i = 0 ; /// The source number (for debugging).
2017-04-01 07:20:54 +00:00
InputData ( ) { }
2017-09-08 02:29:47 +00:00
InputData ( const BlockInputStreamPtr & in_ , size_t i_ ) : in ( in_ ) , i ( i_ ) { }
2017-04-01 07:20:54 +00:00
} ;
2018-11-28 14:33:40 +00:00
void publishPayload ( Block & block , size_t thread_num )
2017-04-01 07:20:54 +00:00
{
2018-11-28 14:33:40 +00:00
handler . onBlock ( block , thread_num ) ;
2017-04-01 07:20:54 +00:00
}
2018-06-19 20:30:35 +00:00
void thread ( ThreadGroupStatusPtr thread_group , size_t thread_num )
2017-04-01 07:20:54 +00:00
{
std : : exception_ptr exception ;
CurrentMetrics : : Increment metric_increment { CurrentMetrics : : QueryThread } ;
try
{
2018-06-19 20:30:35 +00:00
setThreadName ( " ParalInputsProc " ) ;
2019-01-29 17:24:54 +00:00
if ( thread_group )
CurrentThread : : attachTo ( thread_group ) ;
2018-06-19 20:30:35 +00:00
2017-12-04 20:29:13 +00:00
while ( ! finish )
{
2017-11-28 08:52:00 +00:00
InputData unprepared_input ;
{
2019-01-02 06:44:36 +00:00
std : : lock_guard lock ( unprepared_inputs_mutex ) ;
2017-11-28 08:52:00 +00:00
if ( unprepared_inputs . empty ( ) )
break ;
unprepared_input = unprepared_inputs . front ( ) ;
unprepared_inputs . pop ( ) ;
}
unprepared_input . in - > readPrefix ( ) ;
{
2019-01-02 06:44:36 +00:00
std : : lock_guard lock ( available_inputs_mutex ) ;
2017-11-28 08:52:00 +00:00
available_inputs . push ( unprepared_input ) ;
}
}
2017-12-04 20:29:13 +00:00
2017-04-01 07:20:54 +00:00
loop ( thread_num ) ;
}
catch ( . . . )
{
exception = std : : current_exception ( ) ;
}
if ( exception )
{
handler . onException ( exception , thread_num ) ;
}
handler . onFinishThread ( thread_num ) ;
2017-05-13 22:19:04 +00:00
/// The last thread on the output indicates that there is no more data.
2017-04-01 07:20:54 +00:00
if ( 0 = = - - active_threads )
{
2017-05-13 22:19:04 +00:00
/// And then it processes an additional source, if there is one.
2017-04-01 07:20:54 +00:00
if ( additional_input_at_end )
{
try
{
2017-11-28 08:52:00 +00:00
additional_input_at_end - > readPrefix ( ) ;
2017-04-01 07:20:54 +00:00
while ( Block block = additional_input_at_end - > read ( ) )
2018-11-28 14:33:40 +00:00
publishPayload ( block , thread_num ) ;
2017-04-01 07:20:54 +00:00
}
catch ( . . . )
{
exception = std : : current_exception ( ) ;
}
if ( exception )
{
handler . onException ( exception , thread_num ) ;
}
}
2017-11-22 02:42:53 +00:00
handler . onFinish ( ) ; /// TODO If in `onFinish` or `onFinishThread` there is an exception, then std::terminate is called.
2017-04-01 07:20:54 +00:00
}
}
void loop ( size_t thread_num )
{
2017-05-13 22:19:04 +00:00
while ( ! finish ) /// You may need to stop work earlier than all sources run out.
2017-04-01 07:20:54 +00:00
{
InputData input ;
2017-05-13 22:19:04 +00:00
/// Select the next source.
2017-04-01 07:20:54 +00:00
{
2019-01-02 06:44:36 +00:00
std : : lock_guard lock ( available_inputs_mutex ) ;
2017-04-01 07:20:54 +00:00
2017-05-13 22:19:04 +00:00
/// If there are no free sources, then this thread is no longer needed. (But other threads can work with their sources.)
2017-04-01 07:20:54 +00:00
if ( available_inputs . empty ( ) )
break ;
input = available_inputs . front ( ) ;
2017-05-13 22:19:04 +00:00
/// We remove the source from the queue of available sources.
2017-04-01 07:20:54 +00:00
available_inputs . pop ( ) ;
}
2017-05-13 22:19:04 +00:00
/// The main work.
2017-04-01 07:20:54 +00:00
Block block = input . in - > read ( ) ;
{
if ( finish )
break ;
2017-05-13 22:19:04 +00:00
/// If this source is not run out yet, then put the resulting block in the ready queue.
2017-04-01 07:20:54 +00:00
{
2019-01-02 06:44:36 +00:00
std : : lock_guard lock ( available_inputs_mutex ) ;
2017-04-01 07:20:54 +00:00
if ( block )
{
available_inputs . push ( input ) ;
}
else
{
if ( available_inputs . empty ( ) )
break ;
}
}
if ( finish )
break ;
if ( block )
2018-11-28 14:33:40 +00:00
publishPayload ( block , thread_num ) ;
2017-04-01 07:20:54 +00:00
}
}
}
BlockInputStreams inputs ;
BlockInputStreamPtr additional_input_at_end ;
unsigned max_threads ;
Handler & handler ;
2019-01-14 19:22:09 +00:00
/// Threads.
using ThreadsData = std : : vector < ThreadFromGlobalPool > ;
2017-04-01 07:20:54 +00:00
ThreadsData threads ;
2017-05-13 22:19:04 +00:00
/** A set of available sources that are not currently processed by any thread.
* Each thread takes one source from this set , takes a block out of the source ( at this moment the source does the calculations )
* and ( if the source is not run out ) , puts it back into the set of available sources .
2017-04-01 07:20:54 +00:00
*
2017-05-13 22:19:04 +00:00
* The question arises what is better to use :
* - the queue ( just processed source will be processed the next time later than the rest )
* - stack ( just processed source will be processed as soon as possible ) .
2017-04-01 07:20:54 +00:00
*
2017-05-13 22:19:04 +00:00
* The stack is better than the queue when you need to do work on reading one source more consequentially ,
* and theoretically , this allows you to achieve more consequent / consistent reads from the disk .
2017-04-01 07:20:54 +00:00
*
2017-05-13 22:19:04 +00:00
* But when using the stack , there is a problem with distributed query processing :
* data is read only from a part of the servers , and on the other servers
* a timeout occurs during send , and the request processing ends with an exception .
2017-04-01 07:20:54 +00:00
*
2017-05-13 22:19:04 +00:00
* Therefore , a queue is used . This can be improved in the future .
2017-04-01 07:20:54 +00:00
*/
using AvailableInputs = std : : queue < InputData > ;
AvailableInputs available_inputs ;
2017-12-04 20:29:13 +00:00
/** For parallel preparing (readPrefix) child streams.
* First , streams are located here .
* After a stream was prepared , it is moved to " available_inputs " for reading .
*/
2017-11-28 08:52:00 +00:00
using UnpreparedInputs = std : : queue < InputData > ;
UnpreparedInputs unprepared_inputs ;
2017-05-13 22:19:04 +00:00
/// For operations with available_inputs.
2017-04-01 07:20:54 +00:00
std : : mutex available_inputs_mutex ;
2017-11-28 08:52:00 +00:00
/// For operations with unprepared_inputs.
std : : mutex unprepared_inputs_mutex ;
2017-05-13 22:19:04 +00:00
/// How many sources ran out.
2017-04-01 07:20:54 +00:00
std : : atomic < size_t > active_threads { 0 } ;
2017-05-13 22:19:04 +00:00
/// Finish the threads work (before the sources run out).
2017-04-01 07:20:54 +00:00
std : : atomic < bool > finish { false } ;
2017-05-13 22:19:04 +00:00
/// Wait for the completion of all threads.
2017-04-01 07:20:54 +00:00
std : : atomic < bool > joined_threads { false } ;
Logger * log = & Logger : : get ( " ParallelInputsProcessor " ) ;
2014-11-30 18:22:57 +00:00
} ;
}