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 <boost/lockfree/queue.hpp>
|
|
|
|
#include <Common/Stopwatch.h>
|
|
|
|
|
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-04-29 13:57:29 +00:00
|
|
|
PipelineExecutor::PipelineExecutor(Processors processors)
|
2019-06-18 13:08:22 +00:00
|
|
|
: processors(std::move(processors)), num_waited_tasks(0), num_tasks_to_wait(0), cancelled(false), finished(false), main_executor_flag(false), num_waiting_threads(0)
|
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-02-27 12:51:27 +00:00
|
|
|
auto add_edge = [&](auto & from_port, const IProcessor * to_proc, Edges & edges)
|
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;
|
|
|
|
Edge * edge_ptr = nullptr;
|
2019-02-06 16:31:18 +00:00
|
|
|
|
2019-02-27 12:51:27 +00:00
|
|
|
for (auto & edge : edges)
|
|
|
|
if (edge.to == proc_num)
|
|
|
|
edge_ptr = &edge;
|
2019-02-06 16:31:18 +00:00
|
|
|
|
2019-02-27 12:51:27 +00:00
|
|
|
if (!edge_ptr)
|
2019-02-06 16:31:18 +00:00
|
|
|
{
|
2019-02-27 12:51:27 +00:00
|
|
|
edge_ptr = &edges.emplace_back();
|
|
|
|
edge_ptr->to = proc_num;
|
|
|
|
}
|
2019-02-06 16:31:18 +00:00
|
|
|
|
2019-02-27 12:51:27 +00:00
|
|
|
from_port.setVersion(&edge_ptr->version);
|
|
|
|
};
|
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;
|
|
|
|
|
|
|
|
for (auto it = std::next(inputs.begin(), from_input); it != inputs.end(); ++it)
|
|
|
|
{
|
|
|
|
const IProcessor * proc = &it->getOutputPort().getProcessor();
|
|
|
|
add_edge(*it, proc, graph[node].backEdges);
|
|
|
|
}
|
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;
|
|
|
|
|
|
|
|
for (auto it = std::next(outputs.begin(), from_output); it != outputs.end(); ++it)
|
|
|
|
{
|
|
|
|
const IProcessor * proc = &it->getInputPort().getProcessor();
|
|
|
|
add_edge(*it, proc, graph[node].directEdges);
|
|
|
|
}
|
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-19 18:30:02 +00:00
|
|
|
void PipelineExecutor::addChildlessProcessorsToQueue(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-02-08 16:10:57 +00:00
|
|
|
graph[proc].status = ExecStatus::Preparing;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void PipelineExecutor::processFinishedExecutionQueue()
|
|
|
|
{
|
2019-05-16 14:57:27 +00:00
|
|
|
UInt64 finished_job = graph.size();
|
2019-02-08 16:10:57 +00:00
|
|
|
while (!finished_execution_queue.empty())
|
|
|
|
{
|
2019-05-16 14:57:27 +00:00
|
|
|
/// Should be successful as single consumer is used.
|
2019-06-14 09:35:38 +00:00
|
|
|
while (!finished_execution_queue.pop(finished_job));
|
2019-05-16 14:57:27 +00:00
|
|
|
|
2019-06-14 09:35:38 +00:00
|
|
|
auto & state = graph[finished_job].execution_state;
|
|
|
|
|
|
|
|
/// ++num_waited_tasks;
|
|
|
|
++state->num_executed_jobs;
|
2019-02-08 16:10:57 +00:00
|
|
|
|
2019-05-16 14:57:27 +00:00
|
|
|
if (graph[finished_job].execution_state->exception)
|
|
|
|
std::rethrow_exception(graph[finished_job].execution_state->exception);
|
2019-05-16 12:41:47 +00:00
|
|
|
|
2019-05-16 14:57:27 +00:00
|
|
|
graph[finished_job].status = ExecStatus::Preparing;
|
2019-06-14 17:08:32 +00:00
|
|
|
prepare_stack.push(finished_job);
|
2019-02-08 16:10:57 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-05-16 14:57:27 +00:00
|
|
|
void PipelineExecutor::processFinishedExecutionQueueSafe()
|
2019-02-08 16:10:57 +00:00
|
|
|
{
|
2019-05-16 14:57:27 +00:00
|
|
|
// if (pool)
|
|
|
|
// {
|
|
|
|
// /// std::lock_guard lock(finished_execution_mutex);
|
|
|
|
// processFinishedExecutionQueue(queue);
|
|
|
|
// }
|
|
|
|
// else
|
|
|
|
processFinishedExecutionQueue();
|
2019-02-08 16:10:57 +00:00
|
|
|
}
|
|
|
|
|
2019-04-12 11:18:18 +00:00
|
|
|
static void executeJob(IProcessor * processor)
|
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
|
|
|
processor->work();
|
|
|
|
}
|
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-19 18:30:02 +00:00
|
|
|
//bool PipelineExecutor::tryAssignJob(ExecutionState * state)
|
|
|
|
//{
|
|
|
|
// auto current_stream = state->current_stream;
|
|
|
|
// for (auto & executor_context : executor_contexts)
|
|
|
|
// {
|
|
|
|
// if (executor_context->current_stream == current_stream)
|
|
|
|
// {
|
|
|
|
// ExecutionState * expected = nullptr;
|
|
|
|
// if (executor_context->next_task_to_execute.compare_exchange_strong(expected, state))
|
|
|
|
// {
|
|
|
|
// ++num_tasks_to_wait;
|
|
|
|
// return true;
|
|
|
|
// }
|
|
|
|
// }
|
|
|
|
// }
|
|
|
|
//
|
|
|
|
// return false;
|
|
|
|
//}
|
2019-06-14 09:35:38 +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-06-14 17:08:32 +00:00
|
|
|
/// if (!threads.empty())
|
2019-02-08 16:10:57 +00:00
|
|
|
{
|
2019-06-18 13:08:22 +00:00
|
|
|
auto job = [execution_state]()
|
2019-05-16 14:57:27 +00:00
|
|
|
{
|
2019-06-14 17:08:32 +00:00
|
|
|
// SCOPE_EXIT(
|
|
|
|
/// while (!finished_execution_queue.push(pid));
|
|
|
|
/// event_counter.notify()
|
|
|
|
// );
|
2019-04-05 10:52:07 +00:00
|
|
|
|
2019-05-16 12:41:47 +00:00
|
|
|
try
|
|
|
|
{
|
2019-06-19 18:30:02 +00:00
|
|
|
Stopwatch watch;
|
2019-06-18 13:08:22 +00:00
|
|
|
executeJob(execution_state->processor);
|
2019-06-19 18:30:02 +00:00
|
|
|
execution_state->execution_time_ns += watch.elapsed();
|
2019-06-17 09:24:56 +00:00
|
|
|
|
|
|
|
++execution_state->num_executed_jobs;
|
2019-05-16 12:41:47 +00:00
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
2019-05-16 14:57:27 +00:00
|
|
|
/// Note: It's important to save exception before pushing pid to finished_execution_queue
|
|
|
|
execution_state->exception = std::current_exception();
|
2019-05-16 12:41:47 +00:00
|
|
|
}
|
2019-02-08 16:10:57 +00:00
|
|
|
};
|
|
|
|
|
2019-06-18 11:43:43 +00:00
|
|
|
execution_state->job = std::move(job);
|
2019-06-14 17:08:32 +00:00
|
|
|
/// auto * state = graph[pid].execution_state.get();
|
2019-06-14 09:35:38 +00:00
|
|
|
|
2019-06-14 17:08:32 +00:00
|
|
|
// bool is_stream_updated = false;
|
|
|
|
// if (state->need_update_stream)
|
|
|
|
// {
|
|
|
|
// is_stream_updated = true;
|
|
|
|
// state->current_stream = next_stream;
|
|
|
|
// ++next_stream;
|
|
|
|
// }
|
2019-05-16 14:57:27 +00:00
|
|
|
|
2019-06-14 09:35:38 +00:00
|
|
|
/// Try assign job to executor right now.
|
2019-06-14 17:08:32 +00:00
|
|
|
// if (is_stream_updated || !tryAssignJob(state))
|
|
|
|
// execution_states_queue.emplace_back(state);
|
2019-05-16 14:57:27 +00:00
|
|
|
|
2019-06-14 09:35:38 +00:00
|
|
|
/// while (!task_queue.push(graph[pid].execution_state.get()))
|
|
|
|
/// sleep(0);
|
2019-02-08 16:10:57 +00:00
|
|
|
}
|
2019-06-14 17:08:32 +00:00
|
|
|
// else
|
|
|
|
// {
|
|
|
|
// /// Execute task in main thread.
|
|
|
|
// executeJob(graph[pid].processor);
|
|
|
|
// while (!finished_execution_queue.push(pid));
|
|
|
|
// }
|
2019-02-08 16:10:57 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void PipelineExecutor::addAsyncJob(UInt64 pid)
|
|
|
|
{
|
|
|
|
graph[pid].processor->schedule(event_counter);
|
|
|
|
graph[pid].status = ExecStatus::Async;
|
|
|
|
++num_tasks_to_wait;
|
|
|
|
}
|
|
|
|
|
2019-06-19 18:30:02 +00:00
|
|
|
void PipelineExecutor::expandPipeline(Stack & stack, UInt64 pid)
|
2019-02-27 12:51:27 +00:00
|
|
|
{
|
|
|
|
auto & cur_node = graph[pid];
|
|
|
|
auto new_processors = cur_node.processor->expandPipeline();
|
|
|
|
|
|
|
|
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
|
|
|
}
|
|
|
|
|
|
|
|
processors.insert(processors.end(), new_processors.begin(), new_processors.end());
|
|
|
|
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-04-16 11:40:15 +00:00
|
|
|
if (addEdges(node))
|
|
|
|
{
|
|
|
|
if (graph[node].status == ExecStatus::Idle || graph[node].status == ExecStatus::New)
|
|
|
|
{
|
|
|
|
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-06-19 18:30:02 +00:00
|
|
|
bool PipelineExecutor::addProcessorToPrepareQueueIfUpdated(Edge & edge, Stack & stack)
|
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-06-19 18:30:02 +00:00
|
|
|
ExecStatus status = node.status.load();
|
|
|
|
|
2019-06-14 09:35:38 +00:00
|
|
|
/// Don't add processor if nothing was read from port.
|
2019-06-19 18:30:02 +00:00
|
|
|
if (status != ExecStatus::New && edge.version == edge.prev_version)
|
|
|
|
return false;
|
|
|
|
|
|
|
|
if (status == ExecStatus::Finished)
|
2019-06-14 09:35:38 +00:00
|
|
|
return false;
|
|
|
|
|
2019-06-19 18:30:02 +00:00
|
|
|
/// Signal that node need to be prepared.
|
|
|
|
node.need_to_be_prepared = true;
|
2019-06-14 09:35:38 +00:00
|
|
|
edge.prev_version = edge.version;
|
|
|
|
|
2019-06-19 18:30:02 +00:00
|
|
|
/// Try to get ownership for node.
|
2019-06-14 09:35:38 +00:00
|
|
|
|
2019-06-19 18:30:02 +00:00
|
|
|
/// Assume that current status is New or Idle. Otherwise, can't prepare node.
|
|
|
|
if (status != ExecStatus::New)
|
|
|
|
status = ExecStatus::Idle;
|
2019-06-14 09:35:38 +00:00
|
|
|
|
2019-06-19 18:30:02 +00:00
|
|
|
/// Statuses but New and Idle are not interesting because they own node.
|
|
|
|
/// Prepare will be called in owning thread before changing status.
|
|
|
|
while (!node.status.compare_exchange_weak(status, ExecStatus::Preparing))
|
|
|
|
if (!(status == ExecStatus::New || status == ExecStatus::Idle) || !node.need_to_be_prepared)
|
|
|
|
return false;
|
|
|
|
|
|
|
|
stack.push(edge.to);
|
|
|
|
return true;
|
2019-06-14 09:35:38 +00:00
|
|
|
|
|
|
|
}
|
|
|
|
|
2019-06-19 18:30:02 +00:00
|
|
|
void PipelineExecutor::prepareProcessor(UInt64 pid, Stack & stack, bool async)
|
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-06-19 18:30:02 +00:00
|
|
|
/// Stopwatch watch;
|
|
|
|
|
|
|
|
/// Disable flag before prepare call. Otherwise, we can skip prepare request.
|
|
|
|
/// Prepare can be called more times than needed, but it's ok.
|
|
|
|
node.need_to_be_prepared = false;
|
|
|
|
|
2019-06-17 09:24:56 +00:00
|
|
|
auto status = node.processor->prepare();
|
|
|
|
|
2019-06-19 18:30:02 +00:00
|
|
|
/// node.execution_state->preparation_time_ns += watch.elapsed();
|
2019-06-17 09:24:56 +00:00
|
|
|
node.last_processor_status = status;
|
|
|
|
}
|
2019-02-08 16:10:57 +00:00
|
|
|
|
2019-06-19 18:30:02 +00:00
|
|
|
auto add_neighbours_to_prepare_queue = [&, this] ()
|
2019-02-08 16:10:57 +00:00
|
|
|
{
|
|
|
|
for (auto & edge : node.backEdges)
|
2019-06-19 18:30:02 +00:00
|
|
|
addProcessorToPrepareQueueIfUpdated(edge, stack);
|
2019-06-14 17:08:32 +00:00
|
|
|
|
|
|
|
for (auto & edge : node.directEdges)
|
2019-06-19 18:30:02 +00:00
|
|
|
addProcessorToPrepareQueueIfUpdated(edge, stack);
|
|
|
|
};
|
|
|
|
|
|
|
|
auto try_release_ownership = [&] ()
|
|
|
|
{
|
|
|
|
ExecStatus expected = ExecStatus::Idle;
|
|
|
|
node.status = ExecStatus::Idle;
|
|
|
|
|
|
|
|
if (node.need_to_be_prepared)
|
|
|
|
{
|
|
|
|
while (!node.status.compare_exchange_weak(expected, ExecStatus::Preparing))
|
|
|
|
if (!(expected == ExecStatus::Idle) || !node.need_to_be_prepared)
|
|
|
|
return;
|
|
|
|
|
|
|
|
stack.push(pid);
|
|
|
|
}
|
2019-02-08 16:10:57 +00:00
|
|
|
};
|
2019-02-06 16:31:18 +00:00
|
|
|
|
2019-06-17 09:24:56 +00:00
|
|
|
switch (node.last_processor_status)
|
2019-02-06 16:31:18 +00:00
|
|
|
{
|
|
|
|
case IProcessor::Status::NeedData:
|
2019-02-08 16:10:57 +00:00
|
|
|
{
|
|
|
|
add_neighbours_to_prepare_queue();
|
2019-06-19 18:30:02 +00:00
|
|
|
try_release_ownership();
|
|
|
|
|
2019-02-08 16:10:57 +00:00
|
|
|
break;
|
|
|
|
}
|
2019-02-06 16:31:18 +00:00
|
|
|
case IProcessor::Status::PortFull:
|
|
|
|
{
|
2019-02-08 16:10:57 +00:00
|
|
|
add_neighbours_to_prepare_queue();
|
2019-06-19 18:30:02 +00:00
|
|
|
try_release_ownership();
|
|
|
|
|
2019-02-06 16:31:18 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case IProcessor::Status::Finished:
|
|
|
|
{
|
2019-02-08 16:10:57 +00:00
|
|
|
add_neighbours_to_prepare_queue();
|
2019-02-06 16:31:18 +00:00
|
|
|
node.status = ExecStatus::Finished;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
case IProcessor::Status::Ready:
|
|
|
|
{
|
2019-02-08 16:10:57 +00:00
|
|
|
node.status = ExecStatus::Executing;
|
2019-06-18 11:43:43 +00:00
|
|
|
/// addJob(pid);
|
2019-02-06 16:31:18 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case IProcessor::Status::Async:
|
|
|
|
{
|
2019-06-19 18:30:02 +00:00
|
|
|
throw Exception("Async is temporary not supported.", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
|
|
|
// node.status = ExecStatus::Executing;
|
|
|
|
// addAsyncJob(pid);
|
|
|
|
// break;
|
2019-02-06 16:31:18 +00:00
|
|
|
}
|
|
|
|
case IProcessor::Status::Wait:
|
|
|
|
{
|
2019-02-08 16:10:57 +00:00
|
|
|
if (!async)
|
|
|
|
throw Exception("Processor returned status Wait before Async.", ErrorCodes::LOGICAL_ERROR);
|
2019-02-27 11:24:14 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case IProcessor::Status::ExpandPipeline:
|
|
|
|
{
|
2019-03-15 17:06:32 +00:00
|
|
|
|
2019-06-19 18:30:02 +00:00
|
|
|
ExecutionState * desired = node.execution_state.get();
|
|
|
|
ExecutionState * expected = nullptr;
|
|
|
|
|
|
|
|
while (!node_to_expand.compare_exchange_strong(expected, desired))
|
|
|
|
{
|
|
|
|
expected = nullptr;
|
|
|
|
doExpandPipeline(stack);
|
|
|
|
}
|
|
|
|
|
|
|
|
doExpandPipeline(stack);
|
|
|
|
|
|
|
|
node.need_to_be_prepared = true;
|
|
|
|
try_release_ownership();
|
2019-02-27 12:51:27 +00:00
|
|
|
break;
|
2019-02-06 16:31:18 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-06-19 18:30:02 +00:00
|
|
|
void PipelineExecutor::doExpandPipeline(Stack & stack)
|
2019-06-14 09:35:38 +00:00
|
|
|
{
|
2019-06-19 18:30:02 +00:00
|
|
|
std::unique_lock lock(mutex_to_expand_pipeline);
|
|
|
|
++num_waiting_threads_to_expand_pipeline;
|
2019-06-14 09:35:38 +00:00
|
|
|
|
2019-06-19 18:30:02 +00:00
|
|
|
condvar_to_expand_pipeline.wait(lock, [&]()
|
|
|
|
{
|
|
|
|
return num_waiting_threads_to_expand_pipeline == num_preparing_threads || node_to_expand == nullptr;
|
|
|
|
});
|
2019-06-14 09:35:38 +00:00
|
|
|
|
2019-06-19 18:30:02 +00:00
|
|
|
--num_waiting_threads_to_expand_pipeline;
|
2019-06-14 09:35:38 +00:00
|
|
|
|
2019-06-19 18:30:02 +00:00
|
|
|
if (node_to_expand)
|
2019-02-06 16:31:18 +00:00
|
|
|
{
|
2019-06-19 18:30:02 +00:00
|
|
|
expandPipeline(stack, node_to_expand.load()->processors_id);
|
|
|
|
node_to_expand = nullptr;
|
|
|
|
lock.unlock();
|
|
|
|
condvar_to_expand_pipeline.notify_all();
|
2019-02-06 16:31:18 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-06-19 18:30:02 +00:00
|
|
|
//void PipelineExecutor::assignJobs()
|
|
|
|
//{
|
|
|
|
// for (auto * state : execution_states_queue)
|
|
|
|
// {
|
|
|
|
// if (!tryAssignJob(state))
|
|
|
|
// {
|
|
|
|
// while (!task_queue.push(state))
|
|
|
|
// sleep(0);
|
|
|
|
//
|
|
|
|
// task_condvar.notify_one();
|
|
|
|
// ++num_tasks_to_wait;
|
|
|
|
// }
|
|
|
|
// }
|
|
|
|
//
|
|
|
|
// execution_states_queue.clear();
|
|
|
|
//}
|
2019-06-14 09:35:38 +00:00
|
|
|
|
2019-06-19 18:30:02 +00:00
|
|
|
//void PipelineExecutor::processPrepareQueue()
|
|
|
|
//{
|
|
|
|
// while (!prepare_stack.empty())
|
|
|
|
// {
|
|
|
|
// UInt64 proc = prepare_stack.top();
|
|
|
|
// prepare_stack.pop();
|
|
|
|
//
|
|
|
|
// prepareProcessor(proc, false);
|
|
|
|
// }
|
|
|
|
//
|
|
|
|
// assignJobs();
|
|
|
|
//}
|
|
|
|
//
|
|
|
|
//void PipelineExecutor::processAsyncQueue()
|
|
|
|
//{
|
|
|
|
// UInt64 num_processors = processors.size();
|
|
|
|
// for (UInt64 node = 0; node < num_processors; ++node)
|
|
|
|
// if (graph[node].status == ExecStatus::Async)
|
|
|
|
// prepareProcessor(node, true);
|
|
|
|
//
|
|
|
|
// assignJobs();
|
|
|
|
//}
|
2019-02-08 16:10:57 +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-04-30 12:28:11 +00:00
|
|
|
/// Wait for all tasks to finish in case of exception.
|
|
|
|
SCOPE_EXIT(
|
2019-05-16 14:57:27 +00:00
|
|
|
finished = true;
|
|
|
|
|
|
|
|
task_condvar.notify_all();
|
|
|
|
|
|
|
|
for (auto & thread : threads)
|
|
|
|
thread.join();
|
2019-04-30 12:28:11 +00:00
|
|
|
);
|
|
|
|
|
2019-05-16 14:57:27 +00:00
|
|
|
executeImpl(num_threads);
|
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)
|
|
|
|
if (node.status != ExecStatus::Finished)
|
|
|
|
all_processors_finished = false;
|
|
|
|
|
|
|
|
if (!all_processors_finished)
|
|
|
|
throw Exception("Pipeline stuck. Current state:\n" + dumpPipeline(), ErrorCodes::LOGICAL_ERROR);
|
|
|
|
}
|
|
|
|
|
2019-06-14 17:08:32 +00:00
|
|
|
void PipelineExecutor::executeSingleThread(size_t num_threads)
|
2019-04-12 09:57:13 +00:00
|
|
|
{
|
2019-06-14 17:08:32 +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-06-14 17:08:32 +00:00
|
|
|
Stopwatch total_time_watch;
|
2019-05-16 14:57:27 +00:00
|
|
|
|
2019-06-14 17:08:32 +00:00
|
|
|
ExecutionState * state = nullptr;
|
2019-05-16 14:57:27 +00:00
|
|
|
|
2019-06-19 18:30:02 +00:00
|
|
|
auto finish_execution = [&]()
|
|
|
|
{
|
|
|
|
finished = true;
|
|
|
|
finish_condvar.notify_one();
|
|
|
|
main_executor_condvar.notify_all();
|
|
|
|
};
|
|
|
|
|
|
|
|
auto prepare_processor = [&](UInt64 pid, Stack & stack)
|
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
|
|
|
prepareProcessor(pid, stack, false);
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
graph[pid].execution_state->exception = std::current_exception();
|
|
|
|
finish_execution();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2019-06-14 17:08:32 +00:00
|
|
|
while (!finished)
|
|
|
|
{
|
2019-06-18 13:08:22 +00:00
|
|
|
|
2019-06-14 17:08:32 +00:00
|
|
|
/// First, find any processor to execute.
|
|
|
|
/// Just travers graph and prepare any processor.
|
2019-06-18 13:08:22 +00:00
|
|
|
while (!finished)
|
2019-05-16 14:57:27 +00:00
|
|
|
{
|
2019-06-18 11:43:43 +00:00
|
|
|
|
2019-06-18 13:08:22 +00:00
|
|
|
while (num_waited_tasks < num_tasks_to_wait)
|
2019-05-16 14:57:27 +00:00
|
|
|
{
|
2019-06-18 13:08:22 +00:00
|
|
|
if (task_queue.pop(state))
|
2019-06-14 17:08:32 +00:00
|
|
|
{
|
2019-06-18 13:08:22 +00:00
|
|
|
++num_waited_tasks;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
state = nullptr;
|
|
|
|
}
|
2019-05-16 14:57:27 +00:00
|
|
|
|
2019-06-18 13:08:22 +00:00
|
|
|
if (state)
|
|
|
|
break;
|
2019-05-16 14:57:27 +00:00
|
|
|
|
2019-06-19 18:30:02 +00:00
|
|
|
std::unique_lock lock(main_executor_mutex);
|
|
|
|
|
|
|
|
++num_waiting_threads;
|
|
|
|
|
2019-06-18 13:08:22 +00:00
|
|
|
if (num_waiting_threads == num_threads)
|
2019-06-19 18:30:02 +00:00
|
|
|
finish_execution();
|
2019-06-18 11:43:43 +00:00
|
|
|
|
2019-06-19 18:30:02 +00:00
|
|
|
main_executor_condvar.wait(lock, [&]() { return finished || num_waited_tasks < num_tasks_to_wait; });
|
|
|
|
|
|
|
|
--num_waiting_threads;
|
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-06-18 13:08:22 +00:00
|
|
|
addJob(state);
|
2019-06-18 11:43:43 +00:00
|
|
|
|
2019-02-08 16:10:57 +00:00
|
|
|
{
|
2019-06-14 17:08:32 +00:00
|
|
|
Stopwatch execution_time_watch;
|
|
|
|
state->job();
|
|
|
|
execution_time_ns += execution_time_watch.elapsed();
|
2019-02-08 16:10:57 +00:00
|
|
|
}
|
2019-06-14 17:08:32 +00:00
|
|
|
|
|
|
|
if (state->exception)
|
2019-06-19 18:30:02 +00:00
|
|
|
finish_execution();
|
2019-06-14 17:08:32 +00:00
|
|
|
|
|
|
|
if (finished)
|
2019-02-08 16:10:57 +00:00
|
|
|
break;
|
2019-06-14 17:08:32 +00:00
|
|
|
|
|
|
|
Stopwatch processing_time_watch;
|
|
|
|
|
|
|
|
/// Try to execute neighbour processor.
|
|
|
|
{
|
2019-06-18 13:08:22 +00:00
|
|
|
/// std::unique_lock lock(main_executor_mutex);
|
2019-06-14 17:08:32 +00:00
|
|
|
|
2019-06-19 18:30:02 +00:00
|
|
|
Stack stack;
|
|
|
|
|
|
|
|
++num_preparing_threads;
|
|
|
|
if (node_to_expand)
|
|
|
|
doExpandPipeline(stack);
|
|
|
|
|
|
|
|
|
|
|
|
prepare_processor(state->processors_id, stack);
|
2019-06-14 17:08:32 +00:00
|
|
|
|
|
|
|
/// Execute again if can.
|
2019-06-18 13:08:22 +00:00
|
|
|
if (graph[state->processors_id].status != ExecStatus::Executing)
|
|
|
|
state = nullptr;
|
2019-06-17 09:24:56 +00:00
|
|
|
|
|
|
|
/// Process all neighbours. Children will be on the top of stack, then parents.
|
2019-06-19 18:30:02 +00:00
|
|
|
while (!stack.empty() && !finished)
|
2019-06-17 09:24:56 +00:00
|
|
|
{
|
2019-06-19 18:30:02 +00:00
|
|
|
while (!stack.empty() && !finished)
|
2019-06-14 17:08:32 +00:00
|
|
|
{
|
2019-06-19 18:30:02 +00:00
|
|
|
auto current_processor = stack.top();
|
|
|
|
stack.pop();
|
|
|
|
|
|
|
|
prepare_processor(current_processor, stack);
|
2019-06-18 13:08:22 +00:00
|
|
|
|
2019-06-19 18:30:02 +00:00
|
|
|
if (graph[current_processor].status == ExecStatus::Executing)
|
2019-06-18 13:08:22 +00:00
|
|
|
{
|
2019-06-19 18:30:02 +00:00
|
|
|
auto cur_state = graph[current_processor].execution_state.get();
|
|
|
|
|
|
|
|
if (state)
|
|
|
|
{
|
|
|
|
++num_tasks_to_wait;
|
|
|
|
main_executor_condvar.notify_one();
|
|
|
|
while (!task_queue.push(cur_state));
|
|
|
|
}
|
|
|
|
else
|
|
|
|
state = cur_state;
|
2019-06-18 13:08:22 +00:00
|
|
|
}
|
2019-06-14 17:08:32 +00:00
|
|
|
}
|
2019-06-17 09:24:56 +00:00
|
|
|
|
2019-06-19 18:30:02 +00:00
|
|
|
if (node_to_expand)
|
|
|
|
doExpandPipeline(stack);
|
|
|
|
}
|
|
|
|
--num_preparing_threads;
|
2019-02-08 16:10:57 +00:00
|
|
|
}
|
2019-06-14 17:08:32 +00:00
|
|
|
|
|
|
|
/// Let another thread to continue.
|
2019-06-18 13:08:22 +00:00
|
|
|
/// main_executor_condvar.notify_all();
|
2019-06-14 17:08:32 +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
|
|
|
|
|
|
|
total_time_ns = total_time_watch.elapsed();
|
|
|
|
wait_time_ns = total_time_ns - execution_time_ns - processing_time_ns;
|
|
|
|
|
|
|
|
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.");
|
|
|
|
}
|
|
|
|
|
|
|
|
void PipelineExecutor::executeImpl(size_t num_threads)
|
|
|
|
{
|
|
|
|
/// No need to make task_queue longer than num_threads.
|
|
|
|
/// Therefore, finished_execution_queue can't be longer than num_threads too.
|
2019-06-18 13:08:22 +00:00
|
|
|
task_queue.reserve_unsafe(8192);
|
2019-06-14 17:08:32 +00:00
|
|
|
finished_execution_queue.reserve_unsafe(num_threads);
|
|
|
|
|
2019-06-19 18:30:02 +00:00
|
|
|
Stack stack;
|
|
|
|
|
|
|
|
addChildlessProcessorsToQueue(stack);
|
|
|
|
|
|
|
|
while (!stack.empty())
|
2019-06-18 13:08:22 +00:00
|
|
|
{
|
2019-06-19 18:30:02 +00:00
|
|
|
UInt64 proc = stack.top();
|
|
|
|
stack.pop();
|
2019-06-18 13:08:22 +00:00
|
|
|
|
2019-06-19 18:30:02 +00:00
|
|
|
prepareProcessor(proc, stack, false);
|
2019-06-18 13:08:22 +00:00
|
|
|
|
|
|
|
if (graph[proc].status == ExecStatus::Executing)
|
|
|
|
{
|
|
|
|
auto cur_state = graph[proc].execution_state.get();
|
|
|
|
++num_tasks_to_wait;
|
|
|
|
while (!task_queue.push(cur_state));
|
|
|
|
}
|
|
|
|
}
|
2019-06-14 17:08:32 +00:00
|
|
|
|
2019-06-19 18:30:02 +00:00
|
|
|
/// background_executor_flag = false;
|
|
|
|
num_preparing_threads = 0;
|
|
|
|
node_to_expand = nullptr;
|
|
|
|
|
2019-06-14 17:08:32 +00:00
|
|
|
threads.reserve(num_threads);
|
2019-06-19 18:30:02 +00:00
|
|
|
/// executor_contexts.reserve(num_threads);
|
2019-06-14 17:08:32 +00:00
|
|
|
|
|
|
|
auto thread_group = CurrentThread::getGroup();
|
|
|
|
|
|
|
|
for (size_t i = 0; i < num_threads; ++i)
|
|
|
|
{
|
2019-06-19 18:30:02 +00:00
|
|
|
// executor_contexts.emplace_back(std::make_unique<ExecutorContext>());
|
|
|
|
// auto * executor_context = executor_contexts.back().get();
|
|
|
|
//
|
|
|
|
// executor_context->executor_number = i;
|
|
|
|
// executor_context->next_task_to_execute = nullptr;
|
2019-06-14 17:08:32 +00:00
|
|
|
|
|
|
|
threads.emplace_back([this, thread_group, num_threads]
|
|
|
|
{
|
|
|
|
ThreadStatus thread_status;
|
|
|
|
|
|
|
|
if (thread_group)
|
|
|
|
CurrentThread::attachTo(thread_group);
|
|
|
|
|
|
|
|
SCOPE_EXIT(
|
|
|
|
if (thread_group)
|
|
|
|
CurrentThread::detachQueryIfNotDetached();
|
|
|
|
);
|
|
|
|
|
|
|
|
executeSingleThread(num_threads);
|
|
|
|
});
|
|
|
|
}
|
|
|
|
|
|
|
|
{
|
|
|
|
std::mutex finish_mutex;
|
|
|
|
std::unique_lock lock(finish_mutex);
|
|
|
|
finish_condvar.wait(lock, [&]() -> bool { return finished; });
|
|
|
|
}
|
|
|
|
|
|
|
|
{
|
|
|
|
std::lock_guard lock(main_executor_mutex);
|
|
|
|
|
|
|
|
for (auto & node : graph)
|
|
|
|
if (node.execution_state->exception)
|
|
|
|
std::rethrow_exception(node.execution_state->exception);
|
|
|
|
}
|
|
|
|
// while (!cancelled)
|
|
|
|
// {
|
|
|
|
// processFinishedExecutionQueueSafe();
|
|
|
|
// processPrepareQueue();
|
|
|
|
// processAsyncQueue();
|
|
|
|
//
|
|
|
|
// if (prepare_stack.empty())
|
|
|
|
// {
|
|
|
|
// /// For single-thread executor.
|
|
|
|
// if (num_threads == 1)
|
|
|
|
// {
|
|
|
|
// if (!finished_execution_queue.empty())
|
|
|
|
// continue;
|
|
|
|
// else
|
|
|
|
// break;
|
|
|
|
// }
|
|
|
|
//
|
|
|
|
// if (num_tasks_to_wait > num_waited_tasks)
|
|
|
|
// {
|
|
|
|
// /// Try wait anything.
|
|
|
|
// event_counter.wait();
|
|
|
|
// ++num_waited_tasks;
|
|
|
|
// }
|
|
|
|
// else
|
|
|
|
// {
|
|
|
|
// /// Here prepare_queue is empty and we have nobody to wait for. Exiting.
|
|
|
|
// break;
|
|
|
|
// }
|
|
|
|
// }
|
|
|
|
// }
|
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
|
|
|
}
|
|
|
|
|
|
|
|
}
|