mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Storages and databases are hollow by default now.
This commit is contained in:
parent
18173d71fb
commit
ce25afb2e9
@ -97,7 +97,7 @@ namespace
|
||||
res.push_back(makeBackupEntryForMetadata(*info.create_query));
|
||||
if (info.has_data)
|
||||
{
|
||||
auto data_backup = info.storage->backup(context, info.partitions);
|
||||
auto data_backup = info.storage->backupData(context, info.partitions);
|
||||
if (!data_backup.empty())
|
||||
{
|
||||
String data_path = getDataPathInBackup(*info.create_query);
|
||||
@ -129,7 +129,7 @@ namespace
|
||||
const auto & database = table_.first;
|
||||
const auto & storage = table_.second;
|
||||
|
||||
if (database->hasHollowBackup())
|
||||
if (!database->hasTablesToBackup())
|
||||
throw Exception(
|
||||
ErrorCodes::CANNOT_BACKUP_TABLE,
|
||||
"Cannot backup the {} because it's contained in a hollow database (engine: {})",
|
||||
@ -144,7 +144,7 @@ namespace
|
||||
/// Make a create query for this table.
|
||||
auto create_query = renameInCreateQuery(database->getCreateTableQuery(table_name_.second, context));
|
||||
|
||||
bool has_data = !storage->hasHollowBackup() && !backup_settings.structure_only;
|
||||
bool has_data = storage->hasDataToBackup() && !backup_settings.structure_only;
|
||||
if (has_data)
|
||||
{
|
||||
/// We check for SELECT privilege only if we're going to read data from the table.
|
||||
@ -220,7 +220,7 @@ namespace
|
||||
}
|
||||
|
||||
/// Backup tables in this database.
|
||||
if (!database_->hasHollowBackup())
|
||||
if (database_->hasTablesToBackup())
|
||||
{
|
||||
for (auto it = database_->getTablesIterator(context); it->isValid(); it->next())
|
||||
{
|
||||
|
@ -231,7 +231,7 @@ namespace
|
||||
{
|
||||
if (!hasData())
|
||||
return {};
|
||||
return storage->restoreFromBackup(context, partitions, backup, data_path_in_backup, *restore_settings);
|
||||
return storage->restoreData(context, partitions, backup, data_path_in_backup, *restore_settings);
|
||||
}
|
||||
|
||||
ContextMutablePtr context;
|
||||
|
@ -50,6 +50,9 @@ public:
|
||||
|
||||
void alterTable(ContextPtr local_context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) override;
|
||||
|
||||
/// This database can contain tables to backup.
|
||||
bool hasTablesToBackup() const override { return true; }
|
||||
|
||||
private:
|
||||
String data_path;
|
||||
using NameToASTCreate = std::unordered_map<String, ASTPtr>;
|
||||
|
@ -36,6 +36,9 @@ public:
|
||||
const StorageID & table_id,
|
||||
const StorageInMemoryMetadata & metadata) override;
|
||||
|
||||
/// This database can contain tables to backup.
|
||||
bool hasTablesToBackup() const override { return true; }
|
||||
|
||||
protected:
|
||||
virtual void commitAlterTable(
|
||||
const StorageID & table_id,
|
||||
|
@ -331,7 +331,7 @@ public:
|
||||
|
||||
/// Returns true if the backup of the database is hollow, which means it doesn't contain
|
||||
/// any tables which can be stored to a backup.
|
||||
virtual bool hasHollowBackup() const { return false; }
|
||||
virtual bool hasTablesToBackup() const { return false; }
|
||||
|
||||
virtual ~IDatabase() = default;
|
||||
|
||||
|
@ -216,14 +216,14 @@ bool IStorage::isStaticStorage() const
|
||||
return false;
|
||||
}
|
||||
|
||||
BackupEntries IStorage::backup(ContextPtr, const ASTs &)
|
||||
BackupEntries IStorage::backupData(ContextPtr, const ASTs &)
|
||||
{
|
||||
throw Exception("Table engine " + getName() + " doesn't support backups", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
RestoreTaskPtr IStorage::restoreFromBackup(ContextMutablePtr, const ASTs &, const BackupPtr &, const String &, const StorageRestoreSettings &)
|
||||
RestoreTaskPtr IStorage::restoreData(ContextMutablePtr, const ASTs &, const BackupPtr &, const String &, const StorageRestoreSettings &)
|
||||
{
|
||||
throw Exception("Table engine " + getName() + " doesn't support restoring", ErrorCodes::NOT_IMPLEMENTED);
|
||||
throw Exception("Table engine " + getName() + " doesn't support backups", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
std::string PrewhereInfo::dump() const
|
||||
|
@ -219,13 +219,13 @@ public:
|
||||
NameDependencies getDependentViewsByColumn(ContextPtr context) const;
|
||||
|
||||
/// Returns true if the backup is hollow, which means it doesn't contain any data.
|
||||
virtual bool hasHollowBackup() const { return false; }
|
||||
virtual bool hasDataToBackup() const { return false; }
|
||||
|
||||
/// Prepares entries to backup data of the storage.
|
||||
virtual BackupEntries backup(ContextPtr context, const ASTs & partitions);
|
||||
virtual BackupEntries backupData(ContextPtr context, const ASTs & partitions);
|
||||
|
||||
/// Extract data from the backup and put it to the storage.
|
||||
virtual RestoreTaskPtr restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings);
|
||||
virtual RestoreTaskPtr restoreData(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings);
|
||||
|
||||
/// Returns whether the column is virtual - by default all columns are real.
|
||||
/// Initially reserved virtual column name may be shadowed by real column.
|
||||
|
@ -3630,7 +3630,7 @@ Pipe MergeTreeData::alterPartition(
|
||||
}
|
||||
|
||||
|
||||
BackupEntries MergeTreeData::backup(ContextPtr local_context, const ASTs & partitions)
|
||||
BackupEntries MergeTreeData::backupData(ContextPtr local_context, const ASTs & partitions)
|
||||
{
|
||||
DataPartsVector data_parts;
|
||||
if (partitions.empty())
|
||||
@ -3789,9 +3789,9 @@ private:
|
||||
};
|
||||
|
||||
|
||||
RestoreTaskPtr MergeTreeData::restoreDataPartsFromBackup(const std::unordered_set<String> & partition_ids,
|
||||
const BackupPtr & backup, const String & data_path_in_backup,
|
||||
SimpleIncrement * increment)
|
||||
RestoreTaskPtr MergeTreeData::restoreDataParts(const std::unordered_set<String> & partition_ids,
|
||||
const BackupPtr & backup, const String & data_path_in_backup,
|
||||
SimpleIncrement * increment)
|
||||
{
|
||||
return std::make_unique<MergeTreeDataRestoreTask>(
|
||||
std::static_pointer_cast<MergeTreeData>(shared_from_this()), backup, data_path_in_backup, partition_ids, increment);
|
||||
|
@ -674,12 +674,15 @@ public:
|
||||
ContextPtr context,
|
||||
TableLockHolder & table_lock_holder);
|
||||
|
||||
/// Storage has data to backup.
|
||||
bool hasDataToBackup() const override { return true; }
|
||||
|
||||
/// Prepares entries to backup data of the storage.
|
||||
BackupEntries backup(ContextPtr context, const ASTs & partitions) override;
|
||||
BackupEntries backupData(ContextPtr context, const ASTs & partitions) override;
|
||||
static BackupEntries backupDataParts(const DataPartsVector & data_parts);
|
||||
|
||||
/// Extract data from the backup and put it to the storage.
|
||||
RestoreTaskPtr restoreDataPartsFromBackup(
|
||||
RestoreTaskPtr restoreDataParts(
|
||||
const std::unordered_set<String> & partition_ids,
|
||||
const BackupPtr & backup,
|
||||
const String & data_path_in_backup,
|
||||
|
@ -888,7 +888,7 @@ IStorage::ColumnSizeByName StorageLog::getColumnSizes() const
|
||||
}
|
||||
|
||||
|
||||
BackupEntries StorageLog::backup(ContextPtr context, const ASTs & partitions)
|
||||
BackupEntries StorageLog::backupData(ContextPtr context, const ASTs & partitions)
|
||||
{
|
||||
if (!partitions.empty())
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName());
|
||||
@ -1063,7 +1063,7 @@ private:
|
||||
ContextMutablePtr context;
|
||||
};
|
||||
|
||||
RestoreTaskPtr StorageLog::restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings &)
|
||||
RestoreTaskPtr StorageLog::restoreData(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings &)
|
||||
{
|
||||
if (!partitions.empty())
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName());
|
||||
|
@ -52,8 +52,9 @@ public:
|
||||
bool supportsSubcolumns() const override { return true; }
|
||||
ColumnSizeByName getColumnSizes() const override;
|
||||
|
||||
BackupEntries backup(ContextPtr context, const ASTs & partitions) override;
|
||||
RestoreTaskPtr restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings) override;
|
||||
bool hasDataToBackup() const override { return true; }
|
||||
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;
|
||||
|
||||
protected:
|
||||
/** Attach the table with the appropriate name, along the appropriate path (with / at the end),
|
||||
|
@ -469,7 +469,7 @@ private:
|
||||
};
|
||||
|
||||
|
||||
BackupEntries StorageMemory::backup(ContextPtr context, const ASTs & partitions)
|
||||
BackupEntries StorageMemory::backupData(ContextPtr context, const ASTs & partitions)
|
||||
{
|
||||
if (!partitions.empty())
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName());
|
||||
@ -552,7 +552,7 @@ private:
|
||||
};
|
||||
|
||||
|
||||
RestoreTaskPtr StorageMemory::restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings &)
|
||||
RestoreTaskPtr StorageMemory::restoreData(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings &)
|
||||
{
|
||||
if (!partitions.empty())
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName());
|
||||
|
@ -66,8 +66,9 @@ public:
|
||||
|
||||
void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override;
|
||||
|
||||
BackupEntries backup(ContextPtr context, const ASTs & partitions) override;
|
||||
RestoreTaskPtr restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings) override;
|
||||
bool hasDataToBackup() const override { return true; }
|
||||
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;
|
||||
|
||||
std::optional<UInt64> totalRows(const Settings &) const override;
|
||||
std::optional<UInt64> totalBytes(const Settings &) const override;
|
||||
|
@ -1644,9 +1644,9 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_
|
||||
}
|
||||
|
||||
|
||||
RestoreTaskPtr StorageMergeTree::restoreFromBackup(ContextMutablePtr local_context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings &)
|
||||
RestoreTaskPtr StorageMergeTree::restoreData(ContextMutablePtr local_context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings &)
|
||||
{
|
||||
return restoreDataPartsFromBackup(getPartitionIDsFromQuery(partitions, local_context), backup, data_path_in_backup, &increment);
|
||||
return restoreDataParts(getPartitionIDsFromQuery(partitions, local_context), backup, data_path_in_backup, &increment);
|
||||
}
|
||||
|
||||
|
||||
|
@ -97,7 +97,7 @@ public:
|
||||
|
||||
CheckResults checkData(const ASTPtr & query, ContextPtr context) override;
|
||||
|
||||
RestoreTaskPtr restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings) override;
|
||||
RestoreTaskPtr restoreData(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings) override;
|
||||
|
||||
bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override;
|
||||
|
||||
|
@ -491,7 +491,7 @@ void StorageStripeLog::saveFileSizes(const WriteLock & /* already locked for wri
|
||||
}
|
||||
|
||||
|
||||
BackupEntries StorageStripeLog::backup(ContextPtr context, const ASTs & partitions)
|
||||
BackupEntries StorageStripeLog::backupData(ContextPtr context, const ASTs & partitions)
|
||||
{
|
||||
if (!partitions.empty())
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName());
|
||||
@ -637,7 +637,7 @@ private:
|
||||
};
|
||||
|
||||
|
||||
RestoreTaskPtr StorageStripeLog::restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings &)
|
||||
RestoreTaskPtr StorageStripeLog::restoreData(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings &)
|
||||
{
|
||||
if (!partitions.empty())
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName());
|
||||
|
@ -52,8 +52,9 @@ public:
|
||||
|
||||
void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder&) override;
|
||||
|
||||
BackupEntries backup(ContextPtr context, const ASTs & partitions) override;
|
||||
RestoreTaskPtr restoreFromBackup(ContextMutablePtr context, const ASTs & partitions, const BackupPtr & backup, const String & data_path_in_backup, const StorageRestoreSettings & restore_settings) override;
|
||||
bool hasDataToBackup() const override { return true; }
|
||||
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;
|
||||
|
||||
protected:
|
||||
StorageStripeLog(
|
||||
|
Loading…
Reference in New Issue
Block a user