Do not throttle S3-S3 backups if native copy is possible

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
This commit is contained in:
Azat Khuzhin 2023-04-03 15:07:22 +02:00
parent 61405b827d
commit c8597fbb9a
9 changed files with 19 additions and 19 deletions

View File

@ -37,7 +37,7 @@ public:
virtual void removeFiles(const Strings & file_names) = 0;
virtual DataSourceDescription getDataSourceDescription() const = 0;
virtual void copyDataToFile(const CreateReadBufferFunction & create_read_buffer, UInt64 offset, UInt64 size, const String & dest_file_name);
virtual bool supportNativeCopy(DataSourceDescription /* data_source_description */) const { return false; }
virtual bool supportNativeCopy(DataSourceDescription /* data_source_description */, bool /* has_throttling */) const { return false; }
// Ignore throttling, copyDataToFile() should be used if throttling was requested.
virtual void copyFileNative(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name);
};

View File

@ -119,9 +119,9 @@ DataSourceDescription BackupReaderDisk::getDataSourceDescription() const
return disk->getDataSourceDescription();
}
bool BackupWriterDisk::supportNativeCopy(DataSourceDescription data_source_description) const
bool BackupWriterDisk::supportNativeCopy(DataSourceDescription data_source_description, bool has_throttling) const
{
return data_source_description == disk->getDataSourceDescription();
return !has_throttling && data_source_description == disk->getDataSourceDescription();
}
void BackupWriterDisk::copyFileNative(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name)

View File

@ -41,7 +41,7 @@ public:
void removeFiles(const Strings & file_names) override;
DataSourceDescription getDataSourceDescription() const override;
bool supportNativeCopy(DataSourceDescription data_source_description) const override;
bool supportNativeCopy(DataSourceDescription data_source_description, bool has_throttling) const override;
void copyFileNative(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name) override;
private:

View File

@ -139,9 +139,9 @@ DataSourceDescription BackupReaderFile::getDataSourceDescription() const
}
bool BackupWriterFile::supportNativeCopy(DataSourceDescription data_source_description) const
bool BackupWriterFile::supportNativeCopy(DataSourceDescription data_source_description, bool has_throttling) const
{
return data_source_description == getDataSourceDescription();
return !has_throttling && data_source_description == getDataSourceDescription();
}
void BackupWriterFile::copyFileNative(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name)

View File

@ -37,7 +37,7 @@ public:
void removeFile(const String & file_name) override;
void removeFiles(const Strings & file_names) override;
DataSourceDescription getDataSourceDescription() const override;
bool supportNativeCopy(DataSourceDescription data_source_description) const override;
bool supportNativeCopy(DataSourceDescription data_source_description, bool has_throttling) const override;
void copyFileNative(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name) override;
private:

View File

@ -163,7 +163,7 @@ BackupWriterS3::BackupWriterS3(
const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, const ContextPtr & context_)
: s3_uri(s3_uri_)
, client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_))
, read_settings(context_->getReadSettings())
, read_settings(context_->getBackupReadSettings())
, request_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).request_settings)
, log(&Poco::Logger::get("BackupWriterS3"))
{
@ -176,7 +176,7 @@ DataSourceDescription BackupWriterS3::getDataSourceDescription() const
return DataSourceDescription{DataSourceType::S3, s3_uri.endpoint, false, false};
}
bool BackupWriterS3::supportNativeCopy(DataSourceDescription data_source_description) const
bool BackupWriterS3::supportNativeCopy(DataSourceDescription data_source_description, bool /* has_throttling */) const
{
return getDataSourceDescription() == data_source_description;
}
@ -189,7 +189,7 @@ void BackupWriterS3::copyFileNative(DiskPtr src_disk, const String & src_file_na
auto objects = src_disk->getStorageObjects(src_file_name);
if (objects.size() > 1)
{
auto create_read_buffer = [src_disk, src_file_name] { return src_disk->readFile(src_file_name); };
auto create_read_buffer = [this, src_disk, src_file_name] { return src_disk->readFile(src_file_name, read_settings); };
copyDataToFile(create_read_buffer, src_offset, src_size, dest_file_name);
}
else

View File

@ -52,7 +52,7 @@ public:
void removeFiles(const Strings & file_names) override;
DataSourceDescription getDataSourceDescription() const override;
bool supportNativeCopy(DataSourceDescription data_source_description) const override;
bool supportNativeCopy(DataSourceDescription data_source_description, bool has_throttling) const override;
void copyFileNative(DiskPtr src_disk, const String & src_file_name, UInt64 src_offset, UInt64 src_size, const String & dest_file_name) override;
private:

View File

@ -81,7 +81,7 @@ BackupImpl::BackupImpl(
const std::optional<BackupInfo> & base_backup_info_,
std::shared_ptr<IBackupReader> reader_,
const ContextPtr & context_)
: context(context_)
: has_throttler(static_cast<bool>(context_->getBackupsThrottler()))
, backup_name_for_logging(backup_name_for_logging_)
, use_archive(!archive_params_.archive_name.empty())
, archive_params(archive_params_)
@ -91,7 +91,7 @@ BackupImpl::BackupImpl(
, version(INITIAL_BACKUP_VERSION)
, base_backup_info(base_backup_info_)
{
open();
open(context_);
}
@ -105,7 +105,7 @@ BackupImpl::BackupImpl(
const std::shared_ptr<IBackupCoordination> & coordination_,
const std::optional<UUID> & backup_uuid_,
bool deduplicate_files_)
: context(context_)
: has_throttler(static_cast<bool>(context_->getBackupsThrottler()))
, backup_name_for_logging(backup_name_for_logging_)
, use_archive(!archive_params_.archive_name.empty())
, archive_params(archive_params_)
@ -119,7 +119,7 @@ BackupImpl::BackupImpl(
, deduplicate_files(deduplicate_files_)
, log(&Poco::Logger::get("BackupImpl"))
{
open();
open(context_);
}
@ -135,7 +135,7 @@ BackupImpl::~BackupImpl()
}
}
void BackupImpl::open()
void BackupImpl::open(const ContextPtr & context)
{
std::lock_guard lock{mutex};
@ -836,7 +836,7 @@ void BackupImpl::writeFile(const BackupFileInfo & info, BackupEntryPtr entry)
/// We need to copy whole file without archive, we can do it faster
/// if source and destination are compatible
if (!use_archive && !context->getBackupsThrottler() && writer->supportNativeCopy(reader_description))
if (!use_archive && writer->supportNativeCopy(reader_description, has_throttler))
{
/// Should be much faster than writing data through server.
LOG_TRACE(log, "Will copy file {} using native copy", info.data_file_name);

View File

@ -85,7 +85,7 @@ public:
bool supportsWritingInMultipleThreads() const override { return !use_archive; }
private:
void open();
void open(const ContextPtr & context);
void close();
void openArchive();
@ -109,7 +109,7 @@ private:
/// Calculates and sets `compressed_size`.
void setCompressedSize();
ContextPtr context;
const bool has_throttler;
const String backup_name_for_logging;
const bool use_archive;
const ArchiveParams archive_params;