Add settings "throw_if_table_has_different_definition" and "throw_if_database_has_different_definition".

This commit is contained in:
Vitaly Baranov 2022-01-23 20:35:11 +07:00 committed by Vitaly Baranov
parent fd1d24f624
commit ef57a87394
10 changed files with 293 additions and 115 deletions

View File

@ -1,8 +1,10 @@
#include <Backups/BackupUtils.h>
#include <Backups/BackupEntryFromMemory.h>
#include <Backups/BackupSettings.h>
#include <Backups/DDLCompareUtils.h>
#include <Backups/DDLRenamingVisitor.h>
#include <Backups/IBackup.h>
#include <Backups/formatTableNameOrTemporaryTableName.h>
#include <Common/escapeForFileName.h>
#include <Access/Common/AccessFlags.h>
#include <Databases/IDatabase.h>
@ -82,7 +84,7 @@ namespace
{
if (info.different_create_query)
throw Exception(ErrorCodes::CANNOT_BACKUP_DATABASE,
"Couldn't backup a database because two different create queries were generated for it: {} and {}",
"Cannot backup a database because two different create queries were generated for it: {} and {}",
serializeAST(*info.create_query), serializeAST(*info.different_create_query));
}
@ -130,28 +132,24 @@ namespace
if (database->hasHollowBackup())
throw Exception(
ErrorCodes::CANNOT_BACKUP_TABLE,
"Couldn't backup table {}.{} because of the database's engine {} is hollow",
backQuoteIfNeed(table_name_.first), backQuoteIfNeed(table_name_.second),
"Cannot backup the {} because it's contained in a hollow database (engine: {})",
formatTableNameOrTemporaryTableName(table_name_),
database->getEngineName());
/// Check that we are not trying to backup the same table again.
DatabaseAndTableName new_table_name = renaming_settings.getNewTableName(table_name_);
if (tables.contains(new_table_name))
{
String message;
if (new_table_name.first == DatabaseCatalog::TEMPORARY_DATABASE)
message = fmt::format("Couldn't backup temporary table {} twice", backQuoteIfNeed(new_table_name.second));
else
message = fmt::format("Couldn't backup table {}.{} twice", backQuoteIfNeed(new_table_name.first), backQuoteIfNeed(new_table_name.second));
throw Exception(ErrorCodes::CANNOT_BACKUP_TABLE, message);
}
throw Exception(ErrorCodes::CANNOT_BACKUP_TABLE, "Cannot backup the {} twice", formatTableNameOrTemporaryTableName(new_table_name));
/// Make a create query for this table.
auto create_query = renameInCreateQuery(database->getCreateTableQuery(table_name_.second, context));
bool has_data = !storage->hasHollowBackup() && !backup_settings.structure_only;
if (has_data)
{
/// We check for SELECT privilege only if we're going to read data from the table.
context->checkAccess(AccessType::SELECT, table_name_.first, table_name_.second);
}
CreateTableInfo info;
info.create_query = create_query;
@ -185,7 +183,7 @@ namespace
{
auto create_db_query = renameInCreateQuery(table_.first->getCreateDatabaseQuery());
create_db_query->setDatabase(new_table_name.first);
if (serializeAST(*info_db.create_query) != serializeAST(*create_db_query))
if (!areDatabaseDefinitionsSame(*info_db.create_query, *create_db_query))
info_db.different_create_query = create_db_query;
}
}
@ -206,7 +204,7 @@ namespace
/// Check that we are not trying to restore the same database again.
String new_database_name = renaming_settings.getNewDatabaseName(database_name_);
if (databases.contains(new_database_name) && databases[new_database_name].is_explicit)
throw Exception(ErrorCodes::CANNOT_BACKUP_DATABASE, "Couldn't backup database {} twice", backQuoteIfNeed(new_database_name));
throw Exception(ErrorCodes::CANNOT_BACKUP_DATABASE, "Cannot backup the database {} twice", backQuoteIfNeed(new_database_name));
/// Of course we're not going to backup the definition of the system or the temporary database.
if (!isSystemOrTemporaryDatabase(database_name_))

View File

