From 1902f5306d4ad38f24d87279d6c3aa770dc115d4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Nov 2024 04:55:21 +0100 Subject: [PATCH 1/8] Persistent databases in clickhouse-local --- programs/local/LocalServer.cpp | 78 ++++++--- src/Databases/DatabaseAtomic.cpp | 7 +- src/Databases/DatabasesOverlay.cpp | 254 ++++++++++++++++++++++++++++- src/Databases/DatabasesOverlay.h | 34 ++++ src/Interpreters/loadMetadata.cpp | 29 +++- 5 files changed, 362 insertions(+), 40 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index e6f8ecef097..b6952499ed6 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -52,6 +52,7 @@ #include #include #include +#include #include "config.h" @@ -261,7 +262,19 @@ static DatabasePtr createMemoryDatabaseIfNotExists(ContextPtr context, const Str static DatabasePtr createClickHouseLocalDatabaseOverlay(const String & name_, ContextPtr context) { auto overlay = std::make_shared(name_, context); - overlay->registerNextDatabase(std::make_shared(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(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(name_, default_database_metadata_path, default_database_uuid, context)); overlay->registerNextDatabase(std::make_shared(name_, "", context)); return overlay; } @@ -273,7 +286,7 @@ void LocalServer::tryInitPath() if (getClientConfiguration().has("path")) { - // User-supplied path. + /// User-supplied path. path = getClientConfiguration().getString("path"); Poco::trimInPlace(path); @@ -285,17 +298,17 @@ void LocalServer::tryInitPath() " correct the --path."); } } - else + else if (getClientConfiguration().has("tmp")) { - // 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(); } @@ -323,7 +336,15 @@ 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); + } + else + { + /// No explicit path specified. Use a subdirectory in the current directory (it will be created lazily only if needed). + /// The subdirectory is named `clickhouse.local`. This name is to not collide with the possible names + /// of the binary file, `clickhouse` or `clickhouse-local`. + path = "clickhouse.local"; + getClientConfiguration().setString("path", path); } global_context->setPath(fs::path(path) / ""); @@ -822,30 +843,37 @@ 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(); + if (fs::exists(fs::path(path) / "metadata")) { - DatabaseCatalog::instance().createBackgroundTasks(); - waitLoad(loadMetadata(global_context)); - DatabaseCatalog::instance().startupBackgroundTasks(); + /// Lock path directory before read + /// Note: this is slightly unsafe. The first instance of clickhouse-local will not be protected. + status.emplace(fs::path(path) / "status", StatusFile::write_full_info); + + 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")) { diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index bd077ccd7b5..5237da53a6b 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -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 { diff --git a/src/Databases/DatabasesOverlay.cpp b/src/Databases/DatabasesOverlay.cpp index 495733e15fd..0ac009f8015 100644 --- a/src/Databases/DatabasesOverlay.cpp +++ b/src/Databases/DatabasesOverlay.cpp @@ -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,248 @@ DatabaseTablesIteratorPtr DatabasesOverlay::getTablesIterator(ContextPtr context return std::make_unique(std::move(tables), getDatabaseName()); } +bool DatabasesOverlay::canContainMergeTreeTables() const +{ + for (auto & db : databases) + if (db->canContainMergeTreeTables()) + return true; + return false; +} + +bool DatabasesOverlay::canContainDistributedTables() const +{ + for (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 (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 (auto & db : databases) + { + if (db->isReadOnly()) + continue; + + try + { + return db->waitTableStarted(name); + } + 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 (auto & db : databases) + { + if (db->isReadOnly()) + continue; + + try + { + return db->waitDatabaseStarted(); + } + 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 + { + return db->stopLoading(); + } + 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 (auto & db : databases) + { + if (db->isReadOnly()) + continue; + db->checkMetadataFilenameAvailability(table_name); + return; + } +} + + } diff --git a/src/Databases/DatabasesOverlay.h b/src/Databases/DatabasesOverlay.h index 40c653e5cb5..2883844106c 100644 --- a/src/Databases/DatabasesOverlay.h +++ b/src/Databases/DatabasesOverlay.h @@ -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 databases; LoggerPtr log; diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 12de96ef03a..f34e8187308 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -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 databases; + + /// Some databases don't have an .sql metadata file. + std::map 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"); From 211a42192707aeaff21c1cc23f1a91be2c714e62 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Nov 2024 14:03:22 +0100 Subject: [PATCH 2/8] Progress on implementation --- programs/local/LocalServer.cpp | 19 ++++++++----------- 1 file changed, 8 insertions(+), 11 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index b6952499ed6..d6febb755a6 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -287,6 +287,10 @@ void LocalServer::tryInitPath() if (getClientConfiguration().has("path")) { /// User-supplied path. + /// By default it is a subdirectory in the current directory (it will be created lazily only if needed). + /// The subdirectory is named `clickhouse.local`. This name is to not collide with the possible names + /// of the binary file, `clickhouse` or `clickhouse-local`. + path = getClientConfiguration().getString("path"); Poco::trimInPlace(path); @@ -338,14 +342,6 @@ void LocalServer::tryInitPath() path = default_path.string(); LOG_DEBUG(log, "Working directory will be created as needed: {}", path); } - else - { - /// No explicit path specified. Use a subdirectory in the current directory (it will be created lazily only if needed). - /// The subdirectory is named `clickhouse.local`. This name is to not collide with the possible names - /// of the binary file, `clickhouse` or `clickhouse-local`. - path = "clickhouse.local"; - getClientConfiguration().setString("path", path); - } global_context->setPath(fs::path(path) / ""); @@ -943,7 +939,8 @@ void LocalServer::addOptions(OptionsDescription & options_description) ("logger.level", po::value(), "Log level") ("no-system-tables", "do not attach system tables (better startup time)") - ("path", po::value(), "Storage path") + ("path", po::value()->default_value("clickhouse.local"), "storage path") + ("tmp", "use a temporary directory for tables and delete it on exit") ("only-system-tables", "attach only system tables from specified path") ("top_level_domains_path", po::value(), "Path to lists with custom TLDs") ; @@ -981,12 +978,12 @@ void LocalServer::processOptions(const OptionsDescription &, const CommandLineOp getClientConfiguration().setString("table-file", options["file"].as()); if (options.count("structure")) getClientConfiguration().setString("table-structure", options["structure"].as()); + if (options.count("tmp")) + getClientConfiguration().setBool("tmp", true); if (options.count("no-system-tables")) 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()); if (options.count("input-format")) getClientConfiguration().setString("table-data-format", options["input-format"].as()); From 6738a0457a41d27c7c32d62dc48ff29f8c3aea1e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 13 Nov 2024 00:57:18 +0100 Subject: [PATCH 3/8] Add a test --- programs/local/LocalServer.cpp | 19 +++------- ...271_clickhouse_local_persistency.reference | 35 ++++++++++++++++++ .../03271_clickhouse_local_persistency.sh | 37 +++++++++++++++++++ 3 files changed, 78 insertions(+), 13 deletions(-) create mode 100644 tests/queries/0_stateless/03271_clickhouse_local_persistency.reference create mode 100755 tests/queries/0_stateless/03271_clickhouse_local_persistency.sh diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index d6febb755a6..8ef3b0bd7e5 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -287,10 +287,6 @@ void LocalServer::tryInitPath() if (getClientConfiguration().has("path")) { /// User-supplied path. - /// By default it is a subdirectory in the current directory (it will be created lazily only if needed). - /// The subdirectory is named `clickhouse.local`. This name is to not collide with the possible names - /// of the binary file, `clickhouse` or `clickhouse-local`. - path = getClientConfiguration().getString("path"); Poco::trimInPlace(path); @@ -302,7 +298,7 @@ void LocalServer::tryInitPath() " correct the --path."); } } - else if (getClientConfiguration().has("tmp")) + else { /// 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) @@ -844,12 +840,12 @@ void LocalServer::processConfig() attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE)); String path = global_context->getPath(); + + /// Lock path directory before read + status.emplace(fs::path(path) / "status", StatusFile::write_full_info); + if (fs::exists(fs::path(path) / "metadata")) { - /// Lock path directory before read - /// Note: this is slightly unsafe. The first instance of clickhouse-local will not be protected. - status.emplace(fs::path(path) / "status", StatusFile::write_full_info); - LOG_DEBUG(log, "Loading metadata from {}", path); if (fs::exists(std::filesystem::path(path) / "metadata" / "system.sql")) @@ -939,8 +935,7 @@ void LocalServer::addOptions(OptionsDescription & options_description) ("logger.level", po::value(), "Log level") ("no-system-tables", "do not attach system tables (better startup time)") - ("path", po::value()->default_value("clickhouse.local"), "storage path") - ("tmp", "use a temporary directory for tables and delete it on exit") + ("path", po::value(), "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(), "Path to lists with custom TLDs") ; @@ -978,8 +973,6 @@ void LocalServer::processOptions(const OptionsDescription &, const CommandLineOp getClientConfiguration().setString("table-file", options["file"].as()); if (options.count("structure")) getClientConfiguration().setString("table-structure", options["structure"].as()); - if (options.count("tmp")) - getClientConfiguration().setBool("tmp", true); if (options.count("no-system-tables")) getClientConfiguration().setBool("no-system-tables", true); if (options.count("only-system-tables")) diff --git a/tests/queries/0_stateless/03271_clickhouse_local_persistency.reference b/tests/queries/0_stateless/03271_clickhouse_local_persistency.reference new file mode 100644 index 00000000000..7abc1ea60b4 --- /dev/null +++ b/tests/queries/0_stateless/03271_clickhouse_local_persistency.reference @@ -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 diff --git a/tests/queries/0_stateless/03271_clickhouse_local_persistency.sh b/tests/queries/0_stateless/03271_clickhouse_local_persistency.sh new file mode 100755 index 00000000000..0dcc0e8e89c --- /dev/null +++ b/tests/queries/0_stateless/03271_clickhouse_local_persistency.sh @@ -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}" +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 From 5193f4e04180053b60ae718e62ea2c922be2b654 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 13 Nov 2024 01:37:06 +0100 Subject: [PATCH 4/8] Update 03271_clickhouse_local_persistency.sh --- tests/queries/0_stateless/03271_clickhouse_local_persistency.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03271_clickhouse_local_persistency.sh b/tests/queries/0_stateless/03271_clickhouse_local_persistency.sh index 0dcc0e8e89c..71be0a13af2 100755 --- a/tests/queries/0_stateless/03271_clickhouse_local_persistency.sh +++ b/tests/queries/0_stateless/03271_clickhouse_local_persistency.sh @@ -4,7 +4,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -cd "${CLICKHOUSE_TMP}" +cd "${CLICKHOUSE_TMP}" || exit rm -rf "clickhouse.local" rm -f test From 291848f81da58531eabae9223cd51ca6d9905428 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 13 Nov 2024 02:19:35 +0100 Subject: [PATCH 5/8] Fix error --- programs/local/LocalServer.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 8ef3b0bd7e5..2075d0aab78 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -842,6 +842,7 @@ void LocalServer::processConfig() 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")) From 17ac4b84bc4d2bc2417ca88f81c654f611e55dc2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 13 Nov 2024 04:27:01 +0100 Subject: [PATCH 6/8] Fix build --- src/Databases/DatabasesOverlay.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Databases/DatabasesOverlay.cpp b/src/Databases/DatabasesOverlay.cpp index 0ac009f8015..fc8dbb95d90 100644 --- a/src/Databases/DatabasesOverlay.cpp +++ b/src/Databases/DatabasesOverlay.cpp @@ -315,7 +315,7 @@ DatabaseTablesIteratorPtr DatabasesOverlay::getTablesIterator(ContextPtr context bool DatabasesOverlay::canContainMergeTreeTables() const { - for (auto & db : databases) + for (const auto & db : databases) if (db->canContainMergeTreeTables()) return true; return false; @@ -323,7 +323,7 @@ bool DatabasesOverlay::canContainMergeTreeTables() const bool DatabasesOverlay::canContainDistributedTables() const { - for (auto & db : databases) + for (const auto & db : databases) if (db->canContainDistributedTables()) return true; return false; @@ -338,7 +338,7 @@ void DatabasesOverlay::loadStoredObjects(ContextMutablePtr local_context, Loadin bool DatabasesOverlay::supportsLoadingInTopologicalOrder() const { - for (auto & db : databases) + for (const auto & db : databases) if (db->supportsLoadingInTopologicalOrder()) return true; return false; @@ -477,7 +477,7 @@ LoadTaskPtr DatabasesOverlay::startupDatabaseAsync( void DatabasesOverlay::waitTableStarted(const String & name) const { - for (auto & db : databases) + for (const auto & db : databases) { if (db->isReadOnly()) continue; @@ -501,7 +501,7 @@ void DatabasesOverlay::waitTableStarted(const String & name) const void DatabasesOverlay::waitDatabaseStarted() const { - for (auto & db : databases) + for (const auto & db : databases) { if (db->isReadOnly()) continue; @@ -547,7 +547,7 @@ void DatabasesOverlay::stopLoading() void DatabasesOverlay::checkMetadataFilenameAvailability(const String & table_name) const { - for (auto & db : databases) + for (const auto & db : databases) { if (db->isReadOnly()) continue; From adb75a2d2929b914d2ec89cbabd7bd95d1991af9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 13 Nov 2024 18:10:44 +0100 Subject: [PATCH 7/8] Fix tidy --- src/Databases/DatabasesOverlay.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Databases/DatabasesOverlay.cpp b/src/Databases/DatabasesOverlay.cpp index fc8dbb95d90..4d190b18b8f 100644 --- a/src/Databases/DatabasesOverlay.cpp +++ b/src/Databases/DatabasesOverlay.cpp @@ -484,7 +484,8 @@ void DatabasesOverlay::waitTableStarted(const String & name) const try { - return db->waitTableStarted(name); + db->waitTableStarted(name); + return } catch (...) { @@ -508,7 +509,8 @@ void DatabasesOverlay::waitDatabaseStarted() const try { - return db->waitDatabaseStarted(); + db->waitDatabaseStarted(); + return; } catch (...) { @@ -531,7 +533,8 @@ void DatabasesOverlay::stopLoading() try { - return db->stopLoading(); + db->stopLoading(); + return; } catch (...) { From 731bec86e9fec5656e22f099a1a88ea06ecf0b51 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 13 Nov 2024 22:03:59 +0100 Subject: [PATCH 8/8] Fix build --- src/Databases/DatabasesOverlay.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabasesOverlay.cpp b/src/Databases/DatabasesOverlay.cpp index 4d190b18b8f..8351d48a1f8 100644 --- a/src/Databases/DatabasesOverlay.cpp +++ b/src/Databases/DatabasesOverlay.cpp @@ -485,7 +485,7 @@ void DatabasesOverlay::waitTableStarted(const String & name) const try { db->waitTableStarted(name); - return + return; } catch (...) {