ClickHouse/src/Databases/DatabaseAtomic.cpp

540 lines
20 KiB
C++
Raw Normal View History

2019-10-23 13:46:38 +00:00
#include <Databases/DatabaseAtomic.h>
2019-10-30 12:17:52 +00:00
#include <Databases/DatabaseOnDisk.h>
2019-11-11 11:34:03 +00:00
#include <Poco/File.h>
#include <Poco/Path.h>
2019-11-11 11:34:03 +00:00
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
2020-01-27 20:31:39 +00:00
#include <Parsers/formatAST.h>
2020-04-23 18:00:43 +00:00
#include <Common/renameat2.h>
2020-04-10 01:35:37 +00:00
#include <Storages/StorageMaterializedView.h>
#include <Interpreters/Context.h>
2020-07-16 14:25:39 +00:00
#include <Interpreters/ExternalDictionariesLoader.h>
2020-05-17 01:01:26 +00:00
#include <filesystem>
2019-10-23 13:46:38 +00:00
namespace DB
{
2019-11-11 11:34:03 +00:00
namespace ErrorCodes
{
extern const int UNKNOWN_TABLE;
2020-07-17 13:11:44 +00:00
extern const int UNKNOWN_DATABASE;
2019-11-11 11:34:03 +00:00
extern const int TABLE_ALREADY_EXISTS;
2020-03-23 00:12:13 +00:00
extern const int CANNOT_ASSIGN_ALTER;
2020-04-10 23:02:15 +00:00
extern const int DATABASE_NOT_EMPTY;
2020-04-11 15:38:41 +00:00
extern const int NOT_IMPLEMENTED;
2020-07-02 20:39:31 +00:00
extern const int FILE_ALREADY_EXISTS;
2020-07-16 21:41:26 +00:00
extern const int INCORRECT_QUERY;
2019-11-11 11:34:03 +00:00
}
2019-10-23 13:46:38 +00:00
2020-04-01 22:41:29 +00:00
class AtomicDatabaseTablesSnapshotIterator final : public DatabaseTablesSnapshotIterator
{
public:
2020-04-08 01:02:00 +00:00
explicit AtomicDatabaseTablesSnapshotIterator(DatabaseTablesSnapshotIterator && base)
2020-04-01 22:41:29 +00:00
: DatabaseTablesSnapshotIterator(std::move(base)) {}
UUID uuid() const override { return table()->getStorageID().uuid; }
};
2020-07-02 20:39:31 +00:00
DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, Context & context_)
2020-04-23 18:00:43 +00:00
: DatabaseOrdinary(name_, std::move(metadata_path_), "store/", "DatabaseAtomic (" + name_ + ")", context_)
2020-07-07 12:11:58 +00:00
, path_to_table_symlinks(global_context.getPath() + "data/" + escapeForFileName(name_) + "/")
2020-07-02 20:39:31 +00:00
, path_to_metadata_symlink(global_context.getPath() + "metadata/" + escapeForFileName(name_))
, db_uuid(uuid)
2019-10-23 13:46:38 +00:00
{
2020-07-02 20:39:31 +00:00
assert(db_uuid != UUIDHelpers::Nil);
2020-04-10 00:08:43 +00:00
Poco::File(path_to_table_symlinks).createDirectories();
2020-07-07 12:11:58 +00:00
tryCreateMetadataSymlink();
2019-11-11 11:34:03 +00:00
}
String DatabaseAtomic::getTableDataPath(const String & table_name) const
2019-11-11 11:34:03 +00:00
{
2019-12-02 19:11:18 +00:00
std::lock_guard lock(mutex);
2019-11-11 11:34:03 +00:00
auto it = table_name_to_path.find(table_name);
if (it == table_name_to_path.end())
2020-07-07 12:11:58 +00:00
throw Exception("Table " + table_name + " not found in database " + database_name, ErrorCodes::UNKNOWN_TABLE);
2020-01-16 18:13:18 +00:00
assert(it->second != data_path && !it->second.empty());
return it->second;
2019-11-11 11:34:03 +00:00
}
String DatabaseAtomic::getTableDataPath(const ASTCreateQuery & query) const
2019-11-11 11:34:03 +00:00
{
2020-04-06 23:22:44 +00:00
auto tmp = data_path + DatabaseCatalog::getPathForUUID(query.uuid);
2020-01-16 18:13:18 +00:00
assert(tmp != data_path && !tmp.empty());
return tmp;
2019-11-11 11:34:03 +00:00
}
void DatabaseAtomic::drop(const Context &)
{
2020-08-18 15:15:27 +00:00
assert(tables.empty());
2020-07-02 20:39:31 +00:00
try
{
Poco::File(path_to_metadata_symlink).remove();
Poco::File(path_to_table_symlinks).remove(true);
}
catch (...)
{
2020-07-07 12:11:58 +00:00
LOG_WARNING(log, getCurrentExceptionMessage(true));
2020-07-02 20:39:31 +00:00
}
2020-01-27 20:31:39 +00:00
Poco::File(getMetadataPath()).remove(true);
2019-11-11 11:34:03 +00:00
}
void DatabaseAtomic::attachTable(const String & name, const StoragePtr & table, const String & relative_table_path)
{
2020-01-16 18:13:18 +00:00
assert(relative_table_path != data_path && !relative_table_path.empty());
2020-04-06 23:22:44 +00:00
DetachedTables not_in_use;
2020-04-22 20:43:10 +00:00
std::unique_lock lock(mutex);
not_in_use = cleanupDetachedTables();
2020-07-07 12:11:58 +00:00
auto table_id = table->getStorageID();
assertDetachedTableNotInUse(table_id.uuid);
2020-04-22 20:43:10 +00:00
DatabaseWithDictionaries::attachTableUnlocked(name, table, lock);
2019-12-02 19:11:18 +00:00
table_name_to_path.emplace(std::make_pair(name, relative_table_path));
2019-10-23 13:46:38 +00:00
}
2019-11-11 11:34:03 +00:00
StoragePtr DatabaseAtomic::detachTable(const String & name)
{
2020-04-22 20:43:10 +00:00
DetachedTables not_in_use;
std::unique_lock lock(mutex);
auto table = DatabaseWithDictionaries::detachTableUnlocked(name, lock);
2020-03-23 00:12:13 +00:00
table_name_to_path.erase(name);
detached_tables.emplace(table->getStorageID().uuid, table);
not_in_use = cleanupDetachedTables();
2020-03-23 00:12:13 +00:00
return table;
2019-11-11 11:34:03 +00:00
}
2019-10-23 13:46:38 +00:00
2020-03-20 00:07:52 +00:00
void DatabaseAtomic::dropTable(const Context &, const String & table_name, bool no_delay)
2020-01-22 11:30:11 +00:00
{
String table_metadata_path = getObjectMetadataPath(table_name);
2020-03-23 00:12:13 +00:00
String table_metadata_path_drop;
StoragePtr table;
{
2020-04-22 20:43:10 +00:00
std::unique_lock lock(mutex);
table = getTableUnlocked(table_name, lock);
2020-03-23 00:12:13 +00:00
table_metadata_path_drop = DatabaseCatalog::instance().getPathForDroppedMetadata(table->getStorageID());
2020-04-22 20:43:10 +00:00
Poco::File(table_metadata_path).renameTo(table_metadata_path_drop); /// Mark table as dropped
DatabaseWithDictionaries::detachTableUnlocked(table_name, lock); /// Should never throw
2020-03-23 00:12:13 +00:00
table_name_to_path.erase(table_name);
}
2020-04-10 00:08:43 +00:00
tryRemoveSymlink(table_name);
/// Remove the inner table (if any) to avoid deadlock
2020-10-27 11:04:03 +00:00
/// (due to attempt to execute DROP from the worker thread)
if (auto * mv = dynamic_cast<StorageMaterializedView *>(table.get()))
mv->dropInnerTable(no_delay);
2020-04-22 20:43:10 +00:00
/// Notify DatabaseCatalog that table was dropped. It will remove table data in background.
/// Cleanup is performed outside of database to allow easily DROP DATABASE without waiting for cleanup to complete.
2020-03-20 00:07:52 +00:00
DatabaseCatalog::instance().enqueueDroppedTableCleanup(table->getStorageID(), table, table_metadata_path_drop, no_delay);
2020-01-22 11:30:11 +00:00
}
2019-11-11 14:28:28 +00:00
void DatabaseAtomic::renameTable(const Context & context, const String & table_name, IDatabase & to_database,
2020-07-16 21:41:26 +00:00
const String & to_table_name, bool exchange, bool dictionary)
2019-11-11 14:28:28 +00:00
{
if (typeid(*this) != typeid(to_database))
2019-12-02 19:11:18 +00:00
{
if (!typeid_cast<DatabaseOrdinary *>(&to_database))
throw Exception("Moving tables between databases of different engines is not supported", ErrorCodes::NOT_IMPLEMENTED);
2020-03-31 20:38:05 +00:00
/// Allow moving tables between Atomic and Ordinary (with table lock)
2020-07-16 21:41:26 +00:00
DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name, exchange, dictionary);
2019-12-02 19:11:18 +00:00
return;
}
2020-07-16 21:41:26 +00:00
if (exchange && dictionary)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot exchange dictionaries");
2020-03-31 20:38:05 +00:00
auto & other_db = dynamic_cast<DatabaseAtomic &>(to_database);
2020-04-10 01:35:37 +00:00
bool inside_database = this == &other_db;
2019-11-11 14:28:28 +00:00
2020-03-23 00:12:13 +00:00
String old_metadata_path = getObjectMetadataPath(table_name);
2020-03-23 22:40:40 +00:00
String new_metadata_path = to_database.getObjectMetadataPath(to_table_name);
2019-11-11 14:28:28 +00:00
2020-04-10 23:02:15 +00:00
auto detach = [](DatabaseAtomic & db, const String & table_name_)
2020-03-23 00:12:13 +00:00
{
2020-07-16 21:41:26 +00:00
auto it = db.table_name_to_path.find(table_name_);
String table_data_path_saved;
/// Path can be not set for DDL dictionaries, but it does not matter for StorageDictionary.
if (it != db.table_name_to_path.end())
table_data_path_saved = it->second;
assert(!table_data_path_saved.empty() || db.dictionaries.find(table_name_) != db.dictionaries.end());
2020-03-31 20:38:05 +00:00
db.tables.erase(table_name_);
db.table_name_to_path.erase(table_name_);
2020-07-16 21:41:26 +00:00
if (!table_data_path_saved.empty())
db.tryRemoveSymlink(table_name_);
2020-04-07 14:05:51 +00:00
return table_data_path_saved;
2020-03-31 20:38:05 +00:00
};
2020-04-10 23:02:15 +00:00
auto attach = [](DatabaseAtomic & db, const String & table_name_, const String & table_data_path_, const StoragePtr & table_)
2020-03-31 20:38:05 +00:00
{
db.tables.emplace(table_name_, table_);
2020-07-16 21:41:26 +00:00
if (table_data_path_.empty())
return;
2020-03-31 20:38:05 +00:00
db.table_name_to_path.emplace(table_name_, table_data_path_);
2020-04-10 23:02:15 +00:00
db.tryCreateSymlink(table_name_, table_data_path_);
2020-03-31 20:38:05 +00:00
};
2020-04-11 15:38:41 +00:00
auto assert_can_move_mat_view = [inside_database](const StoragePtr & table_)
2020-04-10 01:35:37 +00:00
{
if (inside_database)
return;
if (const auto * mv = dynamic_cast<const StorageMaterializedView *>(table_.get()))
if (mv->hasInnerTable())
throw Exception("Cannot move MaterializedView with inner table to other database", ErrorCodes::NOT_IMPLEMENTED);
};
2020-03-31 20:38:05 +00:00
String table_data_path;
String other_table_data_path;
if (inside_database && table_name == to_table_name)
return;
std::unique_lock<std::mutex> db_lock;
std::unique_lock<std::mutex> other_db_lock;
if (inside_database)
db_lock = std::unique_lock{mutex};
else if (this < &other_db)
{
db_lock = std::unique_lock{mutex};
other_db_lock = std::unique_lock{other_db.mutex};
2020-03-23 00:12:13 +00:00
}
else
{
2020-03-31 20:38:05 +00:00
other_db_lock = std::unique_lock{other_db.mutex};
db_lock = std::unique_lock{mutex};
2020-03-23 00:12:13 +00:00
}
2020-03-31 20:38:05 +00:00
2020-07-16 21:41:26 +00:00
bool is_dictionary = dictionaries.find(table_name) != dictionaries.end();
if (exchange && other_db.dictionaries.find(to_table_name) != other_db.dictionaries.end())
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot exchange dictionaries");
if (dictionary != is_dictionary)
throw Exception(ErrorCodes::INCORRECT_QUERY,
"Use RENAME DICTIONARY for dictionaries and RENAME TABLE for tables.");
if (is_dictionary && !inside_database)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot move dictionary to other database");
2020-04-22 20:43:10 +00:00
StoragePtr table = getTableUnlocked(table_name, db_lock);
2020-09-26 19:18:28 +00:00
table->checkTableCanBeRenamed();
2020-04-11 15:38:41 +00:00
assert_can_move_mat_view(table);
2020-04-10 00:08:43 +00:00
StoragePtr other_table;
if (exchange)
2020-04-10 01:35:37 +00:00
{
2020-04-22 20:43:10 +00:00
other_table = other_db.getTableUnlocked(to_table_name, other_db_lock);
2020-09-26 19:18:28 +00:00
other_table->checkTableCanBeRenamed();
2020-04-11 15:38:41 +00:00
assert_can_move_mat_view(other_table);
2020-04-10 01:35:37 +00:00
}
2020-04-10 00:08:43 +00:00
2020-04-22 20:43:10 +00:00
/// Table renaming actually begins here
2020-03-31 20:38:05 +00:00
if (exchange)
renameExchange(old_metadata_path, new_metadata_path);
else
renameNoReplace(old_metadata_path, new_metadata_path);
2020-04-22 20:43:10 +00:00
/// After metadata was successfully moved, the following methods should not throw (if them do, it's a logical error)
2020-03-31 20:38:05 +00:00
table_data_path = detach(*this, table_name);
if (exchange)
other_table_data_path = detach(other_db, to_table_name);
2020-07-16 21:41:26 +00:00
auto old_table_id = table->getStorageID();
table->renameInMemory({other_db.database_name, to_table_name, old_table_id.uuid});
2020-03-31 20:38:05 +00:00
if (exchange)
2020-07-07 12:11:58 +00:00
other_table->renameInMemory({database_name, table_name, other_table->getStorageID().uuid});
2020-03-31 20:38:05 +00:00
if (!inside_database)
{
2020-07-16 21:41:26 +00:00
DatabaseCatalog::instance().updateUUIDMapping(old_table_id.uuid, other_db.shared_from_this(), table);
2020-03-31 20:38:05 +00:00
if (exchange)
DatabaseCatalog::instance().updateUUIDMapping(other_table->getStorageID().uuid, shared_from_this(), other_table);
}
attach(other_db, to_table_name, table_data_path, table);
if (exchange)
attach(*this, table_name, other_table_data_path, other_table);
2020-07-16 21:41:26 +00:00
if (is_dictionary)
{
auto new_table_id = StorageID(other_db.database_name, to_table_name, old_table_id.uuid);
renameDictionaryInMemoryUnlocked(old_table_id, new_table_id);
}
2019-11-11 14:28:28 +00:00
}
2020-03-23 00:12:13 +00:00
void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table,
const String & table_metadata_tmp_path, const String & table_metadata_path)
{
2020-04-06 23:22:44 +00:00
DetachedTables not_in_use;
2020-03-23 00:12:13 +00:00
auto table_data_path = getTableDataPath(query);
2020-10-27 20:52:49 +00:00
bool locked_uuid = false;
2020-03-23 00:12:13 +00:00
try
{
2020-04-22 20:43:10 +00:00
std::unique_lock lock{mutex};
2020-07-17 13:11:44 +00:00
if (query.database != database_name)
throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database was renamed to `{}`, cannot create table in `{}`",
database_name, query.database);
2020-10-27 20:52:49 +00:00
/// Do some checks before renaming file from .tmp to .sql
not_in_use = cleanupDetachedTables();
assertDetachedTableNotInUse(query.uuid);
2020-10-27 20:52:49 +00:00
/// We will get en exception if some table with the same UUID exists (even if it's detached table or table from another database)
DatabaseCatalog::instance().addUUIDMapping(query.uuid);
locked_uuid = true;
/// It throws if `table_metadata_path` already exists (it's possible if table was detached)
renameNoReplace(table_metadata_tmp_path, table_metadata_path); /// Commit point (a sort of)
2020-04-22 20:43:10 +00:00
attachTableUnlocked(query.table, table, lock); /// Should never throw
2020-03-23 00:12:13 +00:00
table_name_to_path.emplace(query.table, table_data_path);
}
catch (...)
{
Poco::File(table_metadata_tmp_path).remove();
2020-10-27 20:52:49 +00:00
if (locked_uuid)
DatabaseCatalog::instance().removeUUIDMappingFinally(query.uuid);
2020-03-23 00:12:13 +00:00
throw;
}
2020-04-10 00:08:43 +00:00
tryCreateSymlink(query.table, table_data_path);
2020-03-23 00:12:13 +00:00
}
void DatabaseAtomic::commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path)
{
2020-09-19 14:19:06 +00:00
bool check_file_exists = true;
2020-07-17 13:11:44 +00:00
SCOPE_EXIT({ std::error_code code; if (check_file_exists) std::filesystem::remove(table_metadata_tmp_path, code); });
2020-03-23 00:12:13 +00:00
2020-04-22 20:43:10 +00:00
std::unique_lock lock{mutex};
auto actual_table_id = getTableUnlocked(table_id.table_name, lock)->getStorageID();
2020-03-23 00:12:13 +00:00
if (table_id.uuid != actual_table_id.uuid)
throw Exception("Cannot alter table because it was renamed", ErrorCodes::CANNOT_ASSIGN_ALTER);
2020-09-19 14:19:06 +00:00
check_file_exists = renameExchangeIfSupported(table_metadata_tmp_path, table_metadata_path);
if (!check_file_exists)
2020-07-17 13:11:44 +00:00
std::filesystem::rename(table_metadata_tmp_path, table_metadata_path);
2020-03-23 00:12:13 +00:00
}
void DatabaseAtomic::assertDetachedTableNotInUse(const UUID & uuid)
{
2020-04-22 20:43:10 +00:00
/// Without this check the following race is possible since table RWLocks are not used:
/// 1. INSERT INTO table ...;
/// 2. DETACH TABLE table; (INSERT still in progress, it holds StoragePtr)
/// 3. ATTACH TABLE table; (new instance of Storage with the same UUID is created, instances share data on disk)
/// 4. INSERT INTO table ...; (both Storage instances writes data without any synchronization)
/// To avoid it, we remember UUIDs of detached tables and does not allow ATTACH table with such UUID until detached instance still in use.
if (detached_tables.count(uuid))
throw Exception("Cannot attach table with UUID " + toString(uuid) +
2020-09-17 18:47:21 +00:00
", because it was detached but still used by some query. Retry later.", ErrorCodes::TABLE_ALREADY_EXISTS);
}
DatabaseAtomic::DetachedTables DatabaseAtomic::cleanupDetachedTables()
{
2020-04-06 23:22:44 +00:00
DetachedTables not_in_use;
auto it = detached_tables.begin();
while (it != detached_tables.end())
{
if (it->second.unique())
2020-04-06 23:22:44 +00:00
{
not_in_use.emplace(it->first, it->second);
it = detached_tables.erase(it);
2020-04-06 23:22:44 +00:00
}
else
++it;
}
2020-04-22 20:43:10 +00:00
/// It should be destroyed in caller with released database mutex
2020-04-06 23:22:44 +00:00
return not_in_use;
}
void DatabaseAtomic::assertCanBeDetached(bool cleanup)
2020-04-10 23:02:15 +00:00
{
if (cleanup)
2020-04-10 23:02:15 +00:00
{
DetachedTables not_in_use;
{
std::lock_guard lock(mutex);
not_in_use = cleanupDetachedTables();
2020-04-10 23:02:15 +00:00
}
}
std::lock_guard lock(mutex);
if (!detached_tables.empty())
throw Exception("Database " + backQuoteIfNeed(database_name) + " cannot be detached, "
"because some tables are still in use. Retry later.", ErrorCodes::DATABASE_NOT_EMPTY);
}
2020-05-28 20:10:45 +00:00
DatabaseTablesIteratorPtr DatabaseAtomic::getTablesIterator(const Context & context, const IDatabase::FilterByNameFunction & filter_by_table_name)
2020-04-01 22:41:29 +00:00
{
2020-05-28 20:10:45 +00:00
auto base_iter = DatabaseWithOwnTablesBase::getTablesIterator(context, filter_by_table_name);
2020-04-01 22:41:29 +00:00
return std::make_unique<AtomicDatabaseTablesSnapshotIterator>(std::move(typeid_cast<DatabaseTablesSnapshotIterator &>(*base_iter)));
}
2020-04-13 14:09:56 +00:00
UUID DatabaseAtomic::tryGetTableUUID(const String & table_name) const
{
2020-05-28 20:10:45 +00:00
if (auto table = tryGetTable(table_name, global_context))
2020-04-13 14:09:56 +00:00
return table->getStorageID().uuid;
return UUIDHelpers::Nil;
}
void DatabaseAtomic::loadStoredObjects(Context & context, bool has_force_restore_data_flag, bool force_attach)
2020-04-10 00:08:43 +00:00
{
/// Recreate symlinks to table data dirs in case of force restore, because some of them may be broken
if (has_force_restore_data_flag)
Poco::File(path_to_table_symlinks).remove(true);
DatabaseOrdinary::loadStoredObjects(context, has_force_restore_data_flag, force_attach);
2020-04-10 00:08:43 +00:00
if (has_force_restore_data_flag)
{
NameToPathMap table_names;
{
std::lock_guard lock{mutex};
table_names = table_name_to_path;
}
2020-08-13 17:09:40 +00:00
Poco::File(path_to_table_symlinks).createDirectories();
2020-04-10 00:08:43 +00:00
for (const auto & table : table_names)
tryCreateSymlink(table.first, table.second);
}
}
void DatabaseAtomic::tryCreateSymlink(const String & table_name, const String & actual_data_path)
{
try
{
String link = path_to_table_symlinks + escapeForFileName(table_name);
String data = Poco::Path(global_context.getPath()).makeAbsolute().toString() + actual_data_path;
2020-04-10 00:08:43 +00:00
Poco::File{data}.linkTo(link, Poco::File::LINK_SYMBOLIC);
}
catch (...)
{
LOG_WARNING(log, getCurrentExceptionMessage(true));
2020-04-10 00:08:43 +00:00
}
}
void DatabaseAtomic::tryRemoveSymlink(const String & table_name)
{
try
{
String path = path_to_table_symlinks + escapeForFileName(table_name);
Poco::File{path}.remove();
}
catch (...)
{
LOG_WARNING(log, getCurrentExceptionMessage(true));
2020-04-10 00:08:43 +00:00
}
}
2020-07-07 12:11:58 +00:00
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);
Poco::File metadata_symlink(path_to_metadata_symlink);
if (metadata_symlink.exists())
{
if (!metadata_symlink.isLink())
throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "Directory {} exists", path_to_metadata_symlink);
}
else
{
try
{
Poco::File{metadata_path}.linkTo(path_to_metadata_symlink, Poco::File::LINK_SYMBOLIC);
}
catch (...)
{
tryLogCurrentException(log);
}
}
}
void DatabaseAtomic::renameDatabase(const String & new_name)
{
2020-07-08 14:28:07 +00:00
/// CREATE, ATTACH, DROP, DETACH and RENAME DATABASE must hold DDLGuard
2020-07-07 12:11:58 +00:00
try
{
Poco::File(path_to_metadata_symlink).remove();
}
catch (...)
{
LOG_WARNING(log, getCurrentExceptionMessage(true));
}
2020-07-08 14:28:07 +00:00
auto new_name_escaped = escapeForFileName(new_name);
auto old_database_metadata_path = global_context.getPath() + "metadata/" + escapeForFileName(getDatabaseName()) + ".sql";
auto new_database_metadata_path = global_context.getPath() + "metadata/" + new_name_escaped + ".sql";
renameNoReplace(old_database_metadata_path, new_database_metadata_path);
String old_path_to_table_symlinks;
2020-07-07 12:11:58 +00:00
{
std::lock_guard lock(mutex);
2020-07-08 14:28:07 +00:00
DatabaseCatalog::instance().updateDatabaseName(database_name, new_name);
2020-07-07 12:11:58 +00:00
database_name = new_name;
for (auto & table : tables)
{
auto table_id = table.second->getStorageID();
table_id.database_name = database_name;
table.second->renameInMemory(table_id);
}
2020-07-16 14:25:39 +00:00
for (auto & dict : dictionaries)
{
auto old_name = StorageID(dict.second.create_query);
auto name = old_name;
name.database_name = database_name;
renameDictionaryInMemoryUnlocked(old_name, name);
}
2020-07-08 14:28:07 +00:00
path_to_metadata_symlink = global_context.getPath() + "metadata/" + new_name_escaped;
old_path_to_table_symlinks = path_to_table_symlinks;
path_to_table_symlinks = global_context.getPath() + "data/" + new_name_escaped + "/";
2020-07-07 12:11:58 +00:00
}
2020-07-08 14:28:07 +00:00
Poco::File(old_path_to_table_symlinks).renameTo(path_to_table_symlinks);
2020-07-07 12:11:58 +00:00
tryCreateMetadataSymlink();
}
2020-07-16 14:25:39 +00:00
void DatabaseAtomic::renameDictionaryInMemoryUnlocked(const StorageID & old_name, const StorageID & new_name)
{
auto it = dictionaries.find(old_name.table_name);
assert(it != dictionaries.end());
assert(it->second.config->getString("dictionary.uuid") == toString(old_name.uuid));
assert(old_name.uuid == new_name.uuid);
it->second.config->setString("dictionary.database", new_name.database_name);
it->second.config->setString("dictionary.name", new_name.table_name);
2020-07-16 21:41:26 +00:00
auto & create = it->second.create_query->as<ASTCreateQuery &>();
create.database = new_name.database_name;
create.table = new_name.table_name;
assert(create.uuid == new_name.uuid);
if (old_name.table_name != new_name.table_name)
{
auto attach_info = std::move(it->second);
dictionaries.erase(it);
dictionaries.emplace(new_name.table_name, std::move(attach_info));
}
2020-07-16 14:25:39 +00:00
auto result = external_loader.getLoadResult(toString(old_name.uuid));
if (!result.object)
return;
const auto & dict = dynamic_cast<const IDictionaryBase &>(*result.object);
dict.updateDictionaryName(new_name);
}
void DatabaseAtomic::waitDetachedTableNotInUse(const UUID & uuid)
{
{
std::lock_guard lock{mutex};
if (detached_tables.count(uuid) == 0)
return;
}
/// Table is in use while its shared_ptr counter is greater than 1.
/// We cannot trigger condvar on shared_ptr destruction, so it's busy wait.
while (true)
{
DetachedTables not_in_use;
{
std::lock_guard lock{mutex};
not_in_use = cleanupDetachedTables();
if (detached_tables.count(uuid) == 0)
return;
}
2020-09-29 22:30:34 +00:00
std::this_thread::sleep_for(std::chrono::milliseconds(100));
}
}
2020-07-16 14:25:39 +00:00
2019-10-23 13:46:38 +00:00
}