@ -0,0 +1,83 @@
#include <Backups/DDLCompareUtils.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/formatAST.h>
namespace DB
{
namespace
{
std::shared_ptr<const ASTCreateQuery> prepareDDLToCompare(const ASTCreateQuery & ast)
{
auto res = typeid_cast<std::shared_ptr<const ASTCreateQuery>>(ast.shared_from_this());
std::shared_ptr<ASTCreateQuery> clone;
auto get_clone = [&]
{
if (!clone)
{
clone = typeid_cast<std::shared_ptr<ASTCreateQuery>>(res->clone());
res = clone;
}
return clone;
};
/// Remove UUID.
if (res->uuid != UUIDHelpers::Nil)
get_clone()->uuid = UUIDHelpers::Nil;
/// Clear IF NOT EXISTS flag.
if (res->if_not_exists)
get_clone()->if_not_exists = false;
return res;
}
}
bool areTableDefinitionsSame(const IAST & table1, const IAST & table2)
{
auto ast1 = typeid_cast<std::shared_ptr<const ASTCreateQuery>>(table1.shared_from_this());
if (!ast1 || !ast1->table)
return false;
auto ast2 = typeid_cast<std::shared_ptr<const ASTCreateQuery>>(table2.shared_from_this());
if (!ast2 || !ast2->table)
return false;
if ((ast1->uuid != ast2->uuid) || (ast1->if_not_exists != ast2->if_not_exists))
{
ast1 = prepareDDLToCompare(*ast1);
ast2 = prepareDDLToCompare(*ast2);
}
return serializeAST(*ast1) == serializeAST(*ast1);
}
bool areDatabaseDefinitionsSame(const IAST & database1, const IAST & database2)
{
auto ast1 = typeid_cast<std::shared_ptr<const ASTCreateQuery>>(database1.shared_from_this());
if (!ast1 || ast1->table || !ast1->database)
return false;
auto ast2 = typeid_cast<std::shared_ptr<const ASTCreateQuery>>(database2.shared_from_this());
if (!ast2 || ast2->table || !ast2->database)
return false;
if ((ast1->uuid != ast2->uuid) || (ast1->if_not_exists != ast2->if_not_exists))
{
ast1 = prepareDDLToCompare(*ast1);
ast2 = prepareDDLToCompare(*ast2);
}
return serializeAST(*ast1) == serializeAST(*ast1);
}
bool areTableDataCompatible(const IAST & src_table, const IAST & dest_table)
{
return areTableDefinitionsSame(src_table, dest_table);
}
}

View File

@ -0,0 +1,17 @@
#pragma once
namespace DB
{
class IAST;
/// Checks that two table definitions are actually the same.
bool areTableDefinitionsSame(const IAST & table1, const IAST & table2);
/// Checks that two database definitions are actually the same.
bool areDatabaseDefinitionsSame(const IAST & database1, const IAST & database2);
/// Whether the data from the first table can be attached to the second table.
bool areTableDataCompatible(const IAST & src_table, const IAST & dest_table);
}

View File

@ -29,6 +29,10 @@ RestoreSettings RestoreSettings::fromRestoreQuery(const ASTBackupQuery & query)
res.throw_if_database_exists = setting.value.safeGet<bool>();
else if (setting.name == "throw_if_table_exists")
res.throw_if_table_exists = setting.value.safeGet<bool>();
else if (setting.name == "throw_if_database_has_different_definition")
res.throw_if_database_has_different_definition = setting.value.safeGet<bool>();
else if (setting.name == "throw_if_table_has_different_definition")
res.throw_if_table_has_different_definition = setting.value.safeGet<bool>();
else
throw Exception(ErrorCodes::UNKNOWN_SETTING, "Unknown setting {}", setting.name);
}

View File

@ -30,6 +30,14 @@ struct RestoreSettings : public StorageRestoreSettings
/// Whether RESTORE TABLE will throw an exception if a destination table already exists.
bool throw_if_table_exists = true;
/// Whether RESTORE DATABASE will throw an exception if a destination database has
/// a different definition comparing with the definition read from backup.
bool throw_if_database_has_different_definition = true;
/// Whether RESTORE TABLE will throw an exception if a destination table has
/// a different definition comparing with the definition read from backup.
bool throw_if_table_has_different_definition = true;
static RestoreSettings fromRestoreQuery(const ASTBackupQuery & query);
};

View File

