Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 6 additions & 0 deletions docs/en/sql-reference/statements/system.md
Original file line number Diff line number Diff line change
Expand Up @@ -206,6 +206,12 @@ SYSTEM RELOAD USERS [ON CLUSTER cluster_name]
Normally shuts down ClickHouse (like `service clickhouse-server stop` / `kill {$pid_clickhouse-server}`)
## PRESHUTDOWN {#preshutdown}
<CloudNotSupportedBadge/>
Prepare node for graceful shutdown. Unregister in autodiscovered clusters, stop accepting distributed requests to object storages (s3Cluster, icebergCluster, etc.).
## KILL {#kill}
Aborts ClickHouse process (like `kill -9 {$ pid_clickhouse-server}`)
Expand Down
4 changes: 4 additions & 0 deletions programs/server/Server.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -2292,6 +2292,8 @@ try

}

global_context->startSwarmMode();

{
std::lock_guard lock(servers_lock);
/// We should start interserver communications before (and more important shutdown after) tables.
Expand Down Expand Up @@ -2701,6 +2703,8 @@ try

is_cancelled = true;

global_context->stopSwarmMode();

LOG_DEBUG(log, "Waiting for current connections to close.");

size_t current_connections = 0;
Expand Down
1 change: 1 addition & 0 deletions src/Access/Common/AccessType.h
Original file line number Diff line number Diff line change
Expand Up @@ -200,6 +200,7 @@ enum class AccessType : uint8_t
M(SYSTEM_TTL_MERGES, "SYSTEM STOP TTL MERGES, SYSTEM START TTL MERGES, STOP TTL MERGES, START TTL MERGES", TABLE, SYSTEM) \
M(SYSTEM_FETCHES, "SYSTEM STOP FETCHES, SYSTEM START FETCHES, STOP FETCHES, START FETCHES", TABLE, SYSTEM) \
M(SYSTEM_MOVES, "SYSTEM STOP MOVES, SYSTEM START MOVES, STOP MOVES, START MOVES", TABLE, SYSTEM) \
M(SYSTEM_SWARM, "SYSTEM STOP SWARM MODE, SYSTEM START SWARM MODE, STOP SWARM MODE, START SWARM MODE", GLOBAL, SYSTEM) \
M(SYSTEM_PULLING_REPLICATION_LOG, "SYSTEM STOP PULLING REPLICATION LOG, SYSTEM START PULLING REPLICATION LOG", TABLE, SYSTEM) \
M(SYSTEM_CLEANUP, "SYSTEM STOP CLEANUP, SYSTEM START CLEANUP", TABLE, SYSTEM) \
M(SYSTEM_VIEWS, "SYSTEM REFRESH VIEW, SYSTEM START VIEWS, SYSTEM STOP VIEWS, SYSTEM START VIEW, SYSTEM STOP VIEW, SYSTEM CANCEL VIEW, REFRESH VIEW, START VIEWS, STOP VIEWS, START VIEW, STOP VIEW, CANCEL VIEW", VIEW, SYSTEM) \
Expand Down
1 change: 1 addition & 0 deletions src/Common/CurrentMetrics.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -430,6 +430,7 @@
M(StartupScriptsExecutionState, "State of startup scripts execution: 0 = not finished, 1 = success, 2 = failure.") \
\
M(IsServerShuttingDown, "Indicates if the server is shutting down: 0 = no, 1 = yes") \
M(IsSwarmModeEnabled, "Indicates if the swarm mode enabled or not: 0 = disabled, 1 = enabled") \
\
M(TotalMergeFailures, "Number of all failed merges since startup, including the ones that were aborted") \
M(NonAbortedMergeFailures, "Number of failed merges since startup, excluding the merges that were aborted") \
Expand Down
62 changes: 61 additions & 1 deletion src/Interpreters/ClusterDiscovery.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -108,6 +108,13 @@ class ClusterDiscovery::Flags
cv.notify_one();
}

void wakeup()
{
std::unique_lock<std::mutex> lk(mu);
any_need_update = true;
cv.notify_one();
}

private:
std::condition_variable cv;
std::mutex mu;
Expand Down Expand Up @@ -391,7 +398,9 @@ bool ClusterDiscovery::upsertCluster(ClusterInfo & cluster_info)
return true;
};

