mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Make restore tasks explicit.
This commit is contained in:
parent
9fb50422ce
commit
7a63feb3f7
@ -2,24 +2,17 @@
|
||||
#include <Backups/BackupEntryFromMemory.h>
|
||||
#include <Backups/BackupRenamingConfig.h>
|
||||
#include <Backups/IBackup.h>
|
||||
#include <Backups/hasCompatibleDataToRestoreTable.h>
|
||||
#include <Backups/renameInCreateQuery.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Access/Common/AccessFlags.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <base/insertAtEnd.h>
|
||||
#include <base/sort.h>
|
||||
#include <boost/range/adaptor/reversed.hpp>
|
||||
#include <filesystem>
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -29,8 +22,6 @@ namespace ErrorCodes
|
||||
extern const int BACKUP_ELEMENT_DUPLICATE;
|
||||
extern const int BACKUP_IS_EMPTY;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int TABLE_ALREADY_EXISTS;
|
||||
extern const int CANNOT_RESTORE_TABLE;
|
||||
}
|
||||
|
||||
namespace
|
||||
@ -265,28 +256,6 @@ namespace
|
||||
elements[database.index].except_list.emplace(table_name);
|
||||
}
|
||||
|
||||
/// Reorder the elements: databases should be before tables and dictionaries they contain.
|
||||
for (auto & [database_name, database] : databases)
|
||||
{
|
||||
if (database.index == static_cast<size_t>(-1))
|
||||
continue;
|
||||
size_t min_index = std::numeric_limits<size_t>::max();
|
||||
auto min_index_it = database.tables.end();
|
||||
for (auto it = database.tables.begin(); it != database.tables.end(); ++it)
|
||||
{
|
||||
if (min_index > it->second)
|
||||
{
|
||||
min_index = it->second;
|
||||
min_index_it = it;
|
||||
}
|
||||
}
|
||||
if (database.index > min_index)
|
||||
{
|
||||
std::swap(elements[database.index], elements[min_index]);
|
||||
std::swap(database.index, min_index_it->second);
|
||||
}
|
||||
}
|
||||
|
||||
for (auto skip_index : skip_indices | boost::adaptors::reversed)
|
||||
elements.erase(elements.begin() + skip_index);
|
||||
}
|
||||
@ -302,48 +271,6 @@ namespace
|
||||
return res;
|
||||
}
|
||||
|
||||
String getDataPathInBackup(const DatabaseAndTableName & table_name)
|
||||
{
|
||||
if (table_name.first.empty() || table_name.second.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name and table name must not be empty");
|
||||
assert(!table_name.first.empty() && !table_name.second.empty());
|
||||
return String{"data/"} + escapeForFileName(table_name.first) + "/" + escapeForFileName(table_name.second) + "/";
|
||||
}
|
||||
|
||||
String getDataPathInBackup(const IAST & create_query)
|
||||
{
|
||||
const auto & create = create_query.as<const ASTCreateQuery &>();
|
||||
if (!create.table)
|
||||
return {};
|
||||
if (create.temporary)
|
||||
return getDataPathInBackup({DatabaseCatalog::TEMPORARY_DATABASE, create.getTable()});
|
||||
return getDataPathInBackup({create.getDatabase(), create.getTable()});
|
||||
}
|
||||
|
||||
String getMetadataPathInBackup(const DatabaseAndTableName & table_name)
|
||||
{
|
||||
if (table_name.first.empty() || table_name.second.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name and table name must not be empty");
|
||||
return String{"metadata/"} + escapeForFileName(table_name.first) + "/" + escapeForFileName(table_name.second) + ".sql";
|
||||
}
|
||||
|
||||
String getMetadataPathInBackup(const String & database_name)
|
||||
{
|
||||
if (database_name.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name must not be empty");
|
||||
return String{"metadata/"} + escapeForFileName(database_name) + ".sql";
|
||||
}
|
||||
|
||||
String getMetadataPathInBackup(const IAST & create_query)
|
||||
{
|
||||
const auto & create = create_query.as<const ASTCreateQuery &>();
|
||||
if (!create.table)
|
||||
return getMetadataPathInBackup(create.getDatabase());
|
||||
if (create.temporary)
|
||||
return getMetadataPathInBackup({DatabaseCatalog::TEMPORARY_DATABASE, create.getTable()});
|
||||
return getMetadataPathInBackup({create.getDatabase(), create.getTable()});
|
||||
}
|
||||
|
||||
void backupCreateQuery(const IAST & create_query, BackupEntries & backup_entries)
|
||||
{
|
||||
auto metadata_entry = std::make_unique<BackupEntryFromMemory>(serializeAST(create_query));
|
||||
@ -412,179 +339,6 @@ namespace
|
||||
backupDatabase(database, {}, context, renaming_config, backup_entries);
|
||||
}
|
||||
}
|
||||
|
||||
void makeDatabaseIfNotExists(const String & database_name, ContextMutablePtr context)
|
||||
{
|
||||
if (DatabaseCatalog::instance().isDatabaseExist(database_name))
|
||||
return;
|
||||
|
||||
/// We create and execute `create` query for the database name.
|
||||
auto create_query = std::make_shared<ASTCreateQuery>();
|
||||
create_query->setDatabase(database_name);
|
||||
create_query->if_not_exists = true;
|
||||
InterpreterCreateQuery create_interpreter{create_query, context};
|
||||
create_interpreter.execute();
|
||||
}
|
||||
|
||||
ASTPtr readCreateQueryFromBackup(const DatabaseAndTableName & table_name, const BackupPtr & backup)
|
||||
{
|
||||
String create_query_path = getMetadataPathInBackup(table_name);
|
||||
auto read_buffer = backup->readFile(create_query_path)->getReadBuffer();
|
||||
String create_query_str;
|
||||
readStringUntilEOF(create_query_str, *read_buffer);
|
||||
read_buffer.reset();
|
||||
ParserCreateQuery create_parser;
|
||||
return parseQuery(create_parser, create_query_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||
}
|
||||
|
||||
ASTPtr readCreateQueryFromBackup(const String & database_name, const BackupPtr & backup)
|
||||
{
|
||||
String create_query_path = getMetadataPathInBackup(database_name);
|
||||
auto read_buffer = backup->readFile(create_query_path)->getReadBuffer();
|
||||
String create_query_str;
|
||||
readStringUntilEOF(create_query_str, *read_buffer);
|
||||
read_buffer.reset();
|
||||
ParserCreateQuery create_parser;
|
||||
return parseQuery(create_parser, create_query_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||
}
|
||||
|
||||
void restoreTable(
|
||||
const DatabaseAndTableName & table_name,
|
||||
const ASTs & partitions,
|
||||
ContextMutablePtr context,
|
||||
const BackupPtr & backup,
|
||||
const BackupRenamingConfigPtr & renaming_config,
|
||||
RestoreObjectsTasks & restore_tasks)
|
||||
{
|
||||
ASTPtr create_query = readCreateQueryFromBackup(table_name, backup);
|
||||
auto new_create_query = typeid_cast<std::shared_ptr<ASTCreateQuery>>(renameInCreateQuery(create_query, renaming_config, context));
|
||||
|
||||
restore_tasks.emplace_back([table_name, new_create_query, partitions, context, backup]() -> RestoreDataTasks
|
||||
{
|
||||
DatabaseAndTableName new_table_name{new_create_query->getDatabase(), new_create_query->getTable()};
|
||||
if (new_create_query->temporary)
|
||||
new_table_name.first = DatabaseCatalog::TEMPORARY_DATABASE;
|
||||
|
||||
context->checkAccess(AccessType::INSERT, new_table_name.first, new_table_name.second);
|
||||
|
||||
StoragePtr storage;
|
||||
for (size_t try_index = 0; try_index != 10; ++try_index)
|
||||
{
|
||||
if (DatabaseCatalog::instance().isTableExist({new_table_name.first, new_table_name.second}, context))
|
||||
{
|
||||
DatabasePtr existing_database;
|
||||
StoragePtr existing_storage;
|
||||
std::tie(existing_database, existing_storage) = DatabaseCatalog::instance().tryGetDatabaseAndTable({new_table_name.first, new_table_name.second}, context);
|
||||
if (existing_storage)
|
||||
{
|
||||
if (auto existing_table_create_query = existing_database->tryGetCreateTableQuery(new_table_name.second, context))
|
||||
{
|
||||
if (hasCompatibleDataToRestoreTable(*new_create_query, existing_table_create_query->as<ASTCreateQuery &>()))
|
||||
{
|
||||
storage = existing_storage;
|
||||
break;
|
||||
}
|
||||
else
|
||||
{
|
||||
String error_message = (new_table_name.first == DatabaseCatalog::TEMPORARY_DATABASE)
|
||||
? ("Temporary table " + backQuoteIfNeed(new_table_name.second) + " already exists")
|
||||
: ("Table " + backQuoteIfNeed(new_table_name.first) + "." + backQuoteIfNeed(new_table_name.second)
|
||||
+ " already exists");
|
||||
throw Exception(error_message, ErrorCodes::CANNOT_RESTORE_TABLE);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
makeDatabaseIfNotExists(new_table_name.first, context);
|
||||
|
||||
try
|
||||
{
|
||||
InterpreterCreateQuery create_interpreter{new_create_query, context};
|
||||
create_interpreter.execute();
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
if (e.code() != ErrorCodes::TABLE_ALREADY_EXISTS)
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
if (!storage)
|
||||
{
|
||||
String error_message = (new_table_name.first == DatabaseCatalog::TEMPORARY_DATABASE)
|
||||
? ("Could not create temporary table " + backQuoteIfNeed(new_table_name.second) + " for restoring")
|
||||
: ("Could not create table " + backQuoteIfNeed(new_table_name.first) + "." + backQuoteIfNeed(new_table_name.second)
|
||||
+ " for restoring");
|
||||
throw Exception(error_message, ErrorCodes::CANNOT_RESTORE_TABLE);
|
||||
}
|
||||
|
||||
String data_path_in_backup = getDataPathInBackup(table_name);
|
||||
RestoreDataTasks restore_data_tasks = storage->restoreFromBackup(backup, data_path_in_backup, partitions, context);
|
||||
|
||||
/// Keep `storage` alive while we're executing `restore_data_tasks`.
|
||||
for (auto & restore_data_task : restore_data_tasks)
|
||||
restore_data_task = [restore_data_task, storage]() { restore_data_task(); };
|
||||
|
||||
return restore_data_tasks;
|
||||
});
|
||||
}
|
||||
|
||||
void restoreDatabase(const String & database_name, const std::set<String> & except_list, ContextMutablePtr context, const BackupPtr & backup, const BackupRenamingConfigPtr & renaming_config, RestoreObjectsTasks & restore_tasks)
|
||||
{
|
||||
ASTPtr create_query = readCreateQueryFromBackup(database_name, backup);
|
||||
auto new_create_query = typeid_cast<std::shared_ptr<ASTCreateQuery>>(renameInCreateQuery(create_query, renaming_config, context));
|
||||
|
||||
restore_tasks.emplace_back([database_name, new_create_query, except_list, context, backup, renaming_config]() -> RestoreDataTasks
|
||||
{
|
||||
const String & new_database_name = new_create_query->getDatabase();
|
||||
context->checkAccess(AccessType::SHOW_TABLES, new_database_name);
|
||||
|
||||
if (!DatabaseCatalog::instance().isDatabaseExist(new_database_name))
|
||||
{
|
||||
/// We create and execute `create` query for the database name.
|
||||
new_create_query->if_not_exists = true;
|
||||
InterpreterCreateQuery create_interpreter{new_create_query, context};
|
||||
create_interpreter.execute();
|
||||
}
|
||||
|
||||
RestoreObjectsTasks restore_objects_tasks;
|
||||
Strings table_metadata_filenames = backup->listFiles("metadata/" + escapeForFileName(database_name) + "/", "/");
|
||||
for (const String & table_metadata_filename : table_metadata_filenames)
|
||||
{
|
||||
String table_name = unescapeForFileName(fs::path{table_metadata_filename}.stem());
|
||||
if (except_list.contains(table_name))
|
||||
continue;
|
||||
restoreTable({database_name, table_name}, {}, context, backup, renaming_config, restore_objects_tasks);
|
||||
}
|
||||
|
||||
RestoreDataTasks restore_data_tasks;
|
||||
for (auto & restore_object_task : restore_objects_tasks)
|
||||
insertAtEnd(restore_data_tasks, std::move(restore_object_task)());
|
||||
return restore_data_tasks;
|
||||
});
|
||||
}
|
||||
|
||||
void restoreAllDatabases(const std::set<String> & except_list, ContextMutablePtr context, const BackupPtr & backup, const BackupRenamingConfigPtr & renaming_config, RestoreObjectsTasks & restore_tasks)
|
||||
{
|
||||
restore_tasks.emplace_back([except_list, context, backup, renaming_config]() -> RestoreDataTasks
|
||||
{
|
||||
RestoreObjectsTasks restore_objects_tasks;
|
||||
Strings database_metadata_filenames = backup->listFiles("metadata/", "/");
|
||||
for (const String & database_metadata_filename : database_metadata_filenames)
|
||||
{
|
||||
String database_name = unescapeForFileName(fs::path{database_metadata_filename}.stem());
|
||||
if (except_list.contains(database_name))
|
||||
continue;
|
||||
restoreDatabase(database_name, {}, context, backup, renaming_config, restore_objects_tasks);
|
||||
}
|
||||
|
||||
RestoreDataTasks restore_data_tasks;
|
||||
for (auto & restore_object_task : restore_objects_tasks)
|
||||
insertAtEnd(restore_data_tasks, std::move(restore_object_task)());
|
||||
return restore_data_tasks;
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -728,106 +482,46 @@ void writeBackupEntries(BackupMutablePtr backup, BackupEntries && backup_entries
|
||||
backup->finalizeWriting();
|
||||
}
|
||||
|
||||
|
||||
RestoreObjectsTasks makeRestoreTasks(const Elements & elements, ContextMutablePtr context, const BackupPtr & backup)
|
||||
String getDataPathInBackup(const DatabaseAndTableName & table_name)
|
||||
{
|
||||
RestoreObjectsTasks restore_tasks;
|
||||
|
||||
auto elements2 = adjustElements(elements, context->getCurrentDatabase());
|
||||
auto renaming_config = std::make_shared<BackupRenamingConfig>();
|
||||
renaming_config->setFromBackupQueryElements(elements2);
|
||||
|
||||
for (const auto & element : elements2)
|
||||
{
|
||||
switch (element.type)
|
||||
{
|
||||
case ElementType::TABLE:
|
||||
{
|
||||
const String & database_name = element.name.first;
|
||||
const String & table_name = element.name.second;
|
||||
restoreTable({database_name, table_name}, element.partitions, context, backup, renaming_config, restore_tasks);
|
||||
break;
|
||||
}
|
||||
|
||||
case ElementType::DATABASE:
|
||||
{
|
||||
const String & database_name = element.name.first;
|
||||
restoreDatabase(database_name, element.except_list, context, backup, renaming_config, restore_tasks);
|
||||
break;
|
||||
}
|
||||
|
||||
case ElementType::ALL_DATABASES:
|
||||
{
|
||||
restoreAllDatabases(element.except_list, context, backup, renaming_config, restore_tasks);
|
||||
break;
|
||||
}
|
||||
|
||||
default:
|
||||
throw Exception("Unexpected element type", ErrorCodes::LOGICAL_ERROR); /// other element types have been removed in deduplicateElements()
|
||||
}
|
||||
}
|
||||
|
||||
return restore_tasks;
|
||||
if (table_name.first.empty() || table_name.second.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name and table name must not be empty");
|
||||
assert(!table_name.first.empty() && !table_name.second.empty());
|
||||
return String{"data/"} + escapeForFileName(table_name.first) + "/" + escapeForFileName(table_name.second) + "/";
|
||||
}
|
||||
|
||||
|
||||
void executeRestoreTasks(RestoreObjectsTasks && restore_tasks, size_t num_threads)
|
||||
String getDataPathInBackup(const IAST & create_query)
|
||||
{
|
||||
if (!num_threads)
|
||||
num_threads = 1;
|
||||
const auto & create = create_query.as<const ASTCreateQuery &>();
|
||||
if (!create.table)
|
||||
return {};
|
||||
if (create.temporary)
|
||||
return getDataPathInBackup({DatabaseCatalog::TEMPORARY_DATABASE, create.getTable()});
|
||||
return getDataPathInBackup({create.getDatabase(), create.getTable()});
|
||||
}
|
||||
|
||||
RestoreDataTasks restore_data_tasks;
|
||||
for (auto & restore_object_task : restore_tasks)
|
||||
insertAtEnd(restore_data_tasks, std::move(restore_object_task)());
|
||||
restore_tasks.clear();
|
||||
String getMetadataPathInBackup(const DatabaseAndTableName & table_name)
|
||||
{
|
||||
if (table_name.first.empty() || table_name.second.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name and table name must not be empty");
|
||||
return String{"metadata/"} + escapeForFileName(table_name.first) + "/" + escapeForFileName(table_name.second) + ".sql";
|
||||
}
|
||||
|
||||
std::vector<ThreadFromGlobalPool> threads;
|
||||
size_t num_active_threads = 0;
|
||||
std::mutex mutex;
|
||||
std::condition_variable cond;
|
||||
std::exception_ptr exception;
|
||||
String getMetadataPathInBackup(const String & database_name)
|
||||
{
|
||||
if (database_name.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name must not be empty");
|
||||
return String{"metadata/"} + escapeForFileName(database_name) + ".sql";
|
||||
}
|
||||
|
||||
for (auto & restore_data_task : restore_data_tasks)
|
||||
{
|
||||
{
|
||||
std::unique_lock lock{mutex};
|
||||
if (exception)
|
||||
break;
|
||||
cond.wait(lock, [&] { return num_active_threads < num_threads; });
|
||||
if (exception)
|
||||
break;
|
||||
++num_active_threads;
|
||||
}
|
||||
|
||||
threads.emplace_back([&restore_data_task, &mutex, &cond, &num_active_threads, &exception]() mutable
|
||||
{
|
||||
try
|
||||
{
|
||||
restore_data_task();
|
||||
restore_data_task = {};
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
if (!exception)
|
||||
exception = std::current_exception();
|
||||
}
|
||||
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
--num_active_threads;
|
||||
cond.notify_all();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
for (auto & thread : threads)
|
||||
thread.join();
|
||||
|
||||
restore_data_tasks.clear();
|
||||
|
||||
if (exception)
|
||||
std::rethrow_exception(exception);
|
||||
String getMetadataPathInBackup(const IAST & create_query)
|
||||
{
|
||||
const auto & create = create_query.as<const ASTCreateQuery &>();
|
||||
if (!create.table)
|
||||
return getMetadataPathInBackup(create.getDatabase());
|
||||
if (create.temporary)
|
||||
return getMetadataPathInBackup({DatabaseCatalog::TEMPORARY_DATABASE, create.getTable()});
|
||||
return getMetadataPathInBackup({create.getDatabase(), create.getTable()});
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -12,13 +12,8 @@ using BackupMutablePtr = std::shared_ptr<IBackup>;
|
||||
class IBackupEntry;
|
||||
using BackupEntryPtr = std::unique_ptr<IBackupEntry>;
|
||||
using BackupEntries = std::vector<std::pair<String, BackupEntryPtr>>;
|
||||
using RestoreDataTask = std::function<void()>;
|
||||
using RestoreDataTasks = std::vector<RestoreDataTask>;
|
||||
using RestoreObjectTask = std::function<RestoreDataTasks()>;
|
||||
using RestoreObjectsTasks = std::vector<RestoreObjectTask>;
|
||||
class Context;
|
||||
using ContextPtr = std::shared_ptr<const Context>;
|
||||
using ContextMutablePtr = std::shared_ptr<Context>;
|
||||
|
||||
|
||||
/// Prepares backup entries.
|
||||
@ -30,10 +25,13 @@ UInt64 estimateBackupSize(const BackupEntries & backup_entries, const BackupPtr
|
||||
/// Write backup entries to an opened backup.
|
||||
void writeBackupEntries(BackupMutablePtr backup, BackupEntries && backup_entries, size_t num_threads);
|
||||
|
||||
/// Prepare restore tasks.
|
||||
RestoreObjectsTasks makeRestoreTasks(const ASTBackupQuery::Elements & elements, ContextMutablePtr context, const BackupPtr & backup);
|
||||
/// Returns the path to metadata in backup.
|
||||
String getMetadataPathInBackup(const DatabaseAndTableName & table_name);
|
||||
String getMetadataPathInBackup(const String & database_name);
|
||||
String getMetadataPathInBackup(const IAST & create_query);
|
||||
|
||||
/// Execute restore tasks.
|
||||
void executeRestoreTasks(RestoreObjectsTasks && restore_tasks, size_t num_threads);
|
||||
/// Returns the path to table's data in backup.
|
||||
String getDataPathInBackup(const DatabaseAndTableName & table_name);
|
||||
String getDataPathInBackup(const IAST & create_query);
|
||||
|
||||
}
|
||||
|
31
src/Backups/IRestoreFromBackupTask.h
Normal file
31
src/Backups/IRestoreFromBackupTask.h
Normal file
@ -0,0 +1,31 @@
|
||||
#pragma once
|
||||
|
||||
#include <memory>
|
||||
#include <vector>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Represents a task of restoring something (database / table / table's part) from backup.
|
||||
class IRestoreFromBackupTask
|
||||
{
|
||||
public:
|
||||
IRestoreFromBackupTask() = default;
|
||||
virtual ~IRestoreFromBackupTask() = default;
|
||||
|
||||
/// Perform restoring, the function also can return a list of nested tasks that should be run later.
|
||||
virtual std::vector<std::unique_ptr<IRestoreFromBackupTask>> run() = 0;
|
||||
|
||||
/// Is it necessary to run this task sequentially?
|
||||
/// Sequential tasks are executed first and strictly in one thread.
|
||||
virtual bool isSequential() const { return false; }
|
||||
|
||||
/// Reverts the effect of run(). If that's not possible, the function does nothing.
|
||||
virtual void rollback() {}
|
||||
};
|
||||
|
||||
using RestoreFromBackupTaskPtr = std::unique_ptr<IRestoreFromBackupTask>;
|
||||
using RestoreFromBackupTasks = std::vector<RestoreFromBackupTaskPtr>;
|
||||
|
||||
}
|
637
src/Backups/RestoreFromBackupUtils.cpp
Normal file
637
src/Backups/RestoreFromBackupUtils.cpp
Normal file
@ -0,0 +1,637 @@
|
||||
#include <Backups/BackupUtils.h>
|
||||
#include <Backups/BackupRenamingConfig.h>
|
||||
#include <Backups/IBackup.h>
|
||||
#include <Backups/IBackupEntry.h>
|
||||
#include <Backups/IRestoreFromBackupTask.h>
|
||||
#include <Backups/hasCompatibleDataToRestoreTable.h>
|
||||
#include <Backups/renameInCreateQuery.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <boost/range/adaptor/reversed.hpp>
|
||||
#include <filesystem>
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_RESTORE_TABLE;
|
||||
extern const int CANNOT_RESTORE_DATABASE;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
using Kind = ASTBackupQuery::Kind;
|
||||
using Element = ASTBackupQuery::Element;
|
||||
using Elements = ASTBackupQuery::Elements;
|
||||
using ElementType = ASTBackupQuery::ElementType;
|
||||
|
||||
|
||||
/// Replaces an empty database with the current database.
|
||||
void replaceEmptyDatabaseWithCurrentDatabase(Elements & elements, const String & current_database)
|
||||
{
|
||||
for (auto & element : elements)
|
||||
{
|
||||
if (element.type == ElementType::TABLE)
|
||||
{
|
||||
if (element.name.first.empty() && !element.name.second.empty())
|
||||
element.name.first = current_database;
|
||||
if (element.new_name.first.empty() && !element.new_name.second.empty())
|
||||
element.new_name.first = current_database;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/// Restores a database (without tables inside), should be executed before executing
|
||||
/// RestoreTableFromBackupTask.
|
||||
class RestoreDatabaseFromBackupTask : public IRestoreFromBackupTask
|
||||
{
|
||||
public:
|
||||
RestoreDatabaseFromBackupTask(ContextMutablePtr context_, const ASTPtr & create_query_)
|
||||
: context(context_), create_query(typeid_cast<std::shared_ptr<ASTCreateQuery>>(create_query_))
|
||||
{
|
||||
}
|
||||
|
||||
RestoreFromBackupTasks run() override
|
||||
{
|
||||
createDatabase();
|
||||
return {};
|
||||
}
|
||||
|
||||
bool isSequential() const override { return true; }
|
||||
|
||||
private:
|
||||
void createDatabase()
|
||||
{
|
||||
InterpreterCreateQuery create_interpreter{create_query, context};
|
||||
create_interpreter.execute();
|
||||
}
|
||||
|
||||
ContextMutablePtr context;
|
||||
std::shared_ptr<ASTCreateQuery> create_query;
|
||||
};
|
||||
|
||||
|
||||
/// Restores a table and fills it with data.
|
||||
class RestoreTableFromBackupTask : public IRestoreFromBackupTask
|
||||
{
|
||||
public:
|
||||
RestoreTableFromBackupTask(
|
||||
ContextMutablePtr context_,
|
||||
const ASTPtr & create_query_,
|
||||
const ASTs & partitions_,
|
||||
const BackupPtr & backup_,
|
||||
const DatabaseAndTableName & table_name_in_backup_)
|
||||
: context(context_), create_query(typeid_cast<std::shared_ptr<ASTCreateQuery>>(create_query_)),
|
||||
partitions(partitions_), backup(backup_), table_name_in_backup(table_name_in_backup_)
|
||||
{
|
||||
table_name = DatabaseAndTableName{create_query->getDatabase(), create_query->getTable()};
|
||||
if (create_query->temporary)
|
||||
table_name.first = DatabaseCatalog::TEMPORARY_DATABASE;
|
||||
}
|
||||
|
||||
RestoreFromBackupTasks run() override
|
||||
{
|
||||
createStorage();
|
||||
auto storage = getStorage();
|
||||
RestoreFromBackupTasks tasks;
|
||||
if (auto task = insertDataIntoStorage(storage))
|
||||
tasks.push_back(std::move(task));
|
||||
return tasks;
|
||||
}
|
||||
|
||||
bool isSequential() const override { return true; }
|
||||
|
||||
private:
|
||||
void createStorage()
|
||||
{
|
||||
InterpreterCreateQuery create_interpreter{create_query, 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);
|
||||
}
|
||||
|
||||
RestoreFromBackupTaskPtr insertDataIntoStorage(StoragePtr storage)
|
||||
{
|
||||
context->checkAccess(AccessType::INSERT, table_name.first, table_name.second);
|
||||
String data_path_in_backup = getDataPathInBackup(table_name_in_backup);
|
||||
return storage->restoreFromBackup(backup, data_path_in_backup, partitions, context);
|
||||
}
|
||||
|
||||
ContextMutablePtr context;
|
||||
std::shared_ptr<ASTCreateQuery> create_query;
|
||||
DatabaseAndTableName table_name;
|
||||
ASTs partitions;
|
||||
BackupPtr backup;
|
||||
DatabaseAndTableName table_name_in_backup;
|
||||
};
|
||||
|
||||
|
||||
/// Makes tasks for restoring databases and tables according to the elements of ASTBackupQuery.
|
||||
/// Keep this class consistent with BackupEntriesBuilder.
|
||||
class RestoreTasksBuilder
|
||||
{
|
||||
public:
|
||||
RestoreTasksBuilder(ContextMutablePtr context_, const BackupPtr & backup_)
|
||||
: context(context_), backup(backup_) {}
|
||||
|
||||
/// Prepares internal structures for making tasks for restoring.
|
||||
void prepare(const ASTBackupQuery::Elements & elements)
|
||||
{
|
||||
auto elements2 = elements;
|
||||
replaceEmptyDatabaseWithCurrentDatabase(elements2, context->getCurrentDatabase());
|
||||
|
||||
auto new_renaming_config = std::make_shared<BackupRenamingConfig>();
|
||||
new_renaming_config->setFromBackupQueryElements(elements2);
|
||||
renaming_config = new_renaming_config;
|
||||
|
||||
for (const auto & element : elements2)
|
||||
{
|
||||
switch (element.type)
|
||||
{
|
||||
case ElementType::TABLE: [[fallthrough]];
|
||||
case ElementType::DICTIONARY:
|
||||
{
|
||||
const String & database_name = element.name.first;
|
||||
const String & table_name = element.name.second;
|
||||
prepareToRestoreTable(DatabaseAndTableName{database_name, table_name}, element.partitions);
|
||||
break;
|
||||
}
|
||||
|
||||
case ElementType::TEMPORARY_TABLE:
|
||||
{
|
||||
String database_name = DatabaseCatalog::TEMPORARY_DATABASE;
|
||||
const String & table_name = element.name.second;
|
||||
prepareToRestoreTable(DatabaseAndTableName{database_name, table_name}, element.partitions);
|
||||
break;
|
||||
}
|
||||
|
||||
case ElementType::DATABASE:
|
||||
{
|
||||
const String & database_name = element.name.first;
|
||||
prepareToRestoreDatabase(database_name, element.except_list);
|
||||
break;
|
||||
}
|
||||
|
||||
case ElementType::ALL_TEMPORARY_TABLES:
|
||||
{
|
||||
prepareToRestoreDatabase(DatabaseCatalog::TEMPORARY_DATABASE, element.except_list);
|
||||
break;
|
||||
}
|
||||
|
||||
case ElementType::ALL_DATABASES:
|
||||
{
|
||||
prepareToRestoreAllDatabases(element.except_list);
|
||||
break;
|
||||
}
|
||||
|
||||
case ElementType::EVERYTHING:
|
||||
{
|
||||
prepareToRestoreAllDatabases({});
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Makes tasks for restoring, should be called after prepare().
|
||||
RestoreFromBackupTasks makeTasks() const
|
||||
{
|
||||
/// Check that there are not `different_create_query`. (If it's set it means error.)
|
||||
for (auto & info : databases | boost::adaptors::map_values)
|
||||
{
|
||||
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 {}",
|
||||
serializeAST(*info.create_query), serializeAST(*info.different_create_query));
|
||||
}
|
||||
|
||||
RestoreFromBackupTasks res;
|
||||
for (auto & info : databases | boost::adaptors::map_values)
|
||||
res.push_back(std::make_unique<RestoreDatabaseFromBackupTask>(context, info.create_query));
|
||||
|
||||
/// TODO: We need to restore tables according to their dependencies.
|
||||
for (auto & info : tables | boost::adaptors::map_values)
|
||||
res.push_back(std::make_unique<RestoreTableFromBackupTask>(context, info.create_query, info.partitions, backup, info.name_in_backup));
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
private:
|
||||
/// Prepares to restore a single table and probably its database's definition.
|
||||
void prepareToRestoreTable(const DatabaseAndTableName & table_name_, const ASTs & partitions_)
|
||||
{
|
||||
/// Check that we are not trying to restore the same table again.
|
||||
DatabaseAndTableName new_table_name = renaming_config->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);
|
||||
}
|
||||
|
||||
/// Make a create query for this table.
|
||||
auto create_query = renameInCreateQuery(readCreateQueryFromBackup(table_name_));
|
||||
create_query->if_not_exists = true;
|
||||
|
||||
CreateTableInfo info;
|
||||
info.create_query = create_query;
|
||||
info.name_in_backup = table_name_;
|
||||
info.partitions = partitions_;
|
||||
tables[new_table_name] = std::move(info);
|
||||
|
||||
/// If it's not system or temporary database then probably we need to restore the database's definition too.
|
||||
if (!isSystemOrTemporaryDatabase(new_table_name.first))
|
||||
{
|
||||
if (!databases.contains(new_table_name.first))
|
||||
{
|
||||
/// Add a create query for restoring the database if we haven't done it yet.
|
||||
std::shared_ptr<ASTCreateQuery> create_db_query;
|
||||
String db_name_in_backup = table_name_.first;
|
||||
if (hasCreateQueryInBackup(db_name_in_backup))
|
||||
{
|
||||
create_db_query = renameInCreateQuery(readCreateQueryFromBackup(db_name_in_backup));
|
||||
}
|
||||
else
|
||||
{
|
||||
create_db_query = std::make_shared<ASTCreateQuery>();
|
||||
db_name_in_backup.clear();
|
||||
}
|
||||
create_db_query->setDatabase(new_table_name.first);
|
||||
create_db_query->if_not_exists = true;
|
||||
|
||||
CreateDatabaseInfo info_db;
|
||||
info_db.create_query = create_db_query;
|
||||
info_db.name_in_backup = std::move(db_name_in_backup);
|
||||
info_db.is_explicit = false;
|
||||
databases[new_table_name.first] = std::move(info_db);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// We already have added a create query for restoring the database,
|
||||
/// set `different_create_query` if it's not the same.
|
||||
auto & info_db = databases[new_table_name.first];
|
||||
if (!info_db.is_explicit && (info_db.name_in_backup != table_name_.first) && !info_db.different_create_query)
|
||||
{
|
||||
std::shared_ptr<ASTCreateQuery> create_db_query;
|
||||
if (hasCreateQueryInBackup(table_name_.first))
|
||||
create_db_query = renameInCreateQuery(readCreateQueryFromBackup(table_name_.first));
|
||||
else
|
||||
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))
|
||||
info_db.different_create_query = create_db_query;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Prepares to restore a database and all tables in it.
|
||||
void prepareToRestoreDatabase(const String & database_name_, const std::set<String> & except_list_)
|
||||
{
|
||||
/// Check that we are not trying to restore the same database again.
|
||||
String new_database_name = renaming_config->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));
|
||||
|
||||
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_));
|
||||
|
||||
/// Of course we're not going to restore the definition of the system or the temporary database.
|
||||
if (!isSystemOrTemporaryDatabase(new_database_name))
|
||||
{
|
||||
/// Make a create query for this database.
|
||||
std::shared_ptr<ASTCreateQuery> create_db_query;
|
||||
String db_name_in_backup = database_name_;
|
||||
if (hasCreateQueryInBackup(db_name_in_backup))
|
||||
{
|
||||
create_db_query = renameInCreateQuery(readCreateQueryFromBackup(db_name_in_backup));
|
||||
}
|
||||
else
|
||||
{
|
||||
create_db_query = std::make_shared<ASTCreateQuery>();
|
||||
create_db_query->setDatabase(database_name_);
|
||||
db_name_in_backup.clear();
|
||||
}
|
||||
|
||||
create_db_query->if_not_exists = true;
|
||||
|
||||
CreateDatabaseInfo info_db;
|
||||
info_db.create_query = create_db_query;
|
||||
info_db.name_in_backup = std::move(db_name_in_backup);
|
||||
info_db.is_explicit = true;
|
||||
databases[new_database_name] = std::move(info_db);
|
||||
}
|
||||
|
||||
/// Restore tables in this database.
|
||||
for (const String & table_metadata_filename : table_metadata_filenames)
|
||||
{
|
||||
String table_name = unescapeForFileName(fs::path{table_metadata_filename}.stem());
|
||||
if (except_list_.contains(table_name))
|
||||
continue;
|
||||
prepareToRestoreTable(DatabaseAndTableName{database_name_, table_name}, ASTs{});
|
||||
}
|
||||
}
|
||||
|
||||
/// Prepares to restore all the databases contained in the backup.
|
||||
void prepareToRestoreAllDatabases(const std::set<String> & except_list_)
|
||||
{
|
||||
Strings database_metadata_filenames = backup->listFiles("metadata/", "/");
|
||||
for (const String & database_metadata_filename : database_metadata_filenames)
|
||||
{
|
||||
String database_name = unescapeForFileName(fs::path{database_metadata_filename}.stem());
|
||||
if (except_list_.contains(database_name))
|
||||
continue;
|
||||
prepareToRestoreDatabase(database_name, std::set<String>{});
|
||||
}
|
||||
}
|
||||
|
||||
/// Reads a create query for creating a specified table from the backup.
|
||||
std::shared_ptr<ASTCreateQuery> readCreateQueryFromBackup(const DatabaseAndTableName & table_name) const
|
||||
{
|
||||
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));
|
||||
auto read_buffer = backup->readFile(create_query_path)->getReadBuffer();
|
||||
String create_query_str;
|
||||
readStringUntilEOF(create_query_str, *read_buffer);
|
||||
read_buffer.reset();
|
||||
ParserCreateQuery create_parser;
|
||||
return typeid_cast<std::shared_ptr<ASTCreateQuery>>(parseQuery(create_parser, create_query_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH));
|
||||
}
|
||||
|
||||
/// Reads a create query for creating a specified database from the backup.
|
||||
std::shared_ptr<ASTCreateQuery> readCreateQueryFromBackup(const String & database_name) const
|
||||
{
|
||||
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));
|
||||
auto read_buffer = backup->readFile(create_query_path)->getReadBuffer();
|
||||
String create_query_str;
|
||||
readStringUntilEOF(create_query_str, *read_buffer);
|
||||
read_buffer.reset();
|
||||
ParserCreateQuery create_parser;
|
||||
return typeid_cast<std::shared_ptr<ASTCreateQuery>>(parseQuery(create_parser, create_query_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH));
|
||||
}
|
||||
|
||||
/// Whether there is a create query for creating a specified database in the backup.
|
||||
bool hasCreateQueryInBackup(const String & database_name) const
|
||||
{
|
||||
String create_query_path = getMetadataPathInBackup(database_name);
|
||||
return backup->fileExists(create_query_path);
|
||||
}
|
||||
|
||||
/// Do renaming in the create query according to the renaming config.
|
||||
std::shared_ptr<ASTCreateQuery> renameInCreateQuery(const ASTPtr & ast) const
|
||||
{
|
||||
return typeid_cast<std::shared_ptr<ASTCreateQuery>>(::DB::renameInCreateQuery(ast, renaming_config, context));
|
||||
}
|
||||
|
||||
static bool isSystemOrTemporaryDatabase(const String & database_name)
|
||||
{
|
||||
return (database_name == DatabaseCatalog::SYSTEM_DATABASE) || (database_name == DatabaseCatalog::TEMPORARY_DATABASE);
|
||||
}
|
||||
|
||||
/// Information which is used to make an instance of RestoreTableFromBackupTask.
|
||||
struct CreateTableInfo
|
||||
{
|
||||
ASTPtr create_query;
|
||||
DatabaseAndTableName name_in_backup;
|
||||
ASTs partitions;
|
||||
};
|
||||
|
||||
/// Information which is used to make an instance of RestoreDatabaseFromBackupTask.
|
||||
struct CreateDatabaseInfo
|
||||
{
|
||||
ASTPtr create_query;
|
||||
String name_in_backup;
|
||||
|
||||
/// Whether the creation of this database is specified explicitly, via RESTORE DATABASE or
|
||||
/// RESTORE ALL DATABASES.
|
||||
/// It's false if the creation of this database is caused by creating a table contained in it.
|
||||
bool is_explicit = false;
|
||||
|
||||
/// If this is set it means the following error:
|
||||
/// it means that for implicitly created database there were two different create query
|
||||
/// generated so we cannot restore the database.
|
||||
ASTPtr different_create_query;
|
||||
};
|
||||
|
||||
ContextMutablePtr context;
|
||||
BackupPtr backup;
|
||||
BackupRenamingConfigPtr renaming_config;
|
||||
std::map<String, CreateDatabaseInfo> databases;
|
||||
std::map<DatabaseAndTableName, CreateTableInfo> tables;
|
||||
};
|
||||
|
||||
|
||||
/// Reverts completed restore tasks (in reversed order).
|
||||
void rollbackRestoreTasks(RestoreFromBackupTasks && restore_tasks)
|
||||
{
|
||||
for (auto & restore_task : restore_tasks | boost::adaptors::reversed)
|
||||
{
|
||||
try
|
||||
{
|
||||
std::move(restore_task)->rollback();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException("Restore", "Couldn't rollback changes after failed RESTORE");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
RestoreFromBackupTasks makeRestoreTasks(ContextMutablePtr context, const BackupPtr & backup, const Elements & elements)
|
||||
{
|
||||
RestoreTasksBuilder builder{context, backup};
|
||||
builder.prepare(elements);
|
||||
return builder.makeTasks();
|
||||
}
|
||||
|
||||
|
||||
void executeRestoreTasks(RestoreFromBackupTasks && restore_tasks, size_t num_threads)
|
||||
{
|
||||
if (!num_threads)
|
||||
num_threads = 1;
|
||||
|
||||
RestoreFromBackupTasks completed_tasks;
|
||||
bool need_rollback_completed_tasks = true;
|
||||
|
||||
SCOPE_EXIT({
|
||||
if (need_rollback_completed_tasks)
|
||||
rollbackRestoreTasks(std::move(completed_tasks));
|
||||
});
|
||||
|
||||
std::deque<std::unique_ptr<IRestoreFromBackupTask>> sequential_tasks;
|
||||
std::deque<std::unique_ptr<IRestoreFromBackupTask>> enqueued_tasks;
|
||||
|
||||
/// There are two kinds of restore tasks: sequential and non-sequential ones.
|
||||
/// Sequential tasks are executed first and always in one thread.
|
||||
for (auto & task : restore_tasks)
|
||||
{
|
||||
if (task->isSequential())
|
||||
sequential_tasks.push_back(std::move(task));
|
||||
else
|
||||
enqueued_tasks.push_back(std::move(task));
|
||||
}
|
||||
|
||||
/// Sequential tasks.
|
||||
while (!sequential_tasks.empty())
|
||||
{
|
||||
auto current_task = std::move(sequential_tasks.front());
|
||||
sequential_tasks.pop_front();
|
||||
|
||||
RestoreFromBackupTasks new_tasks = current_task->run();
|
||||
|
||||
completed_tasks.push_back(std::move(current_task));
|
||||
for (auto & task : new_tasks)
|
||||
{
|
||||
if (task->isSequential())
|
||||
sequential_tasks.push_back(std::move(task));
|
||||
else
|
||||
enqueued_tasks.push_back(std::move(task));
|
||||
}
|
||||
}
|
||||
|
||||
/// Non-sequential tasks.
|
||||
std::unordered_map<IRestoreFromBackupTask *, std::unique_ptr<IRestoreFromBackupTask>> running_tasks;
|
||||
std::vector<ThreadFromGlobalPool> threads;
|
||||
std::mutex mutex;
|
||||
std::condition_variable cond;
|
||||
std::exception_ptr exception;
|
||||
|
||||
while (true)
|
||||
{
|
||||
IRestoreFromBackupTask * current_task = nullptr;
|
||||
{
|
||||
std::unique_lock lock{mutex};
|
||||
cond.wait(lock, [&]
|
||||
{
|
||||
if (exception)
|
||||
return true;
|
||||
if (enqueued_tasks.empty())
|
||||
return running_tasks.empty();
|
||||
return (running_tasks.size() < num_threads);
|
||||
});
|
||||
|
||||
if (exception || enqueued_tasks.empty())
|
||||
break;
|
||||
|
||||
auto current_task_ptr = std::move(enqueued_tasks.front());
|
||||
current_task = current_task_ptr.get();
|
||||
enqueued_tasks.pop_front();
|
||||
running_tasks[current_task] = std::move(current_task_ptr);
|
||||
}
|
||||
|
||||
assert(current_task);
|
||||
threads.emplace_back([current_task, &mutex, &cond, &enqueued_tasks, &running_tasks, &completed_tasks, &exception]() mutable
|
||||
{
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
if (exception)
|
||||
return;
|
||||
}
|
||||
|
||||
RestoreFromBackupTasks new_tasks;
|
||||
std::exception_ptr new_exception;
|
||||
try
|
||||
{
|
||||
new_tasks = current_task->run();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
new_exception = std::current_exception();
|
||||
}
|
||||
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
auto current_task_it = running_tasks.find(current_task);
|
||||
auto current_task_ptr = std::move(current_task_it->second);
|
||||
running_tasks.erase(current_task_it);
|
||||
|
||||
if (!new_exception)
|
||||
{
|
||||
completed_tasks.push_back(std::move(current_task_ptr));
|
||||
enqueued_tasks.insert(
|
||||
enqueued_tasks.end(), std::make_move_iterator(new_tasks.begin()), std::make_move_iterator(new_tasks.end()));
|
||||
}
|
||||
|
||||
if (!exception)
|
||||
exception = new_exception;
|
||||
|
||||
cond.notify_all();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
for (auto & thread : threads)
|
||||
thread.join();
|
||||
|
||||
if (exception)
|
||||
std::rethrow_exception(exception);
|
||||
else
|
||||
need_rollback_completed_tasks = false;
|
||||
}
|
||||
|
||||
}
|
23
src/Backups/RestoreFromBackupUtils.h
Normal file
23
src/Backups/RestoreFromBackupUtils.h
Normal file
@ -0,0 +1,23 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/ASTBackupQuery.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IBackup;
|
||||
using BackupPtr = std::shared_ptr<const IBackup>;
|
||||
class IRestoreFromBackupTask;
|
||||
using RestoreFromBackupTaskPtr = std::unique_ptr<IRestoreFromBackupTask>;
|
||||
using RestoreFromBackupTasks = std::vector<RestoreFromBackupTaskPtr>;
|
||||
class Context;
|
||||
using ContextMutablePtr = std::shared_ptr<Context>;
|
||||
|
||||
/// Prepares restore tasks.
|
||||
RestoreFromBackupTasks makeRestoreTasks(ContextMutablePtr context, const BackupPtr & backup, const ASTBackupQuery::Elements & elements);
|
||||
|
||||
/// Executes restore tasks.
|
||||
void executeRestoreTasks(RestoreFromBackupTasks && tasks, size_t num_threads);
|
||||
|
||||
}
|
@ -16,6 +16,10 @@ bool hasCompatibleDataToRestoreTable(const ASTCreateQuery & query1, const ASTCre
|
||||
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);
|
||||
}
|
||||
|
||||
|
@ -615,6 +615,7 @@
|
||||
M(644, REMOTE_FS_OBJECT_CACHE_ERROR) \
|
||||
M(645, NUMBER_OF_DIMENSIONS_MISMATHED) \
|
||||
M(646, RBAC_VERSION_IS_TOO_NEW) \
|
||||
M(647, CANNOT_RESTORE_DATABASE) \
|
||||
\
|
||||
M(999, KEEPER_EXCEPTION) \
|
||||
M(1000, POCO_EXCEPTION) \
|
||||
|
@ -2,8 +2,10 @@
|
||||
#include <Backups/BackupFactory.h>
|
||||
#include <Backups/BackupSettings.h>
|
||||
#include <Backups/BackupUtils.h>
|
||||
#include <Backups/RestoreFromBackupUtils.h>
|
||||
#include <Backups/IBackup.h>
|
||||
#include <Backups/IBackupEntry.h>
|
||||
#include <Backups/IRestoreFromBackupTask.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
@ -45,7 +47,7 @@ namespace
|
||||
void executeRestore(const ASTBackupQuery & query, ContextMutablePtr context)
|
||||
{
|
||||
BackupPtr backup = createBackup(query, context);
|
||||
auto restore_tasks = makeRestoreTasks(query.elements, context, backup);
|
||||
auto restore_tasks = makeRestoreTasks(context, backup, query.elements);
|
||||
executeRestoreTasks(std::move(restore_tasks), context->getSettingsRef().max_backup_threads);
|
||||
}
|
||||
}
|
||||
|
@ -221,7 +221,7 @@ BackupEntries IStorage::backup(const ASTs &, ContextPtr)
|
||||
throw Exception("Table engine " + getName() + " doesn't support backups", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
RestoreDataTasks IStorage::restoreFromBackup(const BackupPtr &, const String &, const ASTs &, ContextMutablePtr)
|
||||
RestoreFromBackupTaskPtr IStorage::restoreFromBackup(const BackupPtr &, const String &, const ASTs &, ContextMutablePtr)
|
||||
{
|
||||
throw Exception("Table engine " + getName() + " doesn't support restoring", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
@ -72,7 +72,8 @@ class IBackup;
|
||||
using BackupPtr = std::shared_ptr<const IBackup>;
|
||||
class IBackupEntry;
|
||||
using BackupEntries = std::vector<std::pair<String, std::unique_ptr<IBackupEntry>>>;
|
||||
using RestoreDataTasks = std::vector<std::function<void()>>;
|
||||
class IRestoreFromBackupTask;
|
||||
using RestoreFromBackupTaskPtr = std::unique_ptr<IRestoreFromBackupTask>;
|
||||
|
||||
struct ColumnSize
|
||||
{
|
||||
@ -220,7 +221,7 @@ public:
|
||||
virtual BackupEntries backup(const ASTs & partitions, ContextPtr context);
|
||||
|
||||
/// Extract data from the backup and put it to the storage.
|
||||
virtual RestoreDataTasks restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context);
|
||||
virtual RestoreFromBackupTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context);
|
||||
|
||||
/// Returns whether the column is virtual - by default all columns are real.
|
||||
/// Initially reserved virtual column name may be shadowed by real column.
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Backups/BackupEntryFromImmutableFile.h>
|
||||
#include <Backups/BackupEntryFromSmallFile.h>
|
||||
#include <Backups/IBackup.h>
|
||||
#include <Backups/IRestoreFromBackupTask.h>
|
||||
#include <Compression/CompressedReadBuffer.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
@ -3681,40 +3682,78 @@ BackupEntries MergeTreeData::backupDataParts(const DataPartsVector & data_parts)
|
||||
}
|
||||
|
||||
|
||||
RestoreDataTasks MergeTreeData::restoreDataPartsFromBackup(const BackupPtr & backup, const String & data_path_in_backup,
|
||||
const std::unordered_set<String> & partition_ids,
|
||||
SimpleIncrement * increment)
|
||||
class MergeTreeDataRestoreTask : public IRestoreFromBackupTask
|
||||
{
|
||||
RestoreDataTasks restore_tasks;
|
||||
|
||||
Strings part_names = backup->listFiles(data_path_in_backup);
|
||||
for (const String & part_name : part_names)
|
||||
public:
|
||||
MergeTreeDataRestoreTask(
|
||||
const std::shared_ptr<MergeTreeData> & storage_,
|
||||
const BackupPtr & backup_,
|
||||
const String & data_path_in_backup_,
|
||||
const std::unordered_set<String> & partition_ids_,
|
||||
SimpleIncrement * increment_)
|
||||
: storage(storage_)
|
||||
, backup(backup_)
|
||||
, data_path_in_backup(data_path_in_backup_)
|
||||
, partition_ids(partition_ids_)
|
||||
, increment(increment_)
|
||||
{
|
||||
auto part_info = MergeTreePartInfo::tryParsePartName(part_name, format_version);
|
||||
}
|
||||
|
||||
if (!part_info)
|
||||
continue;
|
||||
|
||||
if (!partition_ids.empty() && !partition_ids.contains(part_info->partition_id))
|
||||
continue;
|
||||
|
||||
UInt64 total_size_of_part = 0;
|
||||
Strings filenames = backup->listFiles(data_path_in_backup + part_name + "/", "");
|
||||
for (const String & filename : filenames)
|
||||
total_size_of_part += backup->getFileSize(data_path_in_backup + part_name + "/" + filename);
|
||||
|
||||
std::shared_ptr<IReservation> reservation = getStoragePolicy()->reserveAndCheck(total_size_of_part);
|
||||
|
||||
auto restore_task = [this,
|
||||
backup,
|
||||
data_path_in_backup,
|
||||
part_name,
|
||||
part_info = std::move(part_info),
|
||||
filenames = std::move(filenames),
|
||||
reservation,
|
||||
increment]()
|
||||
RestoreFromBackupTasks run() override
|
||||
{
|
||||
RestoreFromBackupTasks restore_part_tasks;
|
||||
Strings part_names = backup->listFiles(data_path_in_backup);
|
||||
for (const String & part_name : part_names)
|
||||
{
|
||||
const auto part_info = MergeTreePartInfo::tryParsePartName(part_name, storage->format_version);
|
||||
if (!part_info)
|
||||
continue;
|
||||
|
||||
if (!partition_ids.empty() && !partition_ids.contains(part_info->partition_id))
|
||||
continue;
|
||||
|
||||
restore_part_tasks.push_back(
|
||||
std::make_unique<RestorePartTask>(storage, backup, data_path_in_backup, part_name, *part_info, increment));
|
||||
}
|
||||
return restore_part_tasks;
|
||||
}
|
||||
|
||||
private:
|
||||
std::shared_ptr<MergeTreeData> storage;
|
||||
BackupPtr backup;
|
||||
String data_path_in_backup;
|
||||
std::unordered_set<String> partition_ids;
|
||||
SimpleIncrement * increment;
|
||||
|
||||
class RestorePartTask : public IRestoreFromBackupTask
|
||||
{
|
||||
public:
|
||||
RestorePartTask(
|
||||
const std::shared_ptr<MergeTreeData> & storage_,
|
||||
const BackupPtr & backup_,
|
||||
const String & data_path_in_backup_,
|
||||
const String & part_name_,
|
||||
const MergeTreePartInfo & part_info_,
|
||||
SimpleIncrement * increment_)
|
||||
: storage(storage_)
|
||||
, backup(backup_)
|
||||
, data_path_in_backup(data_path_in_backup_)
|
||||
, part_name(part_name_)
|
||||
, part_info(part_info_)
|
||||
, increment(increment_)
|
||||
{
|
||||
}
|
||||
|
||||
RestoreFromBackupTasks run() override
|
||||
{
|
||||
UInt64 total_size_of_part = 0;
|
||||
Strings filenames = backup->listFiles(data_path_in_backup + part_name + "/", "");
|
||||
for (const String & filename : filenames)
|
||||
total_size_of_part += backup->getFileSize(data_path_in_backup + part_name + "/" + filename);
|
||||
|
||||
std::shared_ptr<IReservation> reservation = storage->getStoragePolicy()->reserveAndCheck(total_size_of_part);
|
||||
auto disk = reservation->getDisk();
|
||||
String relative_data_path = storage->getRelativeDataPath();
|
||||
|
||||
auto temp_part_dir_owner = std::make_shared<TemporaryFileOnDisk>(disk, relative_data_path + "restoring_" + part_name + "_");
|
||||
String temp_part_dir = temp_part_dir_owner->getPath();
|
||||
@ -3729,18 +3768,33 @@ RestoreDataTasks MergeTreeData::restoreDataPartsFromBackup(const BackupPtr & bac
|
||||
auto read_buffer = backup_entry->getReadBuffer();
|
||||
auto write_buffer = disk->writeFile(temp_part_dir + "/" + filename);
|
||||
copyData(*read_buffer, *write_buffer);
|
||||
reservation->update(reservation->getSize() - backup_entry->getSize());
|
||||
}
|
||||
|
||||
auto single_disk_volume = std::make_shared<SingleDiskVolume>(disk->getName(), disk, 0);
|
||||
auto part = createPart(part_name, *part_info, single_disk_volume, relative_temp_part_dir);
|
||||
auto part = storage->createPart(part_name, part_info, single_disk_volume, relative_temp_part_dir);
|
||||
part->loadColumnsChecksumsIndexes(false, true);
|
||||
renameTempPartAndAdd(part, increment);
|
||||
};
|
||||
storage->renameTempPartAndAdd(part, increment);
|
||||
return {};
|
||||
}
|
||||
|
||||
restore_tasks.emplace_back(std::move(restore_task));
|
||||
}
|
||||
private:
|
||||
std::shared_ptr<MergeTreeData> storage;
|
||||
BackupPtr backup;
|
||||
String data_path_in_backup;
|
||||
String part_name;
|
||||
MergeTreePartInfo part_info;
|
||||
SimpleIncrement * increment;
|
||||
};
|
||||
};
|
||||
|
||||
return restore_tasks;
|
||||
|
||||
RestoreFromBackupTaskPtr MergeTreeData::restoreDataPartsFromBackup(const BackupPtr & backup, const String & data_path_in_backup,
|
||||
const std::unordered_set<String> & partition_ids,
|
||||
SimpleIncrement * increment)
|
||||
{
|
||||
return std::make_unique<MergeTreeDataRestoreTask>(
|
||||
std::static_pointer_cast<MergeTreeData>(shared_from_this()), backup, data_path_in_backup, partition_ids, increment);
|
||||
}
|
||||
|
||||
|
||||
|
@ -679,7 +679,7 @@ public:
|
||||
static BackupEntries backupDataParts(const DataPartsVector & data_parts);
|
||||
|
||||
/// Extract data from the backup and put it to the storage.
|
||||
RestoreDataTasks restoreDataPartsFromBackup(
|
||||
RestoreFromBackupTaskPtr restoreDataPartsFromBackup(
|
||||
const BackupPtr & backup,
|
||||
const String & data_path_in_backup,
|
||||
const std::unordered_set<String> & partition_ids,
|
||||
|
@ -28,6 +28,7 @@
|
||||
#include <Backups/BackupEntryFromImmutableFile.h>
|
||||
#include <Backups/BackupEntryFromSmallFile.h>
|
||||
#include <Backups/IBackup.h>
|
||||
#include <Backups/IRestoreFromBackupTask.h>
|
||||
#include <Disks/TemporaryFileOnDisk.h>
|
||||
|
||||
#include <cassert>
|
||||
@ -952,43 +953,57 @@ BackupEntries StorageLog::backup(const ASTs & partitions, ContextPtr context)
|
||||
return backup_entries;
|
||||
}
|
||||
|
||||
RestoreDataTasks StorageLog::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context)
|
||||
class LogRestoreTask : public IRestoreFromBackupTask
|
||||
{
|
||||
if (!partitions.empty())
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName());
|
||||
using WriteLock = StorageLog::WriteLock;
|
||||
using Mark = StorageLog::Mark;
|
||||
|
||||
auto restore_task = [this, backup, data_path_in_backup, context]()
|
||||
public:
|
||||
LogRestoreTask(
|
||||
std::shared_ptr<StorageLog> storage_, const BackupPtr & backup_, const String & data_path_in_backup_, ContextMutablePtr context_)
|
||||
: storage(storage_), backup(backup_), data_path_in_backup(data_path_in_backup_), context(context_)
|
||||
{
|
||||
}
|
||||
|
||||
RestoreFromBackupTasks run() override
|
||||
{
|
||||
auto lock_timeout = getLockTimeout(context);
|
||||
WriteLock lock{rwlock, lock_timeout};
|
||||
WriteLock lock{storage->rwlock, lock_timeout};
|
||||
if (!lock)
|
||||
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
|
||||
|
||||
const auto num_data_files = storage->num_data_files;
|
||||
if (!num_data_files)
|
||||
return;
|
||||
return {};
|
||||
|
||||
auto & file_checker = storage->file_checker;
|
||||
|
||||
/// Load the marks if not loaded yet. We have to do that now because we're going to update these marks.
|
||||
loadMarks(lock);
|
||||
storage->loadMarks(lock);
|
||||
|
||||
/// If there were no files, save zero file sizes to be able to rollback in case of error.
|
||||
saveFileSizes(lock);
|
||||
storage->saveFileSizes(lock);
|
||||
|
||||
try
|
||||
{
|
||||
/// Append data files.
|
||||
auto & data_files = storage->data_files;
|
||||
for (const auto & data_file : data_files)
|
||||
{
|
||||
String file_path_in_backup = data_path_in_backup + fileName(data_file.path);
|
||||
auto backup_entry = backup->readFile(file_path_in_backup);
|
||||
const auto & disk = storage->disk;
|
||||
auto in = backup_entry->getReadBuffer();
|
||||
auto out = disk->writeFile(data_file.path, max_compress_block_size, WriteMode::Append);
|
||||
auto out = disk->writeFile(data_file.path, storage->max_compress_block_size, WriteMode::Append);
|
||||
copyData(*in, *out);
|
||||
}
|
||||
|
||||
const bool use_marks_file = storage->use_marks_file;
|
||||
if (use_marks_file)
|
||||
{
|
||||
/// Append marks.
|
||||
size_t num_extra_marks = 0;
|
||||
const auto & marks_file_path = storage->marks_file_path;
|
||||
String file_path_in_backup = data_path_in_backup + fileName(marks_file_path);
|
||||
size_t file_size = backup->getFileSize(file_path_in_backup);
|
||||
if (file_size % (num_data_files * sizeof(Mark)) != 0)
|
||||
@ -1027,19 +1042,34 @@ RestoreDataTasks StorageLog::restoreFromBackup(const BackupPtr & backup, const S
|
||||
}
|
||||
|
||||
/// Finish writing.
|
||||
saveMarks(lock);
|
||||
saveFileSizes(lock);
|
||||
storage->saveMarks(lock);
|
||||
storage->saveFileSizes(lock);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Rollback partial writes.
|
||||
file_checker.repair();
|
||||
removeUnsavedMarks(lock);
|
||||
storage->removeUnsavedMarks(lock);
|
||||
throw;
|
||||
}
|
||||
|
||||
};
|
||||
return {restore_task};
|
||||
return {};
|
||||
}
|
||||
|
||||
private:
|
||||
std::shared_ptr<StorageLog> storage;
|
||||
BackupPtr backup;
|
||||
String data_path_in_backup;
|
||||
ContextMutablePtr context;
|
||||
};
|
||||
|
||||
RestoreFromBackupTaskPtr StorageLog::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context)
|
||||
{
|
||||
if (!partitions.empty())
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName());
|
||||
|
||||
return std::make_unique<LogRestoreTask>(
|
||||
typeid_cast<std::shared_ptr<StorageLog>>(shared_from_this()), backup, data_path_in_backup, context);
|
||||
}
|
||||
|
||||
|
||||
|
@ -23,6 +23,7 @@ class StorageLog final : public shared_ptr_helper<StorageLog>, public IStorage
|
||||
{
|
||||
friend class LogSource;
|
||||
friend class LogSink;
|
||||
friend class LogRestoreTask;
|
||||
friend struct shared_ptr_helper<StorageLog>;
|
||||
|
||||
public:
|
||||
@ -52,7 +53,7 @@ public:
|
||||
ColumnSizeByName getColumnSizes() const override;
|
||||
|
||||
BackupEntries backup(const ASTs & partitions, ContextPtr context) override;
|
||||
RestoreDataTasks restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override;
|
||||
RestoreFromBackupTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override;
|
||||
|
||||
protected:
|
||||
/** Attach the table with the appropriate name, along the appropriate path (with / at the end),
|
||||
|
@ -3,7 +3,7 @@
|
||||
#include <optional>
|
||||
|
||||
#include <base/sort.h>
|
||||
|
||||
#include <Backups/IRestoreFromBackupTask.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
@ -1644,7 +1644,7 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_
|
||||
}
|
||||
|
||||
|
||||
RestoreDataTasks StorageMergeTree::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr local_context)
|
||||
RestoreFromBackupTaskPtr StorageMergeTree::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr local_context)
|
||||
{
|
||||
return restoreDataPartsFromBackup(backup, data_path_in_backup, getPartitionIDsFromQuery(partitions, local_context), &increment);
|
||||
}
|
||||
|
@ -97,7 +97,7 @@ public:
|
||||
|
||||
CheckResults checkData(const ASTPtr & query, ContextPtr context) override;
|
||||
|
||||
RestoreDataTasks restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override;
|
||||
RestoreFromBackupTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override;
|
||||
|
||||
bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override;
|
||||
|
||||
|
@ -38,6 +38,7 @@
|
||||
#include <Backups/BackupEntryFromImmutableFile.h>
|
||||
#include <Backups/BackupEntryFromSmallFile.h>
|
||||
#include <Backups/IBackup.h>
|
||||
#include <Backups/IRestoreFromBackupTask.h>
|
||||
#include <Disks/TemporaryFileOnDisk.h>
|
||||
|
||||
#include <base/insertAtEnd.h>
|
||||
@ -552,37 +553,51 @@ BackupEntries StorageStripeLog::backup(const ASTs & partitions, ContextPtr conte
|
||||
return backup_entries;
|
||||
}
|
||||
|
||||
RestoreDataTasks StorageStripeLog::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context)
|
||||
class StripeLogRestoreTask : public IRestoreFromBackupTask
|
||||
{
|
||||
if (!partitions.empty())
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName());
|
||||
using WriteLock = StorageStripeLog::WriteLock;
|
||||
|
||||
auto restore_task = [this, backup, data_path_in_backup, context]()
|
||||
public:
|
||||
StripeLogRestoreTask(
|
||||
const std::shared_ptr<StorageStripeLog> storage_,
|
||||
const BackupPtr & backup_,
|
||||
const String & data_path_in_backup_,
|
||||
ContextMutablePtr context_)
|
||||
: storage(storage_), backup(backup_), data_path_in_backup(data_path_in_backup_), context(context_)
|
||||
{
|
||||
WriteLock lock{rwlock, getLockTimeout(context)};
|
||||
}
|
||||
|
||||
RestoreFromBackupTasks run() override
|
||||
{
|
||||
WriteLock lock{storage->rwlock, getLockTimeout(context)};
|
||||
if (!lock)
|
||||
throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED);
|
||||
|
||||
auto & file_checker = storage->file_checker;
|
||||
|
||||
/// Load the indices if not loaded yet. We have to do that now because we're going to update these indices.
|
||||
loadIndices(lock);
|
||||
storage->loadIndices(lock);
|
||||
|
||||
/// If there were no files, save zero file sizes to be able to rollback in case of error.
|
||||
saveFileSizes(lock);
|
||||
storage->saveFileSizes(lock);
|
||||
|
||||
try
|
||||
{
|
||||
/// Append the data file.
|
||||
auto old_data_size = file_checker.getFileSize(data_file_path);
|
||||
auto old_data_size = file_checker.getFileSize(storage->data_file_path);
|
||||
{
|
||||
const auto & data_file_path = storage->data_file_path;
|
||||
String file_path_in_backup = data_path_in_backup + fileName(data_file_path);
|
||||
auto backup_entry = backup->readFile(file_path_in_backup);
|
||||
const auto & disk = storage->disk;
|
||||
auto in = backup_entry->getReadBuffer();
|
||||
auto out = disk->writeFile(data_file_path, max_compress_block_size, WriteMode::Append);
|
||||
auto out = disk->writeFile(data_file_path, storage->max_compress_block_size, WriteMode::Append);
|
||||
copyData(*in, *out);
|
||||
}
|
||||
|
||||
/// Append the index.
|
||||
{
|
||||
const auto & index_file_path = storage->index_file_path;
|
||||
String index_path_in_backup = data_path_in_backup + fileName(index_file_path);
|
||||
IndexForNativeFormat extra_indices;
|
||||
auto backup_entry = backup->readFile(index_path_in_backup);
|
||||
@ -597,23 +612,38 @@ RestoreDataTasks StorageStripeLog::restoreFromBackup(const BackupPtr & backup, c
|
||||
column.location.offset_in_compressed_file += old_data_size;
|
||||
}
|
||||
|
||||
insertAtEnd(indices.blocks, std::move(extra_indices.blocks));
|
||||
insertAtEnd(storage->indices.blocks, std::move(extra_indices.blocks));
|
||||
}
|
||||
|
||||
/// Finish writing.
|
||||
saveIndices(lock);
|
||||
saveFileSizes(lock);
|
||||
storage->saveIndices(lock);
|
||||
storage->saveFileSizes(lock);
|
||||
return {};
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Rollback partial writes.
|
||||
file_checker.repair();
|
||||
removeUnsavedIndices(lock);
|
||||
storage->removeUnsavedIndices(lock);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
};
|
||||
return {restore_task};
|
||||
private:
|
||||
std::shared_ptr<StorageStripeLog> storage;
|
||||
BackupPtr backup;
|
||||
String data_path_in_backup;
|
||||
ContextMutablePtr context;
|
||||
};
|
||||
|
||||
|
||||
RestoreFromBackupTaskPtr StorageStripeLog::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context)
|
||||
{
|
||||
if (!partitions.empty())
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName());
|
||||
|
||||
return std::make_unique<StripeLogRestoreTask>(
|
||||
typeid_cast<std::shared_ptr<StorageStripeLog>>(shared_from_this()), backup, data_path_in_backup, context);
|
||||
}
|
||||
|
||||
|
||||
|
@ -23,6 +23,7 @@ class StorageStripeLog final : public shared_ptr_helper<StorageStripeLog>, publi
|
||||
{
|
||||
friend class StripeLogSource;
|
||||
friend class StripeLogSink;
|
||||
friend class StripeLogRestoreTask;
|
||||
friend struct shared_ptr_helper<StorageStripeLog>;
|
||||
|
||||
public:
|
||||
@ -51,7 +52,7 @@ public:
|
||||
void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder&) override;
|
||||
|
||||
BackupEntries backup(const ASTs & partitions, ContextPtr context) override;
|
||||
RestoreDataTasks restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override;
|
||||
RestoreFromBackupTaskPtr restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override;
|
||||
|
||||
protected:
|
||||
StorageStripeLog(
|
||||
|
Loading…
Reference in New Issue
Block a user