Merge pull request #68024 from ClickHouse/revert-68023-revert-65860-atomic-database-in-clickhouse-local

Revert "Revert "Use `Atomic` database by default in `clickhouse-local`""
This commit is contained in:
Alexey Milovidov 2024-11-11 12:58:33 +00:00 committed by GitHub
commit bcf05c1227
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
10 changed files with 97 additions and 12 deletions

View File

@ -14,6 +14,7 @@
#include <Databases/registerDatabases.h> #include <Databases/registerDatabases.h>
#include <Databases/DatabaseFilesystem.h> #include <Databases/DatabaseFilesystem.h>
#include <Databases/DatabaseMemory.h> #include <Databases/DatabaseMemory.h>
#include <Databases/DatabaseAtomic.h>
#include <Databases/DatabasesOverlay.h> #include <Databases/DatabasesOverlay.h>
#include <Storages/System/attachSystemTables.h> #include <Storages/System/attachSystemTables.h>
#include <Storages/System/attachInformationSchemaTables.h> #include <Storages/System/attachInformationSchemaTables.h>
@ -257,12 +258,12 @@ static DatabasePtr createMemoryDatabaseIfNotExists(ContextPtr context, const Str
return system_database; return system_database;
} }
static DatabasePtr createClickHouseLocalDatabaseOverlay(const String & name_, ContextPtr context_) static DatabasePtr createClickHouseLocalDatabaseOverlay(const String & name_, ContextPtr context)
{ {
auto databaseCombiner = std::make_shared<DatabasesOverlay>(name_, context_); auto overlay = std::make_shared<DatabasesOverlay>(name_, context);
databaseCombiner->registerNextDatabase(std::make_shared<DatabaseFilesystem>(name_, "", context_)); overlay->registerNextDatabase(std::make_shared<DatabaseAtomic>(name_, fs::weakly_canonical(context->getPath()), UUIDHelpers::generateV4(), context));
databaseCombiner->registerNextDatabase(std::make_shared<DatabaseMemory>(name_, context_)); overlay->registerNextDatabase(std::make_shared<DatabaseFilesystem>(name_, "", context));
return databaseCombiner; return overlay;
} }
/// If path is specified and not empty, will try to setup server environment and load existing metadata /// If path is specified and not empty, will try to setup server environment and load existing metadata
@ -811,7 +812,12 @@ void LocalServer::processConfig()
DatabaseCatalog::instance().initializeAndLoadTemporaryDatabase(); DatabaseCatalog::instance().initializeAndLoadTemporaryDatabase();
std::string default_database = server_settings[ServerSetting::default_database]; std::string default_database = server_settings[ServerSetting::default_database];
DatabaseCatalog::instance().attachDatabase(default_database, createClickHouseLocalDatabaseOverlay(default_database, global_context)); {
DatabasePtr database = createClickHouseLocalDatabaseOverlay(default_database, global_context);
if (UUID uuid = database->getUUID(); uuid != UUIDHelpers::Nil)
DatabaseCatalog::instance().addUUIDMapping(uuid);
DatabaseCatalog::instance().attachDatabase(default_database, database);
}
global_context->setCurrentDatabase(default_database); global_context->setCurrentDatabase(default_database);
if (getClientConfiguration().has("path")) if (getClientConfiguration().has("path"))

View File

