mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-11 18:14:03 +00:00
fix distributed "create table as select" query
This commit is contained in:
parent
d439531228
commit
f9fb73ce2e
@ -178,7 +178,7 @@ struct DDLTask
|
||||
Cluster::Address address_in_cluster;
|
||||
size_t host_shard_num;
|
||||
size_t host_replica_num;
|
||||
|
||||
//
|
||||
/// Stage 3.3: execute query
|
||||
ExecutionStatus execution_status;
|
||||
bool was_executed = false;
|
||||
@ -335,6 +335,7 @@ void DDLWorker::processTasks()
|
||||
{
|
||||
LOG_DEBUG(log, "Processing tasks");
|
||||
|
||||
|
||||
Strings queue_nodes = zookeeper->getChildren(queue_dir, nullptr, event_queue_updated);
|
||||
filterAndSortQueueNodes(queue_nodes);
|
||||
if (queue_nodes.empty())
|
||||
@ -536,11 +537,29 @@ bool DDLWorker::tryExecuteQuery(const String & query, const DDLTask & task, Exec
|
||||
return true;
|
||||
}
|
||||
|
||||
void DDLWorker::attachToThreadGroup()
|
||||
{
|
||||
std::lock_guard lock(thread_group_mutex);
|
||||
|
||||
if (thread_group)
|
||||
{
|
||||
/// Put all threads to one thread pool
|
||||
CurrentThread::attachToIfDetached(thread_group);
|
||||
}
|
||||
else
|
||||
{
|
||||
CurrentThread::initializeQuery();
|
||||
thread_group = CurrentThread::getGroup();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void DDLWorker::processTask(DDLTask & task)
|
||||
{
|
||||
LOG_DEBUG(log, "Processing task " << task.entry_name << " (" << task.entry.query << ")");
|
||||
|
||||
attachToThreadGroup();
|
||||
|
||||
String dummy;
|
||||
String active_node_path = task.entry_path + "/active/" + task.host_id_str;
|
||||
String finished_node_path = task.entry_path + "/finished/" + task.host_id_str;
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/Cluster.h>
|
||||
#include <DataStreams/BlockIO.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
#include <atomic>
|
||||
@ -67,6 +68,8 @@ private:
|
||||
|
||||
void run();
|
||||
|
||||
void attachToThreadGroup();
|
||||
|
||||
private:
|
||||
Context & context;
|
||||
Logger * log;
|
||||
@ -98,6 +101,9 @@ private:
|
||||
/// How many tasks could be in the queue
|
||||
size_t max_tasks_in_queue = 1000;
|
||||
|
||||
ThreadGroupStatusPtr thread_group;
|
||||
std::mutex thread_group_mutex;
|
||||
|
||||
friend class DDLQueryStatusInputSream;
|
||||
friend struct DDLTask;
|
||||
};
|
||||
|
Loading…
Reference in New Issue
Block a user