2019-02-06 16:31:18 +00:00
|
|
|
#include <Processors/Executors/PipelineExecutor.h>
|
|
|
|
#include <unordered_map>
|
|
|
|
#include <queue>
|
2019-02-08 16:10:57 +00:00
|
|
|
#include <IO/WriteBufferFromString.h>
|
|
|
|
#include <Processors/printPipeline.h>
|
|
|
|
#include <Common/EventCounter.h>
|
2019-04-05 10:52:07 +00:00
|
|
|
#include <ext/scope_guard.h>
|
2019-04-29 11:04:03 +00:00
|
|
|
#include <Common/CurrentThread.h>
|
2019-02-06 16:31:18 +00:00
|
|
|
|
2019-05-16 14:57:27 +00:00
|
|
|
#include <Common/Stopwatch.h>
|
2019-09-09 15:43:19 +00:00
|
|
|
#include <Processors/ISource.h>
|
2019-11-09 19:46:46 +00:00
|
|
|
#include <Common/setThreadName.h>
|
2019-05-16 14:57:27 +00:00
|
|
|
|
2019-12-24 17:00:43 +00:00
|
|
|
#if !defined(__APPLE__) && !defined(__FreeBSD__)
|
2019-12-23 16:55:11 +00:00
|
|
|
#include <sched.h>
|
2019-12-24 17:00:43 +00:00
|
|
|
#endif
|
2019-12-23 16:55:11 +00:00
|
|
|
|
2019-02-06 16:31:18 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2019-04-26 17:21:14 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int TOO_MANY_ROWS_OR_BYTES;
|
2019-04-26 17:33:46 +00:00
|
|
|
extern const int QUOTA_EXPIRED;
|
2019-05-14 13:39:56 +00:00
|
|
|
extern const int QUERY_WAS_CANCELLED;
|
2019-04-26 17:33:46 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
static bool checkCanAddAdditionalInfoToException(const DB::Exception & exception)
|
|
|
|
{
|
2019-05-14 13:39:56 +00:00
|
|
|
/// Don't add additional info to limits and quota exceptions, and in case of kill query (to pass tests).
|
2019-04-26 17:33:46 +00:00
|
|
|
return exception.code() != ErrorCodes::TOO_MANY_ROWS_OR_BYTES
|
2019-05-14 13:39:56 +00:00
|
|
|
&& exception.code() != ErrorCodes::QUOTA_EXPIRED
|
|
|
|
&& exception.code() != ErrorCodes::QUERY_WAS_CANCELLED;
|
2019-04-26 17:21:14 +00:00
|
|
|
}
|
|
|
|
|
2019-08-03 11:02:40 +00:00
|
|
|
PipelineExecutor::PipelineExecutor(Processors & processors_)
|
|
|
|
: processors(processors_)
|
2019-06-26 15:57:40 +00:00
|
|
|
, cancelled(false)
|
|
|
|
, finished(false)
|
2019-06-27 16:17:02 +00:00
|
|
|
, num_processing_executors(0)
|
|
|
|
, expand_pipeline_task(nullptr)
|
2019-02-06 16:31:18 +00:00
|
|
|
{
|
2019-02-08 16:10:57 +00:00
|
|
|
buildGraph();
|
2019-02-06 16:31:18 +00:00
|
|
|
}
|
|
|
|
|
2019-04-16 11:40:15 +00:00
|
|
|
bool PipelineExecutor::addEdges(UInt64 node)
|
2019-02-06 16:31:18 +00:00
|
|
|
{
|
|
|
|
auto throwUnknownProcessor = [](const IProcessor * proc, const IProcessor * parent, bool from_input_port)
|
|
|
|
{
|
|
|
|
String msg = "Processor " + proc->getName() + " was found as " + (from_input_port ? "input" : "output")
|
2019-02-27 12:51:27 +00:00
|
|
|
+ " for processor " + parent->getName() + ", but not found in list of processors.";
|
2019-02-06 16:31:18 +00:00
|
|
|
|
|
|
|
throw Exception(msg, ErrorCodes::LOGICAL_ERROR);
|
|
|
|
};
|
|
|
|
|
2019-02-27 12:51:27 +00:00
|
|
|
const IProcessor * cur = graph[node].processor;
|
2019-02-06 16:31:18 +00:00
|
|
|
|
2019-11-27 16:24:44 +00:00
|
|
|
auto add_edge = [&](auto & from_port, const IProcessor * to_proc, Edges & edges,
|
2019-12-02 17:32:16 +00:00
|
|
|
bool is_backward, UInt64 input_port_number, UInt64 output_port_number,
|
|
|
|
std::vector<void *> * update_list)
|
2019-02-06 16:31:18 +00:00
|
|
|
{
|
2019-02-27 12:51:27 +00:00
|
|
|
auto it = processors_map.find(to_proc);
|
|
|
|
if (it == processors_map.end())
|
|
|
|
throwUnknownProcessor(to_proc, cur, true);
|
2019-02-06 16:31:18 +00:00
|
|
|
|
2019-02-27 12:51:27 +00:00
|
|
|
UInt64 proc_num = it->second;
|
2019-12-02 17:32:16 +00:00
|
|
|
auto & edge = edges.emplace_back(proc_num, is_backward, input_port_number, output_port_number, update_list);
|
2019-11-27 16:24:44 +00:00
|
|
|
|
2019-12-02 17:32:16 +00:00
|
|
|
from_port.setUpdateInfo(&edge.update_info);
|
2019-02-27 12:51:27 +00:00
|
|
|
};
|
2019-02-06 16:31:18 +00:00
|
|
|
|
2019-04-16 11:40:15 +00:00
|
|
|
bool was_edge_added = false;
|
|
|
|
|
2019-02-27 12:51:27 +00:00
|
|
|
auto & inputs = processors[node]->getInputs();
|
2019-04-16 11:40:15 +00:00
|
|
|
auto from_input = graph[node].backEdges.size();
|
|
|
|
|
|
|
|
if (from_input < inputs.size())
|
2019-02-27 12:51:27 +00:00
|
|
|
{
|
2019-04-16 11:40:15 +00:00
|
|
|
was_edge_added = true;
|
|
|
|
|
2019-11-27 16:24:44 +00:00
|
|
|
for (auto it = std::next(inputs.begin(), from_input); it != inputs.end(); ++it, ++from_input)
|
2019-04-16 11:40:15 +00:00
|
|
|
{
|
|
|
|
const IProcessor * proc = &it->getOutputPort().getProcessor();
|
2019-11-27 16:24:44 +00:00
|
|
|
auto output_port_number = proc->getOutputPortNumber(&it->getOutputPort());
|
2019-12-02 17:32:16 +00:00
|
|
|
add_edge(*it, proc, graph[node].backEdges, true, from_input, output_port_number, &graph[node].post_updated_input_ports);
|
2019-04-16 11:40:15 +00:00
|
|
|
}
|
2019-02-27 12:51:27 +00:00
|
|
|
}
|
2019-02-08 16:10:57 +00:00
|
|
|
|
2019-02-27 12:51:27 +00:00
|
|
|
auto & outputs = processors[node]->getOutputs();
|
2019-04-16 11:40:15 +00:00
|
|
|
auto from_output = graph[node].directEdges.size();
|
|
|
|
|
|
|
|
if (from_output < outputs.size())
|
2019-02-27 12:51:27 +00:00
|
|
|
{
|
2019-04-16 11:40:15 +00:00
|
|
|
was_edge_added = true;
|
|
|
|
|
2019-11-27 16:24:44 +00:00
|
|
|
for (auto it = std::next(outputs.begin(), from_output); it != outputs.end(); ++it, ++from_output)
|
2019-04-16 11:40:15 +00:00
|
|
|
{
|
|
|
|
const IProcessor * proc = &it->getInputPort().getProcessor();
|
2019-11-27 16:24:44 +00:00
|
|
|
auto input_port_number = proc->getInputPortNumber(&it->getInputPort());
|
2019-12-02 17:32:16 +00:00
|
|
|
add_edge(*it, proc, graph[node].directEdges, false, input_port_number, from_output, &graph[node].post_updated_output_ports);
|
2019-04-16 11:40:15 +00:00
|
|
|
}
|
2019-02-27 12:51:27 +00:00
|
|
|
}
|
2019-04-16 11:40:15 +00:00
|
|
|
|
|
|
|
return was_edge_added;
|
2019-02-27 12:51:27 +00:00
|
|
|
}
|
2019-02-06 16:31:18 +00:00
|
|
|
|
2019-02-27 12:51:27 +00:00
|
|
|
void PipelineExecutor::buildGraph()
|
|
|
|
{
|
|
|
|
UInt64 num_processors = processors.size();
|
2019-02-08 16:10:57 +00:00
|
|
|
|
2019-06-18 13:08:22 +00:00
|
|
|
graph.reserve(num_processors);
|
2019-02-27 12:51:27 +00:00
|
|
|
for (UInt64 node = 0; node < num_processors; ++node)
|
|
|
|
{
|
|
|
|
IProcessor * proc = processors[node].get();
|
|
|
|
processors_map[proc] = node;
|
2019-06-18 13:08:22 +00:00
|
|
|
graph.emplace_back(proc, node);
|
2019-02-06 16:31:18 +00:00
|
|
|
}
|
2019-02-27 12:51:27 +00:00
|
|
|
|
|
|
|
for (UInt64 node = 0; node < num_processors; ++node)
|
2019-04-16 11:40:15 +00:00
|
|
|
addEdges(node);
|
2019-02-06 16:31:18 +00:00
|
|
|
}
|
|
|
|
|
2019-06-26 15:57:40 +00:00
|
|
|
void PipelineExecutor::addChildlessProcessorsToStack(Stack & stack)
|
2019-02-08 16:10:57 +00:00
|
|
|
{
|
|
|
|
UInt64 num_processors = processors.size();
|
|
|
|
for (UInt64 proc = 0; proc < num_processors; ++proc)
|
|
|
|
{
|
|
|
|
if (graph[proc].directEdges.empty())
|
|
|
|
{
|
2019-06-19 18:30:02 +00:00
|
|
|
stack.push(proc);
|
2019-11-27 16:24:44 +00:00
|
|
|
/// do not lock mutex, as this function is executedin single thread
|
2019-02-08 16:10:57 +00:00
|
|
|
graph[proc].status = ExecStatus::Preparing;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-09-06 13:28:49 +00:00
|
|
|
static void executeJob(IProcessor * processor)
|
2019-04-12 11:18:18 +00:00
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
2019-09-06 13:28:49 +00:00
|
|
|
processor->work();
|
2019-04-12 11:18:18 +00:00
|
|
|
}
|
2019-04-26 17:33:46 +00:00
|
|
|
catch (Exception & exception)
|
2019-04-12 11:18:18 +00:00
|
|
|
{
|
2019-04-26 17:33:46 +00:00
|
|
|
if (checkCanAddAdditionalInfoToException(exception))
|
|
|
|
exception.addMessage("While executing " + processor->getName() + " ("
|
|
|
|
+ toString(reinterpret_cast<std::uintptr_t>(processor)) + ") ");
|
2019-04-12 11:27:25 +00:00
|
|
|
throw;
|
2019-04-12 11:18:18 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-06-18 13:08:22 +00:00
|
|
|
void PipelineExecutor::addJob(ExecutionState * execution_state)
|
2019-02-08 16:10:57 +00:00
|
|
|
{
|
2019-09-06 13:28:49 +00:00
|
|
|
auto job = [execution_state]()
|
2019-02-08 16:10:57 +00:00
|
|
|
{
|
2019-06-26 15:57:40 +00:00
|
|
|
try
|
2019-05-16 14:57:27 +00:00
|
|
|
{
|
2019-09-02 11:02:08 +00:00
|
|
|
// Stopwatch watch;
|
2019-09-06 13:28:49 +00:00
|
|
|
executeJob(execution_state->processor);
|
2019-09-02 11:02:08 +00:00
|
|
|
// execution_state->execution_time_ns += watch.elapsed();
|
2019-05-16 14:57:27 +00:00
|
|
|
|
2019-06-26 15:57:40 +00:00
|
|
|
++execution_state->num_executed_jobs;
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
execution_state->exception = std::current_exception();
|
|
|
|
}
|
|
|
|
};
|
2019-02-08 16:10:57 +00:00
|
|
|
|
2019-06-26 15:57:40 +00:00
|
|
|
execution_state->job = std::move(job);
|
2019-02-08 16:10:57 +00:00
|
|
|
}
|
|
|
|
|
2019-12-30 18:29:24 +00:00
|
|
|
bool PipelineExecutor::expandPipeline(Stack & stack, UInt64 pid)
|
2019-02-27 12:51:27 +00:00
|
|
|
{
|
|
|
|
auto & cur_node = graph[pid];
|
2019-12-30 18:29:24 +00:00
|
|
|
Processors new_processors;
|
|
|
|
|
|
|
|
try
|
|
|
|
{
|
|
|
|
new_processors = cur_node.processor->expandPipeline();
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
cur_node.execution_state->exception = std::current_exception();
|
|
|
|
return false;
|
|
|
|
}
|
2019-02-27 12:51:27 +00:00
|
|
|
|
|
|
|
for (const auto & processor : new_processors)
|
|
|
|
{
|
2019-04-16 11:40:15 +00:00
|
|
|
if (processors_map.count(processor.get()))
|
|
|
|
throw Exception("Processor " + processor->getName() + " was already added to pipeline.",
|
|
|
|
ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
2019-02-27 12:51:27 +00:00
|
|
|
processors_map[processor.get()] = graph.size();
|
2019-06-18 13:08:22 +00:00
|
|
|
graph.emplace_back(processor.get(), graph.size());
|
2019-02-27 12:51:27 +00:00
|
|
|
}
|
|
|
|
|
2019-08-01 14:25:41 +00:00
|
|
|
{
|
|
|
|
std::lock_guard guard(processors_mutex);
|
|
|
|
processors.insert(processors.end(), new_processors.begin(), new_processors.end());
|
|
|
|
}
|
2019-02-27 12:51:27 +00:00
|
|
|
|
2019-08-01 14:25:41 +00:00
|
|
|
UInt64 num_processors = processors.size();
|
2019-04-16 11:40:15 +00:00
|
|
|
for (UInt64 node = 0; node < num_processors; ++node)
|
2019-02-27 12:51:27 +00:00
|
|
|
{
|
2019-11-27 16:24:44 +00:00
|
|
|
size_t num_direct_edges = graph[node].directEdges.size();
|
|
|
|
size_t num_back_edges = graph[node].backEdges.size();
|
|
|
|
|
2019-04-16 11:40:15 +00:00
|
|
|
if (addEdges(node))
|
|
|
|
{
|
2019-11-27 16:24:44 +00:00
|
|
|
std::lock_guard guard(graph[node].status_mutex);
|
|
|
|
|
|
|
|
for (; num_back_edges < graph[node].backEdges.size(); ++num_back_edges)
|
|
|
|
graph[node].updated_input_ports.emplace_back(num_back_edges);
|
|
|
|
|
|
|
|
for (; num_direct_edges < graph[node].directEdges.size(); ++num_direct_edges)
|
|
|
|
graph[node].updated_output_ports.emplace_back(num_direct_edges);
|
|
|
|
|
|
|
|
if (graph[node].status == ExecStatus::Idle)
|
2019-04-16 11:40:15 +00:00
|
|
|
{
|
|
|
|
graph[node].status = ExecStatus::Preparing;
|
2019-06-19 18:30:02 +00:00
|
|
|
stack.push(node);
|
2019-04-16 11:40:15 +00:00
|
|
|
}
|
|
|
|
}
|
2019-02-27 12:51:27 +00:00
|
|
|
}
|
2019-12-30 18:29:24 +00:00
|
|
|
|
|
|
|
return true;
|
2019-02-27 12:51:27 +00:00
|
|
|
}
|
|
|
|
|
2019-12-30 18:29:24 +00:00
|
|
|
bool PipelineExecutor::tryAddProcessorToStackIfUpdated(Edge & edge, Queue & queue, size_t thread_number)
|
2019-06-14 09:35:38 +00:00
|
|
|
{
|
2019-06-19 18:30:02 +00:00
|
|
|
/// In this method we have ownership on edge, but node can be concurrently accessed.
|
|
|
|
|
2019-06-14 09:35:38 +00:00
|
|
|
auto & node = graph[edge.to];
|
|
|
|
|
2019-12-26 12:58:09 +00:00
|
|
|
std::unique_lock lock(node.status_mutex);
|
2019-06-19 18:30:02 +00:00
|
|
|
|
2019-11-27 16:24:44 +00:00
|
|
|
ExecStatus status = node.status;
|
2019-06-19 18:30:02 +00:00
|
|
|
|
|
|
|
if (status == ExecStatus::Finished)
|
2019-12-30 18:29:24 +00:00
|
|
|
return true;
|
2019-06-14 09:35:38 +00:00
|
|
|
|
2019-11-27 16:24:44 +00:00
|
|
|
if (edge.backward)
|
|
|
|
node.updated_output_ports.push_back(edge.output_port_number);
|
|
|
|
else
|
|
|
|
node.updated_input_ports.push_back(edge.input_port_number);
|
2019-06-14 09:35:38 +00:00
|
|
|
|
2019-11-27 16:24:44 +00:00
|
|
|
if (status == ExecStatus::Idle)
|
|
|
|
{
|
|
|
|
node.status = ExecStatus::Preparing;
|
2019-12-30 18:29:24 +00:00
|
|
|
return prepareProcessor(edge.to, thread_number, queue, std::move(lock));
|
2019-11-27 16:24:44 +00:00
|
|
|
}
|
2019-12-30 18:29:24 +00:00
|
|
|
|
|
|
|
return true;
|
2019-06-14 09:35:38 +00:00
|
|
|
}
|
|
|
|
|
2019-12-30 18:29:24 +00:00
|
|
|
bool PipelineExecutor::prepareProcessor(UInt64 pid, size_t thread_number, Queue & queue, std::unique_lock<std::mutex> node_lock)
|
2019-02-06 16:31:18 +00:00
|
|
|
{
|
2019-06-19 18:30:02 +00:00
|
|
|
/// In this method we have ownership on node.
|
2019-02-06 16:31:18 +00:00
|
|
|
auto & node = graph[pid];
|
2019-06-17 09:24:56 +00:00
|
|
|
|
2019-11-27 16:24:44 +00:00
|
|
|
bool need_traverse = false;
|
|
|
|
bool need_expand_pipeline = false;
|
|
|
|
|
|
|
|
std::vector<Edge *> updated_back_edges;
|
|
|
|
std::vector<Edge *> updated_direct_edges;
|
|
|
|
|
2019-06-17 09:24:56 +00:00
|
|
|
{
|
2019-06-19 18:30:02 +00:00
|
|
|
/// Stopwatch watch;
|
|
|
|
|
2019-12-26 12:58:09 +00:00
|
|
|
std::unique_lock<std::mutex> lock(std::move(node_lock));
|
2019-06-19 18:30:02 +00:00
|
|
|
|
2019-12-30 18:29:24 +00:00
|
|
|
try
|
|
|
|
{
|
|
|
|
node.last_processor_status = node.processor->prepare(node.updated_input_ports, node.updated_output_ports);
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
node.execution_state->exception = std::current_exception();
|
|
|
|
return false;
|
|
|
|
}
|
2019-06-17 09:24:56 +00:00
|
|
|
|
2019-06-19 18:30:02 +00:00
|
|
|
/// node.execution_state->preparation_time_ns += watch.elapsed();
|
2019-12-30 18:29:24 +00:00
|
|
|
|
|
|
|
node.updated_input_ports.clear();
|
|
|
|
node.updated_output_ports.clear();
|
2019-06-14 17:08:32 +00:00
|
|
|
|
2019-11-27 16:24:44 +00:00
|
|
|
switch (node.last_processor_status)
|
|
|
|
{
|
|
|
|
case IProcessor::Status::NeedData:
|
|
|
|
case IProcessor::Status::PortFull:
|
|
|
|
{
|
|
|
|
need_traverse = true;
|
|
|
|
node.status = ExecStatus::Idle;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
case IProcessor::Status::Finished:
|
|
|
|
{
|
|
|
|
need_traverse = true;
|
|
|
|
node.status = ExecStatus::Finished;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
case IProcessor::Status::Ready:
|
|
|
|
{
|
|
|
|
node.status = ExecStatus::Executing;
|
2019-12-26 14:23:34 +00:00
|
|
|
queue.push(node.execution_state.get());
|
|
|
|
break;
|
2019-11-27 16:24:44 +00:00
|
|
|
}
|
|
|
|
case IProcessor::Status::Async:
|
|
|
|
{
|
|
|
|
throw Exception("Async is temporary not supported.", ErrorCodes::LOGICAL_ERROR);
|
2019-06-19 18:30:02 +00:00
|
|
|
|
2019-11-27 16:24:44 +00:00
|
|
|
// node.status = ExecStatus::Executing;
|
|
|
|
// addAsyncJob(pid);
|
|
|
|
// break;
|
|
|
|
}
|
|
|
|
case IProcessor::Status::Wait:
|
|
|
|
{
|
2019-12-26 12:58:09 +00:00
|
|
|
throw Exception("Wait is temporary not supported.", ErrorCodes::LOGICAL_ERROR);
|
2019-11-27 16:24:44 +00:00
|
|
|
}
|
|
|
|
case IProcessor::Status::ExpandPipeline:
|
|
|
|
{
|
|
|
|
need_expand_pipeline = true;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
2019-06-19 18:30:02 +00:00
|
|
|
|
2019-11-27 16:24:44 +00:00
|
|
|
if (need_traverse)
|
2019-06-19 18:30:02 +00:00
|
|
|
{
|
2019-12-02 17:32:16 +00:00
|
|
|
for (auto & edge_id : node.post_updated_input_ports)
|
2019-11-27 16:24:44 +00:00
|
|
|
{
|
2019-12-02 17:32:16 +00:00
|
|
|
auto edge = static_cast<Edge *>(edge_id);
|
|
|
|
updated_back_edges.emplace_back(edge);
|
|
|
|
edge->update_info.trigger();
|
2019-11-27 16:24:44 +00:00
|
|
|
}
|
2019-06-19 18:30:02 +00:00
|
|
|
|
2019-12-02 17:32:16 +00:00
|
|
|
for (auto & edge_id : node.post_updated_output_ports)
|
2019-11-27 16:24:44 +00:00
|
|
|
{
|
2019-12-02 17:32:16 +00:00
|
|
|
auto edge = static_cast<Edge *>(edge_id);
|
|
|
|
updated_direct_edges.emplace_back(edge);
|
|
|
|
edge->update_info.trigger();
|
2019-11-27 16:24:44 +00:00
|
|
|
}
|
2019-12-02 17:32:16 +00:00
|
|
|
|
|
|
|
node.post_updated_input_ports.clear();
|
|
|
|
node.post_updated_output_ports.clear();
|
2019-06-19 18:30:02 +00:00
|
|
|
}
|
2019-11-27 16:24:44 +00:00
|
|
|
}
|
2019-02-06 16:31:18 +00:00
|
|
|
|
2019-11-27 16:24:44 +00:00
|
|
|
if (need_traverse)
|
2019-02-06 16:31:18 +00:00
|
|
|
{
|
2019-11-27 16:24:44 +00:00
|
|
|
for (auto & edge : updated_direct_edges)
|
2019-12-30 18:29:24 +00:00
|
|
|
{
|
|
|
|
if (!tryAddProcessorToStackIfUpdated(*edge, queue, thread_number))
|
|
|
|
return false;
|
|
|
|
}
|
2019-12-26 12:58:09 +00:00
|
|
|
|
|
|
|
for (auto & edge : updated_back_edges)
|
2019-12-30 18:29:24 +00:00
|
|
|
{
|
|
|
|
if (!tryAddProcessorToStackIfUpdated(*edge, queue, thread_number))
|
|
|
|
return false;
|
|
|
|
}
|
2019-11-27 16:24:44 +00:00
|
|
|
}
|
2019-06-19 18:30:02 +00:00
|
|
|
|
2019-11-27 16:24:44 +00:00
|
|
|
if (need_expand_pipeline)
|
|
|
|
{
|
2019-12-26 12:58:09 +00:00
|
|
|
Stack stack;
|
|
|
|
|
2019-11-27 16:24:44 +00:00
|
|
|
executor_contexts[thread_number]->task_list.emplace_back(
|
2019-06-27 16:17:02 +00:00
|
|
|
node.execution_state.get(),
|
2019-12-26 12:58:09 +00:00
|
|
|
&stack
|
2019-11-27 16:24:44 +00:00
|
|
|
);
|
2019-03-15 17:06:32 +00:00
|
|
|
|
2019-11-27 16:24:44 +00:00
|
|
|
ExpandPipelineTask * desired = &executor_contexts[thread_number]->task_list.back();
|
|
|
|
ExpandPipelineTask * expected = nullptr;
|
2019-06-19 18:30:02 +00:00
|
|
|
|
2019-11-27 16:24:44 +00:00
|
|
|
while (!expand_pipeline_task.compare_exchange_strong(expected, desired))
|
|
|
|
{
|
2019-12-30 18:29:24 +00:00
|
|
|
if (!doExpandPipeline(expected, true))
|
|
|
|
return false;
|
|
|
|
|
2019-11-27 16:24:44 +00:00
|
|
|
expected = nullptr;
|
|
|
|
}
|
2019-06-19 18:30:02 +00:00
|
|
|
|
2019-12-30 18:29:24 +00:00
|
|
|
if (!doExpandPipeline(desired, true))
|
|
|
|
return false;
|
2019-12-26 12:58:09 +00:00
|
|
|
|
2019-12-30 20:09:37 +00:00
|
|
|
/// Add itself back to be prepared again.
|
|
|
|
stack.push(pid);
|
|
|
|
|
|
|
|
while (!stack.empty())
|
|
|
|
{
|
|
|
|
auto item = stack.top();
|
|
|
|
if (!prepareProcessor(item, thread_number, queue, std::unique_lock<std::mutex>(graph[item].status_mutex)))
|
|
|
|
return false;
|
|
|
|
|
|
|
|
stack.pop();
|
|
|
|
}
|
2019-02-06 16:31:18 +00:00
|
|
|
}
|
2019-12-30 18:29:24 +00:00
|
|
|
|
|
|
|
return true;
|
2019-02-06 16:31:18 +00:00
|
|
|
}
|
|
|
|
|
2019-12-30 18:29:24 +00:00
|
|
|
bool PipelineExecutor::doExpandPipeline(ExpandPipelineTask * task, bool processing)
|
2019-06-14 09:35:38 +00:00
|
|
|
{
|
2019-06-27 16:17:02 +00:00
|
|
|
std::unique_lock lock(task->mutex);
|
2019-07-07 14:57:50 +00:00
|
|
|
|
|
|
|
if (processing)
|
|
|
|
++task->num_waiting_processing_threads;
|
2019-06-14 09:35:38 +00:00
|
|
|
|
2019-06-27 16:17:02 +00:00
|
|
|
task->condvar.wait(lock, [&]()
|
2019-06-19 18:30:02 +00:00
|
|
|
{
|
2019-07-07 14:57:50 +00:00
|
|
|
return task->num_waiting_processing_threads >= num_processing_executors || expand_pipeline_task != task;
|
2019-06-19 18:30:02 +00:00
|
|
|
});
|
2019-06-14 09:35:38 +00:00
|
|
|
|
2019-12-30 19:45:27 +00:00
|
|
|
bool result = true;
|
2019-12-30 18:29:24 +00:00
|
|
|
|
2019-06-27 16:17:02 +00:00
|
|
|
/// After condvar.wait() task may point to trash. Can change it only if it is still in expand_pipeline_task.
|
|
|
|
if (expand_pipeline_task == task)
|
2019-02-06 16:31:18 +00:00
|
|
|
{
|
2019-12-30 18:29:24 +00:00
|
|
|
result = expandPipeline(*task->stack, task->node_to_expand->processors_id);
|
2019-06-26 15:57:40 +00:00
|
|
|
|
2019-06-27 16:17:02 +00:00
|
|
|
expand_pipeline_task = nullptr;
|
2019-06-26 15:57:40 +00:00
|
|
|
|
2019-06-19 18:30:02 +00:00
|
|
|
lock.unlock();
|
2019-06-27 16:17:02 +00:00
|
|
|
task->condvar.notify_all();
|
2019-02-06 16:31:18 +00:00
|
|
|
}
|
2019-12-30 18:29:24 +00:00
|
|
|
|
|
|
|
return result;
|
2019-02-06 16:31:18 +00:00
|
|
|
}
|
|
|
|
|
2019-08-01 14:25:41 +00:00
|
|
|
void PipelineExecutor::cancel()
|
|
|
|
{
|
|
|
|
cancelled = true;
|
|
|
|
finish();
|
|
|
|
|
|
|
|
std::lock_guard guard(processors_mutex);
|
|
|
|
for (auto & processor : processors)
|
|
|
|
processor->cancel();
|
|
|
|
}
|
|
|
|
|
2019-06-25 17:19:32 +00:00
|
|
|
void PipelineExecutor::finish()
|
2019-06-24 15:14:58 +00:00
|
|
|
{
|
2019-06-26 15:57:40 +00:00
|
|
|
{
|
2019-07-07 13:14:02 +00:00
|
|
|
std::lock_guard lock(task_queue_mutex);
|
|
|
|
finished = true;
|
2019-06-26 15:57:40 +00:00
|
|
|
}
|
2019-07-07 13:14:02 +00:00
|
|
|
|
2019-09-10 13:12:18 +00:00
|
|
|
std::lock_guard guard(executor_contexts_mutex);
|
|
|
|
|
2019-09-04 12:34:15 +00:00
|
|
|
for (auto & context : executor_contexts)
|
|
|
|
{
|
|
|
|
{
|
|
|
|
std::lock_guard lock(context->mutex);
|
|
|
|
context->wake_flag = true;
|
|
|
|
}
|
|
|
|
|
|
|
|
context->condvar.notify_one();
|
|
|
|
}
|
2019-06-24 15:14:58 +00:00
|
|
|
}
|
|
|
|
|
2019-05-16 14:57:27 +00:00
|
|
|
void PipelineExecutor::execute(size_t num_threads)
|
2019-02-06 16:31:18 +00:00
|
|
|
{
|
2019-04-12 09:57:13 +00:00
|
|
|
try
|
|
|
|
{
|
2019-05-16 14:57:27 +00:00
|
|
|
executeImpl(num_threads);
|
2019-06-26 15:57:40 +00:00
|
|
|
|
|
|
|
/// 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);
|
2019-04-12 09:57:13 +00:00
|
|
|
}
|
2019-04-26 17:33:46 +00:00
|
|
|
catch (Exception & exception)
|
2019-04-12 09:57:13 +00:00
|
|
|
{
|
2019-04-26 17:33:46 +00:00
|
|
|
if (checkCanAddAdditionalInfoToException(exception))
|
|
|
|
exception.addMessage("\nCurrent state:\n" + dumpPipeline());
|
2019-04-26 17:21:14 +00:00
|
|
|
|
2019-04-12 09:57:13 +00:00
|
|
|
throw;
|
|
|
|
}
|
|
|
|
|
2019-05-14 13:39:56 +00:00
|
|
|
if (cancelled)
|
|
|
|
return;
|
|
|
|
|
2019-04-12 09:57:13 +00:00
|
|
|
bool all_processors_finished = true;
|
|
|
|
for (auto & node : graph)
|
2019-11-27 16:24:44 +00:00
|
|
|
if (node.status != ExecStatus::Finished) /// Single thread, do not hold mutex
|
2019-04-12 09:57:13 +00:00
|
|
|
all_processors_finished = false;
|
|
|
|
|
|
|
|
if (!all_processors_finished)
|
|
|
|
throw Exception("Pipeline stuck. Current state:\n" + dumpPipeline(), ErrorCodes::LOGICAL_ERROR);
|
|
|
|
}
|
|
|
|
|
2019-07-07 13:14:02 +00:00
|
|
|
void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads)
|
2019-04-12 09:57:13 +00:00
|
|
|
{
|
2019-12-24 13:55:28 +00:00
|
|
|
#if !defined(__APPLE__) && !defined(__FreeBSD__)
|
2019-12-23 16:55:11 +00:00
|
|
|
/// Specify CPU core for thread if can.
|
|
|
|
/// It may reduce the number of context swithches.
|
2020-01-02 06:27:53 +00:00
|
|
|
/*
|
2020-01-01 18:42:15 +00:00
|
|
|
if (num_threads > 1)
|
|
|
|
{
|
|
|
|
cpu_set_t cpu_set;
|
|
|
|
CPU_ZERO(&cpu_set);
|
|
|
|
CPU_SET(thread_num, &cpu_set);
|
|
|
|
|
|
|
|
if (sched_setaffinity(0, sizeof(cpu_set_t), &cpu_set) == -1)
|
|
|
|
LOG_TRACE(log, "Cannot set affinity for thread " << num_threads);
|
|
|
|
}
|
2020-01-02 06:27:53 +00:00
|
|
|
*/
|
2019-12-24 13:55:28 +00:00
|
|
|
#endif
|
|
|
|
|
2019-12-28 18:42:07 +00:00
|
|
|
// UInt64 total_time_ns = 0;
|
|
|
|
// UInt64 execution_time_ns = 0;
|
|
|
|
// UInt64 processing_time_ns = 0;
|
|
|
|
// UInt64 wait_time_ns = 0;
|
2019-05-16 14:57:27 +00:00
|
|
|
|
2019-12-28 18:42:07 +00:00
|
|
|
// Stopwatch total_time_watch;
|
2019-06-24 15:14:58 +00:00
|
|
|
ExecutionState * state = nullptr;
|
2019-06-19 18:30:02 +00:00
|
|
|
|
2019-12-26 14:23:34 +00:00
|
|
|
auto prepare_processor = [&](UInt64 pid, Queue & queue)
|
2019-06-19 18:30:02 +00:00
|
|
|
{
|
2019-12-30 18:29:24 +00:00
|
|
|
if (!prepareProcessor(pid, thread_num, queue, std::unique_lock<std::mutex>(graph[pid].status_mutex)))
|
2019-06-25 17:19:32 +00:00
|
|
|
finish();
|
2019-06-19 18:30:02 +00:00
|
|
|
};
|
|
|
|
|
2019-09-11 16:31:15 +00:00
|
|
|
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();
|
|
|
|
};
|
|
|
|
|
2019-06-14 17:08:32 +00:00
|
|
|
while (!finished)
|
|
|
|
{
|
|
|
|
/// First, find any processor to execute.
|
|
|
|
/// Just travers graph and prepare any processor.
|
2019-09-04 12:34:15 +00:00
|
|
|
while (!finished)
|
2019-05-16 14:57:27 +00:00
|
|
|
{
|
2019-09-09 15:43:19 +00:00
|
|
|
{
|
2019-09-10 12:14:47 +00:00
|
|
|
std::unique_lock lock(task_queue_mutex);
|
|
|
|
|
2019-09-04 09:37:56 +00:00
|
|
|
if (!task_queue.empty())
|
|
|
|
{
|
2019-12-26 18:23:31 +00:00
|
|
|
state = task_queue.pop(thread_num);
|
2019-09-03 11:15:37 +00:00
|
|
|
|
2019-12-27 10:59:43 +00:00
|
|
|
if (!task_queue.empty() && !threads_queue.empty() /*&& task_queue.quota() > threads_queue.size()*/)
|
2019-09-04 12:34:15 +00:00
|
|
|
{
|
2019-09-12 11:13:34 +00:00
|
|
|
auto thread_to_wake = threads_queue.pop_any();
|
2019-09-04 12:34:15 +00:00
|
|
|
lock.unlock();
|
2019-09-11 16:31:15 +00:00
|
|
|
wake_up_executor(thread_to_wake);
|
2019-09-04 12:34:15 +00:00
|
|
|
}
|
2019-09-03 12:06:45 +00:00
|
|
|
|
2019-09-04 09:37:56 +00:00
|
|
|
break;
|
|
|
|
}
|
2019-06-19 18:30:02 +00:00
|
|
|
|
2019-09-10 08:45:48 +00:00
|
|
|
if (threads_queue.size() + 1 == num_threads)
|
2019-09-04 09:37:56 +00:00
|
|
|
{
|
|
|
|
lock.unlock();
|
2019-09-04 12:34:15 +00:00
|
|
|
finish();
|
2019-09-04 09:37:56 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
|
2019-09-10 08:45:48 +00:00
|
|
|
threads_queue.push(thread_num);
|
2019-09-04 12:34:15 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
{
|
|
|
|
std::unique_lock lock(executor_contexts[thread_num]->mutex);
|
|
|
|
|
|
|
|
executor_contexts[thread_num]->condvar.wait(lock, [&]
|
2019-09-04 09:37:56 +00:00
|
|
|
{
|
2019-09-04 12:34:15 +00:00
|
|
|
return finished || executor_contexts[thread_num]->wake_flag;
|
2019-09-04 09:37:56 +00:00
|
|
|
});
|
|
|
|
|
2019-09-04 12:34:15 +00:00
|
|
|
executor_contexts[thread_num]->wake_flag = false;
|
2019-09-04 09:37:56 +00:00
|
|
|
}
|
2019-05-16 14:57:27 +00:00
|
|
|
}
|
|
|
|
|
2019-06-14 17:08:32 +00:00
|
|
|
if (finished)
|
2019-06-17 09:24:56 +00:00
|
|
|
break;
|
2019-06-14 17:08:32 +00:00
|
|
|
|
2019-06-18 13:08:22 +00:00
|
|
|
while (state)
|
2019-02-06 16:31:18 +00:00
|
|
|
{
|
2019-06-14 17:08:32 +00:00
|
|
|
if (finished)
|
|
|
|
break;
|
2019-02-08 16:10:57 +00:00
|
|
|
|
2019-07-10 06:52:44 +00:00
|
|
|
addJob(state);
|
|
|
|
|
|
|
|
{
|
2019-09-02 11:02:08 +00:00
|
|
|
// Stopwatch execution_time_watch;
|
2019-09-06 13:28:49 +00:00
|
|
|
state->job();
|
2019-09-02 11:02:08 +00:00
|
|
|
// execution_time_ns += execution_time_watch.elapsed();
|
2019-07-10 06:52:44 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
if (state->exception)
|
|
|
|
finish();
|
|
|
|
|
|
|
|
if (finished)
|
|
|
|
break;
|
|
|
|
|
2019-09-02 11:02:08 +00:00
|
|
|
// Stopwatch processing_time_watch;
|
2019-06-14 17:08:32 +00:00
|
|
|
|
|
|
|
/// Try to execute neighbour processor.
|
|
|
|
{
|
2019-09-03 18:05:44 +00:00
|
|
|
Queue queue;
|
2019-06-19 18:30:02 +00:00
|
|
|
|
2019-06-27 16:17:02 +00:00
|
|
|
++num_processing_executors;
|
|
|
|
while (auto task = expand_pipeline_task.load())
|
2019-07-07 14:57:50 +00:00
|
|
|
doExpandPipeline(task, true);
|
2019-06-19 18:30:02 +00:00
|
|
|
|
2019-06-14 17:08:32 +00:00
|
|
|
/// Execute again if can.
|
2019-12-26 14:23:34 +00:00
|
|
|
prepare_processor(state->processors_id, queue);
|
2019-12-26 14:47:48 +00:00
|
|
|
state = nullptr;
|
2019-06-17 09:24:56 +00:00
|
|
|
|
2019-09-11 16:31:15 +00:00
|
|
|
/// Take local task from queue if has one.
|
2019-12-26 18:23:31 +00:00
|
|
|
if (!queue.empty())
|
2019-09-11 16:31:15 +00:00
|
|
|
{
|
|
|
|
state = queue.front();
|
|
|
|
queue.pop();
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Push other tasks to global queue.
|
2019-09-03 08:42:26 +00:00
|
|
|
if (!queue.empty())
|
2019-06-17 09:24:56 +00:00
|
|
|
{
|
2019-09-04 12:34:15 +00:00
|
|
|
std::unique_lock lock(task_queue_mutex);
|
2019-09-02 15:49:18 +00:00
|
|
|
|
2019-09-03 18:05:44 +00:00
|
|
|
while (!queue.empty() && !finished)
|
|
|
|
{
|
2019-12-26 18:23:31 +00:00
|
|
|
task_queue.push(queue.front(), thread_num);
|
2019-09-03 08:42:26 +00:00
|
|
|
queue.pop();
|
2019-09-03 18:05:44 +00:00
|
|
|
}
|
2019-06-27 16:17:02 +00:00
|
|
|
|
2019-12-27 10:59:43 +00:00
|
|
|
if (!threads_queue.empty() /* && task_queue.quota() > threads_queue.size()*/)
|
2019-09-04 12:34:15 +00:00
|
|
|
{
|
2019-09-12 11:07:36 +00:00
|
|
|
auto thread_to_wake = threads_queue.pop_any();
|
2019-09-04 12:34:15 +00:00
|
|
|
lock.unlock();
|
2019-12-26 16:15:31 +00:00
|
|
|
|
2019-12-26 16:34:08 +00:00
|
|
|
wake_up_executor(thread_to_wake);
|
2019-09-04 12:34:15 +00:00
|
|
|
}
|
2019-09-03 18:05:44 +00:00
|
|
|
}
|
2019-09-03 11:15:37 +00:00
|
|
|
|
2019-06-27 16:17:02 +00:00
|
|
|
--num_processing_executors;
|
|
|
|
while (auto task = expand_pipeline_task.load())
|
2019-07-07 14:57:50 +00:00
|
|
|
doExpandPipeline(task, false);
|
2019-02-08 16:10:57 +00:00
|
|
|
}
|
2019-06-14 17:08:32 +00:00
|
|
|
|
2019-09-02 11:02:08 +00:00
|
|
|
// processing_time_ns += processing_time_watch.elapsed();
|
2019-02-06 16:31:18 +00:00
|
|
|
}
|
|
|
|
}
|
2019-06-14 17:08:32 +00:00
|
|
|
|
2019-12-28 18:42:07 +00:00
|
|
|
// total_time_ns = total_time_watch.elapsed();
|
|
|
|
// wait_time_ns = total_time_ns - execution_time_ns - processing_time_ns;
|
|
|
|
/*
|
2019-06-14 17:08:32 +00:00
|
|
|
LOG_TRACE(log, "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.");
|
2019-12-28 18:42:07 +00:00
|
|
|
*/
|
2019-06-14 17:08:32 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void PipelineExecutor::executeImpl(size_t num_threads)
|
|
|
|
{
|
2019-06-19 18:30:02 +00:00
|
|
|
Stack stack;
|
|
|
|
|
2019-09-10 08:50:31 +00:00
|
|
|
threads_queue.init(num_threads);
|
2019-12-26 18:23:31 +00:00
|
|
|
task_queue.init(num_threads);
|
2019-09-10 08:50:31 +00:00
|
|
|
|
2019-09-10 13:12:18 +00:00
|
|
|
{
|
|
|
|
std::lock_guard guard(executor_contexts_mutex);
|
|
|
|
|
|
|
|
executor_contexts.reserve(num_threads);
|
|
|
|
for (size_t i = 0; i < num_threads; ++i)
|
|
|
|
executor_contexts.emplace_back(std::make_unique<ExecutorContext>());
|
|
|
|
}
|
2019-07-10 06:52:44 +00:00
|
|
|
|
2019-08-22 14:38:44 +00:00
|
|
|
auto thread_group = CurrentThread::getGroup();
|
|
|
|
|
|
|
|
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)
|
2019-09-09 12:09:48 +00:00
|
|
|
if (thread.joinable())
|
|
|
|
thread.join();
|
2019-08-22 14:38:44 +00:00
|
|
|
}
|
|
|
|
);
|
|
|
|
|
2019-06-26 15:57:40 +00:00
|
|
|
addChildlessProcessorsToStack(stack);
|
2019-06-19 18:30:02 +00:00
|
|
|
|
2019-06-18 13:08:22 +00:00
|
|
|
{
|
2019-08-22 14:38:44 +00:00
|
|
|
std::lock_guard lock(task_queue_mutex);
|
2019-06-18 13:08:22 +00:00
|
|
|
|
2019-12-26 14:23:34 +00:00
|
|
|
Queue queue;
|
2019-12-26 18:23:31 +00:00
|
|
|
size_t next_thread = 0;
|
2019-12-26 14:23:34 +00:00
|
|
|
|
2019-08-22 14:38:44 +00:00
|
|
|
while (!stack.empty())
|
2019-07-10 06:52:44 +00:00
|
|
|
{
|
2019-08-22 14:38:44 +00:00
|
|
|
UInt64 proc = stack.top();
|
|
|
|
stack.pop();
|
|
|
|
|
2019-12-26 14:23:34 +00:00
|
|
|
prepareProcessor(proc, 0, queue, std::unique_lock<std::mutex>(graph[proc].status_mutex));
|
|
|
|
|
|
|
|
while (!queue.empty())
|
2019-08-22 14:38:44 +00:00
|
|
|
{
|
2019-12-26 18:23:31 +00:00
|
|
|
task_queue.push(queue.front(), next_thread);
|
2019-12-26 14:23:34 +00:00
|
|
|
queue.pop();
|
2019-12-26 18:23:31 +00:00
|
|
|
|
|
|
|
++next_thread;
|
|
|
|
if (next_thread >= num_threads)
|
|
|
|
next_thread = 0;
|
2019-08-22 14:38:44 +00:00
|
|
|
}
|
2019-07-10 06:52:44 +00:00
|
|
|
}
|
2019-06-18 13:08:22 +00:00
|
|
|
}
|
2019-06-14 17:08:32 +00:00
|
|
|
|
2020-01-01 11:18:24 +00:00
|
|
|
if (num_threads > 1)
|
2019-06-26 15:57:40 +00:00
|
|
|
{
|
2020-01-01 11:18:24 +00:00
|
|
|
|
|
|
|
for (size_t i = 0; i < num_threads; ++i)
|
2019-06-14 17:08:32 +00:00
|
|
|
{
|
2020-01-01 11:18:24 +00:00
|
|
|
threads.emplace_back([this, thread_group, thread_num = i, num_threads]
|
|
|
|
{
|
|
|
|
/// ThreadStatus thread_status;
|
2019-06-14 17:08:32 +00:00
|
|
|
|
2020-01-01 11:18:24 +00:00
|
|
|
setThreadName("QueryPipelineEx");
|
2019-11-09 19:46:46 +00:00
|
|
|
|
2020-01-01 11:18:24 +00:00
|
|
|
if (thread_group)
|
|
|
|
CurrentThread::attachTo(thread_group);
|
2019-06-14 17:08:32 +00:00
|
|
|
|
2020-01-01 11:18:24 +00:00
|
|
|
SCOPE_EXIT(
|
|
|
|
if (thread_group)
|
|
|
|
CurrentThread::detachQueryIfNotDetached();
|
|
|
|
);
|
2019-06-14 17:08:32 +00:00
|
|
|
|
2020-01-01 11:18:24 +00:00
|
|
|
executeSingleThread(thread_num, num_threads);
|
|
|
|
});
|
|
|
|
}
|
2019-06-14 17:08:32 +00:00
|
|
|
|
2020-01-01 11:18:24 +00:00
|
|
|
for (auto & thread : threads)
|
|
|
|
if (thread.joinable())
|
|
|
|
thread.join();
|
|
|
|
}
|
|
|
|
else
|
|
|
|
executeSingleThread(0, num_threads);
|
2019-08-22 14:38:44 +00:00
|
|
|
|
|
|
|
finished_flag = true;
|
2019-04-12 09:57:13 +00:00
|
|
|
}
|
2019-02-06 16:31:18 +00:00
|
|
|
|
2019-04-12 09:57:13 +00:00
|
|
|
String PipelineExecutor::dumpPipeline() const
|
|
|
|
{
|
2019-05-16 14:57:27 +00:00
|
|
|
for (auto & node : graph)
|
|
|
|
{
|
|
|
|
if (node.execution_state)
|
|
|
|
node.processor->setDescription(
|
2019-06-17 09:24:56 +00:00
|
|
|
"(" + std::to_string(node.execution_state->num_executed_jobs) + " jobs, execution time: "
|
|
|
|
+ std::to_string(node.execution_state->execution_time_ns / 1e9) + " sec., preparation time: "
|
|
|
|
+ std::to_string(node.execution_state->preparation_time_ns / 1e9) + " sec.)");
|
2019-05-16 14:57:27 +00:00
|
|
|
}
|
|
|
|
|
2019-04-12 09:57:13 +00:00
|
|
|
std::vector<IProcessor::Status> statuses;
|
|
|
|
std::vector<IProcessor *> proc_list;
|
|
|
|
statuses.reserve(graph.size());
|
|
|
|
proc_list.reserve(graph.size());
|
2019-02-06 16:31:18 +00:00
|
|
|
|
2019-04-12 09:57:13 +00:00
|
|
|
for (auto & proc : graph)
|
2019-02-06 16:31:18 +00:00
|
|
|
{
|
2019-04-12 09:57:13 +00:00
|
|
|
proc_list.emplace_back(proc.processor);
|
|
|
|
statuses.emplace_back(proc.last_processor_status);
|
|
|
|
}
|
2019-02-06 16:31:18 +00:00
|
|
|
|
2019-04-12 09:57:13 +00:00
|
|
|
WriteBufferFromOwnString out;
|
|
|
|
printPipeline(processors, statuses, out);
|
|
|
|
out.finish();
|
2019-02-08 16:10:57 +00:00
|
|
|
|
2019-04-12 09:57:13 +00:00
|
|
|
return out.str();
|
2019-02-06 16:31:18 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|