if (!cluster_info.current_node_is_observer && !contains(node_uuids, current_node_name))
if (!cluster_info.current_node_is_observer
&& context->isSwarmModeEnabled()
&& !contains(node_uuids, current_node_name))
{
LOG_ERROR(log, "Can't find current node in cluster '{}', will register again", cluster_info.name);
registerInZk(zk, cluster_info);
Expand Down Expand Up @@ -455,12 +464,30 @@ void ClusterDiscovery::registerInZk(zkutil::ZooKeeperPtr & zk, ClusterInfo & inf
return;
}

if (!context->isSwarmModeEnabled())
{
LOG_DEBUG(log, "STOP SWARM MODE called, skip self-registering current node {} in cluster {}", current_node_name, info.name);
return;
}

LOG_DEBUG(log, "Registering current node {} in cluster {}", current_node_name, info.name);

zk->createOrUpdate(node_path, info.current_node.serialize(), zkutil::CreateMode::Ephemeral);
LOG_DEBUG(log, "Current node {} registered in cluster {}", current_node_name, info.name);
}

void ClusterDiscovery::unregisterFromZk(zkutil::ZooKeeperPtr & zk, ClusterInfo & info)
{
if (info.current_node_is_observer)
return;

String node_path = getShardsListPath(info.zk_root) / current_node_name;
LOG_DEBUG(log, "Removing current node {} from cluster {}", current_node_name, info.name);

zk->remove(node_path);
LOG_DEBUG(log, "Current node {} removed from cluster {}", current_node_name, info.name);
}

void ClusterDiscovery::initialUpdate()
{
LOG_DEBUG(log, "Initializing");
Expand Down Expand Up @@ -506,6 +533,18 @@ void ClusterDiscovery::initialUpdate()
is_initialized = true;
}

void ClusterDiscovery::registerAll()
{
register_change_flag = RegisterChangeFlag::RCF_REGISTER_ALL;
clusters_to_update->wakeup();
}

void ClusterDiscovery::unregisterAll()
{
register_change_flag = RegisterChangeFlag::RCF_UNREGISTER_ALL;
clusters_to_update->wakeup();
}

void ClusterDiscovery::findDynamicClusters(
std::unordered_map<String, ClusterDiscovery::ClusterInfo> & info,
std::unordered_set<size_t> * unchanged_roots)
Expand Down Expand Up @@ -729,6 +768,27 @@ bool ClusterDiscovery::runMainThread(std::function<void()> up_to_date_callback)
{
up_to_date_callback();
}

RegisterChangeFlag flag = register_change_flag.exchange(RegisterChangeFlag::RCF_NONE);

if (flag == RegisterChangeFlag::RCF_REGISTER_ALL)
{
LOG_DEBUG(log, "Register in all dynamic clusters");
for (auto & [_, info] : clusters_info)
{
auto zk = context->getDefaultOrAuxiliaryZooKeeper(info.zk_name);
registerInZk(zk, info);
}
}
else if (flag == RegisterChangeFlag::RCF_UNREGISTER_ALL)
{
LOG_DEBUG(log, "Unregister in all dynamic clusters");
for (auto & [_, info] : clusters_info)
{
auto zk = context->getDefaultOrAuxiliaryZooKeeper(info.zk_name);
unregisterFromZk(zk, info);
}
}
}
LOG_DEBUG(log, "Worker thread stopped");
return finished;
Expand Down
13 changes: 13 additions & 0 deletions src/Interpreters/ClusterDiscovery.h
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,9 @@ class ClusterDiscovery

~ClusterDiscovery();

void registerAll();
void unregisterAll();

