mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-28 10:31:57 +00:00
Rework using a separate flag file instead of renames
This commit is contained in:
parent
24abcc970b
commit
27ff3a5214
@ -151,24 +151,6 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n
|
|||||||
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);
|
||||||
|
|
||||||
if (!exchange)
|
|
||||||
{
|
|
||||||
if (auto target_db = dynamic_cast<DatabaseOnDisk *>(&to_database))
|
|
||||||
{
|
|
||||||
auto attach_query = parseQueryFromMetadata(log, context, old_metadata_path);
|
|
||||||
auto & create = attach_query->as<ASTCreateQuery &>();
|
|
||||||
create.database = to_database.getDatabaseName();
|
|
||||||
create.table = to_table_name;
|
|
||||||
|
|
||||||
// we run checks for rename same way as for create table (attach has more relaxed checks)
|
|
||||||
create.attach = false;
|
|
||||||
|
|
||||||
/// if after that check and before the actual rename the dest table
|
|
||||||
/// will be created & detached permanently the rename will overwrite it.
|
|
||||||
target_db->checkTableAttachPossible(context, create);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
auto detach = [](DatabaseAtomic & db, const String & table_name_, bool has_symlink)
|
auto detach = [](DatabaseAtomic & db, const String & table_name_, bool has_symlink)
|
||||||
{
|
{
|
||||||
auto it = db.table_name_to_path.find(table_name_);
|
auto it = db.table_name_to_path.find(table_name_);
|
||||||
@ -235,6 +217,9 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n
|
|||||||
if (is_dictionary && !inside_database)
|
if (is_dictionary && !inside_database)
|
||||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot move dictionary to other database");
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot move dictionary to other database");
|
||||||
|
|
||||||
|
if (!exchange)
|
||||||
|
other_db.checkMetadataFilenameAvailabilityUnlocked(to_table_name, inside_database ? db_lock : other_db_lock);
|
||||||
|
|
||||||
StoragePtr table = getTableUnlocked(table_name, db_lock);
|
StoragePtr table = getTableUnlocked(table_name, db_lock);
|
||||||
table->checkTableCanBeRenamed();
|
table->checkTableCanBeRenamed();
|
||||||
assert_can_move_mat_view(table);
|
assert_can_move_mat_view(table);
|
||||||
|
@ -42,6 +42,14 @@ void DatabaseLazy::loadStoredObjects(
|
|||||||
iterateMetadataFiles(context, [this](const String & file_name)
|
iterateMetadataFiles(context, [this](const String & file_name)
|
||||||
{
|
{
|
||||||
const std::string table_name = file_name.substr(0, file_name.size() - 4);
|
const std::string table_name = file_name.substr(0, file_name.size() - 4);
|
||||||
|
|
||||||
|
auto detached_permanently_flag = Poco::File(getMetadataPath() + "/" + file_name + detached_suffix);
|
||||||
|
if (detached_permanently_flag.exists())
|
||||||
|
{
|
||||||
|
LOG_DEBUG(log, "Skipping permanently detached table {}.", backQuote(table_name));
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
attachTable(table_name, nullptr, {});
|
attachTable(table_name, nullptr, {});
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
@ -151,8 +151,6 @@ void DatabaseOnDisk::createTable(
|
|||||||
const auto & create = query->as<ASTCreateQuery &>();
|
const auto & create = query->as<ASTCreateQuery &>();
|
||||||
assert(table_name == create.table);
|
assert(table_name == create.table);
|
||||||
|
|
||||||
checkTableAttachPossible(context, create);
|
|
||||||
|
|
||||||
/// Create a file with metadata if necessary - if the query is not ATTACH.
|
/// Create a file with metadata if necessary - if the query is not ATTACH.
|
||||||
/// Write the query of `ATTACH table` to it.
|
/// Write the query of `ATTACH table` to it.
|
||||||
|
|
||||||
@ -165,19 +163,23 @@ void DatabaseOnDisk::createTable(
|
|||||||
/// A race condition would be possible if a table with the same name is simultaneously created using CREATE and using ATTACH.
|
/// A race condition would be possible if a table with the same name is simultaneously created using CREATE and using ATTACH.
|
||||||
/// But there is protection from it - see using DDLGuard in InterpreterCreateQuery.
|
/// But there is protection from it - see using DDLGuard in InterpreterCreateQuery.
|
||||||
|
|
||||||
String table_metadata_path = getObjectMetadataPath(table_name);
|
if (isDictionaryExist(table_name))
|
||||||
|
throw Exception(ErrorCodes::DICTIONARY_ALREADY_EXISTS, "Dictionary {}.{} already exists", backQuote(getDatabaseName()), backQuote(table_name));
|
||||||
|
|
||||||
if (create.attach_short_syntax && Poco::File(table_metadata_path).exists())
|
if (isTableExist(table_name, global_context))
|
||||||
|
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exists", backQuote(getDatabaseName()), backQuote(table_name));
|
||||||
|
|
||||||
|
if (!create.attach)
|
||||||
|
checkMetadataFilenameAvailability(table_name);
|
||||||
|
|
||||||
|
if (create.attach_short_syntax)
|
||||||
{
|
{
|
||||||
/// Metadata already exists, table was detached (not permanently)
|
/// Metadata already exists, table was detached
|
||||||
attachTable(table_name, table, getTableDataPath(create));
|
attachTable(table_name, table, getTableDataPath(create));
|
||||||
return;
|
return;
|
||||||
|
|
||||||
/// if the table was detached permanently, then usual metadata file doesn't exists
|
|
||||||
/// (.sql_detached instead) and we use longer, but safer way of attaching that back
|
|
||||||
/// with recreating the metadata file.
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
String table_metadata_path = getObjectMetadataPath(table_name);
|
||||||
String table_metadata_tmp_path = table_metadata_path + create_suffix;
|
String table_metadata_tmp_path = table_metadata_path + create_suffix;
|
||||||
String statement;
|
String statement;
|
||||||
|
|
||||||
@ -216,16 +218,16 @@ void DatabaseOnDisk::commitCreateTable(const ASTCreateQuery & query, const Stora
|
|||||||
|
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
/// If the table was detached permanently we will have a file with
|
/// If the table was detached permanently we will have a flag file with
|
||||||
/// .sql_detached suffix, which is not needed anymore since we attached the table back
|
/// .sql.detached extension, which is not needed anymore since we attached the table back
|
||||||
auto table_metadata_file_detached = Poco::File(table_metadata_path + detached_suffix);
|
auto detached_permanently_flag = Poco::File(table_metadata_path + detached_suffix);
|
||||||
if (table_metadata_file_detached.exists())
|
if (detached_permanently_flag.exists())
|
||||||
table_metadata_file_detached.remove();
|
detached_permanently_flag.remove();
|
||||||
}
|
}
|
||||||
catch (...)
|
catch (Exception & e)
|
||||||
{
|
{
|
||||||
// It's not a big issue if we can't remove the .sql_detached file.
|
e.addMessage("while trying to remove permanenty detached flag. Table {}.{} may still be marked as permanently detached, and will not be reattached during server restart.", backQuote(getDatabaseName()), backQuote(query.table));
|
||||||
LOG_WARNING(log, getCurrentExceptionMessage(__PRETTY_FUNCTION__));
|
throw;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -233,37 +235,18 @@ void DatabaseOnDisk::detachTablePermanently(const String & table_name)
|
|||||||
{
|
{
|
||||||
auto table = detachTable(table_name);
|
auto table = detachTable(table_name);
|
||||||
|
|
||||||
String table_metadata_path = getObjectMetadataPath(table_name);
|
Poco::File detached_permanently_flag(getObjectMetadataPath(table_name) + detached_suffix);
|
||||||
String table_metadata_path_detached = table_metadata_path + detached_suffix;
|
|
||||||
|
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
if (table)
|
detached_permanently_flag.createFile();
|
||||||
{
|
|
||||||
auto table_id = table->getStorageID();
|
|
||||||
|
|
||||||
/// usual detach don't remove UUID from the mapping
|
|
||||||
/// (it's done to prevent accidental reuse of table UUID)
|
|
||||||
/// but since reattach of permanently detached table
|
|
||||||
/// happens in a similar way as normal table creation
|
|
||||||
/// (with sql recreation, and adding uuid to a mapping)
|
|
||||||
/// we need to have uuid free and avaliable for further attaches.
|
|
||||||
if (table_id.hasUUID())
|
|
||||||
DatabaseCatalog::instance().removeUUIDMappingFinally(table_id.uuid);
|
|
||||||
}
|
|
||||||
|
|
||||||
/// it will silently overwrite the file if exists, and it's ok
|
|
||||||
Poco::File(table_metadata_path).renameTo(table_metadata_path_detached);
|
|
||||||
}
|
}
|
||||||
catch (Exception & e)
|
catch (Exception & e)
|
||||||
{
|
{
|
||||||
e.addMessage("while trying to detach table {} permanently.", table_name);
|
e.addMessage("while trying to set permanenty detached flag. Table {}.{} may be reattached during server restart.", backQuote(getDatabaseName()), backQuote(table_name));
|
||||||
throw;
|
throw;
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void DatabaseOnDisk::dropTable(const Context & context, const String & table_name, bool /*no_delay*/)
|
void DatabaseOnDisk::dropTable(const Context & context, const String & table_name, bool /*no_delay*/)
|
||||||
{
|
{
|
||||||
String table_metadata_path = getObjectMetadataPath(table_name);
|
String table_metadata_path = getObjectMetadataPath(table_name);
|
||||||
@ -302,40 +285,24 @@ void DatabaseOnDisk::dropTable(const Context & context, const String & table_nam
|
|||||||
Poco::File(table_metadata_path_drop).remove();
|
Poco::File(table_metadata_path_drop).remove();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void DatabaseOnDisk::checkMetadataFilenameAvailability(const String & to_table_name) const
|
||||||
void DatabaseOnDisk::checkTableAttachPossible(const Context & context, const ASTCreateQuery & create) const
|
|
||||||
{
|
{
|
||||||
String to_table_name = create.table;
|
std::unique_lock lock(mutex);
|
||||||
|
checkMetadataFilenameAvailabilityUnlocked(create, lock);
|
||||||
|
}
|
||||||
|
|
||||||
|
void DatabaseOnDisk::checkMetadataFilenameAvailabilityUnlocked(const String & to_table_name, std::unique_lock<std::mutex> &) const
|
||||||
|
{
|
||||||
String table_metadata_path = getObjectMetadataPath(to_table_name);
|
String table_metadata_path = getObjectMetadataPath(to_table_name);
|
||||||
|
|
||||||
if (isDictionaryExist(to_table_name))
|
if (Poco::File(table_metadata_path).exists())
|
||||||
throw Exception(ErrorCodes::DICTIONARY_ALREADY_EXISTS, "Dictionary {}.{} already exists", backQuote(getDatabaseName()), backQuote(to_table_name));
|
|
||||||
|
|
||||||
if (isTableExist(to_table_name, global_context))
|
|
||||||
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exists", backQuote(getDatabaseName()), backQuote(to_table_name));
|
|
||||||
|
|
||||||
if (!create.attach && Poco::File(table_metadata_path).exists())
|
|
||||||
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exists (detached)", backQuote(getDatabaseName()), backQuote(to_table_name));
|
|
||||||
|
|
||||||
/// if we have a table with a same name detached permanently we only allow
|
|
||||||
/// attaching it (and uuid should be the same), but not creating
|
|
||||||
String table_metadata_path_detached = table_metadata_path + detached_suffix;
|
|
||||||
|
|
||||||
if (Poco::File(table_metadata_path_detached).exists())
|
|
||||||
{
|
{
|
||||||
if (!create.attach)
|
auto detached_permanently_flag = Poco::File(table_metadata_path + detached_suffix);
|
||||||
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exists (detached permanently)", backQuote(getDatabaseName()), backQuote(to_table_name));
|
|
||||||
|
|
||||||
if (!create.attach_short_syntax)
|
if (detached_permanently_flag.exists())
|
||||||
{
|
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exists (detached permanently)", backQuote(database_name), backQuote(to_table_name));
|
||||||
ASTPtr ast_detached = parseQueryFromMetadata(log, context, table_metadata_path_detached);
|
else
|
||||||
auto & create_detached = ast_detached->as<ASTCreateQuery &>();
|
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exists (detached)", backQuote(database_name), backQuote(to_table_name));
|
||||||
|
|
||||||
// either both should be Nil, either values should be equal
|
|
||||||
if (create.uuid != create_detached.uuid)
|
|
||||||
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exist (detached permanently). To attach it back you need to use short ATTACH syntax or a full statement with the same UUID", backQuote(getDatabaseName()), backQuote(to_table_name));
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -385,13 +352,8 @@ void DatabaseOnDisk::renameTable(
|
|||||||
if (from_atomic_to_ordinary)
|
if (from_atomic_to_ordinary)
|
||||||
create.uuid = UUIDHelpers::Nil;
|
create.uuid = UUIDHelpers::Nil;
|
||||||
|
|
||||||
|
if (auto * target_db = dynamic_cast<DatabaseOnDisk *>(&to_database))
|
||||||
if (auto target_db = dynamic_cast<DatabaseOnDisk *>(&to_database))
|
target_db->checkMetadataFilenameAvailability(to_table_name);
|
||||||
{
|
|
||||||
// we run checks for rename same way as for create table (attach has more relaxed checks)
|
|
||||||
create.attach = false;
|
|
||||||
target_db->checkTableAttachPossible(context, create);
|
|
||||||
}
|
|
||||||
|
|
||||||
/// Notify the table that it is renamed. It will move data to new path (if it stores data on disk) and update StorageID
|
/// Notify the table that it is renamed. It will move data to new path (if it stores data on disk) and update StorageID
|
||||||
table->rename(to_database.getTableDataPath(create), StorageID(create));
|
table->rename(to_database.getTableDataPath(create), StorageID(create));
|
||||||
@ -423,21 +385,15 @@ void DatabaseOnDisk::renameTable(
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/// It returns create table statement (even if table is detached permanently)
|
/// It returns create table statement (even if table is detached)
|
||||||
ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const String & table_name, const Context &, bool throw_on_error) const
|
ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const String & table_name, const Context &, bool throw_on_error) const
|
||||||
{
|
{
|
||||||
ASTPtr ast;
|
ASTPtr ast;
|
||||||
bool has_table = tryGetTable(table_name, global_context) != nullptr;
|
bool has_table = tryGetTable(table_name, global_context) != nullptr;
|
||||||
auto table_metadata_path = getObjectMetadataPath(table_name);
|
auto table_metadata_path = getObjectMetadataPath(table_name);
|
||||||
|
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
if (Poco::File(table_metadata_path).exists())
|
ast = getCreateQueryFromMetadata(table_metadata_path, throw_on_error);
|
||||||
ast = getCreateQueryFromMetadata(table_metadata_path, throw_on_error);
|
|
||||||
else if (Poco::File(table_metadata_path + detached_suffix).exists())
|
|
||||||
ast = getCreateQueryFromMetadata(table_metadata_path + detached_suffix, throw_on_error);
|
|
||||||
else if (throw_on_error)
|
|
||||||
throw Exception("Metadata file does not exist", ErrorCodes::FILE_DOESNT_EXIST);
|
|
||||||
}
|
}
|
||||||
catch (const Exception & e)
|
catch (const Exception & e)
|
||||||
{
|
{
|
||||||
@ -532,12 +488,11 @@ void DatabaseOnDisk::iterateMetadataFiles(const Context & context, const Iterati
|
|||||||
if (endsWith(dir_it.name(), ".sql.bak"))
|
if (endsWith(dir_it.name(), ".sql.bak"))
|
||||||
continue;
|
continue;
|
||||||
|
|
||||||
/// Permanently detached tables are not attached automatically
|
/// Permanently detached table flag
|
||||||
if (endsWith(dir_it.name(), ".sql_detached"))
|
if (endsWith(dir_it.name(), ".sql.detached"))
|
||||||
continue;
|
continue;
|
||||||
|
|
||||||
static const char * tmp_drop_ext = ".sql.tmp_drop";
|
if (endsWith(dir_it.name(), ".sql.tmp_drop"))
|
||||||
if (endsWith(dir_it.name(), tmp_drop_ext))
|
|
||||||
{
|
{
|
||||||
/// There are files that we tried to delete previously
|
/// There are files that we tried to delete previously
|
||||||
metadata_files.emplace(dir_it.name(), false);
|
metadata_files.emplace(dir_it.name(), false);
|
||||||
@ -616,8 +571,6 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(Poco::Logger * logger, const Conte
|
|||||||
auto & create = ast->as<ASTCreateQuery &>();
|
auto & create = ast->as<ASTCreateQuery &>();
|
||||||
if (!create.table.empty() && create.uuid != UUIDHelpers::Nil)
|
if (!create.table.empty() && create.uuid != UUIDHelpers::Nil)
|
||||||
{
|
{
|
||||||
/// if the table is detached permanently getBaseName will still return a proper name
|
|
||||||
/// because we use table_name.sql_detached naming
|
|
||||||
String table_name = Poco::Path(metadata_file_path).makeFile().getBaseName();
|
String table_name = Poco::Path(metadata_file_path).makeFile().getBaseName();
|
||||||
table_name = unescapeForFileName(table_name);
|
table_name = unescapeForFileName(table_name);
|
||||||
|
|
||||||
|
@ -70,18 +70,13 @@ public:
|
|||||||
static ASTPtr parseQueryFromMetadata(Poco::Logger * log, const Context & context, const String & metadata_file_path, bool throw_on_error = true, bool remove_empty = false);
|
static ASTPtr parseQueryFromMetadata(Poco::Logger * log, const Context & context, const String & metadata_file_path, bool throw_on_error = true, bool remove_empty = false);
|
||||||
|
|
||||||
/// will throw when the table we want to attach already exists (in active / detached / detached permanently form)
|
/// will throw when the table we want to attach already exists (in active / detached / detached permanently form)
|
||||||
/// still allow to overwrite the detached table if the table we want to attach is the old one
|
void checkMetadataFilenameAvailability(const String & to_table_name) const;
|
||||||
/// uses locks.
|
void checkMetadataFilenameAvailabilityUnlocked(const String & to_table_name, std::unique_lock<std::mutex> &) const;
|
||||||
void checkTableAttachPossible(const Context & context, const ASTCreateQuery & create) const;
|
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
static constexpr const char * create_suffix = ".tmp";
|
static constexpr const char * create_suffix = ".tmp";
|
||||||
static constexpr const char * drop_suffix = ".tmp_drop";
|
static constexpr const char * drop_suffix = ".tmp_drop";
|
||||||
|
static constexpr const char * detached_suffix = ".detached";
|
||||||
/// engine=Atomic takes the table name from basename of metadata file (also for detached table)
|
|
||||||
/// in case of double dots (table_name.sql.detached) it would extract 'table_name.sql'
|
|
||||||
/// so we use simpler option "table_name.sql_detached" and get 'table_name' correctly.
|
|
||||||
static constexpr const char * detached_suffix = "_detached";
|
|
||||||
|
|
||||||
using IteratingFunction = std::function<void(const String &)>;
|
using IteratingFunction = std::function<void(const String &)>;
|
||||||
|
|
||||||
@ -99,7 +94,6 @@ protected:
|
|||||||
|
|
||||||
const String metadata_path;
|
const String metadata_path;
|
||||||
const String data_path;
|
const String data_path;
|
||||||
|
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -135,6 +135,19 @@ void DatabaseOrdinary::loadStoredObjects(Context & context, bool has_force_resto
|
|||||||
{
|
{
|
||||||
auto * create_query = ast->as<ASTCreateQuery>();
|
auto * create_query = ast->as<ASTCreateQuery>();
|
||||||
create_query->database = database_name;
|
create_query->database = database_name;
|
||||||
|
|
||||||
|
auto detached_permanently_flag = Poco::File(full_path.string() + detached_suffix);
|
||||||
|
if (detached_permanently_flag.exists())
|
||||||
|
{
|
||||||
|
/// even if we don't load the table we still mark the uuid of it as taken.
|
||||||
|
if (create_query->uuid != UUIDHelpers::Nil)
|
||||||
|
DatabaseCatalog::instance().addUUIDMapping(create_query->uuid);
|
||||||
|
|
||||||
|
const std::string table_name = file_name.substr(0, file_name.size() - 4);
|
||||||
|
LOG_DEBUG(log, "Skipping permanently detached table {}.", backQuote(table_name));
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
std::lock_guard lock{file_names_mutex};
|
std::lock_guard lock{file_names_mutex};
|
||||||
file_names[file_name] = ast;
|
file_names[file_name] = ast;
|
||||||
total_dictionaries += create_query->is_dictionary;
|
total_dictionaries += create_query->is_dictionary;
|
||||||
|
@ -307,7 +307,6 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query,
|
|||||||
stopDatabaseSynchronization(database);
|
stopDatabaseSynchronization(database);
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
|
|
||||||
if (database->shouldBeEmptyOnDetach())
|
if (database->shouldBeEmptyOnDetach())
|
||||||
{
|
{
|
||||||
/// DETACH or DROP all tables and dictionaries inside database.
|
/// DETACH or DROP all tables and dictionaries inside database.
|
||||||
|
Loading…
Reference in New Issue
Block a user