mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge 5999763a8d
into 44b4bd38b9
This commit is contained in:
commit
2c90f9345a
@ -52,6 +52,7 @@
|
||||
#include <boost/program_options/options_description.hpp>
|
||||
#include <base/argsToConfig.h>
|
||||
#include <filesystem>
|
||||
#include <Common/filesystemHelpers.h>
|
||||
|
||||
#include "config.h"
|
||||
|
||||
@ -262,7 +263,19 @@ static DatabasePtr createMemoryDatabaseIfNotExists(ContextPtr context, const Str
|
||||
static DatabasePtr createClickHouseLocalDatabaseOverlay(const String & name_, ContextPtr context)
|
||||
{
|
||||
auto overlay = std::make_shared<DatabasesOverlay>(name_, context);
|
||||
overlay->registerNextDatabase(std::make_shared<DatabaseAtomic>(name_, fs::weakly_canonical(context->getPath()), UUIDHelpers::generateV4(), context));
|
||||
|
||||
UUID default_database_uuid;
|
||||
|
||||
fs::path existing_path_symlink = fs::weakly_canonical(context->getPath()) / "metadata" / "default";
|
||||
if (FS::isSymlinkNoThrow(existing_path_symlink))
|
||||
default_database_uuid = parse<UUID>(FS::readSymlink(existing_path_symlink).parent_path().filename());
|
||||
else
|
||||
default_database_uuid = UUIDHelpers::generateV4();
|
||||
|
||||
fs::path default_database_metadata_path = fs::weakly_canonical(context->getPath()) / "store"
|
||||
/ DatabaseCatalog::getPathForUUID(default_database_uuid);
|
||||
|
||||
overlay->registerNextDatabase(std::make_shared<DatabaseAtomic>(name_, default_database_metadata_path, default_database_uuid, context));
|
||||
overlay->registerNextDatabase(std::make_shared<DatabaseFilesystem>(name_, "", context));
|
||||
return overlay;
|
||||
}
|
||||
@ -274,7 +287,7 @@ void LocalServer::tryInitPath()
|
||||
|
||||
if (getClientConfiguration().has("path"))
|
||||
{
|
||||
// User-supplied path.
|
||||
/// User-supplied path.
|
||||
path = getClientConfiguration().getString("path");
|
||||
Poco::trimInPlace(path);
|
||||
|
||||
@ -288,15 +301,15 @@ void LocalServer::tryInitPath()
|
||||
}
|
||||
else
|
||||
{
|
||||
// The path is not provided explicitly - use a unique path in the system temporary directory
|
||||
// (or in the current dir if a temporary doesn't exist)
|
||||
/// The user requested to use a temporary path - use a unique path in the system temporary directory
|
||||
/// (or in the current dir if a temporary doesn't exist)
|
||||
LoggerRawPtr log = &logger();
|
||||
std::filesystem::path parent_folder;
|
||||
std::filesystem::path default_path;
|
||||
|
||||
try
|
||||
{
|
||||
// try to guess a tmp folder name, and check if it's a directory (throw exception otherwise)
|
||||
/// Try to guess a tmp folder name, and check if it's a directory (throw an exception otherwise).
|
||||
parent_folder = std::filesystem::temp_directory_path();
|
||||
|
||||
}
|
||||
@ -324,7 +337,7 @@ void LocalServer::tryInitPath()
|
||||
temporary_directory_to_delete = default_path;
|
||||
|
||||
path = default_path.string();
|
||||
LOG_DEBUG(log, "Working directory created: {}", path);
|
||||
LOG_DEBUG(log, "Working directory will be created as needed: {}", path);
|
||||
}
|
||||
|
||||
global_context->setPath(fs::path(path) / "");
|
||||
@ -826,30 +839,38 @@ void LocalServer::processConfig()
|
||||
|
||||
if (getClientConfiguration().has("path"))
|
||||
{
|
||||
String path = global_context->getPath();
|
||||
fs::create_directories(fs::path(path));
|
||||
|
||||
/// Lock path directory before read
|
||||
status.emplace(fs::path(path) / "status", StatusFile::write_full_info);
|
||||
|
||||
LOG_DEBUG(log, "Loading metadata from {}", path);
|
||||
auto load_system_metadata_tasks = loadMetadataSystem(global_context);
|
||||
attachSystemTablesServer(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE), false);
|
||||
attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA));
|
||||
attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE));
|
||||
waitLoad(TablesLoaderForegroundPoolId, load_system_metadata_tasks);
|
||||
|
||||
if (!getClientConfiguration().has("only-system-tables"))
|
||||
String path = global_context->getPath();
|
||||
|
||||
/// Lock path directory before read
|
||||
fs::create_directories(fs::path(path));
|
||||
status.emplace(fs::path(path) / "status", StatusFile::write_full_info);
|
||||
|
||||
if (fs::exists(fs::path(path) / "metadata"))
|
||||
{
|
||||
DatabaseCatalog::instance().createBackgroundTasks();
|
||||
waitLoad(loadMetadata(global_context));
|
||||
DatabaseCatalog::instance().startupBackgroundTasks();
|
||||
LOG_DEBUG(log, "Loading metadata from {}", path);
|
||||
|
||||
if (fs::exists(std::filesystem::path(path) / "metadata" / "system.sql"))
|
||||
{
|
||||
LoadTaskPtrs load_system_metadata_tasks = loadMetadataSystem(global_context);
|
||||
waitLoad(TablesLoaderForegroundPoolId, load_system_metadata_tasks);
|
||||
}
|
||||
|
||||
if (!getClientConfiguration().has("only-system-tables"))
|
||||
{
|
||||
DatabaseCatalog::instance().createBackgroundTasks();
|
||||
waitLoad(loadMetadata(global_context));
|
||||
DatabaseCatalog::instance().startupBackgroundTasks();
|
||||
}
|
||||
|
||||
/// For ClickHouse local if path is not set the loader will be disabled.
|
||||
global_context->getUserDefinedSQLObjectsStorage().loadObjects();
|
||||
|
||||
LOG_DEBUG(log, "Loaded metadata.");
|
||||
}
|
||||
|
||||
/// For ClickHouse local if path is not set the loader will be disabled.
|
||||
global_context->getUserDefinedSQLObjectsStorage().loadObjects();
|
||||
|
||||
LOG_DEBUG(log, "Loaded metadata.");
|
||||
}
|
||||
else if (!getClientConfiguration().has("no-system-tables"))
|
||||
{
|
||||
@ -919,7 +940,7 @@ void LocalServer::addOptions(OptionsDescription & options_description)
|
||||
("logger.level", po::value<std::string>(), "Log level")
|
||||
|
||||
("no-system-tables", "do not attach system tables (better startup time)")
|
||||
("path", po::value<std::string>(), "Storage path")
|
||||
("path", po::value<std::string>(), "Storage path. If it was not specified, we will use a temporary directory, that is cleaned up on exit.")
|
||||
("only-system-tables", "attach only system tables from specified path")
|
||||
("top_level_domains_path", po::value<std::string>(), "Path to lists with custom TLDs")
|
||||
;
|
||||
@ -961,8 +982,6 @@ void LocalServer::processOptions(const OptionsDescription &, const CommandLineOp
|
||||
getClientConfiguration().setBool("no-system-tables", true);
|
||||
if (options.count("only-system-tables"))
|
||||
getClientConfiguration().setBool("only-system-tables", true);
|
||||
if (options.count("database"))
|
||||
getClientConfiguration().setString("default_database", options["database"].as<std::string>());
|
||||
|
||||
if (options.count("input-format"))
|
||||
getClientConfiguration().setString("table-data-format", options["input-format"].as<std::string>());
|
||||
|
@ -76,7 +76,8 @@ void DatabaseAtomic::createDirectories()
|
||||
void DatabaseAtomic::createDirectoriesUnlocked()
|
||||
{
|
||||
DatabaseOnDisk::createDirectoriesUnlocked();
|
||||
fs::create_directories(fs::path(getContext()->getPath()) / "metadata");
|
||||
fs::path catalog_path = fs::path(getContext()->getPath()) / "metadata";
|
||||
fs::create_directories(catalog_path);
|
||||
fs::create_directories(path_to_table_symlinks);
|
||||
tryCreateMetadataSymlink();
|
||||
}
|
||||
@ -604,12 +605,12 @@ void DatabaseAtomic::tryCreateMetadataSymlink()
|
||||
{
|
||||
/// Symlinks in data/db_name/ directory and metadata/db_name/ are not used by ClickHouse,
|
||||
/// it's needed only for convenient introspection.
|
||||
assert(path_to_metadata_symlink != metadata_path);
|
||||
chassert(path_to_metadata_symlink != metadata_path);
|
||||
fs::path metadata_symlink(path_to_metadata_symlink);
|
||||
if (fs::exists(metadata_symlink))
|
||||
{
|
||||
if (!FS::isSymlink(metadata_symlink))
|
||||
throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "Directory {} exists", path_to_metadata_symlink);
|
||||
throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "Directory {} already exists", path_to_metadata_symlink);
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -63,7 +63,7 @@ void DatabasesOverlay::createTable(ContextPtr context_, const String & table_nam
|
||||
}
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"There is no databases for CREATE TABLE `{}` query in database `{}` (engine {})",
|
||||
"There are no databases for CREATE TABLE `{}` query in database `{}` (engine {})",
|
||||
table_name,
|
||||
getDatabaseName(),
|
||||
getEngineName());
|
||||
@ -81,7 +81,7 @@ void DatabasesOverlay::dropTable(ContextPtr context_, const String & table_name,
|
||||
}
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"There is no databases for DROP TABLE `{}` query in database `{}` (engine {})",
|
||||
"There are no databases for DROP TABLE `{}` query in database `{}` (engine {})",
|
||||
table_name,
|
||||
getDatabaseName(),
|
||||
getEngineName());
|
||||
@ -104,7 +104,7 @@ void DatabasesOverlay::attachTable(
|
||||
}
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"There is no databases for ATTACH TABLE `{}` query in database `{}` (engine {})",
|
||||
"There are no databases for ATTACH TABLE `{}` query in database `{}` (engine {})",
|
||||
table_name,
|
||||
getDatabaseName(),
|
||||
getEngineName());
|
||||
@ -120,7 +120,7 @@ StoragePtr DatabasesOverlay::detachTable(ContextPtr context_, const String & tab
|
||||
}
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"There is no databases for DETACH TABLE `{}` query in database `{}` (engine {})",
|
||||
"There are no databases for DETACH TABLE `{}` query in database `{}` (engine {})",
|
||||
table_name,
|
||||
getDatabaseName(),
|
||||
getEngineName());
|
||||
@ -255,7 +255,7 @@ void DatabasesOverlay::alterTable(ContextPtr local_context, const StorageID & ta
|
||||
}
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"There is no databases for ALTER TABLE `{}` query in database `{}` (engine {})",
|
||||
"There are no databases for ALTER TABLE `{}` query in database `{}` (engine {})",
|
||||
table_id.table_name,
|
||||
getDatabaseName(),
|
||||
getEngineName());
|
||||
@ -313,4 +313,251 @@ DatabaseTablesIteratorPtr DatabasesOverlay::getTablesIterator(ContextPtr context
|
||||
return std::make_unique<DatabaseTablesSnapshotIterator>(std::move(tables), getDatabaseName());
|
||||
}
|
||||
|
||||
bool DatabasesOverlay::canContainMergeTreeTables() const
|
||||
{
|
||||
for (const auto & db : databases)
|
||||
if (db->canContainMergeTreeTables())
|
||||
return true;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool DatabasesOverlay::canContainDistributedTables() const
|
||||
{
|
||||
for (const auto & db : databases)
|
||||
if (db->canContainDistributedTables())
|
||||
return true;
|
||||
return false;
|
||||
}
|
||||
|
||||
void DatabasesOverlay::loadStoredObjects(ContextMutablePtr local_context, LoadingStrictnessLevel mode)
|
||||
{
|
||||
for (auto & db : databases)
|
||||
if (!db->isReadOnly())
|
||||
db->loadStoredObjects(local_context, mode);
|
||||
}
|
||||
|
||||
bool DatabasesOverlay::supportsLoadingInTopologicalOrder() const
|
||||
{
|
||||
for (const auto & db : databases)
|
||||
if (db->supportsLoadingInTopologicalOrder())
|
||||
return true;
|
||||
return false;
|
||||
}
|
||||
|
||||
void DatabasesOverlay::beforeLoadingMetadata(ContextMutablePtr local_context, LoadingStrictnessLevel mode)
|
||||
{
|
||||
for (auto & db : databases)
|
||||
if (!db->isReadOnly())
|
||||
db->beforeLoadingMetadata(local_context, mode);
|
||||
}
|
||||
|
||||
void DatabasesOverlay::loadTablesMetadata(ContextPtr local_context, ParsedTablesMetadata & metadata, bool is_startup)
|
||||
{
|
||||
for (auto & db : databases)
|
||||
if (!db->isReadOnly())
|
||||
db->loadTablesMetadata(local_context, metadata, is_startup);
|
||||
}
|
||||
|
||||
void DatabasesOverlay::loadTableFromMetadata(
|
||||
ContextMutablePtr local_context,
|
||||
const String & file_path,
|
||||
const QualifiedTableName & name,
|
||||
const ASTPtr & ast,
|
||||
LoadingStrictnessLevel mode)
|
||||
{
|
||||
for (auto & db : databases)
|
||||
{
|
||||
if (db->isReadOnly())
|
||||
continue;
|
||||
|
||||
try
|
||||
{
|
||||
db->loadTableFromMetadata(local_context, file_path, name, ast, mode);
|
||||
return;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
continue;
|
||||
}
|
||||
}
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"There are no databases capable of loading table `{}` from path `{}` in database `{}` (engine {})",
|
||||
name,
|
||||
file_path,
|
||||
getDatabaseName(),
|
||||
getEngineName());
|
||||
}
|
||||
|
||||
LoadTaskPtr DatabasesOverlay::loadTableFromMetadataAsync(
|
||||
AsyncLoader & async_loader,
|
||||
LoadJobSet load_after,
|
||||
ContextMutablePtr local_context,
|
||||
const String & file_path,
|
||||
const QualifiedTableName & name,
|
||||
const ASTPtr & ast,
|
||||
LoadingStrictnessLevel mode)
|
||||
{
|
||||
for (auto & db : databases)
|
||||
{
|
||||
if (db->isReadOnly())
|
||||
continue;
|
||||
|
||||
try
|
||||
{
|
||||
return db->loadTableFromMetadataAsync(async_loader, load_after, local_context, file_path, name, ast, mode);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
continue;
|
||||
}
|
||||
}
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"There are no databases capable of loading table `{}` from path `{}` in database `{}` (engine {})",
|
||||
name,
|
||||
file_path,
|
||||
getDatabaseName(),
|
||||
getEngineName());
|
||||
}
|
||||
|
||||
LoadTaskPtr DatabasesOverlay::startupTableAsync(
|
||||
AsyncLoader & async_loader,
|
||||
LoadJobSet startup_after,
|
||||
const QualifiedTableName & name,
|
||||
LoadingStrictnessLevel mode)
|
||||
{
|
||||
for (auto & db : databases)
|
||||
{
|
||||
if (db->isReadOnly())
|
||||
continue;
|
||||
|
||||
try
|
||||
{
|
||||
return db->startupTableAsync(async_loader, startup_after, name, mode);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
continue;
|
||||
}
|
||||
}
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"There are no databases capable of starting up table `{}` in database `{}` (engine {})",
|
||||
name,
|
||||
getDatabaseName(),
|
||||
getEngineName());
|
||||
}
|
||||
|
||||
LoadTaskPtr DatabasesOverlay::startupDatabaseAsync(
|
||||
AsyncLoader & async_loader,
|
||||
LoadJobSet startup_after,
|
||||
LoadingStrictnessLevel mode)
|
||||
{
|
||||
for (auto & db : databases)
|
||||
{
|
||||
if (db->isReadOnly())
|
||||
continue;
|
||||
|
||||
try
|
||||
{
|
||||
return db->startupDatabaseAsync(async_loader, startup_after, mode);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
continue;
|
||||
}
|
||||
}
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"There are no databases capable of starting up asynchronously in database `{}` (engine {})",
|
||||
getDatabaseName(),
|
||||
getEngineName());
|
||||
}
|
||||
|
||||
void DatabasesOverlay::waitTableStarted(const String & name) const
|
||||
{
|
||||
for (const auto & db : databases)
|
||||
{
|
||||
if (db->isReadOnly())
|
||||
continue;
|
||||
|
||||
try
|
||||
{
|
||||
db->waitTableStarted(name);
|
||||
return;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
continue;
|
||||
}
|
||||
}
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"There are no databases capable of waiting for table startup `{}` in database `{}` (engine {})",
|
||||
name,
|
||||
getDatabaseName(),
|
||||
getEngineName());
|
||||
}
|
||||
|
||||
void DatabasesOverlay::waitDatabaseStarted() const
|
||||
{
|
||||
for (const auto & db : databases)
|
||||
{
|
||||
if (db->isReadOnly())
|
||||
continue;
|
||||
|
||||
try
|
||||
{
|
||||
db->waitDatabaseStarted();
|
||||
return;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
continue;
|
||||
}
|
||||
}
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"There are no databases capable of waiting for startup in database `{}` (engine {})",
|
||||
getDatabaseName(),
|
||||
getEngineName());
|
||||
}
|
||||
|
||||
void DatabasesOverlay::stopLoading()
|
||||
{
|
||||
for (auto & db : databases)
|
||||
{
|
||||
if (db->isReadOnly())
|
||||
continue;
|
||||
|
||||
try
|
||||
{
|
||||
db->stopLoading();
|
||||
return;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
continue;
|
||||
}
|
||||
}
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"There are no databases capable of stop loading in database `{}` (engine {})",
|
||||
getDatabaseName(),
|
||||
getEngineName());
|
||||
}
|
||||
|
||||
void DatabasesOverlay::checkMetadataFilenameAvailability(const String & table_name) const
|
||||
{
|
||||
for (const auto & db : databases)
|
||||
{
|
||||
if (db->isReadOnly())
|
||||
continue;
|
||||
db->checkMetadataFilenameAvailability(table_name);
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -66,6 +66,40 @@ public:
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
bool canContainMergeTreeTables() const override;
|
||||
bool canContainDistributedTables() const override;
|
||||
void loadStoredObjects(ContextMutablePtr local_context, LoadingStrictnessLevel mode) override;
|
||||
bool supportsLoadingInTopologicalOrder() const override;
|
||||
void beforeLoadingMetadata(ContextMutablePtr local_context, LoadingStrictnessLevel mode) override;
|
||||
void loadTablesMetadata(ContextPtr local_context, ParsedTablesMetadata & metadata, bool is_startup) override;
|
||||
void loadTableFromMetadata(
|
||||
ContextMutablePtr local_context,
|
||||
const String & file_path,
|
||||
const QualifiedTableName & name,
|
||||
const ASTPtr & ast,
|
||||
LoadingStrictnessLevel mode) override;
|
||||
LoadTaskPtr loadTableFromMetadataAsync(
|
||||
AsyncLoader & async_loader,
|
||||
LoadJobSet load_after,
|
||||
ContextMutablePtr local_context,
|
||||
const String & file_path,
|
||||
const QualifiedTableName & name,
|
||||
const ASTPtr & ast,
|
||||
LoadingStrictnessLevel mode) override;
|
||||
[[nodiscard]] LoadTaskPtr startupTableAsync(
|
||||
AsyncLoader & async_loader,
|
||||
LoadJobSet startup_after,
|
||||
const QualifiedTableName & name,
|
||||
LoadingStrictnessLevel mode) override;
|
||||
[[nodiscard]] LoadTaskPtr startupDatabaseAsync(
|
||||
AsyncLoader & async_loader,
|
||||
LoadJobSet startup_after,
|
||||
LoadingStrictnessLevel mode) override;
|
||||
void waitTableStarted(const String & name) const override;
|
||||
void waitDatabaseStarted() const override;
|
||||
void stopLoading() override;
|
||||
void checkMetadataFilenameAvailability(const String & table_name) const override;
|
||||
|
||||
protected:
|
||||
std::vector<DatabasePtr> databases;
|
||||
LoggerPtr log;
|
||||
|
@ -101,6 +101,10 @@ static void loadDatabase(
|
||||
const String & database_path,
|
||||
bool force_restore_data)
|
||||
{
|
||||
/// If it is already loaded.
|
||||
if (DatabaseCatalog::instance().isDatabaseExist(database))
|
||||
return;
|
||||
|
||||
String database_attach_query;
|
||||
String database_metadata_file = database_path + ".sql";
|
||||
|
||||
@ -195,21 +199,25 @@ LoadTaskPtrs loadMetadata(ContextMutablePtr context, const String & default_data
|
||||
|
||||
/// Loop over databases.
|
||||
std::map<String, String> databases;
|
||||
|
||||
/// Some databases don't have an .sql metadata file.
|
||||
std::map<String, String> orphan_directories_and_symlinks;
|
||||
fs::directory_iterator dir_end;
|
||||
for (fs::directory_iterator it(path); it != dir_end; ++it)
|
||||
{
|
||||
if (it->is_symlink())
|
||||
continue;
|
||||
|
||||
if (it->is_directory())
|
||||
if (it->is_symlink() || it->is_directory())
|
||||
{
|
||||
String db_name = it->path().filename().string();
|
||||
orphan_directories_and_symlinks.emplace(unescapeForFileName(db_name), fs::path(path) / db_name);
|
||||
continue;
|
||||
}
|
||||
|
||||
const auto current_file = it->path().filename().string();
|
||||
|
||||
/// TODO: DETACH DATABASE PERMANENTLY ?
|
||||
if (fs::path(current_file).extension() == ".sql")
|
||||
{
|
||||
String db_name = fs::path(current_file).stem();
|
||||
orphan_directories_and_symlinks.erase(db_name);
|
||||
if (!isSystemOrInformationSchema(db_name))
|
||||
databases.emplace(unescapeForFileName(db_name), fs::path(path) / db_name);
|
||||
}
|
||||
@ -249,7 +257,13 @@ LoadTaskPtrs loadMetadata(ContextMutablePtr context, const String & default_data
|
||||
loaded_databases.insert({name, DatabaseCatalog::instance().getDatabase(name)});
|
||||
}
|
||||
|
||||
auto mode = getLoadingStrictnessLevel(/* attach */ true, /* force_attach */ true, has_force_restore_data_flag, /*secondary*/ false);
|
||||
for (const auto & [name, db_path] : orphan_directories_and_symlinks)
|
||||
{
|
||||
loadDatabase(context, name, db_path, has_force_restore_data_flag);
|
||||
loaded_databases.insert({name, DatabaseCatalog::instance().getDatabase(name)});
|
||||
}
|
||||
|
||||
auto mode = getLoadingStrictnessLevel(/* attach */ true, /* force_attach */ true, has_force_restore_data_flag, /* secondary */ false);
|
||||
TablesLoader loader{context, std::move(loaded_databases), mode};
|
||||
auto load_tasks = loader.loadTablesAsync();
|
||||
auto startup_tasks = loader.startupTablesAsync();
|
||||
@ -501,7 +515,8 @@ void convertDatabasesEnginesIfNeed(const LoadTaskPtrs & load_metadata, ContextMu
|
||||
|
||||
LoadTaskPtrs loadMetadataSystem(ContextMutablePtr context, bool async_load_system_database)
|
||||
{
|
||||
loadSystemDatabaseImpl(context, DatabaseCatalog::SYSTEM_DATABASE, "Atomic");
|
||||
loadSystemDatabaseImpl(context, DatabaseCatalog::SYSTEM_DATABASE,
|
||||
context->getApplicationType() == Context::ApplicationType::SERVER ? "Atomic" : "Memory");
|
||||
loadSystemDatabaseImpl(context, DatabaseCatalog::INFORMATION_SCHEMA, "Memory");
|
||||
loadSystemDatabaseImpl(context, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE, "Memory");
|
||||
|
||||
|
@ -0,0 +1,35 @@
|
||||
1
|
||||
0 Hello0
|
||||
1 Hello1
|
||||
2 Hello2
|
||||
3 Hello3
|
||||
4 Hello4
|
||||
5 Hello5
|
||||
6 Hello6
|
||||
7 Hello7
|
||||
8 Hello8
|
||||
9 Hello9
|
||||
default Overlay
|
||||
World0
|
||||
World1
|
||||
World2
|
||||
World3
|
||||
World4
|
||||
World5
|
||||
World6
|
||||
World7
|
||||
World8
|
||||
World9
|
||||
|
||||
World0
|
||||
World1
|
||||
World2
|
||||
World3
|
||||
World4
|
||||
World5
|
||||
World6
|
||||
World7
|
||||
World8
|
||||
World9
|
||||
Hello
|
||||
World
|
37
tests/queries/0_stateless/03271_clickhouse_local_persistency.sh
Executable file
37
tests/queries/0_stateless/03271_clickhouse_local_persistency.sh
Executable file
@ -0,0 +1,37 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
cd "${CLICKHOUSE_TMP}" || exit
|
||||
rm -rf "clickhouse.local"
|
||||
rm -f test
|
||||
|
||||
# You can specify the path explicitly.
|
||||
$CLICKHOUSE_LOCAL --path "clickhouse.local" --query "SELECT 1"
|
||||
|
||||
# You can create tables.
|
||||
$CLICKHOUSE_LOCAL --path "clickhouse.local" --query "CREATE TABLE test (x UInt64, s String) ENGINE = MergeTree ORDER BY x"
|
||||
|
||||
# The data is persisted between restarts
|
||||
$CLICKHOUSE_LOCAL --path "clickhouse.local" --query "INSERT INTO test SELECT number, 'Hello' || number FROM numbers(10)"
|
||||
$CLICKHOUSE_LOCAL --path "clickhouse.local" --query "SELECT * FROM test ORDER BY x"
|
||||
|
||||
# The default database is an Overlay on top of Atomic, which lets you exchange tables.
|
||||
$CLICKHOUSE_LOCAL --path "clickhouse.local" --query "SELECT name, engine FROM system.databases WHERE name = 'default'"
|
||||
$CLICKHOUSE_LOCAL --path "clickhouse.local" --query "CREATE OR REPLACE TABLE test (s String) ENGINE = MergeTree ORDER BY ()"
|
||||
$CLICKHOUSE_LOCAL --path "clickhouse.local" --query "SELECT * FROM test"
|
||||
$CLICKHOUSE_LOCAL --path "clickhouse.local" --query "INSERT INTO test SELECT 'World' || number FROM numbers(10)"
|
||||
$CLICKHOUSE_LOCAL --path "clickhouse.local" --query "SELECT * FROM test"
|
||||
|
||||
# It is an overlay database. If you don't have a table with the same name, it will look for a file with that name.
|
||||
# Files are searched relative to the current working directory.
|
||||
echo '"Hello"
|
||||
"World"' > "test"
|
||||
|
||||
echo
|
||||
$CLICKHOUSE_LOCAL --path "clickhouse.local" --query "SELECT * FROM test; DROP TABLE test; SELECT * FROM test;"
|
||||
|
||||
rm -rf "clickhouse.local"
|
||||
rm test
|
Loading…
Reference in New Issue
Block a user