private:
struct NodeInfo
{
Expand Down Expand Up @@ -125,6 +128,7 @@ class ClusterDiscovery
void initialUpdate();

void registerInZk(zkutil::ZooKeeperPtr & zk, ClusterInfo & info);
void unregisterFromZk(zkutil::ZooKeeperPtr & zk, ClusterInfo & info);

Strings getNodeNames(zkutil::ZooKeeperPtr & zk,
const String & zk_root,
Expand Down Expand Up @@ -207,6 +211,15 @@ class ClusterDiscovery
std::shared_ptr<std::vector<std::shared_ptr<MulticlusterDiscovery>>> multicluster_discovery_paths;

MultiVersion<Macros>::Version macros;

enum RegisterChangeFlag
{
RCF_NONE,
RCF_REGISTER_ALL,
RCF_UNREGISTER_ALL,
};

std::atomic<RegisterChangeFlag> register_change_flag = RegisterChangeFlag::RCF_NONE;
};

}
45 changes: 43 additions & 2 deletions src/Interpreters/Context.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -188,6 +188,7 @@ namespace CurrentMetrics
extern const Metric IcebergCatalogThreads;
extern const Metric IcebergCatalogThreadsActive;
extern const Metric IcebergCatalogThreadsScheduled;
extern const Metric IsSwarmModeEnabled;
}


Expand Down Expand Up @@ -579,6 +580,7 @@ struct ContextSharedPart : boost::noncopyable
std::map<String, UInt16> server_ports;

std::atomic<bool> shutdown_called = false;
std::atomic<bool> swarm_mode_enabled = true;

Stopwatch uptime_watch TSA_GUARDED_BY(mutex);

Expand Down Expand Up @@ -747,6 +749,8 @@ struct ContextSharedPart : boost::noncopyable
*/
void shutdown() TSA_NO_THREAD_SAFETY_ANALYSIS
{
swarm_mode_enabled = false;
CurrentMetrics::set(CurrentMetrics::IsSwarmModeEnabled, 0);
bool is_shutdown_called = shutdown_called.exchange(true);
if (is_shutdown_called)
return;
Expand Down Expand Up @@ -4644,7 +4648,6 @@ std::shared_ptr<Cluster> Context::getCluster(const std::string & cluster_name) c
throw Exception(ErrorCodes::CLUSTER_DOESNT_EXIST, "Requested cluster '{}' not found", cluster_name);
}


