Remove ReadSettings from backup entries.

This commit is contained in:
Vitaly Baranov 2023-05-03 13:51:36 +02:00
parent 7cea264230
commit 5198997fd8
20 changed files with 30 additions and 48 deletions

View File

@ -21,24 +21,23 @@ namespace
}
BackupEntryFromAppendOnlyFile::BackupEntryFromAppendOnlyFile(
const DiskPtr & disk_, const String & file_path_, const ReadSettings & settings_, const std::optional<UInt64> & file_size_)
const DiskPtr & disk_, const String & file_path_, const std::optional<UInt64> & file_size_)
: disk(disk_)
, file_path(file_path_)
, data_source_description(disk->getDataSourceDescription())
, size(calculateSize(disk_, file_path_, data_source_description.is_encrypted, file_size_))
, settings(settings_.adjustBufferSize(size))
{
}
BackupEntryFromAppendOnlyFile::~BackupEntryFromAppendOnlyFile() = default;
std::unique_ptr<SeekableReadBuffer> BackupEntryFromAppendOnlyFile::getReadBuffer() const
std::unique_ptr<SeekableReadBuffer> BackupEntryFromAppendOnlyFile::getReadBuffer(const ReadSettings & read_settings) const
{
std::unique_ptr<SeekableReadBuffer> buf;
if (data_source_description.is_encrypted)
buf = disk->readEncryptedFile(file_path, settings);
buf = disk->readEncryptedFile(file_path, read_settings.adjustBufferSize(size));
else
buf = disk->readFile(file_path, settings);
buf = disk->readFile(file_path, read_settings.adjustBufferSize(size));
return std::make_unique<LimitSeekableReadBuffer>(std::move(buf), 0, size);
}

View File

@ -15,12 +15,11 @@ public:
BackupEntryFromAppendOnlyFile(
const DiskPtr & disk_,
const String & file_path_,
const ReadSettings & settings_,
const std::optional<UInt64> & file_size_ = {});
~BackupEntryFromAppendOnlyFile() override;
std::unique_ptr<SeekableReadBuffer> getReadBuffer() const override;
std::unique_ptr<SeekableReadBuffer> getReadBuffer(const ReadSettings & read_settings) const override;
UInt64 getSize() const override { return size; }
DataSourceDescription getDataSourceDescription() const override { return data_source_description; }
@ -35,7 +34,6 @@ private:
const String file_path;
const DataSourceDescription data_source_description;
const UInt64 size;
const ReadSettings settings;
};
}

View File