@ -1,11 +1,12 @@
#include <Backups/RestoreUtils.h>
#include <Backups/BackupUtils.h>
#include <Backups/DDLCompareUtils.h>
#include <Backups/DDLRenamingVisitor.h>
#include <Backups/IBackup.h>
#include <Backups/IBackupEntry.h>
#include <Backups/IRestoreTask.h>
#include <Backups/RestoreSettings.h>
#include <Backups/hasCompatibleDataToRestoreTable.h>
#include <Backups/formatTableNameOrTemporaryTableName.h>
#include <Common/escapeForFileName.h>
#include <Databases/IDatabase.h>
#include <IO/ReadHelpers.h>
@ -37,19 +38,29 @@ namespace
using Elements = ASTBackupQuery::Elements;
using ElementType = ASTBackupQuery::ElementType;
/// Restores a database (without tables inside), should be executed before executing
/// RestoreTableTask.
class RestoreDatabaseTask : public IRestoreTask
{
public:
RestoreDatabaseTask(ContextMutablePtr context_, const ASTPtr & create_query_, const RestoreSettings &)
: context(context_), create_query(typeid_cast<std::shared_ptr<ASTCreateQuery>>(create_query_))
RestoreDatabaseTask(
ContextMutablePtr context_,
const ASTPtr & create_query_,
const RestoreSettings & restore_settings_,
bool skip_same_definition_check_)
: context(context_)
, create_query(typeid_cast<std::shared_ptr<ASTCreateQuery>>(create_query_))
, restore_settings(restore_settings_)
, skip_same_definition_check(skip_same_definition_check_)
{
}
RestoreTasks run() override
{
createDatabase();
getDatabase();
checkDatabaseCreateQuery();
return {};
}
@ -58,12 +69,50 @@ namespace
private:
void createDatabase()
{
InterpreterCreateQuery create_interpreter{create_query, context};
/// We need to call clone() for `create_query` because the interpreter can decide
/// to change a passed AST a little bit.
InterpreterCreateQuery create_interpreter{create_query->clone(), context};
create_interpreter.execute();
}
DatabasePtr getDatabase()
{
if (!database)
database = DatabaseCatalog::instance().getDatabase(create_query->getDatabase());
return database;
}
ASTPtr getDatabaseCreateQuery()
{
if (!database_create_query)
database_create_query = getDatabase()->getCreateDatabaseQuery();
return database_create_query;
}
void checkDatabaseCreateQuery()
{
if (skip_same_definition_check || !restore_settings.throw_if_database_has_different_definition)
return;
getDatabaseCreateQuery();
if (areDatabaseDefinitionsSame(*create_query, *database_create_query))
return;
throw Exception(
ErrorCodes::CANNOT_RESTORE_DATABASE,
"The database {} already exists but has a different definition: {}, "
"compare to its definition in the backup: {}",
backQuoteIfNeed(create_query->getDatabase()),
serializeAST(*database_create_query),
serializeAST(*create_query));
}
ContextMutablePtr context;
std::shared_ptr<ASTCreateQuery> create_query;
RestoreSettings restore_settings;
bool skip_same_definition_check = false;
DatabasePtr database;
ASTPtr database_create_query;
};
@ -90,9 +139,10 @@ namespace
RestoreTasks run() override
{
createStorage();
auto storage = getStorage();
getStorage();
checkStorageCreateQuery();
RestoreTasks tasks;
if (auto task = insertDataIntoStorage(storage))
if (auto task = insertData())
tasks.push_back(std::move(task));
return tasks;
}
@ -102,59 +152,84 @@ namespace
private:
void createStorage()
{
InterpreterCreateQuery create_interpreter{create_query, context};
/// We need to call clone() for `create_query` because the interpreter can decide
/// to change a passed AST a little bit.
InterpreterCreateQuery create_interpreter{create_query->clone(), context};
create_interpreter.execute();
}
StoragePtr tryGetStorage()
{
if (!DatabaseCatalog::instance().isTableExist({table_name.first, table_name.second}, context))
return nullptr;
DatabasePtr existing_database;
StoragePtr existing_storage;
std::tie(existing_database, existing_storage) = DatabaseCatalog::instance().tryGetDatabaseAndTable({table_name.first, table_name.second}, context);
if (!existing_storage)
return nullptr;
auto existing_table_create_query = existing_database->tryGetCreateTableQuery(table_name.second, context);
if (!existing_table_create_query)
return nullptr;
if (!hasCompatibleDataToRestoreTable(*create_query, existing_table_create_query->as<ASTCreateQuery &>()))
throw Exception(
ErrorCodes::CANNOT_RESTORE_TABLE,
"Table {}.{} from backup is incompatible with existing table {}.{}. "
"The create query of the table from backup: {}."
"The create query of the existing table: {}",
backQuoteIfNeed(table_name_in_backup.first),
backQuoteIfNeed(table_name_in_backup.second),
backQuoteIfNeed(table_name.first),
backQuoteIfNeed(table_name.second),
serializeAST(*create_query),
serializeAST(*existing_table_create_query));
return existing_storage;
}
StoragePtr getStorage()
{
if (auto storage = tryGetStorage())
return storage;
String error_message = (table_name.first == DatabaseCatalog::TEMPORARY_DATABASE)
? ("Could not create temporary table " + backQuoteIfNeed(table_name.second) + " for restoring")
: ("Could not create table " + backQuoteIfNeed(table_name.first) + "." + backQuoteIfNeed(table_name.second)
+ " for restoring");
throw Exception(error_message, ErrorCodes::CANNOT_RESTORE_TABLE);
if (!storage)
std::tie(database, storage) = DatabaseCatalog::instance().getDatabaseAndTable({table_name.first, table_name.second}, context);
return storage;
}
RestoreTaskPtr insertDataIntoStorage(StoragePtr storage)
ASTPtr getStorageCreateQuery()
{
if (storage->hasHollowBackup() || restore_settings.structure_only)
return {};
if (!storage_create_query)
{
getStorage();
storage_create_query = database->getCreateTableQuery(table_name.second, context);
}
return storage_create_query;
}
void checkStorageCreateQuery()
{
if (!restore_settings.throw_if_table_has_different_definition)
return;
getStorageCreateQuery();
if (areTableDefinitionsSame(*create_query, *storage_create_query))
return;
throw Exception(
ErrorCodes::CANNOT_RESTORE_TABLE,
"The {} already exists but has a different definition: {}, "
"compare to its definition in the backup: {}",
formatTableNameOrTemporaryTableName(table_name),
serializeAST(*storage_create_query),
serializeAST(*create_query));
}
bool hasData()
{
if (has_data)
return *has_data;
has_data = false;
if (restore_settings.structure_only)
return false;
data_path_in_backup = getDataPathInBackup(table_name_in_backup);
if (backup->listFiles(data_path_in_backup).empty())
return false;
getStorageCreateQuery();
if (!areTableDataCompatible(*create_query, *storage_create_query))
throw Exception(
ErrorCodes::CANNOT_RESTORE_TABLE,
"Cannot attach data of the {} in the backup to the existing {} because of they are not compatible. "
"Here is the definition of the {} in the backup: {}, and here is the definition of the existing {}: {}",
formatTableNameOrTemporaryTableName(table_name_in_backup),
formatTableNameOrTemporaryTableName(table_name),
formatTableNameOrTemporaryTableName(table_name_in_backup),
serializeAST(*create_query),
formatTableNameOrTemporaryTableName(table_name),
serializeAST(*storage_create_query));
/// We check for INSERT privilege only if we're going to write into table.
context->checkAccess(AccessType::INSERT, table_name.first, table_name.second);
String data_path_in_backup = getDataPathInBackup(table_name_in_backup);
has_data = true;
return true;
}
RestoreTaskPtr insertData()
{
if (!hasData())
return {};
return storage->restoreFromBackup(context, partitions, backup, data_path_in_backup, restore_settings);
}
@ -165,6 +240,11 @@ namespace
BackupPtr backup;
DatabaseAndTableName table_name_in_backup;
RestoreSettings restore_settings;
DatabasePtr database;
StoragePtr storage;
ASTPtr storage_create_query;
std::optional<bool> has_data;
String data_path_in_backup;
};
@ -220,13 +300,14 @@ namespace
{
if (info.different_create_query)
throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE,
"Couldn't restore a database because two different create queries were generated for it: {} and {}",
"Cannot restore a database because two different create queries were generated for it: {} and {}",
serializeAST(*info.create_query), serializeAST(*info.different_create_query));
}
RestoreTasks res;
for (auto & info : databases | boost::adaptors::map_values)
res.push_back(std::make_unique<RestoreDatabaseTask>(context, info.create_query, restore_settings));
res.push_back(std::make_unique<RestoreDatabaseTask>(context, info.create_query, restore_settings,
/* skip_same_definition_check = */ !info.is_explicit));
/// TODO: We need to restore tables according to their dependencies.
for (auto & info : tables | boost::adaptors::map_values)
@ -242,14 +323,7 @@ namespace
/// Check that we are not trying to restore the same table again.
DatabaseAndTableName new_table_name = renaming_settings.getNewTableName(table_name_);
if (tables.contains(new_table_name))
{
String message;
if (new_table_name.first == DatabaseCatalog::TEMPORARY_DATABASE)
message = fmt::format("Couldn't restore temporary table {} twice", backQuoteIfNeed(new_table_name.second));
else
message = fmt::format("Couldn't restore table {}.{} twice", backQuoteIfNeed(new_table_name.first), backQuoteIfNeed(new_table_name.second));
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, message);
}
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore the {} twice", formatTableNameOrTemporaryTableName(new_table_name));
/// Make a create query for this table.
auto create_query = renameInCreateQuery(readCreateQueryFromBackup(table_name_));
@ -301,7 +375,7 @@ namespace
create_db_query = std::make_shared<ASTCreateQuery>();
create_db_query->setDatabase(new_table_name.first);
create_db_query->if_not_exists = true;
if (serializeAST(*info_db.create_query) != serializeAST(*create_db_query))
if (!areDatabaseDefinitionsSame(*info_db.create_query, *create_db_query))
info_db.different_create_query = create_db_query;
}
}
@ -314,13 +388,13 @@ namespace
/// Check that we are not trying to restore the same database again.
String new_database_name = renaming_settings.getNewDatabaseName(database_name_);
if (databases.contains(new_database_name) && databases[new_database_name].is_explicit)
throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, "Couldn't restore database {} twice", backQuoteIfNeed(new_database_name));
throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, "Cannot restore the database {} twice", backQuoteIfNeed(new_database_name));
Strings table_metadata_filenames = backup->listFiles("metadata/" + escapeForFileName(database_name_) + "/", "/");
bool throw_if_no_create_database_query = table_metadata_filenames.empty();
if (throw_if_no_create_database_query && !hasCreateQueryInBackup(database_name_))
throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, "Could not restore database {} because there is no such database in the backup", backQuoteIfNeed(database_name_));
throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, "Cannot restore the database {} because there is no such database in the backup", backQuoteIfNeed(database_name_));
/// Of course we're not going to restore the definition of the system or the temporary database.
if (!isSystemOrTemporaryDatabase(new_database_name))
@ -376,7 +450,8 @@ namespace
{
String create_query_path = getMetadataPathInBackup(table_name);
if (!backup->fileExists(create_query_path))
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Could not restore table {}.{} because there is no such table in the backup", backQuoteIfNeed(table_name.first), backQuoteIfNeed(table_name.second));
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore the {} because there is no such table in the backup",
formatTableNameOrTemporaryTableName(table_name));
auto read_buffer = backup->readFile(create_query_path)->getReadBuffer();
String create_query_str;
readStringUntilEOF(create_query_str, *read_buffer);
@ -390,7 +465,7 @@ namespace
{
String create_query_path = getMetadataPathInBackup(database_name);
if (!backup->fileExists(create_query_path))
throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, "Could not restore database {} because there is no such database in the backup", backQuoteIfNeed(database_name));
throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE, "Cannot restore the database {} because there is no such database in the backup", backQuoteIfNeed(database_name));
auto read_buffer = backup->readFile(create_query_path)->getReadBuffer();
String create_query_str;
readStringUntilEOF(create_query_str, *read_buffer);