std::shared_ptr<Cluster> Context::tryGetCluster(const std::string & cluster_name) const
{
std::shared_ptr<Cluster> res = nullptr;
Expand All @@ -4663,6 +4666,21 @@ std::shared_ptr<Cluster> Context::tryGetCluster(const std::string & cluster_name
return res;
}

void Context::unregisterInAutodiscoveryClusters()
{
std::lock_guard lock(shared->clusters_mutex);
if (!shared->cluster_discovery)
return;
shared->cluster_discovery->unregisterAll();
}

void Context::registerInAutodiscoveryClusters()
{
std::lock_guard lock(shared->clusters_mutex);
if (!shared->cluster_discovery)
return;
shared->cluster_discovery->registerAll();
}

void Context::reloadClusterConfig() const
{
Expand Down Expand Up @@ -5540,12 +5558,35 @@ void Context::stopServers(const ServerType & server_type) const
shared->stop_servers_callback(server_type);
}


void Context::shutdown() TSA_NO_THREAD_SAFETY_ANALYSIS
{
shared->shutdown();
}

bool Context::stopSwarmMode()
{
bool expected_is_enabled = true;
bool is_stopped_now = shared->swarm_mode_enabled.compare_exchange_strong(expected_is_enabled, false);
if (is_stopped_now)
CurrentMetrics::set(CurrentMetrics::IsSwarmModeEnabled, 0);
// return true if stop successful
return is_stopped_now;
}

bool Context::startSwarmMode()
{
bool expected_is_enabled = false;
bool is_started_now = shared->swarm_mode_enabled.compare_exchange_strong(expected_is_enabled, true);
if (is_started_now)
CurrentMetrics::set(CurrentMetrics::IsSwarmModeEnabled, 1);
// return true if start successful
return is_started_now;
}

bool Context::isSwarmModeEnabled() const
{
return shared->swarm_mode_enabled;
}

Context::ApplicationType Context::getApplicationType() const
{
Expand Down
11 changes: 11 additions & 0 deletions src/Interpreters/Context.h
Original file line number Diff line number Diff line change
Expand Up @@ -1296,6 +1296,8 @@ class Context: public ContextData, public std::enable_shared_from_this<Context>
size_t getClustersVersion() const;

void startClusterDiscovery();
void registerInAutodiscoveryClusters();
void unregisterInAutodiscoveryClusters();

/// Sets custom cluster, but doesn't update configuration
void setCluster(const String & cluster_name, const std::shared_ptr<Cluster> & cluster);
Expand Down Expand Up @@ -1408,6 +1410,15 @@ class Context: public ContextData, public std::enable_shared_from_this<Context>

void shutdown();

/// Stop some works to allow graceful shutdown later.
/// Returns true if stop successful.
bool stopSwarmMode();
/// Resume some works if we change our mind.
/// Returns true if start successful.
bool startSwarmMode();
/// Return current swarm mode state.
bool isSwarmModeEnabled() const;

bool isInternalQuery() const { return is_internal_query; }
void setInternalQuery(bool internal) { is_internal_query = internal; }

Expand Down
20 changes: 20 additions & 0 deletions src/Interpreters/InterpreterSystemQuery.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -715,6 +715,20 @@ BlockIO InterpreterSystemQuery::execute()
case Type::START_MOVES:
startStopAction(ActionLocks::PartsMove, true);
break;
case Type::STOP_SWARM_MODE:
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Perhaps this should also be a single atomic operation?

{
getContext()->checkAccess(AccessType::SYSTEM_SWARM);
if (getContext()->stopSwarmMode())
getContext()->unregisterInAutodiscoveryClusters();
break;
}
case Type::START_SWARM_MODE:
{
getContext()->checkAccess(AccessType::SYSTEM_SWARM);
if (getContext()->startSwarmMode())
getContext()->registerInAutodiscoveryClusters();
break;
}
case Type::STOP_FETCHES:
startStopAction(ActionLocks::PartsFetch, false);
break;
Expand Down Expand Up @@ -1623,6 +1637,12 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster()
required_access.emplace_back(AccessType::SYSTEM_MOVES, query.getDatabase(), query.getTable());
break;
}
case Type::STOP_SWARM_MODE:
case Type::START_SWARM_MODE:
{
required_access.emplace_back(AccessType::SYSTEM_SWARM);
break;
}
case Type::STOP_PULLING_REPLICATION_LOG:
case Type::START_PULLING_REPLICATION_LOG:
{
Expand Down
2 changes: 2 additions & 0 deletions src/Parsers/ASTSystemQuery.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -495,6 +495,8 @@ void ASTSystemQuery::formatImpl(WriteBuffer & ostr, const FormatSettings & setti
case Type::DROP_PAGE_CACHE:
case Type::STOP_REPLICATED_DDL_QUERIES:
case Type::START_REPLICATED_DDL_QUERIES:
case Type::STOP_SWARM_MODE:
case Type::START_SWARM_MODE:
break;
case Type::UNKNOWN:
case Type::END:
Expand Down
2 changes: 2 additions & 0 deletions src/Parsers/ASTSystemQuery.h
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,8 @@ class ASTSystemQuery : public IAST, public ASTQueryWithOnCluster
START_FETCHES,
STOP_MOVES,
START_MOVES,
STOP_SWARM_MODE,
START_SWARM_MODE,
STOP_REPLICATED_SENDS,
START_REPLICATED_SENDS,
STOP_REPLICATION_QUEUES,
Expand Down
5 changes: 5 additions & 0 deletions src/QueryPipeline/RemoteQueryExecutor.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -1034,6 +1034,11 @@ void RemoteQueryExecutor::setProfileInfoCallback(ProfileInfoCallback callback)
profile_info_callback = std::move(callback);
}

bool RemoteQueryExecutor::skipUnavailableShards() const
{
return context->getSettingsRef()[Setting::skip_unavailable_shards];
}

bool RemoteQueryExecutor::needToSkipUnavailableShard() const
{
return context->getSettingsRef()[Setting::skip_unavailable_shards] && (0 == connections->size());
Expand Down
2 changes: 2 additions & 0 deletions src/QueryPipeline/RemoteQueryExecutor.h
Original file line number Diff line number Diff line change
Expand Up @@ -241,6 +241,8 @@ class RemoteQueryExecutor

IConnections & getConnections() { return *connections; }

bool skipUnavailableShards() const;

bool needToSkipUnavailableShard() const;

bool isReplicaUnavailable() const { return extension && extension->parallel_reading_coordinator && connections->size() == 0; }
Expand Down
Loading
Loading