mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Merge pull request #11004 from ClickHouse/remove-experimental-use-processors-flag-2
Remove allow_processors flag from executeQuery()
This commit is contained in:
commit
f7456f8320
@ -28,7 +28,7 @@
|
|||||||
#include <Compression/CompressionFactory.h>
|
#include <Compression/CompressionFactory.h>
|
||||||
#include <common/logger_useful.h>
|
#include <common/logger_useful.h>
|
||||||
|
|
||||||
#include <Processors/Executors/PullingPipelineExecutor.h>
|
#include <Processors/Executors/PullingAsyncPipelineExecutor.h>
|
||||||
|
|
||||||
#include "TCPHandler.h"
|
#include "TCPHandler.h"
|
||||||
|
|
||||||
@ -566,7 +566,7 @@ void TCPHandler::processOrdinaryQueryWithProcessors()
|
|||||||
}
|
}
|
||||||
|
|
||||||
{
|
{
|
||||||
PullingPipelineExecutor executor(pipeline);
|
PullingAsyncPipelineExecutor executor(pipeline);
|
||||||
CurrentMetrics::Increment query_thread_metric_increment{CurrentMetrics::QueryThread};
|
CurrentMetrics::Increment query_thread_metric_increment{CurrentMetrics::QueryThread};
|
||||||
|
|
||||||
Block block;
|
Block block;
|
||||||
|
@ -1,9 +1,31 @@
|
|||||||
#include <DataStreams/BlockIO.h>
|
#include <DataStreams/BlockIO.h>
|
||||||
#include <Interpreters/ProcessList.h>
|
#include <Interpreters/ProcessList.h>
|
||||||
|
#include <Processors/Executors/PipelineExecutingBlockInputStream.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int LOGICAL_ERROR;
|
||||||
|
}
|
||||||
|
|
||||||
|
BlockInputStreamPtr BlockIO::getInputStream()
|
||||||
|
{
|
||||||
|
if (out)
|
||||||
|
throw Exception("Cannot get input stream from BlockIO because output stream is not empty",
|
||||||
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
if (in)
|
||||||
|
return in;
|
||||||
|
|
||||||
|
if (pipeline.initialized())
|
||||||
|
return std::make_shared<PipelineExecutingBlockInputStream>(std::move(pipeline));
|
||||||
|
|
||||||
|
throw Exception("Cannot get input stream from BlockIO because query pipeline was not initialized",
|
||||||
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
|
||||||
void BlockIO::reset()
|
void BlockIO::reset()
|
||||||
{
|
{
|
||||||
/** process_list_entry should be destroyed after in, after out and after pipeline,
|
/** process_list_entry should be destroyed after in, after out and after pipeline,
|
||||||
|
@ -50,6 +50,9 @@ struct BlockIO
|
|||||||
exception_callback();
|
exception_callback();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Returns in or converts pipeline to stream. Throws if out is not empty.
|
||||||
|
BlockInputStreamPtr getInputStream();
|
||||||
|
|
||||||
private:
|
private:
|
||||||
void reset();
|
void reset();
|
||||||
};
|
};
|
||||||
|
@ -131,10 +131,10 @@ BlockInputStreamPtr ClickHouseDictionarySource::loadAll()
|
|||||||
*/
|
*/
|
||||||
if (is_local)
|
if (is_local)
|
||||||
{
|
{
|
||||||
BlockIO res = executeQuery(load_all_query, context, true, QueryProcessingStage::Complete, false, false);
|
auto stream = executeQuery(load_all_query, context, true).getInputStream();
|
||||||
/// FIXME res.in may implicitly use some objects owned be res, but them will be destructed after return
|
/// FIXME res.in may implicitly use some objects owned be res, but them will be destructed after return
|
||||||
res.in = std::make_shared<ConvertingBlockInputStream>(res.in, sample_block, ConvertingBlockInputStream::MatchColumnsMode::Position);
|
stream = std::make_shared<ConvertingBlockInputStream>(stream, sample_block, ConvertingBlockInputStream::MatchColumnsMode::Position);
|
||||||
return res.in;
|
return stream;
|
||||||
}
|
}
|
||||||
return std::make_shared<RemoteBlockInputStream>(pool, load_all_query, sample_block, context);
|
return std::make_shared<RemoteBlockInputStream>(pool, load_all_query, sample_block, context);
|
||||||
}
|
}
|
||||||
@ -144,9 +144,9 @@ BlockInputStreamPtr ClickHouseDictionarySource::loadUpdatedAll()
|
|||||||
std::string load_update_query = getUpdateFieldAndDate();
|
std::string load_update_query = getUpdateFieldAndDate();
|
||||||
if (is_local)
|
if (is_local)
|
||||||
{
|
{
|
||||||
auto res = executeQuery(load_update_query, context, true, QueryProcessingStage::Complete, false, false);
|
auto stream = executeQuery(load_update_query, context, true).getInputStream();
|
||||||
res.in = std::make_shared<ConvertingBlockInputStream>(res.in, sample_block, ConvertingBlockInputStream::MatchColumnsMode::Position);
|
stream = std::make_shared<ConvertingBlockInputStream>(stream, sample_block, ConvertingBlockInputStream::MatchColumnsMode::Position);
|
||||||
return res.in;
|
return stream;
|
||||||
}
|
}
|
||||||
return std::make_shared<RemoteBlockInputStream>(pool, load_update_query, sample_block, context);
|
return std::make_shared<RemoteBlockInputStream>(pool, load_update_query, sample_block, context);
|
||||||
}
|
}
|
||||||
@ -191,10 +191,10 @@ BlockInputStreamPtr ClickHouseDictionarySource::createStreamForSelectiveLoad(con
|
|||||||
{
|
{
|
||||||
if (is_local)
|
if (is_local)
|
||||||
{
|
{
|
||||||
auto res = executeQuery(query, context, true, QueryProcessingStage::Complete, false, false);
|
auto res = executeQuery(query, context, true).getInputStream();
|
||||||
res.in = std::make_shared<ConvertingBlockInputStream>(
|
res = std::make_shared<ConvertingBlockInputStream>(
|
||||||
res.in, sample_block, ConvertingBlockInputStream::MatchColumnsMode::Position);
|
res, sample_block, ConvertingBlockInputStream::MatchColumnsMode::Position);
|
||||||
return res.in;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
return std::make_shared<RemoteBlockInputStream>(pool, query, sample_block, context);
|
return std::make_shared<RemoteBlockInputStream>(pool, query, sample_block, context);
|
||||||
@ -206,8 +206,7 @@ std::string ClickHouseDictionarySource::doInvalidateQuery(const std::string & re
|
|||||||
if (is_local)
|
if (is_local)
|
||||||
{
|
{
|
||||||
Context query_context = context;
|
Context query_context = context;
|
||||||
auto input_block = executeQuery(request, query_context, true,
|
auto input_block = executeQuery(request, query_context, true).getInputStream();
|
||||||
QueryProcessingStage::Complete, false, false).in;
|
|
||||||
return readInvalidateQuery(*input_block);
|
return readInvalidateQuery(*input_block);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
|
@ -302,10 +302,11 @@ Block InterpreterKillQueryQuery::getSelectResult(const String & columns, const S
|
|||||||
if (where_expression)
|
if (where_expression)
|
||||||
select_query += " WHERE " + queryToString(where_expression);
|
select_query += " WHERE " + queryToString(where_expression);
|
||||||
|
|
||||||
BlockIO block_io = executeQuery(select_query, context.getGlobalContext(), true, QueryProcessingStage::Complete, false, false);
|
BlockIO block_io = executeQuery(select_query, context.getGlobalContext(), true);
|
||||||
Block res = block_io.in->read();
|
auto stream = block_io.getInputStream();
|
||||||
|
Block res = stream->read();
|
||||||
|
|
||||||
if (res && block_io.in->read())
|
if (res && stream->read())
|
||||||
throw Exception("Expected one block from input stream", ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Expected one block from input stream", ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
return res;
|
return res;
|
||||||
|
@ -197,8 +197,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
|||||||
bool internal,
|
bool internal,
|
||||||
QueryProcessingStage::Enum stage,
|
QueryProcessingStage::Enum stage,
|
||||||
bool has_query_tail,
|
bool has_query_tail,
|
||||||
ReadBuffer * istr,
|
ReadBuffer * istr)
|
||||||
bool allow_processors)
|
|
||||||
{
|
{
|
||||||
time_t current_time = time(nullptr);
|
time_t current_time = time(nullptr);
|
||||||
|
|
||||||
@ -318,7 +317,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
|||||||
context.resetInputCallbacks();
|
context.resetInputCallbacks();
|
||||||
|
|
||||||
auto interpreter = InterpreterFactory::get(ast, context, stage);
|
auto interpreter = InterpreterFactory::get(ast, context, stage);
|
||||||
bool use_processors = allow_processors && interpreter->canExecuteWithProcessors();
|
bool use_processors = interpreter->canExecuteWithProcessors();
|
||||||
|
|
||||||
std::shared_ptr<const EnabledQuota> quota;
|
std::shared_ptr<const EnabledQuota> quota;
|
||||||
if (!interpreter->ignoreQuota())
|
if (!interpreter->ignoreQuota())
|
||||||
@ -581,13 +580,12 @@ BlockIO executeQuery(
|
|||||||
Context & context,
|
Context & context,
|
||||||
bool internal,
|
bool internal,
|
||||||
QueryProcessingStage::Enum stage,
|
QueryProcessingStage::Enum stage,
|
||||||
bool may_have_embedded_data,
|
bool may_have_embedded_data)
|
||||||
bool allow_processors)
|
|
||||||
{
|
{
|
||||||
ASTPtr ast;
|
ASTPtr ast;
|
||||||
BlockIO streams;
|
BlockIO streams;
|
||||||
std::tie(ast, streams) = executeQueryImpl(query.data(), query.data() + query.size(), context,
|
std::tie(ast, streams) = executeQueryImpl(query.data(), query.data() + query.size(), context,
|
||||||
internal, stage, !may_have_embedded_data, nullptr, allow_processors);
|
internal, stage, !may_have_embedded_data, nullptr);
|
||||||
|
|
||||||
if (const auto * ast_query_with_output = dynamic_cast<const ASTQueryWithOutput *>(ast.get()))
|
if (const auto * ast_query_with_output = dynamic_cast<const ASTQueryWithOutput *>(ast.get()))
|
||||||
{
|
{
|
||||||
@ -602,6 +600,22 @@ BlockIO executeQuery(
|
|||||||
return streams;
|
return streams;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
BlockIO executeQuery(
|
||||||
|
const String & query,
|
||||||
|
Context & context,
|
||||||
|
bool internal,
|
||||||
|
QueryProcessingStage::Enum stage,
|
||||||
|
bool may_have_embedded_data,
|
||||||
|
bool allow_processors)
|
||||||
|
{
|
||||||
|
BlockIO res = executeQuery(query, context, internal, stage, may_have_embedded_data);
|
||||||
|
|
||||||
|
if (!allow_processors && res.pipeline.initialized())
|
||||||
|
res.in = res.getInputStream();
|
||||||
|
|
||||||
|
return res;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
void executeQuery(
|
void executeQuery(
|
||||||
ReadBuffer & istr,
|
ReadBuffer & istr,
|
||||||
@ -648,7 +662,7 @@ void executeQuery(
|
|||||||
ASTPtr ast;
|
ASTPtr ast;
|
||||||
BlockIO streams;
|
BlockIO streams;
|
||||||
|
|
||||||
std::tie(ast, streams) = executeQueryImpl(begin, end, context, false, QueryProcessingStage::Complete, may_have_tail, &istr, true);
|
std::tie(ast, streams) = executeQueryImpl(begin, end, context, false, QueryProcessingStage::Complete, may_have_tail, &istr);
|
||||||
|
|
||||||
auto & pipeline = streams.pipeline;
|
auto & pipeline = streams.pipeline;
|
||||||
|
|
||||||
|
@ -38,16 +38,6 @@ void executeQuery(
|
|||||||
/// Correctly formatting the results (according to INTO OUTFILE and FORMAT sections)
|
/// Correctly formatting the results (according to INTO OUTFILE and FORMAT sections)
|
||||||
/// must be done separately.
|
/// must be done separately.
|
||||||
BlockIO executeQuery(
|
BlockIO executeQuery(
|
||||||
const String & query, /// Query text without INSERT data. The latter must be written to BlockIO::out.
|
|
||||||
Context & context, /// DB, tables, data types, storage engines, functions, aggregate functions...
|
|
||||||
bool internal = false, /// If true, this query is caused by another query and thus needn't be registered in the ProcessList.
|
|
||||||
QueryProcessingStage::Enum stage = QueryProcessingStage::Complete, /// To which stage the query must be executed.
|
|
||||||
bool may_have_embedded_data = false, /// If insert query may have embedded data
|
|
||||||
bool allow_processors = true /// If can use processors pipeline
|
|
||||||
);
|
|
||||||
|
|
||||||
|
|
||||||
QueryPipeline executeQueryWithProcessors(
|
|
||||||
const String & query, /// Query text without INSERT data. The latter must be written to BlockIO::out.
|
const String & query, /// Query text without INSERT data. The latter must be written to BlockIO::out.
|
||||||
Context & context, /// DB, tables, data types, storage engines, functions, aggregate functions...
|
Context & context, /// DB, tables, data types, storage engines, functions, aggregate functions...
|
||||||
bool internal = false, /// If true, this query is caused by another query and thus needn't be registered in the ProcessList.
|
bool internal = false, /// If true, this query is caused by another query and thus needn't be registered in the ProcessList.
|
||||||
@ -55,4 +45,14 @@ QueryPipeline executeQueryWithProcessors(
|
|||||||
bool may_have_embedded_data = false /// If insert query may have embedded data
|
bool may_have_embedded_data = false /// If insert query may have embedded data
|
||||||
);
|
);
|
||||||
|
|
||||||
|
/// Old interface with allow_processors flag. For compatibility.
|
||||||
|
BlockIO executeQuery(
|
||||||
|
const String & query,
|
||||||
|
Context & context,
|
||||||
|
bool internal,
|
||||||
|
QueryProcessingStage::Enum stage,
|
||||||
|
bool may_have_embedded_data,
|
||||||
|
bool allow_processors /// If can use processors pipeline
|
||||||
|
);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
131
src/Processors/Executors/PipelineExecutingBlockInputStream.cpp
Normal file
131
src/Processors/Executors/PipelineExecutingBlockInputStream.cpp
Normal file
@ -0,0 +1,131 @@
|
|||||||
|
#include <Processors/Executors/PipelineExecutingBlockInputStream.h>
|
||||||
|
#include <Processors/Executors/PullingAsyncPipelineExecutor.h>
|
||||||
|
#include <Processors/Executors/PullingPipelineExecutor.h>
|
||||||
|
#include <Processors/QueryPipeline.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int LOGICAL_ERROR;
|
||||||
|
}
|
||||||
|
|
||||||
|
PipelineExecutingBlockInputStream::PipelineExecutingBlockInputStream(QueryPipeline pipeline_)
|
||||||
|
: pipeline(std::make_unique<QueryPipeline>(std::move(pipeline_)))
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
PipelineExecutingBlockInputStream::~PipelineExecutingBlockInputStream() = default;
|
||||||
|
|
||||||
|
Block PipelineExecutingBlockInputStream::getHeader() const
|
||||||
|
{
|
||||||
|
if (executor)
|
||||||
|
return executor->getHeader();
|
||||||
|
|
||||||
|
if (async_executor)
|
||||||
|
return async_executor->getHeader();
|
||||||
|
|
||||||
|
return pipeline->getHeader();
|
||||||
|
}
|
||||||
|
|
||||||
|
void PipelineExecutingBlockInputStream::createExecutor()
|
||||||
|
{
|
||||||
|
if (pipeline->getNumThreads() > 1)
|
||||||
|
async_executor = std::make_unique<PullingAsyncPipelineExecutor>(*pipeline);
|
||||||
|
else
|
||||||
|
executor = std::make_unique<PullingPipelineExecutor>(*pipeline);
|
||||||
|
|
||||||
|
is_execution_started = true;
|
||||||
|
}
|
||||||
|
|
||||||
|
void PipelineExecutingBlockInputStream::readPrefixImpl()
|
||||||
|
{
|
||||||
|
createExecutor();
|
||||||
|
}
|
||||||
|
|
||||||
|
Block PipelineExecutingBlockInputStream::readImpl()
|
||||||
|
{
|
||||||
|
if (!is_execution_started)
|
||||||
|
createExecutor();
|
||||||
|
|
||||||
|
Block block;
|
||||||
|
bool can_continue = true;
|
||||||
|
while (can_continue)
|
||||||
|
{
|
||||||
|
if (executor)
|
||||||
|
can_continue = executor->pull(block);
|
||||||
|
else
|
||||||
|
can_continue = async_executor->pull(block);
|
||||||
|
|
||||||
|
if (block)
|
||||||
|
return block;
|
||||||
|
}
|
||||||
|
|
||||||
|
totals = executor ? executor->getTotalsBlock()
|
||||||
|
: async_executor->getTotalsBlock();
|
||||||
|
|
||||||
|
extremes = executor ? executor->getExtremesBlock()
|
||||||
|
: async_executor->getExtremesBlock();
|
||||||
|
|
||||||
|
return {};
|
||||||
|
}
|
||||||
|
|
||||||
|
inline static void throwIfExecutionStarted(bool is_execution_started, const char * method)
|
||||||
|
{
|
||||||
|
if (is_execution_started)
|
||||||
|
throw Exception(String("Cannot call ") + method +
|
||||||
|
" for PipelineExecutingBlockInputStream because execution was started",
|
||||||
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
|
||||||
|
void PipelineExecutingBlockInputStream::cancel(bool kill)
|
||||||
|
{
|
||||||
|
IBlockInputStream::cancel(kill);
|
||||||
|
|
||||||
|
if (is_execution_started)
|
||||||
|
{
|
||||||
|
executor ? executor->cancel()
|
||||||
|
: async_executor->cancel();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void PipelineExecutingBlockInputStream::setProgressCallback(const ProgressCallback & callback)
|
||||||
|
{
|
||||||
|
throwIfExecutionStarted(is_execution_started, "setProgressCallback");
|
||||||
|
pipeline->setProgressCallback(callback);
|
||||||
|
}
|
||||||
|
|
||||||
|
void PipelineExecutingBlockInputStream::setProcessListElement(QueryStatus * elem)
|
||||||
|
{
|
||||||
|
throwIfExecutionStarted(is_execution_started, "setProcessListElement");
|
||||||
|
IBlockInputStream::setProcessListElement(elem);
|
||||||
|
pipeline->setProcessListElement(elem);
|
||||||
|
}
|
||||||
|
|
||||||
|
void PipelineExecutingBlockInputStream::setLimits(const IBlockInputStream::LocalLimits & limits_)
|
||||||
|
{
|
||||||
|
throwIfExecutionStarted(is_execution_started, "setLimits");
|
||||||
|
|
||||||
|
if (limits_.mode == LimitsMode::LIMITS_TOTAL)
|
||||||
|
throw Exception("Total limits are not supported by PipelineExecutingBlockInputStream",
|
||||||
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
/// Local limits may be checked by IBlockInputStream itself.
|
||||||
|
IBlockInputStream::setLimits(limits_);
|
||||||
|
}
|
||||||
|
|
||||||
|
void PipelineExecutingBlockInputStream::setQuota(const std::shared_ptr<const EnabledQuota> &)
|
||||||
|
{
|
||||||
|
throw Exception("Quota is not supported by PipelineExecutingBlockInputStream",
|
||||||
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
|
||||||
|
void PipelineExecutingBlockInputStream::addTotalRowsApprox(size_t)
|
||||||
|
{
|
||||||
|
throw Exception("Progress is not supported by PipelineExecutingBlockInputStream",
|
||||||
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
}
|
45
src/Processors/Executors/PipelineExecutingBlockInputStream.h
Normal file
45
src/Processors/Executors/PipelineExecutingBlockInputStream.h
Normal file
@ -0,0 +1,45 @@
|
|||||||
|
#pragma once
|
||||||
|
#include <DataStreams/IBlockInputStream.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
class QueryPipeline;
|
||||||
|
class PullingAsyncPipelineExecutor;
|
||||||
|
class PullingPipelineExecutor;
|
||||||
|
|
||||||
|
/// Implement IBlockInputStream from QueryPipeline.
|
||||||
|
/// It's a temporary wrapper.
|
||||||
|
class PipelineExecutingBlockInputStream : public IBlockInputStream
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
explicit PipelineExecutingBlockInputStream(QueryPipeline pipeline_);
|
||||||
|
~PipelineExecutingBlockInputStream() override;
|
||||||
|
|
||||||
|
String getName() const override { return "PipelineExecuting"; }
|
||||||
|
Block getHeader() const override;
|
||||||
|
|
||||||
|
void cancel(bool kill) override;
|
||||||
|
|
||||||
|
/// Implement IBlockInputStream methods via QueryPipeline.
|
||||||
|
void setProgressCallback(const ProgressCallback & callback) final;
|
||||||
|
void setProcessListElement(QueryStatus * elem) final;
|
||||||
|
void setLimits(const LocalLimits & limits_) final;
|
||||||
|
void setQuota(const std::shared_ptr<const EnabledQuota> & quota_) final;
|
||||||
|
void addTotalRowsApprox(size_t value) final;
|
||||||
|
|
||||||
|
protected:
|
||||||
|
void readPrefixImpl() override;
|
||||||
|
Block readImpl() override;
|
||||||
|
|
||||||
|
private:
|
||||||
|
std::unique_ptr<QueryPipeline> pipeline;
|
||||||
|
/// One of executors is used.
|
||||||
|
std::unique_ptr<PullingPipelineExecutor> executor; /// for singe thread.
|
||||||
|
std::unique_ptr<PullingAsyncPipelineExecutor> async_executor; /// for many threads.
|
||||||
|
bool is_execution_started = false;
|
||||||
|
|
||||||
|
void createExecutor();
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
@ -491,6 +491,34 @@ void PipelineExecutor::execute(size_t num_threads)
|
|||||||
throw;
|
throw;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
finalizeExecution();
|
||||||
|
}
|
||||||
|
|
||||||
|
bool PipelineExecutor::executeStep(std::atomic_bool * yield_flag)
|
||||||
|
{
|
||||||
|
if (finished)
|
||||||
|
return false;
|
||||||
|
|
||||||
|
if (!is_execution_initialized)
|
||||||
|
initializeExecution(1);
|
||||||
|
|
||||||
|
executeStepImpl(0, 1, yield_flag);
|
||||||
|
|
||||||
|
if (!finished)
|
||||||
|
return true;
|
||||||
|
|
||||||
|
/// Execution can be stopped because of exception. Check and rethrow if any.
|
||||||
|
for (auto & node : graph)
|
||||||
|
if (node.execution_state->exception)
|
||||||
|
std::rethrow_exception(node.execution_state->exception);
|
||||||
|
|
||||||
|
finalizeExecution();
|
||||||
|
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
void PipelineExecutor::finalizeExecution()
|
||||||
|
{
|
||||||
if (process_list_element && process_list_element->isKilled())
|
if (process_list_element && process_list_element->isKilled())
|
||||||
throw Exception("Query was cancelled", ErrorCodes::QUERY_WAS_CANCELLED);
|
throw Exception("Query was cancelled", ErrorCodes::QUERY_WAS_CANCELLED);
|
||||||
|
|
||||||
@ -506,33 +534,39 @@ void PipelineExecutor::execute(size_t num_threads)
|
|||||||
throw Exception("Pipeline stuck. Current state:\n" + dumpPipeline(), ErrorCodes::LOGICAL_ERROR);
|
throw Exception("Pipeline stuck. Current state:\n" + dumpPipeline(), ErrorCodes::LOGICAL_ERROR);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void PipelineExecutor::wakeUpExecutor(size_t thread_num)
|
||||||
|
{
|
||||||
|
std::lock_guard guard(executor_contexts[thread_num]->mutex);
|
||||||
|
executor_contexts[thread_num]->wake_flag = true;
|
||||||
|
executor_contexts[thread_num]->condvar.notify_one();
|
||||||
|
}
|
||||||
|
|
||||||
void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads)
|
void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads)
|
||||||
{
|
{
|
||||||
#ifndef NDEBUG
|
executeStepImpl(thread_num, num_threads);
|
||||||
UInt64 total_time_ns = 0;
|
|
||||||
UInt64 execution_time_ns = 0;
|
|
||||||
UInt64 processing_time_ns = 0;
|
|
||||||
UInt64 wait_time_ns = 0;
|
|
||||||
|
|
||||||
|
#ifndef NDEBUG
|
||||||
|
auto & context = executor_contexts[thread_num];
|
||||||
|
LOG_TRACE(log, std::fixed << std::setprecision(3)
|
||||||
|
<< "Thread finished."
|
||||||
|
<< " Total time: " << (context->total_time_ns / 1e9) << " sec."
|
||||||
|
<< " Execution time: " << (context->execution_time_ns / 1e9) << " sec."
|
||||||
|
<< " Processing time: " << (context->processing_time_ns / 1e9) << " sec."
|
||||||
|
<< " Wait time: " << (context->wait_time_ns / 1e9) << " sec.");
|
||||||
|
#endif
|
||||||
|
}
|
||||||
|
|
||||||
|
void PipelineExecutor::executeStepImpl(size_t thread_num, size_t num_threads, std::atomic_bool * yield_flag)
|
||||||
|
{
|
||||||
|
#ifndef NDEBUG
|
||||||
Stopwatch total_time_watch;
|
Stopwatch total_time_watch;
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
ExecutionState * state = nullptr;
|
auto & context = executor_contexts[thread_num];
|
||||||
|
auto & state = context->state;
|
||||||
|
bool yield = false;
|
||||||
|
|
||||||
auto prepare_processor = [&](UInt64 pid, Queue & queue)
|
while (!finished && !yield)
|
||||||
{
|
|
||||||
if (!prepareProcessor(pid, thread_num, queue, std::unique_lock<std::mutex>(*graph[pid].status_mutex)))
|
|
||||||
finish();
|
|
||||||
};
|
|
||||||
|
|
||||||
auto wake_up_executor = [&](size_t executor)
|
|
||||||
{
|
|
||||||
std::lock_guard guard(executor_contexts[executor]->mutex);
|
|
||||||
executor_contexts[executor]->wake_flag = true;
|
|
||||||
executor_contexts[executor]->condvar.notify_one();
|
|
||||||
};
|
|
||||||
|
|
||||||
while (!finished)
|
|
||||||
{
|
{
|
||||||
/// First, find any processor to execute.
|
/// First, find any processor to execute.
|
||||||
/// Just travers graph and prepare any processor.
|
/// Just travers graph and prepare any processor.
|
||||||
@ -555,7 +589,7 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads
|
|||||||
thread_to_wake = threads_queue.pop_any();
|
thread_to_wake = threads_queue.pop_any();
|
||||||
|
|
||||||
lock.unlock();
|
lock.unlock();
|
||||||
wake_up_executor(thread_to_wake);
|
wakeUpExecutor(thread_to_wake);
|
||||||
}
|
}
|
||||||
|
|
||||||
break;
|
break;
|
||||||
@ -572,21 +606,21 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads
|
|||||||
}
|
}
|
||||||
|
|
||||||
{
|
{
|
||||||
std::unique_lock lock(executor_contexts[thread_num]->mutex);
|
std::unique_lock lock(context->mutex);
|
||||||
|
|
||||||
executor_contexts[thread_num]->condvar.wait(lock, [&]
|
context->condvar.wait(lock, [&]
|
||||||
{
|
{
|
||||||
return finished || executor_contexts[thread_num]->wake_flag;
|
return finished || context->wake_flag;
|
||||||
});
|
});
|
||||||
|
|
||||||
executor_contexts[thread_num]->wake_flag = false;
|
context->wake_flag = false;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (finished)
|
if (finished)
|
||||||
break;
|
break;
|
||||||
|
|
||||||
while (state)
|
while (state && !yield)
|
||||||
{
|
{
|
||||||
if (finished)
|
if (finished)
|
||||||
break;
|
break;
|
||||||
@ -601,7 +635,7 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads
|
|||||||
state->job();
|
state->job();
|
||||||
|
|
||||||
#ifndef NDEBUG
|
#ifndef NDEBUG
|
||||||
execution_time_ns += execution_time_watch.elapsed();
|
context->execution_time_ns += execution_time_watch.elapsed();
|
||||||
#endif
|
#endif
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -623,8 +657,13 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads
|
|||||||
while (auto * task = expand_pipeline_task.load())
|
while (auto * task = expand_pipeline_task.load())
|
||||||
doExpandPipeline(task, true);
|
doExpandPipeline(task, true);
|
||||||
|
|
||||||
/// Execute again if can.
|
/// Prepare processor after execution.
|
||||||
prepare_processor(state->processors_id, queue);
|
{
|
||||||
|
auto lock = std::unique_lock<std::mutex>(*graph[state->processors_id].status_mutex);
|
||||||
|
if (!prepareProcessor(state->processors_id, thread_num, queue, std::move(lock)))
|
||||||
|
finish();
|
||||||
|
}
|
||||||
|
|
||||||
state = nullptr;
|
state = nullptr;
|
||||||
|
|
||||||
/// Take local task from queue if has one.
|
/// Take local task from queue if has one.
|
||||||
@ -656,7 +695,7 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads
|
|||||||
|
|
||||||
lock.unlock();
|
lock.unlock();
|
||||||
|
|
||||||
wake_up_executor(thread_to_wake);
|
wakeUpExecutor(thread_to_wake);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -666,27 +705,24 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads
|
|||||||
}
|
}
|
||||||
|
|
||||||
#ifndef NDEBUG
|
#ifndef NDEBUG
|
||||||
processing_time_ns += processing_time_watch.elapsed();
|
context->processing_time_ns += processing_time_watch.elapsed();
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
|
/// We have executed single processor. Check if we need to yield execution.
|
||||||
|
if (yield_flag && *yield_flag)
|
||||||
|
yield = true;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
#ifndef NDEBUG
|
#ifndef NDEBUG
|
||||||
total_time_ns = total_time_watch.elapsed();
|
context->total_time_ns += total_time_watch.elapsed();
|
||||||
wait_time_ns = total_time_ns - execution_time_ns - processing_time_ns;
|
context->wait_time_ns = context->total_time_ns - context->execution_time_ns - context->processing_time_ns;
|
||||||
|
|
||||||
LOG_TRACE(log, std::fixed << std::setprecision(3)
|
|
||||||
<< "Thread finished."
|
|
||||||
<< " Total time: " << (total_time_ns / 1e9) << " sec."
|
|
||||||
<< " Execution time: " << (execution_time_ns / 1e9) << " sec."
|
|
||||||
<< " Processing time: " << (processing_time_ns / 1e9) << " sec."
|
|
||||||
<< " Wait time: " << (wait_time_ns / 1e9) << " sec.");
|
|
||||||
#endif
|
#endif
|
||||||
}
|
}
|
||||||
|
|
||||||
void PipelineExecutor::executeImpl(size_t num_threads)
|
void PipelineExecutor::initializeExecution(size_t num_threads)
|
||||||
{
|
{
|
||||||
Stack stack;
|
is_execution_initialized = true;
|
||||||
|
|
||||||
threads_queue.init(num_threads);
|
threads_queue.init(num_threads);
|
||||||
task_queue.init(num_threads);
|
task_queue.init(num_threads);
|
||||||
@ -699,25 +735,7 @@ void PipelineExecutor::executeImpl(size_t num_threads)
|
|||||||
executor_contexts.emplace_back(std::make_unique<ExecutorContext>());
|
executor_contexts.emplace_back(std::make_unique<ExecutorContext>());
|
||||||
}
|
}
|
||||||
|
|
||||||
auto thread_group = CurrentThread::getGroup();
|
Stack stack;
|
||||||
|
|
||||||
using ThreadsData = std::vector<ThreadFromGlobalPool>;
|
|
||||||
ThreadsData threads;
|
|
||||||
threads.reserve(num_threads);
|
|
||||||
|
|
||||||
bool finished_flag = false;
|
|
||||||
|
|
||||||
SCOPE_EXIT(
|
|
||||||
if (!finished_flag)
|
|
||||||
{
|
|
||||||
finish();
|
|
||||||
|
|
||||||
for (auto & thread : threads)
|
|
||||||
if (thread.joinable())
|
|
||||||
thread.join();
|
|
||||||
}
|
|
||||||
);
|
|
||||||
|
|
||||||
addChildlessProcessorsToStack(stack);
|
addChildlessProcessorsToStack(stack);
|
||||||
|
|
||||||
{
|
{
|
||||||
@ -744,9 +762,32 @@ void PipelineExecutor::executeImpl(size_t num_threads)
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void PipelineExecutor::executeImpl(size_t num_threads)
|
||||||
|
{
|
||||||
|
initializeExecution(num_threads);
|
||||||
|
|
||||||
|
using ThreadsData = std::vector<ThreadFromGlobalPool>;
|
||||||
|
ThreadsData threads;
|
||||||
|
threads.reserve(num_threads);
|
||||||
|
|
||||||
|
bool finished_flag = false;
|
||||||
|
|
||||||
|
SCOPE_EXIT(
|
||||||
|
if (!finished_flag)
|
||||||
|
{
|
||||||
|
finish();
|
||||||
|
|
||||||
|
for (auto & thread : threads)
|
||||||
|
if (thread.joinable())
|
||||||
|
thread.join();
|
||||||
|
}
|
||||||
|
);
|
||||||
|
|
||||||
if (num_threads > 1)
|
if (num_threads > 1)
|
||||||
{
|
{
|
||||||
|
auto thread_group = CurrentThread::getGroup();
|
||||||
|
|
||||||
for (size_t i = 0; i < num_threads; ++i)
|
for (size_t i = 0; i < num_threads; ++i)
|
||||||
{
|
{
|
||||||
|
@ -35,7 +35,10 @@ public:
|
|||||||
/// In case of exception during execution throws any occurred.
|
/// In case of exception during execution throws any occurred.
|
||||||
void execute(size_t num_threads);
|
void execute(size_t num_threads);
|
||||||
|
|
||||||
String getName() const { return "PipelineExecutor"; }
|
/// Execute single step. Step will be stopped when yield_flag is true.
|
||||||
|
/// Execution is happened in a single thread.
|
||||||
|
/// Return true if execution should be continued.
|
||||||
|
bool executeStep(std::atomic_bool * yield_flag = nullptr);
|
||||||
|
|
||||||
const Processors & getProcessors() const { return processors; }
|
const Processors & getProcessors() const { return processors; }
|
||||||
|
|
||||||
@ -203,6 +206,8 @@ private:
|
|||||||
ThreadsQueue threads_queue;
|
ThreadsQueue threads_queue;
|
||||||
std::mutex task_queue_mutex;
|
std::mutex task_queue_mutex;
|
||||||
|
|
||||||
|
/// Flag that checks that initializeExecution was called.
|
||||||
|
bool is_execution_initialized = false;
|
||||||
std::atomic_bool cancelled;
|
std::atomic_bool cancelled;
|
||||||
std::atomic_bool finished;
|
std::atomic_bool finished;
|
||||||
|
|
||||||
@ -235,7 +240,17 @@ private:
|
|||||||
std::mutex mutex;
|
std::mutex mutex;
|
||||||
bool wake_flag = false;
|
bool wake_flag = false;
|
||||||
|
|
||||||
/// std::queue<ExecutionState *> pinned_tasks;
|
/// Currently processing state.
|
||||||
|
ExecutionState * state = nullptr;
|
||||||
|
|
||||||
|
#ifndef NDEBUG
|
||||||
|
/// Time for different processing stages.
|
||||||
|
UInt64 total_time_ns = 0;
|
||||||
|
UInt64 execution_time_ns = 0;
|
||||||
|
UInt64 processing_time_ns = 0;
|
||||||
|
UInt64 wait_time_ns = 0;
|
||||||
|
#endif
|
||||||
|
|
||||||
};
|
};
|
||||||
|
|
||||||
std::vector<std::unique_ptr<ExecutorContext>> executor_contexts;
|
std::vector<std::unique_ptr<ExecutorContext>> executor_contexts;
|
||||||
@ -267,7 +282,15 @@ private:
|
|||||||
bool prepareProcessor(UInt64 pid, size_t thread_number, Queue & queue, std::unique_lock<std::mutex> node_lock);
|
bool prepareProcessor(UInt64 pid, size_t thread_number, Queue & queue, std::unique_lock<std::mutex> node_lock);
|
||||||
bool doExpandPipeline(ExpandPipelineTask * task, bool processing);
|
bool doExpandPipeline(ExpandPipelineTask * task, bool processing);
|
||||||
|
|
||||||
|
/// Continue executor (in case there are tasks in queue).
|
||||||
|
void wakeUpExecutor(size_t thread_num);
|
||||||
|
|
||||||
|
void initializeExecution(size_t num_threads); /// Initialize executor contexts and task_queue.
|
||||||
|
void finalizeExecution(); /// Check all processors are finished.
|
||||||
|
|
||||||
|
/// Methods connected to execution.
|
||||||
void executeImpl(size_t num_threads);
|
void executeImpl(size_t num_threads);
|
||||||
|
void executeStepImpl(size_t thread_num, size_t num_threads, std::atomic_bool * yield_flag = nullptr);
|
||||||
void executeSingleThread(size_t thread_num, size_t num_threads);
|
void executeSingleThread(size_t thread_num, size_t num_threads);
|
||||||
void finish();
|
void finish();
|
||||||
|
|
||||||
|
203
src/Processors/Executors/PullingAsyncPipelineExecutor.cpp
Normal file
203
src/Processors/Executors/PullingAsyncPipelineExecutor.cpp
Normal file
@ -0,0 +1,203 @@
|
|||||||
|
#include <Processors/Executors/PullingAsyncPipelineExecutor.h>
|
||||||
|
#include <Processors/Executors/PipelineExecutor.h>
|
||||||
|
#include <Processors/Formats/LazyOutputFormat.h>
|
||||||
|
#include <Processors/Transforms/AggregatingTransform.h>
|
||||||
|
#include <Processors/QueryPipeline.h>
|
||||||
|
|
||||||
|
#include <Common/setThreadName.h>
|
||||||
|
#include <ext/scope_guard.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
struct PullingAsyncPipelineExecutor::Data
|
||||||
|
{
|
||||||
|
PipelineExecutorPtr executor;
|
||||||
|
std::exception_ptr exception;
|
||||||
|
std::atomic_bool is_executed = false;
|
||||||
|
std::atomic_bool has_exception = false;
|
||||||
|
ThreadFromGlobalPool thread;
|
||||||
|
|
||||||
|
~Data()
|
||||||
|
{
|
||||||
|
if (thread.joinable())
|
||||||
|
thread.join();
|
||||||
|
}
|
||||||
|
|
||||||
|
void rethrowExceptionIfHas()
|
||||||
|
{
|
||||||
|
if (has_exception)
|
||||||
|
{
|
||||||
|
has_exception = false;
|
||||||
|
std::rethrow_exception(std::move(exception));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
PullingAsyncPipelineExecutor::PullingAsyncPipelineExecutor(QueryPipeline & pipeline_) : pipeline(pipeline_)
|
||||||
|
{
|
||||||
|
lazy_format = std::make_shared<LazyOutputFormat>(pipeline.getHeader());
|
||||||
|
pipeline.setOutput(lazy_format);
|
||||||
|
}
|
||||||
|
|
||||||
|
PullingAsyncPipelineExecutor::~PullingAsyncPipelineExecutor()
|
||||||
|
{
|
||||||
|
try
|
||||||
|
{
|
||||||
|
cancel();
|
||||||
|
}
|
||||||
|
catch (...)
|
||||||
|
{
|
||||||
|
tryLogCurrentException("PullingAsyncPipelineExecutor");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
const Block & PullingAsyncPipelineExecutor::getHeader() const
|
||||||
|
{
|
||||||
|
return lazy_format->getPort(IOutputFormat::PortKind::Main).getHeader();
|
||||||
|
}
|
||||||
|
|
||||||
|
static void threadFunction(PullingAsyncPipelineExecutor::Data & data, ThreadGroupStatusPtr thread_group, size_t num_threads)
|
||||||
|
{
|
||||||
|
if (thread_group)
|
||||||
|
CurrentThread::attachTo(thread_group);
|
||||||
|
|
||||||
|
SCOPE_EXIT(
|
||||||
|
if (thread_group)
|
||||||
|
CurrentThread::detachQueryIfNotDetached();
|
||||||
|
);
|
||||||
|
|
||||||
|
setThreadName("QueryPipelineEx");
|
||||||
|
|
||||||
|
try
|
||||||
|
{
|
||||||
|
data.executor->execute(num_threads);
|
||||||
|
}
|
||||||
|
catch (...)
|
||||||
|
{
|
||||||
|
data.exception = std::current_exception();
|
||||||
|
data.has_exception = true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
bool PullingAsyncPipelineExecutor::pull(Chunk & chunk, uint64_t milliseconds)
|
||||||
|
{
|
||||||
|
if (!data)
|
||||||
|
{
|
||||||
|
data = std::make_unique<Data>();
|
||||||
|
data->executor = pipeline.execute();
|
||||||
|
|
||||||
|
auto func = [&, thread_group = CurrentThread::getGroup()]()
|
||||||
|
{
|
||||||
|
threadFunction(*data, thread_group, pipeline.getNumThreads());
|
||||||
|
};
|
||||||
|
|
||||||
|
data->thread = ThreadFromGlobalPool(std::move(func));
|
||||||
|
}
|
||||||
|
|
||||||
|
if (data->has_exception)
|
||||||
|
{
|
||||||
|
/// Finish lazy format in case of exception. Otherwise thread.join() may hung.
|
||||||
|
lazy_format->finish();
|
||||||
|
data->has_exception = false;
|
||||||
|
std::rethrow_exception(std::move(data->exception));
|
||||||
|
}
|
||||||
|
|
||||||
|
if (lazy_format->isFinished())
|
||||||
|
{
|
||||||
|
data->is_executed = true;
|
||||||
|
/// Wait thread ant rethrow exception if any.
|
||||||
|
cancel();
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
chunk = lazy_format->getChunk(milliseconds);
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
bool PullingAsyncPipelineExecutor::pull(Block & block, uint64_t milliseconds)
|
||||||
|
{
|
||||||
|
Chunk chunk;
|
||||||
|
|
||||||
|
if (!pull(chunk, milliseconds))
|
||||||
|
return false;
|
||||||
|
|
||||||
|
if (!chunk)
|
||||||
|
{
|
||||||
|
/// In case if timeout exceeded.
|
||||||
|
block.clear();
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
block = lazy_format->getPort(IOutputFormat::PortKind::Main).getHeader().cloneWithColumns(chunk.detachColumns());
|
||||||
|
|
||||||
|
if (auto chunk_info = chunk.getChunkInfo())
|
||||||
|
{
|
||||||
|
if (const auto * agg_info = typeid_cast<const AggregatedChunkInfo *>(chunk_info.get()))
|
||||||
|
{
|
||||||
|
block.info.bucket_num = agg_info->bucket_num;
|
||||||
|
block.info.is_overflows = agg_info->is_overflows;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
void PullingAsyncPipelineExecutor::cancel()
|
||||||
|
{
|
||||||
|
/// Cancel execution if it wasn't finished.
|
||||||
|
if (data && !data->is_executed && data->executor)
|
||||||
|
data->executor->cancel();
|
||||||
|
|
||||||
|
/// Finish lazy format. Otherwise thread.join() may hung.
|
||||||
|
if (!lazy_format->isFinished())
|
||||||
|
lazy_format->finish();
|
||||||
|
|
||||||
|
/// Join thread here to wait for possible exception.
|
||||||
|
if (data && data->thread.joinable())
|
||||||
|
data->thread.join();
|
||||||
|
|
||||||
|
/// Rethrow exception to not swallow it in destructor.
|
||||||
|
if (data)
|
||||||
|
data->rethrowExceptionIfHas();
|
||||||
|
}
|
||||||
|
|
||||||
|
Chunk PullingAsyncPipelineExecutor::getTotals()
|
||||||
|
{
|
||||||
|
return lazy_format->getTotals();
|
||||||
|
}
|
||||||
|
|
||||||
|
Chunk PullingAsyncPipelineExecutor::getExtremes()
|
||||||
|
{
|
||||||
|
return lazy_format->getExtremes();
|
||||||
|
}
|
||||||
|
|
||||||
|
Block PullingAsyncPipelineExecutor::getTotalsBlock()
|
||||||
|
{
|
||||||
|
auto totals = getTotals();
|
||||||
|
|
||||||
|
if (totals.empty())
|
||||||
|
return {};
|
||||||
|
|
||||||
|
const auto & header = lazy_format->getPort(IOutputFormat::PortKind::Totals).getHeader();
|
||||||
|
return header.cloneWithColumns(totals.detachColumns());
|
||||||
|
}
|
||||||
|
|
||||||
|
Block PullingAsyncPipelineExecutor::getExtremesBlock()
|
||||||
|
{
|
||||||
|
auto extremes = getExtremes();
|
||||||
|
|
||||||
|
if (extremes.empty())
|
||||||
|
return {};
|
||||||
|
|
||||||
|
const auto & header = lazy_format->getPort(IOutputFormat::PortKind::Extremes).getHeader();
|
||||||
|
return header.cloneWithColumns(extremes.detachColumns());
|
||||||
|
}
|
||||||
|
|
||||||
|
BlockStreamProfileInfo & PullingAsyncPipelineExecutor::getProfileInfo()
|
||||||
|
{
|
||||||
|
return lazy_format->getProfileInfo();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
58
src/Processors/Executors/PullingAsyncPipelineExecutor.h
Normal file
58
src/Processors/Executors/PullingAsyncPipelineExecutor.h
Normal file
@ -0,0 +1,58 @@
|
|||||||
|
#pragma once
|
||||||
|
#include <memory>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
class QueryPipeline;
|
||||||
|
class Block;
|
||||||
|
class Chunk;
|
||||||
|
class LazyOutputFormat;
|
||||||
|
struct BlockStreamProfileInfo;
|
||||||
|
|
||||||
|
/// Asynchronous pulling executor for QueryPipeline.
|
||||||
|
/// Always creates extra thread. If query is executed in single thread, use PullingPipelineExecutor.
|
||||||
|
/// Typical usage is:
|
||||||
|
///
|
||||||
|
/// PullingAsyncPipelineExecutor executor(query_pipeline);
|
||||||
|
/// while (executor.pull(chunk, timeout))
|
||||||
|
/// ... process chunk ...
|
||||||
|
class PullingAsyncPipelineExecutor
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
explicit PullingAsyncPipelineExecutor(QueryPipeline & pipeline_);
|
||||||
|
~PullingAsyncPipelineExecutor();
|
||||||
|
|
||||||
|
/// Get structure of returned block or chunk.
|
||||||
|
const Block & getHeader() const;
|
||||||
|
|
||||||
|
/// Methods return false if query is finished.
|
||||||
|
/// If milliseconds > 0, returns empty object and `true` after timeout exceeded. Otherwise method is blocking.
|
||||||
|
/// You can use any pull method.
|
||||||
|
bool pull(Chunk & chunk, uint64_t milliseconds = 0);
|
||||||
|
bool pull(Block & block, uint64_t milliseconds = 0);
|
||||||
|
|
||||||
|
/// Stop execution. It is not necessary, but helps to stop execution before executor is destroyed.
|
||||||
|
void cancel();
|
||||||
|
|
||||||
|
/// Get totals and extremes. Returns empty chunk if doesn't have any.
|
||||||
|
Chunk getTotals();
|
||||||
|
Chunk getExtremes();
|
||||||
|
|
||||||
|
/// Get totals and extremes. Returns empty chunk if doesn't have any.
|
||||||
|
Block getTotalsBlock();
|
||||||
|
Block getExtremesBlock();
|
||||||
|
|
||||||
|
/// Get query profile info.
|
||||||
|
BlockStreamProfileInfo & getProfileInfo();
|
||||||
|
|
||||||
|
/// Internal executor data.
|
||||||
|
struct Data;
|
||||||
|
|
||||||
|
private:
|
||||||
|
QueryPipeline & pipeline;
|
||||||
|
std::shared_ptr<LazyOutputFormat> lazy_format;
|
||||||
|
std::unique_ptr<Data> data;
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
@ -1,43 +1,15 @@
|
|||||||
#include <Processors/Executors/PullingPipelineExecutor.h>
|
#include <Processors/Executors/PullingPipelineExecutor.h>
|
||||||
#include <Processors/Executors/PipelineExecutor.h>
|
#include <Processors/Formats/PullingOutputFormat.h>
|
||||||
#include <Processors/Formats/LazyOutputFormat.h>
|
|
||||||
#include <Processors/Transforms/AggregatingTransform.h>
|
|
||||||
#include <Processors/QueryPipeline.h>
|
#include <Processors/QueryPipeline.h>
|
||||||
|
#include <Processors/Transforms/AggregatingTransform.h>
|
||||||
#include <Common/setThreadName.h>
|
|
||||||
#include <ext/scope_guard.h>
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
struct PullingPipelineExecutor::Data
|
|
||||||
{
|
|
||||||
PipelineExecutorPtr executor;
|
|
||||||
std::exception_ptr exception;
|
|
||||||
std::atomic_bool is_executed = false;
|
|
||||||
std::atomic_bool has_exception = false;
|
|
||||||
ThreadFromGlobalPool thread;
|
|
||||||
|
|
||||||
~Data()
|
|
||||||
{
|
|
||||||
if (thread.joinable())
|
|
||||||
thread.join();
|
|
||||||
}
|
|
||||||
|
|
||||||
void rethrowExceptionIfHas()
|
|
||||||
{
|
|
||||||
if (has_exception)
|
|
||||||
{
|
|
||||||
has_exception = false;
|
|
||||||
std::rethrow_exception(std::move(exception));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
PullingPipelineExecutor::PullingPipelineExecutor(QueryPipeline & pipeline_) : pipeline(pipeline_)
|
PullingPipelineExecutor::PullingPipelineExecutor(QueryPipeline & pipeline_) : pipeline(pipeline_)
|
||||||
{
|
{
|
||||||
lazy_format = std::make_shared<LazyOutputFormat>(pipeline.getHeader());
|
pulling_format = std::make_shared<PullingOutputFormat>(pipeline.getHeader(), has_data_flag);
|
||||||
pipeline.setOutput(lazy_format);
|
pipeline.setOutput(pulling_format);
|
||||||
}
|
}
|
||||||
|
|
||||||
PullingPipelineExecutor::~PullingPipelineExecutor()
|
PullingPipelineExecutor::~PullingPipelineExecutor()
|
||||||
@ -52,70 +24,28 @@ PullingPipelineExecutor::~PullingPipelineExecutor()
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
static void threadFunction(PullingPipelineExecutor::Data & data, ThreadGroupStatusPtr thread_group, size_t num_threads)
|
const Block & PullingPipelineExecutor::getHeader() const
|
||||||
{
|
{
|
||||||
if (thread_group)
|
return pulling_format->getPort(IOutputFormat::PortKind::Main).getHeader();
|
||||||
CurrentThread::attachTo(thread_group);
|
|
||||||
|
|
||||||
SCOPE_EXIT(
|
|
||||||
if (thread_group)
|
|
||||||
CurrentThread::detachQueryIfNotDetached();
|
|
||||||
);
|
|
||||||
|
|
||||||
setThreadName("QueryPipelineEx");
|
|
||||||
|
|
||||||
try
|
|
||||||
{
|
|
||||||
data.executor->execute(num_threads);
|
|
||||||
}
|
|
||||||
catch (...)
|
|
||||||
{
|
|
||||||
data.exception = std::current_exception();
|
|
||||||
data.has_exception = true;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
bool PullingPipelineExecutor::pull(Chunk & chunk)
|
||||||
bool PullingPipelineExecutor::pull(Chunk & chunk, uint64_t milliseconds)
|
|
||||||
{
|
{
|
||||||
if (!data)
|
if (!executor)
|
||||||
{
|
executor = pipeline.execute();
|
||||||
data = std::make_unique<Data>();
|
|
||||||
data->executor = pipeline.execute();
|
|
||||||
|
|
||||||
auto func = [&, thread_group = CurrentThread::getGroup()]()
|
if (!executor->executeStep(&has_data_flag))
|
||||||
{
|
|
||||||
threadFunction(*data, thread_group, pipeline.getNumThreads());
|
|
||||||
};
|
|
||||||
|
|
||||||
data->thread = ThreadFromGlobalPool(std::move(func));
|
|
||||||
}
|
|
||||||
|
|
||||||
if (data->has_exception)
|
|
||||||
{
|
|
||||||
/// Finish lazy format in case of exception. Otherwise thread.join() may hung.
|
|
||||||
lazy_format->finish();
|
|
||||||
data->has_exception = false;
|
|
||||||
std::rethrow_exception(std::move(data->exception));
|
|
||||||
}
|
|
||||||
|
|
||||||
if (lazy_format->isFinished())
|
|
||||||
{
|
|
||||||
data->is_executed = true;
|
|
||||||
/// Wait thread ant rethrow exception if any.
|
|
||||||
cancel();
|
|
||||||
return false;
|
return false;
|
||||||
}
|
|
||||||
|
|
||||||
chunk = lazy_format->getChunk(milliseconds);
|
chunk = pulling_format->getChunk();
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
bool PullingPipelineExecutor::pull(Block & block, uint64_t milliseconds)
|
bool PullingPipelineExecutor::pull(Block & block)
|
||||||
{
|
{
|
||||||
Chunk chunk;
|
Chunk chunk;
|
||||||
|
|
||||||
if (!pull(chunk, milliseconds))
|
if (!pull(chunk))
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
if (!chunk)
|
if (!chunk)
|
||||||
@ -125,7 +55,7 @@ bool PullingPipelineExecutor::pull(Block & block, uint64_t milliseconds)
|
|||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
block = lazy_format->getPort(IOutputFormat::PortKind::Main).getHeader().cloneWithColumns(chunk.detachColumns());
|
block = pulling_format->getPort(IOutputFormat::PortKind::Main).getHeader().cloneWithColumns(chunk.detachColumns());
|
||||||
|
|
||||||
if (auto chunk_info = chunk.getChunkInfo())
|
if (auto chunk_info = chunk.getChunkInfo())
|
||||||
{
|
{
|
||||||
@ -142,30 +72,22 @@ bool PullingPipelineExecutor::pull(Block & block, uint64_t milliseconds)
|
|||||||
void PullingPipelineExecutor::cancel()
|
void PullingPipelineExecutor::cancel()
|
||||||
{
|
{
|
||||||
/// Cancel execution if it wasn't finished.
|
/// Cancel execution if it wasn't finished.
|
||||||
if (data && !data->is_executed && data->executor)
|
if (executor)
|
||||||
data->executor->cancel();
|
executor->cancel();
|
||||||
|
|
||||||
/// Finish lazy format. Otherwise thread.join() may hung.
|
/// Read all data and finish execution.
|
||||||
if (!lazy_format->isFinished())
|
Chunk chunk;
|
||||||
lazy_format->finish();
|
while (pull(chunk));
|
||||||
|
|
||||||
/// Join thread here to wait for possible exception.
|
|
||||||
if (data && data->thread.joinable())
|
|
||||||
data->thread.join();
|
|
||||||
|
|
||||||
/// Rethrow exception to not swallow it in destructor.
|
|
||||||
if (data)
|
|
||||||
data->rethrowExceptionIfHas();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
Chunk PullingPipelineExecutor::getTotals()
|
Chunk PullingPipelineExecutor::getTotals()
|
||||||
{
|
{
|
||||||
return lazy_format->getTotals();
|
return pulling_format->getTotals();
|
||||||
}
|
}
|
||||||
|
|
||||||
Chunk PullingPipelineExecutor::getExtremes()
|
Chunk PullingPipelineExecutor::getExtremes()
|
||||||
{
|
{
|
||||||
return lazy_format->getExtremes();
|
return pulling_format->getExtremes();
|
||||||
}
|
}
|
||||||
|
|
||||||
Block PullingPipelineExecutor::getTotalsBlock()
|
Block PullingPipelineExecutor::getTotalsBlock()
|
||||||
@ -175,7 +97,7 @@ Block PullingPipelineExecutor::getTotalsBlock()
|
|||||||
if (totals.empty())
|
if (totals.empty())
|
||||||
return {};
|
return {};
|
||||||
|
|
||||||
const auto & header = lazy_format->getPort(IOutputFormat::PortKind::Totals).getHeader();
|
const auto & header = pulling_format->getPort(IOutputFormat::PortKind::Totals).getHeader();
|
||||||
return header.cloneWithColumns(totals.detachColumns());
|
return header.cloneWithColumns(totals.detachColumns());
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -186,13 +108,13 @@ Block PullingPipelineExecutor::getExtremesBlock()
|
|||||||
if (extremes.empty())
|
if (extremes.empty())
|
||||||
return {};
|
return {};
|
||||||
|
|
||||||
const auto & header = lazy_format->getPort(IOutputFormat::PortKind::Extremes).getHeader();
|
const auto & header = pulling_format->getPort(IOutputFormat::PortKind::Extremes).getHeader();
|
||||||
return header.cloneWithColumns(extremes.detachColumns());
|
return header.cloneWithColumns(extremes.detachColumns());
|
||||||
}
|
}
|
||||||
|
|
||||||
BlockStreamProfileInfo & PullingPipelineExecutor::getProfileInfo()
|
BlockStreamProfileInfo & PullingPipelineExecutor::getProfileInfo()
|
||||||
{
|
{
|
||||||
return lazy_format->getProfileInfo();
|
return pulling_format->getProfileInfo();
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -1,16 +1,20 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
#include <memory>
|
#include <memory>
|
||||||
|
#include <atomic>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
class QueryPipeline;
|
|
||||||
class Block;
|
class Block;
|
||||||
class Chunk;
|
class Chunk;
|
||||||
class LazyOutputFormat;
|
class QueryPipeline;
|
||||||
|
class PipelineExecutor;
|
||||||
|
class PullingOutputFormat;
|
||||||
struct BlockStreamProfileInfo;
|
struct BlockStreamProfileInfo;
|
||||||
|
|
||||||
/// Pulling executor for QueryPipeline.
|
using PipelineExecutorPtr = std::shared_ptr<PipelineExecutor>;
|
||||||
|
|
||||||
|
/// Pulling executor for QueryPipeline. Always execute pipeline in single thread.
|
||||||
/// Typical usage is:
|
/// Typical usage is:
|
||||||
///
|
///
|
||||||
/// PullingPipelineExecutor executor(query_pipeline);
|
/// PullingPipelineExecutor executor(query_pipeline);
|
||||||
@ -22,11 +26,13 @@ public:
|
|||||||
explicit PullingPipelineExecutor(QueryPipeline & pipeline_);
|
explicit PullingPipelineExecutor(QueryPipeline & pipeline_);
|
||||||
~PullingPipelineExecutor();
|
~PullingPipelineExecutor();
|
||||||
|
|
||||||
|
/// Get structure of returned block or chunk.
|
||||||
|
const Block & getHeader() const;
|
||||||
|
|
||||||
/// Methods return false if query is finished.
|
/// Methods return false if query is finished.
|
||||||
/// If milliseconds > 0, returns empty object and `true` after timeout exceeded.
|
|
||||||
/// You can use any pull method.
|
/// You can use any pull method.
|
||||||
bool pull(Chunk & chunk, uint64_t milliseconds = 0);
|
bool pull(Chunk & chunk);
|
||||||
bool pull(Block & block, uint64_t milliseconds = 0);
|
bool pull(Block & block);
|
||||||
|
|
||||||
/// Stop execution. It is not necessary, but helps to stop execution before executor is destroyed.
|
/// Stop execution. It is not necessary, but helps to stop execution before executor is destroyed.
|
||||||
void cancel();
|
void cancel();
|
||||||
@ -42,13 +48,11 @@ public:
|
|||||||
/// Get query profile info.
|
/// Get query profile info.
|
||||||
BlockStreamProfileInfo & getProfileInfo();
|
BlockStreamProfileInfo & getProfileInfo();
|
||||||
|
|
||||||
/// Internal executor data.
|
|
||||||
struct Data;
|
|
||||||
|
|
||||||
private:
|
private:
|
||||||
|
std::atomic_bool has_data_flag = false;
|
||||||
QueryPipeline & pipeline;
|
QueryPipeline & pipeline;
|
||||||
std::shared_ptr<LazyOutputFormat> lazy_format;
|
std::shared_ptr<PullingOutputFormat> pulling_format;
|
||||||
std::unique_ptr<Data> data;
|
PipelineExecutorPtr executor;
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -9,7 +9,7 @@ namespace DB
|
|||||||
|
|
||||||
/// LazyOutputFormat is used to retrieve ready data from executing pipeline.
|
/// LazyOutputFormat is used to retrieve ready data from executing pipeline.
|
||||||
/// You can periodically call `getChunk` from separate thread.
|
/// You can periodically call `getChunk` from separate thread.
|
||||||
/// Used in PullingPipelineExecutor.
|
/// Used in PullingAsyncPipelineExecutor.
|
||||||
class LazyOutputFormat : public IOutputFormat
|
class LazyOutputFormat : public IOutputFormat
|
||||||
{
|
{
|
||||||
|
|
||||||
|
42
src/Processors/Formats/PullingOutputFormat.cpp
Normal file
42
src/Processors/Formats/PullingOutputFormat.cpp
Normal file
@ -0,0 +1,42 @@
|
|||||||
|
#include <Processors/Formats/PullingOutputFormat.h>
|
||||||
|
#include <IO/WriteBuffer.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
namespace ErrorCodes
|
||||||
|
{
|
||||||
|
extern const int LOGICAL_ERROR;
|
||||||
|
}
|
||||||
|
|
||||||
|
WriteBuffer PullingOutputFormat::out(nullptr, 0);
|
||||||
|
|
||||||
|
void PullingOutputFormat::consume(Chunk chunk)
|
||||||
|
{
|
||||||
|
if (data)
|
||||||
|
throw Exception("PullingOutputFormat cannot consume chunk because it already has data",
|
||||||
|
ErrorCodes::LOGICAL_ERROR);
|
||||||
|
|
||||||
|
if (chunk)
|
||||||
|
info.update(chunk.getNumRows(), chunk.allocatedBytes());
|
||||||
|
|
||||||
|
data = std::move(chunk);
|
||||||
|
has_data_flag = true;
|
||||||
|
}
|
||||||
|
|
||||||
|
Chunk PullingOutputFormat::getChunk()
|
||||||
|
{
|
||||||
|
auto chunk = std::move(data);
|
||||||
|
has_data_flag = false;
|
||||||
|
return chunk;
|
||||||
|
}
|
||||||
|
|
||||||
|
Chunk PullingOutputFormat::getTotals() { return std::move(totals); }
|
||||||
|
Chunk PullingOutputFormat::getExtremes() { return std::move(extremes); }
|
||||||
|
|
||||||
|
void PullingOutputFormat::setRowsBeforeLimit(size_t rows_before_limit)
|
||||||
|
{
|
||||||
|
info.setRowsBeforeLimit(rows_before_limit);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
45
src/Processors/Formats/PullingOutputFormat.h
Normal file
45
src/Processors/Formats/PullingOutputFormat.h
Normal file
@ -0,0 +1,45 @@
|
|||||||
|
#pragma once
|
||||||
|
#include <Processors/Formats/IOutputFormat.h>
|
||||||
|
#include <DataStreams/BlockStreamProfileInfo.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
/// Output format which is used in PullingPipelineExecutor.
|
||||||
|
class PullingOutputFormat : public IOutputFormat
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
explicit PullingOutputFormat(const Block & header, std::atomic_bool & consume_data_flag_)
|
||||||
|
: IOutputFormat(header, out)
|
||||||
|
, has_data_flag(consume_data_flag_)
|
||||||
|
{}
|
||||||
|
|
||||||
|
String getName() const override { return "PullingOutputFormat"; }
|
||||||
|
|
||||||
|
Chunk getChunk();
|
||||||
|
Chunk getTotals();
|
||||||
|
Chunk getExtremes();
|
||||||
|
|
||||||
|
BlockStreamProfileInfo & getProfileInfo() { return info; }
|
||||||
|
|
||||||
|
void setRowsBeforeLimit(size_t rows_before_limit) override;
|
||||||
|
|
||||||
|
protected:
|
||||||
|
void consume(Chunk chunk) override;
|
||||||
|
void consumeTotals(Chunk chunk) override { totals = std::move(chunk); }
|
||||||
|
void consumeExtremes(Chunk chunk) override { extremes = std::move(chunk); }
|
||||||
|
|
||||||
|
private:
|
||||||
|
Chunk data;
|
||||||
|
Chunk totals;
|
||||||
|
Chunk extremes;
|
||||||
|
|
||||||
|
std::atomic_bool & has_data_flag;
|
||||||
|
|
||||||
|
BlockStreamProfileInfo info;
|
||||||
|
|
||||||
|
/// Is not used.
|
||||||
|
static WriteBuffer out;
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
@ -10,7 +10,9 @@ SRCS(
|
|||||||
Chunk.cpp
|
Chunk.cpp
|
||||||
ConcatProcessor.cpp
|
ConcatProcessor.cpp
|
||||||
DelayedPortsProcessor.cpp
|
DelayedPortsProcessor.cpp
|
||||||
|
Executors/PipelineExecutingBlockInputStream.cpp
|
||||||
Executors/PipelineExecutor.cpp
|
Executors/PipelineExecutor.cpp
|
||||||
|
Executors/PullingAsyncPipelineExecutor.cpp
|
||||||
Executors/PullingPipelineExecutor.cpp
|
Executors/PullingPipelineExecutor.cpp
|
||||||
Executors/TreeExecutorBlockInputStream.cpp
|
Executors/TreeExecutorBlockInputStream.cpp
|
||||||
ForkProcessor.cpp
|
ForkProcessor.cpp
|
||||||
@ -57,6 +59,7 @@ SRCS(
|
|||||||
Formats/IRowOutputFormat.cpp
|
Formats/IRowOutputFormat.cpp
|
||||||
Formats/LazyOutputFormat.cpp
|
Formats/LazyOutputFormat.cpp
|
||||||
Formats/OutputStreamToOutputFormat.cpp
|
Formats/OutputStreamToOutputFormat.cpp
|
||||||
|
Formats/PullingOutputFormat.cpp
|
||||||
Formats/RowInputFormatWithDiagnosticInfo.cpp
|
Formats/RowInputFormatWithDiagnosticInfo.cpp
|
||||||
IAccumulatingTransform.cpp
|
IAccumulatingTransform.cpp
|
||||||
IInflatingTransform.cpp
|
IInflatingTransform.cpp
|
||||||
|
Loading…
Reference in New Issue
Block a user