mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
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:
commit
bcf05c1227
@ -14,6 +14,7 @@
|
||||
#include <Databases/registerDatabases.h>
|
||||
#include <Databases/DatabaseFilesystem.h>
|
||||
#include <Databases/DatabaseMemory.h>
|
||||
#include <Databases/DatabaseAtomic.h>
|
||||
#include <Databases/DatabasesOverlay.h>
|
||||
#include <Storages/System/attachSystemTables.h>
|
||||
#include <Storages/System/attachInformationSchemaTables.h>
|
||||
@ -257,12 +258,12 @@ static DatabasePtr createMemoryDatabaseIfNotExists(ContextPtr context, const Str
|
||||
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_);
|
||||
databaseCombiner->registerNextDatabase(std::make_shared<DatabaseFilesystem>(name_, "", context_));
|
||||
databaseCombiner->registerNextDatabase(std::make_shared<DatabaseMemory>(name_, context_));
|
||||
return databaseCombiner;
|
||||
auto overlay = std::make_shared<DatabasesOverlay>(name_, context);
|
||||
overlay->registerNextDatabase(std::make_shared<DatabaseAtomic>(name_, fs::weakly_canonical(context->getPath()), UUIDHelpers::generateV4(), context));
|
||||
overlay->registerNextDatabase(std::make_shared<DatabaseFilesystem>(name_, "", context));
|
||||
return overlay;
|
||||
}
|
||||
|
||||
/// 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();
|
||||
|
||||
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);
|
||||
|
||||
if (getClientConfiguration().has("path"))
|
||||
|
@ -6,7 +6,6 @@
|
||||
#include <Databases/DatabaseReplicated.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DDLTask.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
@ -19,6 +18,7 @@
|
||||
#include <Common/filesystemHelpers.h>
|
||||
#include <Core/Settings.h>
|
||||
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
namespace DB
|
||||
@ -60,9 +60,6 @@ DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, c
|
||||
, db_uuid(uuid)
|
||||
{
|
||||
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_)
|
||||
@ -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
|
||||
{
|
||||
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());
|
||||
DetachedTables not_in_use;
|
||||
std::lock_guard lock(mutex);
|
||||
createDirectoriesUnlocked();
|
||||
not_in_use = cleanupDetachedTables();
|
||||
auto table_id = table->getStorageID();
|
||||
assertDetachedTableNotInUse(table_id.uuid);
|
||||
@ -208,11 +220,15 @@ void DatabaseAtomic::renameTable(ContextPtr local_context, const String & table_
|
||||
if (exchange && !supportsAtomicRename(&message))
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "RENAME EXCHANGE is not supported because exchanging files is not supported by the OS ({})", message);
|
||||
|
||||
createDirectories();
|
||||
waitDatabaseStarted();
|
||||
|
||||
auto & other_db = dynamic_cast<DatabaseAtomic &>(to_database);
|
||||
bool inside_database = this == &other_db;
|
||||
|
||||
if (!inside_database)
|
||||
other_db.createDirectories();
|
||||
|
||||
String old_metadata_path = getObjectMetadataPath(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,
|
||||
ContextPtr query_context)
|
||||
{
|
||||
createDirectories();
|
||||
DetachedTables not_in_use;
|
||||
auto table_data_path = getTableDataPath(query);
|
||||
try
|
||||
@ -469,6 +486,9 @@ void DatabaseAtomic::beforeLoadingMetadata(ContextMutablePtr /*context*/, Loadin
|
||||
if (mode < LoadingStrictnessLevel::FORCE_RESTORE)
|
||||
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
|
||||
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
|
||||
|
||||
createDirectories();
|
||||
waitDatabaseStarted();
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -76,6 +76,9 @@ protected:
|
||||
using DetachedTables = std::unordered_map<UUID, StoragePtr>;
|
||||
[[nodiscard]] DetachedTables cleanupDetachedTables() TSA_REQUIRES(mutex);
|
||||
|
||||
void createDirectories();
|
||||
void createDirectoriesUnlocked() TSA_REQUIRES(mutex);
|
||||
|
||||
void tryCreateMetadataSymlink();
|
||||
|
||||
virtual bool allowMoveTableToOtherDatabaseEngine(IDatabase & /*to_database*/) const { return false; }
|
||||
|
@ -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_)
|
||||
, expiration_time(expiration_time_)
|
||||
{
|
||||
createDirectories();
|
||||
}
|
||||
|
||||
|
||||
|
@ -180,7 +180,18 @@ DatabaseOnDisk::DatabaseOnDisk(
|
||||
, metadata_path(metadata_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);
|
||||
}
|
||||
|
||||
@ -198,6 +209,8 @@ void DatabaseOnDisk::createTable(
|
||||
const StoragePtr & table,
|
||||
const ASTPtr & query)
|
||||
{
|
||||
createDirectories();
|
||||
|
||||
const auto & settings = local_context->getSettingsRef();
|
||||
const auto & create = query->as<ASTCreateQuery &>();
|
||||
assert(table_name == create.getTable());
|
||||
@ -265,7 +278,6 @@ void DatabaseOnDisk::createTable(
|
||||
}
|
||||
|
||||
commitCreateTable(create, table, table_metadata_tmp_path, table_metadata_path, local_context);
|
||||
|
||||
removeDetachedPermanentlyFlag(local_context, table_name, table_metadata_path, false);
|
||||
}
|
||||
|
||||
@ -293,6 +305,8 @@ void DatabaseOnDisk::commitCreateTable(const ASTCreateQuery & query, const Stora
|
||||
{
|
||||
try
|
||||
{
|
||||
createDirectories();
|
||||
|
||||
/// Add a table to the map of known tables.
|
||||
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");
|
||||
}
|
||||
|
||||
createDirectories();
|
||||
waitDatabaseStarted();
|
||||
|
||||
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
|
||||
{
|
||||
if (!fs::exists(metadata_path))
|
||||
return;
|
||||
|
||||
auto process_tmp_drop_metadata_file = [&](const String & file_name)
|
||||
{
|
||||
assert(getUUID() == UUIDHelpers::Nil);
|
||||
|
@ -99,6 +99,9 @@ protected:
|
||||
virtual void removeDetachedPermanentlyFlag(ContextPtr context, const String & table_name, const String & table_metadata_path, bool attach);
|
||||
virtual void setDetachedTableNotInUseForce(const UUID & /*uuid*/) {}
|
||||
|
||||
void createDirectories();
|
||||
void createDirectoriesUnlocked() TSA_REQUIRES(mutex);
|
||||
|
||||
const String metadata_path;
|
||||
const String data_path;
|
||||
};
|
||||
|
@ -416,6 +416,7 @@ public:
|
||||
std::lock_guard lock{mutex};
|
||||
return database_name;
|
||||
}
|
||||
|
||||
/// Get UUID of database.
|
||||
virtual UUID getUUID() const { return UUIDHelpers::Nil; }
|
||||
|
||||
|
@ -62,6 +62,7 @@ DatabaseMaterializedMySQL::DatabaseMaterializedMySQL(
|
||||
, settings(std::move(settings_))
|
||||
, materialize_thread(context_, database_name_, mysql_database_name_, std::move(pool_), std::move(client_), binlog_client_, settings.get())
|
||||
{
|
||||
createDirectories();
|
||||
}
|
||||
|
||||
DatabaseMaterializedMySQL::~DatabaseMaterializedMySQL() = default;
|
||||
|
@ -0,0 +1,6 @@
|
||||
123
|
||||
Hello
|
||||
['Hello','world']
|
||||
Hello
|
||||
Hello
|
||||
['Hello','world']
|
24
tests/queries/0_stateless/03199_atomic_clickhouse_local.sh
Executable file
24
tests/queries/0_stateless/03199_atomic_clickhouse_local.sh
Executable 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;
|
||||
"
|
Loading…
Reference in New Issue
Block a user