mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge pull request #64889 from ClickHouse/pufit/startup-scripts
Startup scripts
This commit is contained in:
commit
f22f71a09d
30
docs/en/operations/startup-scripts.md
Normal file
30
docs/en/operations/startup-scripts.md
Normal file
@ -0,0 +1,30 @@
|
||||
---
|
||||
slug: /en/operations/startup-scripts
|
||||
sidebar_label: Startup Scripts
|
||||
---
|
||||
|
||||
# Startup Scripts
|
||||
|
||||
ClickHouse can run arbitrary SQL queries from the server configuration during startup. This can be useful for migrations or automatic schema creation.
|
||||
|
||||
```xml
|
||||
<clickhouse>
|
||||
<startup_scripts>
|
||||
<scripts>
|
||||
<query>CREATE ROLE OR REPLACE test_role</query>
|
||||
</scripts>
|
||||
<scripts>
|
||||
<query>CREATE TABLE TestTable (id UInt64) ENGINE=TinyLog</query>
|
||||
<condition>SELECT 1;</condition>
|
||||
</scripts>
|
||||
</startup_scripts>
|
||||
</clickhouse>
|
||||
```
|
||||
|
||||
ClickHouse executes all queries from the `startup_scripts` sequentially in the specified order. If any of the queries fail, the execution of the following queries won't be interrupted.
|
||||
|
||||
You can specify a conditional query in the config. In that case, the corresponding query executes only when the condition query returns the value `1` or `true`.
|
||||
|
||||
:::note
|
||||
If the condition query returns any other value than `1` or `true`, the result will be interpreted as `false`, and the corresponding won't be executed.
|
||||
:::
|
@ -587,6 +587,54 @@ static void sanityChecks(Server & server)
|
||||
}
|
||||
}
|
||||
|
||||
void loadStartupScripts(const Poco::Util::AbstractConfiguration & config, ContextMutablePtr context, Poco::Logger * log)
|
||||
{
|
||||
try
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
config.keys("startup_scripts", keys);
|
||||
|
||||
SetResultDetailsFunc callback;
|
||||
for (const auto & key : keys)
|
||||
{
|
||||
std::string full_prefix = "startup_scripts." + key;
|
||||
|
||||
if (config.has(full_prefix + ".condition"))
|
||||
{
|
||||
auto condition = config.getString(full_prefix + ".condition");
|
||||
auto condition_read_buffer = ReadBufferFromString(condition);
|
||||
auto condition_write_buffer = WriteBufferFromOwnString();
|
||||
|
||||
LOG_DEBUG(log, "Checking startup query condition `{}`", condition);
|
||||
executeQuery(condition_read_buffer, condition_write_buffer, true, context, callback, QueryFlags{ .internal = true }, std::nullopt, {});
|
||||
|
||||
auto result = condition_write_buffer.str();
|
||||
|
||||
if (result != "1\n" && result != "true\n")
|
||||
{
|
||||
if (result != "0\n" && result != "false\n")
|
||||
context->addWarningMessage(fmt::format("The condition query returned `{}`, which can't be interpreted as a boolean (`0`, `false`, `1`, `true`). Will skip this query.", result));
|
||||
|
||||
continue;
|
||||
}
|
||||
|
||||
LOG_DEBUG(log, "Condition is true, will execute the query next");
|
||||
}
|
||||
|
||||
auto query = config.getString(full_prefix + ".query");
|
||||
auto read_buffer = ReadBufferFromString(query);
|
||||
auto write_buffer = WriteBufferFromOwnString();
|
||||
|
||||
LOG_DEBUG(log, "Executing query `{}`", query);
|
||||
executeQuery(read_buffer, write_buffer, true, context, callback, QueryFlags{ .internal = true }, std::nullopt, {});
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log, "Failed to parse startup scripts file");
|
||||
}
|
||||
}
|
||||
|
||||
static void initializeAzureSDKLogger(
|
||||
[[ maybe_unused ]] const ServerSettings & server_settings,
|
||||
[[ maybe_unused ]] int server_logs_level)
|
||||
@ -1990,6 +2038,11 @@ try
|
||||
/// otherwise there is a race condition between the system database initialization
|
||||
/// and creation of new tables in the database.
|
||||
waitLoad(TablesLoaderForegroundPoolId, system_startup_tasks);
|
||||
|
||||
/// Startup scripts can depend on the system log tables.
|
||||
if (config().has("startup_scripts") && !server_settings.prepare_system_log_tables_on_startup.changed)
|
||||
global_context->setServerSetting("prepare_system_log_tables_on_startup", true);
|
||||
|
||||
/// After attaching system databases we can initialize system log.
|
||||
global_context->initializeSystemLogs();
|
||||
global_context->setSystemZooKeeperLogAfterInitializationIfNeeded();
|
||||
@ -2138,6 +2191,9 @@ try
|
||||
load_metadata_tasks.clear();
|
||||
load_metadata_tasks.shrink_to_fit();
|
||||
|
||||
if (config().has("startup_scripts"))
|
||||
loadStartupScripts(config(), global_context, log);
|
||||
|
||||
{
|
||||
std::lock_guard lock(servers_lock);
|
||||
for (auto & server : servers)
|
||||
|
@ -153,6 +153,7 @@ namespace DB
|
||||
M(Bool, enable_azure_sdk_logging, false, "Enables logging from Azure sdk", 0) \
|
||||
M(String, merge_workload, "default", "Name of workload to be used to access resources for all merges (may be overridden by a merge tree setting)", 0) \
|
||||
M(String, mutation_workload, "default", "Name of workload to be used to access resources for all mutations (may be overridden by a merge tree setting)", 0) \
|
||||
M(Bool, prepare_system_log_tables_on_startup, false, "If true, ClickHouse creates all configured `system.*_log` tables before the startup. It can be helpful if some startup scripts depend on these tables.", 0) \
|
||||
M(Double, gwp_asan_force_sample_probability, 0.0003, "Probability that an allocation from specific places will be sampled by GWP Asan (i.e. PODArray allocations)", 0) \
|
||||
M(UInt64, config_reload_interval_ms, 2000, "How often clickhouse will reload config and check for new changes", 0) \
|
||||
|
||||
|
@ -2238,6 +2238,12 @@ void Context::setSetting(std::string_view name, const Field & value)
|
||||
contextSanityClampSettingsWithLock(*this, settings, lock);
|
||||
}
|
||||
|
||||
void Context::setServerSetting(std::string_view name, const Field & value)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
shared->server_settings.set(name, value);
|
||||
}
|
||||
|
||||
void Context::applySettingChange(const SettingChange & change)
|
||||
{
|
||||
try
|
||||
|
@ -823,6 +823,7 @@ public:
|
||||
/// Set settings by name.
|
||||
void setSetting(std::string_view name, const String & value);
|
||||
void setSetting(std::string_view name, const Field & value);
|
||||
void setServerSetting(std::string_view name, const Field & value);
|
||||
void applySettingChange(const SettingChange & change);
|
||||
void applySettingsChanges(const SettingsChanges & changes);
|
||||
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Common/MemoryTrackerBlockerInThread.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Core/ServerSettings.h>
|
||||
#include <Interpreters/AsynchronousInsertLog.h>
|
||||
#include <Interpreters/AsynchronousMetricLog.h>
|
||||
#include <Interpreters/BackupLog.h>
|
||||
@ -356,10 +357,15 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf
|
||||
if (blob_storage_log)
|
||||
logs.emplace_back(blob_storage_log.get());
|
||||
|
||||
bool should_prepare = global_context->getServerSettings().prepare_system_log_tables_on_startup;
|
||||
try
|
||||
{
|
||||
for (auto & log : logs)
|
||||
{
|
||||
log->startup();
|
||||
if (should_prepare)
|
||||
log->prepareTable();
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
|
@ -135,6 +135,12 @@ public:
|
||||
|
||||
void stopFlushThread() override;
|
||||
|
||||
/** Creates new table if it does not exist.
|
||||
* Renames old table if its structure is not suitable.
|
||||
* This cannot be done in constructor to avoid deadlock while renaming a table under locked Context when SystemLog object is created.
|
||||
*/
|
||||
void prepareTable() override;
|
||||
|
||||
protected:
|
||||
LoggerPtr log;
|
||||
|
||||
@ -145,12 +151,6 @@ protected:
|
||||
|
||||
StoragePtr getStorage() const;
|
||||
|
||||
/** Creates new table if it does not exist.
|
||||
* Renames old table if its structure is not suitable.
|
||||
* This cannot be done in constructor to avoid deadlock while renaming a table under locked Context when SystemLog object is created.
|
||||
*/
|
||||
void prepareTable() override;
|
||||
|
||||
/// Some tables can override settings for internal queries
|
||||
virtual void addSettingsForQuery(ContextMutablePtr & mutable_context, IAST::QueryKind query_kind) const;
|
||||
|
||||
|
0
tests/integration/test_startup_scripts/__init__.py
Normal file
0
tests/integration/test_startup_scripts/__init__.py
Normal file
@ -0,0 +1,8 @@
|
||||
<clickhouse>
|
||||
<query_log>
|
||||
<database>system</database>
|
||||
<table>query_log</table>
|
||||
<partition_by>toYYYYMM(event_date)</partition_by>
|
||||
<flush_interval_milliseconds>1000</flush_interval_milliseconds>
|
||||
</query_log>
|
||||
</clickhouse>
|
@ -0,0 +1,17 @@
|
||||
<clickhouse>
|
||||
<startup_scripts>
|
||||
<scripts>
|
||||
<query>CREATE ROLE OR REPLACE testrole</query>
|
||||
</scripts>
|
||||
<scripts>
|
||||
<query>GRANT CREATE USER, ALTER USER, DROP USER, SHOW USERS, SHOW CREATE USER ON *.* TO 'testrole' WITH GRANT OPTION;</query>
|
||||
</scripts>
|
||||
<scripts>
|
||||
<query>CREATE TABLE TestTable (id UInt64) ENGINE=TinyLog</query>
|
||||
<condition>SELECT 1;</condition>
|
||||
</scripts>
|
||||
<scripts>
|
||||
<query>SELECT * FROM system.query_log LIMIT 1</query>
|
||||
</scripts>
|
||||
</startup_scripts>
|
||||
</clickhouse>
|
41
tests/integration/test_startup_scripts/configs/users.xml
Normal file
41
tests/integration/test_startup_scripts/configs/users.xml
Normal file
@ -0,0 +1,41 @@
|
||||
<clickhouse>
|
||||
|
||||
<profiles>
|
||||
<default>
|
||||
</default>
|
||||
|
||||
<readonly>
|
||||
<readonly>1</readonly>
|
||||
</readonly>
|
||||
</profiles>
|
||||
|
||||
<users>
|
||||
<default>
|
||||
|
||||
<password></password>
|
||||
|
||||
<networks>
|
||||
<ip>::/0</ip>
|
||||
</networks>
|
||||
|
||||
<profile>default</profile>
|
||||
|
||||
<quota>default</quota>
|
||||
|
||||
</default>
|
||||
</users>
|
||||
|
||||
<quotas>
|
||||
<default>
|
||||
<interval>
|
||||
<duration>3600</duration>
|
||||
|
||||
<queries>0</queries>
|
||||
<errors>0</errors>
|
||||
<result_rows>0</result_rows>
|
||||
<read_rows>0</read_rows>
|
||||
<execution_time>0</execution_time>
|
||||
</interval>
|
||||
</default>
|
||||
</quotas>
|
||||
</clickhouse>
|
21
tests/integration/test_startup_scripts/test.py
Normal file
21
tests/integration/test_startup_scripts/test.py
Normal file
@ -0,0 +1,21 @@
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
|
||||
def test_startup_scripts():
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
|
||||
node = cluster.add_instance(
|
||||
"node",
|
||||
main_configs=[
|
||||
"configs/config.d/query_log.xml",
|
||||
"configs/config.d/startup_scripts.xml",
|
||||
],
|
||||
with_zookeeper=False,
|
||||
)
|
||||
|
||||
try:
|
||||
cluster.start()
|
||||
assert node.query("SHOW TABLES") == "TestTable\n"
|
||||
|
||||
finally:
|
||||
cluster.shutdown()
|
Loading…
Reference in New Issue
Block a user