2022-05-31 09:33:23 +00:00
|
|
|
#include <Backups/RestorerFromBackup.h>
|
|
|
|
#include <Backups/IRestoreCoordination.h>
|
|
|
|
#include <Backups/BackupSettings.h>
|
|
|
|
#include <Backups/IBackup.h>
|
|
|
|
#include <Backups/IBackupEntry.h>
|
2022-06-08 02:11:41 +00:00
|
|
|
#include <Backups/BackupUtils.h>
|
2022-06-15 18:25:13 +00:00
|
|
|
#include <Access/AccessBackup.h>
|
2022-06-18 22:01:08 +00:00
|
|
|
#include <Access/AccessRights.h>
|
2022-05-31 09:33:23 +00:00
|
|
|
#include <Parsers/ParserCreateQuery.h>
|
|
|
|
#include <Parsers/parseQuery.h>
|
|
|
|
#include <Parsers/formatAST.h>
|
|
|
|
#include <Parsers/ASTCreateQuery.h>
|
2022-06-09 16:19:54 +00:00
|
|
|
#include <Parsers/ASTFunction.h>
|
2022-05-31 09:33:23 +00:00
|
|
|
#include <Interpreters/DatabaseCatalog.h>
|
|
|
|
#include <Interpreters/Context.h>
|
|
|
|
#include <Interpreters/InterpreterCreateQuery.h>
|
|
|
|
#include <Databases/IDatabase.h>
|
2022-06-06 19:15:17 +00:00
|
|
|
#include <Databases/DDLDependencyVisitor.h>
|
2022-05-31 09:33:23 +00:00
|
|
|
#include <Storages/IStorage.h>
|
|
|
|
#include <Common/escapeForFileName.h>
|
|
|
|
#include <Common/quoteString.h>
|
|
|
|
#include <base/insertAtEnd.h>
|
2022-06-06 09:50:20 +00:00
|
|
|
#include <boost/algorithm/string/join.hpp>
|
|
|
|
#include <filesystem>
|
|
|
|
|
|
|
|
namespace fs = std::filesystem;
|
2022-05-31 09:33:23 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int BACKUP_ENTRY_NOT_FOUND;
|
|
|
|
extern const int CANNOT_RESTORE_TABLE;
|
|
|
|
extern const int CANNOT_RESTORE_DATABASE;
|
2022-06-19 13:48:52 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
2022-05-31 09:33:23 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2022-06-09 16:19:54 +00:00
|
|
|
namespace
|
|
|
|
{
|
2022-06-18 11:56:04 +00:00
|
|
|
constexpr const std::string_view sql_ext = ".sql";
|
|
|
|
|
2022-06-18 22:01:08 +00:00
|
|
|
String tryGetTableEngine(const IAST & ast)
|
2022-06-09 16:19:54 +00:00
|
|
|
{
|
|
|
|
const ASTCreateQuery * create = ast.as<ASTCreateQuery>();
|
|
|
|
if (!create)
|
2022-06-18 22:01:08 +00:00
|
|
|
return {};
|
2022-06-09 16:19:54 +00:00
|
|
|
if (!create->storage || !create->storage->engine)
|
2022-06-18 22:01:08 +00:00
|
|
|
return {};
|
|
|
|
return create->storage->engine->name;
|
|
|
|
}
|
|
|
|
|
|
|
|
bool hasSystemTableEngine(const IAST & ast)
|
|
|
|
{
|
|
|
|
return tryGetTableEngine(ast).starts_with("System");
|
|
|
|
}
|
|
|
|
|
|
|
|
bool hasSystemAccessTableEngine(const IAST & ast)
|
|
|
|
{
|
|
|
|
String engine_name = tryGetTableEngine(ast);
|
|
|
|
return (engine_name == "SystemUsers") || (engine_name == "SystemRoles") || (engine_name == "SystemSettingsProfiles")
|
|
|
|
|| (engine_name == "SystemRowPolicies") || (engine_name == "SystemQuotas");
|
2022-06-09 16:19:54 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
bool RestorerFromBackup::TableKey::operator ==(const TableKey & right) const
|
|
|
|
{
|
|
|
|
return (name == right.name) && (is_temporary == right.is_temporary);
|
|
|
|
}
|
|
|
|
|
|
|
|
bool RestorerFromBackup::TableKey::operator <(const TableKey & right) const
|
|
|
|
{
|
|
|
|
return (name < right.name) || ((name == right.name) && (is_temporary < right.is_temporary));
|
|
|
|
}
|
|
|
|
|
|
|
|
std::string_view RestorerFromBackup::toString(Stage stage)
|
|
|
|
{
|
|
|
|
switch (stage)
|
|
|
|
{
|
|
|
|
case Stage::kPreparing: return "Preparing";
|
|
|
|
case Stage::kFindingTablesInBackup: return "Finding tables in backup";
|
|
|
|
case Stage::kCreatingDatabases: return "Creating databases";
|
|
|
|
case Stage::kCreatingTables: return "Creating tables";
|
|
|
|
case Stage::kInsertingDataToTables: return "Inserting data to tables";
|
|
|
|
case Stage::kError: return "Error";
|
|
|
|
}
|
2022-06-19 13:48:52 +00:00
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown restore stage: {}", static_cast<int>(stage));
|
2022-06-09 16:19:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2022-05-31 09:33:23 +00:00
|
|
|
RestorerFromBackup::RestorerFromBackup(
|
|
|
|
const ASTBackupQuery::Elements & restore_query_elements_,
|
|
|
|
const RestoreSettings & restore_settings_,
|
|
|
|
std::shared_ptr<IRestoreCoordination> restore_coordination_,
|
|
|
|
const BackupPtr & backup_,
|
|
|
|
const ContextMutablePtr & context_,
|
|
|
|
std::chrono::seconds timeout_)
|
|
|
|
: restore_query_elements(restore_query_elements_)
|
|
|
|
, restore_settings(restore_settings_)
|
|
|
|
, restore_coordination(restore_coordination_)
|
|
|
|
, backup(backup_)
|
|
|
|
, context(context_)
|
|
|
|
, timeout(timeout_)
|
2022-06-22 22:56:41 +00:00
|
|
|
, create_table_timeout_ms(context->getConfigRef().getUInt64("backups.create_table_timeout", 300000))
|
2022-05-31 09:33:23 +00:00
|
|
|
, log(&Poco::Logger::get("RestorerFromBackup"))
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
RestorerFromBackup::~RestorerFromBackup() = default;
|
|
|
|
|
|
|
|
void RestorerFromBackup::restoreMetadata()
|
2022-06-18 22:01:08 +00:00
|
|
|
{
|
|
|
|
run(/* only_check_access= */ false);
|
|
|
|
}
|
|
|
|
|
|
|
|
void RestorerFromBackup::checkAccessOnly()
|
|
|
|
{
|
|
|
|
run(/* only_check_access= */ true);
|
|
|
|
}
|
|
|
|
|
|
|
|
void RestorerFromBackup::run(bool only_check_access)
|
2022-05-31 09:33:23 +00:00
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
|
|
|
/// restoreMetadata() must not be called multiple times.
|
|
|
|
if (current_stage != Stage::kPreparing)
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Already restoring");
|
|
|
|
|
|
|
|
/// Calculate the root path in the backup for restoring, it's either empty or has the format "shards/<shard_num>/replicas/<replica_num>/".
|
|
|
|
findRootPathsInBackup();
|
|
|
|
|
|
|
|
/// Do renaming in the create queries according to the renaming config.
|
2022-06-08 02:11:41 +00:00
|
|
|
renaming_map = makeRenamingMapFromBackupQuery(restore_query_elements);
|
2022-05-31 09:33:23 +00:00
|
|
|
|
|
|
|
/// Find all the databases and tables which we will read from the backup.
|
|
|
|
setStage(Stage::kFindingTablesInBackup);
|
|
|
|
collectDatabaseAndTableInfos();
|
2022-06-19 13:48:52 +00:00
|
|
|
|
2022-06-18 22:01:08 +00:00
|
|
|
/// Check access rights.
|
|
|
|
checkAccessForCollectedInfos();
|
|
|
|
if (only_check_access)
|
|
|
|
return;
|
2022-05-31 09:33:23 +00:00
|
|
|
|
|
|
|
/// Create databases using the create queries read from the backup.
|
|
|
|
setStage(Stage::kCreatingDatabases);
|
|
|
|
createDatabases();
|
|
|
|
|
|
|
|
/// Create tables using the create queries read from the backup.
|
|
|
|
setStage(Stage::kCreatingTables);
|
|
|
|
createTables();
|
|
|
|
|
|
|
|
/// All what's left is to insert data to tables.
|
|
|
|
/// No more data restoring tasks are allowed after this point.
|
|
|
|
setStage(Stage::kInsertingDataToTables);
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
|
|
|
/// Other hosts should know that we've encountered an error.
|
|
|
|
setStage(Stage::kError, getCurrentExceptionMessage(false));
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
}
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-06-18 22:01:08 +00:00
|
|
|
|
2022-05-31 09:33:23 +00:00
|
|
|
RestorerFromBackup::DataRestoreTasks RestorerFromBackup::getDataRestoreTasks()
|
|
|
|
{
|
|
|
|
if (current_stage != Stage::kInsertingDataToTables)
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Metadata wasn't restored");
|
|
|
|
|
2022-06-18 22:01:08 +00:00
|
|
|
if (data_restore_tasks.empty() && !access_restore_task)
|
2022-06-09 16:19:54 +00:00
|
|
|
return {};
|
|
|
|
|
|
|
|
LOG_TRACE(log, "Will insert data to tables");
|
|
|
|
|
2022-05-31 09:33:23 +00:00
|
|
|
/// Storages and table locks must exist while we're executing data restoring tasks.
|
2022-06-09 16:19:54 +00:00
|
|
|
auto storages = std::make_shared<std::vector<StoragePtr>>();
|
|
|
|
auto table_locks = std::make_shared<std::vector<TableLockHolder>>();
|
|
|
|
storages->reserve(table_infos.size());
|
|
|
|
table_locks->reserve(table_infos.size());
|
|
|
|
for (const auto & table_info : table_infos | boost::adaptors::map_values)
|
2022-05-31 09:33:23 +00:00
|
|
|
{
|
2022-06-09 16:19:54 +00:00
|
|
|
storages->push_back(table_info.storage);
|
|
|
|
table_locks->push_back(table_info.table_lock);
|
2022-05-31 09:33:23 +00:00
|
|
|
}
|
|
|
|
|
2022-06-09 16:19:54 +00:00
|
|
|
DataRestoreTasks res_tasks;
|
|
|
|
for (const auto & task : data_restore_tasks)
|
|
|
|
res_tasks.push_back([task, storages, table_locks] { task(); });
|
|
|
|
|
2022-06-18 22:01:08 +00:00
|
|
|
if (access_restore_task)
|
|
|
|
res_tasks.push_back([task = access_restore_task, access_control = &context->getAccessControl()] { task->restore(*access_control); });
|
|
|
|
|
2022-05-31 09:33:23 +00:00
|
|
|
return res_tasks;
|
|
|
|
}
|
|
|
|
|
|
|
|
void RestorerFromBackup::setStage(Stage new_stage, const String & error_message)
|
|
|
|
{
|
|
|
|
if (new_stage == Stage::kError)
|
|
|
|
LOG_ERROR(log, "{} failed with error: {}", toString(current_stage), error_message);
|
|
|
|
else
|
|
|
|
LOG_TRACE(log, "{}", toString(new_stage));
|
2022-06-19 13:48:52 +00:00
|
|
|
|
2022-05-31 09:33:23 +00:00
|
|
|
current_stage = new_stage;
|
2022-06-18 22:01:08 +00:00
|
|
|
|
|
|
|
if (!restore_coordination)
|
|
|
|
return;
|
2022-06-19 13:48:52 +00:00
|
|
|
|
2022-05-31 09:33:23 +00:00
|
|
|
if (new_stage == Stage::kError)
|
|
|
|
{
|
|
|
|
restore_coordination->syncStageError(restore_settings.host_id, error_message);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
auto all_hosts
|
|
|
|
= BackupSettings::Util::filterHostIDs(restore_settings.cluster_host_ids, restore_settings.shard_num, restore_settings.replica_num);
|
|
|
|
restore_coordination->syncStage(restore_settings.host_id, static_cast<int>(new_stage), all_hosts, timeout);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void RestorerFromBackup::findRootPathsInBackup()
|
|
|
|
{
|
|
|
|
size_t shard_num = 1;
|
|
|
|
size_t replica_num = 1;
|
|
|
|
if (!restore_settings.host_id.empty())
|
|
|
|
{
|
|
|
|
std::tie(shard_num, replica_num)
|
|
|
|
= BackupSettings::Util::findShardNumAndReplicaNum(restore_settings.cluster_host_ids, restore_settings.host_id);
|
|
|
|
}
|
2022-06-19 13:48:52 +00:00
|
|
|
|
2022-05-31 09:33:23 +00:00
|
|
|
root_paths_in_backup.clear();
|
|
|
|
|
|
|
|
/// Start with "" as the root path and then we will add shard- and replica-related part to it.
|
2022-06-06 09:50:20 +00:00
|
|
|
fs::path root_path = "/";
|
2022-05-31 09:33:23 +00:00
|
|
|
root_paths_in_backup.push_back(root_path);
|
|
|
|
|
|
|
|
/// Add shard-related part to the root path.
|
2022-06-06 09:50:20 +00:00
|
|
|
Strings shards_in_backup = backup->listFiles(root_path / "shards");
|
2022-05-31 09:33:23 +00:00
|
|
|
if (shards_in_backup.empty())
|
|
|
|
{
|
|
|
|
if (restore_settings.shard_num_in_backup > 1)
|
|
|
|
throw Exception(ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "No shard #{} in backup", restore_settings.shard_num_in_backup);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
String shard_name;
|
|
|
|
if (restore_settings.shard_num_in_backup)
|
|
|
|
shard_name = std::to_string(restore_settings.shard_num_in_backup);
|
|
|
|
else if (shards_in_backup.size() == 1)
|
|
|
|
shard_name = shards_in_backup.front();
|
|
|
|
else
|
|
|
|
shard_name = std::to_string(shard_num);
|
|
|
|
if (std::find(shards_in_backup.begin(), shards_in_backup.end(), shard_name) == shards_in_backup.end())
|
|
|
|
throw Exception(ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "No shard #{} in backup", shard_name);
|
2022-06-06 09:50:20 +00:00
|
|
|
root_path = root_path / "shards" / shard_name;
|
2022-05-31 09:33:23 +00:00
|
|
|
root_paths_in_backup.push_back(root_path);
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Add replica-related part to the root path.
|
2022-06-06 09:50:20 +00:00
|
|
|
Strings replicas_in_backup = backup->listFiles(root_path / "replicas");
|
2022-05-31 09:33:23 +00:00
|
|
|
if (replicas_in_backup.empty())
|
|
|
|
{
|
|
|
|
if (restore_settings.replica_num_in_backup > 1)
|
|
|
|
throw Exception(ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "No replica #{} in backup", restore_settings.replica_num_in_backup);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
String replica_name;
|
|
|
|
if (restore_settings.replica_num_in_backup)
|
|
|
|
{
|
|
|
|
replica_name = std::to_string(restore_settings.replica_num_in_backup);
|
|
|
|
if (std::find(replicas_in_backup.begin(), replicas_in_backup.end(), replica_name) == replicas_in_backup.end())
|
|
|
|
throw Exception(ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "No replica #{} in backup", replica_name);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
replica_name = std::to_string(replica_num);
|
|
|
|
if (std::find(replicas_in_backup.begin(), replicas_in_backup.end(), replica_name) == replicas_in_backup.end())
|
|
|
|
replica_name = replicas_in_backup.front();
|
|
|
|
}
|
2022-06-06 09:50:20 +00:00
|
|
|
root_path = root_path / "replicas" / replica_name;
|
2022-05-31 09:33:23 +00:00
|
|
|
root_paths_in_backup.push_back(root_path);
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Revert the list of root paths, because we need it in the following order:
|
2022-06-06 09:50:20 +00:00
|
|
|
/// "/shards/<shard_num>/replicas/<replica_num>/" (first we search tables here)
|
|
|
|
/// "/shards/<shard_num>/" (then here)
|
|
|
|
/// "/" (and finally here)
|
2022-05-31 09:33:23 +00:00
|
|
|
std::reverse(root_paths_in_backup.begin(), root_paths_in_backup.end());
|
|
|
|
|
2022-06-06 09:50:20 +00:00
|
|
|
LOG_TRACE(
|
|
|
|
log,
|
|
|
|
"Will use paths in backup: {}",
|
|
|
|
boost::algorithm::join(
|
|
|
|
root_paths_in_backup
|
|
|
|
| boost::adaptors::transformed([](const fs::path & path) -> String { return doubleQuoteString(String{path}); }),
|
|
|
|
", "));
|
2022-05-31 09:33:23 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void RestorerFromBackup::collectDatabaseAndTableInfos()
|
|
|
|
{
|
|
|
|
database_infos.clear();
|
|
|
|
table_infos.clear();
|
|
|
|
for (const auto & element : restore_query_elements)
|
|
|
|
{
|
|
|
|
switch (element.type)
|
|
|
|
{
|
|
|
|
case ASTBackupQuery::ElementType::TABLE:
|
|
|
|
{
|
2022-06-11 11:36:02 +00:00
|
|
|
collectTableInfo({element.database_name, element.table_name}, false, element.partitions);
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
case ASTBackupQuery::ElementType::TEMPORARY_TABLE:
|
|
|
|
{
|
|
|
|
collectTableInfo({element.database_name, element.table_name}, true, element.partitions);
|
2022-05-31 09:33:23 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
case ASTBackupQuery::ElementType::DATABASE:
|
|
|
|
{
|
2022-06-18 11:56:04 +00:00
|
|
|
collectDatabaseInfo(element.database_name, element.except_tables, /* throw_if_no_database_metadata_in_backup= */ true);
|
2022-05-31 09:33:23 +00:00
|
|
|
break;
|
|
|
|
}
|
2022-06-11 11:36:02 +00:00
|
|
|
case ASTBackupQuery::ElementType::ALL:
|
2022-05-31 09:33:23 +00:00
|
|
|
{
|
2022-06-11 11:36:02 +00:00
|
|
|
collectAllDatabasesInfo(element.except_databases, element.except_tables);
|
2022-05-31 09:33:23 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
LOG_INFO(log, "Will restore {} databases and {} tables", database_infos.size(), table_infos.size());
|
|
|
|
}
|
|
|
|
|
2022-06-09 16:19:54 +00:00
|
|
|
void RestorerFromBackup::collectTableInfo(const QualifiedTableName & table_name_in_backup, bool is_temporary_table, const std::optional<ASTs> & partitions)
|
2022-05-31 09:33:23 +00:00
|
|
|
{
|
2022-06-09 16:19:54 +00:00
|
|
|
String database_name_in_backup = is_temporary_table ? DatabaseCatalog::TEMPORARY_DATABASE : table_name_in_backup.database;
|
|
|
|
|
2022-06-06 09:50:20 +00:00
|
|
|
std::optional<fs::path> metadata_path;
|
|
|
|
std::optional<fs::path> root_path_in_use;
|
|
|
|
for (const auto & root_path_in_backup : root_paths_in_backup)
|
2022-05-31 09:33:23 +00:00
|
|
|
{
|
2022-06-09 16:19:54 +00:00
|
|
|
fs::path try_metadata_path;
|
|
|
|
if (is_temporary_table)
|
|
|
|
{
|
|
|
|
try_metadata_path
|
|
|
|
= root_path_in_backup / "temporary_tables" / "metadata" / (escapeForFileName(table_name_in_backup.table) + ".sql");
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
try_metadata_path = root_path_in_backup / "metadata" / escapeForFileName(table_name_in_backup.database)
|
|
|
|
/ (escapeForFileName(table_name_in_backup.table) + ".sql");
|
|
|
|
}
|
|
|
|
|
2022-05-31 09:33:23 +00:00
|
|
|
if (backup->fileExists(try_metadata_path))
|
|
|
|
{
|
|
|
|
metadata_path = try_metadata_path;
|
2022-06-06 09:50:20 +00:00
|
|
|
root_path_in_use = root_path_in_backup;
|
2022-05-31 09:33:23 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!metadata_path)
|
2022-06-06 13:52:34 +00:00
|
|
|
throw Exception(ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "Table {} not found in backup", table_name_in_backup.getFullName());
|
2022-05-31 09:33:23 +00:00
|
|
|
|
2022-06-09 16:19:54 +00:00
|
|
|
TableKey table_key;
|
|
|
|
fs::path data_path_in_backup;
|
|
|
|
if (is_temporary_table)
|
|
|
|
{
|
|
|
|
data_path_in_backup = *root_path_in_use / "temporary_tables" / "data" / escapeForFileName(table_name_in_backup.table);
|
|
|
|
table_key.name.table = renaming_map.getNewTemporaryTableName(table_name_in_backup.table);
|
|
|
|
table_key.is_temporary = true;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
data_path_in_backup
|
|
|
|
= *root_path_in_use / "data" / escapeForFileName(table_name_in_backup.database) / escapeForFileName(table_name_in_backup.table);
|
|
|
|
table_key.name = renaming_map.getNewTableName(table_name_in_backup);
|
|
|
|
}
|
2022-05-31 09:33:23 +00:00
|
|
|
|
|
|
|
auto read_buffer = backup->readFile(*metadata_path)->getReadBuffer();
|
|
|
|
String create_query_str;
|
|
|
|
readStringUntilEOF(create_query_str, *read_buffer);
|
|
|
|
read_buffer.reset();
|
|
|
|
ParserCreateQuery create_parser;
|
|
|
|
ASTPtr create_table_query = parseQuery(create_parser, create_query_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
2022-06-08 02:11:41 +00:00
|
|
|
renameDatabaseAndTableNameInCreateQuery(context->getGlobalContext(), renaming_map, create_table_query);
|
2022-05-31 09:33:23 +00:00
|
|
|
|
2022-06-09 16:19:54 +00:00
|
|
|
if (auto it = table_infos.find(table_key); it != table_infos.end())
|
2022-05-31 09:33:23 +00:00
|
|
|
{
|
|
|
|
const TableInfo & table_info = it->second;
|
|
|
|
if (table_info.create_table_query && (serializeAST(*table_info.create_table_query) != serializeAST(*create_table_query)))
|
|
|
|
{
|
|
|
|
throw Exception(
|
|
|
|
ErrorCodes::CANNOT_RESTORE_TABLE,
|
2022-06-09 16:19:54 +00:00
|
|
|
"Extracted two different create queries for the same {}table {}: {} and {}",
|
|
|
|
(is_temporary_table ? "temporary " : ""),
|
|
|
|
table_key.name.getFullName(),
|
2022-05-31 09:33:23 +00:00
|
|
|
serializeAST(*table_info.create_table_query),
|
|
|
|
serializeAST(*create_table_query));
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-06-09 16:19:54 +00:00
|
|
|
TableInfo & res_table_info = table_infos[table_key];
|
2022-05-31 09:33:23 +00:00
|
|
|
res_table_info.create_table_query = create_table_query;
|
|
|
|
res_table_info.data_path_in_backup = data_path_in_backup;
|
2022-06-09 16:19:54 +00:00
|
|
|
res_table_info.dependencies = getDependenciesSetFromCreateQuery(context->getGlobalContext(), table_key.name, create_table_query);
|
2022-05-31 09:33:23 +00:00
|
|
|
|
|
|
|
if (partitions)
|
|
|
|
{
|
|
|
|
if (!res_table_info.partitions)
|
|
|
|
res_table_info.partitions.emplace();
|
|
|
|
insertAtEnd(*res_table_info.partitions, *partitions);
|
|
|
|
}
|
2022-06-18 22:01:08 +00:00
|
|
|
|
|
|
|
if (hasSystemAccessTableEngine(*create_table_query))
|
|
|
|
{
|
|
|
|
if (!access_restore_task)
|
|
|
|
access_restore_task = std::make_shared<AccessRestoreTask>(backup, restore_settings, restore_coordination);
|
|
|
|
access_restore_task->addDataPath(data_path_in_backup);
|
|
|
|
}
|
2022-05-31 09:33:23 +00:00
|
|
|
}
|
|
|
|
|
2022-06-18 11:56:04 +00:00
|
|
|
void RestorerFromBackup::collectDatabaseInfo(const String & database_name_in_backup, const std::set<DatabaseAndTableName> & except_table_names, bool throw_if_no_database_metadata_in_backup)
|
2022-05-31 09:33:23 +00:00
|
|
|
{
|
2022-06-06 09:50:20 +00:00
|
|
|
std::optional<fs::path> metadata_path;
|
2022-05-31 09:33:23 +00:00
|
|
|
std::unordered_set<String> table_names_in_backup;
|
2022-06-06 09:50:20 +00:00
|
|
|
for (const auto & root_path_in_backup : root_paths_in_backup)
|
2022-05-31 09:33:23 +00:00
|
|
|
{
|
2022-06-06 09:50:20 +00:00
|
|
|
fs::path try_metadata_path = root_path_in_backup / "metadata" / (escapeForFileName(database_name_in_backup) + ".sql");
|
2022-05-31 09:33:23 +00:00
|
|
|
if (!metadata_path && backup->fileExists(try_metadata_path))
|
|
|
|
metadata_path = try_metadata_path;
|
|
|
|
|
2022-06-06 09:50:20 +00:00
|
|
|
Strings file_names = backup->listFiles(root_path_in_backup / "metadata" / escapeForFileName(database_name_in_backup));
|
2022-05-31 09:33:23 +00:00
|
|
|
for (const String & file_name : file_names)
|
|
|
|
{
|
|
|
|
if (!file_name.ends_with(sql_ext))
|
|
|
|
continue;
|
|
|
|
String file_name_without_ext = file_name.substr(0, file_name.length() - sql_ext.length());
|
|
|
|
table_names_in_backup.insert(unescapeForFileName(file_name_without_ext));
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-06-18 11:56:04 +00:00
|
|
|
if (!metadata_path && throw_if_no_database_metadata_in_backup)
|
2022-05-31 09:33:23 +00:00
|
|
|
throw Exception(ErrorCodes::BACKUP_ENTRY_NOT_FOUND, "Database {} not found in backup", backQuoteIfNeed(database_name_in_backup));
|
|
|
|
|
|
|
|
if (metadata_path)
|
|
|
|
{
|
|
|
|
auto read_buffer = backup->readFile(*metadata_path)->getReadBuffer();
|
|
|
|
String create_query_str;
|
|
|
|
readStringUntilEOF(create_query_str, *read_buffer);
|
|
|
|
read_buffer.reset();
|
|
|
|
ParserCreateQuery create_parser;
|
2022-06-18 11:56:04 +00:00
|
|
|
ASTPtr create_database_query = parseQuery(create_parser, create_query_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
2022-06-08 02:11:41 +00:00
|
|
|
renameDatabaseAndTableNameInCreateQuery(context->getGlobalContext(), renaming_map, create_database_query);
|
2022-05-31 09:33:23 +00:00
|
|
|
|
2022-06-18 11:56:04 +00:00
|
|
|
String database_name = renaming_map.getNewDatabaseName(database_name_in_backup);
|
|
|
|
DatabaseInfo & database_info = database_infos[database_name];
|
2022-06-19 13:48:52 +00:00
|
|
|
|
2022-06-18 11:56:04 +00:00
|
|
|
if (database_info.create_database_query && (serializeAST(*database_info.create_database_query) != serializeAST(*create_database_query)))
|
|
|
|
{
|
|
|
|
throw Exception(
|
|
|
|
ErrorCodes::CANNOT_RESTORE_DATABASE,
|
|
|
|
"Extracted two different create queries for the same database {}: {} and {}",
|
|
|
|
backQuoteIfNeed(database_name),
|
|
|
|
serializeAST(*database_info.create_database_query),
|
|
|
|
serializeAST(*create_database_query));
|
|
|
|
}
|
2022-05-31 09:33:23 +00:00
|
|
|
|
2022-06-18 11:56:04 +00:00
|
|
|
database_info.create_database_query = create_database_query;
|
|
|
|
}
|
2022-05-31 09:33:23 +00:00
|
|
|
|
|
|
|
for (const String & table_name_in_backup : table_names_in_backup)
|
|
|
|
{
|
2022-06-11 11:36:02 +00:00
|
|
|
if (except_table_names.contains({database_name_in_backup, table_name_in_backup}))
|
2022-05-31 09:33:23 +00:00
|
|
|
continue;
|
|
|
|
|
2022-06-18 11:56:04 +00:00
|
|
|
collectTableInfo({database_name_in_backup, table_name_in_backup}, /* is_temporary_table= */ false, /* partitions= */ {});
|
2022-05-31 09:33:23 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-06-11 11:36:02 +00:00
|
|
|
void RestorerFromBackup::collectAllDatabasesInfo(const std::set<String> & except_database_names, const std::set<DatabaseAndTableName> & except_table_names)
|
2022-05-31 09:33:23 +00:00
|
|
|
{
|
|
|
|
std::unordered_set<String> database_names_in_backup;
|
2022-06-18 11:56:04 +00:00
|
|
|
std::unordered_set<String> temporary_table_names_in_backup;
|
|
|
|
|
2022-06-06 09:50:20 +00:00
|
|
|
for (const auto & root_path_in_backup : root_paths_in_backup)
|
2022-05-31 09:33:23 +00:00
|
|
|
{
|
2022-06-06 09:50:20 +00:00
|
|
|
Strings file_names = backup->listFiles(root_path_in_backup / "metadata");
|
2022-05-31 09:33:23 +00:00
|
|
|
for (String & file_name : file_names)
|
|
|
|
{
|
|
|
|
if (file_name.ends_with(sql_ext))
|
|
|
|
file_name.resize(file_name.length() - sql_ext.length());
|
|
|
|
database_names_in_backup.emplace(unescapeForFileName(file_name));
|
|
|
|
}
|
2022-06-18 11:56:04 +00:00
|
|
|
|
|
|
|
file_names = backup->listFiles(root_path_in_backup / "temporary_tables" / "metadata");
|
|
|
|
for (String & file_name : file_names)
|
|
|
|
{
|
|
|
|
if (!file_name.ends_with(sql_ext))
|
|
|
|
continue;
|
|
|
|
file_name.resize(file_name.length() - sql_ext.length());
|
|
|
|
temporary_table_names_in_backup.emplace(unescapeForFileName(file_name));
|
|
|
|
}
|
2022-05-31 09:33:23 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
for (const String & database_name_in_backup : database_names_in_backup)
|
|
|
|
{
|
|
|
|
if (except_database_names.contains(database_name_in_backup))
|
|
|
|
continue;
|
|
|
|
|
2022-06-18 11:56:04 +00:00
|
|
|
collectDatabaseInfo(database_name_in_backup, except_table_names, /* throw_if_no_database_metadata_in_backup= */ false);
|
2022-05-31 09:33:23 +00:00
|
|
|
}
|
2022-06-18 11:56:04 +00:00
|
|
|
|
|
|
|
for (const String & temporary_table_name_in_backup : temporary_table_names_in_backup)
|
|
|
|
collectTableInfo({"", temporary_table_name_in_backup}, /* is_temporary_table= */ true, /* partitions= */ {});
|
2022-05-31 09:33:23 +00:00
|
|
|
}
|
|
|
|
|
2022-06-18 22:01:08 +00:00
|
|
|
void RestorerFromBackup::checkAccessForCollectedInfos() const
|
|
|
|
{
|
|
|
|
AccessRightsElements required_access;
|
|
|
|
for (const auto & database_name : database_infos | boost::adaptors::map_keys)
|
|
|
|
{
|
|
|
|
if (DatabaseCatalog::isPredefinedDatabaseName(database_name))
|
|
|
|
continue;
|
|
|
|
|
|
|
|
AccessFlags flags;
|
|
|
|
|
|
|
|
if (restore_settings.create_database != RestoreDatabaseCreationMode::kMustExist)
|
|
|
|
flags |= AccessType::CREATE_DATABASE;
|
|
|
|
|
|
|
|
if (!flags)
|
|
|
|
flags = AccessType::SHOW_DATABASES;
|
|
|
|
|
|
|
|
required_access.emplace_back(flags, database_name);
|
|
|
|
}
|
|
|
|
|
|
|
|
for (const auto & [table_name, table_info] : table_infos)
|
|
|
|
{
|
|
|
|
if (hasSystemTableEngine(*table_info.create_table_query))
|
|
|
|
continue;
|
|
|
|
|
|
|
|
if (table_name.is_temporary)
|
|
|
|
{
|
|
|
|
if (restore_settings.create_table != RestoreTableCreationMode::kMustExist)
|
|
|
|
required_access.emplace_back(AccessType::CREATE_TEMPORARY_TABLE);
|
|
|
|
continue;
|
|
|
|
}
|
|
|
|
|
|
|
|
AccessFlags flags;
|
|
|
|
const ASTCreateQuery & create = table_info.create_table_query->as<const ASTCreateQuery &>();
|
|
|
|
|
|
|
|
if (restore_settings.create_table != RestoreTableCreationMode::kMustExist)
|
|
|
|
{
|
|
|
|
if (create.is_dictionary)
|
|
|
|
flags |= AccessType::CREATE_DICTIONARY;
|
|
|
|
else if (create.is_ordinary_view || create.is_materialized_view || create.is_live_view)
|
|
|
|
flags |= AccessType::CREATE_VIEW;
|
|
|
|
else
|
|
|
|
flags |= AccessType::CREATE_TABLE;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!restore_settings.structure_only && !create.is_dictionary && !create.is_ordinary_view
|
|
|
|
&& backup->hasFiles(table_info.data_path_in_backup))
|
|
|
|
{
|
|
|
|
flags |= AccessType::INSERT;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!flags)
|
|
|
|
{
|
|
|
|
if (create.is_dictionary)
|
|
|
|
flags = AccessType::SHOW_DICTIONARIES;
|
|
|
|
else
|
|
|
|
flags = AccessType::SHOW_TABLES;
|
|
|
|
}
|
|
|
|
|
|
|
|
required_access.emplace_back(flags, table_name.name.database, table_name.name.table);
|
|
|
|
}
|
|
|
|
|
|
|
|
if (access_restore_task)
|
|
|
|
insertAtEnd(required_access, access_restore_task->getRequiredAccess());
|
|
|
|
|
|
|
|
/// We convert to AccessRights and back to check access rights in a predictable way
|
|
|
|
/// (some elements could be duplicated or not sorted).
|
|
|
|
required_access = AccessRights{required_access}.getElements();
|
|
|
|
|
|
|
|
context->checkAccess(required_access);
|
|
|
|
|
|
|
|
}
|
|
|
|
|
2022-05-31 09:33:23 +00:00
|
|
|
void RestorerFromBackup::createDatabases()
|
|
|
|
{
|
|
|
|
for (const auto & [database_name, database_info] : database_infos)
|
|
|
|
{
|
2022-06-09 16:19:54 +00:00
|
|
|
bool need_create_database = (restore_settings.create_database != RestoreDatabaseCreationMode::kMustExist);
|
|
|
|
if (need_create_database && DatabaseCatalog::isPredefinedDatabaseName(database_name))
|
|
|
|
need_create_database = false; /// Predefined databases always exist.
|
|
|
|
|
|
|
|
if (need_create_database)
|
2022-05-31 09:33:23 +00:00
|
|
|
{
|
|
|
|
/// Execute CREATE DATABASE query.
|
|
|
|
auto create_database_query = database_info.create_database_query;
|
|
|
|
if (restore_settings.create_table == RestoreTableCreationMode::kCreateIfNotExists)
|
|
|
|
{
|
|
|
|
create_database_query = create_database_query->clone();
|
|
|
|
create_database_query->as<ASTCreateQuery &>().if_not_exists = true;
|
|
|
|
}
|
2022-06-08 02:11:41 +00:00
|
|
|
LOG_TRACE(log, "Creating database {}: {}", backQuoteIfNeed(database_name), serializeAST(*create_database_query));
|
2022-05-31 09:33:23 +00:00
|
|
|
executeCreateQuery(create_database_query);
|
|
|
|
}
|
|
|
|
|
|
|
|
DatabasePtr database = DatabaseCatalog::instance().getDatabase(database_name);
|
|
|
|
|
|
|
|
if (!restore_settings.allow_different_database_def)
|
|
|
|
{
|
|
|
|
/// Check that the database's definition is the same as expected.
|
2022-06-14 08:53:22 +00:00
|
|
|
ASTPtr create_database_query = database->getCreateDatabaseQueryForBackup();
|
2022-05-31 09:33:23 +00:00
|
|
|
ASTPtr expected_create_query = database_info.create_database_query;
|
2022-06-08 02:11:41 +00:00
|
|
|
if (serializeAST(*create_database_query) != serializeAST(*expected_create_query))
|
2022-05-31 09:33:23 +00:00
|
|
|
{
|
|
|
|
throw Exception(
|
|
|
|
ErrorCodes::CANNOT_RESTORE_DATABASE,
|
|
|
|
"The database {} has a different definition: {} "
|
|
|
|
"comparing to its definition in the backup: {}",
|
|
|
|
backQuoteIfNeed(database_name),
|
2022-06-08 02:11:41 +00:00
|
|
|
serializeAST(*create_database_query),
|
2022-05-31 09:33:23 +00:00
|
|
|
serializeAST(*expected_create_query));
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void RestorerFromBackup::createTables()
|
|
|
|
{
|
2022-06-06 19:15:17 +00:00
|
|
|
while (true)
|
2022-05-31 09:33:23 +00:00
|
|
|
{
|
2022-06-06 19:15:17 +00:00
|
|
|
/// We need to create tables considering their dependencies.
|
|
|
|
auto tables_to_create = findTablesWithoutDependencies();
|
|
|
|
if (tables_to_create.empty())
|
|
|
|
break; /// We've already created all the tables.
|
|
|
|
|
2022-06-09 16:19:54 +00:00
|
|
|
for (const auto & table_key : tables_to_create)
|
2022-05-31 09:33:23 +00:00
|
|
|
{
|
2022-06-09 16:19:54 +00:00
|
|
|
auto & table_info = table_infos.at(table_key);
|
|
|
|
|
|
|
|
DatabasePtr database;
|
|
|
|
if (table_key.is_temporary)
|
|
|
|
database = DatabaseCatalog::instance().getDatabaseForTemporaryTables();
|
|
|
|
else
|
|
|
|
database = DatabaseCatalog::instance().getDatabase(table_key.name.database);
|
|
|
|
|
|
|
|
bool need_create_table = (restore_settings.create_table != RestoreTableCreationMode::kMustExist);
|
|
|
|
if (need_create_table && hasSystemTableEngine(*table_info.create_table_query))
|
|
|
|
need_create_table = false; /// Tables with System* table engine already exist or can't be created by SQL anyway.
|
|
|
|
|
|
|
|
if (need_create_table)
|
2022-06-06 19:15:17 +00:00
|
|
|
{
|
|
|
|
/// Execute CREATE TABLE query (we call IDatabase::createTableRestoredFromBackup() to allow the database to do some
|
|
|
|
/// database-specific things).
|
|
|
|
auto create_table_query = table_info.create_table_query;
|
|
|
|
if (restore_settings.create_table == RestoreTableCreationMode::kCreateIfNotExists)
|
|
|
|
{
|
|
|
|
create_table_query = create_table_query->clone();
|
|
|
|
create_table_query->as<ASTCreateQuery &>().if_not_exists = true;
|
|
|
|
}
|
2022-06-09 16:19:54 +00:00
|
|
|
LOG_TRACE(
|
|
|
|
log,
|
|
|
|
"Creating {}table {}: {}",
|
|
|
|
(table_key.is_temporary ? "temporary " : ""),
|
|
|
|
table_key.name.getFullName(),
|
|
|
|
serializeAST(*create_table_query));
|
|
|
|
|
2022-06-22 22:56:41 +00:00
|
|
|
database->createTableRestoredFromBackup(create_table_query, context, restore_coordination, create_table_timeout_ms);
|
2022-06-06 19:15:17 +00:00
|
|
|
}
|
2022-05-31 09:33:23 +00:00
|
|
|
|
2022-06-06 19:15:17 +00:00
|
|
|
table_info.created = true;
|
2022-06-09 16:19:54 +00:00
|
|
|
|
|
|
|
auto resolved_id = table_key.is_temporary
|
|
|
|
? context->resolveStorageID(StorageID{"", table_key.name.table}, Context::ResolveExternal)
|
|
|
|
: context->resolveStorageID(StorageID{table_key.name.database, table_key.name.table}, Context::ResolveGlobal);
|
|
|
|
|
|
|
|
auto storage = database->getTable(resolved_id.table_name, context);
|
|
|
|
table_info.storage = storage;
|
|
|
|
table_info.table_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout);
|
2022-06-06 19:15:17 +00:00
|
|
|
|
|
|
|
if (!restore_settings.allow_different_table_def)
|
2022-05-31 09:33:23 +00:00
|
|
|
{
|
2022-06-22 22:56:41 +00:00
|
|
|
ASTPtr create_table_query = database->getCreateTableQuery(resolved_id.table_name, context);
|
2022-06-23 10:17:54 +00:00
|
|
|
storage->adjustCreateQueryForBackup(create_table_query);
|
2022-06-06 19:15:17 +00:00
|
|
|
ASTPtr expected_create_query = table_info.create_table_query;
|
|
|
|
if (serializeAST(*create_table_query) != serializeAST(*expected_create_query))
|
|
|
|
{
|
|
|
|
throw Exception(
|
|
|
|
ErrorCodes::CANNOT_RESTORE_TABLE,
|
2022-06-09 16:19:54 +00:00
|
|
|
"The {}table {} has a different definition: {} "
|
2022-06-06 19:15:17 +00:00
|
|
|
"comparing to its definition in the backup: {}",
|
2022-06-09 16:19:54 +00:00
|
|
|
(table_key.is_temporary ? "temporary " : ""),
|
|
|
|
table_key.name.getFullName(),
|
2022-06-06 19:15:17 +00:00
|
|
|
serializeAST(*create_table_query),
|
|
|
|
serializeAST(*expected_create_query));
|
|
|
|
}
|
2022-05-31 09:33:23 +00:00
|
|
|
}
|
|
|
|
|
2022-06-06 19:15:17 +00:00
|
|
|
if (!restore_settings.structure_only)
|
2022-05-31 09:33:23 +00:00
|
|
|
{
|
2022-06-06 19:15:17 +00:00
|
|
|
const auto & data_path_in_backup = table_info.data_path_in_backup;
|
|
|
|
const auto & partitions = table_info.partitions;
|
|
|
|
storage->restoreDataFromBackup(*this, data_path_in_backup, partitions);
|
2022-05-31 09:33:23 +00:00
|
|
|
}
|
|
|
|
}
|
2022-06-06 19:15:17 +00:00
|
|
|
}
|
|
|
|
}
|
2022-05-31 09:33:23 +00:00
|
|
|
|
2022-06-06 19:15:17 +00:00
|
|
|
/// Returns the list of tables without dependencies or those which dependencies have been created before.
|
2022-06-09 16:19:54 +00:00
|
|
|
std::vector<RestorerFromBackup::TableKey> RestorerFromBackup::findTablesWithoutDependencies() const
|
2022-06-06 19:15:17 +00:00
|
|
|
{
|
2022-06-09 16:19:54 +00:00
|
|
|
std::vector<TableKey> tables_without_dependencies;
|
2022-06-06 19:15:17 +00:00
|
|
|
bool all_tables_created = true;
|
|
|
|
|
2022-06-09 16:19:54 +00:00
|
|
|
for (const auto & [key, table_info] : table_infos)
|
2022-06-06 19:15:17 +00:00
|
|
|
{
|
|
|
|
if (table_info.created)
|
|
|
|
continue;
|
|
|
|
|
|
|
|
/// Found a table which is not created yet.
|
|
|
|
all_tables_created = false;
|
|
|
|
|
|
|
|
/// Check if all dependencies have been created before.
|
|
|
|
bool all_dependencies_met = true;
|
|
|
|
for (const auto & dependency : table_info.dependencies)
|
2022-05-31 09:33:23 +00:00
|
|
|
{
|
2022-06-09 16:19:54 +00:00
|
|
|
auto it = table_infos.find(TableKey{dependency, false});
|
2022-06-06 19:15:17 +00:00
|
|
|
if ((it != table_infos.end()) && !it->second.created)
|
|
|
|
{
|
|
|
|
all_dependencies_met = false;
|
|
|
|
break;
|
|
|
|
}
|
2022-05-31 09:33:23 +00:00
|
|
|
}
|
2022-06-06 19:15:17 +00:00
|
|
|
|
|
|
|
if (all_dependencies_met)
|
2022-06-09 16:19:54 +00:00
|
|
|
tables_without_dependencies.push_back(key);
|
2022-06-06 19:15:17 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
if (!tables_without_dependencies.empty())
|
|
|
|
return tables_without_dependencies;
|
|
|
|
|
|
|
|
if (all_tables_created)
|
|
|
|
return {};
|
|
|
|
|
|
|
|
/// Cyclic dependency? We'll try to create those tables anyway but probably it's going to fail.
|
2022-06-09 16:19:54 +00:00
|
|
|
std::vector<TableKey> tables_with_cyclic_dependencies;
|
|
|
|
for (const auto & [key, table_info] : table_infos)
|
2022-06-06 19:15:17 +00:00
|
|
|
{
|
|
|
|
if (!table_info.created)
|
2022-06-09 16:19:54 +00:00
|
|
|
tables_with_cyclic_dependencies.push_back(key);
|
2022-05-31 09:33:23 +00:00
|
|
|
}
|
2022-06-06 19:15:17 +00:00
|
|
|
|
|
|
|
/// Only show a warning here, proper exception will be thrown later on creating those tables.
|
|
|
|
LOG_WARNING(
|
|
|
|
log,
|
|
|
|
"Some tables have cyclic dependency from each other: {}",
|
|
|
|
boost::algorithm::join(
|
|
|
|
tables_with_cyclic_dependencies
|
2022-06-09 16:19:54 +00:00
|
|
|
| boost::adaptors::transformed([](const TableKey & key) -> String { return key.name.getFullName(); }),
|
2022-06-06 19:15:17 +00:00
|
|
|
", "));
|
|
|
|
|
|
|
|
return tables_with_cyclic_dependencies;
|
2022-05-31 09:33:23 +00:00
|
|
|
}
|
|
|
|
|
2022-06-09 16:19:54 +00:00
|
|
|
void RestorerFromBackup::addDataRestoreTask(DataRestoreTask && new_task)
|
2022-05-31 09:33:23 +00:00
|
|
|
{
|
|
|
|
if (current_stage == Stage::kInsertingDataToTables)
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Adding data-restoring tasks is not allowed");
|
2022-06-09 16:19:54 +00:00
|
|
|
data_restore_tasks.push_back(std::move(new_task));
|
2022-05-31 09:33:23 +00:00
|
|
|
}
|
|
|
|
|
2022-06-09 16:19:54 +00:00
|
|
|
void RestorerFromBackup::addDataRestoreTasks(DataRestoreTasks && new_tasks)
|
2022-05-31 09:33:23 +00:00
|
|
|
{
|
|
|
|
if (current_stage == Stage::kInsertingDataToTables)
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Adding data-restoring tasks is not allowed");
|
2022-06-09 16:19:54 +00:00
|
|
|
insertAtEnd(data_restore_tasks, std::move(new_tasks));
|
2022-05-31 09:33:23 +00:00
|
|
|
}
|
|
|
|
|
2022-06-18 22:01:08 +00:00
|
|
|
void RestorerFromBackup::checkPathInBackupToRestoreAccess(const String & path)
|
2022-06-15 18:25:13 +00:00
|
|
|
{
|
2022-06-18 22:01:08 +00:00
|
|
|
if (!access_restore_task || !access_restore_task->hasDataPath(path))
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Path to restore access was not added");
|
|
|
|
}
|
2022-06-15 18:25:13 +00:00
|
|
|
|
2022-05-31 09:33:23 +00:00
|
|
|
void RestorerFromBackup::executeCreateQuery(const ASTPtr & create_query) const
|
|
|
|
{
|
|
|
|
InterpreterCreateQuery interpreter{create_query, context};
|
|
|
|
interpreter.setInternal(true);
|
|
|
|
interpreter.execute();
|
|
|
|
}
|
|
|
|
|
|
|
|
void RestorerFromBackup::throwPartitionsNotSupported(const StorageID & storage_id, const String & table_engine)
|
|
|
|
{
|
|
|
|
throw Exception(
|
|
|
|
ErrorCodes::CANNOT_RESTORE_TABLE,
|
|
|
|
"Table engine {} doesn't support partitions, cannot table {}",
|
|
|
|
table_engine,
|
|
|
|
storage_id.getFullTableName());
|
|
|
|
}
|
|
|
|
|
|
|
|
void RestorerFromBackup::throwTableIsNotEmpty(const StorageID & storage_id)
|
|
|
|
{
|
|
|
|
throw Exception(
|
|
|
|
ErrorCodes::CANNOT_RESTORE_TABLE,
|
|
|
|
"Cannot restore the table {} because it already contains some data. You can set structure_only=true or "
|
|
|
|
"allow_non_empty_tables=true to overcome that in the way you want",
|
|
|
|
storage_id.getFullTableName());
|
|
|
|
}
|
|
|
|
}
|