mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge pull request #51493 from azat/backups-fix-tmp-files
Fix usage of temporary directories during RESTORE
This commit is contained in:
commit
73dbdf0b1e
@ -14,7 +14,7 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
WriteBufferFromTemporaryFile::WriteBufferFromTemporaryFile(TemporaryFileOnDiskHolder && tmp_file_)
|
||||
: WriteBufferFromFile(tmp_file_->getPath(), DBMS_DEFAULT_BUFFER_SIZE, O_RDWR | O_TRUNC | O_CREAT, /* throttler= */ {}, 0600)
|
||||
: WriteBufferFromFile(tmp_file_->getAbsolutePath(), DBMS_DEFAULT_BUFFER_SIZE, O_RDWR | O_TRUNC | O_CREAT, /* throttler= */ {}, 0600)
|
||||
, tmp_file(std::move(tmp_file_))
|
||||
{
|
||||
}
|
||||
|
@ -54,7 +54,7 @@ TemporaryFileOnDisk::TemporaryFileOnDisk(const DiskPtr & disk_, const String & p
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary file name is empty");
|
||||
}
|
||||
|
||||
String TemporaryFileOnDisk::getPath() const
|
||||
String TemporaryFileOnDisk::getAbsolutePath() const
|
||||
{
|
||||
return std::filesystem::path(disk->getPath()) / relative_path;
|
||||
}
|
||||
|
@ -22,7 +22,10 @@ public:
|
||||
~TemporaryFileOnDisk();
|
||||
|
||||
DiskPtr getDisk() const { return disk; }
|
||||
String getPath() const;
|
||||
/// Return absolute path (disk + relative_path)
|
||||
String getAbsolutePath() const;
|
||||
/// Return relative path (without disk)
|
||||
const String & getRelativePath() const { return relative_path; }
|
||||
|
||||
private:
|
||||
DiskPtr disk;
|
||||
|
@ -1034,7 +1034,7 @@ std::shared_ptr<Block> MergeJoin::loadRightBlock(size_t pos) const
|
||||
{
|
||||
auto load_func = [&]() -> std::shared_ptr<Block>
|
||||
{
|
||||
TemporaryFileStreamLegacy input(flushed_right_blocks[pos]->getPath(), materializeBlock(right_sample_block));
|
||||
TemporaryFileStreamLegacy input(flushed_right_blocks[pos]->getAbsolutePath(), materializeBlock(right_sample_block));
|
||||
return std::make_shared<Block>(input.block_in->read());
|
||||
};
|
||||
|
||||
|
@ -39,7 +39,7 @@ namespace
|
||||
TemporaryFileOnDiskHolder flushToFile(const DiskPtr & disk, const Block & header, QueryPipelineBuilder pipeline, const String & codec)
|
||||
{
|
||||
auto tmp_file = std::make_unique<TemporaryFileOnDisk>(disk, CurrentMetrics::TemporaryFilesForJoin);
|
||||
auto write_stat = TemporaryFileStreamLegacy::write(tmp_file->getPath(), header, std::move(pipeline), codec);
|
||||
auto write_stat = TemporaryFileStreamLegacy::write(tmp_file->getAbsolutePath(), header, std::move(pipeline), codec);
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::ExternalProcessingCompressedBytesTotal, write_stat.compressed_bytes);
|
||||
ProfileEvents::increment(ProfileEvents::ExternalProcessingUncompressedBytesTotal, write_stat.uncompressed_bytes);
|
||||
@ -267,7 +267,7 @@ SortedBlocksWriter::SortedFiles SortedBlocksWriter::finishMerge(std::function<vo
|
||||
|
||||
Pipe SortedBlocksWriter::streamFromFile(const TmpFilePtr & file) const
|
||||
{
|
||||
return Pipe(std::make_shared<TemporaryFileLazySource>(file->getPath(), materializeBlock(sample_block)));
|
||||
return Pipe(std::make_shared<TemporaryFileLazySource>(file->getAbsolutePath(), materializeBlock(sample_block)));
|
||||
}
|
||||
|
||||
|
||||
|
@ -235,9 +235,9 @@ TemporaryFileStream::TemporaryFileStream(TemporaryFileOnDiskHolder file_, const
|
||||
: parent(parent_)
|
||||
, header(header_)
|
||||
, file(std::move(file_))
|
||||
, out_writer(std::make_unique<OutputWriter>(std::make_unique<WriteBufferFromFile>(file->getPath()), header))
|
||||
, out_writer(std::make_unique<OutputWriter>(std::make_unique<WriteBufferFromFile>(file->getAbsolutePath()), header))
|
||||
{
|
||||
LOG_TEST(&Poco::Logger::get("TemporaryFileStream"), "Writing to temporary file {}", file->getPath());
|
||||
LOG_TEST(&Poco::Logger::get("TemporaryFileStream"), "Writing to temporary file {}", file->getAbsolutePath());
|
||||
}
|
||||
|
||||
TemporaryFileStream::TemporaryFileStream(FileSegmentsHolderPtr segments_, const Block & header_, TemporaryDataOnDisk * parent_)
|
||||
@ -365,7 +365,7 @@ void TemporaryFileStream::release()
|
||||
String TemporaryFileStream::getPath() const
|
||||
{
|
||||
if (file)
|
||||
return file->getPath();
|
||||
return file->getAbsolutePath();
|
||||
if (segment_holder && !segment_holder->empty())
|
||||
return segment_holder->front().getPathInLocalCache();
|
||||
|
||||
|
@ -350,7 +350,7 @@ void DataPartStorageOnDiskBase::backup(
|
||||
temp_dir_it = temp_dirs->emplace(disk, std::make_shared<TemporaryFileOnDisk>(disk, "tmp/")).first;
|
||||
|
||||
temp_dir_owner = temp_dir_it->second;
|
||||
fs::path temp_dir = temp_dir_owner->getPath();
|
||||
fs::path temp_dir = temp_dir_owner->getRelativePath();
|
||||
temp_part_dir = temp_dir / part_path_in_backup.relative_path();
|
||||
disk->createDirectories(temp_part_dir);
|
||||
}
|
||||
|
@ -5266,7 +5266,7 @@ public:
|
||||
auto it = temp_dirs.find(disk);
|
||||
if (it == temp_dirs.end())
|
||||
it = temp_dirs.emplace(disk, std::make_shared<TemporaryFileOnDisk>(disk, "tmp/")).first;
|
||||
return it->second->getPath();
|
||||
return it->second->getRelativePath();
|
||||
}
|
||||
|
||||
private:
|
||||
|
@ -946,7 +946,7 @@ void StorageLog::backupData(BackupEntriesCollector & backup_entries_collector, c
|
||||
|
||||
fs::path data_path_in_backup_fs = data_path_in_backup;
|
||||
auto temp_dir_owner = std::make_shared<TemporaryFileOnDisk>(disk, "tmp/");
|
||||
fs::path temp_dir = temp_dir_owner->getPath();
|
||||
fs::path temp_dir = temp_dir_owner->getRelativePath();
|
||||
disk->createDirectories(temp_dir);
|
||||
|
||||
bool copy_encrypted = !backup_entries_collector.getBackupSettings().decrypt_files_from_encrypted_disks;
|
||||
|
@ -314,7 +314,7 @@ namespace
|
||||
backup_entries.resize(file_paths.size());
|
||||
|
||||
temp_dir_owner.emplace(temp_disk);
|
||||
fs::path temp_dir = temp_dir_owner->getPath();
|
||||
fs::path temp_dir = temp_dir_owner->getRelativePath();
|
||||
temp_disk->createDirectories(temp_dir);
|
||||
|
||||
/// Writing data.bin
|
||||
@ -453,10 +453,10 @@ void StorageMemory::restoreDataImpl(const BackupPtr & backup, const String & dat
|
||||
if (!dynamic_cast<ReadBufferFromFileBase *>(in.get()))
|
||||
{
|
||||
temp_data_file.emplace(temporary_disk);
|
||||
auto out = std::make_unique<WriteBufferFromFile>(temp_data_file->getPath());
|
||||
auto out = std::make_unique<WriteBufferFromFile>(temp_data_file->getAbsolutePath());
|
||||
copyData(*in, *out);
|
||||
out.reset();
|
||||
in = createReadBufferFromFileBase(temp_data_file->getPath(), {});
|
||||
in = createReadBufferFromFileBase(temp_data_file->getAbsolutePath(), {});
|
||||
}
|
||||
std::unique_ptr<ReadBufferFromFileBase> in_from_file{static_cast<ReadBufferFromFileBase *>(in.release())};
|
||||
CompressedReadBufferFromFile compressed_in{std::move(in_from_file)};
|
||||
|
@ -544,7 +544,7 @@ void StorageStripeLog::backupData(BackupEntriesCollector & backup_entries_collec
|
||||
|
||||
fs::path data_path_in_backup_fs = data_path_in_backup;
|
||||
auto temp_dir_owner = std::make_shared<TemporaryFileOnDisk>(disk, "tmp/");
|
||||
fs::path temp_dir = temp_dir_owner->getPath();
|
||||
fs::path temp_dir = temp_dir_owner->getRelativePath();
|
||||
disk->createDirectories(temp_dir);
|
||||
|
||||
bool copy_encrypted = !backup_entries_collector.getBackupSettings().decrypt_files_from_encrypted_disks;
|
||||
|
@ -29,14 +29,14 @@ $CLICKHOUSE_CLIENT -nm -q "
|
||||
"
|
||||
|
||||
query_id=$(random_str 10)
|
||||
$CLICKHOUSE_CLIENT --send_logs_level=error --format Null --query_id $query_id -q "RESTORE TABLE data AS data_native_copy FROM S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_native_copy') SETTINGS allow_s3_native_copy=true"
|
||||
$CLICKHOUSE_CLIENT --format Null --query_id $query_id -q "RESTORE TABLE data AS data_native_copy FROM S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_native_copy') SETTINGS allow_s3_native_copy=true"
|
||||
$CLICKHOUSE_CLIENT -nm -q "
|
||||
SYSTEM FLUSH LOGS;
|
||||
SELECT query, ProfileEvents['S3CopyObject']>0 FROM system.query_log WHERE type = 'QueryFinish' AND event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id'
|
||||
"
|
||||
|
||||
query_id=$(random_str 10)
|
||||
$CLICKHOUSE_CLIENT --send_logs_level=error --format Null --query_id $query_id -q "RESTORE TABLE data AS data_no_native_copy FROM S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_no_native_copy') SETTINGS allow_s3_native_copy=false"
|
||||
$CLICKHOUSE_CLIENT --format Null --query_id $query_id -q "RESTORE TABLE data AS data_no_native_copy FROM S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data_no_native_copy') SETTINGS allow_s3_native_copy=false"
|
||||
$CLICKHOUSE_CLIENT -nm -q "
|
||||
SYSTEM FLUSH LOGS;
|
||||
SELECT query, ProfileEvents['S3CopyObject']>0 FROM system.query_log WHERE type = 'QueryFinish' AND event_date >= yesterday() AND current_database = '$CLICKHOUSE_DATABASE' AND query_id = '$query_id'
|
||||
|
18
tests/queries/0_stateless/02803_backup_tmp_files.sh
Executable file
18
tests/queries/0_stateless/02803_backup_tmp_files.sh
Executable file
@ -0,0 +1,18 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest
|
||||
# Tag: no-fasttest - requires S3
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
set -e
|
||||
|
||||
$CLICKHOUSE_CLIENT -nm -q "
|
||||
drop table if exists data;
|
||||
create table data (key Int) engine=MergeTree() order by tuple() settings disk='s3_disk';
|
||||
insert into data select * from numbers(10);
|
||||
"
|
||||
|
||||
$CLICKHOUSE_CLIENT --format Null -q "BACKUP TABLE data TO S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data')"
|
||||
$CLICKHOUSE_CLIENT --format Null -q "RESTORE TABLE data AS data_native_copy FROM S3(s3_conn, 'backups/$CLICKHOUSE_DATABASE/data')"
|
Loading…
Reference in New Issue
Block a user