@ -20,13 +20,11 @@ namespace
BackupEntryFromImmutableFile::BackupEntryFromImmutableFile(
const DiskPtr & disk_,
const String & file_path_,
const ReadSettings & settings_,
const std::optional<UInt64> & file_size_,
const std::optional<UInt128> & checksum_)
: disk(disk_)
, file_path(file_path_)
, data_source_description(disk->getDataSourceDescription())
, settings(settings_)
, file_size(file_size_)
, checksum(checksum_)
{
@ -34,12 +32,12 @@ BackupEntryFromImmutableFile::BackupEntryFromImmutableFile(
BackupEntryFromImmutableFile::~BackupEntryFromImmutableFile() = default;
std::unique_ptr<SeekableReadBuffer> BackupEntryFromImmutableFile::getReadBuffer() const
std::unique_ptr<SeekableReadBuffer> BackupEntryFromImmutableFile::getReadBuffer(const ReadSettings & read_settings) const
{
if (data_source_description.is_encrypted)
return disk->readEncryptedFile(file_path, settings);
return disk->readEncryptedFile(file_path, read_settings);
else
return disk->readFile(file_path, settings);
return disk->readFile(file_path, read_settings);
}
UInt64 BackupEntryFromImmutableFile::getSize() const

View File

@ -1,10 +1,10 @@
#pragma once
#include <Backups/BackupEntryWithChecksumCalculation.h>
#include <IO/ReadSettings.h>
#include <base/defines.h>
#include <mutex>
namespace DB
{
class IDisk;
@ -18,13 +18,12 @@ public:
BackupEntryFromImmutableFile(
const DiskPtr & disk_,
const String & file_path_,
const ReadSettings & settings_,
const std::optional<UInt64> & file_size_ = {},
const std::optional<UInt128> & checksum_ = {});
~BackupEntryFromImmutableFile() override;
std::unique_ptr<SeekableReadBuffer> getReadBuffer() const override;
std::unique_ptr<SeekableReadBuffer> getReadBuffer(const ReadSettings & read_settings) const override;
UInt64 getSize() const override;
UInt128 getChecksum() const override;
@ -42,7 +41,6 @@ private:
const DiskPtr disk;
const String file_path;
const DataSourceDescription data_source_description;
const ReadSettings settings;
mutable std::optional<UInt64> file_size;
mutable std::optional<UInt64> checksum;
mutable bool file_size_adjusted = false;

View File

@ -14,7 +14,7 @@ BackupEntryFromMemory::BackupEntryFromMemory(String data_) : data(std::move(data
{
}
std::unique_ptr<SeekableReadBuffer> BackupEntryFromMemory::getReadBuffer() const
std::unique_ptr<SeekableReadBuffer> BackupEntryFromMemory::getReadBuffer(const ReadSettings &) const
{
return std::make_unique<ReadBufferFromString>(data);
}

View File

@ -14,7 +14,7 @@ public:
BackupEntryFromMemory(const void * data_, size_t size_);
explicit BackupEntryFromMemory(String data_);
std::unique_ptr<SeekableReadBuffer> getReadBuffer() const override;
std::unique_ptr<SeekableReadBuffer> getReadBuffer(const ReadSettings &) const override;
UInt64 getSize() const override { return data.size(); }
DataSourceDescription getDataSourceDescription() const override { return DataSourceDescription{DataSourceType::RAM, "", false, false}; }

View File

@ -44,7 +44,7 @@ BackupEntryFromSmallFile::BackupEntryFromSmallFile(const DiskPtr & disk_, const
{
}
std::unique_ptr<SeekableReadBuffer> BackupEntryFromSmallFile::getReadBuffer() const
std::unique_ptr<SeekableReadBuffer> BackupEntryFromSmallFile::getReadBuffer(const ReadSettings &) const
{
return std::make_unique<ReadBufferFromString>(data);
}

View File

@ -16,7 +16,7 @@ public:
explicit BackupEntryFromSmallFile(const String & file_path_);
BackupEntryFromSmallFile(const DiskPtr & disk_, const String & file_path_);
std::unique_ptr<SeekableReadBuffer> getReadBuffer() const override;
std::unique_ptr<SeekableReadBuffer> getReadBuffer(const ReadSettings &) const override;
UInt64 getSize() const override { return data.size(); }
DataSourceDescription getDataSourceDescription() const override { return data_source_description; }

View File

@ -11,7 +11,7 @@ UInt128 BackupEntryWithChecksumCalculation<Base>::getChecksum() const
std::lock_guard lock{checksum_calculation_mutex};
if (!calculated_checksum)
{
auto read_buffer = this->getReadBuffer();
auto read_buffer = this->getReadBuffer(ReadSettings{}.adjustBufferSize(this->getSize()));
HashingReadBuffer hashing_read_buffer(*read_buffer);
hashing_read_buffer.ignoreAll();
calculated_checksum = hashing_read_buffer.getHash();
@ -28,7 +28,7 @@ std::optional<UInt128> BackupEntryWithChecksumCalculation<Base>::getPartialCheck
if (prefix_length >= this->getSize())
return this->getChecksum();
auto read_buffer = this->getReadBuffer();
auto read_buffer = this->getReadBuffer(ReadSettings{}.adjustBufferSize(prefix_length));
HashingReadBuffer hashing_read_buffer(*read_buffer);
hashing_read_buffer.ignore(prefix_length);
auto partial_checksum = hashing_read_buffer.getHash();

View File

@ -15,7 +15,7 @@ public:
BackupEntryWrappedWith(BackupEntryPtr entry_, T && custom_value_) : entry(entry_), custom_value(std::move(custom_value_)) { }
~BackupEntryWrappedWith() override = default;
std::unique_ptr<SeekableReadBuffer> getReadBuffer() const override { return entry->getReadBuffer(); }
std::unique_ptr<SeekableReadBuffer> getReadBuffer(const ReadSettings & read_settings) const override { return entry->getReadBuffer(read_settings); }
UInt64 getSize() const override { return entry->getSize(); }
UInt128 getChecksum() const override { return entry->getChecksum(); }
std::optional<UInt128> getPartialChecksum(size_t prefix_length) const override { return entry->getPartialChecksum(prefix_length); }

View File

@ -868,7 +868,7 @@ void BackupImpl::writeFile(const BackupFileInfo & info, BackupEntryPtr entry)
{
LOG_TRACE(log, "Writing backup for file {} from {}: data file #{}, adding to archive", info.data_file_name, src_file_desc, info.data_file_index);
auto out = archive_writer->writeFile(info.data_file_name);
auto read_buffer = entry->getReadBuffer();
auto read_buffer = entry->getReadBuffer(writer->getReadSettings());
if (info.base_size != 0)
read_buffer->seek(info.base_size, SEEK_SET);
copyData(*read_buffer, *out);
@ -882,7 +882,7 @@ void BackupImpl::writeFile(const BackupFileInfo & info, BackupEntryPtr entry)
else
{
LOG_TRACE(log, "Writing backup for file {} from {}: data file #{}", info.data_file_name, src_file_desc, info.data_file_index);
auto create_read_buffer = [entry] { return entry->getReadBuffer(); };
auto create_read_buffer = [entry, read_settings = writer->getReadSettings()] { return entry->getReadBuffer(read_settings); };
writer->copyDataToFile(info.data_file_name, create_read_buffer, info.base_size, info.size - info.base_size);
}

View File

@ -17,7 +17,7 @@ class IBackupEntriesLazyBatch::BackupEntryFromBatch : public IBackupEntry
public:
BackupEntryFromBatch(const std::shared_ptr<IBackupEntriesLazyBatch> & batch_, size_t index_) : batch(batch_), index(index_) { }
std::unique_ptr<SeekableReadBuffer> getReadBuffer() const override { return getInternalBackupEntry()->getReadBuffer(); }
std::unique_ptr<SeekableReadBuffer> getReadBuffer(const ReadSettings & read_settings) const override { return getInternalBackupEntry()->getReadBuffer(read_settings); }
UInt64 getSize() const override { return getInternalBackupEntry()->getSize(); }
UInt128 getChecksum() const override { return getInternalBackupEntry()->getChecksum(); }
std::optional<UInt128> getPartialChecksum(size_t prefix_length) const override { return getInternalBackupEntry()->getPartialChecksum(prefix_length); }

View File

@ -28,7 +28,7 @@ public:
virtual std::optional<UInt128> getPartialChecksum(size_t /* prefix_length */) const { return {}; }
/// Returns a read buffer for reading the data.
virtual std::unique_ptr<SeekableReadBuffer> getReadBuffer() const = 0;
virtual std::unique_ptr<SeekableReadBuffer> getReadBuffer(const ReadSettings & read_settings) const = 0;
/// Returns true if the data returned by getReadBuffer() is encrypted by an encrypted disk.
virtual bool isEncryptedByDisk() const { return false; }

View File

@ -319,7 +319,6 @@ DataPartStorageOnDiskBase::getReplicatedFilesDescriptionForRemoteDisk(const Name
}
void DataPartStorageOnDiskBase::backup(
const ReadSettings & read_settings,
const MergeTreeDataPartChecksums & checksums,
const NameSet & files_without_checksums,
const String & path_in_backup,
@ -393,7 +392,7 @@ void DataPartStorageOnDiskBase::backup(
file_hash = {it->second.file_hash.first, it->second.file_hash.second};
}
BackupEntryPtr backup_entry = std::make_unique<BackupEntryFromImmutableFile>(disk, filepath_on_disk, read_settings, file_size, file_hash);
BackupEntryPtr backup_entry = std::make_unique<BackupEntryFromImmutableFile>(disk, filepath_on_disk, file_size, file_hash);
if (temp_dir_owner)
backup_entry = wrapBackupEntryWith(std::move(backup_entry), temp_dir_owner);

View File

@ -50,7 +50,6 @@ public:
ReplicatedFilesDescription getReplicatedFilesDescriptionForRemoteDisk(const NameSet & file_names) const override;
void backup(
const ReadSettings & read_settings,
const MergeTreeDataPartChecksums & checksums,
const NameSet & files_without_checksums,
const String & path_in_backup,

View File

@ -197,7 +197,6 @@ public:
/// Also creates a new tmp_dir for internal disk (if disk is mentioned the first time).
using TemporaryFilesOnDisks = std::map<DiskPtr, std::shared_ptr<TemporaryFileOnDisk>>;
virtual void backup(
const ReadSettings & read_settings,
const MergeTreeDataPartChecksums & checksums,
const NameSet & files_without_checksums,
const String & path_in_backup,

View File

@ -5086,7 +5086,6 @@ BackupEntries MergeTreeData::backupParts(const DataPartsVector & data_parts, con
BackupEntries backup_entries_from_part;
part->getDataPartStorage().backup(
read_settings,
part->checksums,
part->getFileNamesWithoutChecksums(),
data_path_in_backup,
@ -5098,7 +5097,6 @@ BackupEntries MergeTreeData::backupParts(const DataPartsVector & data_parts, con
for (const auto & [projection_name, projection_part] : projection_parts)
{
projection_part->getDataPartStorage().backup(
read_settings,
projection_part->checksums,
projection_part->getFileNamesWithoutChecksums(),
fs::path{data_path_in_backup} / part->name,

View File

@ -928,10 +928,8 @@ std::optional<UInt64> StorageLog::totalBytes(const Settings &) const
void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & /* partitions */)
{
auto local_context = backup_entries_collector.getContext();
ReadSettings read_settings = local_context->getBackupReadSettings();
auto lock_timeout = getLockTimeout(backup_entries_collector.getContext());
auto lock_timeout = getLockTimeout(local_context);
loadMarks(lock_timeout);
ReadLock lock{rwlock, lock_timeout};
@ -954,7 +952,7 @@ void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, c
String hardlink_file_path = temp_dir / data_file_name;
disk->createHardLink(data_file.path, hardlink_file_path);
BackupEntryPtr backup_entry = std::make_unique<BackupEntryFromAppendOnlyFile>(
disk, hardlink_file_path, read_settings, file_checker.getFileSize(data_file.path));
disk, hardlink_file_path, file_checker.getFileSize(data_file.path));
backup_entry = wrapBackupEntryWith(std::move(backup_entry), temp_dir_owner);
backup_entries_collector.addBackupEntry(data_path_in_backup_fs / data_file_name, std::move(backup_entry));
}
@ -967,7 +965,7 @@ void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, c
String hardlink_file_path = temp_dir / marks_file_name;
disk->createHardLink(marks_file_path, hardlink_file_path);
BackupEntryPtr backup_entry = std::make_unique<BackupEntryFromAppendOnlyFile>(
disk, hardlink_file_path, read_settings, file_checker.getFileSize(marks_file_path));
disk, hardlink_file_path, file_checker.getFileSize(marks_file_path));
backup_entry = wrapBackupEntryWith(std::move(backup_entry), temp_dir_owner);
backup_entries_collector.addBackupEntry(data_path_in_backup_fs / marks_file_name, std::move(backup_entry));
}

View File

@ -308,8 +308,6 @@ namespace
BackupEntries generate() override
{
ReadSettings read_settings = context->getBackupReadSettings();
BackupEntries backup_entries;
backup_entries.resize(file_paths.size());
@ -326,7 +324,7 @@ namespace
NativeWriter block_out{data_out, 0, metadata_snapshot->getSampleBlock(), false, &index};
for (const auto & block : *blocks)
block_out.write(block);
backup_entries[data_bin_pos] = {file_paths[data_bin_pos], std::make_shared<BackupEntryFromImmutableFile>(temp_disk, data_file_path, read_settings)};
backup_entries[data_bin_pos] = {file_paths[data_bin_pos], std::make_shared<BackupEntryFromImmutableFile>(temp_disk, data_file_path)};
}
/// Writing index.mrk
@ -335,7 +333,7 @@ namespace
auto index_mrk_out_compressed = temp_disk->writeFile(index_mrk_path);
CompressedWriteBuffer index_mrk_out{*index_mrk_out_compressed};
index.write(index_mrk_out);
backup_entries[index_mrk_pos] = {file_paths[index_mrk_pos], std::make_shared<BackupEntryFromImmutableFile>(temp_disk, index_mrk_path, read_settings)};
backup_entries[index_mrk_pos] = {file_paths[index_mrk_pos], std::make_shared<BackupEntryFromImmutableFile>(temp_disk, index_mrk_path)};
}
/// Writing columns.txt

View File

@ -529,10 +529,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 */)
{
auto local_context = backup_entries_collector.getContext();
ReadSettings read_settings = local_context->getBackupReadSettings();
auto lock_timeout = getLockTimeout(backup_entries_collector.getContext());
auto lock_timeout = getLockTimeout(local_context);
loadIndices(lock_timeout);
ReadLock lock{rwlock, lock_timeout};
@ -554,7 +552,7 @@ void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collec
String hardlink_file_path = temp_dir / data_file_name;
disk->createHardLink(data_file_path, hardlink_file_path);
BackupEntryPtr backup_entry = std::make_unique<BackupEntryFromAppendOnlyFile>(
disk, hardlink_file_path, read_settings, file_checker.getFileSize(data_file_path));
disk, hardlink_file_path, file_checker.getFileSize(data_file_path));
backup_entry = wrapBackupEntryWith(std::move(backup_entry), temp_dir_owner);
backup_entries_collector.addBackupEntry(data_path_in_backup_fs / data_file_name, std::move(backup_entry));
}
@ -566,7 +564,7 @@ void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collec
String hardlink_file_path = temp_dir / index_file_name;
disk->createHardLink(index_file_path, hardlink_file_path);
BackupEntryPtr backup_entry = std::make_unique<BackupEntryFromAppendOnlyFile>(
disk, hardlink_file_path, read_settings, file_checker.getFileSize(index_file_path));
disk, hardlink_file_path, file_checker.getFileSize(index_file_path));
backup_entry = wrapBackupEntryWith(std::move(backup_entry), temp_dir_owner);
backup_entries_collector.addBackupEntry(data_path_in_backup_fs / index_file_name, std::move(backup_entry));
}