@ -6,7 +6,6 @@
#include <Databases/DatabaseReplicated.h> #include <Databases/DatabaseReplicated.h>
#include <IO/ReadBufferFromFile.h> #include <IO/ReadBufferFromFile.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Interpreters/DDLTask.h> #include <Interpreters/DDLTask.h>
#include <Interpreters/DatabaseCatalog.h> #include <Interpreters/DatabaseCatalog.h>
@ -19,6 +18,7 @@
#include <Common/filesystemHelpers.h> #include <Common/filesystemHelpers.h>
#include <Core/Settings.h> #include <Core/Settings.h>
namespace fs = std::filesystem; namespace fs = std::filesystem;
namespace DB namespace DB
@ -60,9 +60,6 @@ DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, c
, db_uuid(uuid) , db_uuid(uuid)
{ {
assert(db_uuid != UUIDHelpers::Nil); assert(db_uuid != UUIDHelpers::Nil);
fs::create_directories(fs::path(getContext()->getPath()) / "metadata");
fs::create_directories(path_to_table_symlinks);
tryCreateMetadataSymlink();
} }
DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, ContextPtr context_) DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, ContextPtr context_)
@ -70,6 +67,20 @@ DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, C
{ {
} }
void DatabaseAtomic::createDirectories()
{
std::lock_guard lock(mutex);
createDirectoriesUnlocked();
}
void DatabaseAtomic::createDirectoriesUnlocked()
{
DatabaseOnDisk::createDirectoriesUnlocked();
fs::create_directories(fs::path(getContext()->getPath()) / "metadata");
fs::create_directories(path_to_table_symlinks);
tryCreateMetadataSymlink();
}
String DatabaseAtomic::getTableDataPath(const String & table_name) const String DatabaseAtomic::getTableDataPath(const String & table_name) const
{ {
std::lock_guard lock(mutex); std::lock_guard lock(mutex);
@ -108,6 +119,7 @@ void DatabaseAtomic::attachTable(ContextPtr /* context_ */, const String & name,
assert(relative_table_path != data_path && !relative_table_path.empty()); assert(relative_table_path != data_path && !relative_table_path.empty());
DetachedTables not_in_use; DetachedTables not_in_use;
std::lock_guard lock(mutex); std::lock_guard lock(mutex);
createDirectoriesUnlocked();
not_in_use = cleanupDetachedTables(); not_in_use = cleanupDetachedTables();
auto table_id = table->getStorageID(); auto table_id = table->getStorageID();
assertDetachedTableNotInUse(table_id.uuid); assertDetachedTableNotInUse(table_id.uuid);
@ -208,11 +220,15 @@ void DatabaseAtomic::renameTable(ContextPtr local_context, const String & table_
if (exchange && !supportsAtomicRename(&message)) if (exchange && !supportsAtomicRename(&message))
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "RENAME EXCHANGE is not supported because exchanging files is not supported by the OS ({})", message); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "RENAME EXCHANGE is not supported because exchanging files is not supported by the OS ({})", message);
createDirectories();
waitDatabaseStarted(); waitDatabaseStarted();
auto & other_db = dynamic_cast<DatabaseAtomic &>(to_database); auto & other_db = dynamic_cast<DatabaseAtomic &>(to_database);
bool inside_database = this == &other_db; bool inside_database = this == &other_db;
if (!inside_database)
other_db.createDirectories();
String old_metadata_path = getObjectMetadataPath(table_name); String old_metadata_path = getObjectMetadataPath(table_name);
String new_metadata_path = to_database.getObjectMetadataPath(to_table_name); String new_metadata_path = to_database.getObjectMetadataPath(to_table_name);
@ -333,6 +349,7 @@ void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const Stora
const String & table_metadata_tmp_path, const String & table_metadata_path, const String & table_metadata_tmp_path, const String & table_metadata_path,
ContextPtr query_context) ContextPtr query_context)
{ {
createDirectories();
DetachedTables not_in_use; DetachedTables not_in_use;
auto table_data_path = getTableDataPath(query); auto table_data_path = getTableDataPath(query);
try try
@ -469,6 +486,9 @@ void DatabaseAtomic::beforeLoadingMetadata(ContextMutablePtr /*context*/, Loadin
if (mode < LoadingStrictnessLevel::FORCE_RESTORE) if (mode < LoadingStrictnessLevel::FORCE_RESTORE)
return; return;
if (!fs::exists(path_to_table_symlinks))
return;
/// Recreate symlinks to table data dirs in case of force restore, because some of them may be broken /// Recreate symlinks to table data dirs in case of force restore, because some of them may be broken
for (const auto & table_path : fs::directory_iterator(path_to_table_symlinks)) for (const auto & table_path : fs::directory_iterator(path_to_table_symlinks))
{ {
@ -611,6 +631,7 @@ void DatabaseAtomic::renameDatabase(ContextPtr query_context, const String & new
{ {
/// CREATE, ATTACH, DROP, DETACH and RENAME DATABASE must hold DDLGuard /// CREATE, ATTACH, DROP, DETACH and RENAME DATABASE must hold DDLGuard
createDirectories();
waitDatabaseStarted(); waitDatabaseStarted();
bool check_ref_deps = query_context->getSettingsRef()[Setting::check_referential_table_dependencies]; bool check_ref_deps = query_context->getSettingsRef()[Setting::check_referential_table_dependencies];
@ -702,4 +723,5 @@ void registerDatabaseAtomic(DatabaseFactory & factory)
}; };
factory.registerDatabase("Atomic", create_fn); factory.registerDatabase("Atomic", create_fn);
} }
} }

View File

@ -76,6 +76,9 @@ protected:
using DetachedTables = std::unordered_map<UUID, StoragePtr>; using DetachedTables = std::unordered_map<UUID, StoragePtr>;
[[nodiscard]] DetachedTables cleanupDetachedTables() TSA_REQUIRES(mutex); [[nodiscard]] DetachedTables cleanupDetachedTables() TSA_REQUIRES(mutex);
void createDirectories();
void createDirectoriesUnlocked() TSA_REQUIRES(mutex);
void tryCreateMetadataSymlink(); void tryCreateMetadataSymlink();
virtual bool allowMoveTableToOtherDatabaseEngine(IDatabase & /*to_database*/) const { return false; } virtual bool allowMoveTableToOtherDatabaseEngine(IDatabase & /*to_database*/) const { return false; }

View File

@ -47,6 +47,7 @@ DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_,
: DatabaseOnDisk(name_, metadata_path_, std::filesystem::path("data") / escapeForFileName(name_) / "", "DatabaseLazy (" + name_ + ")", context_) : DatabaseOnDisk(name_, metadata_path_, std::filesystem::path("data") / escapeForFileName(name_) / "", "DatabaseLazy (" + name_ + ")", context_)
, expiration_time(expiration_time_) , expiration_time(expiration_time_)
{ {
createDirectories();
} }

View File

@ -180,7 +180,18 @@ DatabaseOnDisk::DatabaseOnDisk(
, metadata_path(metadata_path_) , metadata_path(metadata_path_)
, data_path(data_path_) , data_path(data_path_)
{ {
fs::create_directories(local_context->getPath() + data_path); }
void DatabaseOnDisk::createDirectories()
{
std::lock_guard lock(mutex);
createDirectoriesUnlocked();
}
void DatabaseOnDisk::createDirectoriesUnlocked()
{
fs::create_directories(std::filesystem::path(getContext()->getPath()) / data_path);
fs::create_directories(metadata_path); fs::create_directories(metadata_path);
} }
@ -198,6 +209,8 @@ void DatabaseOnDisk::createTable(
const StoragePtr & table, const StoragePtr & table,
const ASTPtr & query) const ASTPtr & query)
{ {
createDirectories();
const auto & settings = local_context->getSettingsRef(); const auto & settings = local_context->getSettingsRef();
const auto & create = query->as<ASTCreateQuery &>(); const auto & create = query->as<ASTCreateQuery &>();
assert(table_name == create.getTable()); assert(table_name == create.getTable());
@ -265,7 +278,6 @@ void DatabaseOnDisk::createTable(
} }
commitCreateTable(create, table, table_metadata_tmp_path, table_metadata_path, local_context); commitCreateTable(create, table, table_metadata_tmp_path, table_metadata_path, local_context);
removeDetachedPermanentlyFlag(local_context, table_name, table_metadata_path, false); removeDetachedPermanentlyFlag(local_context, table_name, table_metadata_path, false);
} }
@ -293,6 +305,8 @@ void DatabaseOnDisk::commitCreateTable(const ASTCreateQuery & query, const Stora
{ {
try try
{ {
createDirectories();
/// Add a table to the map of known tables. /// Add a table to the map of known tables.
attachTable(query_context, query.getTable(), table, getTableDataPath(query)); attachTable(query_context, query.getTable(), table, getTableDataPath(query));
@ -426,6 +440,7 @@ void DatabaseOnDisk::renameTable(
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Moving tables between databases of different engines is not supported"); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Moving tables between databases of different engines is not supported");
} }
createDirectories();
waitDatabaseStarted(); waitDatabaseStarted();
auto table_data_relative_path = getTableDataPath(table_name); auto table_data_relative_path = getTableDataPath(table_name);
@ -621,6 +636,9 @@ time_t DatabaseOnDisk::getObjectMetadataModificationTime(const String & object_n
void DatabaseOnDisk::iterateMetadataFiles(const IteratingFunction & process_metadata_file) const void DatabaseOnDisk::iterateMetadataFiles(const IteratingFunction & process_metadata_file) const
{ {
if (!fs::exists(metadata_path))
return;
auto process_tmp_drop_metadata_file = [&](const String & file_name) auto process_tmp_drop_metadata_file = [&](const String & file_name)
{ {
assert(getUUID() == UUIDHelpers::Nil); assert(getUUID() == UUIDHelpers::Nil);

View File

@ -99,6 +99,9 @@ protected:
virtual void removeDetachedPermanentlyFlag(ContextPtr context, const String & table_name, const String & table_metadata_path, bool attach); virtual void removeDetachedPermanentlyFlag(ContextPtr context, const String & table_name, const String & table_metadata_path, bool attach);
virtual void setDetachedTableNotInUseForce(const UUID & /*uuid*/) {} virtual void setDetachedTableNotInUseForce(const UUID & /*uuid*/) {}
void createDirectories();
void createDirectoriesUnlocked() TSA_REQUIRES(mutex);
const String metadata_path; const String metadata_path;
const String data_path; const String data_path;
}; };

View File

@ -416,6 +416,7 @@ public:
std::lock_guard lock{mutex}; std::lock_guard lock{mutex};
return database_name; return database_name;
} }
/// Get UUID of database. /// Get UUID of database.
virtual UUID getUUID() const { return UUIDHelpers::Nil; } virtual UUID getUUID() const { return UUIDHelpers::Nil; }

View File

@ -62,6 +62,7 @@ DatabaseMaterializedMySQL::DatabaseMaterializedMySQL(
, settings(std::move(settings_)) , settings(std::move(settings_))
, materialize_thread(context_, database_name_, mysql_database_name_, std::move(pool_), std::move(client_), binlog_client_, settings.get()) , materialize_thread(context_, database_name_, mysql_database_name_, std::move(pool_), std::move(client_), binlog_client_, settings.get())
{ {
createDirectories();
} }
DatabaseMaterializedMySQL::~DatabaseMaterializedMySQL() = default; DatabaseMaterializedMySQL::~DatabaseMaterializedMySQL() = default;

View File

@ -0,0 +1,6 @@
123
Hello
['Hello','world']
Hello
Hello
['Hello','world']

View File

@ -0,0 +1,24 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
${CLICKHOUSE_LOCAL} -n "
CREATE TABLE test (x UInt8) ORDER BY x;
INSERT INTO test VALUES (123);
SELECT * FROM test;
CREATE OR REPLACE TABLE test (s String) ORDER BY s;
INSERT INTO test VALUES ('Hello');
SELECT * FROM test;
RENAME TABLE test TO test2;
CREATE OR REPLACE TABLE test (s Array(String)) ORDER BY s;
INSERT INTO test VALUES (['Hello', 'world']);
SELECT * FROM test;
SELECT * FROM test2;
EXCHANGE TABLES test AND test2;
SELECT * FROM test;
SELECT * FROM test2;
DROP TABLE test;
DROP TABLE test2;
"