View File

@ -0,0 +1,17 @@
#include <Backups/formatTableNameOrTemporaryTableName.h>
#include <Common/quoteString.h>
#include <Interpreters/DatabaseCatalog.h>
namespace DB
{
String formatTableNameOrTemporaryTableName(const DatabaseAndTableName & table_name)
{
if (table_name.first == DatabaseCatalog::TEMPORARY_DATABASE)
return "temporary table " + backQuoteIfNeed(table_name.second);
else
return "table " + backQuoteIfNeed(table_name.first) + "." + backQuoteIfNeed(table_name.second);
}
}

View File

@ -0,0 +1,13 @@
#pragma once
#include <base/types.h>
namespace DB
{
using DatabaseAndTableName = std::pair<String, String>;
/// Outputs either "table db_name.table_name" or "temporary table table_name".
String formatTableNameOrTemporaryTableName(const DatabaseAndTableName & table_name);
}

View File

@ -1,26 +0,0 @@
#include <Backups/hasCompatibleDataToRestoreTable.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/formatAST.h>
namespace DB
{
bool hasCompatibleDataToRestoreTable(const ASTCreateQuery & query1, const ASTCreateQuery & query2)
{
/// TODO: Write more subtle condition here.
auto q1 = typeid_cast<std::shared_ptr<ASTCreateQuery>>(query1.clone());
auto q2 = typeid_cast<std::shared_ptr<ASTCreateQuery>>(query2.clone());
/// Remove UUIDs.
q1->uuid = UUIDHelpers::Nil;
q2->uuid = UUIDHelpers::Nil;
/// Clear IF NOT EXISTS flag.
q1->if_not_exists = false;
q2->if_not_exists = false;
return serializeAST(*q1) == serializeAST(*q2);
}
}

View File

@ -1,11 +0,0 @@
#pragma once
namespace DB
{
class ASTCreateQuery;
/// Whether the data of the first table can be inserted to the second table.
bool hasCompatibleDataToRestoreTable(const ASTCreateQuery & query1, const ASTCreateQuery & query2);
}