mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
Implement BACKUP and RESTORE for MatViews.
This commit is contained in:
parent
ce25afb2e9
commit
51fc556fcd
@ -10,6 +10,13 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
ArchiveBackup::ArchiveBackup(
|
||||
const String & backup_name_,
|
||||
const DiskPtr & disk_,
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Backups/BackupImpl.h>
|
||||
#include <Backups/BackupImpl.h>
|
||||
#include <Backups/BackupFactory.h>
|
||||
#include <Backups/BackupEntryFromMemory.h>
|
||||
#include <Backups/IBackupEntry.h>
|
||||
@ -488,7 +488,6 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry)
|
||||
/// but the entry itself has been moved or renamed.
|
||||
base_size = size;
|
||||
base_checksum = *checksum;
|
||||
base_exists = true;
|
||||
use_base = true;
|
||||
}
|
||||
}
|
||||
|
@ -21,6 +21,7 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_BACKUP_TABLE;
|
||||
extern const int CANNOT_BACKUP_DATABASE;
|
||||
extern const int BACKUP_IS_EMPTY;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
namespace
|
||||
@ -80,7 +81,7 @@ namespace
|
||||
BackupEntries makeBackupEntries() const
|
||||
{
|
||||
/// Check that there are not `different_create_query`. (If it's set it means error.)
|
||||
for (auto & info : databases | boost::adaptors::map_values)
|
||||
for (const auto & info : databases | boost::adaptors::map_values)
|
||||
{
|
||||
if (info.different_create_query)
|
||||
throw Exception(ErrorCodes::CANNOT_BACKUP_DATABASE,
|
||||
@ -89,10 +90,10 @@ namespace
|
||||
}
|
||||
|
||||
BackupEntries res;
|
||||
for (auto & info : databases | boost::adaptors::map_values)
|
||||
for (const auto & info : databases | boost::adaptors::map_values)
|
||||
res.push_back(makeBackupEntryForMetadata(*info.create_query));
|
||||
|
||||
for (auto & info : tables | boost::adaptors::map_values)
|
||||
for (const auto & info : tables | boost::adaptors::map_values)
|
||||
{
|
||||
res.push_back(makeBackupEntryForMetadata(*info.create_query));
|
||||
if (info.has_data)
|
||||
@ -142,7 +143,7 @@ namespace
|
||||
throw Exception(ErrorCodes::CANNOT_BACKUP_TABLE, "Cannot backup the {} twice", formatTableNameOrTemporaryTableName(new_table_name));
|
||||
|
||||
/// Make a create query for this table.
|
||||
auto create_query = renameInCreateQuery(database->getCreateTableQuery(table_name_.second, context));
|
||||
auto create_query = prepareCreateQueryForBackup(database->getCreateTableQuery(table_name_.second, context));
|
||||
|
||||
bool has_data = storage->hasDataToBackup() && !backup_settings.structure_only;
|
||||
if (has_data)
|
||||
@ -165,7 +166,7 @@ namespace
|
||||
if (!databases.contains(new_table_name.first))
|
||||
{
|
||||
/// Add a create query to backup the database if we haven't done it yet.
|
||||
auto create_db_query = renameInCreateQuery(database->getCreateDatabaseQuery());
|
||||
auto create_db_query = prepareCreateQueryForBackup(database->getCreateDatabaseQuery());
|
||||
create_db_query->setDatabase(new_table_name.first);
|
||||
|
||||
CreateDatabaseInfo info_db;
|
||||
@ -181,7 +182,7 @@ namespace
|
||||
auto & info_db = databases[new_table_name.first];
|
||||
if (!info_db.is_explicit && (info_db.original_name != table_name_.first) && !info_db.different_create_query)
|
||||
{
|
||||
auto create_db_query = renameInCreateQuery(table_.first->getCreateDatabaseQuery());
|
||||
auto create_db_query = prepareCreateQueryForBackup(table_.first->getCreateDatabaseQuery());
|
||||
create_db_query->setDatabase(new_table_name.first);
|
||||
if (!areDatabaseDefinitionsSame(*info_db.create_query, *create_db_query))
|
||||
info_db.different_create_query = create_db_query;
|
||||
@ -210,7 +211,7 @@ namespace
|
||||
if (!isSystemOrTemporaryDatabase(database_name_))
|
||||
{
|
||||
/// Make a create query for this database.
|
||||
auto create_db_query = renameInCreateQuery(database_->getCreateDatabaseQuery());
|
||||
auto create_db_query = prepareCreateQueryForBackup(database_->getCreateDatabaseQuery());
|
||||
|
||||
CreateDatabaseInfo info_db;
|
||||
info_db.create_query = create_db_query;
|
||||
@ -245,9 +246,14 @@ namespace
|
||||
}
|
||||
|
||||
/// Do renaming in the create query according to the renaming config.
|
||||
std::shared_ptr<ASTCreateQuery> renameInCreateQuery(const ASTPtr & ast) const
|
||||
std::shared_ptr<ASTCreateQuery> prepareCreateQueryForBackup(const ASTPtr & ast) const
|
||||
{
|
||||
return typeid_cast<std::shared_ptr<ASTCreateQuery>>(::DB::renameInCreateQuery(ast, context, renaming_settings));
|
||||
ASTPtr query = ast;
|
||||
::DB::renameInCreateQuery(query, context, renaming_settings);
|
||||
auto create_query = typeid_cast<std::shared_ptr<ASTCreateQuery>>(query);
|
||||
create_query->uuid = UUIDHelpers::Nil;
|
||||
create_query->to_inner_uuid = UUIDHelpers::Nil;
|
||||
return create_query;
|
||||
}
|
||||
|
||||
static bool isSystemOrTemporaryDatabase(const String & database_name)
|
||||
|
@ -22,10 +22,13 @@ namespace
|
||||
return clone;
|
||||
};
|
||||
|
||||
/// Remove UUID.
|
||||
/// Remove UUIDs.
|
||||
if (res->uuid != UUIDHelpers::Nil)
|
||||
get_clone()->uuid = UUIDHelpers::Nil;
|
||||
|
||||
if (res->to_inner_uuid != UUIDHelpers::Nil)
|
||||
get_clone()->to_inner_uuid = UUIDHelpers::Nil;
|
||||
|
||||
/// Clear IF NOT EXISTS flag.
|
||||
if (res->if_not_exists)
|
||||
get_clone()->if_not_exists = false;
|
||||
@ -45,7 +48,8 @@ bool areTableDefinitionsSame(const IAST & table1, const IAST & table2)
|
||||
if (!ast2 || !ast2->table)
|
||||
return false;
|
||||
|
||||
if ((ast1->uuid != ast2->uuid) || (ast1->if_not_exists != ast2->if_not_exists))
|
||||
if ((ast1->uuid != ast2->uuid) || (ast1->to_inner_uuid != ast2->to_inner_uuid) ||
|
||||
(ast1->if_not_exists != ast2->if_not_exists))
|
||||
{
|
||||
ast1 = prepareDDLToCompare(*ast1);
|
||||
ast2 = prepareDDLToCompare(*ast2);
|
||||
|
@ -59,8 +59,6 @@ namespace
|
||||
else
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Database name specified in the CREATE DATABASE query must not be empty");
|
||||
|
||||
create.uuid = UUIDHelpers::Nil;
|
||||
|
||||
if (!create.as_table.empty() && !create.as_database.empty())
|
||||
std::tie(create.as_database, create.as_table) = data.renaming_settings.getNewTableName({create.as_database, create.as_table});
|
||||
|
||||
@ -365,19 +363,16 @@ void DDLRenamingVisitor::visit(ASTPtr & ast, const Data & data)
|
||||
visitDictionary(*dictionary, data);
|
||||
}
|
||||
|
||||
ASTPtr renameInCreateQuery(const ASTPtr & ast, const ContextPtr & global_context, const DDLRenamingSettings & renaming_settings)
|
||||
void renameInCreateQuery(ASTPtr & ast, const ContextPtr & global_context, const DDLRenamingSettings & renaming_settings)
|
||||
{
|
||||
auto new_ast = ast->clone();
|
||||
try
|
||||
{
|
||||
DDLRenamingVisitor::Data data{renaming_settings, global_context};
|
||||
DDLRenamingVisitor::Visitor{data}.visit(new_ast);
|
||||
return new_ast;
|
||||
DDLRenamingVisitor::Visitor{data}.visit(ast);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException("Backup", "Error while renaming in AST");
|
||||
return ast;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -40,7 +40,7 @@ private:
|
||||
|
||||
|
||||
/// Changes names in AST according to the renaming settings.
|
||||
ASTPtr renameInCreateQuery(const ASTPtr & ast, const ContextPtr & global_context, const DDLRenamingSettings & config);
|
||||
void renameInCreateQuery(ASTPtr & ast, const ContextPtr & global_context, const DDLRenamingSettings & renaming_settings);
|
||||
|
||||
/// Visits ASTCreateQuery and changes names of tables and databases according to passed DDLRenamingConfig.
|
||||
class DDLRenamingVisitor
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <memory>
|
||||
#include <optional>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -297,7 +297,7 @@ namespace
|
||||
RestoreTasks 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)
|
||||
for (const auto & info : databases | boost::adaptors::map_values)
|
||||
{
|
||||
if (info.different_create_query)
|
||||
throw Exception(ErrorCodes::CANNOT_RESTORE_DATABASE,
|
||||
@ -308,12 +308,12 @@ namespace
|
||||
auto restore_settings_ptr = std::make_shared<const RestoreSettings>(restore_settings);
|
||||
|
||||
RestoreTasks res;
|
||||
for (auto & info : databases | boost::adaptors::map_values)
|
||||
for (const auto & info : databases | boost::adaptors::map_values)
|
||||
res.push_back(std::make_unique<RestoreDatabaseTask>(context, info.create_query, restore_settings_ptr,
|
||||
/* ignore_if_database_def_differs = */ !info.is_explicit));
|
||||
|
||||
/// TODO: We need to restore tables according to their dependencies.
|
||||
for (auto & info : tables | boost::adaptors::map_values)
|
||||
for (const auto & info : tables | boost::adaptors::map_values)
|
||||
res.push_back(std::make_unique<RestoreTableTask>(context, info.create_query, info.partitions, backup, info.name_in_backup, restore_settings_ptr));
|
||||
|
||||
return res;
|
||||
@ -487,7 +487,10 @@ namespace
|
||||
/// 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, context, renaming_settings));
|
||||
ASTPtr query = ast;
|
||||
::DB::renameInCreateQuery(query, context, renaming_settings);
|
||||
auto create_query = typeid_cast<std::shared_ptr<ASTCreateQuery>>(query);
|
||||
return create_query;
|
||||
}
|
||||
|
||||
static bool isSystemOrTemporaryDatabase(const String & database_name)
|
||||
|
@ -6,7 +6,6 @@
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/createReadBufferFromFileBase.h>
|
||||
#include <base/JSON.h>
|
||||
|
||||
|
||||
@ -161,7 +160,7 @@ void FileChecker::load()
|
||||
if (!fileReallyExists(files_info_path))
|
||||
return;
|
||||
|
||||
std::unique_ptr<ReadBuffer> in = disk ? disk->readFile(files_info_path) : createReadBufferFromFileBase(files_info_path, {});
|
||||
std::unique_ptr<ReadBuffer> in = disk ? disk->readFile(files_info_path) : std::make_unique<ReadBufferFromFile>(files_info_path);
|
||||
WriteBufferFromOwnString out;
|
||||
|
||||
/// The JSON library does not support whitespace. We delete them. Inefficient.
|
||||
|
@ -6,7 +6,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Returns true if a specified path has one of the registed file extensions for an archive.
|
||||
/// Returns true if a specified path has one of the registered file extensions for an archive.
|
||||
bool hasRegisteredArchiveFileExtension(const String & path);
|
||||
|
||||
}
|
||||
|
@ -25,6 +25,9 @@
|
||||
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
|
||||
#include <Processors/Sinks/SinkToStorage.h>
|
||||
|
||||
#include <Backups/IBackupEntry.h>
|
||||
#include <Backups/IRestoreTask.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -430,6 +433,20 @@ Strings StorageMaterializedView::getDataPaths() const
|
||||
return {};
|
||||
}
|
||||
|
||||
BackupEntries StorageMaterializedView::backupData(ContextPtr context_, const ASTs & partitions_)
|
||||
{
|
||||
if (!hasInnerTable())
|
||||
return {};
|
||||
return getTargetTable()->backupData(context_, partitions_);
|
||||
}
|
||||
|
||||
RestoreTaskPtr StorageMaterializedView::restoreData(ContextMutablePtr context_, const ASTs & partitions_, const BackupPtr & backup_, const String & data_path_in_backup_, const StorageRestoreSettings & restore_settings_)
|
||||
{
|
||||
if (!hasInnerTable())
|
||||
return {};
|
||||
return getTargetTable()->restoreData(context_, partitions_, backup_, data_path_in_backup_, restore_settings_);
|
||||
}
|
||||
|
||||
ActionLock StorageMaterializedView::getActionLock(StorageActionBlockType type)
|
||||
{
|
||||
if (has_inner_table)
|
||||
|
@ -97,6 +97,10 @@ public:
|
||||
|
||||
Strings getDataPaths() const override;
|
||||
|
||||
bool hasDataToBackup() const override { return hasInnerTable(); }
|
||||
BackupEntries backupData(ContextPtr context_, const ASTs & partitions_) override;
|
||||
RestoreTaskPtr restoreData(ContextMutablePtr context_, const ASTs & partitions_, const BackupPtr & backup, const String & data_path_in_backup_, const StorageRestoreSettings & restore_settings_) override;
|
||||
|
||||
private:
|
||||
/// Will be initialized in constructor
|
||||
StorageID target_table_id = StorageID::createEmpty();
|
||||
|
@ -24,8 +24,8 @@
|
||||
#include <Backups/IBackup.h>
|
||||
#include <Backups/IBackupEntriesBatch.h>
|
||||
#include <Backups/IRestoreTask.h>
|
||||
#include <Disks/IO/createReadBufferFromFileBase.h>
|
||||
#include <IO/copyData.h>
|
||||
#include <IO/createReadBufferFromFileBase.h>
|
||||
#include <Poco/TemporaryFile.h>
|
||||
|
||||
|
||||
@ -35,6 +35,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
|
||||
|
@ -99,6 +99,18 @@ def test_backup_table_under_another_name():
|
||||
assert instance.query("SELECT count(), sum(x) FROM test.table2") == "100\t4950\n"
|
||||
|
||||
|
||||
def test_materialized_view():
|
||||
backup_name = new_backup_name()
|
||||
instance.query("CREATE MATERIALIZED VIEW mv_1(x UInt8) ENGINE=MergeTree ORDER BY tuple() POPULATE AS SELECT 1 AS x")
|
||||
|
||||
instance.query(f"BACKUP TABLE mv_1 TO {backup_name}")
|
||||
instance.query("DROP TABLE mv_1")
|
||||
instance.query(f"RESTORE TABLE mv_1 FROM {backup_name}")
|
||||
|
||||
assert instance.query("SELECT * FROM mv_1") == "1\n"
|
||||
instance.query("DROP TABLE mv_1")
|
||||
|
||||
|
||||
def test_incremental_backup():
|
||||
backup_name = new_backup_name()
|
||||
incremental_backup_name = new_backup_name()
|
||||
|
Loading…
Reference in New Issue
Block a user