mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 01:22:04 +00:00
Add test for partition clause. More checks for data compatibility on restore.
This commit is contained in:
parent
11b51d2878
commit
031ca28fdc
@ -25,6 +25,7 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_RESTORE_TABLE;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
@ -317,12 +318,21 @@ AccessRestoreTask::AccessRestoreTask(
|
||||
|
||||
AccessRestoreTask::~AccessRestoreTask() = default;
|
||||
|
||||
void AccessRestoreTask::addDataPath(const String & data_path)
|
||||
void AccessRestoreTask::addDataPath(const String & data_path, const QualifiedTableName & table_name_for_logs)
|
||||
{
|
||||
if (!data_paths.emplace(data_path).second)
|
||||
return;
|
||||
|
||||
if (!backup->hasFiles(data_path))
|
||||
return;
|
||||
|
||||
String file_path = fs::path{data_path} / "access.txt";
|
||||
if (!backup->fileExists(file_path))
|
||||
{
|
||||
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: File {} in backup is required",
|
||||
table_name_for_logs.getFullName(), file_path);
|
||||
}
|
||||
|
||||
auto backup_entry = backup->readFile(file_path);
|
||||
auto ab = AccessEntitiesInBackup::fromBackupEntry(*backup_entry, file_path);
|
||||
|
||||
|
@ -17,6 +17,7 @@ class IRestoreCoordination;
|
||||
struct IAccessEntity;
|
||||
using AccessEntityPtr = std::shared_ptr<const IAccessEntity>;
|
||||
class AccessRightsElements;
|
||||
struct QualifiedTableName;
|
||||
|
||||
|
||||
/// Makes a backup of access entities of a specified type.
|
||||
@ -35,7 +36,7 @@ public:
|
||||
~AccessRestoreTask();
|
||||
|
||||
/// Adds a data path to loads access entities from.
|
||||
void addDataPath(const String & data_path);
|
||||
void addDataPath(const String & data_path, const QualifiedTableName & table_name_for_logs);
|
||||
bool hasDataPath(const String & data_path) const;
|
||||
|
||||
/// Checks that the current user can do restoring.
|
||||
|
@ -478,6 +478,7 @@ void BackupEntriesCollector::gatherTablesMetadata()
|
||||
for (const auto & db_table : db_tables)
|
||||
{
|
||||
const auto & create_table_query = db_table.first;
|
||||
const auto & storage = db_table.second;
|
||||
const auto & create = create_table_query->as<const ASTCreateQuery &>();
|
||||
String table_name = create.getTable();
|
||||
|
||||
@ -499,14 +500,28 @@ void BackupEntriesCollector::gatherTablesMetadata()
|
||||
/// Add information to `table_infos`.
|
||||
auto & res_table_info = table_infos[QualifiedTableName{database_name, table_name}];
|
||||
res_table_info.database = database;
|
||||
res_table_info.storage = db_table.second;
|
||||
res_table_info.storage = storage;
|
||||
res_table_info.create_table_query = create_table_query;
|
||||
res_table_info.metadata_path_in_backup = metadata_path_in_backup;
|
||||
res_table_info.data_path_in_backup = data_path_in_backup;
|
||||
|
||||
auto partitions_it = database_info.tables.find(table_name);
|
||||
if (partitions_it != database_info.tables.end())
|
||||
res_table_info.partitions = partitions_it->second.partitions;
|
||||
if (!backup_settings.structure_only)
|
||||
{
|
||||
auto it = database_info.tables.find(table_name);
|
||||
if (it != database_info.tables.end())
|
||||
{
|
||||
const auto & partitions = it->second.partitions;
|
||||
if (partitions && !storage->supportsBackupPartition())
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::CANNOT_BACKUP_TABLE,
|
||||
"Table engine {} doesn't support partitions, cannot backup {}",
|
||||
storage->getName(),
|
||||
tableNameWithTypeToString(database_name, table_name, false));
|
||||
}
|
||||
res_table_info.partitions = partitions;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -724,13 +739,4 @@ void BackupEntriesCollector::runPostTasks()
|
||||
}
|
||||
}
|
||||
|
||||
void BackupEntriesCollector::throwPartitionsNotSupported(const StorageID & storage_id, const String & table_engine)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::CANNOT_BACKUP_TABLE,
|
||||
"Table engine {} doesn't support partitions, cannot backup table {}",
|
||||
table_engine,
|
||||
storage_id.getFullTableName());
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -50,9 +50,6 @@ public:
|
||||
/// 1) we need to join (in a backup) the data of replicated tables gathered on different hosts.
|
||||
void addPostTask(std::function<void()> task);
|
||||
|
||||
/// Throws an exception that a specified table engine doesn't support partitions.
|
||||
[[noreturn]] static void throwPartitionsNotSupported(const StorageID & storage_id, const String & table_engine);
|
||||
|
||||
private:
|
||||
void calculateRootPathInBackup();
|
||||
|
||||
|
@ -34,9 +34,6 @@ namespace
|
||||
/// back into "{uuid}", and also we probably can remove the zookeeper path and replica name if they're default.
|
||||
/// So we're kind of reverting what we had done to the table's definition in registerStorageMergeTree.cpp before we created this table.
|
||||
auto & create = data.create_query->as<ASTCreateQuery &>();
|
||||
if (create.uuid == UUIDHelpers::Nil)
|
||||
return;
|
||||
|
||||
auto & engine = *storage.engine;
|
||||
|
||||
auto * engine_args_ast = typeid_cast<ASTExpressionList *>(engine.arguments.get());
|
||||
@ -54,9 +51,12 @@ namespace
|
||||
{
|
||||
String & zookeeper_path_arg = zookeeper_path_ast->value.get<String>();
|
||||
String & replica_name_arg = replica_name_ast->value.get<String>();
|
||||
String table_uuid_str = toString(create.uuid);
|
||||
if (size_t uuid_pos = zookeeper_path_arg.find(table_uuid_str); uuid_pos != String::npos)
|
||||
zookeeper_path_arg.replace(uuid_pos, table_uuid_str.size(), "{uuid}");
|
||||
if (create.uuid != UUIDHelpers::Nil)
|
||||
{
|
||||
String table_uuid_str = toString(create.uuid);
|
||||
if (size_t uuid_pos = zookeeper_path_arg.find(table_uuid_str); uuid_pos != String::npos)
|
||||
zookeeper_path_arg.replace(uuid_pos, table_uuid_str.size(), "{uuid}");
|
||||
}
|
||||
const auto & config = data.global_context->getConfigRef();
|
||||
if ((zookeeper_path_arg == StorageReplicatedMergeTree::getDefaultZooKeeperPath(config))
|
||||
&& (replica_name_arg == StorageReplicatedMergeTree::getDefaultReplicaName(config))
|
||||
|
@ -370,8 +370,9 @@ void RestorerFromBackup::findTableInBackup(const QualifiedTableName & table_name
|
||||
TableInfo & res_table_info = table_infos[table_name];
|
||||
res_table_info.create_table_query = create_table_query;
|
||||
res_table_info.is_predefined_table = DatabaseCatalog::instance().isPredefinedTable(StorageID{table_name.database, table_name.table});
|
||||
res_table_info.data_path_in_backup = data_path_in_backup;
|
||||
res_table_info.dependencies = getDependenciesSetFromCreateQuery(context->getGlobalContext(), table_name, create_table_query);
|
||||
res_table_info.has_data = backup->hasFiles(data_path_in_backup);
|
||||
res_table_info.data_path_in_backup = data_path_in_backup;
|
||||
|
||||
if (partitions)
|
||||
{
|
||||
@ -384,7 +385,7 @@ void RestorerFromBackup::findTableInBackup(const QualifiedTableName & table_name
|
||||
{
|
||||
if (!access_restore_task)
|
||||
access_restore_task = std::make_shared<AccessRestoreTask>(backup, restore_settings, restore_coordination);
|
||||
access_restore_task->addDataPath(data_path_in_backup);
|
||||
access_restore_task->addDataPath(data_path_in_backup, table_name);
|
||||
}
|
||||
}
|
||||
|
||||
@ -510,7 +511,7 @@ void RestorerFromBackup::checkAccessForObjectsFoundInBackup() const
|
||||
if (isSystemFunctionsTableName(table_name))
|
||||
{
|
||||
/// CREATE_FUNCTION privilege is required to restore the "system.functions" table.
|
||||
if (!restore_settings.structure_only && backup->hasFiles(table_info.data_path_in_backup))
|
||||
if (!restore_settings.structure_only && table_info.has_data)
|
||||
required_access.emplace_back(AccessType::CREATE_FUNCTION);
|
||||
}
|
||||
/// Privileges required to restore ACL system tables are checked separately
|
||||
@ -538,8 +539,7 @@ void RestorerFromBackup::checkAccessForObjectsFoundInBackup() const
|
||||
flags |= AccessType::CREATE_TABLE;
|
||||
}
|
||||
|
||||
if (!restore_settings.structure_only && !create.is_dictionary && !create.is_ordinary_view
|
||||
&& backup->hasFiles(table_info.data_path_in_backup))
|
||||
if (!restore_settings.structure_only && table_info.has_data)
|
||||
{
|
||||
flags |= AccessType::INSERT;
|
||||
}
|
||||
@ -685,6 +685,15 @@ void RestorerFromBackup::createTables()
|
||||
{
|
||||
const auto & data_path_in_backup = table_info.data_path_in_backup;
|
||||
const auto & partitions = table_info.partitions;
|
||||
if (partitions && !storage->supportsBackupPartition())
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::CANNOT_RESTORE_TABLE,
|
||||
"Table engine {} doesn't support partitions, cannot restore {}",
|
||||
storage->getName(),
|
||||
tableNameWithTypeToString(table_name.database, table_name.table, false));
|
||||
}
|
||||
|
||||
storage->restoreDataFromBackup(*this, data_path_in_backup, partitions);
|
||||
}
|
||||
}
|
||||
@ -795,15 +804,6 @@ RestorerFromBackup::DataRestoreTasks RestorerFromBackup::getDataRestoreTasks()
|
||||
return res_tasks;
|
||||
}
|
||||
|
||||
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(
|
||||
|
@ -59,9 +59,6 @@ public:
|
||||
/// Checks that a specified path is already registered to be used for restoring access control.
|
||||
void checkPathInBackupIsRegisteredToRestoreAccess(const String & path);
|
||||
|
||||
/// Throws an exception that a specified table engine doesn't support partitions.
|
||||
[[noreturn]] static void throwPartitionsNotSupported(const StorageID & storage_id, const String & table_engine);
|
||||
|
||||
/// Throws an exception that a specified table is already non-empty.
|
||||
[[noreturn]] static void throwTableIsNotEmpty(const StorageID & storage_id);
|
||||
|
||||
@ -104,9 +101,10 @@ private:
|
||||
{
|
||||
ASTPtr create_table_query;
|
||||
bool is_predefined_table = false;
|
||||
std::optional<ASTs> partitions;
|
||||
std::filesystem::path data_path_in_backup;
|
||||
std::unordered_set<QualifiedTableName> dependencies;
|
||||
bool has_data = false;
|
||||
std::filesystem::path data_path_in_backup;
|
||||
std::optional<ASTs> partitions;
|
||||
bool created = false;
|
||||
StoragePtr storage;
|
||||
TableLockHolder table_lock;
|
||||
|
@ -14,7 +14,8 @@
|
||||
#include <Processors/QueryPlan/ReadFromPreparedSource.h>
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
#include <Storages/AlterCommands.h>
|
||||
#include <Backups/BackupEntriesCollector.h>
|
||||
#include <Backups/RestorerFromBackup.h>
|
||||
#include <Backups/IBackup.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -24,7 +25,7 @@ namespace ErrorCodes
|
||||
extern const int TABLE_IS_DROPPED;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
extern const int DEADLOCK_AVOIDED;
|
||||
extern const int INCONSISTENT_METADATA_FOR_BACKUP;
|
||||
extern const int CANNOT_RESTORE_TABLE;
|
||||
}
|
||||
|
||||
bool IStorage::isVirtualColumn(const String & column_name, const StorageMetadataPtr & metadata_snapshot) const
|
||||
@ -257,8 +258,13 @@ void IStorage::backupData(BackupEntriesCollector &, const String &, const std::o
|
||||
{
|
||||
}
|
||||
|
||||
void IStorage::restoreDataFromBackup(RestorerFromBackup &, const String &, const std::optional<ASTs> &)
|
||||
void IStorage::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> &)
|
||||
{
|
||||
/// If an inherited class doesn't override restoreDataFromBackup() that means it doesn't backup any data.
|
||||
auto filenames = restorer.getBackup()->listFiles(data_path_in_backup);
|
||||
if (!filenames.empty())
|
||||
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: Folder {} in backup must be empty",
|
||||
getStorageID().getFullTableName(), data_path_in_backup);
|
||||
}
|
||||
|
||||
std::string PrewhereInfo::dump() const
|
||||
|
@ -232,6 +232,9 @@ public:
|
||||
/// Extracts data from the backup and put it to the storage.
|
||||
virtual void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & partitions);
|
||||
|
||||
/// Returns true if the storage supports backup/restore for specific partitions.
|
||||
virtual bool supportsBackupPartition() const { return false; }
|
||||
|
||||
private:
|
||||
|
||||
StorageID storage_id;
|
||||
|
@ -150,6 +150,7 @@ namespace ErrorCodes
|
||||
extern const int SUPPORT_IS_DISABLED;
|
||||
extern const int TOO_MANY_SIMULTANEOUS_QUERIES;
|
||||
extern const int INCORRECT_QUERY;
|
||||
extern const int CANNOT_RESTORE_TABLE;
|
||||
}
|
||||
|
||||
static void checkSampleExpression(const StorageInMemoryMetadata & metadata, bool allow_sampling_expression_not_in_primary_key, bool check_sample_column_is_correct)
|
||||
@ -4092,7 +4093,10 @@ void MergeTreeData::restorePartsFromBackup(RestorerFromBackup & restorer, const
|
||||
{
|
||||
const auto part_info = MergeTreePartInfo::tryParsePartName(part_name, format_version);
|
||||
if (!part_info)
|
||||
continue;
|
||||
{
|
||||
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: File name {} doesn't look like the name of a part",
|
||||
getStorageID().getFullTableName(), String{data_path_in_backup_fs / part_name});
|
||||
}
|
||||
|
||||
if (partition_ids && !partition_ids->contains(part_info->partition_id))
|
||||
continue;
|
||||
|
@ -716,6 +716,9 @@ public:
|
||||
/// Extract data from the backup and put it to the storage.
|
||||
void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & partitions) override;
|
||||
|
||||
/// Returns true if the storage supports backup/restore for specific partitions.
|
||||
bool supportsBackupPartition() const override { return true; }
|
||||
|
||||
/// Moves partition to specified Disk
|
||||
void movePartitionToDisk(const ASTPtr & partition, const String & name, bool moving_part, ContextPtr context);
|
||||
|
||||
|
@ -51,6 +51,7 @@ namespace ErrorCodes
|
||||
extern const int SIZES_OF_MARKS_FILES_ARE_INCONSISTENT;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int INCORRECT_FILE_NAME;
|
||||
extern const int CANNOT_RESTORE_TABLE;
|
||||
}
|
||||
|
||||
/// NOTE: The lock `StorageLog::rwlock` is NOT kept locked while reading,
|
||||
@ -921,11 +922,8 @@ std::optional<UInt64> StorageLog::totalBytes(const Settings &) const
|
||||
return total_bytes;
|
||||
}
|
||||
|
||||
void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & partitions)
|
||||
void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
|
||||
{
|
||||
if (partitions)
|
||||
BackupEntriesCollector::throwPartitionsNotSupported(getStorageID(), getName());
|
||||
|
||||
auto lock_timeout = getLockTimeout(backup_entries_collector.getContext());
|
||||
loadMarks(lock_timeout);
|
||||
|
||||
@ -986,16 +984,16 @@ void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, c
|
||||
}
|
||||
}
|
||||
|
||||
void StorageLog::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & partitions)
|
||||
void StorageLog::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
|
||||
{
|
||||
if (partitions)
|
||||
RestorerFromBackup::throwPartitionsNotSupported(getStorageID(), getName());
|
||||
auto backup = restorer.getBackup();
|
||||
if (!backup->hasFiles(data_path_in_backup))
|
||||
return;
|
||||
|
||||
if (!num_data_files)
|
||||
return;
|
||||
|
||||
auto backup = restorer.getBackup();
|
||||
if (!restorer.isNonEmptyTableAllowed() && total_bytes && backup->hasFiles(data_path_in_backup))
|
||||
if (!restorer.isNonEmptyTableAllowed() && total_bytes)
|
||||
RestorerFromBackup::throwTableIsNotEmpty(getStorageID());
|
||||
|
||||
auto lock_timeout = getLockTimeout(restorer.getContext());
|
||||
@ -1024,6 +1022,11 @@ void StorageLog::restoreDataImpl(const BackupPtr & backup, const String & data_p
|
||||
for (const auto & data_file : data_files)
|
||||
{
|
||||
String file_path_in_backup = data_path_in_backup_fs / fileName(data_file.path);
|
||||
if (!backup->fileExists(file_path_in_backup))
|
||||
{
|
||||
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: File {} in backup is required",
|
||||
getStorageID().getFullTableName(), file_path_in_backup);
|
||||
}
|
||||
auto backup_entry = backup->readFile(file_path_in_backup);
|
||||
auto in = backup_entry->getReadBuffer();
|
||||
auto out = disk->writeFile(data_file.path, max_compress_block_size, WriteMode::Append);
|
||||
@ -1035,6 +1038,11 @@ void StorageLog::restoreDataImpl(const BackupPtr & backup, const String & data_p
|
||||
/// Append marks.
|
||||
size_t num_extra_marks = 0;
|
||||
String file_path_in_backup = data_path_in_backup_fs / fileName(marks_file_path);
|
||||
if (!backup->fileExists(file_path_in_backup))
|
||||
{
|
||||
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: File {} in backup is required",
|
||||
getStorageID().getFullTableName(), file_path_in_backup);
|
||||
}
|
||||
size_t file_size = backup->getFileSize(file_path_in_backup);
|
||||
if (file_size % (num_data_files * sizeof(Mark)) != 0)
|
||||
throw Exception("Size of marks file is inconsistent", ErrorCodes::SIZES_OF_MARKS_FILES_ARE_INCONSISTENT);
|
||||
|
@ -421,6 +421,13 @@ void StorageMaterializedView::restoreDataFromBackup(RestorerFromBackup & restore
|
||||
return getTargetTable()->restoreDataFromBackup(restorer, data_path_in_backup, partitions);
|
||||
}
|
||||
|
||||
bool StorageMaterializedView::supportsBackupPartition() const
|
||||
{
|
||||
if (hasInnerTable())
|
||||
return getTargetTable()->supportsBackupPartition();
|
||||
return false;
|
||||
}
|
||||
|
||||
std::optional<UInt64> StorageMaterializedView::totalRows(const Settings & settings) const
|
||||
{
|
||||
if (hasInnerTable())
|
||||
|
@ -97,6 +97,7 @@ public:
|
||||
|
||||
void backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & partitions) override;
|
||||
void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & partitions) override;
|
||||
bool supportsBackupPartition() const override;
|
||||
|
||||
std::optional<UInt64> totalRows(const Settings & settings) const override;
|
||||
std::optional<UInt64> totalBytes(const Settings & settings) const override;
|
||||
|
@ -38,6 +38,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int CANNOT_RESTORE_TABLE;
|
||||
}
|
||||
|
||||
|
||||
@ -479,24 +480,21 @@ namespace
|
||||
};
|
||||
}
|
||||
|
||||
void StorageMemory::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & partitions)
|
||||
void StorageMemory::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
|
||||
{
|
||||
if (partitions)
|
||||
BackupEntriesCollector::throwPartitionsNotSupported(getStorageID(), getName());
|
||||
|
||||
auto max_compress_block_size = backup_entries_collector.getContext()->getSettingsRef().max_compress_block_size;
|
||||
backup_entries_collector.addBackupEntries(
|
||||
std::make_shared<MemoryBackupEntriesBatch>(getInMemoryMetadataPtr(), data.get(), data_path_in_backup, max_compress_block_size)
|
||||
->getBackupEntries());
|
||||
}
|
||||
|
||||
void StorageMemory::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & partitions)
|
||||
void StorageMemory::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
|
||||
{
|
||||
if (partitions)
|
||||
RestorerFromBackup::throwPartitionsNotSupported(getStorageID(), getName());
|
||||
|
||||
auto backup = restorer.getBackup();
|
||||
if (!restorer.isNonEmptyTableAllowed() && total_size_bytes && backup->hasFiles(data_path_in_backup))
|
||||
if (!backup->hasFiles(data_path_in_backup))
|
||||
return;
|
||||
|
||||
if (!restorer.isNonEmptyTableAllowed() && total_size_bytes)
|
||||
RestorerFromBackup::throwTableIsNotEmpty(getStorageID());
|
||||
|
||||
restorer.addDataRestoreTask(
|
||||
@ -514,6 +512,11 @@ void StorageMemory::restoreDataImpl(const BackupPtr & backup, const String & dat
|
||||
IndexForNativeFormat index;
|
||||
{
|
||||
String index_file_path = data_path_in_backup_fs / "index.mrk";
|
||||
if (!backup->fileExists(index_file_path))
|
||||
{
|
||||
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: File {} in backup is required",
|
||||
getStorageID().getFullTableName(), index_file_path);
|
||||
}
|
||||
auto backup_entry = backup->readFile(index_file_path);
|
||||
auto in = backup_entry->getReadBuffer();
|
||||
CompressedReadBuffer compressed_in{*in};
|
||||
@ -526,6 +529,11 @@ void StorageMemory::restoreDataImpl(const BackupPtr & backup, const String & dat
|
||||
size_t new_rows = 0;
|
||||
{
|
||||
String data_file_path = data_path_in_backup_fs / "data.bin";
|
||||
if (!backup->fileExists(data_file_path))
|
||||
{
|
||||
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: File {} in backup is required",
|
||||
getStorageID().getFullTableName(), data_file_path);
|
||||
}
|
||||
auto backup_entry = backup->readFile(data_file_path);
|
||||
std::unique_ptr<ReadBuffer> in = backup_entry->getReadBuffer();
|
||||
std::optional<Poco::TemporaryFile> temp_data_copy;
|
||||
|
@ -55,6 +55,7 @@ namespace ErrorCodes
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int INCORRECT_FILE_NAME;
|
||||
extern const int TIMEOUT_EXCEEDED;
|
||||
extern const int CANNOT_RESTORE_TABLE;
|
||||
}
|
||||
|
||||
|
||||
@ -527,11 +528,8 @@ std::optional<UInt64> StorageStripeLog::totalBytes(const Settings &) const
|
||||
}
|
||||
|
||||
|
||||
void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & partitions)
|
||||
void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
|
||||
{
|
||||
if (partitions)
|
||||
BackupEntriesCollector::throwPartitionsNotSupported(getStorageID(), getName());
|
||||
|
||||
auto lock_timeout = getLockTimeout(backup_entries_collector.getContext());
|
||||
loadIndices(lock_timeout);
|
||||
|
||||
@ -589,13 +587,13 @@ void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collec
|
||||
data_path_in_backup_fs / "count.txt", std::make_unique<BackupEntryFromMemory>(toString(num_rows)));
|
||||
}
|
||||
|
||||
void StorageStripeLog::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & partitions)
|
||||
void StorageStripeLog::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
|
||||
{
|
||||
if (partitions)
|
||||
RestorerFromBackup::throwPartitionsNotSupported(getStorageID(), getName());
|
||||
|
||||
auto backup = restorer.getBackup();
|
||||
if (!restorer.isNonEmptyTableAllowed() && total_bytes && backup->hasFiles(data_path_in_backup))
|
||||
if (!backup->hasFiles(data_path_in_backup))
|
||||
return;
|
||||
|
||||
if (!restorer.isNonEmptyTableAllowed() && total_bytes)
|
||||
RestorerFromBackup::throwTableIsNotEmpty(getStorageID());
|
||||
|
||||
auto lock_timeout = getLockTimeout(restorer.getContext());
|
||||
@ -624,6 +622,11 @@ void StorageStripeLog::restoreDataImpl(const BackupPtr & backup, const String &
|
||||
auto old_data_size = file_checker.getFileSize(data_file_path);
|
||||
{
|
||||
String file_path_in_backup = data_path_in_backup_fs / fileName(data_file_path);
|
||||
if (!backup->fileExists(file_path_in_backup))
|
||||
{
|
||||
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: File {} in backup is required",
|
||||
getStorageID().getFullTableName(), file_path_in_backup);
|
||||
}
|
||||
auto backup_entry = backup->readFile(file_path_in_backup);
|
||||
auto in = backup_entry->getReadBuffer();
|
||||
auto out = disk->writeFile(data_file_path, max_compress_block_size, WriteMode::Append);
|
||||
@ -634,6 +637,11 @@ void StorageStripeLog::restoreDataImpl(const BackupPtr & backup, const String &
|
||||
{
|
||||
String index_path_in_backup = data_path_in_backup_fs / fileName(index_file_path);
|
||||
IndexForNativeFormat extra_indices;
|
||||
if (!backup->fileExists(index_path_in_backup))
|
||||
{
|
||||
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: File {} in backup is required",
|
||||
getStorageID().getFullTableName(), index_path_in_backup);
|
||||
}
|
||||
auto backup_entry = backup->readFile(index_path_in_backup);
|
||||
auto index_in = backup_entry->getReadBuffer();
|
||||
CompressedReadBuffer index_compressed_in{*index_in};
|
||||
|
@ -115,7 +115,7 @@ void StorageSystemFunctions::fillData(MutableColumns & res_columns, ContextPtr c
|
||||
}
|
||||
}
|
||||
|
||||
void StorageSystemFunctions::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> &)
|
||||
void StorageSystemFunctions::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
|
||||
{
|
||||
const auto & user_defined_sql_functions_factory = UserDefinedSQLFunctionFactory::instance();
|
||||
const auto & user_defined_sql_functions_names = user_defined_sql_functions_factory.getAllRegisteredNames();
|
||||
@ -131,18 +131,21 @@ void StorageSystemFunctions::backupData(BackupEntriesCollector & backup_entries_
|
||||
}
|
||||
}
|
||||
|
||||
void StorageSystemFunctions::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> &)
|
||||
void StorageSystemFunctions::restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
|
||||
{
|
||||
auto backup = restorer.getBackup();
|
||||
fs::path data_path_in_backup_fs{data_path_in_backup};
|
||||
|
||||
Strings filenames = backup->listFiles(data_path_in_backup);
|
||||
for (const auto & filename : filenames)
|
||||
{
|
||||
if (!filename.ends_with(".sql"))
|
||||
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore user-defined functions, expected *.sql files, got {}", filename);
|
||||
{
|
||||
throw Exception(ErrorCodes::CANNOT_RESTORE_TABLE, "Cannot restore table {}: File name {} doesn't have the extension .sql",
|
||||
getStorageID().getFullTableName(), String{data_path_in_backup_fs / filename});
|
||||
}
|
||||
}
|
||||
|
||||
fs::path data_path_in_backup_fs{data_path_in_backup};
|
||||
auto & user_defined_sql_functions_factory = UserDefinedSQLFunctionFactory::instance();
|
||||
const auto & restore_settings = restorer.getRestoreSettings();
|
||||
auto context = restorer.getContext();
|
||||
|
@ -122,21 +122,15 @@ void StorageSystemQuotas::fillData(MutableColumns & res_columns, ContextPtr cont
|
||||
}
|
||||
|
||||
void StorageSystemQuotas::backupData(
|
||||
BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & partitions)
|
||||
BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
|
||||
{
|
||||
if (partitions)
|
||||
BackupEntriesCollector::throwPartitionsNotSupported(getStorageID(), getName());
|
||||
|
||||
const auto & access_control = backup_entries_collector.getContext()->getAccessControl();
|
||||
access_control.backup(backup_entries_collector, AccessEntityType::QUOTA, data_path_in_backup);
|
||||
}
|
||||
|
||||
void StorageSystemQuotas::restoreDataFromBackup(
|
||||
RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & partitions)
|
||||
RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
|
||||
{
|
||||
if (partitions)
|
||||
RestorerFromBackup::throwPartitionsNotSupported(getStorageID(), getName());
|
||||
|
||||
auto & access_control = restorer.getContext()->getAccessControl();
|
||||
access_control.restore(restorer, data_path_in_backup);
|
||||
}
|
||||
|
@ -60,21 +60,15 @@ void StorageSystemRoles::fillData(MutableColumns & res_columns, ContextPtr conte
|
||||
}
|
||||
|
||||
void StorageSystemRoles::backupData(
|
||||
BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & partitions)
|
||||
BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
|
||||
{
|
||||
if (partitions)
|
||||
BackupEntriesCollector::throwPartitionsNotSupported(getStorageID(), getName());
|
||||
|
||||
const auto & access_control = backup_entries_collector.getContext()->getAccessControl();
|
||||
access_control.backup(backup_entries_collector, AccessEntityType::ROLE, data_path_in_backup);
|
||||
}
|
||||
|
||||
void StorageSystemRoles::restoreDataFromBackup(
|
||||
RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & partitions)
|
||||
RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
|
||||
{
|
||||
if (partitions)
|
||||
RestorerFromBackup::throwPartitionsNotSupported(getStorageID(), getName());
|
||||
|
||||
auto & access_control = restorer.getContext()->getAccessControl();
|
||||
access_control.restore(restorer, data_path_in_backup);
|
||||
}
|
||||
|
@ -140,21 +140,15 @@ void StorageSystemRowPolicies::fillData(MutableColumns & res_columns, ContextPtr
|
||||
}
|
||||
|
||||
void StorageSystemRowPolicies::backupData(
|
||||
BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & partitions)
|
||||
BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
|
||||
{
|
||||
if (partitions)
|
||||
BackupEntriesCollector::throwPartitionsNotSupported(getStorageID(), getName());
|
||||
|
||||
const auto & access_control = backup_entries_collector.getContext()->getAccessControl();
|
||||
access_control.backup(backup_entries_collector, AccessEntityType::ROW_POLICY, data_path_in_backup);
|
||||
}
|
||||
|
||||
void StorageSystemRowPolicies::restoreDataFromBackup(
|
||||
RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & partitions)
|
||||
RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
|
||||
{
|
||||
if (partitions)
|
||||
RestorerFromBackup::throwPartitionsNotSupported(getStorageID(), getName());
|
||||
|
||||
auto & access_control = restorer.getContext()->getAccessControl();
|
||||
access_control.restore(restorer, data_path_in_backup);
|
||||
}
|
||||
|
@ -87,21 +87,15 @@ void StorageSystemSettingsProfiles::fillData(MutableColumns & res_columns, Conte
|
||||
}
|
||||
|
||||
void StorageSystemSettingsProfiles::backupData(
|
||||
BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & partitions)
|
||||
BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
|
||||
{
|
||||
if (partitions)
|
||||
BackupEntriesCollector::throwPartitionsNotSupported(getStorageID(), getName());
|
||||
|
||||
const auto & access_control = backup_entries_collector.getContext()->getAccessControl();
|
||||
access_control.backup(backup_entries_collector, AccessEntityType::SETTINGS_PROFILE, data_path_in_backup);
|
||||
}
|
||||
|
||||
void StorageSystemSettingsProfiles::restoreDataFromBackup(
|
||||
RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & partitions)
|
||||
RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
|
||||
{
|
||||
if (partitions)
|
||||
RestorerFromBackup::throwPartitionsNotSupported(getStorageID(), getName());
|
||||
|
||||
auto & access_control = restorer.getContext()->getAccessControl();
|
||||
access_control.restore(restorer, data_path_in_backup);
|
||||
}
|
||||
|
@ -215,21 +215,15 @@ void StorageSystemUsers::fillData(MutableColumns & res_columns, ContextPtr conte
|
||||
}
|
||||
|
||||
void StorageSystemUsers::backupData(
|
||||
BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & partitions)
|
||||
BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
|
||||
{
|
||||
if (partitions)
|
||||
BackupEntriesCollector::throwPartitionsNotSupported(getStorageID(), getName());
|
||||
|
||||
const auto & access_control = backup_entries_collector.getContext()->getAccessControl();
|
||||
access_control.backup(backup_entries_collector, AccessEntityType::USER, data_path_in_backup);
|
||||
}
|
||||
|
||||
void StorageSystemUsers::restoreDataFromBackup(
|
||||
RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & partitions)
|
||||
RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
|
||||
{
|
||||
if (partitions)
|
||||
RestorerFromBackup::throwPartitionsNotSupported(getStorageID(), getName());
|
||||
|
||||
auto & access_control = restorer.getContext()->getAccessControl();
|
||||
access_control.restore(restorer, data_path_in_backup);
|
||||
}
|
||||
|
@ -799,3 +799,29 @@ def test_system_functions():
|
||||
assert instance.query("SELECT number, parity_str(number) FROM numbers(3)") == TSV(
|
||||
[[0, "even"], [1, "odd"], [2, "even"]]
|
||||
)
|
||||
|
||||
|
||||
def test_backup_partition():
|
||||
create_and_fill_table(n=30)
|
||||
|
||||
backup_name = new_backup_name()
|
||||
instance.query(f"BACKUP TABLE test.table PARTITIONS '1', '4' TO {backup_name}")
|
||||
|
||||
instance.query("DROP TABLE test.table")
|
||||
|
||||
instance.query(f"RESTORE TABLE test.table FROM {backup_name}")
|
||||
|
||||
assert instance.query("SELECT * FROM test.table ORDER BY x") == TSV([[1, '1'], [4, '4'], [11, '11'], [14, '14'], [21, '21'], [24, '24']])
|
||||
|
||||
|
||||
def test_restore_partition():
|
||||
create_and_fill_table(n=30)
|
||||
|
||||
backup_name = new_backup_name()
|
||||
instance.query(f"BACKUP TABLE test.table TO {backup_name}")
|
||||
|
||||
instance.query("DROP TABLE test.table")
|
||||
|
||||
instance.query(f"RESTORE TABLE test.table PARTITIONS '2', '3' FROM {backup_name}")
|
||||
|
||||
assert instance.query("SELECT * FROM test.table ORDER BY x") == TSV([[2, '2'], [3, '3'], [12, '12'], [13, '13'], [22, '22'], [23, '23']])
|
||||
|
Loading…
Reference in New Issue
Block a user