mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Load the system database first. [#CLICKHOUSE-3044]
This commit is contained in:
parent
4cd9df27ab
commit
67fdd0f544
@ -1147,6 +1147,12 @@ zkutil::ZooKeeperPtr Context::getZooKeeper() const
|
||||
return shared->zookeeper;
|
||||
}
|
||||
|
||||
bool Context::hasZooKeeper() const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(shared->zookeeper_mutex);
|
||||
return shared->zookeeper != nullptr;
|
||||
}
|
||||
|
||||
|
||||
void Context::setInterserverIOAddress(const String & host, UInt16 port)
|
||||
{
|
||||
|
@ -270,6 +270,8 @@ public:
|
||||
void setZooKeeper(std::shared_ptr<zkutil::ZooKeeper> zookeeper);
|
||||
/// If the current session is expired at the time of the call, synchronously creates and returns a new session with the startNewSession() call.
|
||||
std::shared_ptr<zkutil::ZooKeeper> getZooKeeper() const;
|
||||
/// Has ready or expired ZooKeeper
|
||||
bool hasZooKeeper() const;
|
||||
|
||||
/// Create a cache of marks of specified size. This can be done only once.
|
||||
void setMarkCache(size_t cache_size_in_bytes);
|
||||
|
@ -15,6 +15,10 @@
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/loadMetadata.h>
|
||||
|
||||
#include <Databases/DatabaseOrdinary.h>
|
||||
|
||||
#include <Storages/System/attachSystemTables.h>
|
||||
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
|
||||
@ -62,6 +66,7 @@ void loadMetadata(Context & context)
|
||||
ThreadPool thread_pool(SettingMaxThreads().getAutoValue());
|
||||
|
||||
/// Loop over databases.
|
||||
std::map<String, String> databases;
|
||||
Poco::DirectoryIterator dir_end;
|
||||
for (Poco::DirectoryIterator it(path); it != dir_end; ++it)
|
||||
{
|
||||
@ -72,13 +77,18 @@ void loadMetadata(Context & context)
|
||||
if (it.name().at(0) == '.')
|
||||
continue;
|
||||
|
||||
String database = unescapeForFileName(it.name());
|
||||
databases.emplace(unescapeForFileName(it.name()), it.path().toString());
|
||||
}
|
||||
|
||||
static constexpr auto SYSTEM_DATABASE = "system";
|
||||
|
||||
auto load_database = [&] (const String & database, const String & database_path)
|
||||
{
|
||||
/// There may exist .sql file with database creation statement.
|
||||
/// Or, if it is absent, then database with default engine is created.
|
||||
|
||||
String database_attach_query;
|
||||
String database_metadata_file = it->path() + ".sql";
|
||||
String database_metadata_file = escapeForFileName(database) + ".sql";
|
||||
|
||||
if (Poco::File(database_metadata_file).exists())
|
||||
{
|
||||
@ -92,11 +102,46 @@ void loadMetadata(Context & context)
|
||||
|
||||
/// For special system database, always restore data
|
||||
/// to not fail on loading query_log and part_log tables, if they are corrupted.
|
||||
if (database == "system")
|
||||
if (database == SYSTEM_DATABASE)
|
||||
force_restore_data = true;
|
||||
|
||||
executeCreateQuery(database_attach_query, context, database, it->path(), thread_pool, force_restore_data);
|
||||
executeCreateQuery(database_attach_query, context, database, database_path, thread_pool, force_restore_data);
|
||||
};
|
||||
|
||||
/// At first, load the system database
|
||||
auto it_system = databases.find(SYSTEM_DATABASE);
|
||||
if (it_system != databases.end())
|
||||
{
|
||||
load_database(it_system->first, it_system->second);
|
||||
databases.erase(it_system);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Initialize system database manually
|
||||
String global_path = context.getPath();
|
||||
Poco::File(global_path + "data/system").createDirectories();
|
||||
Poco::File(global_path + "metadata/system").createDirectories();
|
||||
|
||||
auto system_database = std::make_shared<DatabaseOrdinary>("system", global_path + "metadata/system/");
|
||||
context.addDatabase("system", system_database);
|
||||
|
||||
/// 'has_force_restore_data_flag' is true, to not fail on loading query_log table, if it is corrupted.
|
||||
system_database->loadTables(context, nullptr, true);
|
||||
}
|
||||
|
||||
/// After the system database is created, attach virtual system tables (in addition to query_lof and part_log)
|
||||
{
|
||||
DatabasePtr system_database = context.getDatabase("system");
|
||||
|
||||
if (context.getApplicationType() == Context::ApplicationType::SERVER)
|
||||
attachSystemTablesServer(system_database, &context, context.hasZooKeeper());
|
||||
else if (context.getApplicationType() == Context::ApplicationType::LOCAL_SERVER)
|
||||
attachSystemTablesLocal(system_database);
|
||||
}
|
||||
|
||||
/// Then, load remaining databases
|
||||
for (const auto & elem : databases)
|
||||
load_database(elem.first, elem.second);
|
||||
|
||||
thread_pool.wait();
|
||||
|
||||
|
@ -314,8 +314,10 @@ try
|
||||
loadMetadata(*context);
|
||||
LOG_DEBUG(log, "Loaded metadata.");
|
||||
}
|
||||
|
||||
attachSystemTables();
|
||||
else
|
||||
{
|
||||
attachSystemTables();
|
||||
}
|
||||
|
||||
processQueries();
|
||||
|
||||
|
@ -16,7 +16,6 @@
|
||||
#include <Common/StringUtils.h>
|
||||
#include <Common/getFQDNOrHostName.h>
|
||||
#include <Common/getMultipleKeysFromConfig.h>
|
||||
#include <Databases/DatabaseOrdinary.h>
|
||||
#include <IO/HTTPCommon.h>
|
||||
#include <Interpreters/AsynchronousMetrics.h>
|
||||
#include <Interpreters/ProcessList.h>
|
||||
@ -390,23 +389,6 @@ int Server::main(const std::vector<std::string> & args)
|
||||
|
||||
global_context->setCurrentDatabase(default_database);
|
||||
|
||||
/// Create system tables.
|
||||
if (!global_context->isDatabaseExist("system"))
|
||||
{
|
||||
Poco::File(path + "data/system").createDirectories();
|
||||
Poco::File(path + "metadata/system").createDirectories();
|
||||
|
||||
auto system_database = std::make_shared<DatabaseOrdinary>("system", path + "metadata/system/");
|
||||
global_context->addDatabase("system", system_database);
|
||||
|
||||
/// 'has_force_restore_data_flag' is true, to not fail on loading query_log table, if it is corrupted.
|
||||
system_database->loadTables(*global_context, nullptr, true);
|
||||
}
|
||||
|
||||
DatabasePtr system_database = global_context->getDatabase("system");
|
||||
|
||||
attachSystemTablesServer(system_database, global_context.get(), has_zookeeper);
|
||||
|
||||
bool has_resharding_worker = false;
|
||||
if (has_zookeeper && config().has("resharding"))
|
||||
{
|
||||
@ -654,8 +636,7 @@ int Server::main(const std::vector<std::string> & args)
|
||||
|
||||
/// This object will periodically calculate some metrics.
|
||||
AsynchronousMetrics async_metrics(*global_context);
|
||||
|
||||
attachSystemTablesAsync(system_database, async_metrics);
|
||||
attachSystemTablesAsync(global_context->getDatabase("system"), async_metrics);
|
||||
|
||||
std::vector<std::unique_ptr<MetricsTransmitter>> metrics_transmitters;
|
||||
for (const auto & graphite_key : DB::getMultipleKeysFromConfig(config(), "", "graphite"))
|
||||
|
@ -47,7 +47,7 @@ class ClickHouseCluster:
|
||||
self.is_up = False
|
||||
|
||||
|
||||
def add_instance(self, name, config_dir=None, main_configs=[], user_configs=[], macroses={}, with_zookeeper=False):
|
||||
def add_instance(self, name, config_dir=None, main_configs=[], user_configs=[], macroses={}, with_zookeeper=False, clickhouse_path_dir=None):
|
||||
"""Add an instance to the cluster.
|
||||
|
||||
name - the name of the instance directory and the value of the 'instance' macro in ClickHouse.
|
||||
@ -63,7 +63,7 @@ class ClickHouseCluster:
|
||||
if name in self.instances:
|
||||
raise Exception("Can\'t add instance `%s': there is already an instance with the same name!" % name)
|
||||
|
||||
instance = ClickHouseInstance(self, self.base_dir, name, config_dir, main_configs, user_configs, macroses, with_zookeeper, self.base_configs_dir, self.server_bin_path)
|
||||
instance = ClickHouseInstance(self, self.base_dir, name, config_dir, main_configs, user_configs, macroses, with_zookeeper, self.base_configs_dir, self.server_bin_path, clickhouse_path_dir)
|
||||
self.instances[name] = instance
|
||||
self.base_cmd.extend(['--file', instance.docker_compose_path])
|
||||
if with_zookeeper and not self.with_zookeeper:
|
||||
@ -148,7 +148,7 @@ MACROS_CONFIG_TEMPLATE = '''
|
||||
class ClickHouseInstance:
|
||||
def __init__(
|
||||
self, cluster, base_path, name, custom_config_dir, custom_main_configs, custom_user_configs, macroses,
|
||||
with_zookeeper, base_configs_dir, server_bin_path):
|
||||
with_zookeeper, base_configs_dir, server_bin_path, clickhouse_path_dir):
|
||||
|
||||
self.name = name
|
||||
self.base_cmd = cluster.base_cmd[:]
|
||||
@ -158,6 +158,7 @@ class ClickHouseInstance:
|
||||
self.custom_config_dir = p.abspath(p.join(base_path, custom_config_dir)) if custom_config_dir else None
|
||||
self.custom_main_config_paths = [p.abspath(p.join(base_path, c)) for c in custom_main_configs]
|
||||
self.custom_user_config_paths = [p.abspath(p.join(base_path, c)) for c in custom_user_configs]
|
||||
self.clickhouse_path_dir = p.abspath(p.join(base_path, clickhouse_path_dir)) if clickhouse_path_dir else None
|
||||
self.macroses = macroses if macroses is not None else {}
|
||||
self.with_zookeeper = with_zookeeper
|
||||
|
||||
@ -281,6 +282,8 @@ class ClickHouseInstance:
|
||||
|
||||
db_dir = p.join(self.path, 'database')
|
||||
os.mkdir(db_dir)
|
||||
if self.clickhouse_path_dir is not None:
|
||||
distutils.dir_util.copy_tree(self.clickhouse_path_dir, db_dir)
|
||||
|
||||
logs_dir = p.join(self.path, 'logs')
|
||||
os.mkdir(logs_dir)
|
||||
|
@ -0,0 +1,12 @@
|
||||
ATTACH TABLE sophisticated_default
|
||||
(
|
||||
a UInt8 DEFAULT
|
||||
(
|
||||
SELECT number FROM system.numbers LIMIT 3,1
|
||||
),
|
||||
b UInt8 ALIAS
|
||||
(
|
||||
SELECT dummy+9 FROM system.one
|
||||
),
|
||||
c UInt8
|
||||
) ENGINE = Memory
|
21
dbms/tests/integration/test_server_initialization/test.py
Normal file
21
dbms/tests/integration/test_server_initialization/test.py
Normal file
@ -0,0 +1,21 @@
|
||||
import pytest
|
||||
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
instance = cluster.add_instance('dummy', clickhouse_path_dir='clickhouse_path')
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def started_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
yield cluster
|
||||
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
def test_sophisticated_default(started_cluster):
|
||||
instance.query("INSERT INTO sophisticated_default (c) VALUES (0)")
|
||||
assert instance.query("SELECT a, b, c FROM sophisticated_default") == "3\t9\t0\n"
|
||||
|
@ -5,3 +5,5 @@
|
||||
0 0 0
|
||||
0 0 0
|
||||
0 0 0
|
||||
|
||||
1
|
||||
|
@ -34,3 +34,16 @@ echo
|
||||
pack_unpack_compare "SELECT name, is_aggregate FROM system.functions" "name String, is_aggregate UInt8" "TabSeparated"
|
||||
pack_unpack_compare "SELECT name, is_aggregate FROM system.functions" "name String, is_aggregate UInt8" "Native"
|
||||
pack_unpack_compare "SELECT name, is_aggregate FROM system.functions" "name String, is_aggregate UInt8" "TSKV"
|
||||
echo
|
||||
clickhouse-local -q "CREATE TABLE sophisticated_default
|
||||
(
|
||||
a UInt8 DEFAULT
|
||||
(
|
||||
SELECT number FROM system.numbers LIMIT 3,1
|
||||
),
|
||||
b UInt8 ALIAS
|
||||
(
|
||||
SELECT dummy+9 FROM system.one
|
||||
),
|
||||
c UInt8
|
||||
) ENGINE = Memory; SELECT count() FROM system.tables WHERE name='sophisticated_default';" 2>/dev/null
|
||||
|
Loading…
Reference in New Issue
Block a user