From 7b460b5f8567cf6686b3e0d3bbd89f6753369a4e Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Aug 2022 13:31:25 +0200 Subject: [PATCH 01/44] Small refactoring --- src/Backups/BackupEntryFromImmutableFile.h | 2 +- src/Backups/BackupEntryFromMemory.h | 5 + src/Backups/BackupEntryFromSmallFile.h | 2 +- src/Backups/BackupIO.h | 2 + src/Backups/BackupIO_Disk.h | 1 - src/Backups/BackupIO_File.cpp | 1 + src/Backups/BackupIO_File.h | 5 +- src/Backups/BackupImpl.cpp | 235 +++++++++++++-------- src/Backups/IBackupEntriesLazyBatch.cpp | 4 + src/Backups/IBackupEntry.h | 2 + 10 files changed, 164 insertions(+), 95 deletions(-) diff --git a/src/Backups/BackupEntryFromImmutableFile.h b/src/Backups/BackupEntryFromImmutableFile.h index 5103518c873..e869a5e1fc0 100644 --- a/src/Backups/BackupEntryFromImmutableFile.h +++ b/src/Backups/BackupEntryFromImmutableFile.h @@ -36,7 +36,7 @@ public: std::optional getChecksum() const override { return checksum; } std::unique_ptr getReadBuffer() const override; - String getFilePath() const { return file_path; } + String getFilePath() const override { return file_path; } DiskPtr getDisk() const { return disk; } private: diff --git a/src/Backups/BackupEntryFromMemory.h b/src/Backups/BackupEntryFromMemory.h index 2226112c9c3..6f7e2f8b9a3 100644 --- a/src/Backups/BackupEntryFromMemory.h +++ b/src/Backups/BackupEntryFromMemory.h @@ -19,6 +19,11 @@ public: std::optional getChecksum() const override { return checksum; } std::unique_ptr getReadBuffer() const override; + String getFilePath() const override + { + return ""; + } + private: const String data; const std::optional checksum; diff --git a/src/Backups/BackupEntryFromSmallFile.h b/src/Backups/BackupEntryFromSmallFile.h index 298c4fa8f77..330d1b924c4 100644 --- a/src/Backups/BackupEntryFromSmallFile.h +++ b/src/Backups/BackupEntryFromSmallFile.h @@ -23,7 +23,7 @@ public: const String & file_path_, const std::optional & checksum_ = {}); - String getFilePath() const { return file_path; } + String getFilePath() const override { return file_path; } DiskPtr getDisk() const { return disk; } private: diff --git a/src/Backups/BackupIO.h b/src/Backups/BackupIO.h index 389df97502a..f5a274ea28b 100644 --- a/src/Backups/BackupIO.h +++ b/src/Backups/BackupIO.h @@ -27,6 +27,8 @@ public: virtual bool fileContentsEqual(const String & file_name, const String & expected_file_contents) = 0; virtual std::unique_ptr writeFile(const String & file_name) = 0; virtual void removeFiles(const Strings & file_names) = 0; + virtual void copyFileThroughBuffer(std::unique_ptr && source, const String & file_name); + virtual void copyFileNative(const String & file_name_from, const String & file_name_to); }; } diff --git a/src/Backups/BackupIO_Disk.h b/src/Backups/BackupIO_Disk.h index 53412e6d219..beea5f120c2 100644 --- a/src/Backups/BackupIO_Disk.h +++ b/src/Backups/BackupIO_Disk.h @@ -34,7 +34,6 @@ public: bool fileContentsEqual(const String & file_name, const String & expected_file_contents) override; std::unique_ptr writeFile(const String & file_name) override; void removeFiles(const Strings & file_names) override; - private: DiskPtr disk; std::filesystem::path path; diff --git a/src/Backups/BackupIO_File.cpp b/src/Backups/BackupIO_File.cpp index 7c08c150474..19c486ad682 100644 --- a/src/Backups/BackupIO_File.cpp +++ b/src/Backups/BackupIO_File.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace fs = std::filesystem; diff --git a/src/Backups/BackupIO_File.h b/src/Backups/BackupIO_File.h index 5d37408e6d8..85f4ec34ab4 100644 --- a/src/Backups/BackupIO_File.h +++ b/src/Backups/BackupIO_File.h @@ -9,7 +9,7 @@ namespace DB class BackupReaderFile : public IBackupReader { public: - BackupReaderFile(const String & path_); + explicit BackupReaderFile(const String & path_); ~BackupReaderFile() override; bool fileExists(const String & file_name) override; @@ -23,7 +23,7 @@ private: class BackupWriterFile : public IBackupWriter { public: - BackupWriterFile(const String & path_); + explicit BackupWriterFile(const String & path_); ~BackupWriterFile() override; bool fileExists(const String & file_name) override; @@ -31,7 +31,6 @@ public: bool fileContentsEqual(const String & file_name, const String & expected_file_contents) override; std::unique_ptr writeFile(const String & file_name) override; void removeFiles(const Strings & file_names) override; - private: std::filesystem::path path; }; diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index 263aab2bd50..4f9f6097496 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -111,6 +111,11 @@ public: UInt64 getSize() const override { return size; } std::optional getChecksum() const override { return checksum; } + String getFilePath() const override + { + return data_file_name; + } + private: const std::shared_ptr backup; const String archive_suffix; @@ -587,6 +592,82 @@ BackupEntryPtr BackupImpl::readFile(const SizeAndChecksum & size_and_checksum) c } } +namespace +{ + +std::optional getInfoAboutFileFromBaseBackupIfExists(std::shared_ptr base_backup, const std::string & file_path) +{ + if (base_backup && base_backup->fileExists(file_path)) + return std::pair{base_backup->getFileSize(file_path), base_backup->getFileChecksum(file_path)}; + + return std::nullopt; +} + +enum class CheckBackupResult +{ + HasPrefix, + HasFull, + HasNothing, +}; + +CheckBackupResult checkBaseBackupForFile(const SizeAndChecksum & base_backup_info, const FileInfo & new_entry_info) +{ + /// We cannot reuse base backup because our file is smaller + /// than file stored in previous backup + if (new_entry_info.size > base_backup_info.first) + return CheckBackupResult::HasNothing; + + if (base_backup_info.first == new_entry_info.size) + return CheckBackupResult::HasFull; + + return CheckBackupResult::HasPrefix; + +} + +struct ChecksumsForNewEntry +{ + UInt128 full_checksum; + UInt128 prefix_checksum; +}; + +/// Calculate checksum for backup entry if it's empty. +/// Also able to calculate additional checksum of some prefix. +ChecksumsForNewEntry calculateNewEntryChecksumsIfNeeded(BackupEntryPtr entry, size_t prefix_size) +{ + if (prefix_size > 0) + { + auto read_buffer = entry->getReadBuffer(); + HashingReadBuffer hashing_read_buffer(*read_buffer); + hashing_read_buffer.ignore(prefix_size); + auto prefix_checksum = hashing_read_buffer.getHash(); + if (entry->getChecksum() == std::nullopt) + { + hashing_read_buffer.ignoreAll(); + auto full_checksum = hashing_read_buffer.getHash(); + return ChecksumsForNewEntry{full_checksum, prefix_checksum}; + } + else + { + return ChecksumsForNewEntry{*(entry->getChecksum()), prefix_checksum}; + } + } + else + { + if (entry->getChecksum() == std::nullopt) + { + auto read_buffer = entry->getReadBuffer(); + HashingReadBuffer hashing_read_buffer(*read_buffer); + hashing_read_buffer.ignoreAll(); + return ChecksumsForNewEntry{hashing_read_buffer.getHash(), 0}; + } + else + { + return ChecksumsForNewEntry{*(entry->getChecksum()), 0}; + } + } +} + +} void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) { @@ -602,133 +683,108 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) throw Exception( ErrorCodes::BACKUP_ENTRY_ALREADY_EXISTS, "Backup {}: Entry {} already exists", backup_name, quoteString(file_name)); - FileInfo info; - info.file_name = adjusted_path; - size_t size = entry->getSize(); - info.size = size; + FileInfo info + { + .file_name = adjusted_path, + .size = entry->getSize(), + .base_size = 0, + .base_checksum = 0, + }; - /// Check if the entry's data is empty. - if (!info.size) + /// Empty file, nothing to backup + if (info.size == 0) { coordination->addFileInfo(info); return; } - /// Maybe we have a copy of this file in the backup already. - std::optional checksum = entry->getChecksum(); - if (checksum && coordination->getFileInfo(std::pair{size, *checksum})) - { - info.checksum = *checksum; - coordination->addFileInfo(info); - return; - } + std::optional base_backup_file_info = getInfoAboutFileFromBaseBackupIfExists(base_backup, adjusted_path); - /// Check if a entry with such name exists in the base backup. - bool base_exists = (base_backup && base_backup->fileExists(adjusted_path)); - UInt64 base_size = 0; - UInt128 base_checksum{0, 0}; - if (base_exists) + /// We have info about this file in base backup + /// If file has no checksum -- calculate and fill it. + if (base_backup_file_info.has_value()) { - base_size = base_backup->getFileSize(adjusted_path); - base_checksum = base_backup->getFileChecksum(adjusted_path); - } + CheckBackupResult check_base = checkBaseBackupForFile(*base_backup_file_info, info); - std::unique_ptr read_buffer; /// We'll set that later. - std::optional hashing_read_buffer; - UInt64 hashing_pos = 0; /// Current position in `hashing_read_buffer`. - - /// Determine whether it's possible to receive this entry's data from the base backup completely or partly. - bool use_base = false; - if (base_exists && base_size && (size >= base_size)) - { - if (checksum && (size == base_size)) + /// File with the same name but smaller size exist in previous backup + if (check_base == CheckBackupResult::HasPrefix) { - /// The size is the same, we need to compare checksums to find out - /// if the entry's data has not changed since the base backup. - use_base = (*checksum == base_checksum); + auto checksums = calculateNewEntryChecksumsIfNeeded(entry, base_backup_file_info->first); + info.checksum = checksums.full_checksum; + + /// We have prefix of this file in backup with the same checksum. + /// In ClickHouse this can happen for StorageLog for example. + if (checksums.prefix_checksum == base_backup_file_info->second) + { + info.base_size = base_backup_file_info->first; + info.base_checksum = base_backup_file_info->second; + } } else { - /// The size has increased, we need to calculate a partial checksum to find out - /// if the entry's data has only appended since the base backup. - read_buffer = entry->getReadBuffer(); - hashing_read_buffer.emplace(*read_buffer); - hashing_read_buffer->ignore(base_size); - hashing_pos = base_size; - UInt128 partial_checksum = hashing_read_buffer->getHash(); - if (size == base_size) - checksum = partial_checksum; - if (partial_checksum == base_checksum) - use_base = true; + /// We have full file or have nothing, first of all let's get checksum + /// of current file + auto checksums = calculateNewEntryChecksumsIfNeeded(entry, 0); + info.checksum = checksums.full_checksum; + + if (info.checksum == base_backup_file_info->second) + { + assert(check_base == CheckBackupResult::HasFull); + assert(info.size == base_backup_file_info->first); + + info.base_size = base_backup_file_info->first; + info.base_checksum = base_backup_file_info->second; + /// Actually we can add this info to coordination and exist, + /// but we intentionally don't do it, otherwise control flow + /// of this function will be very complex. + } } } - - /// Finish calculating the checksum. - if (!checksum) + else /// We don't have info about this file_name (sic!) in base backup, + /// however file could be renamed, so we will check one more time using size and checksum { - if (!read_buffer) - read_buffer = entry->getReadBuffer(); - if (!hashing_read_buffer) - hashing_read_buffer.emplace(*read_buffer); - hashing_read_buffer->ignore(size - hashing_pos); - checksum = hashing_read_buffer->getHash(); + auto checksums = calculateNewEntryChecksumsIfNeeded(entry, 0); + info.checksum = checksums.full_checksum; } - hashing_read_buffer.reset(); - info.checksum = *checksum; /// Maybe we have a copy of this file in the backup already. - if (coordination->getFileInfo(std::pair{size, *checksum})) + if (coordination->getFileInfo(std::pair{info.size, info.checksum})) { coordination->addFileInfo(info); return; } - /// Check if a entry with the same checksum exists in the base backup. - if (base_backup && !use_base && base_backup->fileExists(std::pair{size, *checksum})) + /// On the previous lines we checked that backup for file with adjusted_name exist in previous backup. + /// However file can be renamed, but has the same size and checksums, let's check for this case. + if (base_backup && base_backup->fileExists(std::pair{info.size, info.checksum})) { - /// The entry's data has not changed since the base backup, - /// but the entry itself has been moved or renamed. - base_size = size; - base_checksum = *checksum; - use_base = true; - } + info.base_size = info.size; + info.base_checksum = info.checksum; - if (use_base) - { - info.base_size = base_size; - info.base_checksum = base_checksum; - } - - if (use_base && (size == base_size)) - { - /// The entry's data has not been changed since the base backup. coordination->addFileInfo(info); return; } - bool is_data_file_required; + /// All "short paths" failed. We don't have this file in previous or existing backup + /// or have only prefix of it in previous backup. Let's go long path. + info.data_file_name = info.file_name; info.archive_suffix = current_archive_suffix; + + bool is_data_file_required; coordination->addFileInfo(info, is_data_file_required); if (!is_data_file_required) return; /// We copy data only if it's a new combination of size & checksum. - /// Either the entry wasn't exist in the base backup - /// or the entry has data appended to the end of the data from the base backup. - /// In both those cases we have to copy data to this backup. + auto read_buffer = entry->getReadBuffer(); - /// Find out where the start position to copy data is. - auto copy_pos = use_base ? base_size : 0; - - /// Move the current read position to the start position to copy data. - if (!read_buffer) - read_buffer = entry->getReadBuffer(); - read_buffer->seek(copy_pos, SEEK_SET); + /// If we have prefix in base we will seek to the start of the suffix which differs + if (info.base_size != 0) + read_buffer->seek(info.base_size, SEEK_SET); if (!num_files_written) checkLockFile(true); - /// Copy the entry's data after `copy_pos`. std::unique_ptr out; if (use_archives) { @@ -741,20 +797,21 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) next_suffix = true;*/ if (next_suffix) current_archive_suffix = coordination->getNextArchiveSuffix(); + if (info.archive_suffix != current_archive_suffix) { info.archive_suffix = current_archive_suffix; coordination->updateFileInfo(info); } out = getArchiveWriter(current_archive_suffix)->writeFile(info.data_file_name); + copyData(*read_buffer, *out); + out->finalize(); } else { - out = writer->writeFile(info.data_file_name); + writer->copyFileThroughBuffer(std::move(read_buffer), info.data_file_name); } - copyData(*read_buffer, *out); - out->finalize(); ++num_files_written; } diff --git a/src/Backups/IBackupEntriesLazyBatch.cpp b/src/Backups/IBackupEntriesLazyBatch.cpp index 5fb3a4cb3c0..db2f5258316 100644 --- a/src/Backups/IBackupEntriesLazyBatch.cpp +++ b/src/Backups/IBackupEntriesLazyBatch.cpp @@ -20,6 +20,10 @@ public: UInt64 getSize() const override { return getInternalBackupEntry()->getSize(); } std::optional getChecksum() const override { return getInternalBackupEntry()->getChecksum(); } std::unique_ptr getReadBuffer() const override { return getInternalBackupEntry()->getReadBuffer(); } + String getFilePath() const override + { + return getInternalBackupEntry()->getFilePath(); + } private: BackupEntryPtr getInternalBackupEntry() const diff --git a/src/Backups/IBackupEntry.h b/src/Backups/IBackupEntry.h index 04f7dc61475..24b31807ddb 100644 --- a/src/Backups/IBackupEntry.h +++ b/src/Backups/IBackupEntry.h @@ -24,6 +24,8 @@ public: /// Returns a read buffer for reading the data. virtual std::unique_ptr getReadBuffer() const = 0; + + virtual String getFilePath() const = 0; }; using BackupEntryPtr = std::shared_ptr; From d8664c3227b407e6bdc54cca8ac7ca69e2da6200 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Aug 2022 16:58:30 +0200 Subject: [PATCH 02/44] Add shortcut for backups --- src/Backups/BackupEntryFromImmutableFile.cpp | 21 +++++ src/Backups/BackupEntryFromImmutableFile.h | 2 + src/Backups/BackupEntryFromMemory.h | 5 ++ src/Backups/BackupEntryFromSmallFile.cpp | 1 + src/Backups/BackupEntryFromSmallFile.h | 1 - src/Backups/BackupIO.h | 9 +++ src/Backups/BackupIO_Disk.cpp | 22 ++++++ src/Backups/BackupIO_Disk.h | 6 ++ src/Backups/BackupIO_File.cpp | 46 +++++++++++ src/Backups/BackupIO_File.h | 6 ++ src/Backups/BackupImpl.cpp | 78 ++++++++++++------- src/Backups/IBackupEntriesLazyBatch.cpp | 6 ++ src/Backups/IBackupEntry.h | 3 + .../registerBackupEnginesFileAndDisk.cpp | 2 +- src/Common/filesystemHelpers.cpp | 16 ++++ src/Common/filesystemHelpers.h | 2 + src/Disks/DiskDecorator.h | 2 +- src/Disks/DiskEncrypted.h | 8 +- src/Disks/DiskLocal.cpp | 18 ++++- src/Disks/DiskLocal.h | 6 +- src/Disks/DiskMemory.h | 3 +- src/Disks/DiskType.h | 34 ++++---- src/Disks/IDisk.cpp | 2 +- src/Disks/IDisk.h | 12 +-- .../AzureBlobStorage/AzureObjectStorage.cpp | 4 + .../AzureBlobStorage/AzureObjectStorage.h | 4 + .../registerDiskAzureBlobStorage.cpp | 1 - .../Cached/CachedObjectStorage.cpp | 7 ++ .../Cached/CachedObjectStorage.h | 2 + .../ObjectStorages/DiskObjectStorage.cpp | 3 - src/Disks/ObjectStorages/DiskObjectStorage.h | 4 +- .../ObjectStorages/HDFS/HDFSObjectStorage.h | 14 +++- .../ObjectStorages/HDFS/registerDiskHDFS.cpp | 1 - src/Disks/ObjectStorages/IObjectStorage.h | 3 + .../ObjectStorages/LocalObjectStorage.cpp | 8 ++ src/Disks/ObjectStorages/LocalObjectStorage.h | 3 + .../ObjectStorages/S3/S3ObjectStorage.cpp | 6 +- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 14 +++- .../ObjectStorages/S3/registerDiskS3.cpp | 3 +- .../ObjectStorages/Web/WebObjectStorage.h | 10 +++ .../Web/registerDiskWebServer.cpp | 1 - .../MergeTree/DataPartStorageOnDisk.cpp | 2 +- src/Storages/MergeTree/DataPartsExchange.cpp | 4 +- src/Storages/StorageReplicatedMergeTree.cpp | 15 ++-- src/Storages/StorageReplicatedMergeTree.h | 2 +- src/Storages/System/StorageSystemDisks.cpp | 2 +- 46 files changed, 334 insertions(+), 90 deletions(-) diff --git a/src/Backups/BackupEntryFromImmutableFile.cpp b/src/Backups/BackupEntryFromImmutableFile.cpp index 088324f364a..6df57e241e2 100644 --- a/src/Backups/BackupEntryFromImmutableFile.cpp +++ b/src/Backups/BackupEntryFromImmutableFile.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB @@ -44,4 +45,24 @@ std::unique_ptr BackupEntryFromImmutableFile::getReadBuffer( return createReadBufferFromFileBase(file_path, /* settings= */ {}); } + +DataSourceDescription BackupEntryFromImmutableFile::getDataSourceDescription() const +{ + if (disk) + return disk->getDataSourceDescription(); + + DataSourceDescription result{ + .type = DataSourceType::Local, + .is_encrypted = false, + .is_cached = false, + }; + + if (auto block_device_id = tryGetBlockDeviceId(file_path); block_device_id.has_value()) + result.description = *block_device_id; + else + result.description = file_path; + + return result; +} + } diff --git a/src/Backups/BackupEntryFromImmutableFile.h b/src/Backups/BackupEntryFromImmutableFile.h index e869a5e1fc0..1ff8aafe9a5 100644 --- a/src/Backups/BackupEntryFromImmutableFile.h +++ b/src/Backups/BackupEntryFromImmutableFile.h @@ -39,6 +39,8 @@ public: String getFilePath() const override { return file_path; } DiskPtr getDisk() const { return disk; } + DataSourceDescription getDataSourceDescription() const override; + private: const DiskPtr disk; const String file_path; diff --git a/src/Backups/BackupEntryFromMemory.h b/src/Backups/BackupEntryFromMemory.h index 6f7e2f8b9a3..b71c8c9a675 100644 --- a/src/Backups/BackupEntryFromMemory.h +++ b/src/Backups/BackupEntryFromMemory.h @@ -24,6 +24,11 @@ public: return ""; } + DataSourceDescription getDataSourceDescription() const override + { + return DataSourceDescription{DataSourceType::RAM, "", false, false}; + } + private: const String data; const std::optional checksum; diff --git a/src/Backups/BackupEntryFromSmallFile.cpp b/src/Backups/BackupEntryFromSmallFile.cpp index ca13fcd8f1e..d24b3a6498d 100644 --- a/src/Backups/BackupEntryFromSmallFile.cpp +++ b/src/Backups/BackupEntryFromSmallFile.cpp @@ -36,4 +36,5 @@ BackupEntryFromSmallFile::BackupEntryFromSmallFile( : BackupEntryFromMemory(readFile(disk_, file_path_), checksum_), disk(disk_), file_path(file_path_) { } + } diff --git a/src/Backups/BackupEntryFromSmallFile.h b/src/Backups/BackupEntryFromSmallFile.h index 330d1b924c4..d69805e18a3 100644 --- a/src/Backups/BackupEntryFromSmallFile.h +++ b/src/Backups/BackupEntryFromSmallFile.h @@ -25,7 +25,6 @@ public: String getFilePath() const override { return file_path; } DiskPtr getDisk() const { return disk; } - private: const DiskPtr disk; const String file_path; diff --git a/src/Backups/BackupIO.h b/src/Backups/BackupIO.h index f5a274ea28b..cc40b508b10 100644 --- a/src/Backups/BackupIO.h +++ b/src/Backups/BackupIO.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB { @@ -15,6 +16,7 @@ public: virtual bool fileExists(const String & file_name) = 0; virtual UInt64 getFileSize(const String & file_name) = 0; virtual std::unique_ptr readFile(const String & file_name) = 0; + virtual DataSourceDescription getDataSourceDescription() const = 0; }; /// Represents operations of storing to disk or uploading for writing a backup. @@ -27,7 +29,14 @@ public: virtual bool fileContentsEqual(const String & file_name, const String & expected_file_contents) = 0; virtual std::unique_ptr writeFile(const String & file_name) = 0; virtual void removeFiles(const Strings & file_names) = 0; + virtual DataSourceDescription getDataSourceDescription() const = 0; virtual void copyFileThroughBuffer(std::unique_ptr && source, const String & file_name); + + virtual bool supportNativeCopy(DataSourceDescription /* data_source_description */) const + { + return false; + } + virtual void copyFileNative(const String & file_name_from, const String & file_name_to); }; diff --git a/src/Backups/BackupIO_Disk.cpp b/src/Backups/BackupIO_Disk.cpp index 22a86825387..ef34a690279 100644 --- a/src/Backups/BackupIO_Disk.cpp +++ b/src/Backups/BackupIO_Disk.cpp @@ -77,4 +77,26 @@ void BackupWriterDisk::removeFiles(const Strings & file_names) disk->removeDirectory(path); } +DataSourceDescription BackupWriterDisk::getDataSourceDescription() const +{ + return disk->getDataSourceDescription(); +} + +DataSourceDescription BackupReaderDisk::getDataSourceDescription() const +{ + return disk->getDataSourceDescription(); +} + +bool BackupWriterDisk::supportNativeCopy(DataSourceDescription data_source_description) const +{ + return data_source_description == disk->getDataSourceDescription(); +} + +void BackupWriterDisk::copyFileNative(const String & file_name_from, const String & file_name_to) +{ + auto file_path = path / file_name_to; + disk->createDirectories(file_path.parent_path()); + disk->copyFile(file_name_from, *disk, file_path); +} + } diff --git a/src/Backups/BackupIO_Disk.h b/src/Backups/BackupIO_Disk.h index beea5f120c2..f984da5fda0 100644 --- a/src/Backups/BackupIO_Disk.h +++ b/src/Backups/BackupIO_Disk.h @@ -17,6 +17,7 @@ public: bool fileExists(const String & file_name) override; UInt64 getFileSize(const String & file_name) override; std::unique_ptr readFile(const String & file_name) override; + DataSourceDescription getDataSourceDescription() const override; private: DiskPtr disk; @@ -34,6 +35,11 @@ public: bool fileContentsEqual(const String & file_name, const String & expected_file_contents) override; std::unique_ptr writeFile(const String & file_name) override; void removeFiles(const Strings & file_names) override; + DataSourceDescription getDataSourceDescription() const override; + + bool supportNativeCopy(DataSourceDescription data_source_description) const override; + + void copyFileNative(const String & file_name_from, const String & file_name_to) override; private: DiskPtr disk; std::filesystem::path path; diff --git a/src/Backups/BackupIO_File.cpp b/src/Backups/BackupIO_File.cpp index 19c486ad682..eda7e9f7601 100644 --- a/src/Backups/BackupIO_File.cpp +++ b/src/Backups/BackupIO_File.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace fs = std::filesystem; @@ -79,4 +80,49 @@ void BackupWriterFile::removeFiles(const Strings & file_names) fs::remove(path); } +DataSourceDescription BackupWriterFile::getDataSourceDescription() const +{ + DataSourceDescription data_source_description; + + data_source_description.type = DataSourceType::Local; + + if (auto block_device_id = tryGetBlockDeviceId(path); block_device_id.has_value()) + data_source_description.description = *block_device_id; + else + data_source_description.description = path; + data_source_description.is_encrypted = false; + data_source_description.is_cached = false; + + return data_source_description; +} + +DataSourceDescription BackupReaderFile::getDataSourceDescription() const +{ + DataSourceDescription data_source_description; + + data_source_description.type = DataSourceType::Local; + + if (auto block_device_id = tryGetBlockDeviceId(path); block_device_id.has_value()) + data_source_description.description = *block_device_id; + else + data_source_description.description = path; + data_source_description.is_encrypted = false; + data_source_description.is_cached = false; + + return data_source_description; +} + + +bool BackupWriterFile::supportNativeCopy(DataSourceDescription data_source_description) const +{ + return data_source_description == getDataSourceDescription(); +} + +void BackupWriterFile::copyFileNative(const String & file_name_from, const String & file_name_to) +{ + auto file_path = path / file_name_to; + fs::create_directories(file_path.parent_path()); + fs::copy(file_name_from, file_path, fs::copy_options::recursive | fs::copy_options::overwrite_existing); +} + } diff --git a/src/Backups/BackupIO_File.h b/src/Backups/BackupIO_File.h index 85f4ec34ab4..93ed78877cc 100644 --- a/src/Backups/BackupIO_File.h +++ b/src/Backups/BackupIO_File.h @@ -15,6 +15,7 @@ public: bool fileExists(const String & file_name) override; UInt64 getFileSize(const String & file_name) override; std::unique_ptr readFile(const String & file_name) override; + DataSourceDescription getDataSourceDescription() const override; private: std::filesystem::path path; @@ -31,6 +32,11 @@ public: bool fileContentsEqual(const String & file_name, const String & expected_file_contents) override; std::unique_ptr writeFile(const String & file_name) override; void removeFiles(const Strings & file_names) override; + DataSourceDescription getDataSourceDescription() const override; + bool supportNativeCopy(DataSourceDescription data_source_description) const override; + + void copyFileNative(const String & file_name_from, const String & file_name_to) override; + private: std::filesystem::path path; }; diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index 4f9f6097496..b118ad514c3 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -116,6 +116,12 @@ public: return data_file_name; } + DataSourceDescription getDataSourceDescription() const override + { + return backup->reader->getDataSourceDescription(); + } + + private: const std::shared_ptr backup; const String archive_suffix; @@ -776,40 +782,52 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) if (!is_data_file_required) return; /// We copy data only if it's a new combination of size & checksum. - auto read_buffer = entry->getReadBuffer(); - - /// If we have prefix in base we will seek to the start of the suffix which differs - if (info.base_size != 0) - read_buffer->seek(info.base_size, SEEK_SET); - - if (!num_files_written) - checkLockFile(true); - - std::unique_ptr out; - if (use_archives) + /// We need to copy whole file without archive, we can do it faster + /// if source and destination are compatible + if (!use_archives && info.base_size == 0) { - String archive_suffix = current_archive_suffix; - bool next_suffix = false; - if (current_archive_suffix.empty() && is_internal_backup) - next_suffix = true; - /*if (archive_params.max_volume_size && current_archive_writer - && (current_archive_writer->getTotalSize() + size - base_size > archive_params.max_volume_size)) - next_suffix = true;*/ - if (next_suffix) - current_archive_suffix = coordination->getNextArchiveSuffix(); - - if (info.archive_suffix != current_archive_suffix) - { - info.archive_suffix = current_archive_suffix; - coordination->updateFileInfo(info); - } - out = getArchiveWriter(current_archive_suffix)->writeFile(info.data_file_name); - copyData(*read_buffer, *out); - out->finalize(); + auto writer_description = writer->getDataSourceDescription(); + auto reader_description = entry->getDataSourceDescription(); + /// Should be much faster than writing data through server + if (writer->supportNativeCopy(reader_description)) + writer->copyFileNative(entry->getFilePath(), info.data_file_name); } else { - writer->copyFileThroughBuffer(std::move(read_buffer), info.data_file_name); + auto read_buffer = entry->getReadBuffer(); + + /// If we have prefix in base we will seek to the start of the suffix which differs + if (info.base_size != 0) + read_buffer->seek(info.base_size, SEEK_SET); + + if (!num_files_written) + checkLockFile(true); + + if (use_archives) + { + String archive_suffix = current_archive_suffix; + bool next_suffix = false; + if (current_archive_suffix.empty() && is_internal_backup) + next_suffix = true; + /*if (archive_params.max_volume_size && current_archive_writer + && (current_archive_writer->getTotalSize() + size - base_size > archive_params.max_volume_size)) + next_suffix = true;*/ + if (next_suffix) + current_archive_suffix = coordination->getNextArchiveSuffix(); + + if (info.archive_suffix != current_archive_suffix) + { + info.archive_suffix = current_archive_suffix; + coordination->updateFileInfo(info); + } + auto out = getArchiveWriter(current_archive_suffix)->writeFile(info.data_file_name); + copyData(*read_buffer, *out); + out->finalize(); + } + else + { + writer->copyFileThroughBuffer(std::move(read_buffer), info.data_file_name); + } } ++num_files_written; diff --git a/src/Backups/IBackupEntriesLazyBatch.cpp b/src/Backups/IBackupEntriesLazyBatch.cpp index db2f5258316..ac67af3487a 100644 --- a/src/Backups/IBackupEntriesLazyBatch.cpp +++ b/src/Backups/IBackupEntriesLazyBatch.cpp @@ -25,6 +25,12 @@ public: return getInternalBackupEntry()->getFilePath(); } + + DataSourceDescription getDataSourceDescription() const override + { + return getInternalBackupEntry()->getDataSourceDescription(); + } + private: BackupEntryPtr getInternalBackupEntry() const { diff --git a/src/Backups/IBackupEntry.h b/src/Backups/IBackupEntry.h index 24b31807ddb..b9c74325277 100644 --- a/src/Backups/IBackupEntry.h +++ b/src/Backups/IBackupEntry.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -26,6 +27,8 @@ public: virtual std::unique_ptr getReadBuffer() const = 0; virtual String getFilePath() const = 0; + + virtual DataSourceDescription getDataSourceDescription() const = 0; }; using BackupEntryPtr = std::shared_ptr; diff --git a/src/Backups/registerBackupEnginesFileAndDisk.cpp b/src/Backups/registerBackupEnginesFileAndDisk.cpp index 380ae36a8e3..69af37fc6ff 100644 --- a/src/Backups/registerBackupEnginesFileAndDisk.cpp +++ b/src/Backups/registerBackupEnginesFileAndDisk.cpp @@ -46,7 +46,7 @@ namespace void checkPath(const String & disk_name, const DiskPtr & disk, fs::path & path) { path = path.lexically_normal(); - if (!path.is_relative() && (disk->getType() == DiskType::Local)) + if (!path.is_relative() && (disk->getDataSourceDescription().type == DataSourceType::Local)) path = path.lexically_proximate(disk->getPath()); bool path_ok = path.empty() || (path.is_relative() && (*path.begin() != "..")); diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 610608cd312..afe38dbb528 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -79,6 +79,22 @@ String getBlockDeviceId([[maybe_unused]] const String & path) #endif } + +std::optional tryGetBlockDeviceId([[maybe_unused]] const String & path) +{ +#if defined(OS_LINUX) + struct stat sb; + if (lstat(path.c_str(), &sb)) + return {}; + WriteBufferFromOwnString ss; + ss << major(sb.st_dev) << ":" << minor(sb.st_dev); + return ss.str(); +#else + return {}; +#endif + +} + #if !defined(OS_LINUX) [[noreturn]] #endif diff --git a/src/Common/filesystemHelpers.h b/src/Common/filesystemHelpers.h index f96fe269eab..b90df54c1f1 100644 --- a/src/Common/filesystemHelpers.h +++ b/src/Common/filesystemHelpers.h @@ -25,6 +25,8 @@ std::unique_ptr createTemporaryFile(const std::string & path); #endif String getBlockDeviceId([[maybe_unused]] const String & path); +std::optional tryGetBlockDeviceId([[maybe_unused]] const String & path); + enum class BlockDeviceType { UNKNOWN = 0, // we were unable to determine device type diff --git a/src/Disks/DiskDecorator.h b/src/Disks/DiskDecorator.h index 2298ab56f68..6bb8b541767 100644 --- a/src/Disks/DiskDecorator.h +++ b/src/Disks/DiskDecorator.h @@ -72,7 +72,7 @@ public: void sync(int fd) const; String getUniqueId(const String & path) const override { return delegate->getUniqueId(path); } bool checkUniqueId(const String & id) const override { return delegate->checkUniqueId(id); } - DiskType getType() const override { return delegate->getType(); } + DataSourceDescription getDataSourceDescription() const override { return delegate->getDataSourceDescription(); } bool isRemote() const override { return delegate->isRemote(); } bool supportZeroCopyReplication() const override { return delegate->supportZeroCopyReplication(); } bool supportParallelWrite() const override { return delegate->supportParallelWrite(); } diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index 1a714395f82..d2795e01086 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -234,7 +234,13 @@ public: void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String & config_prefix, const DisksMap & map) override; - DiskType getType() const override { return DiskType::Encrypted; } + DataSourceDescription getDataSourceDescription() const override + { + auto delegate_description = delegate->getDataSourceDescription(); + delegate_description.is_encrypted = true; + return delegate_description; + } + bool isRemote() const override { return delegate->isRemote(); } SyncGuardPtr getDirectorySyncGuard(const String & path) const override; diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 9015154a343..fd6379cc046 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -230,14 +230,14 @@ std::optional DiskLocal::tryReserve(UInt64 bytes) if (bytes == 0) { - LOG_DEBUG(log, "Reserving 0 bytes on disk {}", backQuote(name)); + LOG_DEBUG(logger, "Reserving 0 bytes on disk {}", backQuote(name)); ++reservation_count; return {unreserved_space}; } if (unreserved_space >= bytes) { - LOG_DEBUG(log, "Reserving {} on disk {}, having unreserved {}.", + LOG_DEBUG(logger, "Reserving {} on disk {}, having unreserved {}.", ReadableSize(bytes), backQuote(name), ReadableSize(unreserved_space)); ++reservation_count; reserved_bytes += bytes; @@ -497,6 +497,14 @@ DiskLocal::DiskLocal(const String & name_, const String & path_, UInt64 keep_fre , keep_free_space_bytes(keep_free_space_bytes_) , logger(&Poco::Logger::get("DiskLocal")) { + data_source_description.type = DataSourceType::Local; + + if (auto block_device_id = tryGetBlockDeviceId(disk_path); block_device_id.has_value()) + data_source_description.description = *block_device_id; + else + data_source_description.description = disk_path; + data_source_description.is_encrypted = false; + data_source_description.is_cached = false; } DiskLocal::DiskLocal( @@ -507,6 +515,11 @@ DiskLocal::DiskLocal( disk_checker = std::make_unique(this, context, local_disk_check_period_ms); } +DataSourceDescription DiskLocal::getDataSourceDescription() const +{ + return data_source_description; +} + void DiskLocal::startup(ContextPtr) { try @@ -615,7 +628,6 @@ DiskObjectStoragePtr DiskLocal::createDiskObjectStorage() "Local", metadata_storage, object_storage, - DiskType::Local, false, /* threadpool_size */16 ); diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index 9d3ce1d36b1..00e99f8fdc1 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -101,7 +101,8 @@ public: void truncateFile(const String & path, size_t size) override; - DiskType getType() const override { return DiskType::Local; } + DataSourceDescription getDataSourceDescription() const override; + bool isRemote() const override { return false; } bool supportZeroCopyReplication() const override { return false; } @@ -145,14 +146,13 @@ private: const String disk_checker_path = ".disk_checker_file"; std::atomic keep_free_space_bytes; Poco::Logger * logger; + DataSourceDescription data_source_description; UInt64 reserved_bytes = 0; UInt64 reservation_count = 0; static std::mutex reservation_mutex; - Poco::Logger * log = &Poco::Logger::get("DiskLocal"); - std::atomic broken{false}; std::atomic readonly{false}; std::unique_ptr disk_checker; diff --git a/src/Disks/DiskMemory.h b/src/Disks/DiskMemory.h index 0a1b16a4fea..9401d2827a6 100644 --- a/src/Disks/DiskMemory.h +++ b/src/Disks/DiskMemory.h @@ -91,7 +91,8 @@ public: void truncateFile(const String & path, size_t size) override; - DiskType getType() const override { return DiskType::RAM; } + DataSourceDescription getDataSourceDescription() const override { return DataSourceDescription{DataSourceType::RAM, "", false, false}; } + bool isRemote() const override { return false; } bool supportZeroCopyReplication() const override { return false; } diff --git a/src/Disks/DiskType.h b/src/Disks/DiskType.h index 2bb4be8f5c9..53f555db088 100644 --- a/src/Disks/DiskType.h +++ b/src/Disks/DiskType.h @@ -5,40 +5,44 @@ namespace DB { -enum class DiskType +enum class DataSourceType { Local, RAM, S3, HDFS, - Encrypted, WebServer, AzureBlobStorage, - Cache, }; -inline String toString(DiskType disk_type) +inline String toString(DataSourceType data_source_type) { - switch (disk_type) + switch (data_source_type) { - case DiskType::Local: + case DataSourceType::Local: return "local"; - case DiskType::RAM: + case DataSourceType::RAM: return "memory"; - case DiskType::S3: + case DataSourceType::S3: return "s3"; - case DiskType::HDFS: + case DataSourceType::HDFS: return "hdfs"; - case DiskType::Encrypted: - return "encrypted"; - case DiskType::WebServer: + case DataSourceType::WebServer: return "web"; - case DiskType::AzureBlobStorage: + case DataSourceType::AzureBlobStorage: return "azure_blob_storage"; - case DiskType::Cache: - return "cache"; } __builtin_unreachable(); } +struct DataSourceDescription +{ + DataSourceType type; + std::string description; + bool is_encrypted; + bool is_cached; + + bool operator==(const DataSourceDescription & o) const; +}; + } diff --git a/src/Disks/IDisk.cpp b/src/Disks/IDisk.cpp index 7d89fb28271..7a05d86b24e 100644 --- a/src/Disks/IDisk.cpp +++ b/src/Disks/IDisk.cpp @@ -113,7 +113,7 @@ void IDisk::copyDirectoryContent(const String & from_dir, const std::shared_ptr< void IDisk::truncateFile(const String &, size_t) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Truncate operation is not implemented for disk of type {}", getType()); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Truncate operation is not implemented for disk of type {}", getDataSourceDescription().type); } SyncGuardPtr IDisk::getDirectorySyncGuard(const String & /* path */) const diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 3e2b7535fcc..213b0fda638 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -227,7 +227,7 @@ public: virtual NameSet getCacheLayersNames() const { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method `getCacheLayersNames()` is not implemented for disk: {}", getType()); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method `getCacheLayersNames()` is not implemented for disk: {}", getDataSourceDescription().type); } /// Returns a list of storage objects (contains path, size, ...). @@ -235,7 +235,7 @@ public: /// be multiple files in remote fs for single clickhouse file. virtual StoredObjects getStorageObjects(const String &) const { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method `getStorageObjects() not implemented for disk: {}`", getType()); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method `getStorageObjects() not implemented for disk: {}`", getDataSourceDescription().type); } /// For one local path there might be multiple remote paths in case of Log family engines. @@ -243,7 +243,7 @@ public: virtual void getRemotePathsRecursive(const String &, std::vector &) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method `getRemotePathsRecursive() not implemented for disk: {}`", getType()); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method `getRemotePathsRecursive() not implemented for disk: {}`", getDataSourceDescription().type); } /// Batch request to remove multiple files. @@ -271,8 +271,8 @@ public: /// Truncate file to specified size. virtual void truncateFile(const String & path, size_t size); - /// Return disk type - "local", "s3", etc. - virtual DiskType getType() const = 0; + /// Return data source description + virtual DataSourceDescription getDataSourceDescription() const = 0; /// Involves network interaction. virtual bool isRemote() const = 0; @@ -357,7 +357,7 @@ public: throw Exception( ErrorCodes::NOT_IMPLEMENTED, "Method createDiskObjectStorage() is not implemented for disk type: {}", - getType()); + getDataSourceDescription().type); } virtual bool supportsStat() const { return false; } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 7c88ea70cec..f84ffe01a27 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -32,6 +32,10 @@ AzureObjectStorage::AzureObjectStorage( , settings(std::move(settings_)) , log(&Poco::Logger::get("AzureObjectStorage")) { + data_source_description.type = DataSourceType::AzureBlobStorage; + data_source_description.description = client.get()->GetUrl(); + data_source_description.is_cached = false; + data_source_description.is_encrypted = false; } std::string AzureObjectStorage::generateBlobNameForPath(const std::string & /* path */) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 9b5a16f6be5..0a45b8dd605 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -58,6 +58,8 @@ public: AzureClientPtr && client_, SettingsPtr && settings_); + DataSourceDescription getDataSourceDescription() const override { return data_source_description; } + std::string getName() const override { return "AzureObjectStorage"; } bool exists(const StoredObject & object) const override; @@ -129,6 +131,8 @@ private: MultiVersion settings; Poco::Logger * log; + + DataSourceDescription data_source_description; }; } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/registerDiskAzureBlobStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/registerDiskAzureBlobStorage.cpp index 8ec93aedfef..e6615316391 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/registerDiskAzureBlobStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/registerDiskAzureBlobStorage.cpp @@ -90,7 +90,6 @@ void registerDiskAzureBlobStorage(DiskFactory & factory) "DiskAzureBlobStorage", std::move(metadata_storage), std::move(azure_object_storage), - DiskType::AzureBlobStorage, send_metadata, copy_thread_pool_size ); diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp index f24b81bf2de..a68f9fdaf2d 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp @@ -34,6 +34,13 @@ CachedObjectStorage::CachedObjectStorage( cache->initialize(); } +DataSourceDescription CachedObjectStorage::getDataSourceDescription() const +{ + auto wrapped_object_storage_data_source = object_storage->getDataSourceDescription(); + wrapped_object_storage_data_source.is_cached = true; + return wrapped_object_storage_data_source; +} + FileCache::Key CachedObjectStorage::getCacheKey(const std::string & path) const { return cache->hash(path); diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index f99addb97d4..258b70ed579 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -20,6 +20,8 @@ class CachedObjectStorage final : public IObjectStorage public: CachedObjectStorage(ObjectStoragePtr object_storage_, FileCachePtr cache_, const FileCacheSettings & cache_settings_, const String & cache_config_name_); + DataSourceDescription getDataSourceDescription() const override; + std::string getName() const override { return fmt::format("CachedObjectStorage-{}({})", cache_config_name, object_storage->getName()); } bool exists(const StoredObject & object) const override; diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 75579247752..abb592dbf66 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -103,14 +103,12 @@ DiskObjectStorage::DiskObjectStorage( const String & log_name, MetadataStoragePtr metadata_storage_, ObjectStoragePtr object_storage_, - DiskType disk_type_, bool send_metadata_, uint64_t thread_pool_size_) : IDisk(getAsyncExecutor(log_name, thread_pool_size_)) , name(name_) , object_storage_root_path(object_storage_root_path_) , log (&Poco::Logger::get("DiskObjectStorage(" + log_name + ")")) - , disk_type(disk_type_) , metadata_storage(std::move(metadata_storage_)) , object_storage(std::move(object_storage_)) , send_metadata(send_metadata_) @@ -469,7 +467,6 @@ DiskObjectStoragePtr DiskObjectStorage::createDiskObjectStorage() getName(), metadata_storage, object_storage, - disk_type, send_metadata, threadpool_size); } diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.h b/src/Disks/ObjectStorages/DiskObjectStorage.h index 1b53cd514a0..1d21c71174c 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.h +++ b/src/Disks/ObjectStorages/DiskObjectStorage.h @@ -34,14 +34,13 @@ public: const String & log_name, MetadataStoragePtr metadata_storage_, ObjectStoragePtr object_storage_, - DiskType disk_type_, bool send_metadata_, uint64_t thread_pool_size_); /// Create fake transaction DiskTransactionPtr createTransaction() override; - DiskType getType() const override { return disk_type; } + DataSourceDescription getDataSourceDescription() const override { return object_storage->getDataSourceDescription(); } bool supportZeroCopyReplication() const override { return true; } @@ -206,7 +205,6 @@ private: const String object_storage_root_path; Poco::Logger * log; - const DiskType disk_type; MetadataStoragePtr metadata_storage; ObjectStoragePtr object_storage; diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h index 4687d63c128..bbf2f593a68 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h @@ -48,10 +48,20 @@ public: , hdfs_builder(createHDFSBuilder(hdfs_root_path_, config)) , hdfs_fs(createHDFSFS(hdfs_builder.get())) , settings(std::move(settings_)) - {} + { + data_source_description.type = DataSourceType::HDFS; + data_source_description.description = hdfs_root_path_; + data_source_description.is_cached = false; + data_source_description.is_encrypted = false; + } std::string getName() const override { return "HDFSObjectStorage"; } + DataSourceDescription getDataSourceDescription() const override + { + return data_source_description; + } + bool exists(const StoredObject & object) const override; std::unique_ptr readObject( /// NOLINT @@ -121,6 +131,8 @@ private: HDFSFSPtr hdfs_fs; SettingsPtr settings; + + DataSourceDescription data_source_description; }; } diff --git a/src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp b/src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp index a2e9fb3e4ad..a9189e0101b 100644 --- a/src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp +++ b/src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp @@ -49,7 +49,6 @@ void registerDiskHDFS(DiskFactory & factory) "DiskHDFS", std::move(metadata_storage), std::move(hdfs_storage), - DiskType::HDFS, /* send_metadata = */ false, copy_thread_pool_size); diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index ae5a2587ddd..89d0e0d6d03 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -15,6 +15,7 @@ #include #include +#include #include #include #include @@ -58,6 +59,8 @@ class IObjectStorage public: IObjectStorage() = default; + virtual DataSourceDescription getDataSourceDescription() const = 0; + virtual std::string getName() const = 0; /// Object exists or not diff --git a/src/Disks/ObjectStorages/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/LocalObjectStorage.cpp index a25f2ba18c4..64512a07919 100644 --- a/src/Disks/ObjectStorages/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/LocalObjectStorage.cpp @@ -28,6 +28,14 @@ namespace ErrorCodes LocalObjectStorage::LocalObjectStorage() : log(&Poco::Logger::get("LocalObjectStorage")) { + data_source_description.type = DataSourceType::Local; + if (auto block_device_id = tryGetBlockDeviceId("/"); block_device_id.has_value()) + data_source_description.description = *block_device_id; + else + data_source_description.description = "/"; + + data_source_description.is_cached = false; + data_source_description.is_encrypted = false; } bool LocalObjectStorage::exists(const StoredObject & object) const diff --git a/src/Disks/ObjectStorages/LocalObjectStorage.h b/src/Disks/ObjectStorages/LocalObjectStorage.h index 2225d0c72b7..de38581e7bb 100644 --- a/src/Disks/ObjectStorages/LocalObjectStorage.h +++ b/src/Disks/ObjectStorages/LocalObjectStorage.h @@ -17,6 +17,8 @@ class LocalObjectStorage : public IObjectStorage public: LocalObjectStorage(); + DataSourceDescription getDataSourceDescription() const override { return data_source_description; } + std::string getName() const override { return "LocalObjectStorage"; } bool exists(const StoredObject & object) const override; @@ -86,6 +88,7 @@ public: private: Poco::Logger * log; + DataSourceDescription data_source_description; }; } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index a18275ff5ff..ac1fe9447fb 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -2,6 +2,8 @@ #if USE_AWS_S3 +#include + #include #include #include @@ -25,6 +27,7 @@ #include #include + #include #include @@ -511,7 +514,8 @@ std::unique_ptr S3ObjectStorage::cloneObjectStorage( return std::make_unique( getClient(config, config_prefix, context), getSettings(config, config_prefix, context), - version_id, s3_capabilities, new_namespace); + version_id, s3_capabilities, new_namespace, + S3::URI(Poco::URI(config.getString(config_prefix + ".endpoint"))).endpoint); } } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index 63b254b719a..448826bfa71 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -48,13 +48,23 @@ public: std::unique_ptr && s3_settings_, String version_id_, const S3Capabilities & s3_capabilities_, - String bucket_) + String bucket_, + String connection_string) : bucket(bucket_) , client(std::move(client_)) , s3_settings(std::move(s3_settings_)) , s3_capabilities(s3_capabilities_) , version_id(std::move(version_id_)) { + data_source_description.type = DataSourceType::S3; + data_source_description.description = connection_string; + data_source_description.is_cached = false; + data_source_description.is_encrypted = false; + } + + DataSourceDescription getDataSourceDescription() const override + { + return data_source_description; } std::string getName() const override { return "S3ObjectStorage"; } @@ -169,6 +179,8 @@ private: S3Capabilities s3_capabilities; const String version_id; + + DataSourceDescription data_source_description; }; } diff --git a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp index 37d23d87c0f..378f7058322 100644 --- a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp +++ b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp @@ -133,7 +133,7 @@ void registerDiskS3(DiskFactory & factory) auto s3_storage = std::make_unique( getClient(config, config_prefix, context), getSettings(config, config_prefix, context), - uri.version_id, s3_capabilities, uri.bucket); + uri.version_id, s3_capabilities, uri.bucket, uri.endpoint); bool skip_access_check = config.getBool(config_prefix + ".skip_access_check", false); @@ -162,7 +162,6 @@ void registerDiskS3(DiskFactory & factory) "DiskS3", std::move(metadata_storage), std::move(s3_storage), - DiskType::S3, send_metadata, copy_thread_pool_size); diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.h b/src/Disks/ObjectStorages/Web/WebObjectStorage.h index fd9e9e9ce42..7d0380a37fd 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.h +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.h @@ -20,6 +20,16 @@ class WebObjectStorage : public IObjectStorage, WithContext public: WebObjectStorage(const String & url_, ContextPtr context_); + DataSourceDescription getDataSourceDescription() const override + { + return DataSourceDescription{ + .type = DataSourceType::WebServer, + .description = url, + .is_encrypted = false, + .is_cached = false, + }; + } + std::string getName() const override { return "WebObjectStorage"; } bool exists(const StoredObject & object) const override; diff --git a/src/Disks/ObjectStorages/Web/registerDiskWebServer.cpp b/src/Disks/ObjectStorages/Web/registerDiskWebServer.cpp index 1051440e16c..5ef3fad4a0a 100644 --- a/src/Disks/ObjectStorages/Web/registerDiskWebServer.cpp +++ b/src/Disks/ObjectStorages/Web/registerDiskWebServer.cpp @@ -47,7 +47,6 @@ void registerDiskWebServer(DiskFactory & factory) "DiskWebServer", metadata_storage, object_storage, - DiskType::WebServer, /* send_metadata */false, /* threadpool_size */16); }; diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp index 1cd6fd0305b..0154fd6e281 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp @@ -415,7 +415,7 @@ std::string DataPartStorageOnDisk::getDiskName() const std::string DataPartStorageOnDisk::getDiskType() const { - return toString(volume->getDisk()->getType()); + return toString(volume->getDisk()->getDataSourceDescription().type); } bool DataPartStorageOnDisk::isStoredOnRemoteDisk() const diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 5fe3ee4da28..02821de8629 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -458,11 +458,11 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( Disks disks = data.getDisks(); for (const auto & data_disk : disks) if (data_disk->supportZeroCopyReplication()) - capability.push_back(toString(data_disk->getType())); + capability.push_back(toString(data_disk->getDataSourceDescription().type)); } else if (disk->supportZeroCopyReplication()) { - capability.push_back(toString(disk->getType())); + capability.push_back(toString(disk->getDataSourceDescription().type)); } } if (!capability.empty()) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 393cdf76c7a..3c7ff418901 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7605,7 +7605,7 @@ void StorageReplicatedMergeTree::lockSharedDataTemporary(const String & part_nam String id = part_id; boost::replace_all(id, "/", "_"); - Strings zc_zookeeper_paths = getZeroCopyPartPath(*getSettings(), toString(disk->getType()), getTableSharedID(), + Strings zc_zookeeper_paths = getZeroCopyPartPath(*getSettings(), toString(disk->getDataSourceDescription().type), getTableSharedID(), part_name, zookeeper_path); for (const auto & zc_zookeeper_path : zc_zookeeper_paths) @@ -7795,11 +7795,11 @@ DataPartStoragePtr StorageReplicatedMergeTree::tryToFetchIfShared( const String & path) { const auto settings = getSettings(); - auto disk_type = disk->getType(); + auto data_source_description = disk->getDataSourceDescription(); if (!(disk->supportZeroCopyReplication() && settings->allow_remote_fs_zero_copy_replication)) return nullptr; - String replica = getSharedDataReplica(part, disk_type); + String replica = getSharedDataReplica(part, data_source_description.type); /// We can't fetch part when none replicas have this part on a same type remote disk if (replica.empty()) @@ -7808,9 +7808,8 @@ DataPartStoragePtr StorageReplicatedMergeTree::tryToFetchIfShared( return executeFetchShared(replica, part.name, disk, path); } - String StorageReplicatedMergeTree::getSharedDataReplica( - const IMergeTreeDataPart & part, DiskType disk_type) const + const IMergeTreeDataPart & part, DataSourceType data_source_type) const { String best_replica; @@ -7818,7 +7817,7 @@ String StorageReplicatedMergeTree::getSharedDataReplica( if (!zookeeper) return ""; - Strings zc_zookeeper_paths = getZeroCopyPartPath(*getSettings(), toString(disk_type), getTableSharedID(), part.name, + Strings zc_zookeeper_paths = getZeroCopyPartPath(*getSettings(), toString(data_source_type), getTableSharedID(), part.name, zookeeper_path); std::set replicas; @@ -7929,7 +7928,7 @@ std::optional StorageReplicatedMergeTree::getZeroCopyPartPath(const Stri if (!disk || !disk->supportZeroCopyReplication()) return std::nullopt; - return getZeroCopyPartPath(*getSettings(), toString(disk->getType()), getTableSharedID(), part_name, zookeeper_path)[0]; + return getZeroCopyPartPath(*getSettings(), toString(disk->getDataSourceDescription().type), getTableSharedID(), part_name, zookeeper_path)[0]; } std::optional StorageReplicatedMergeTree::tryCreateZeroCopyExclusiveLock(const String & part_name, const DiskPtr & disk) @@ -8334,7 +8333,7 @@ bool StorageReplicatedMergeTree::removeSharedDetachedPart(DiskPtr disk, const St String id = disk->getUniqueId(checksums); bool can_remove = false; std::tie(can_remove, files_not_to_remove) = StorageReplicatedMergeTree::unlockSharedDataByID(id, table_uuid, part_name, - detached_replica_name, toString(disk->getType()), zookeeper, local_context->getReplicatedMergeTreeSettings(), &Poco::Logger::get("StorageReplicatedMergeTree"), + detached_replica_name, toString(disk->getDataSourceDescription().type), zookeeper, local_context->getReplicatedMergeTreeSettings(), &Poco::Logger::get("StorageReplicatedMergeTree"), detached_zookeeper_path); keep_shared = !can_remove; diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 3f03fb70f7a..c1db1bad9ad 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -286,7 +286,7 @@ public: DataPartStoragePtr tryToFetchIfShared(const IMergeTreeDataPart & part, const DiskPtr & disk, const String & path) override; /// Get best replica having this partition on a same type remote disk - String getSharedDataReplica(const IMergeTreeDataPart & part, DiskType disk_type) const; + String getSharedDataReplica(const IMergeTreeDataPart & part, DataSourceType data_source_type) const; inline String getReplicaName() const { return replica_name; } diff --git a/src/Storages/System/StorageSystemDisks.cpp b/src/Storages/System/StorageSystemDisks.cpp index 86238ab8df1..9546c854ebf 100644 --- a/src/Storages/System/StorageSystemDisks.cpp +++ b/src/Storages/System/StorageSystemDisks.cpp @@ -54,7 +54,7 @@ Pipe StorageSystemDisks::read( col_free->insert(disk_ptr->getAvailableSpace()); col_total->insert(disk_ptr->getTotalSpace()); col_keep->insert(disk_ptr->getKeepingFreeSpace()); - col_type->insert(toString(disk_ptr->getType())); + col_type->insert(toString(disk_ptr->getDataSourceDescription().type)); String cache_path; if (disk_ptr->supportsCache()) From 8fd308845994c4622ff3c91547f07648c41fe8d0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 20 Aug 2022 17:21:03 +0200 Subject: [PATCH 03/44] Commit missed files --- src/Backups/BackupIO.cpp | 27 +++++++++++++++++++++++++++ src/Disks/DiskType.cpp | 11 +++++++++++ 2 files changed, 38 insertions(+) create mode 100644 src/Backups/BackupIO.cpp create mode 100644 src/Disks/DiskType.cpp diff --git a/src/Backups/BackupIO.cpp b/src/Backups/BackupIO.cpp new file mode 100644 index 00000000000..26510d67d75 --- /dev/null +++ b/src/Backups/BackupIO.cpp @@ -0,0 +1,27 @@ +#include + +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +void IBackupWriter::copyFileThroughBuffer(std::unique_ptr && source, const String & file_name) +{ + auto write_buffer = writeFile(file_name); + copyData(*source, *write_buffer); + write_buffer->finalize(); +} + +void IBackupWriter::copyFileNative(const String & /* file_name_from */, const String & /* file_name_to */) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Native copy not implemented for backup writer"); +} + +} diff --git a/src/Disks/DiskType.cpp b/src/Disks/DiskType.cpp new file mode 100644 index 00000000000..b5ae9ca4c71 --- /dev/null +++ b/src/Disks/DiskType.cpp @@ -0,0 +1,11 @@ +#include "DiskType.h" + +namespace DB +{ + +bool DataSourceDescription::operator==(const DataSourceDescription & o) const +{ + return std::tie(type, description, is_encrypted) == std::tie(o.type, o.description, o.is_encrypted); +} + +} From 704d7fdc41f69753337be60911976194134e02f5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 21 Aug 2022 18:18:35 +0200 Subject: [PATCH 04/44] Fix copy and use disks --- src/Backups/BackupEntryFromAppendOnlyFile.cpp | 10 ----- src/Backups/BackupEntryFromAppendOnlyFile.h | 7 --- src/Backups/BackupEntryFromImmutableFile.cpp | 35 +++------------ src/Backups/BackupEntryFromImmutableFile.h | 13 ++---- src/Backups/BackupEntryFromMemory.h | 2 + src/Backups/BackupEntryFromSmallFile.cpp | 2 + src/Backups/BackupEntryFromSmallFile.h | 3 +- src/Backups/BackupIO.cpp | 2 +- src/Backups/BackupIO.h | 3 +- src/Backups/BackupIO_Disk.cpp | 13 +++++- src/Backups/BackupIO_Disk.h | 2 +- src/Backups/BackupIO_File.cpp | 10 ++++- src/Backups/BackupIO_File.h | 2 +- src/Backups/BackupImpl.cpp | 43 ++++++++++++++++--- src/Backups/BackupsWorker.cpp | 6 ++- src/Backups/IBackupEntriesLazyBatch.cpp | 4 ++ src/Backups/IBackupEntry.h | 3 ++ src/Disks/DiskLocal.cpp | 7 +++ src/Disks/DiskLocal.h | 2 + src/Disks/DiskMemory.cpp | 9 ++++ src/Disks/DiskMemory.h | 2 + src/Disks/IDisk.cpp | 14 ------ src/Disks/IDisk.h | 2 +- .../ObjectStorages/DiskObjectStorage.cpp | 16 +++++++ src/Disks/ObjectStorages/DiskObjectStorage.h | 2 + .../test_backup_restore_new/test.py | 14 ++++++ 26 files changed, 142 insertions(+), 86 deletions(-) diff --git a/src/Backups/BackupEntryFromAppendOnlyFile.cpp b/src/Backups/BackupEntryFromAppendOnlyFile.cpp index fa816091bdf..910b6b6507e 100644 --- a/src/Backups/BackupEntryFromAppendOnlyFile.cpp +++ b/src/Backups/BackupEntryFromAppendOnlyFile.cpp @@ -5,16 +5,6 @@ namespace DB { -BackupEntryFromAppendOnlyFile::BackupEntryFromAppendOnlyFile( - const String & file_path_, - const std::optional & file_size_, - const std::optional & checksum_, - const std::shared_ptr & temporary_file_) - : BackupEntryFromImmutableFile(file_path_, file_size_, checksum_, temporary_file_) - , limit(BackupEntryFromImmutableFile::getSize()) -{ -} - BackupEntryFromAppendOnlyFile::BackupEntryFromAppendOnlyFile( const DiskPtr & disk_, const String & file_path_, diff --git a/src/Backups/BackupEntryFromAppendOnlyFile.h b/src/Backups/BackupEntryFromAppendOnlyFile.h index d868f82d45f..49a587faf85 100644 --- a/src/Backups/BackupEntryFromAppendOnlyFile.h +++ b/src/Backups/BackupEntryFromAppendOnlyFile.h @@ -11,13 +11,6 @@ namespace DB class BackupEntryFromAppendOnlyFile : public BackupEntryFromImmutableFile { public: - /// The constructor is allowed to not set `file_size_` or `checksum_`, in that case it will be calculated from the data. - explicit BackupEntryFromAppendOnlyFile( - const String & file_path_, - const std::optional & file_size_ = {}, - const std::optional & checksum_ = {}, - const std::shared_ptr & temporary_file_ = {}); - BackupEntryFromAppendOnlyFile( const DiskPtr & disk_, const String & file_path_, diff --git a/src/Backups/BackupEntryFromImmutableFile.cpp b/src/Backups/BackupEntryFromImmutableFile.cpp index 6df57e241e2..86b9c13fb9a 100644 --- a/src/Backups/BackupEntryFromImmutableFile.cpp +++ b/src/Backups/BackupEntryFromImmutableFile.cpp @@ -8,15 +8,6 @@ namespace DB { -BackupEntryFromImmutableFile::BackupEntryFromImmutableFile( - const String & file_path_, - const std::optional & file_size_, - const std::optional & checksum_, - const std::shared_ptr & temporary_file_) - : file_path(file_path_), file_size(file_size_), checksum(checksum_), temporary_file(temporary_file_) -{ -} - BackupEntryFromImmutableFile::BackupEntryFromImmutableFile( const DiskPtr & disk_, const String & file_path_, @@ -33,36 +24,24 @@ UInt64 BackupEntryFromImmutableFile::getSize() const { std::lock_guard lock{get_file_size_mutex}; if (!file_size) - file_size = disk ? disk->getFileSize(file_path) : Poco::File(file_path).getSize(); + file_size = disk->getFileSize(file_path); return *file_size; } std::unique_ptr BackupEntryFromImmutableFile::getReadBuffer() const { - if (disk) - return disk->readFile(file_path); - else - return createReadBufferFromFileBase(file_path, /* settings= */ {}); + return disk->readFile(file_path); } DataSourceDescription BackupEntryFromImmutableFile::getDataSourceDescription() const { - if (disk) - return disk->getDataSourceDescription(); + return disk->getDataSourceDescription(); +} - DataSourceDescription result{ - .type = DataSourceType::Local, - .is_encrypted = false, - .is_cached = false, - }; - - if (auto block_device_id = tryGetBlockDeviceId(file_path); block_device_id.has_value()) - result.description = *block_device_id; - else - result.description = file_path; - - return result; +String BackupEntryFromImmutableFile::getFilePath() const +{ + return file_path; } } diff --git a/src/Backups/BackupEntryFromImmutableFile.h b/src/Backups/BackupEntryFromImmutableFile.h index 1ff8aafe9a5..68c50ff79af 100644 --- a/src/Backups/BackupEntryFromImmutableFile.h +++ b/src/Backups/BackupEntryFromImmutableFile.h @@ -16,13 +16,6 @@ using DiskPtr = std::shared_ptr; class BackupEntryFromImmutableFile : public IBackupEntry { public: - /// The constructor is allowed to not set `file_size_` or `checksum_`, in that case it will be calculated from the data. - explicit BackupEntryFromImmutableFile( - const String & file_path_, - const std::optional & file_size_ = {}, - const std::optional & checksum_ = {}, - const std::shared_ptr & temporary_file_ = {}); - BackupEntryFromImmutableFile( const DiskPtr & disk_, const String & file_path_, @@ -36,11 +29,11 @@ public: std::optional getChecksum() const override { return checksum; } std::unique_ptr getReadBuffer() const override; - String getFilePath() const override { return file_path; } - DiskPtr getDisk() const { return disk; } - + String getFilePath() const override; DataSourceDescription getDataSourceDescription() const override; + std::shared_ptr tryGetDiskIfExists() const override { return disk; } + private: const DiskPtr disk; const String file_path; diff --git a/src/Backups/BackupEntryFromMemory.h b/src/Backups/BackupEntryFromMemory.h index b71c8c9a675..fb2f32d980d 100644 --- a/src/Backups/BackupEntryFromMemory.h +++ b/src/Backups/BackupEntryFromMemory.h @@ -29,6 +29,8 @@ public: return DataSourceDescription{DataSourceType::RAM, "", false, false}; } + std::shared_ptr tryGetDiskIfExists() const override { return nullptr; } + private: const String data; const std::optional checksum; diff --git a/src/Backups/BackupEntryFromSmallFile.cpp b/src/Backups/BackupEntryFromSmallFile.cpp index d24b3a6498d..b28197e7837 100644 --- a/src/Backups/BackupEntryFromSmallFile.cpp +++ b/src/Backups/BackupEntryFromSmallFile.cpp @@ -29,12 +29,14 @@ namespace BackupEntryFromSmallFile::BackupEntryFromSmallFile(const String & file_path_, const std::optional & checksum_) : BackupEntryFromMemory(readFile(file_path_), checksum_), file_path(file_path_) { + LOG_DEBUG(&Poco::Logger::get("DEBUG"), "SMALL FILE {}", file_path); } BackupEntryFromSmallFile::BackupEntryFromSmallFile( const DiskPtr & disk_, const String & file_path_, const std::optional & checksum_) : BackupEntryFromMemory(readFile(disk_, file_path_), checksum_), disk(disk_), file_path(file_path_) { + LOG_DEBUG(&Poco::Logger::get("DEBUG"), "SMALL FILE {}", file_path); } } diff --git a/src/Backups/BackupEntryFromSmallFile.h b/src/Backups/BackupEntryFromSmallFile.h index d69805e18a3..66ab1c4f3d3 100644 --- a/src/Backups/BackupEntryFromSmallFile.h +++ b/src/Backups/BackupEntryFromSmallFile.h @@ -24,7 +24,8 @@ public: const std::optional & checksum_ = {}); String getFilePath() const override { return file_path; } - DiskPtr getDisk() const { return disk; } + + std::shared_ptr tryGetDiskIfExists() const override { return disk; } private: const DiskPtr disk; const String file_path; diff --git a/src/Backups/BackupIO.cpp b/src/Backups/BackupIO.cpp index 26510d67d75..4d1b5a88216 100644 --- a/src/Backups/BackupIO.cpp +++ b/src/Backups/BackupIO.cpp @@ -19,7 +19,7 @@ void IBackupWriter::copyFileThroughBuffer(std::unique_ptr && write_buffer->finalize(); } -void IBackupWriter::copyFileNative(const String & /* file_name_from */, const String & /* file_name_to */) +void IBackupWriter::copyFileNative(std::shared_ptr /* from_disk */, const String & /* file_name_from */, const String & /* file_name_to */) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Native copy not implemented for backup writer"); } diff --git a/src/Backups/BackupIO.h b/src/Backups/BackupIO.h index cc40b508b10..c8e4b102970 100644 --- a/src/Backups/BackupIO.h +++ b/src/Backups/BackupIO.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB { @@ -37,7 +38,7 @@ public: return false; } - virtual void copyFileNative(const String & file_name_from, const String & file_name_to); + virtual void copyFileNative(std::shared_ptr from_disk, const String & file_name_from, const String & file_name_to); }; } diff --git a/src/Backups/BackupIO_Disk.cpp b/src/Backups/BackupIO_Disk.cpp index ef34a690279..b87f14d9e7f 100644 --- a/src/Backups/BackupIO_Disk.cpp +++ b/src/Backups/BackupIO_Disk.cpp @@ -6,6 +6,12 @@ namespace DB { + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + BackupReaderDisk::BackupReaderDisk(const DiskPtr & disk_, const String & path_) : disk(disk_), path(path_) { } @@ -92,11 +98,14 @@ bool BackupWriterDisk::supportNativeCopy(DataSourceDescription data_source_descr return data_source_description == disk->getDataSourceDescription(); } -void BackupWriterDisk::copyFileNative(const String & file_name_from, const String & file_name_to) +void BackupWriterDisk::copyFileNative(std::shared_ptr from_disk, const String & file_name_from, const String & file_name_to) { auto file_path = path / file_name_to; disk->createDirectories(file_path.parent_path()); - disk->copyFile(file_name_from, *disk, file_path); + if (from_disk) + from_disk->copyFile(file_name_from, *disk, file_path); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot natively copy data to disk without source disk"); } } diff --git a/src/Backups/BackupIO_Disk.h b/src/Backups/BackupIO_Disk.h index f984da5fda0..abc5b831c7b 100644 --- a/src/Backups/BackupIO_Disk.h +++ b/src/Backups/BackupIO_Disk.h @@ -39,7 +39,7 @@ public: bool supportNativeCopy(DataSourceDescription data_source_description) const override; - void copyFileNative(const String & file_name_from, const String & file_name_to) override; + void copyFileNative(std::shared_ptr from_disk, const String & file_name_from, const String & file_name_to) override; private: DiskPtr disk; std::filesystem::path path; diff --git a/src/Backups/BackupIO_File.cpp b/src/Backups/BackupIO_File.cpp index eda7e9f7601..a401fb1a5d5 100644 --- a/src/Backups/BackupIO_File.cpp +++ b/src/Backups/BackupIO_File.cpp @@ -118,11 +118,17 @@ bool BackupWriterFile::supportNativeCopy(DataSourceDescription data_source_descr return data_source_description == getDataSourceDescription(); } -void BackupWriterFile::copyFileNative(const String & file_name_from, const String & file_name_to) +void BackupWriterFile::copyFileNative(std::shared_ptr from_disk, const String & file_name_from, const String & file_name_to) { auto file_path = path / file_name_to; fs::create_directories(file_path.parent_path()); - fs::copy(file_name_from, file_path, fs::copy_options::recursive | fs::copy_options::overwrite_existing); + std::string abs_source_path; + if (from_disk) + abs_source_path = fullPath(from_disk, file_name_from); + else + abs_source_path = fs::absolute(file_name_from); + + fs::copy(abs_source_path, file_path, fs::copy_options::recursive | fs::copy_options::overwrite_existing); } } diff --git a/src/Backups/BackupIO_File.h b/src/Backups/BackupIO_File.h index 93ed78877cc..a3c6373fffe 100644 --- a/src/Backups/BackupIO_File.h +++ b/src/Backups/BackupIO_File.h @@ -35,7 +35,7 @@ public: DataSourceDescription getDataSourceDescription() const override; bool supportNativeCopy(DataSourceDescription data_source_description) const override; - void copyFileNative(const String & file_name_from, const String & file_name_to) override; + void copyFileNative(std::shared_ptr from_disk, const String & file_name_from, const String & file_name_to) override; private: std::filesystem::path path; diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index b118ad514c3..b569c29887f 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -116,6 +116,11 @@ public: return data_file_name; } + std::shared_ptr tryGetDiskIfExists() const override + { + return nullptr; + } + DataSourceDescription getDataSourceDescription() const override { return backup->reader->getDataSourceDescription(); @@ -677,6 +682,7 @@ ChecksumsForNewEntry calculateNewEntryChecksumsIfNeeded(BackupEntryPtr entry, si void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) { + std::lock_guard lock{mutex}; if (open_mode != OpenMode::WRITE) throw Exception("Backup is not opened for writing", ErrorCodes::LOGICAL_ERROR); @@ -684,6 +690,11 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) if (writing_finalized) throw Exception("Backup is already finalized", ErrorCodes::LOGICAL_ERROR); + std::string from_file_name = "memory buffer"; + if (auto fname = entry->getFilePath(); !fname.empty()) + from_file_name = "file " + fname; + LOG_TRACE(log, "Writing backup for file {} from file {}", file_name, from_file_name); + auto adjusted_path = removeLeadingSlash(file_name); if (coordination->getFileInfo(adjusted_path)) throw Exception( @@ -710,6 +721,7 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) /// If file has no checksum -- calculate and fill it. if (base_backup_file_info.has_value()) { + LOG_TRACE(log, "File {} found in base backup, checking for equality", adjusted_path); CheckBackupResult check_base = checkBaseBackupForFile(*base_backup_file_info, info); /// File with the same name but smaller size exist in previous backup @@ -722,9 +734,14 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) /// In ClickHouse this can happen for StorageLog for example. if (checksums.prefix_checksum == base_backup_file_info->second) { + LOG_TRACE(log, "File prefix of {} in base backup, will write rest part of file to current backup", adjusted_path); info.base_size = base_backup_file_info->first; info.base_checksum = base_backup_file_info->second; } + else + { + LOG_TRACE(log, "Prefix checksum of file {} doesn't match with checksum in base backup", adjusted_path); + } } else { @@ -735,6 +752,7 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) if (info.checksum == base_backup_file_info->second) { + LOG_TRACE(log, "Found whole file {} in base backup", adjusted_path); assert(check_base == CheckBackupResult::HasFull); assert(info.size == base_backup_file_info->first); @@ -744,11 +762,17 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) /// but we intentionally don't do it, otherwise control flow /// of this function will be very complex. } + else + { + LOG_TRACE(log, "Whole file {} in base backup doesn't match by checksum", adjusted_path); + } } } else /// We don't have info about this file_name (sic!) in base backup, /// however file could be renamed, so we will check one more time using size and checksum { + + LOG_TRACE(log, "Nothing found for file {} in base backup", adjusted_path); auto checksums = calculateNewEntryChecksumsIfNeeded(entry, 0); info.checksum = checksums.full_checksum; } @@ -756,6 +780,7 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) /// Maybe we have a copy of this file in the backup already. if (coordination->getFileInfo(std::pair{info.size, info.checksum})) { + LOG_TRACE(log, "File {} already exist in current backup, adding reference", adjusted_path); coordination->addFileInfo(info); return; } @@ -764,6 +789,8 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) /// However file can be renamed, but has the same size and checksums, let's check for this case. if (base_backup && base_backup->fileExists(std::pair{info.size, info.checksum})) { + + LOG_TRACE(log, "File {} doesn't exist in current backup, but we have file with same size and checksum", adjusted_path); info.base_size = info.size; info.base_checksum = info.checksum; @@ -780,20 +807,25 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) bool is_data_file_required; coordination->addFileInfo(info, is_data_file_required); if (!is_data_file_required) + { + LOG_TRACE(log, "File {} doesn't exist in current backup, but we have file with same size and checksum", adjusted_path); return; /// We copy data only if it's a new combination of size & checksum. + } + auto writer_description = writer->getDataSourceDescription(); + auto reader_description = entry->getDataSourceDescription(); /// We need to copy whole file without archive, we can do it faster /// if source and destination are compatible - if (!use_archives && info.base_size == 0) + if (!use_archives && info.base_size == 0 && writer->supportNativeCopy(reader_description)) { - auto writer_description = writer->getDataSourceDescription(); - auto reader_description = entry->getDataSourceDescription(); + + LOG_TRACE(log, "Will copy file {} using native copy", adjusted_path); /// Should be much faster than writing data through server - if (writer->supportNativeCopy(reader_description)) - writer->copyFileNative(entry->getFilePath(), info.data_file_name); + writer->copyFileNative(entry->tryGetDiskIfExists(), entry->getFilePath(), info.data_file_name); } else { + LOG_TRACE(log, "Will copy file {} through memory buffers", adjusted_path); auto read_buffer = entry->getReadBuffer(); /// If we have prefix in base we will seek to the start of the suffix which differs @@ -805,6 +837,7 @@ void BackupImpl::writeFile(const String & file_name, BackupEntryPtr entry) if (use_archives) { + LOG_TRACE(log, "Adding file {} to archive", adjusted_path); String archive_suffix = current_archive_suffix; bool next_suffix = false; if (current_archive_suffix.empty() && is_internal_backup) diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index add23411d8f..a310ee90a38 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -90,7 +90,8 @@ namespace } catch (...) { - coordination->setError(current_host, Exception{getCurrentExceptionCode(), getCurrentExceptionMessage(true, true)}); + if (coordination) + coordination->setError(current_host, Exception{getCurrentExceptionCode(), getCurrentExceptionMessage(true, true)}); } } @@ -164,9 +165,9 @@ OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const Context backup_coordination = makeBackupCoordination(backup_settings.coordination_zk_path, context, backup_settings.internal); } + auto backup_info = BackupInfo::fromAST(*backup_query->backup_name); try { - auto backup_info = BackupInfo::fromAST(*backup_query->backup_name); addInfo(backup_id, backup_info.toString(), backup_settings.internal, BackupStatus::CREATING_BACKUP); /// Prepare context to use. @@ -213,6 +214,7 @@ OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const Context } catch (...) { + tryLogCurrentException(log, fmt::format("Failed to start {} {}", (backup_settings.internal ? "internal backup" : "backup"), backup_info.toString())); /// Something bad happened, the backup has not built. setStatusSafe(backup_id, BackupStatus::BACKUP_FAILED); sendCurrentExceptionToCoordination(backup_coordination, backup_settings.host_id); diff --git a/src/Backups/IBackupEntriesLazyBatch.cpp b/src/Backups/IBackupEntriesLazyBatch.cpp index ac67af3487a..e265ec31f54 100644 --- a/src/Backups/IBackupEntriesLazyBatch.cpp +++ b/src/Backups/IBackupEntriesLazyBatch.cpp @@ -25,6 +25,10 @@ public: return getInternalBackupEntry()->getFilePath(); } + std::shared_ptr tryGetDiskIfExists() const override + { + return getInternalBackupEntry()->tryGetDiskIfExists(); + } DataSourceDescription getDataSourceDescription() const override { diff --git a/src/Backups/IBackupEntry.h b/src/Backups/IBackupEntry.h index b9c74325277..32f2d03af07 100644 --- a/src/Backups/IBackupEntry.h +++ b/src/Backups/IBackupEntry.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -28,6 +29,8 @@ public: virtual String getFilePath() const = 0; + virtual std::shared_ptr tryGetDiskIfExists() const = 0; + virtual DataSourceDescription getDataSourceDescription() const = 0; }; diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index fd6379cc046..750d08ef80c 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -726,6 +726,13 @@ void DiskLocal::chmod(const String & path, mode_t mode) DB::throwFromErrnoWithPath("Cannot chmod file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); } +MetadataStoragePtr DiskLocal::getMetadataStorage() +{ + auto object_storage = std::make_shared(); + return std::make_shared( + std::static_pointer_cast(shared_from_this()), object_storage, getPath()); +} + void registerDiskLocal(DiskFactory & factory) { auto creator = [](const String & name, diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index 00e99f8fdc1..f79647b8541 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -131,6 +131,8 @@ public: bool supportsChmod() const override { return true; } void chmod(const String & path, mode_t mode) override; + MetadataStoragePtr getMetadataStorage() override; + private: std::optional tryReserve(UInt64 bytes); diff --git a/src/Disks/DiskMemory.cpp b/src/Disks/DiskMemory.cpp index 9b857d617c9..f4ca2a7459a 100644 --- a/src/Disks/DiskMemory.cpp +++ b/src/Disks/DiskMemory.cpp @@ -7,6 +7,8 @@ #include #include +#include +#include namespace DB { @@ -443,6 +445,13 @@ void DiskMemory::truncateFile(const String & path, size_t size) file_it->second.data.resize(size); } +MetadataStoragePtr DiskMemory::getMetadataStorage() +{ + auto object_storage = std::make_shared(); + return std::make_shared( + std::static_pointer_cast(shared_from_this()), object_storage, getPath()); +} + using DiskMemoryPtr = std::shared_ptr; diff --git a/src/Disks/DiskMemory.h b/src/Disks/DiskMemory.h index 9401d2827a6..78fb52a768d 100644 --- a/src/Disks/DiskMemory.h +++ b/src/Disks/DiskMemory.h @@ -97,6 +97,8 @@ public: bool supportZeroCopyReplication() const override { return false; } + MetadataStoragePtr getMetadataStorage() override; + private: void createDirectoriesImpl(const String & path); void replaceFileImpl(const String & from_path, const String & to_path); diff --git a/src/Disks/IDisk.cpp b/src/Disks/IDisk.cpp index 7a05d86b24e..3704a511478 100644 --- a/src/Disks/IDisk.cpp +++ b/src/Disks/IDisk.cpp @@ -121,18 +121,4 @@ SyncGuardPtr IDisk::getDirectorySyncGuard(const String & /* path */) const return nullptr; } -MetadataStoragePtr IDisk::getMetadataStorage() -{ - if (isRemote()) - { - return std::make_shared(std::static_pointer_cast(shared_from_this()), ""); - } - else - { - auto object_storage = std::make_shared(); - return std::make_shared( - std::static_pointer_cast(shared_from_this()), object_storage, getPath()); - } -} - } diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 213b0fda638..bfbdba0e050 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -321,7 +321,7 @@ public: /// Actually it's a part of IDiskRemote implementation but we have so /// complex hierarchy of disks (with decorators), so we cannot even /// dynamic_cast some pointer to IDisk to pointer to IDiskRemote. - virtual MetadataStoragePtr getMetadataStorage(); + virtual MetadataStoragePtr getMetadataStorage() = 0; /// Very similar case as for getMetadataDiskIfExistsOrSelf(). If disk has "metadata" /// it will return mapping for each required path: path -> metadata as string. diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index abb592dbf66..bac0d8e4c12 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -214,6 +214,22 @@ void DiskObjectStorage::moveFile(const String & from_path, const String & to_pat transaction->commit(); } + +void DiskObjectStorage::copy(const String & from_path, const std::shared_ptr & to_disk, const String & to_path) +{ + /// It's the same object storage disk + if (this == to_disk.get()) + { + auto transaction = createObjectStorageTransaction(); + transaction->copyFile(from_path, to_path); + transaction->commit(); + } + else + { + IDisk::copy(from_path, to_disk, to_path); + } +} + void DiskObjectStorage::moveFile(const String & from_path, const String & to_path) { moveFile(from_path, to_path, send_metadata); diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.h b/src/Disks/ObjectStorages/DiskObjectStorage.h index 1d21c71174c..6e2d719e3ad 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.h +++ b/src/Disks/ObjectStorages/DiskObjectStorage.h @@ -153,6 +153,8 @@ public: WriteMode mode, const WriteSettings & settings) override; + void copy(const String & from_path, const std::shared_ptr & to_disk, const String & to_path) override; + void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context_, const String &, const DisksMap &) override; void restoreMetadataIfNeeded(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context); diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index fc44bf5ee13..2fe3bb99e45 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -89,6 +89,8 @@ def test_restore_table(engine): assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" instance.query(f"BACKUP TABLE test.table TO {backup_name}") + assert instance.contains_in_log("using native copy") + instance.query("DROP TABLE test.table") assert instance.query("EXISTS test.table") == "0\n" @@ -129,6 +131,8 @@ def test_restore_table_under_another_name(): assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" instance.query(f"BACKUP TABLE test.table TO {backup_name}") + assert instance.contains_in_log("using native copy") + assert instance.query("EXISTS test.table2") == "0\n" instance.query(f"RESTORE TABLE test.table AS test.table2 FROM {backup_name}") @@ -142,6 +146,8 @@ def test_backup_table_under_another_name(): assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" instance.query(f"BACKUP TABLE test.table AS test.table2 TO {backup_name}") + assert instance.contains_in_log("using native copy") + assert instance.query("EXISTS test.table2") == "0\n" instance.query(f"RESTORE TABLE test.table2 FROM {backup_name}") @@ -170,6 +176,8 @@ def test_incremental_backup(): assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" instance.query(f"BACKUP TABLE test.table TO {backup_name}") + assert instance.contains_in_log("using native copy") + instance.query("INSERT INTO test.table VALUES (65, 'a'), (66, 'b')") assert instance.query("SELECT count(), sum(x) FROM test.table") == "102\t5081\n" @@ -244,6 +252,8 @@ def test_file_engine(): assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" instance.query(f"BACKUP TABLE test.table TO {backup_name}") + assert instance.contains_in_log("using native copy") + instance.query("DROP TABLE test.table") assert instance.query("EXISTS test.table") == "0\n" @@ -257,6 +267,9 @@ def test_database(): assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" instance.query(f"BACKUP DATABASE test TO {backup_name}") + + assert instance.contains_in_log("using native copy") + instance.query("DROP DATABASE test") instance.query(f"RESTORE DATABASE test FROM {backup_name}") @@ -269,6 +282,7 @@ def test_zip_archive(): assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" instance.query(f"BACKUP TABLE test.table TO {backup_name}") + assert os.path.isfile(get_path_to_backup(backup_name)) instance.query("DROP TABLE test.table") From 354f4e90ebae4d7698340be61a1560b4754a02ba Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 21 Aug 2022 18:21:01 +0200 Subject: [PATCH 05/44] Remove redundant lines --- src/Backups/BackupEntryFromSmallFile.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Backups/BackupEntryFromSmallFile.cpp b/src/Backups/BackupEntryFromSmallFile.cpp index b28197e7837..d24b3a6498d 100644 --- a/src/Backups/BackupEntryFromSmallFile.cpp +++ b/src/Backups/BackupEntryFromSmallFile.cpp @@ -29,14 +29,12 @@ namespace BackupEntryFromSmallFile::BackupEntryFromSmallFile(const String & file_path_, const std::optional & checksum_) : BackupEntryFromMemory(readFile(file_path_), checksum_), file_path(file_path_) { - LOG_DEBUG(&Poco::Logger::get("DEBUG"), "SMALL FILE {}", file_path); } BackupEntryFromSmallFile::BackupEntryFromSmallFile( const DiskPtr & disk_, const String & file_path_, const std::optional & checksum_) : BackupEntryFromMemory(readFile(disk_, file_path_), checksum_), disk(disk_), file_path(file_path_) { - LOG_DEBUG(&Poco::Logger::get("DEBUG"), "SMALL FILE {}", file_path); } } From bddf1cc312d1b78d92de52f7c25168ce658c248f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Aug 2022 23:12:41 +0200 Subject: [PATCH 06/44] Allow to parse Date as DateTime and DateTime64 --- src/IO/ReadHelpers.cpp | 34 ++++++++++---- src/IO/ReadHelpers.h | 47 ++++++++++++------- .../02387_parse_date_as_datetime.reference | 4 ++ .../02387_parse_date_as_datetime.sql | 13 +++++ 4 files changed, 70 insertions(+), 28 deletions(-) create mode 100644 tests/queries/0_stateless/02387_parse_date_as_datetime.reference create mode 100644 tests/queries/0_stateless/02387_parse_date_as_datetime.sql diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index fb5d0b9aea4..ac899b5a61f 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -969,10 +969,12 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D { static constexpr bool throw_exception = std::is_same_v; - /// YYYY-MM-DD hh:mm:ss - static constexpr auto date_time_broken_down_length = 19; /// YYYY-MM-DD static constexpr auto date_broken_down_length = 10; + /// hh:mm:ss + static constexpr auto time_broken_down_length = 8; + /// YYYY-MM-DD hh:mm:ss + static constexpr auto date_time_broken_down_length = date_broken_down_length + 1 + time_broken_down_length; char s[date_time_broken_down_length]; char * s_pos = s; @@ -995,16 +997,15 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D if (s_pos == s + 4 && !buf.eof() && !isNumericASCII(*buf.position())) { const auto already_read_length = s_pos - s; - const size_t remaining_date_time_size = date_time_broken_down_length - already_read_length; const size_t remaining_date_size = date_broken_down_length - already_read_length; - size_t size = buf.read(s_pos, remaining_date_time_size); - if (size != remaining_date_time_size && size != remaining_date_size) + size_t size = buf.read(s_pos, remaining_date_size); + if (size != remaining_date_size) { s_pos[size] = 0; if constexpr (throw_exception) - throw ParsingException(std::string("Cannot parse datetime ") + s, ErrorCodes::CANNOT_PARSE_DATETIME); + throw ParsingException(std::string("Cannot parse DateTime ") + s, ErrorCodes::CANNOT_PARSE_DATETIME); else return false; } @@ -1017,11 +1018,24 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D UInt8 minute = 0; UInt8 second = 0; - if (size == remaining_date_time_size) + if (!buf.eof() && (*buf.position() == ' ' || *buf.position() == 'T')) { - hour = (s[11] - '0') * 10 + (s[12] - '0'); - minute = (s[14] - '0') * 10 + (s[15] - '0'); - second = (s[17] - '0') * 10 + (s[18] - '0'); + ++buf.position(); + size = buf.read(s, time_broken_down_length); + + if (size != time_broken_down_length) + { + s_pos[size] = 0; + + if constexpr (throw_exception) + throw ParsingException(std::string("Cannot parse time component of DateTime ") + s, ErrorCodes::CANNOT_PARSE_DATETIME); + else + return false; + } + + hour = (s[0] - '0') * 10 + (s[1] - '0'); + minute = (s[3] - '0') * 10 + (s[4] - '0'); + second = (s[6] - '0') * 10 + (s[7] - '0'); } if (unlikely(year == 0)) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 502e76a6c5e..1b104c78acb 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -856,10 +856,10 @@ inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, cons const char * s = buf.position(); /// YYYY-MM-DD hh:mm:ss - static constexpr auto DateTimeStringInputSize = 19; - ///YYYY-MM-DD - static constexpr auto DateStringInputSize = 10; - bool optimistic_path_for_date_time_input = s + DateTimeStringInputSize <= buf.buffer().end(); + static constexpr auto date_time_broken_down_length = 19; + /// YYYY-MM-DD + static constexpr auto date_broken_down_length = 10; + bool optimistic_path_for_date_time_input = s + date_time_broken_down_length <= buf.buffer().end(); if (optimistic_path_for_date_time_input) { @@ -872,7 +872,8 @@ inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, cons UInt8 hour = 0; UInt8 minute = 0; UInt8 second = 0; - ///simply determine whether it is YYYY-MM-DD hh:mm:ss or YYYY-MM-DD by the content of the tenth character in an optimistic scenario + + /// Simply determine whether it is YYYY-MM-DD hh:mm:ss or YYYY-MM-DD by the content of the tenth character in an optimistic scenario bool dt_long = (s[10] == ' ' || s[10] == 'T'); if (dt_long) { @@ -881,15 +882,13 @@ inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, cons second = (s[17] - '0') * 10 + (s[18] - '0'); } - if (unlikely(year == 0)) - datetime = 0; - else - datetime = date_lut.makeDateTime(year, month, day, hour, minute, second); + datetime = date_lut.makeDateTime(year, month, day, hour, minute, second); if (dt_long) - buf.position() += DateTimeStringInputSize; + buf.position() += date_time_broken_down_length; else - buf.position() += DateStringInputSize; + buf.position() += date_broken_down_length; + return ReturnType(true); } else @@ -988,21 +987,33 @@ inline bool tryReadDateTime64Text(DateTime64 & datetime64, UInt32 scale, ReadBuf inline void readDateTimeText(LocalDateTime & datetime, ReadBuffer & buf) { - char s[19]; - size_t size = buf.read(s, 19); - if (19 != size) + char s[10]; + size_t size = buf.read(s, 10); + if (10 != size) { s[size] = 0; - throw ParsingException(std::string("Cannot parse datetime ") + s, ErrorCodes::CANNOT_PARSE_DATETIME); + throw ParsingException(std::string("Cannot parse DateTime ") + s, ErrorCodes::CANNOT_PARSE_DATETIME); } datetime.year((s[0] - '0') * 1000 + (s[1] - '0') * 100 + (s[2] - '0') * 10 + (s[3] - '0')); datetime.month((s[5] - '0') * 10 + (s[6] - '0')); datetime.day((s[8] - '0') * 10 + (s[9] - '0')); - datetime.hour((s[11] - '0') * 10 + (s[12] - '0')); - datetime.minute((s[14] - '0') * 10 + (s[15] - '0')); - datetime.second((s[17] - '0') * 10 + (s[18] - '0')); + /// Allow to read Date as DateTime + if (buf.eof() || !(*buf.position() == ' ' || *buf.position() == 'T')) + return; + + ++buf.position(); + size = buf.read(s, 8); + if (8 != size) + { + s[size] = 0; + throw ParsingException(std::string("Cannot parse time component of DateTime ") + s, ErrorCodes::CANNOT_PARSE_DATETIME); + } + + datetime.hour((s[0] - '0') * 10 + (s[1] - '0')); + datetime.minute((s[3] - '0') * 10 + (s[4] - '0')); + datetime.second((s[6] - '0') * 10 + (s[7] - '0')); } diff --git a/tests/queries/0_stateless/02387_parse_date_as_datetime.reference b/tests/queries/0_stateless/02387_parse_date_as_datetime.reference new file mode 100644 index 00000000000..226d8c26438 --- /dev/null +++ b/tests/queries/0_stateless/02387_parse_date_as_datetime.reference @@ -0,0 +1,4 @@ +123 2022-05-03 00:00:00 +456 2022-05-03 01:02:03 +123 2022-05-03 00:00:00.000 +456 2022-05-03 01:02:03.000 diff --git a/tests/queries/0_stateless/02387_parse_date_as_datetime.sql b/tests/queries/0_stateless/02387_parse_date_as_datetime.sql new file mode 100644 index 00000000000..bae3da43f53 --- /dev/null +++ b/tests/queries/0_stateless/02387_parse_date_as_datetime.sql @@ -0,0 +1,13 @@ +CREATE TEMPORARY TABLE test (`i` Int64, `d` DateTime); +INSERT INTO test FORMAT JSONEachRow {"i": 123, "d": "2022-05-03"}; +INSERT INTO test FORMAT JSONEachRow {"i": 456, "d": "2022-05-03 01:02:03"}; +SELECT * FROM test; + +DROP TABLE test; + +CREATE TEMPORARY TABLE test (`i` Int64, `d` DateTime64); +INSERT INTO test FORMAT JSONEachRow {"i": 123, "d": "2022-05-03"}; +INSERT INTO test FORMAT JSONEachRow {"i": 456, "d": "2022-05-03 01:02:03"}; +SELECT * FROM test; + +DROP TABLE test; From c4a60b8123854056abbc47bc049c336ec3cc2af9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Aug 2022 00:23:10 +0200 Subject: [PATCH 07/44] Allow conversion from String with DateTime64 to Date and Date32 --- src/Functions/FunctionsConversion.h | 30 ++++++++++++++++++++++++----- 1 file changed, 25 insertions(+), 5 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 0d6ddfdf7c2..91a3bb474dc 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -1017,9 +1017,7 @@ inline bool tryParseImpl(DataTypeDate32::FieldType & x, ReadBuff { ExtendedDayNum tmp(0); if (!tryReadDateText(tmp, rb)) - { return false; - } x = tmp; return true; } @@ -1102,9 +1100,27 @@ struct ConvertThroughParsing if (in.eof()) return true; - /// Special case, that allows to parse string with DateTime as Date. - if (std::is_same_v && (in.buffer().size()) == strlen("YYYY-MM-DD hh:mm:ss")) - return true; + /// Special case, that allows to parse string with DateTime or DateTime64 as Date or Date32. + if constexpr (std::is_same_v || std::is_same_v) + { + if (!in.eof() && (*in.position() == ' ' || *in.position() == 'T')) + { + if (in.buffer().size() == strlen("YYYY-MM-DD hh:mm:ss")) + return true; + + if (in.buffer().size() >= strlen("YYYY-MM-DD hh:mm:ss.x") + && in.buffer().begin()[19] == '.') + { + in.position() = in.buffer().begin() + 20; + + while (!in.eof() && isNumericASCII(*in.position())) + ++in.position(); + + if (in.eof()) + return true; + } + } + } return false; } @@ -1242,8 +1258,10 @@ struct ConvertThroughParsing vec_to[i] = value; } else if constexpr (IsDataTypeDecimal) + { SerializationDecimal::readText( vec_to[i], read_buffer, ToDataType::maxPrecision(), col_to->getScale()); + } else { parseImpl(vec_to[i], read_buffer, local_time_zone); @@ -1296,8 +1314,10 @@ struct ConvertThroughParsing vec_to[i] = value; } else if constexpr (IsDataTypeDecimal) + { parsed = SerializationDecimal::tryReadText( vec_to[i], read_buffer, ToDataType::maxPrecision(), col_to->getScale()); + } else parsed = tryParseImpl(vec_to[i], read_buffer, local_time_zone); } From a028d22c0bddb364e2d0cca225cff51ae4689217 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Aug 2022 00:26:52 +0200 Subject: [PATCH 08/44] Add a test --- ...th_datetime64_to_date_and_date32.reference | 12 +++++++ ...ing_with_datetime64_to_date_and_date32.sql | 33 +++++++++++++++++++ 2 files changed, 45 insertions(+) create mode 100644 tests/queries/0_stateless/02388_conversion_from_string_with_datetime64_to_date_and_date32.reference create mode 100644 tests/queries/0_stateless/02388_conversion_from_string_with_datetime64_to_date_and_date32.sql diff --git a/tests/queries/0_stateless/02388_conversion_from_string_with_datetime64_to_date_and_date32.reference b/tests/queries/0_stateless/02388_conversion_from_string_with_datetime64_to_date_and_date32.reference new file mode 100644 index 00000000000..78cdbaf0202 --- /dev/null +++ b/tests/queries/0_stateless/02388_conversion_from_string_with_datetime64_to_date_and_date32.reference @@ -0,0 +1,12 @@ +2022-08-22 +2022-08-22 +2022-08-22 +2022-08-22 +2022-08-22 +2022-08-22 +2022-08-22 +2022-08-22 +2022-08-22 +2022-08-22 +2022-08-22 +2022-08-22 diff --git a/tests/queries/0_stateless/02388_conversion_from_string_with_datetime64_to_date_and_date32.sql b/tests/queries/0_stateless/02388_conversion_from_string_with_datetime64_to_date_and_date32.sql new file mode 100644 index 00000000000..b1f905993b4 --- /dev/null +++ b/tests/queries/0_stateless/02388_conversion_from_string_with_datetime64_to_date_and_date32.sql @@ -0,0 +1,33 @@ +SELECT toDate('2022-08-22 01:02:03'); +SELECT toDate32('2022-08-22 01:02:03'); + +SELECT toDate('2022-08-22 01:02:03.1'); +SELECT toDate32('2022-08-22 01:02:03.1'); + +SELECT toDate('2022-08-22 01:02:03.123456'); +SELECT toDate32('2022-08-22 01:02:03.123456'); + +SELECT toDate('2022-08-22T01:02:03'); +SELECT toDate32('2022-08-22T01:02:03'); + +SELECT toDate('2022-08-22T01:02:03.1'); +SELECT toDate32('2022-08-22T01:02:03.1'); + +SELECT toDate('2022-08-22T01:02:03.123456'); +SELECT toDate32('2022-08-22T01:02:03.123456'); + + +SELECT toDate('2022-08-22+01:02:03'); -- { serverError 6 } +SELECT toDate32('2022-08-22+01:02:03'); -- { serverError 6 } + +SELECT toDate('2022-08-22 01:02:0'); -- { serverError 6 } +SELECT toDate32('2022-08-22 01:02:0'); -- { serverError 6 } + +SELECT toDate('2022-08-22 01:02:03.'); -- { serverError 6 } +SELECT toDate32('2022-08-22 01:02:03.'); -- { serverError 6 } + +SELECT toDate('2022-08-22 01:02:03.111a'); -- { serverError 6 } +SELECT toDate32('2022-08-22 01:02:03.2b'); -- { serverError 6 } + +SELECT toDate('2022-08-22 01:02:03.a'); -- { serverError 6 } +SELECT toDate32('2022-08-22 01:02:03.b'); -- { serverError 6 } From 2faa23112c9d7beb897143a4af40d91c99503b80 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Aug 2022 00:39:06 +0200 Subject: [PATCH 09/44] Fix test --- src/IO/ReadHelpers.h | 1 + tests/queries/0_stateless/02387_parse_date_as_datetime.sql | 4 ++-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 1b104c78acb..dff2c4d5c95 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -736,6 +736,7 @@ inline ReturnType readDateTextImpl(ExtendedDayNum & date, ReadBuffer & buf) readDateTextImpl(local_date, buf); else if (!readDateTextImpl(local_date, buf)) return false; + /// When the parameter is out of rule or out of range, Date32 uses 1925-01-01 as the default value (-DateLUT::instance().getDayNumOffsetEpoch(), -16436) and Date uses 1970-01-01. date = DateLUT::instance().makeDayNum(local_date.year(), local_date.month(), local_date.day(), -static_cast(DateLUT::instance().getDayNumOffsetEpoch())); return ReturnType(true); diff --git a/tests/queries/0_stateless/02387_parse_date_as_datetime.sql b/tests/queries/0_stateless/02387_parse_date_as_datetime.sql index bae3da43f53..9727f677be2 100644 --- a/tests/queries/0_stateless/02387_parse_date_as_datetime.sql +++ b/tests/queries/0_stateless/02387_parse_date_as_datetime.sql @@ -1,13 +1,13 @@ CREATE TEMPORARY TABLE test (`i` Int64, `d` DateTime); INSERT INTO test FORMAT JSONEachRow {"i": 123, "d": "2022-05-03"}; INSERT INTO test FORMAT JSONEachRow {"i": 456, "d": "2022-05-03 01:02:03"}; -SELECT * FROM test; +SELECT * FROM test ORDER BY i; DROP TABLE test; CREATE TEMPORARY TABLE test (`i` Int64, `d` DateTime64); INSERT INTO test FORMAT JSONEachRow {"i": 123, "d": "2022-05-03"}; INSERT INTO test FORMAT JSONEachRow {"i": 456, "d": "2022-05-03 01:02:03"}; -SELECT * FROM test; +SELECT * FROM test ORDER BY i; DROP TABLE test; From 75c0ed528d93d4174ca95c755af62f1d5d20d005 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Aug 2022 01:21:32 +0200 Subject: [PATCH 10/44] Fix test --- src/IO/ReadHelpers.h | 5 ++++- tests/queries/0_stateless/01750_parsing_exception.sh | 2 +- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index dff2c4d5c95..d49b4fc7937 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -883,7 +883,10 @@ inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, cons second = (s[17] - '0') * 10 + (s[18] - '0'); } - datetime = date_lut.makeDateTime(year, month, day, hour, minute, second); + if (unlikely(year == 0)) + datetime = 0; + else + datetime = date_lut.makeDateTime(year, month, day, hour, minute, second); if (dt_long) buf.position() += date_time_broken_down_length; diff --git a/tests/queries/0_stateless/01750_parsing_exception.sh b/tests/queries/0_stateless/01750_parsing_exception.sh index cd50d769f6d..103e551b626 100755 --- a/tests/queries/0_stateless/01750_parsing_exception.sh +++ b/tests/queries/0_stateless/01750_parsing_exception.sh @@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh # if it will not match, the exit code of grep will be non-zero and the test will fail -$CLICKHOUSE_CLIENT -q "SELECT toDateTime(format('{}-{}-01 00:00:00', '2021', '1'))" |& grep -F -q 'Cannot parse datetime 2021-1-01 00:00:00: Cannot parse DateTime from String:' +$CLICKHOUSE_CLIENT -q "SELECT toDateTime(format('{}-{}-01 00:00:00', '2021', '1'))" |& grep -F -q "Cannot parse string '2021-1-01 00:00:00' as DateTime" From b4d6afd5b33885380c407dca6b574e3350113f21 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Aug 2022 13:51:48 +0200 Subject: [PATCH 11/44] Black --- src/Storages/System/StorageSystemDisks.cpp | 7 +++++- tests/integration/test_disk_types/test.py | 28 +++++++++++++++++----- 2 files changed, 28 insertions(+), 7 deletions(-) diff --git a/src/Storages/System/StorageSystemDisks.cpp b/src/Storages/System/StorageSystemDisks.cpp index 9546c854ebf..ef2c695d6b7 100644 --- a/src/Storages/System/StorageSystemDisks.cpp +++ b/src/Storages/System/StorageSystemDisks.cpp @@ -23,6 +23,7 @@ StorageSystemDisks::StorageSystemDisks(const StorageID & table_id_) {"total_space", std::make_shared()}, {"keep_free_space", std::make_shared()}, {"type", std::make_shared()}, + {"is_encrypted", std::make_shared()}, {"cache_path", std::make_shared()}, })); setInMemoryMetadata(storage_metadata); @@ -45,6 +46,7 @@ Pipe StorageSystemDisks::read( MutableColumnPtr col_total = ColumnUInt64::create(); MutableColumnPtr col_keep = ColumnUInt64::create(); MutableColumnPtr col_type = ColumnString::create(); + MutableColumnPtr col_is_encrypted = ColumnUInt8::create(); MutableColumnPtr col_cache_path = ColumnString::create(); for (const auto & [disk_name, disk_ptr] : context->getDisksMap()) @@ -54,7 +56,9 @@ Pipe StorageSystemDisks::read( col_free->insert(disk_ptr->getAvailableSpace()); col_total->insert(disk_ptr->getTotalSpace()); col_keep->insert(disk_ptr->getKeepingFreeSpace()); - col_type->insert(toString(disk_ptr->getDataSourceDescription().type)); + auto data_source_description = disk_ptr->getDataSourceDescription(); + col_type->insert(toString(data_source_description.type)); + col_is_encrypted->insert(data_source_description.is_encrypted); String cache_path; if (disk_ptr->supportsCache()) @@ -70,6 +74,7 @@ Pipe StorageSystemDisks::read( res_columns.emplace_back(std::move(col_total)); res_columns.emplace_back(std::move(col_keep)); res_columns.emplace_back(std::move(col_type)); + res_columns.emplace_back(std::move(col_is_encrypted)); res_columns.emplace_back(std::move(col_cache_path)); UInt64 num_rows = res_columns.at(0)->size(); diff --git a/tests/integration/test_disk_types/test.py b/tests/integration/test_disk_types/test.py index a26f80165e8..5f7b430d7ef 100644 --- a/tests/integration/test_disk_types/test.py +++ b/tests/integration/test_disk_types/test.py @@ -6,7 +6,7 @@ disk_types = { "disk_s3": "s3", "disk_memory": "memory", "disk_hdfs": "hdfs", - "disk_encrypted": "encrypted", + "disk_encrypted": "s3", } @@ -34,14 +34,30 @@ def test_different_types(cluster): if disk == "": # skip empty line (after split at last position) continue fields = disk.split("\t") - assert len(fields) >= 6 + assert len(fields) >= 7 assert disk_types.get(fields[0], "UNKNOWN") == fields[5] + if "encrypted" in fields[0]: + assert fields[6] == "1" + else: + assert fields[6] == "0" def test_select_by_type(cluster): node = cluster.instances["node"] for name, disk_type in list(disk_types.items()): - assert ( - node.query("SELECT name FROM system.disks WHERE type='" + disk_type + "'") - == name + "\n" - ) + if disk_type != "s3": + assert ( + node.query( + "SELECT name FROM system.disks WHERE type='" + disk_type + "'" + ) + == name + "\n" + ) + else: + assert ( + node.query( + "SELECT name FROM system.disks WHERE type='" + + disk_type + + "' ORDER BY name" + ) + == "disk_encrypted\ndisk_s3\n" + ) From e865959ccc37ba3b047dea6ae807b1a4418acbc1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Aug 2022 16:53:57 +0200 Subject: [PATCH 12/44] Fix test --- .../queries/0_stateless/02117_show_create_table_system.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 9edc2aa0cb4..3ca3f856b95 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -186,6 +186,7 @@ CREATE TABLE system.disks `total_space` UInt64, `keep_free_space` UInt64, `type` String, + `is_encrypted` UInt8, `cache_path` String ) ENGINE = SystemDisks From 9e5479235933473f876408cff7258c72f9fcec7a Mon Sep 17 00:00:00 2001 From: Maksim Buren <43178149+maks-buren630501@users.noreply.github.com> Date: Tue, 23 Aug 2022 09:34:03 +0300 Subject: [PATCH 13/44] Troubleshooting racing consumer_task use replication_handler_initialized in thread start in method assertInitialized(), by this sometimes assertInitialized throw error because replication_handler_initialized was still "false". --- src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index e0e2acc3436..0f17c564da9 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -321,13 +321,13 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) nested_storages, (is_materialized_postgresql_database ? postgres_database : postgres_database + '.' + tables_list)); + replication_handler_initialized = true; + consumer_task->activateAndSchedule(); cleanup_task->activateAndSchedule(); /// Do not rely anymore on saved storage pointers. materialized_storages.clear(); - - replication_handler_initialized = true; } From 4711f577633d9bc3560e0469b5256d77237be6f6 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 23 Aug 2022 10:43:12 +0200 Subject: [PATCH 14/44] Update PostgreSQLReplicationHandler.cpp --- src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 0f17c564da9..b81e029acff 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -322,7 +322,7 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) (is_materialized_postgresql_database ? postgres_database : postgres_database + '.' + tables_list)); replication_handler_initialized = true; - + consumer_task->activateAndSchedule(); cleanup_task->activateAndSchedule(); From b2b97e53ea3e319cea39262aa23eb4378fb8528f Mon Sep 17 00:00:00 2001 From: maks-buren630501 Date: Tue, 23 Aug 2022 12:53:30 +0300 Subject: [PATCH 15/44] Fix bug whith wrong postgresql answer --- .../MaterializedPostgreSQLConsumer.cpp | 20 ++++++++++++++++++- .../MaterializedPostgreSQLConsumer.h | 4 ++++ 2 files changed, 23 insertions(+), 1 deletion(-) diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index a57328fb402..8600ba21652 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -45,6 +45,7 @@ MaterializedPostgreSQLConsumer::MaterializedPostgreSQLConsumer( , schema_as_a_part_of_table_name(schema_as_a_part_of_table_name_) , allow_automatic_update(allow_automatic_update_) { + commited = false; final_lsn = start_lsn; auto tx = std::make_shared(connection->getRef()); current_lsn = advanceLSN(tx); @@ -53,6 +54,7 @@ MaterializedPostgreSQLConsumer::MaterializedPostgreSQLConsumer( for (const auto & [table_name, storage_info] : storages_info_) storages.emplace(table_name, storage_info); + LOG_TRACE(log, "Consumer created"); } @@ -443,6 +445,7 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl pos += unused_flags_len + commit_lsn_len + transaction_end_lsn_len + transaction_commit_timestamp_len; final_lsn = current_lsn; + commited = true; break; } case 'R': // Relation @@ -593,6 +596,12 @@ void MaterializedPostgreSQLConsumer::syncTables() LOG_DEBUG(log, "Table sync end for {} tables, last lsn: {} = {}, (attempted lsn {})", tables_to_sync.size(), current_lsn, getLSNValue(current_lsn), getLSNValue(final_lsn)); + updateLsn(); +} + + +void MaterializedPostgreSQLConsumer::updateLsn() +{ try { auto tx = std::make_shared(connection->getRef()); @@ -614,6 +623,7 @@ String MaterializedPostgreSQLConsumer::advanceLSN(std::shared_ptr(); LOG_TRACE(log, "Advanced LSN up to: {}", getLSNValue(final_lsn)); + commited = false; return final_lsn; } @@ -771,7 +781,7 @@ bool MaterializedPostgreSQLConsumer::readFromReplicationSlot() try { - // LOG_DEBUG(log, "Current message: {}", (*row)[1]); + LOG_DEBUG(log, "Current message: {}", (*row)[1]); processReplicationMessage((*row)[1].c_str(), (*row)[1].size()); } catch (const Exception & e) @@ -790,6 +800,7 @@ bool MaterializedPostgreSQLConsumer::readFromReplicationSlot() } catch (const pqxx::broken_connection &) { + LOG_DEBUG(log, "Connection was brocken"); connection->tryUpdateConnection(); return false; } @@ -825,6 +836,13 @@ bool MaterializedPostgreSQLConsumer::readFromReplicationSlot() if (!tables_to_sync.empty()) syncTables(); + else + { + if(commited) + { + updateLsn(); + } + } return true; } diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h index 869d764f358..412394ff0b2 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h @@ -94,6 +94,8 @@ private: void syncTables(); + void updateLsn(); + String advanceLSN(std::shared_ptr ntx); void processReplicationMessage(const char * replication_message, size_t size); @@ -136,6 +138,8 @@ private: ContextPtr context; const std::string replication_slot_name, publication_name; + bool commited; + std::shared_ptr connection; std::string current_lsn, final_lsn; From 0827e87d45f681f917bba25eec858540731b5515 Mon Sep 17 00:00:00 2001 From: Maksim Buren <43178149+maks-buren630501@users.noreply.github.com> Date: Tue, 23 Aug 2022 14:20:32 +0300 Subject: [PATCH 16/44] delete miss log --- src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index 8600ba21652..70046e49e8e 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -54,7 +54,6 @@ MaterializedPostgreSQLConsumer::MaterializedPostgreSQLConsumer( for (const auto & [table_name, storage_info] : storages_info_) storages.emplace(table_name, storage_info); - LOG_TRACE(log, "Consumer created"); } From 07286dd04236e8a7b7e1b6f41930cc30f4483b0e Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 23 Aug 2022 15:12:09 +0200 Subject: [PATCH 17/44] Add paranoid option to check blobs to S3 after upload --- src/Core/Settings.h | 1 + .../ObjectStorages/S3/S3ObjectStorage.cpp | 17 +++++++ src/Disks/ObjectStorages/S3/diskSettings.cpp | 1 + src/IO/WriteBufferFromS3.cpp | 15 ++++++ src/Storages/StorageS3Settings.cpp | 10 ++++ src/Storages/StorageS3Settings.h | 4 +- tests/config/install.sh | 1 + tests/config/users.d/enable_blobs_check.xml | 7 +++ .../__init__.py | 1 + .../configs/setting.xml | 23 +++++++++ .../configs/storage_conf.xml | 31 +++++++++++ .../test_checking_s3_blobs_paranoid/test.py | 51 +++++++++++++++++++ 12 files changed, 161 insertions(+), 1 deletion(-) create mode 100644 tests/config/users.d/enable_blobs_check.xml create mode 100644 tests/integration/test_checking_s3_blobs_paranoid/__init__.py create mode 100644 tests/integration/test_checking_s3_blobs_paranoid/configs/setting.xml create mode 100644 tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml create mode 100644 tests/integration/test_checking_s3_blobs_paranoid/test.py diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2b808a1ada7..94e358009f4 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -90,6 +90,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, s3_max_connections, 1024, "The maximum number of connections per server.", 0) \ M(Bool, s3_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables.", 0) \ M(Bool, s3_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in s3 engine tables", 0) \ + M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \ M(Bool, enable_s3_requests_logging, false, "Enable very explicit logging of S3 requests. Makes sense for debug only.", 0) \ M(UInt64, hdfs_replication, 0, "The actual number of replications can be specified when the hdfs file is created.", 0) \ M(Bool, hdfs_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables", 0) \ diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 31eca9d4ffa..ddaeef62562 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -369,6 +369,15 @@ void S3ObjectStorage::copyObjectImpl( } throwIfError(outcome); + + auto settings_ptr = s3_settings.get(); + if (settings_ptr->s3_settings.check_objects_after_upload) + { + auto object_head = requestObjectHeadData(dst_bucket, dst_key); + if (!object_head.IsSuccess()) + throw Exception(ErrorCodes::S3_ERROR, "Object {} from bucket {} disappeared immediately after upload, it's a bug in S3 or S3 API.", dst_key, dst_bucket); + } + } void S3ObjectStorage::copyObjectMultipartImpl( @@ -450,6 +459,14 @@ void S3ObjectStorage::copyObjectMultipartImpl( throwIfError(outcome); } + + if (settings_ptr->s3_settings.check_objects_after_upload) + { + auto object_head = requestObjectHeadData(dst_bucket, dst_key); + if (!object_head.IsSuccess()) + throw Exception(ErrorCodes::S3_ERROR, "Object {} from bucket {} disappeared immediately after upload, it's a bug in S3 or S3 API.", dst_key, dst_bucket); + } + } void S3ObjectStorage::copyObject( // NOLINT diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 11f7b2e8ad7..4cdb867e48c 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -40,6 +40,7 @@ std::unique_ptr getSettings(const Poco::Util::AbstractC rw_settings.upload_part_size_multiply_factor = config.getUInt64(config_prefix + ".s3_upload_part_size_multiply_factor", context->getSettingsRef().s3_upload_part_size_multiply_factor); rw_settings.upload_part_size_multiply_parts_count_threshold = config.getUInt64(config_prefix + ".s3_upload_part_size_multiply_parts_count_threshold", context->getSettingsRef().s3_upload_part_size_multiply_parts_count_threshold); rw_settings.max_single_part_upload_size = config.getUInt64(config_prefix + ".s3_max_single_part_upload_size", context->getSettingsRef().s3_max_single_part_upload_size); + rw_settings.check_objects_after_upload = config.getUInt64(config_prefix + ".s3_check_objects_after_upload", context->getSettingsRef().s3_check_objects_after_upload); return std::make_unique( rw_settings, diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 7da38b3df45..40e592ec197 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include @@ -164,6 +165,20 @@ void WriteBufferFromS3::finalizeImpl() if (!multipart_upload_id.empty()) completeMultipartUpload(); + + if (s3_settings.check_objects_after_upload) + { + LOG_TRACE(log, "Checking object {} exists after upload", key); + + Aws::S3::Model::HeadObjectRequest request; + request.SetBucket(bucket); + request.SetKey(key); + + auto response = client_ptr->HeadObject(request); + + if (!response.IsSuccess()) + throw Exception(ErrorCodes::S3_ERROR, "Object {} from bucket {} disappeared immediately after upload, it's a bug in S3 or S3 API.", key, bucket); + } } void WriteBufferFromS3::createMultipartUpload() diff --git a/src/Storages/StorageS3Settings.cpp b/src/Storages/StorageS3Settings.cpp index 6c979d69795..353e324c853 100644 --- a/src/Storages/StorageS3Settings.cpp +++ b/src/Storages/StorageS3Settings.cpp @@ -34,6 +34,13 @@ void StorageS3Settings::loadFromConfig(const String & config_elem, const Poco::U return with_default ? config.getUInt64(config_elem + "." + key + "." + elem, default_value) : config.getUInt64(config_elem + "." + key + "." + elem); }; + + auto get_bool_for_key = [&](const String & key, const String & elem, bool with_default = true, bool default_value = false) + { + return with_default ? config.getBool(config_elem + "." + key + "." + elem, default_value) : config.getBool(config_elem + "." + key + "." + elem); + }; + + for (const String & key : config_keys) { if (config.has(config_elem + "." + key + ".endpoint")) @@ -82,6 +89,7 @@ void StorageS3Settings::loadFromConfig(const String & config_elem, const Poco::U rw_settings.upload_part_size_multiply_parts_count_threshold = get_uint_for_key(key, "upload_part_size_multiply_parts_count_threshold", true, settings.s3_upload_part_size_multiply_parts_count_threshold); rw_settings.max_single_part_upload_size = get_uint_for_key(key, "max_single_part_upload_size", true, settings.s3_max_single_part_upload_size); rw_settings.max_connections = get_uint_for_key(key, "max_connections", true, settings.s3_max_connections); + rw_settings.check_objects_after_upload = get_bool_for_key(key, "check_objects_after_upload", true, false); s3_settings.emplace(endpoint, S3Settings{std::move(auth_settings), std::move(rw_settings)}); } @@ -112,6 +120,7 @@ S3Settings::ReadWriteSettings::ReadWriteSettings(const Settings & settings) upload_part_size_multiply_parts_count_threshold = settings.s3_upload_part_size_multiply_parts_count_threshold; max_single_part_upload_size = settings.s3_max_single_part_upload_size; max_connections = settings.s3_max_connections; + check_objects_after_upload = settings.s3_check_objects_after_upload; } void S3Settings::ReadWriteSettings::updateFromSettingsIfEmpty(const Settings & settings) @@ -128,6 +137,7 @@ void S3Settings::ReadWriteSettings::updateFromSettingsIfEmpty(const Settings & s max_single_part_upload_size = settings.s3_max_single_part_upload_size; if (!max_connections) max_connections = settings.s3_max_connections; + check_objects_after_upload = settings.s3_check_objects_after_upload; } } diff --git a/src/Storages/StorageS3Settings.h b/src/Storages/StorageS3Settings.h index 21195fad215..9ef51c77191 100644 --- a/src/Storages/StorageS3Settings.h +++ b/src/Storages/StorageS3Settings.h @@ -60,6 +60,7 @@ struct S3Settings size_t upload_part_size_multiply_parts_count_threshold = 0; size_t max_single_part_upload_size = 0; size_t max_connections = 0; + bool check_objects_after_upload = false; ReadWriteSettings() = default; explicit ReadWriteSettings(const Settings & settings); @@ -71,7 +72,8 @@ struct S3Settings && upload_part_size_multiply_factor == other.upload_part_size_multiply_factor && upload_part_size_multiply_parts_count_threshold == other.upload_part_size_multiply_parts_count_threshold && max_single_part_upload_size == other.max_single_part_upload_size - && max_connections == other.max_connections; + && max_connections == other.max_connections + && check_objects_after_upload == other.check_objects_after_upload; } void updateFromSettingsIfEmpty(const Settings & settings); diff --git a/tests/config/install.sh b/tests/config/install.sh index eca1bfc3501..072787efbb3 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -59,6 +59,7 @@ ln -sf $SRC_PATH/users.d/session_log_test.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/memory_profiler.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/no_fsync_metadata.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/filelog.xml $DEST_SERVER_PATH/users.d/ +ln -sf $SRC_PATH/users.d/enable_blobs_check.xml $DEST_SERVER_PATH/users.d/ # FIXME DataPartsExchange may hang for http_send_timeout seconds # when nobody is going to read from the other side of socket (due to "Fetching of part was cancelled"), diff --git a/tests/config/users.d/enable_blobs_check.xml b/tests/config/users.d/enable_blobs_check.xml new file mode 100644 index 00000000000..0877bfcfa9b --- /dev/null +++ b/tests/config/users.d/enable_blobs_check.xml @@ -0,0 +1,7 @@ + + + + 1 + + + diff --git a/tests/integration/test_checking_s3_blobs_paranoid/__init__.py b/tests/integration/test_checking_s3_blobs_paranoid/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_checking_s3_blobs_paranoid/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/setting.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/setting.xml new file mode 100644 index 00000000000..f8749488f57 --- /dev/null +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/setting.xml @@ -0,0 +1,23 @@ + + + + + + 1 + 1 + + + + + + + + ::/0 + + default + default + + + + + diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml new file mode 100644 index 00000000000..733205ce3e1 --- /dev/null +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/storage_conf.xml @@ -0,0 +1,31 @@ + + + + + test + + + + + + s3 + http://minio1:9001/root/data/ + minio + minio123 + + + + + + +
+ s3 +
+
+
+
+
+ + s3 + +
diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py new file mode 100644 index 00000000000..522ec56963a --- /dev/null +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -0,0 +1,51 @@ +#!/usr/bin/env python3 + +import logging +import os +import time + + +from helpers.cluster import ClickHouseCluster +import pytest + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "node", + main_configs=[ + "configs/storage_conf.xml", + ], + user_configs=[ + "configs/setting.xml", + ], + with_minio=True, + ) + logging.info("Starting cluster...") + cluster.start() + logging.info("Cluster started") + + yield cluster + finally: + cluster.shutdown() + + +def test_paranoid_check_in_logs(cluster): + node = cluster.instances["node"] + + node.query( + """ + CREATE TABLE s3_failover_test ( + id Int64, + data String + ) ENGINE=MergeTree() + ORDER BY id + """ + ) + + node.query("INSERT INTO s3_failover_test VALUES (1, 'Hello')") + + assert node.contains_in_log("exists after upload") + + assert node.query("SELECT * FROM s3_failover_test ORDER BY id") == "1\tHello\n" From f0b18a108282f6ddc3c1357cb3e0dc9f84a8fad3 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 23 Aug 2022 13:21:29 +0000 Subject: [PATCH 18/44] Automatic style fix --- tests/integration/test_checking_s3_blobs_paranoid/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index 522ec56963a..adb56b1899c 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -8,6 +8,7 @@ import time from helpers.cluster import ClickHouseCluster import pytest + @pytest.fixture(scope="module") def cluster(): try: From de489ba1f7c065c6cbe5f718e81b9d17a8d9052c Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 23 Aug 2022 14:42:23 +0000 Subject: [PATCH 19/44] Fix possible error 'Decimal math overflow' while parsing DateTime64 --- src/Core/DecimalFunctions.h | 78 +++++++++++++++++-- src/IO/ReadHelpers.h | 8 +- .../02406_try_read_datetime64_bug.reference | 2 + .../02406_try_read_datetime64_bug.sql | 2 + 4 files changed, 81 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/02406_try_read_datetime64_bug.reference create mode 100644 tests/queries/0_stateless/02406_try_read_datetime64_bug.sql diff --git a/src/Core/DecimalFunctions.h b/src/Core/DecimalFunctions.h index 331df9aa637..0f2158fb83b 100644 --- a/src/Core/DecimalFunctions.h +++ b/src/Core/DecimalFunctions.h @@ -90,24 +90,56 @@ struct DataTypeDecimalTrait * Sign of `fractional` is expected to be positive, otherwise result is undefined. * If `scale` is to big (scale > max_precision), result is undefined. */ -template -inline DecimalType decimalFromComponentsWithMultiplier( - const typename DecimalType::NativeType & whole, - const typename DecimalType::NativeType & fractional, - typename DecimalType::NativeType scale_multiplier) + +template +inline bool decimalFromComponentsWithMultiplierImpl( + const typename DecimalType::NativeType & whole, + const typename DecimalType::NativeType & fractional, + typename DecimalType::NativeType scale_multiplier, + DecimalType & result) { using T = typename DecimalType::NativeType; const auto fractional_sign = whole < 0 ? -1 : 1; T whole_scaled = 0; if (common::mulOverflow(whole, scale_multiplier, whole_scaled)) - throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW); + { + if constexpr (throw_on_error) + throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW); + return false; + } T value; if (common::addOverflow(whole_scaled, fractional_sign * (fractional % scale_multiplier), value)) - throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW); + { + if constexpr (throw_on_error) + throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW); + return false; + } - return DecimalType(value); + result = DecimalType(value); + return true; +} + +template +inline DecimalType decimalFromComponentsWithMultiplier( + const typename DecimalType::NativeType & whole, + const typename DecimalType::NativeType & fractional, + typename DecimalType::NativeType scale_multiplier) +{ + DecimalType result; + decimalFromComponentsWithMultiplierImpl(whole, fractional, scale_multiplier, result); + return result; +} + +template +inline bool tryGetDecimalFromComponentsWithMultiplier( + const typename DecimalType::NativeType & whole, + const typename DecimalType::NativeType & fractional, + typename DecimalType::NativeType scale_multiplier, + DecimalType & result) +{ + return decimalFromComponentsWithMultiplierImpl(whole, fractional, scale_multiplier, result); } template @@ -118,6 +150,15 @@ inline DecimalType decimalFromComponentsWithMultiplier( return decimalFromComponentsWithMultiplier(components.whole, components.fractional, scale_multiplier); } +template +inline bool tryGetDecimalFromComponentsWithMultiplier( + const DecimalComponents & components, + typename DecimalType::NativeType scale_multiplier, + DecimalType & result) +{ + return tryGetDecimalFromComponentsWithMultiplier(components.whole, components.fractional, scale_multiplier, result); +} + /** Make a decimal value from whole and fractional components with given scale. * @@ -134,6 +175,18 @@ inline DecimalType decimalFromComponents( return decimalFromComponentsWithMultiplier(whole, fractional, scaleMultiplier(scale)); } +template +inline bool tryGetDecimalFromComponents( + const typename DecimalType::NativeType & whole, + const typename DecimalType::NativeType & fractional, + UInt32 scale, + DecimalType & result) +{ + using T = typename DecimalType::NativeType; + + return tryGetDecimalFromComponentsWithMultiplier(whole, fractional, scaleMultiplier(scale), result); +} + /** Make a decimal value from whole and fractional components with given scale. * @see `decimalFromComponentsWithMultiplier` for details. */ @@ -145,6 +198,15 @@ inline DecimalType decimalFromComponents( return decimalFromComponents(components.whole, components.fractional, scale); } +template +inline bool tryGetDecimalFromComponents( + const DecimalComponents & components, + UInt32 scale, + DecimalType & result) +{ + return tryGetDecimalFromComponents(components.whole, components.fractional, scale, result); +} + /** Split decimal into whole and fractional parts with given scale_multiplier. * This is an optimization to reduce number of calls to scaleMultiplier on known scale. */ diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 502e76a6c5e..6b06b0005b8 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -961,7 +961,13 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re components.whole = components.whole / common::exp10_i32(scale); } - datetime64 = negative_multiplier * DecimalUtils::decimalFromComponents(components, scale); + if constexpr (std::is_same_v) + datetime64 = DecimalUtils::decimalFromComponents(components, scale); + else + DecimalUtils::tryGetDecimalFromComponents(components, scale, datetime64); + + negative_multiplier *= negative_multiplier; + return ReturnType(true); } diff --git a/tests/queries/0_stateless/02406_try_read_datetime64_bug.reference b/tests/queries/0_stateless/02406_try_read_datetime64_bug.reference new file mode 100644 index 00000000000..c31db065c01 --- /dev/null +++ b/tests/queries/0_stateless/02406_try_read_datetime64_bug.reference @@ -0,0 +1,2 @@ +1970-01-01 00:00:00.000000000 +c1 Nullable(String) diff --git a/tests/queries/0_stateless/02406_try_read_datetime64_bug.sql b/tests/queries/0_stateless/02406_try_read_datetime64_bug.sql new file mode 100644 index 00000000000..26cff0dd049 --- /dev/null +++ b/tests/queries/0_stateless/02406_try_read_datetime64_bug.sql @@ -0,0 +1,2 @@ +select toDateTime64OrDefault('Aaaa e a.a.aaaaaaaaa', 9); +desc format(CSV, '"Aaaa e a.a.aaaaaaaaa"'); From f62c2c3221d6d6ff54eac3d3d77a444f7ebc458b Mon Sep 17 00:00:00 2001 From: kgurjev Date: Mon, 22 Aug 2022 18:33:43 +0300 Subject: [PATCH 20/44] Fix bug in settings input_format_tsv_skip_first_lines of format TSV --- src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp | 2 +- tests/queries/0_stateless/02314_csv_tsv_skip_first_lines.sql | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index 5a20d527710..d983e5371b8 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -240,7 +240,7 @@ void TabSeparatedFormatReader::checkNullValueForNonNullable(DataTypePtr type) void TabSeparatedFormatReader::skipPrefixBeforeHeader() { - for (size_t i = 0; i != format_settings.csv.skip_first_lines; ++i) + for (size_t i = 0; i != format_settings.tsv.skip_first_lines; ++i) readRow(); } diff --git a/tests/queries/0_stateless/02314_csv_tsv_skip_first_lines.sql b/tests/queries/0_stateless/02314_csv_tsv_skip_first_lines.sql index ff913a2a3ca..4a0cef35310 100644 --- a/tests/queries/0_stateless/02314_csv_tsv_skip_first_lines.sql +++ b/tests/queries/0_stateless/02314_csv_tsv_skip_first_lines.sql @@ -7,6 +7,6 @@ select * from file(data_02314.csv) settings input_format_csv_skip_first_lines=5; insert into function file(data_02314.tsv) select number, number + 1 from numbers(5) settings engine_file_truncate_on_insert=1; insert into function file(data_02314.tsv) select number, number + 1, number + 2 from numbers(5); -desc file(data_02314.tsv) settings input_format_csv_skip_first_lines=5; -select * from file(data_02314.tsv) settings input_format_csv_skip_first_lines=5; +desc file(data_02314.tsv) settings input_format_tsv_skip_first_lines=5; +select * from file(data_02314.tsv) settings input_format_tsv_skip_first_lines=5; From 335293549bc270b84278ea08cc33d22f254671cb Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Wed, 24 Aug 2022 17:38:16 +0800 Subject: [PATCH 21/44] Fix segment fault when sending data to remote HTTP server via URL table engine Signed-off-by: Frank Chen --- src/IO/WriteBufferFromHTTP.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/IO/WriteBufferFromHTTP.cpp b/src/IO/WriteBufferFromHTTP.cpp index 916fa40dc35..f7456ad6b6c 100644 --- a/src/IO/WriteBufferFromHTTP.cpp +++ b/src/IO/WriteBufferFromHTTP.cpp @@ -35,9 +35,8 @@ WriteBufferFromHTTP::WriteBufferFromHTTP( void WriteBufferFromHTTP::finalizeImpl() { - // for compressed body, the data is stored in buffered first - // here, make sure the content in the buffer has been flushed - this->nextImpl(); + // Make sure the content in the buffer has been flushed + this->next(); receiveResponse(*session, request, response, false); /// TODO: Response body is ignored. From 4486f26d511ab9f052d965a56890158d46452654 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 24 Aug 2022 12:23:08 +0200 Subject: [PATCH 22/44] Update MaterializedPostgreSQLConsumer.cpp --- .../PostgreSQL/MaterializedPostgreSQLConsumer.cpp | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index 70046e49e8e..82843228629 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -833,15 +833,14 @@ bool MaterializedPostgreSQLConsumer::readFromReplicationSlot() } if (!tables_to_sync.empty()) - syncTables(); - - else { - if(commited) - { - updateLsn(); - } + syncTables(); } + else if (commited) + { + updateLsn(); + } + return true; } From de9d2542901667ad9c6581a5c6429e42ccf031de Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 24 Aug 2022 12:24:44 +0200 Subject: [PATCH 23/44] Update MaterializedPostgreSQLConsumer.cpp --- .../PostgreSQL/MaterializedPostgreSQLConsumer.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index 82843228629..e25e8d74a29 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -45,7 +45,7 @@ MaterializedPostgreSQLConsumer::MaterializedPostgreSQLConsumer( , schema_as_a_part_of_table_name(schema_as_a_part_of_table_name_) , allow_automatic_update(allow_automatic_update_) { - commited = false; + committed = false; final_lsn = start_lsn; auto tx = std::make_shared(connection->getRef()); current_lsn = advanceLSN(tx); @@ -444,7 +444,7 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl pos += unused_flags_len + commit_lsn_len + transaction_end_lsn_len + transaction_commit_timestamp_len; final_lsn = current_lsn; - commited = true; + committed = true; break; } case 'R': // Relation @@ -622,7 +622,7 @@ String MaterializedPostgreSQLConsumer::advanceLSN(std::shared_ptr(); LOG_TRACE(log, "Advanced LSN up to: {}", getLSNValue(final_lsn)); - commited = false; + committed = false; return final_lsn; } @@ -799,7 +799,7 @@ bool MaterializedPostgreSQLConsumer::readFromReplicationSlot() } catch (const pqxx::broken_connection &) { - LOG_DEBUG(log, "Connection was brocken"); + LOG_DEBUG(log, "Connection was broken"); connection->tryUpdateConnection(); return false; } @@ -836,7 +836,7 @@ bool MaterializedPostgreSQLConsumer::readFromReplicationSlot() { syncTables(); } - else if (commited) + else if (committed) { updateLsn(); } From 38c4094f8968d698a865da056187fcf76449bc2a Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 24 Aug 2022 12:25:03 +0200 Subject: [PATCH 24/44] Update MaterializedPostgreSQLConsumer.h --- src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h index 412394ff0b2..91bf5eeccde 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h @@ -138,7 +138,7 @@ private: ContextPtr context; const std::string replication_slot_name, publication_name; - bool commited; + bool committed = false; std::shared_ptr connection; From 860d0baa48aaeb833c9b8cc84285d549caa33b4c Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 24 Aug 2022 12:28:44 +0200 Subject: [PATCH 25/44] Better exception messages in FunctionArrayMapped --- src/Functions/array/FunctionArrayMapped.h | 51 +++++++++++++++-------- 1 file changed, 33 insertions(+), 18 deletions(-) diff --git a/src/Functions/array/FunctionArrayMapped.h b/src/Functions/array/FunctionArrayMapped.h index c4ac89df78e..6d500cc15c4 100644 --- a/src/Functions/array/FunctionArrayMapped.h +++ b/src/Functions/array/FunctionArrayMapped.h @@ -85,6 +85,9 @@ class FunctionArrayMapped : public IFunction { public: static constexpr auto name = Name::name; + static constexpr bool is_argument_type_map = std::is_same_v; + static constexpr bool is_argument_type_array = std::is_same_v; + static constexpr auto argument_type_name = is_argument_type_map ? "Map" : "Array"; static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override @@ -112,20 +115,25 @@ public: throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} needs one argument with data", getName()); - size_t nested_types_count = std::is_same_v ? (arguments.size() - 1) * 2 : (arguments.size() - 1); + size_t nested_types_count = is_argument_type_map ? (arguments.size() - 1) * 2 : (arguments.size() - 1); DataTypes nested_types(nested_types_count); for (size_t i = 0; i < arguments.size() - 1; ++i) { const auto * array_type = checkAndGetDataType(&*arguments[i + 1]); if (!array_type) - throw Exception("Argument " + toString(i + 2) + " of function " + getName() + " must be array. Found " - + arguments[i + 1]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if constexpr (std::is_same_v) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Argument {} of function {} must be {}. Found {} instead", + toString(i + 2), + getName(), + argument_type_name, + arguments[i + 1]->getName()); + if constexpr (is_argument_type_map) { nested_types[2 * i] = recursiveRemoveLowCardinality(array_type->getKeyType()); nested_types[2 * i + 1] = recursiveRemoveLowCardinality(array_type->getValueType()); } - else if constexpr (std::is_same_v) + else if constexpr (is_argument_type_array) { nested_types[i] = recursiveRemoveLowCardinality(array_type->getNestedType()); } @@ -149,7 +157,7 @@ public: "Function {} needs at least {} argument, passed {}", getName(), min_args, arguments.size()); - if ((arguments.size() == 1) && std::is_same_v) + if ((arguments.size() == 1) && is_argument_type_array) { const auto * data_type = checkAndGetDataType(arguments[0].type.get()); @@ -163,7 +171,7 @@ public: throw Exception("The only argument for function " + getName() + " must be array of UInt8. Found " + arguments[0].type->getName() + " instead", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if constexpr (std::is_same_v) + if constexpr (is_argument_type_array) return Impl::getReturnType(nested_type, nested_type); else throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unreachable code reached"); @@ -193,10 +201,7 @@ public: throw Exception("Expression for function " + getName() + " must return UInt8 or Nullable(UInt8), found " + return_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - static_assert( - std::is_same_v || - std::is_same_v, - "unsupported type"); + static_assert(is_argument_type_map || is_argument_type_array, "unsupported type"); if (arguments.size() < 2) { @@ -208,10 +213,10 @@ public: if (!first_array_type) throw DB::Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Unsupported type {}", arguments[1].type->getName()); - if constexpr (std::is_same_v) + if constexpr (is_argument_type_array) return Impl::getReturnType(return_type, first_array_type->getNestedType()); - if constexpr (std::is_same_v) + if constexpr (is_argument_type_map) return Impl::getReturnType(return_type, first_array_type->getKeyValueTypes()); throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unreachable code reached"); @@ -229,7 +234,11 @@ public: { const ColumnConst * column_const_array = checkAndGetColumnConst(column_array_ptr.get()); if (!column_const_array) - throw Exception("Expected array column, found " + column_array_ptr->getName(), ErrorCodes::ILLEGAL_COLUMN); + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Expected {} column, found {}", + argument_type_name, + column_array_ptr->getName()); column_array_ptr = column_const_array->convertToFullColumn(); column_array = assert_cast(column_array_ptr.get()); } @@ -279,13 +288,15 @@ public: { const ColumnConst * column_const_array = checkAndGetColumnConst(column_array_ptr.get()); if (!column_const_array) - throw Exception("Expected array column, found " + column_array_ptr->getName(), ErrorCodes::ILLEGAL_COLUMN); + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, "Expected {} column, found {}", argument_type_name, column_array_ptr->getName()); column_array_ptr = recursiveRemoveLowCardinality(column_const_array->convertToFullColumn()); column_array = checkAndGetColumn(column_array_ptr.get()); } if (!array_type) - throw Exception("Expected array type, found " + array_type_ptr->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Expected {} type, found {}", argument_type_name, array_type_ptr->getName()); if (!offsets_column) { @@ -296,7 +307,11 @@ public: /// The first condition is optimization: do not compare data if the pointers are equal. if (getOffsetsPtr(*column_array) != offsets_column && getOffsets(*column_array) != typeid_cast(*offsets_column).getData()) - throw Exception("Arrays passed to " + getName() + " must have equal size", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); + throw Exception( + ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH, + "{}s passed to {} must have equal size", + argument_type_name, + getName()); } if (i == 1) @@ -305,7 +320,7 @@ public: column_first_array = column_array; } - if constexpr (std::is_same_v) + if constexpr (is_argument_type_map) { arrays.emplace_back(ColumnWithTypeAndName( column_array->getNestedData().getColumnPtr(0), recursiveRemoveLowCardinality(array_type->getKeyType()), array_with_type_and_name.name+".key")); From 2b792a2bdd0768aef04228c55e291d679e4c8365 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 24 Aug 2022 12:40:44 +0200 Subject: [PATCH 26/44] Add more logs --- src/Common/FileCache.cpp | 2 +- src/Common/LRUFileCachePriority.cpp | 10 +++++++--- src/Common/LRUFileCachePriority.h | 2 ++ 3 files changed, 10 insertions(+), 4 deletions(-) diff --git a/src/Common/FileCache.cpp b/src/Common/FileCache.cpp index 1aa8a25bb79..7f20d56b11b 100644 --- a/src/Common/FileCache.cpp +++ b/src/Common/FileCache.cpp @@ -872,7 +872,7 @@ void FileCache::remove( Key key, size_t offset, std::lock_guard & cache_lock, std::lock_guard & /* segment_lock */) { - LOG_TEST(log, "Remove. Key: {}, offset: {}", key.toString(), offset); + LOG_DEBUG(log, "Remove from cache. Key: {}, offset: {}", key.toString(), offset); auto * cell = getCell(key, offset, cache_lock); if (!cell) diff --git a/src/Common/LRUFileCachePriority.cpp b/src/Common/LRUFileCachePriority.cpp index 91addc92501..c4d6313e4d6 100644 --- a/src/Common/LRUFileCachePriority.cpp +++ b/src/Common/LRUFileCachePriority.cpp @@ -24,9 +24,7 @@ IFileCachePriority::WriteIterator LRUFileCachePriority::add(const Key & key, siz throw Exception( ErrorCodes::LOGICAL_ERROR, "Attempt to add duplicate queue entry to queue. (Key: {}, offset: {}, size: {})", - entry.key.toString(), - entry.offset, - entry.size); + entry.key.toString(), entry.offset, entry.size); } #endif @@ -36,6 +34,8 @@ IFileCachePriority::WriteIterator LRUFileCachePriority::add(const Key & key, siz CurrentMetrics::add(CurrentMetrics::FilesystemCacheSize, size); CurrentMetrics::add(CurrentMetrics::FilesystemCacheElements); + LOG_DEBUG(log, "Added entry into LRU queue, key: {}, offset: {}", key.toString(), offset); + return std::make_shared(this, iter); } @@ -54,6 +54,8 @@ void LRUFileCachePriority::removeAll(std::lock_guard &) CurrentMetrics::sub(CurrentMetrics::FilesystemCacheSize, cache_size); CurrentMetrics::sub(CurrentMetrics::FilesystemCacheElements, queue.size()); + LOG_DEBUG(log, "Removed all entries from LRU queue"); + queue.clear(); cache_size = 0; } @@ -86,6 +88,8 @@ void LRUFileCachePriority::LRUFileCacheIterator::removeAndGetNext(std::lock_guar CurrentMetrics::sub(CurrentMetrics::FilesystemCacheSize, queue_iter->size); CurrentMetrics::sub(CurrentMetrics::FilesystemCacheElements); + LOG_DEBUG(cache_priority->log, "Removed entry from LRU queue, key: {}, offset: {}", queue_iter->key.toString(), queue_iter->offset); + queue_iter = cache_priority->queue.erase(queue_iter); } diff --git a/src/Common/LRUFileCachePriority.h b/src/Common/LRUFileCachePriority.h index 7ea35e9a5eb..2cdcc981e04 100644 --- a/src/Common/LRUFileCachePriority.h +++ b/src/Common/LRUFileCachePriority.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB { @@ -32,6 +33,7 @@ public: private: LRUQueue queue; + Poco::Logger * log = &Poco::Logger::get("LRUFileCachePriority"); }; class LRUFileCachePriority::LRUFileCacheIterator : public IFileCachePriority::IIterator From 93d0113b147eb5988156991e0c63ed9f8969a952 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 24 Aug 2022 11:28:56 +0000 Subject: [PATCH 27/44] Fix flacky schema inference cache tests --- .../test_file_schema_inference_cache/test.py | 2 +- tests/integration/test_storage_hdfs/test.py | 2 +- tests/integration/test_storage_s3/test.py | 14 +++++--------- 3 files changed, 7 insertions(+), 11 deletions(-) diff --git a/tests/integration/test_file_schema_inference_cache/test.py b/tests/integration/test_file_schema_inference_cache/test.py index 30c9a788d6f..b8f6ac51186 100755 --- a/tests/integration/test_file_schema_inference_cache/test.py +++ b/tests/integration/test_file_schema_inference_cache/test.py @@ -27,7 +27,7 @@ def get_profile_event_for_query(node, query, profile_event): query = query.replace("'", "\\'") return int( node.query( - f"select ProfileEvents['{profile_event}'] from system.query_log where query='{query}' and type = 'QueryFinish' order by event_time desc limit 1" + f"select ProfileEvents['{profile_event}'] from system.query_log where query='{query}' and type = 'QueryFinish' order by query_start_time_microseconds desc limit 1" ) ) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 86fb5ab578c..34243e4b58d 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -636,7 +636,7 @@ def get_profile_event_for_query(node, query, profile_event): query = query.replace("'", "\\'") return int( node.query( - f"select ProfileEvents['{profile_event}'] from system.query_log where query='{query}' and type = 'QueryFinish' order by event_time desc limit 1" + f"select ProfileEvents['{profile_event}'] from system.query_log where query='{query}' and type = 'QueryFinish' order by query_start_time_microseconds desc limit 1" ) ) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index c5c64ee03b0..67857437a45 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1493,16 +1493,12 @@ def test_wrong_format_usage(started_cluster): def check_profile_event_for_query(instance, query, profile_event, amount): instance.query("system flush logs") query = query.replace("'", "\\'") - attempt = 0 - res = 0 - while attempt < 10: - res = int( - instance.query( - f"select ProfileEvents['{profile_event}'] from system.query_log where query='{query}' and type = 'QueryFinish' order by event_time desc limit 1" - ) + res = int( + instance.query( + f"select ProfileEvents['{profile_event}'] from system.query_log where query='{query}' and type = 'QueryFinish' order by query_start_time_microseconds desc limit 1" ) - if res == amount: - break + ) + assert res == amount From 109f384e5d0351a6cd398ee8ae9b8a58f2cc8822 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 24 Aug 2022 13:40:19 +0200 Subject: [PATCH 28/44] Fix typo --- src/IO/ReadHelpers.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 6b06b0005b8..a3e03695837 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -966,7 +966,7 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re else DecimalUtils::tryGetDecimalFromComponents(components, scale, datetime64); - negative_multiplier *= negative_multiplier; + datetime64 *= negative_multiplier; return ReturnType(true); From 074e0f06e5f1751d8c3d13e0e1533b0387019459 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 24 Aug 2022 13:42:29 +0200 Subject: [PATCH 29/44] Update test --- tests/queries/0_stateless/02406_try_read_datetime64_bug.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02406_try_read_datetime64_bug.sql b/tests/queries/0_stateless/02406_try_read_datetime64_bug.sql index 26cff0dd049..846e732ba8e 100644 --- a/tests/queries/0_stateless/02406_try_read_datetime64_bug.sql +++ b/tests/queries/0_stateless/02406_try_read_datetime64_bug.sql @@ -1,2 +1,2 @@ -select toDateTime64OrDefault('Aaaa e a.a.aaaaaaaaa', 9); +select toDateTime64OrDefault('Aaaa e a.a.aaaaaaaaa', 9, 'UTC'); desc format(CSV, '"Aaaa e a.a.aaaaaaaaa"'); From 3e8a2d909311d3d9716ef60042431d8326c64a2a Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 24 Aug 2022 14:16:53 +0200 Subject: [PATCH 30/44] Update FileSegment.cpp --- src/Common/FileSegment.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/FileSegment.cpp b/src/Common/FileSegment.cpp index 0b2e874e9ab..2857fd66369 100644 --- a/src/Common/FileSegment.cpp +++ b/src/Common/FileSegment.cpp @@ -55,6 +55,7 @@ FileSegment::FileSegment( case (State::DOWNLOADED): { reserved_size = downloaded_size = size_; + is_downloaded = true; break; } case (State::SKIP_CACHE): From 35f9815b8ecc5d5babbd19c5df031dcaf2816da5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 24 Aug 2022 14:43:02 +0200 Subject: [PATCH 31/44] Fix backward comp check --- docker/test/stress/run.sh | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 5dc5ffa2f21..4e38f28d5f0 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -303,7 +303,6 @@ else rm -rf /var/lib/clickhouse/* # Make BC check more funny by forcing Ordinary engine for system database - # New version will try to convert it to Atomic on startup mkdir /var/lib/clickhouse/metadata echo "ATTACH DATABASE system ENGINE=Ordinary" > /var/lib/clickhouse/metadata/system.sql @@ -313,16 +312,13 @@ else # Start server from previous release configure - # Avoid "Setting allow_deprecated_database_ordinary is neither a builtin setting..." - rm -f /etc/clickhouse-server/users.d/database_ordinary.xml ||: + # Avoid "Setting s3_check_objects_after_upload is neither a builtin setting..." + rm -f /etc/clickhouse-server/users.d/enable_blobs_check.xml ||: # Remove s3 related configs to avoid "there is no disk type `cache`" rm -f /etc/clickhouse-server/config.d/storage_conf.xml ||: rm -f /etc/clickhouse-server/config.d/azure_storage_conf.xml ||: - # Disable aggressive cleanup of tmp dirs (it worked incorrectly before 22.8) - rm -f /etc/clickhouse-server/config.d/merge_tree_old_dirs_cleanup.xml ||: - start clickhouse-client --query="SELECT 'Server version: ', version()" From 0fa8ae5f1a98bed7d390c45a9791c501a429bf7d Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 24 Aug 2022 13:29:06 +0000 Subject: [PATCH 32/44] Small script to create empty test file with new number assigned --- tests/queries/0_stateless/add_test.sh | 27 +++++++++++++++++++++++++++ 1 file changed, 27 insertions(+) create mode 100755 tests/queries/0_stateless/add_test.sh diff --git a/tests/queries/0_stateless/add_test.sh b/tests/queries/0_stateless/add_test.sh new file mode 100755 index 00000000000..30823d3541b --- /dev/null +++ b/tests/queries/0_stateless/add_test.sh @@ -0,0 +1,27 @@ +#!/bin/bash + +if [ -z "$1" ]; then + echo "Helper script to create empty test and reference files and assign a new number." + echo "Usage: $0 " + exit 1 +fi + +TESTS_PATH=$(dirname ${BASH_SOURCE}) +set -ue + +LAST_TEST_NO=$(ls -1 ${TESTS_PATH} | grep -P -o '^\d+' | sort -nr | head -1) + +# remove leading zeros, increment and add padding zeros to 5 digits +NEW_TEST_NO=$(printf "%05d\n" $((10#$LAST_TEST_NO + 1))) + +# if extension is not provided, use `.sql` +FILENAME="${1}" +FILEEXT="sql" +if [[ $1 == *.* ]] ; then + FILENAME="${1%.*}" + FILEEXT="${1##*.}" +fi + +set -x +touch ${TESTS_PATH}/${NEW_TEST_NO}_${FILENAME}.${FILEEXT} +touch ${TESTS_PATH}/${NEW_TEST_NO}_${FILENAME}.reference From 2cea295a7ad23e4a6fcb9d8e16efddef40682b24 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 24 Aug 2022 15:53:54 +0200 Subject: [PATCH 33/44] More logs --- src/Common/FileCache.cpp | 7 ++++--- src/Common/FileSegment.cpp | 9 +++++++-- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/src/Common/FileCache.cpp b/src/Common/FileCache.cpp index 7f20d56b11b..436992df6b3 100644 --- a/src/Common/FileCache.cpp +++ b/src/Common/FileCache.cpp @@ -108,9 +108,10 @@ void FileCache::useCell( if (file_segment->isDownloaded() && fs::file_size(getPathInLocalCache(file_segment->key(), file_segment->offset(), file_segment->isPersistent())) == 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Cannot have zero size downloaded file segments. Current file segment: {}", - file_segment->range().toString()); + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot have zero size downloaded file segments. {}", + file_segment->getInfoForLog()); result.push_back(cell.file_segment); diff --git a/src/Common/FileSegment.cpp b/src/Common/FileSegment.cpp index c2a12b38320..9f331a5e1a1 100644 --- a/src/Common/FileSegment.cpp +++ b/src/Common/FileSegment.cpp @@ -458,7 +458,11 @@ void FileSegment::completeWithState(State state, bool auto_resize) { if (auto_resize && downloaded_size != range().size()) { - LOG_TEST(log, "Resize cell {} to downloaded: {}", range().toString(), downloaded_size); + LOG_TRACE( + log, + "Resize cell {} to downloaded: {} ({})", range().toString(), + downloaded_size, file_segment->getInfoForLogImpl(segment_lock)); + assert(downloaded_size <= range().size()); segment_range = Range(segment_range.left, segment_range.left + downloaded_size - 1); } @@ -613,6 +617,7 @@ String FileSegment::getInfoForLogImpl(std::lock_guard & segment_lock { WriteBufferFromOwnString info; info << "File segment: " << range().toString() << ", "; + info << "key: " << key().toString() << ", "; info << "state: " << download_state << ", "; info << "downloaded size: " << getDownloadedSize(segment_lock) << ", "; info << "reserved size: " << reserved_size << ", "; @@ -738,7 +743,7 @@ void FileSegment::detach( download_state = State::PARTIALLY_DOWNLOADED_NO_CONTINUATION; downloader_id.clear(); - LOG_TEST(log, "Detached file segment: {}", getInfoForLogImpl(segment_lock)); + LOG_DEBUG(log, "Detached file segment: {}", getInfoForLogImpl(segment_lock)); } void FileSegment::markAsDetached(std::lock_guard & /* segment_lock */) From 6d0907ebef0407fd7a75986eb2cd672d88f83cb6 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 24 Aug 2022 15:55:07 +0200 Subject: [PATCH 34/44] Update FileSegment.cpp --- src/Common/FileSegment.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/FileSegment.cpp b/src/Common/FileSegment.cpp index 9f331a5e1a1..9f80a73c6f0 100644 --- a/src/Common/FileSegment.cpp +++ b/src/Common/FileSegment.cpp @@ -460,8 +460,8 @@ void FileSegment::completeWithState(State state, bool auto_resize) { LOG_TRACE( log, - "Resize cell {} to downloaded: {} ({})", range().toString(), - downloaded_size, file_segment->getInfoForLogImpl(segment_lock)); + "Resize cell {} to downloaded: {} ({})", + range().toString(), downloaded_size, file_segment->getInfoForLogImpl(segment_lock)); assert(downloaded_size <= range().size()); segment_range = Range(segment_range.left, segment_range.left + downloaded_size - 1); From 52604e0168a2cb2601640ae20de9237cc6252a6d Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 24 Aug 2022 14:09:19 +0000 Subject: [PATCH 35/44] Add test for PR #40335 --- .../0_stateless/02405_pmj_issue_40335.reference | 5 +++++ .../queries/0_stateless/02405_pmj_issue_40335.sql | 15 +++++++++++++++ 2 files changed, 20 insertions(+) create mode 100644 tests/queries/0_stateless/02405_pmj_issue_40335.reference create mode 100644 tests/queries/0_stateless/02405_pmj_issue_40335.sql diff --git a/tests/queries/0_stateless/02405_pmj_issue_40335.reference b/tests/queries/0_stateless/02405_pmj_issue_40335.reference new file mode 100644 index 00000000000..94053253394 --- /dev/null +++ b/tests/queries/0_stateless/02405_pmj_issue_40335.reference @@ -0,0 +1,5 @@ +a +b +c +d +e diff --git a/tests/queries/0_stateless/02405_pmj_issue_40335.sql b/tests/queries/0_stateless/02405_pmj_issue_40335.sql new file mode 100644 index 00000000000..32aa3537ae8 --- /dev/null +++ b/tests/queries/0_stateless/02405_pmj_issue_40335.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + +CREATE TABLE t1 (x UInt64) ENGINE = TinyLog; +INSERT INTO t1 VALUES (1), (2), (3); + +CREATE TABLE t2 (x UInt64, value String) ENGINE = TinyLog; +INSERT INTO t2 VALUES (1, 'a'), (2, 'b'), (2, 'c'); +INSERT INTO t2 VALUES (3, 'd'), (3, 'e'), (4, 'f'); + +SET max_block_size=3; +SET max_joined_block_size_rows = 2; +SET join_algorithm='partial_merge'; + +SELECT value FROM t1 LEFT JOIN t2 ON t1.x = t2.x; From aa0ed28a0b7b2c42d53f798c8d9af08c36ccf2a7 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 24 Aug 2022 16:43:44 +0200 Subject: [PATCH 36/44] Update FileSegment.cpp --- src/Common/FileSegment.cpp | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Common/FileSegment.cpp b/src/Common/FileSegment.cpp index 149609d8a02..f13c41b7278 100644 --- a/src/Common/FileSegment.cpp +++ b/src/Common/FileSegment.cpp @@ -537,10 +537,7 @@ void FileSegment::completeBasedOnCurrentState(std::lock_guard & cach } else { - LOG_DEBUG( - log, - "Resize cell {} to downloaded: {} ({})", - range().toString(), current_downloaded_size, getInfoForLogImpl(segment_lock)); + LOG_TEST(log, "Resize cell {} to downloaded: {}", range().toString(), current_downloaded_size); /** * Only last holder of current file segment can resize the cell, From 054f610c5079f8eec276a9ef9ea91e509c0b7ac7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 24 Aug 2022 17:27:56 +0200 Subject: [PATCH 37/44] adjust timeouts for replication consistency check --- tests/queries/0_stateless/01154_move_partition_long.sh | 2 +- tests/queries/0_stateless/replication.lib | 10 ++++++---- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01154_move_partition_long.sh b/tests/queries/0_stateless/01154_move_partition_long.sh index a8f12d6afbd..c68b0944407 100755 --- a/tests/queries/0_stateless/01154_move_partition_long.sh +++ b/tests/queries/0_stateless/01154_move_partition_long.sh @@ -123,7 +123,7 @@ timeout $TIMEOUT bash -c drop_part_thread & wait check_replication_consistency "dst_" "count(), sum(p), sum(k), sum(v)" -try_sync_replicas "src_" +try_sync_replicas "src_" 300 for ((i=0; i<16; i++)) do $CLICKHOUSE_CLIENT -q "DROP TABLE dst_$i" 2>&1| grep -Fv "is already started to be removing" & diff --git a/tests/queries/0_stateless/replication.lib b/tests/queries/0_stateless/replication.lib index 8da21a3fdbe..72d51c48656 100755 --- a/tests/queries/0_stateless/replication.lib +++ b/tests/queries/0_stateless/replication.lib @@ -5,6 +5,7 @@ function try_sync_replicas() { table_name_prefix=$1 + time_left=$2 readarray -t empty_partitions_arr < <(${CLICKHOUSE_CLIENT} -q \ "SELECT DISTINCT substr(new_part_name, 1, position(new_part_name, '_') - 1) AS partition_id @@ -29,7 +30,7 @@ function try_sync_replicas() for t in "${tables_arr[@]}" do # The size of log may be big, so increase timeout. - $CLICKHOUSE_CLIENT --receive_timeout 300 -q "SYSTEM SYNC REPLICA $t" || ($CLICKHOUSE_CLIENT -q \ + $CLICKHOUSE_CLIENT --receive_timeout $time_left -q "SYSTEM SYNC REPLICA $t" || ($CLICKHOUSE_CLIENT -q \ "select 'sync failed, queue:', * from system.replication_queue where database=currentDatabase() and table='$t' order by database, table, node_name" && exit 1) & pids[${i}]=$! i=$((i + 1)) @@ -48,13 +49,14 @@ function check_replication_consistency() # Wait for all queries to finish (query may still be running if thread is killed by timeout) num_tries=0 while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE current_database=currentDatabase() AND query LIKE '%$table_name_prefix%'") -ne 1 ]]; do - sleep 0.5; + sleep 1; num_tries=$((num_tries+1)) - if [ $num_tries -eq 200 ]; then + if [ $num_tries -eq 250 ]; then $CLICKHOUSE_CLIENT -q "SELECT * FROM system.processes WHERE current_database=currentDatabase() AND query LIKE '%$table_name_prefix%' FORMAT Vertical" break fi done + time_left=$((300 - num_tries)) # Do not check anything if all replicas are readonly, # because is this case all replicas are probably lost (it may happen and it's not a bug) @@ -78,7 +80,7 @@ function check_replication_consistency() # SYNC REPLICA is not enough if some MUTATE_PARTs are not assigned yet wait_for_all_mutations "$table_name_prefix%" - try_sync_replicas "$table_name_prefix" || exit 1 + try_sync_replicas "$table_name_prefix" "$time_left" || exit 1 res=$($CLICKHOUSE_CLIENT -q \ "SELECT From 3b982f6b8ba2f376d280932ac95414834f5a2a93 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 24 Aug 2022 17:29:43 +0200 Subject: [PATCH 38/44] Update FileCache.cpp --- src/Common/FileCache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/FileCache.cpp b/src/Common/FileCache.cpp index 436992df6b3..0ac047e0818 100644 --- a/src/Common/FileCache.cpp +++ b/src/Common/FileCache.cpp @@ -109,7 +109,7 @@ void FileCache::useCell( if (file_segment->isDownloaded() && fs::file_size(getPathInLocalCache(file_segment->key(), file_segment->offset(), file_segment->isPersistent())) == 0) throw Exception( - ErrorCodes::LOGICAL_ERROR, + ErrorCodes::LOGICAL_ERROR, "Cannot have zero size downloaded file segments. {}", file_segment->getInfoForLog()); From 814bc37f0e2f2d720bcadc3072cfe580c180ecd3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 24 Aug 2022 17:45:20 +0200 Subject: [PATCH 39/44] Use DiskPtr --- src/Backups/BackupEntryFromImmutableFile.h | 2 +- src/Backups/BackupEntryFromMemory.h | 2 +- src/Backups/BackupEntryFromSmallFile.h | 2 +- src/Backups/BackupIO.cpp | 2 +- src/Backups/BackupIO.h | 2 +- src/Backups/BackupIO_Disk.cpp | 2 +- src/Backups/BackupIO_Disk.h | 2 +- src/Backups/BackupIO_File.cpp | 2 +- src/Backups/BackupIO_File.h | 2 +- src/Backups/BackupImpl.cpp | 2 +- src/Backups/IBackupEntriesLazyBatch.cpp | 2 +- src/Backups/IBackupEntry.h | 2 +- 12 files changed, 12 insertions(+), 12 deletions(-) diff --git a/src/Backups/BackupEntryFromImmutableFile.h b/src/Backups/BackupEntryFromImmutableFile.h index 68c50ff79af..fa36fe33485 100644 --- a/src/Backups/BackupEntryFromImmutableFile.h +++ b/src/Backups/BackupEntryFromImmutableFile.h @@ -32,7 +32,7 @@ public: String getFilePath() const override; DataSourceDescription getDataSourceDescription() const override; - std::shared_ptr tryGetDiskIfExists() const override { return disk; } + DiskPtr tryGetDiskIfExists() const override { return disk; } private: const DiskPtr disk; diff --git a/src/Backups/BackupEntryFromMemory.h b/src/Backups/BackupEntryFromMemory.h index fb2f32d980d..df3b9de40e3 100644 --- a/src/Backups/BackupEntryFromMemory.h +++ b/src/Backups/BackupEntryFromMemory.h @@ -29,7 +29,7 @@ public: return DataSourceDescription{DataSourceType::RAM, "", false, false}; } - std::shared_ptr tryGetDiskIfExists() const override { return nullptr; } + DiskPtr tryGetDiskIfExists() const override { return nullptr; } private: const String data; diff --git a/src/Backups/BackupEntryFromSmallFile.h b/src/Backups/BackupEntryFromSmallFile.h index 66ab1c4f3d3..99e319f07a0 100644 --- a/src/Backups/BackupEntryFromSmallFile.h +++ b/src/Backups/BackupEntryFromSmallFile.h @@ -25,7 +25,7 @@ public: String getFilePath() const override { return file_path; } - std::shared_ptr tryGetDiskIfExists() const override { return disk; } + DiskPtr tryGetDiskIfExists() const override { return disk; } private: const DiskPtr disk; const String file_path; diff --git a/src/Backups/BackupIO.cpp b/src/Backups/BackupIO.cpp index 4d1b5a88216..3641ab9ac7b 100644 --- a/src/Backups/BackupIO.cpp +++ b/src/Backups/BackupIO.cpp @@ -19,7 +19,7 @@ void IBackupWriter::copyFileThroughBuffer(std::unique_ptr && write_buffer->finalize(); } -void IBackupWriter::copyFileNative(std::shared_ptr /* from_disk */, const String & /* file_name_from */, const String & /* file_name_to */) +void IBackupWriter::copyFileNative(DiskPtr /* from_disk */, const String & /* file_name_from */, const String & /* file_name_to */) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Native copy not implemented for backup writer"); } diff --git a/src/Backups/BackupIO.h b/src/Backups/BackupIO.h index c8e4b102970..8c400fca023 100644 --- a/src/Backups/BackupIO.h +++ b/src/Backups/BackupIO.h @@ -38,7 +38,7 @@ public: return false; } - virtual void copyFileNative(std::shared_ptr from_disk, const String & file_name_from, const String & file_name_to); + virtual void copyFileNative(DiskPtr from_disk, const String & file_name_from, const String & file_name_to); }; } diff --git a/src/Backups/BackupIO_Disk.cpp b/src/Backups/BackupIO_Disk.cpp index b87f14d9e7f..f88abb40194 100644 --- a/src/Backups/BackupIO_Disk.cpp +++ b/src/Backups/BackupIO_Disk.cpp @@ -98,7 +98,7 @@ bool BackupWriterDisk::supportNativeCopy(DataSourceDescription data_source_descr return data_source_description == disk->getDataSourceDescription(); } -void BackupWriterDisk::copyFileNative(std::shared_ptr from_disk, const String & file_name_from, const String & file_name_to) +void BackupWriterDisk::copyFileNative(DiskPtr from_disk, const String & file_name_from, const String & file_name_to) { auto file_path = path / file_name_to; disk->createDirectories(file_path.parent_path()); diff --git a/src/Backups/BackupIO_Disk.h b/src/Backups/BackupIO_Disk.h index abc5b831c7b..fd37691eeb7 100644 --- a/src/Backups/BackupIO_Disk.h +++ b/src/Backups/BackupIO_Disk.h @@ -39,7 +39,7 @@ public: bool supportNativeCopy(DataSourceDescription data_source_description) const override; - void copyFileNative(std::shared_ptr from_disk, const String & file_name_from, const String & file_name_to) override; + void copyFileNative(DiskPtr from_disk, const String & file_name_from, const String & file_name_to) override; private: DiskPtr disk; std::filesystem::path path; diff --git a/src/Backups/BackupIO_File.cpp b/src/Backups/BackupIO_File.cpp index a401fb1a5d5..8c043d49ff8 100644 --- a/src/Backups/BackupIO_File.cpp +++ b/src/Backups/BackupIO_File.cpp @@ -118,7 +118,7 @@ bool BackupWriterFile::supportNativeCopy(DataSourceDescription data_source_descr return data_source_description == getDataSourceDescription(); } -void BackupWriterFile::copyFileNative(std::shared_ptr from_disk, const String & file_name_from, const String & file_name_to) +void BackupWriterFile::copyFileNative(DiskPtr from_disk, const String & file_name_from, const String & file_name_to) { auto file_path = path / file_name_to; fs::create_directories(file_path.parent_path()); diff --git a/src/Backups/BackupIO_File.h b/src/Backups/BackupIO_File.h index a3c6373fffe..425a019c71a 100644 --- a/src/Backups/BackupIO_File.h +++ b/src/Backups/BackupIO_File.h @@ -35,7 +35,7 @@ public: DataSourceDescription getDataSourceDescription() const override; bool supportNativeCopy(DataSourceDescription data_source_description) const override; - void copyFileNative(std::shared_ptr from_disk, const String & file_name_from, const String & file_name_to) override; + void copyFileNative(DiskPtr from_disk, const String & file_name_from, const String & file_name_to) override; private: std::filesystem::path path; diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index b569c29887f..f6442545f48 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -116,7 +116,7 @@ public: return data_file_name; } - std::shared_ptr tryGetDiskIfExists() const override + DiskPtr tryGetDiskIfExists() const override { return nullptr; } diff --git a/src/Backups/IBackupEntriesLazyBatch.cpp b/src/Backups/IBackupEntriesLazyBatch.cpp index e265ec31f54..78086015e7b 100644 --- a/src/Backups/IBackupEntriesLazyBatch.cpp +++ b/src/Backups/IBackupEntriesLazyBatch.cpp @@ -25,7 +25,7 @@ public: return getInternalBackupEntry()->getFilePath(); } - std::shared_ptr tryGetDiskIfExists() const override + DiskPtr tryGetDiskIfExists() const override { return getInternalBackupEntry()->tryGetDiskIfExists(); } diff --git a/src/Backups/IBackupEntry.h b/src/Backups/IBackupEntry.h index 32f2d03af07..2a71a1e9756 100644 --- a/src/Backups/IBackupEntry.h +++ b/src/Backups/IBackupEntry.h @@ -29,7 +29,7 @@ public: virtual String getFilePath() const = 0; - virtual std::shared_ptr tryGetDiskIfExists() const = 0; + virtual DiskPtr tryGetDiskIfExists() const = 0; virtual DataSourceDescription getDataSourceDescription() const = 0; }; From 571778ad256b17efe74e5c2629670b0506e4c929 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 24 Aug 2022 17:45:26 +0200 Subject: [PATCH 40/44] Update src/Backups/BackupIO_Disk.cpp Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- src/Backups/BackupIO_Disk.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Backups/BackupIO_Disk.cpp b/src/Backups/BackupIO_Disk.cpp index b87f14d9e7f..4000b77ea8b 100644 --- a/src/Backups/BackupIO_Disk.cpp +++ b/src/Backups/BackupIO_Disk.cpp @@ -100,12 +100,11 @@ bool BackupWriterDisk::supportNativeCopy(DataSourceDescription data_source_descr void BackupWriterDisk::copyFileNative(std::shared_ptr from_disk, const String & file_name_from, const String & file_name_to) { + if (!from_disk) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot natively copy data to disk without source disk"); auto file_path = path / file_name_to; disk->createDirectories(file_path.parent_path()); - if (from_disk) - from_disk->copyFile(file_name_from, *disk, file_path); - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot natively copy data to disk without source disk"); + from_disk->copyFile(file_name_from, *disk, file_path); } } From 51afaedbf298082ea165faeba774bdd82f2a8c89 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 24 Aug 2022 17:51:26 +0200 Subject: [PATCH 41/44] Better defaults --- src/Disks/DiskType.cpp | 4 ++-- src/Disks/DiskType.h | 7 ++++--- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Disks/DiskType.cpp b/src/Disks/DiskType.cpp index b5ae9ca4c71..92979ab505c 100644 --- a/src/Disks/DiskType.cpp +++ b/src/Disks/DiskType.cpp @@ -3,9 +3,9 @@ namespace DB { -bool DataSourceDescription::operator==(const DataSourceDescription & o) const +bool DataSourceDescription::operator==(const DataSourceDescription & other) const { - return std::tie(type, description, is_encrypted) == std::tie(o.type, o.description, o.is_encrypted); + return std::tie(type, description, is_encrypted) == std::tie(other.type, other.description, other.is_encrypted); } } diff --git a/src/Disks/DiskType.h b/src/Disks/DiskType.h index 53f555db088..1a5c7312cb3 100644 --- a/src/Disks/DiskType.h +++ b/src/Disks/DiskType.h @@ -39,10 +39,11 @@ struct DataSourceDescription { DataSourceType type; std::string description; - bool is_encrypted; - bool is_cached; - bool operator==(const DataSourceDescription & o) const; + bool is_encrypted = false; + bool is_cached = false; + + bool operator==(const DataSourceDescription & other) const; }; } From b42fcfd37f6d2d72ed5af3c2751772a1e50d139b Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 24 Aug 2022 16:27:58 +0000 Subject: [PATCH 42/44] shellcheck tests/queries/0_stateless/add_test.sh --- tests/queries/0_stateless/add_test.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/add_test.sh b/tests/queries/0_stateless/add_test.sh index 30823d3541b..2173a4d8cc2 100755 --- a/tests/queries/0_stateless/add_test.sh +++ b/tests/queries/0_stateless/add_test.sh @@ -6,9 +6,10 @@ if [ -z "$1" ]; then exit 1 fi -TESTS_PATH=$(dirname ${BASH_SOURCE}) +TESTS_PATH=$(dirname ${BASH_SOURCE[0]}) set -ue +# shellcheck disable=SC2010 LAST_TEST_NO=$(ls -1 ${TESTS_PATH} | grep -P -o '^\d+' | sort -nr | head -1) # remove leading zeros, increment and add padding zeros to 5 digits From 56e8e247e46ec51a08c3b7bc5bd6ac7cdaba539f Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 24 Aug 2022 16:28:58 +0000 Subject: [PATCH 43/44] Rename add_test.sh -> add-test --- tests/queries/0_stateless/{add_test.sh => add-test} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{add_test.sh => add-test} (100%) diff --git a/tests/queries/0_stateless/add_test.sh b/tests/queries/0_stateless/add-test similarity index 100% rename from tests/queries/0_stateless/add_test.sh rename to tests/queries/0_stateless/add-test From ba4dfe83089b09def0c556f75d582234b0cdea7e Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 25 Aug 2022 00:23:59 +0200 Subject: [PATCH 44/44] Update MaterializedPostgreSQLConsumer.cpp --- src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index e25e8d74a29..f0c5807f89c 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -45,7 +45,6 @@ MaterializedPostgreSQLConsumer::MaterializedPostgreSQLConsumer( , schema_as_a_part_of_table_name(schema_as_a_part_of_table_name_) , allow_automatic_update(allow_automatic_update_) { - committed = false; final_lsn = start_lsn; auto tx = std::make_shared(connection->getRef()); current_lsn = advanceLSN(tx); @@ -780,7 +779,7 @@ bool MaterializedPostgreSQLConsumer::readFromReplicationSlot() try { - LOG_DEBUG(log, "Current message: {}", (*row)[1]); + /// LOG_DEBUG(log, "Current message: {}", (*row)[1]); processReplicationMessage((*row)[1].c_str(), (*row)[1].size()); } catch (const Exception & e)