From 961bf074daf0c901a3e9d14b6caa4ba6cb37cc7c Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 20 Nov 2023 10:56:10 +0100 Subject: [PATCH 001/276] Initial draft version of adding backup support to AzureBlobStorage --- src/Backups/BackupFactory.cpp | 2 + src/Backups/BackupIO_AzureBlobStorage.cpp | 336 ++++++++++++++++++ src/Backups/BackupIO_AzureBlobStorage.h | 69 ++++ src/Backups/BackupImpl.cpp | 8 +- .../registerBackupEngineAzureBlobStorage.cpp | 134 +++++++ src/CMakeLists.txt | 3 + src/Common/ProfileEvents.cpp | 4 + .../copyAzureBlobStorageFile.cpp | 324 +++++++++++++++++ .../copyAzureBlobStorageFile.h | 58 +++ src/Storages/StorageAzureBlob.cpp | 11 + src/Storages/StorageAzureBlob.h | 1 + .../__init__.py | 1 + .../configs/config.xml | 11 + .../configs/disable_profilers.xml | 13 + .../configs/users.xml | 8 + .../test.py | 151 ++++++++ 16 files changed, 1132 insertions(+), 2 deletions(-) create mode 100644 src/Backups/BackupIO_AzureBlobStorage.cpp create mode 100644 src/Backups/BackupIO_AzureBlobStorage.h create mode 100644 src/Backups/registerBackupEngineAzureBlobStorage.cpp create mode 100644 src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp create mode 100644 src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h create mode 100644 tests/integration/test_backup_restore_azure_blob_storage/__init__.py create mode 100644 tests/integration/test_backup_restore_azure_blob_storage/configs/config.xml create mode 100644 tests/integration/test_backup_restore_azure_blob_storage/configs/disable_profilers.xml create mode 100644 tests/integration/test_backup_restore_azure_blob_storage/configs/users.xml create mode 100644 tests/integration/test_backup_restore_azure_blob_storage/test.py diff --git a/src/Backups/BackupFactory.cpp b/src/Backups/BackupFactory.cpp index 898ac7bc490..31e87a21fc2 100644 --- a/src/Backups/BackupFactory.cpp +++ b/src/Backups/BackupFactory.cpp @@ -33,11 +33,13 @@ void BackupFactory::registerBackupEngine(const String & engine_name, const Creat void registerBackupEnginesFileAndDisk(BackupFactory &); void registerBackupEngineS3(BackupFactory &); +void registerBackupEngineAzureBlobStorage(BackupFactory &); void registerBackupEngines(BackupFactory & factory) { registerBackupEnginesFileAndDisk(factory); registerBackupEngineS3(factory); + registerBackupEngineAzureBlobStorage(factory); } BackupFactory::BackupFactory() diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp new file mode 100644 index 00000000000..d41d23e3c36 --- /dev/null +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -0,0 +1,336 @@ +#include + +#if USE_AZURE_BLOB_STORAGE +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +#include + + +namespace fs = std::filesystem; + +namespace DB +{ +namespace ErrorCodes +{ + extern const int AZURE_BLOB_STORAGE_ERROR; + extern const int LOGICAL_ERROR; +} + +//using AzureClientPtr = std::shared_ptr; + +BackupReaderAzureBlobStorage::BackupReaderAzureBlobStorage( + StorageAzureBlob::Configuration configuration_, + const ReadSettings & read_settings_, + const WriteSettings & write_settings_, + const ContextPtr & context_) + : BackupReaderDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupReaderAzureBlobStorage")) + , data_source_description{DataSourceType::AzureBlobStorage, "AzureBlobStorage", false, false} + , configuration(configuration_) +{ + client = StorageAzureBlob::createClient(configuration, /* is_read_only */ false); + settings = StorageAzureBlob::createSettingsAsSharedPtr(context_); + auto settings_as_unique_ptr = StorageAzureBlob::createSettings(context_); + object_storage = std::make_unique("BackupReaderAzureBlobStorage", + std::make_unique(*client.get()), + std::move(settings_as_unique_ptr)); +} + +BackupReaderAzureBlobStorage::~BackupReaderAzureBlobStorage() = default; + +bool BackupReaderAzureBlobStorage::fileExists(const String & file_name) +{ + String key; + if (startsWith(file_name, ".")) + { + key= configuration.blob_path + file_name; + } + else + { + key = file_name; + } + return object_storage->exists(StoredObject(key)); +} + +UInt64 BackupReaderAzureBlobStorage::getFileSize(const String & file_name) +{ + String key; + if (startsWith(file_name, ".")) + { + key= configuration.blob_path + file_name; + } + else + { + key = file_name; + } + ObjectMetadata object_metadata = object_storage->getObjectMetadata(key); + return object_metadata.size_bytes; +} + +std::unique_ptr BackupReaderAzureBlobStorage::readFile(const String & file_name) +{ + String key; + if (startsWith(file_name, ".")) + { + key= configuration.blob_path + file_name; + } + else + { + key = file_name; + } + return std::make_unique( + client, key, read_settings, settings->max_single_read_retries, + settings->max_single_download_retries); +} + +void BackupReaderAzureBlobStorage::copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, + DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) +{ + LOG_INFO(&Poco::Logger::get("BackupReaderAzureBlobStorage"), "Enter copyFileToDisk"); + + /// Use the native copy as a more optimal way to copy a file from AzureBlobStorage to AzureBlobStorage if it's possible. + /// We don't check for `has_throttling` here because the native copy almost doesn't use network. + auto destination_data_source_description = destination_disk->getDataSourceDescription(); + if (destination_data_source_description.sameKind(data_source_description) + && (destination_data_source_description.is_encrypted == encrypted_in_backup)) + { + LOG_TRACE(log, "Copying {} from AzureBlobStorage to disk {}", path_in_backup, destination_disk->getName()); + auto write_blob_function = [&](const Strings & blob_path, WriteMode mode, const std::optional & object_attributes) -> size_t + { + /// Object storage always uses mode `Rewrite` because it simulates append using metadata and different files. + if (blob_path.size() != 2 || mode != WriteMode::Rewrite) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Blob writing function called with unexpected blob_path.size={} or mode={}", + blob_path.size(), mode); + + std::shared_ptr dest_client; + if (configuration.container == blob_path[1]) + { + dest_client = client; + } + else + { + StorageAzureBlob::Configuration dest_configuration = configuration; + dest_configuration.container = blob_path[1]; + dest_configuration.blob_path = blob_path[0]; + dest_client = StorageAzureBlob::createClient(dest_configuration, /* is_read_only */ false); + } + + + copyAzureBlobStorageFile( + client, + dest_client, + configuration.container, + fs::path(configuration.blob_path) / path_in_backup, + 0, + file_size, + /* dest_bucket= */ blob_path[1], + /* dest_key= */ blob_path[0], + settings, + read_settings, + object_attributes, + threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupReaderAzureBlobStorage"), + /* for_disk_azure_blob_storage= */ true); + + return file_size; + }; + + destination_disk->writeFileUsingBlobWritingFunction(destination_path, write_mode, write_blob_function); + return; /// copied! + } + + /// Fallback to copy through buffers. + BackupReaderDefault::copyFileToDisk(path_in_backup, file_size, encrypted_in_backup, destination_disk, destination_path, write_mode); +} + + +BackupWriterAzureBlobStorage::BackupWriterAzureBlobStorage( + StorageAzureBlob::Configuration configuration_, + const ReadSettings & read_settings_, + const WriteSettings & write_settings_, + const ContextPtr & context_) + : BackupWriterDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupWriterAzureBlobStorage")) + , data_source_description{DataSourceType::AzureBlobStorage, "AzureBlobStorage", false, false} + , configuration(configuration_) +{ + client = StorageAzureBlob::createClient(configuration, /* is_read_only */ false); + settings = StorageAzureBlob::createSettingsAsSharedPtr(context_); + auto settings_as_unique_ptr = StorageAzureBlob::createSettings(context_); + object_storage = std::make_unique("BackupWriterAzureBlobStorage", + std::make_unique(*client.get()), + std::move(settings_as_unique_ptr)); +} + +void BackupWriterAzureBlobStorage::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, + bool copy_encrypted, UInt64 start_pos, UInt64 length) +{ + /// Use the native copy as a more optimal way to copy a file from AzureBlobStorage to AzureBlobStorage if it's possible. + auto source_data_source_description = src_disk->getDataSourceDescription(); + if (source_data_source_description.sameKind(data_source_description) && (source_data_source_description.is_encrypted == copy_encrypted)) + { + /// getBlobPath() can return more than 3 elements if the file is stored as multiple objects in AzureBlobStorage bucket. + /// In this case we can't use the native copy. + if (auto blob_path = src_disk->getBlobPath(src_path); blob_path.size() == 2) + { + + std::shared_ptr src_client; + if (configuration.container == blob_path[1]) + { + src_client = client; + } + else + { + StorageAzureBlob::Configuration src_configuration = configuration; + src_configuration.container = blob_path[1]; + src_configuration.blob_path = blob_path[0]; + src_client = StorageAzureBlob::createClient(src_configuration, /* is_read_only */ false); + } + + LOG_TRACE(log, "Copying file {} from disk {} to AzureBlobStorag", src_path, src_disk->getName()); + copyAzureBlobStorageFile( + src_client, + client, + /* src_bucket */ blob_path[1], + /* src_key= */ blob_path[0], + start_pos, + length, + configuration.container, + fs::path(configuration.blob_path) / path_in_backup, + settings, + read_settings, + {}, + threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupWriterAzureBlobStorage")); + return; /// copied! + } + } + + /// Fallback to copy through buffers. + BackupWriterDefault::copyFileFromDisk(path_in_backup, src_disk, src_path, copy_encrypted, start_pos, length); +} + +void BackupWriterAzureBlobStorage::copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) +{ + copyDataToAzureBlobStorageFile(create_read_buffer, start_pos, length, client, configuration.container, path_in_backup, settings, {}, + threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupWriterAzureBlobStorage")); +} + +BackupWriterAzureBlobStorage::~BackupWriterAzureBlobStorage() = default; + +bool BackupWriterAzureBlobStorage::fileExists(const String & file_name) +{ + String key; + if (startsWith(file_name, ".")) + { + key= configuration.blob_path + file_name; + } + else + { + key = file_name; + } + LOG_INFO(&Poco::Logger::get("BackupWriterAzureBlobStorage"), "Result fileExists {} ", object_storage->exists(StoredObject(key))); + + return object_storage->exists(StoredObject(key)); +} + +UInt64 BackupWriterAzureBlobStorage::getFileSize(const String & file_name) +{ + LOG_INFO(&Poco::Logger::get("BackupWriterAzureBlobStorage"), "Enter getFileSize"); + String key; + if (startsWith(file_name, ".")) + { + key= configuration.blob_path + file_name; + } + else + { + key = file_name; + } + RelativePathsWithMetadata children; + object_storage->listObjects(key,children,/*max_keys*/0); + if (children.empty()) + throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Object {} must exist"); + return children[0].metadata.size_bytes; +} + +std::unique_ptr BackupWriterAzureBlobStorage::readFile(const String & file_name, size_t /*expected_file_size*/) +{ + String key; + if (startsWith(file_name, ".")) + { + key= configuration.blob_path + file_name; + } + else + { + key = file_name; + } + + return std::make_unique( + client, key, read_settings, settings->max_single_read_retries, + settings->max_single_download_retries); +} + +std::unique_ptr BackupWriterAzureBlobStorage::writeFile(const String & file_name) +{ + String key; + if (startsWith(file_name, ".")) + { + key= configuration.blob_path + file_name; + } + else + { + key = file_name; + } + return std::make_unique( + client, + key, + settings->max_single_part_upload_size, + DBMS_DEFAULT_BUFFER_SIZE, + write_settings); +} + +void BackupWriterAzureBlobStorage::removeFile(const String & file_name) +{ + String key; + if (startsWith(file_name, ".")) + { + key= configuration.blob_path + file_name; + } + else + { + key = file_name; + } + StoredObject object(key); + object_storage->removeObjectIfExists(object); +} + +void BackupWriterAzureBlobStorage::removeFiles(const Strings & keys) +{ + StoredObjects objects; + for (const auto & key : keys) + objects.emplace_back(key); + + object_storage->removeObjectsIfExist(objects); + +} + +void BackupWriterAzureBlobStorage::removeFilesBatch(const Strings & keys) +{ + StoredObjects objects; + for (const auto & key : keys) + objects.emplace_back(key); + + object_storage->removeObjectsIfExist(objects); +} + +} + +#endif diff --git a/src/Backups/BackupIO_AzureBlobStorage.h b/src/Backups/BackupIO_AzureBlobStorage.h new file mode 100644 index 00000000000..6ef66fc432d --- /dev/null +++ b/src/Backups/BackupIO_AzureBlobStorage.h @@ -0,0 +1,69 @@ +#pragma once + +#include "config.h" + +#if USE_AZURE_BLOB_STORAGE +#include +#include +#include +#include + + +namespace DB +{ + +// using AzureClientPtr = std::shared_ptr; + +/// Represents a backup stored to Azure + class BackupReaderAzureBlobStorage : public BackupReaderDefault + { + public: + BackupReaderAzureBlobStorage(StorageAzureBlob::Configuration configuration_, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_); + ~BackupReaderAzureBlobStorage() override; + + bool fileExists(const String & file_name) override; + UInt64 getFileSize(const String & file_name) override; + std::unique_ptr readFile(const String & file_name) override; + + void copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, + DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) override; + + private: + const DataSourceDescription data_source_description; + std::shared_ptr client; + StorageAzureBlob::Configuration configuration; + std::unique_ptr object_storage; + std::shared_ptr settings; + }; + + + class BackupWriterAzureBlobStorage : public BackupWriterDefault + { + public: + BackupWriterAzureBlobStorage(StorageAzureBlob::Configuration configuration_, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_); + ~BackupWriterAzureBlobStorage() override; + + bool fileExists(const String & file_name) override; + UInt64 getFileSize(const String & file_name) override; + std::unique_ptr writeFile(const String & file_name) override; + + void copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) override; + void copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, + bool copy_encrypted, UInt64 start_pos, UInt64 length) override; + + void removeFile(const String & file_name) override; + void removeFiles(const Strings & file_names) override; + + private: + std::unique_ptr readFile(const String & file_name, size_t expected_file_size) override; + void removeFilesBatch(const Strings & file_names); + const DataSourceDescription data_source_description; + std::shared_ptr client; + StorageAzureBlob::Configuration configuration; + std::unique_ptr object_storage; + std::shared_ptr settings; + }; + +} + +#endif diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index bb97335d8fb..9363ca5e7a7 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -492,6 +492,7 @@ void BackupImpl::checkBackupDoesntExist() const else file_name_to_check_existence = ".backup"; + LOG_INFO(&Poco::Logger::get("BackupImpl"), "checkBackupDoesntExist 1"); if (writer->fileExists(file_name_to_check_existence)) throw Exception(ErrorCodes::BACKUP_ALREADY_EXISTS, "Backup {} already exists", backup_name_for_logging); @@ -499,6 +500,7 @@ void BackupImpl::checkBackupDoesntExist() const if (!is_internal_backup) { assert(!lock_file_name.empty()); + LOG_INFO(&Poco::Logger::get("BackupImpl"), "checkBackupDoesntExist 2"); if (writer->fileExists(lock_file_name)) throw Exception(ErrorCodes::BACKUP_ALREADY_EXISTS, "Backup {} is being written already", backup_name_for_logging); } @@ -522,6 +524,8 @@ bool BackupImpl::checkLockFile(bool throw_if_failed) const if (throw_if_failed) { + LOG_INFO(&Poco::Logger::get("BackupImpl"), "checkLockFile"); + if (!writer->fileExists(lock_file_name)) { throw Exception( @@ -886,12 +890,12 @@ void BackupImpl::writeFile(const BackupFileInfo & info, BackupEntryPtr entry) } else if (src_disk && from_immutable_file) { - LOG_TRACE(log, "Writing backup for file {} from {} (disk {}): data file #{}", info.data_file_name, src_file_desc, src_disk->getName(), info.data_file_index); + LOG_INFO(log, "Writing backup for file {} from {} (disk {}): data file #{}", info.data_file_name, src_file_desc, src_disk->getName(), info.data_file_index); writer->copyFileFromDisk(info.data_file_name, src_disk, src_file_path, info.encrypted_by_disk, info.base_size, info.size - info.base_size); } else { - LOG_TRACE(log, "Writing backup for file {} from {}: data file #{}", info.data_file_name, src_file_desc, info.data_file_index); + LOG_INFO(log, "Writing backup for file {} from {}: data file #{}", info.data_file_name, src_file_desc, info.data_file_index); auto create_read_buffer = [entry, read_settings = writer->getReadSettings()] { return entry->getReadBuffer(read_settings); }; writer->copyDataToFile(info.data_file_name, create_read_buffer, info.base_size, info.size - info.base_size); } diff --git a/src/Backups/registerBackupEngineAzureBlobStorage.cpp b/src/Backups/registerBackupEngineAzureBlobStorage.cpp new file mode 100644 index 00000000000..6f7b5f38c28 --- /dev/null +++ b/src/Backups/registerBackupEngineAzureBlobStorage.cpp @@ -0,0 +1,134 @@ +#include "config.h" + +#include +#include + +#if USE_AZURE_BLOB_STORAGE +#include +#include +#include +#include +#include +#include +#include +#endif + + +namespace DB +{ +namespace fs = std::filesystem; + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int SUPPORT_IS_DISABLED; +} + +#if USE_AZURE_BLOB_STORAGE +namespace +{ + String removeFileNameFromURL(String & url) + { + Poco::URI url2{url}; + String path = url2.getPath(); + size_t slash_pos = path.find_last_of('/'); + String file_name = path.substr(slash_pos + 1); + path.resize(slash_pos + 1); + url2.setPath(path); + url = url2.toString(); + return file_name; + } +} +#endif + + +void registerBackupEngineAzureBlobStorage(BackupFactory & factory) +{ + auto creator_fn = []([[maybe_unused]] const BackupFactory::CreateParams & params) -> std::unique_ptr + { +#if USE_AZURE_BLOB_STORAGE + const String & id_arg = params.backup_info.id_arg; + const auto & args = params.backup_info.args; + + LOG_INFO(&Poco::Logger::get("registerBackupEngineAzureBlobStorage"), "Begin id_arg={} args.size={}", id_arg, args.size()); + + StorageAzureBlob::Configuration configuration; + + if (args.size() == 4) + { + configuration.connection_url = args[0].safeGet(); + configuration.is_connection_string = true; + + configuration.container = args[1].safeGet(); + configuration.blob_path = args[2].safeGet(); + configuration.format = args[3].safeGet(); + + LOG_TRACE(&Poco::Logger::get("registerBackupEngineAzureBlobStorage"), "configuration.connection_url = {}" + "configuration.container = {}" + "configuration.blob_path = {}" + "configuration.format = {}", + configuration.connection_url, configuration.container, configuration.blob_path, configuration.format); + } + + + BackupImpl::ArchiveParams archive_params; + if (hasRegisteredArchiveFileExtension(configuration.blob_path)) + { + if (params.is_internal_backup) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Using archives with backups on clusters is disabled"); + + archive_params.archive_name = removeFileNameFromURL(configuration.blob_path); + archive_params.compression_method = params.compression_method; + archive_params.compression_level = params.compression_level; + archive_params.password = params.password; + } + else + { + if (!params.password.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Password is not applicable, backup cannot be encrypted"); + } + + + if (params.open_mode == IBackup::OpenMode::READ) + { + auto reader = std::make_shared(configuration, + params.read_settings, + params.write_settings, + params.context); + + return std::make_unique( + params.backup_info, + archive_params, + params.base_backup_info, + reader, + params.context, + /*params.use_same_s3_credentials_for_base_backup*/ false); + } + else + { + auto writer = std::make_shared(configuration, + params.read_settings, + params.write_settings, + params.context); + + return std::make_unique( + params.backup_info, + archive_params, + params.base_backup_info, + writer, + params.context, + params.is_internal_backup, + params.backup_coordination, + params.backup_uuid, + params.deduplicate_files, + /*params.use_same_s3_credentials_for_base_backup*/ false); + } +#else + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "AzureBlobStorage support is disabled"); +#endif + }; + + factory.registerBackupEngine("AzureBlobStorage", creator_fn); +} + +} diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 0257b7d329b..984594a6541 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -87,6 +87,7 @@ add_headers_and_sources(clickhouse_common_io IO) add_headers_and_sources(clickhouse_common_io IO/Archives) add_headers_and_sources(clickhouse_common_io IO/Resource) add_headers_and_sources(clickhouse_common_io IO/S3) +add_headers_and_sources(clickhouse_common_io IO/AzureBlobStorage) list (REMOVE_ITEM clickhouse_common_io_sources Common/malloc.cpp Common/new_delete.cpp) @@ -139,6 +140,7 @@ endif() if (TARGET ch_contrib::azure_sdk) add_headers_and_sources(dbms Disks/ObjectStorages/AzureBlobStorage) + add_headers_and_sources(dbms IO/AzureBlobStorage) endif() if (TARGET ch_contrib::hdfs) @@ -485,6 +487,7 @@ if (TARGET ch_contrib::aws_s3) endif() if (TARGET ch_contrib::azure_sdk) + target_link_libraries (clickhouse_common_io PUBLIC ch_contrib::azure_sdk) dbms_target_link_libraries (PRIVATE ch_contrib::azure_sdk) endif() diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 58e860ebcaf..1655d19986a 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -361,6 +361,10 @@ The server successfully detected this situation and will download merged part fr M(S3PutObject, "Number of S3 API PutObject calls.") \ M(S3GetObject, "Number of S3 API GetObject calls.") \ \ + M(AzureUploadPart, "Number of Azure blob storage API UploadPart calls") \ + M(DiskAzureUploadPart, "Number of Disk Azure blob storage API UploadPart calls") \ + M(AzureCopyObject, "Number of Azure blob storage API CopyObject calls") \ + M(DiskAzureCopyObject, "Number of Disk Azure blob storage API CopyObject calls") \ M(AzureDeleteObjects, "Number of Azure blob storage API DeleteObject(s) calls.") \ M(AzureListObjects, "Number of Azure blob storage API ListObjects calls.") \ \ diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp new file mode 100644 index 00000000000..bf0bcac664b --- /dev/null +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp @@ -0,0 +1,324 @@ +#include + +#if USE_AZURE_BLOB_STORAGE + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace ProfileEvents +{ + extern const Event AzureCopyObject; + extern const Event AzureUploadPart; + + extern const Event DiskAzureCopyObject; + extern const Event DiskAzureUploadPart; +} + + +namespace DB +{ + +size_t max_single_operation_copy_size = 256 * 1024 * 1024; + + +namespace +{ + class UploadHelper + { + public: + UploadHelper( + const CreateReadBuffer & create_read_buffer_, + std::shared_ptr client_, + size_t offset_, + size_t total_size_, + const String & dest_bucket_, + const String & dest_key_, + std::shared_ptr settings_, + const std::optional> & object_metadata_, + ThreadPoolCallbackRunner schedule_, + bool for_disk_azure_blob_storage_) + : create_read_buffer(create_read_buffer_) + , client(client_) + , offset (offset_) + , total_size (total_size_) + , dest_bucket(dest_bucket_) + , dest_key(dest_key_) + , settings(settings_) + , object_metadata(object_metadata_) + , schedule(schedule_) + , for_disk_azure_blob_storage(for_disk_azure_blob_storage_) + , log(&Poco::Logger::get("azureBlobStorageUploadHelper")) + , max_single_part_upload_size(settings_.get()->max_single_part_upload_size) + { + } + + ~UploadHelper() {} + + protected: + std::function()> create_read_buffer; + std::shared_ptr client; + size_t offset; + size_t total_size; + const String & dest_bucket; + const String & dest_key; + std::shared_ptr settings; + const std::optional> & object_metadata; + ThreadPoolCallbackRunner schedule; + bool for_disk_azure_blob_storage; + const Poco::Logger * log; + size_t max_single_part_upload_size; + + struct UploadPartTask + { + char *data = nullptr; + size_t size = 0; + std::string block_id; + bool is_finished = false; + std::exception_ptr exception; + + ~UploadPartTask() + { + if (data != nullptr) + free(data); + } + }; + + size_t normal_part_size; + std::vector block_ids; + + std::list TSA_GUARDED_BY(bg_tasks_mutex) bg_tasks; + int num_added_bg_tasks TSA_GUARDED_BY(bg_tasks_mutex) = 0; + int num_finished_bg_tasks TSA_GUARDED_BY(bg_tasks_mutex) = 0; + std::mutex bg_tasks_mutex; + std::condition_variable bg_tasks_condvar; + + public: + void performCopy() + { + performMultipartUpload(); + } + + void completeMultipartUpload() + { + auto block_blob_client = client->GetBlockBlobClient(dest_key); + block_blob_client.CommitBlockList(block_ids); + } + + void performMultipartUpload() + { + normal_part_size = 1024; + + size_t position = offset; + size_t end_position = offset + total_size; + + try + { + while (position < end_position) + { + size_t next_position = std::min(position + normal_part_size, end_position); + size_t part_size = next_position - position; /// `part_size` is either `normal_part_size` or smaller if it's the final part. + + uploadPart(position, part_size); + + position = next_position; + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + waitForAllBackgroundTasks(); + throw; + } + + waitForAllBackgroundTasks(); + completeMultipartUpload(); + } + + + void uploadPart(size_t part_offset, size_t part_size) + { + LOG_TRACE(log, "Writing part. Bucket: {}, Key: {}, Size: {}", dest_bucket, dest_key, part_size); + + if (!part_size) + { + LOG_TRACE(log, "Skipping writing an empty part."); + return; + } + + if (schedule) + { + UploadPartTask * task = nullptr; + + { + std::lock_guard lock(bg_tasks_mutex); + task = &bg_tasks.emplace_back(); + ++num_added_bg_tasks; + } + + /// Notify waiting thread when task finished + auto task_finish_notify = [this, task]() + { + std::lock_guard lock(bg_tasks_mutex); + task->is_finished = true; + ++num_finished_bg_tasks; + + /// Notification under mutex is important here. + /// Otherwise, WriteBuffer could be destroyed in between + /// Releasing lock and condvar notification. + bg_tasks_condvar.notify_one(); + }; + + try + { + auto read_buffer = std::make_unique(create_read_buffer(), part_offset, part_size); + auto buffer = std::make_unique(std::move(read_buffer), part_size); + task->data = new char[part_size]; + task->size = part_size; + buffer->read(task->data,part_size); + task->block_id = getRandomASCIIString(64); + + schedule([this, task, task_finish_notify]() + { + try + { + processUploadTask(*task); + } + catch (...) + { + task->exception = std::current_exception(); + } + task_finish_notify(); + }, Priority{}); + } + catch (...) + { + task_finish_notify(); + throw; + } + } + else + { + UploadPartTask task; + auto read_buffer = std::make_unique(create_read_buffer(), part_offset, part_size); + auto buffer = std::make_unique(std::move(read_buffer), part_size); + task.data = new char[part_size]; + buffer->read(task.data,part_size); + task.size = part_size; + processUploadTask(task); + block_ids.emplace_back(task.block_id); + } + } + + void processUploadTask(UploadPartTask & task) + { + auto block_id = processUploadPartRequest(task); + + std::lock_guard lock(bg_tasks_mutex); /// Protect bg_tasks from race + task.block_id = block_id; + LOG_TRACE(log, "Writing part finished. Bucket: {}, Key: {}, block_id: {}, Parts: {}", dest_bucket, dest_key, block_id, bg_tasks.size()); + } + + String processUploadPartRequest(UploadPartTask & task) + { + ProfileEvents::increment(ProfileEvents::AzureUploadPart); + if (for_disk_azure_blob_storage) + ProfileEvents::increment(ProfileEvents::DiskAzureUploadPart); + + auto block_blob_client = client->GetBlockBlobClient(dest_key); + task.block_id = getRandomASCIIString(64); + Azure::Core::IO::MemoryBodyStream memory(reinterpret_cast(task.data), task.size); + block_blob_client.StageBlock(task.block_id, memory); + + return task.block_id; + } + + + void waitForAllBackgroundTasks() + { + if (!schedule) + return; + + std::unique_lock lock(bg_tasks_mutex); + /// Suppress warnings because bg_tasks_mutex is actually hold, but tsa annotations do not understand std::unique_lock + bg_tasks_condvar.wait(lock, [this]() {return TSA_SUPPRESS_WARNING_FOR_READ(num_added_bg_tasks) == TSA_SUPPRESS_WARNING_FOR_READ(num_finished_bg_tasks); }); + + auto & tasks = TSA_SUPPRESS_WARNING_FOR_WRITE(bg_tasks); + for (auto & task : tasks) + { + if (task.exception) + std::rethrow_exception(task.exception); + block_ids.emplace_back(task.block_id); + } + } + }; +} + + +void copyDataToAzureBlobStorageFile( + const std::function()> & create_read_buffer, + size_t offset, + size_t size, + std::shared_ptr & dest_client, + const String & dest_bucket, + const String & dest_key, + std::shared_ptr settings, + const std::optional> & object_metadata, + ThreadPoolCallbackRunner schedule, + bool for_disk_azure_blob_storage) +{ + UploadHelper helper{create_read_buffer, dest_client, offset, size, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_azure_blob_storage}; + helper.performCopy(); +} + + +void copyAzureBlobStorageFile( + std::shared_ptr src_client, + std::shared_ptr dest_client, + const String & src_bucket, + const String & src_key, + size_t offset, + size_t size, + const String & dest_bucket, + const String & dest_key, + std::shared_ptr settings, + const ReadSettings & read_settings, + const std::optional> & object_metadata, + ThreadPoolCallbackRunner schedule, + bool for_disk_azure_blob_storage) +{ + + if (size < max_single_operation_copy_size) + { + ProfileEvents::increment(ProfileEvents::AzureCopyObject); + if (for_disk_azure_blob_storage) + ProfileEvents::increment(ProfileEvents::DiskAzureCopyObject); + auto block_blob_client_src = src_client->GetBlockBlobClient(src_key); + auto block_blob_client_dest = dest_client->GetBlockBlobClient(dest_key); + auto uri = block_blob_client_src.GetUrl(); + block_blob_client_dest.CopyFromUri(uri); + } + else + { + LOG_TRACE(&Poco::Logger::get("copyAzureBlobStorageFile"), "Reading from Bucket: {}, Key: {}", src_bucket, src_key); + auto create_read_buffer = [&] + { + return std::make_unique(src_client, src_key, read_settings, settings->max_single_read_retries, + settings->max_single_download_retries); + }; + + UploadHelper helper{create_read_buffer, dest_client, offset, size, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_azure_blob_storage}; + helper.performCopy(); + } +} + +} + +#endif diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h new file mode 100644 index 00000000000..31228fbcb23 --- /dev/null +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h @@ -0,0 +1,58 @@ +#pragma once + +#include "config.h" + +#if USE_AZURE_BLOB_STORAGE + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +class SeekableReadBuffer; + +using CreateReadBuffer = std::function()>; + +/// Copies a file from AzureBlobStorage to AzureBlobStorage. +/// The parameters `src_offset` and `src_size` specify a part in the source to copy. +void copyAzureBlobStorageFile( + std::shared_ptr src_client, + std::shared_ptr dest_client, + const String & src_bucket, + const String & src_key, + size_t src_offset, + size_t src_size, + const String & dest_bucket, + const String & dest_key, + std::shared_ptr settings, + const ReadSettings & read_settings, + const std::optional> & object_metadata = std::nullopt, + ThreadPoolCallbackRunner schedule_ = {}, + bool for_disk_azure_blob_storage = false); + + +/// Copies data from any seekable source to AzureBlobStorage. +/// The same functionality can be done by using the function copyData() and the class WriteBufferFromS3 +/// however copyDataToS3File() is faster and spends less memory. +/// The callback `create_read_buffer` can be called from multiple threads in parallel, so that should be thread-safe. +/// The parameters `offset` and `size` specify a part in the source to copy. +void copyDataToAzureBlobStorageFile( + const std::function()> & create_read_buffer, + size_t offset, + size_t size, + std::shared_ptr & client, + const String & dest_bucket, + const String & dest_key, + std::shared_ptr settings, + const std::optional> & object_metadata = std::nullopt, + ThreadPoolCallbackRunner schedule_ = {}, + bool for_disk_azure_blob_storage = false); + +} + +#endif diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 2e0703a8df3..e36604cfb1a 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -258,6 +258,17 @@ AzureObjectStorage::SettingsPtr StorageAzureBlob::createSettings(ContextPtr loca return settings_ptr; } +std::shared_ptr StorageAzureBlob::createSettingsAsSharedPtr(ContextPtr local_context) +{ + const auto & context_settings = local_context->getSettingsRef(); + auto settings_ptr = std::make_shared(); + settings_ptr->max_single_part_upload_size = context_settings.azure_max_single_part_upload_size; + settings_ptr->max_single_read_retries = context_settings.azure_max_single_read_retries; + settings_ptr->list_object_keys_size = static_cast(context_settings.azure_list_object_keys_size); + + return settings_ptr; +} + void registerStorageAzureBlob(StorageFactory & factory) { factory.registerStorage("AzureBlobStorage", [](const StorageFactory::Arguments & args) diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h index b97dee0caed..570e4124d73 100644 --- a/src/Storages/StorageAzureBlob.h +++ b/src/Storages/StorageAzureBlob.h @@ -80,6 +80,7 @@ public: static AzureClientPtr createClient(StorageAzureBlob::Configuration configuration, bool is_read_only); static AzureObjectStorage::SettingsPtr createSettings(ContextPtr local_context); + static std::shared_ptr createSettingsAsSharedPtr(ContextPtr local_context); static void processNamedCollectionResult(StorageAzureBlob::Configuration & configuration, const NamedCollection & collection); diff --git a/tests/integration/test_backup_restore_azure_blob_storage/__init__.py b/tests/integration/test_backup_restore_azure_blob_storage/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_backup_restore_azure_blob_storage/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_backup_restore_azure_blob_storage/configs/config.xml b/tests/integration/test_backup_restore_azure_blob_storage/configs/config.xml new file mode 100644 index 00000000000..5725dce40cd --- /dev/null +++ b/tests/integration/test_backup_restore_azure_blob_storage/configs/config.xml @@ -0,0 +1,11 @@ + + 1 + 0 + 0.0 + 0 + 1 + 1 + 0 + 16 + 16 + \ No newline at end of file diff --git a/tests/integration/test_backup_restore_azure_blob_storage/configs/disable_profilers.xml b/tests/integration/test_backup_restore_azure_blob_storage/configs/disable_profilers.xml new file mode 100644 index 00000000000..b74bb1502ce --- /dev/null +++ b/tests/integration/test_backup_restore_azure_blob_storage/configs/disable_profilers.xml @@ -0,0 +1,13 @@ + + + + + 0 + 0 + 0 + 1000 + 1 + 1 + + + diff --git a/tests/integration/test_backup_restore_azure_blob_storage/configs/users.xml b/tests/integration/test_backup_restore_azure_blob_storage/configs/users.xml new file mode 100644 index 00000000000..c12eb2f79f4 --- /dev/null +++ b/tests/integration/test_backup_restore_azure_blob_storage/configs/users.xml @@ -0,0 +1,8 @@ + + + + + default + + + diff --git a/tests/integration/test_backup_restore_azure_blob_storage/test.py b/tests/integration/test_backup_restore_azure_blob_storage/test.py new file mode 100644 index 00000000000..2ecf08a4f40 --- /dev/null +++ b/tests/integration/test_backup_restore_azure_blob_storage/test.py @@ -0,0 +1,151 @@ +#!/usr/bin/env python3 + +import gzip +import json +import logging +import os +import io +import random +import threading +import time + +from azure.storage.blob import BlobServiceClient +import helpers.client +import pytest +from helpers.cluster import ClickHouseCluster, ClickHouseInstance +from helpers.network import PartitionManager +from helpers.mock_servers import start_mock_servers +from helpers.test_tools import exec_query_with_retry + + + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "node", + main_configs=["configs/config.xml"], + user_configs=["configs/disable_profilers.xml", "configs/users.xml"], + with_azurite=True, + ) + cluster.start() + + yield cluster + finally: + cluster.shutdown() + + +def azure_query( + node, query, expect_error="false", try_num=10, settings={}, query_on_retry=None +): + for i in range(try_num): + try: + if expect_error == "true": + return node.query_and_get_error(query, settings=settings) + else: + return node.query(query, settings=settings) + except Exception as ex: + retriable_errors = [ + "DB::Exception: Azure::Core::Http::TransportException: Connection was closed by the server while trying to read a response", + "DB::Exception: Azure::Core::Http::TransportException: Connection closed before getting full response or response is less than expected", + "DB::Exception: Azure::Core::Http::TransportException: Connection was closed by the server while trying to read a response", + "DB::Exception: Azure::Core::Http::TransportException: Error while polling for socket ready read", + "Azure::Core::Http::TransportException, e.what() = Connection was closed by the server while trying to read a response", + "Azure::Core::Http::TransportException, e.what() = Connection closed before getting full response or response is less than expected", + "Azure::Core::Http::TransportException, e.what() = Connection was closed by the server while trying to read a response", + "Azure::Core::Http::TransportException, e.what() = Error while polling for socket ready read", + ] + retry = False + for error in retriable_errors: + if error in str(ex): + retry = True + print(f"Try num: {i}. Having retriable error: {ex}") + time.sleep(i) + break + if not retry or i == try_num - 1: + raise Exception(ex) + if query_on_retry is not None: + node.query(query_on_retry) + continue + + +def get_azure_file_content(filename, port): + container_name = "cont" + connection_string = ( + f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" + f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" + f"BlobEndpoint=http://127.0.0.1:{port}/devstoreaccount1;" + ) + blob_service_client = BlobServiceClient.from_connection_string( + str(connection_string) + ) + container_client = blob_service_client.get_container_client(container_name) + blob_client = container_client.get_blob_client(filename) + download_stream = blob_client.download_blob() + return download_stream.readall().decode("utf-8") + + +def put_azure_file_content(filename, port, data): + container_name = "cont" + connection_string = ( + f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" + f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" + f"BlobEndpoint=http://127.0.0.1:{port}/devstoreaccount1;" + ) + blob_service_client = BlobServiceClient.from_connection_string(connection_string) + try: + container_client = blob_service_client.create_container(container_name) + except: + container_client = blob_service_client.get_container_client(container_name) + + blob_client = container_client.get_blob_client(filename) + buf = io.BytesIO(data) + blob_client.upload_blob(buf) + +@pytest.fixture(autouse=True, scope="function") +def delete_all_files(cluster): + port = cluster.env_variables["AZURITE_PORT"] + connection_string = ( + f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" + f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" + f"BlobEndpoint=http://127.0.0.1:{port}/devstoreaccount1;" + ) + blob_service_client = BlobServiceClient.from_connection_string(connection_string) + containers = blob_service_client.list_containers() + for container in containers: + container_client = blob_service_client.get_container_client(container) + blob_list = container_client.list_blobs() + for blob in blob_list: + print(blob) + blob_client = container_client.get_blob_client(blob) + blob_client.delete_blob() + + assert len(list(container_client.list_blobs())) == 0 + + yield + + +def test_create_table_connection_string(cluster): + node = cluster.instances["node"] + azure_query( + node, + f"CREATE TABLE test_create_table_conn_string (key UInt64, data String) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont', 'test_create_connection_string', 'CSV')", + ) + +def test_backup_restore(cluster): + node = cluster.instances["node"] + port = cluster.env_variables["AZURITE_PORT"] + azure_query( + node, + f"CREATE TABLE test_simple_write_connection_string (key UInt64, data String) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont', 'test_simple_write_c.csv', 'CSV')", + ) + azure_query(node, f"INSERT INTO test_simple_write_connection_string VALUES (1, 'a')") + print(get_azure_file_content("test_simple_write_c.csv", port)) + assert get_azure_file_content("test_simple_write_c.csv", port) == '1,"a"\n' + + backup_destination = f"AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont', 'test_simple_write_c_backup.csv', 'CSV')" + azure_query(node,f"BACKUP TABLE test_simple_write_connection_string TO {backup_destination}") + print (get_azure_file_content("test_simple_write_c_backup.csv.backup", port)) + azure_query(node, f"RESTORE TABLE test_simple_write_connection_string AS test_simple_write_connection_string_restored FROM {backup_destination};") + assert(azure_query(node,f"SELECT * from test_simple_write_connection_string_restored") == "1\ta\n") \ No newline at end of file From 05b608cd76da8995086887f812e1ab3fceb99551 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 20 Nov 2023 10:12:45 +0000 Subject: [PATCH 002/276] Automatic style fix --- .../test.py | 24 ++++++++++++++----- 1 file changed, 18 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_backup_restore_azure_blob_storage/test.py b/tests/integration/test_backup_restore_azure_blob_storage/test.py index 2ecf08a4f40..cda3cab07e4 100644 --- a/tests/integration/test_backup_restore_azure_blob_storage/test.py +++ b/tests/integration/test_backup_restore_azure_blob_storage/test.py @@ -18,7 +18,6 @@ from helpers.mock_servers import start_mock_servers from helpers.test_tools import exec_query_with_retry - @pytest.fixture(scope="module") def cluster(): try: @@ -103,6 +102,7 @@ def put_azure_file_content(filename, port, data): buf = io.BytesIO(data) blob_client.upload_blob(buf) + @pytest.fixture(autouse=True, scope="function") def delete_all_files(cluster): port = cluster.env_variables["AZURITE_PORT"] @@ -133,6 +133,7 @@ def test_create_table_connection_string(cluster): f"CREATE TABLE test_create_table_conn_string (key UInt64, data String) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont', 'test_create_connection_string', 'CSV')", ) + def test_backup_restore(cluster): node = cluster.instances["node"] port = cluster.env_variables["AZURITE_PORT"] @@ -140,12 +141,23 @@ def test_backup_restore(cluster): node, f"CREATE TABLE test_simple_write_connection_string (key UInt64, data String) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont', 'test_simple_write_c.csv', 'CSV')", ) - azure_query(node, f"INSERT INTO test_simple_write_connection_string VALUES (1, 'a')") + azure_query( + node, f"INSERT INTO test_simple_write_connection_string VALUES (1, 'a')" + ) print(get_azure_file_content("test_simple_write_c.csv", port)) assert get_azure_file_content("test_simple_write_c.csv", port) == '1,"a"\n' backup_destination = f"AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont', 'test_simple_write_c_backup.csv', 'CSV')" - azure_query(node,f"BACKUP TABLE test_simple_write_connection_string TO {backup_destination}") - print (get_azure_file_content("test_simple_write_c_backup.csv.backup", port)) - azure_query(node, f"RESTORE TABLE test_simple_write_connection_string AS test_simple_write_connection_string_restored FROM {backup_destination};") - assert(azure_query(node,f"SELECT * from test_simple_write_connection_string_restored") == "1\ta\n") \ No newline at end of file + azure_query( + node, + f"BACKUP TABLE test_simple_write_connection_string TO {backup_destination}", + ) + print(get_azure_file_content("test_simple_write_c_backup.csv.backup", port)) + azure_query( + node, + f"RESTORE TABLE test_simple_write_connection_string AS test_simple_write_connection_string_restored FROM {backup_destination};", + ) + assert ( + azure_query(node, f"SELECT * from test_simple_write_connection_string_restored") + == "1\ta\n" + ) From 6dfb1c25ec6a4a61a4fe329191c10263eb19ad07 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 20 Nov 2023 11:37:06 +0100 Subject: [PATCH 003/276] Added docs --- docs/en/operations/backup.md | 21 +++++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/docs/en/operations/backup.md b/docs/en/operations/backup.md index 6068b185ede..15d953249a0 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -451,3 +451,24 @@ To disallow concurrent backup/restore, you can use these settings respectively. The default value for both is true, so by default concurrent backup/restores are allowed. When these settings are false on a cluster, only 1 backup/restore is allowed to run on a cluster at a time. + +## Configuring BACKUP/RESTORE to use an AzureBlobStorage Endpoint + +To write backups to an AzureBlobStorage container you need the following pieces of information: +- AzureBlobStorage endpoint connection string / url, +- Container, +- Path, +- Account name (if url is specified) +- Account Key (if url is specified) + +The destination for a backup will be specified like this: +``` +AzureBlobStorage('/', '', '', '', ') +``` + +```sql +BACKUP TABLE data TO AzureBlobStorage('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1/;', + 'test_container', 'data_backup'); +RESTORE TABLE data AS data_restored FROM AzureBlobStorage('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1/;', + 'test_container', 'data_backup'); +``` From d0827e3ea77ff432c4a6a66145827428bcd62b5e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 1 Dec 2023 17:45:23 +0000 Subject: [PATCH 004/276] Add a test. --- .../0_stateless/02932_set_ttl_where.reference | 0 .../0_stateless/02932_set_ttl_where.sql | 22 +++++++++++++++++++ 2 files changed, 22 insertions(+) create mode 100644 tests/queries/0_stateless/02932_set_ttl_where.reference create mode 100644 tests/queries/0_stateless/02932_set_ttl_where.sql diff --git a/tests/queries/0_stateless/02932_set_ttl_where.reference b/tests/queries/0_stateless/02932_set_ttl_where.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02932_set_ttl_where.sql b/tests/queries/0_stateless/02932_set_ttl_where.sql new file mode 100644 index 00000000000..85fddf613e8 --- /dev/null +++ b/tests/queries/0_stateless/02932_set_ttl_where.sql @@ -0,0 +1,22 @@ +create or replace table temp ( + a UInt32 +) +engine = MergeTree +order by a; + +insert into temp select number from system.numbers limit 100_000; + +create or replace table t_temp ( + a UInt32, + timestamp DateTime +) +engine = MergeTree +order by a +TTL timestamp + INTERVAL 2 SECOND WHERE a in (select a from temp); + +select sleep(1); +insert into t_temp select rand(), now() from system.numbers limit 1_000_000; +select sleep(1); +insert into t_temp select rand(), now() from system.numbers limit 1_000_000; +select sleep(1); +optimize table t_temp final; From 508046e6922c0cb163ce5611f1e6ef6a22f8b7f1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 1 Dec 2023 20:31:26 +0000 Subject: [PATCH 005/276] Attempt to support subqueries in TTL. --- src/Interpreters/PreparedSets.cpp | 8 ++- src/Interpreters/PreparedSets.h | 1 + src/Processors/TTL/ITTLAlgorithm.cpp | 5 +- src/Processors/TTL/ITTLAlgorithm.h | 9 ++- .../TTL/TTLAggregationAlgorithm.cpp | 11 ++-- src/Processors/TTL/TTLAggregationAlgorithm.h | 1 + src/Processors/TTL/TTLColumnAlgorithm.cpp | 5 +- src/Processors/TTL/TTLColumnAlgorithm.h | 1 + src/Processors/TTL/TTLDeleteAlgorithm.cpp | 10 +-- src/Processors/TTL/TTLDeleteAlgorithm.h | 2 +- src/Processors/TTL/TTLUpdateInfoAlgorithm.cpp | 5 +- src/Processors/TTL/TTLUpdateInfoAlgorithm.h | 1 + src/Processors/Transforms/TTLTransform.cpp | 2 +- .../MergeTree/MergeTreeDataWriter.cpp | 19 +++--- src/Storages/StorageInMemoryMetadata.cpp | 21 +++---- src/Storages/TTLDescription.cpp | 62 ++++++++++++------- src/Storages/TTLDescription.h | 15 ++++- 17 files changed, 116 insertions(+), 62 deletions(-) diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 955d8892284..ea8d9a62b8b 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -189,11 +189,17 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) } } + set_and_key->set->fillSetElements(); + + return buildSetInplace(context); +} + +SetPtr FutureSetFromSubquery::buildSetInplace(const ContextPtr & context) +{ auto plan = build(context); if (!plan) return nullptr; - set_and_key->set->fillSetElements(); auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); pipeline.complete(std::make_shared(Block())); diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index e237789c63c..3e751d309ba 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -107,6 +107,7 @@ public: SetPtr get() const override; DataTypes getTypes() const override; SetPtr buildOrderedSetInplace(const ContextPtr & context) override; + SetPtr buildSetInplace(const ContextPtr & context); std::unique_ptr build(const ContextPtr & context); diff --git a/src/Processors/TTL/ITTLAlgorithm.cpp b/src/Processors/TTL/ITTLAlgorithm.cpp index 79140137df8..af6c4e4ac35 100644 --- a/src/Processors/TTL/ITTLAlgorithm.cpp +++ b/src/Processors/TTL/ITTLAlgorithm.cpp @@ -11,8 +11,9 @@ namespace ErrorCodes } ITTLAlgorithm::ITTLAlgorithm( - const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_) - : description(description_) + const TTlExpressions & ttl_expressions_, const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_) + : ttl_expressions(ttl_expressions_) + , description(description_) , old_ttl_info(old_ttl_info_) , current_time(current_time_) , force(force_) diff --git a/src/Processors/TTL/ITTLAlgorithm.h b/src/Processors/TTL/ITTLAlgorithm.h index 49cd2c46d9d..6e73286b564 100644 --- a/src/Processors/TTL/ITTLAlgorithm.h +++ b/src/Processors/TTL/ITTLAlgorithm.h @@ -8,6 +8,12 @@ namespace DB { +struct TTlExpressions +{ + ExpressionActionsPtr expression; + ExpressionActionsPtr where_expression; +}; + /** * Represents the actions, which are required to do * with data, when TTL is expired: delete, aggregate, etc. @@ -18,7 +24,7 @@ public: using TTLInfo = IMergeTreeDataPart::TTLInfo; using MutableDataPartPtr = MergeTreeMutableDataPartPtr; - ITTLAlgorithm(const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_); + ITTLAlgorithm(const TTlExpressions & ttl_expressions_, const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_); virtual ~ITTLAlgorithm() = default; virtual void execute(Block & block) = 0; @@ -39,6 +45,7 @@ protected: bool isTTLExpired(time_t ttl) const; UInt32 getTimestampByIndex(const IColumn * column, size_t index) const; + const TTlExpressions ttl_expressions; const TTLDescription description; const TTLInfo old_ttl_info; const time_t current_time; diff --git a/src/Processors/TTL/TTLAggregationAlgorithm.cpp b/src/Processors/TTL/TTLAggregationAlgorithm.cpp index fa3436ec55d..ab2ba5f58fc 100644 --- a/src/Processors/TTL/TTLAggregationAlgorithm.cpp +++ b/src/Processors/TTL/TTLAggregationAlgorithm.cpp @@ -5,13 +5,14 @@ namespace DB { TTLAggregationAlgorithm::TTLAggregationAlgorithm( + const TTlExpressions & ttl_expressions_, const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_, const Block & header_, const MergeTreeData & storage_) - : ITTLAlgorithm(description_, old_ttl_info_, current_time_, force_) + : ITTLAlgorithm(ttl_expressions_, description_, old_ttl_info_, current_time_, force_) , header(header_) { current_key_value.resize(description.group_by_keys.size()); @@ -73,8 +74,8 @@ void TTLAggregationAlgorithm::execute(Block & block) const auto & column_names = header.getNames(); MutableColumns aggregate_columns = header.cloneEmptyColumns(); - auto ttl_column = executeExpressionAndGetColumn(description.expression, block, description.result_column); - auto where_column = executeExpressionAndGetColumn(description.where_expression, block, description.where_result_column); + auto ttl_column = executeExpressionAndGetColumn(ttl_expressions.expression, block, description.result_column); + auto where_column = executeExpressionAndGetColumn(ttl_expressions.where_expression, block, description.where_result_column); size_t rows_aggregated = 0; size_t current_key_start = 0; @@ -145,8 +146,8 @@ void TTLAggregationAlgorithm::execute(Block & block) /// If some rows were aggregated we have to recalculate ttl info's if (some_rows_were_aggregated) { - auto ttl_column_after_aggregation = executeExpressionAndGetColumn(description.expression, block, description.result_column); - auto where_column_after_aggregation = executeExpressionAndGetColumn(description.where_expression, block, description.where_result_column); + auto ttl_column_after_aggregation = executeExpressionAndGetColumn(ttl_expressions.expression, block, description.result_column); + auto where_column_after_aggregation = executeExpressionAndGetColumn(ttl_expressions.where_expression, block, description.where_result_column); for (size_t i = 0; i < block.rows(); ++i) { bool where_filter_passed = !where_column_after_aggregation || where_column_after_aggregation->getBool(i); diff --git a/src/Processors/TTL/TTLAggregationAlgorithm.h b/src/Processors/TTL/TTLAggregationAlgorithm.h index 0e4bf092ed6..9fd074efba8 100644 --- a/src/Processors/TTL/TTLAggregationAlgorithm.h +++ b/src/Processors/TTL/TTLAggregationAlgorithm.h @@ -13,6 +13,7 @@ class TTLAggregationAlgorithm final : public ITTLAlgorithm { public: TTLAggregationAlgorithm( + const TTlExpressions & ttl_expressions_, const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, diff --git a/src/Processors/TTL/TTLColumnAlgorithm.cpp b/src/Processors/TTL/TTLColumnAlgorithm.cpp index 04c4d7b9348..cb99dcf99b1 100644 --- a/src/Processors/TTL/TTLColumnAlgorithm.cpp +++ b/src/Processors/TTL/TTLColumnAlgorithm.cpp @@ -4,6 +4,7 @@ namespace DB { TTLColumnAlgorithm::TTLColumnAlgorithm( + const TTlExpressions & ttl_expressions_, const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, @@ -12,7 +13,7 @@ TTLColumnAlgorithm::TTLColumnAlgorithm( const ExpressionActionsPtr & default_expression_, const String & default_column_name_, bool is_compact_part_) - : ITTLAlgorithm(description_, old_ttl_info_, current_time_, force_) + : ITTLAlgorithm(ttl_expressions_, description_, old_ttl_info_, current_time_, force_) , column_name(column_name_) , default_expression(default_expression_) , default_column_name(default_column_name_) @@ -49,7 +50,7 @@ void TTLColumnAlgorithm::execute(Block & block) if (default_column) default_column = default_column->convertToFullColumnIfConst(); - auto ttl_column = executeExpressionAndGetColumn(description.expression, block, description.result_column); + auto ttl_column = executeExpressionAndGetColumn(ttl_expressions.expression, block, description.result_column); auto & column_with_type = block.getByName(column_name); const IColumn * values_column = column_with_type.column.get(); diff --git a/src/Processors/TTL/TTLColumnAlgorithm.h b/src/Processors/TTL/TTLColumnAlgorithm.h index 30de77dcc2a..efcd7c74454 100644 --- a/src/Processors/TTL/TTLColumnAlgorithm.h +++ b/src/Processors/TTL/TTLColumnAlgorithm.h @@ -11,6 +11,7 @@ class TTLColumnAlgorithm final : public ITTLAlgorithm { public: TTLColumnAlgorithm( + const TTlExpressions & ttl_expressions_, const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, diff --git a/src/Processors/TTL/TTLDeleteAlgorithm.cpp b/src/Processors/TTL/TTLDeleteAlgorithm.cpp index f176df2d003..6a172e9c3c3 100644 --- a/src/Processors/TTL/TTLDeleteAlgorithm.cpp +++ b/src/Processors/TTL/TTLDeleteAlgorithm.cpp @@ -4,8 +4,8 @@ namespace DB { TTLDeleteAlgorithm::TTLDeleteAlgorithm( - const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_) - : ITTLAlgorithm(description_, old_ttl_info_, current_time_, force_) + const TTlExpressions & ttl_expressions_, const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_) + : ITTLAlgorithm(ttl_expressions_, description_, old_ttl_info_, current_time_, force_) { if (!isMinTTLExpired()) new_ttl_info = old_ttl_info; @@ -19,8 +19,8 @@ void TTLDeleteAlgorithm::execute(Block & block) if (!block || !isMinTTLExpired()) return; - auto ttl_column = executeExpressionAndGetColumn(description.expression, block, description.result_column); - auto where_column = executeExpressionAndGetColumn(description.where_expression, block, description.where_result_column); + auto ttl_column = executeExpressionAndGetColumn(ttl_expressions.expression, block, description.result_column); + auto where_column = executeExpressionAndGetColumn(ttl_expressions.where_expression, block, description.where_result_column); MutableColumns result_columns; const auto & column_names = block.getNames(); @@ -54,7 +54,7 @@ void TTLDeleteAlgorithm::execute(Block & block) void TTLDeleteAlgorithm::finalize(const MutableDataPartPtr & data_part) const { - if (description.where_expression) + if (ttl_expressions.where_expression) data_part->ttl_infos.rows_where_ttl[description.result_column] = new_ttl_info; else data_part->ttl_infos.table_ttl = new_ttl_info; diff --git a/src/Processors/TTL/TTLDeleteAlgorithm.h b/src/Processors/TTL/TTLDeleteAlgorithm.h index 292a29bfa27..23389070774 100644 --- a/src/Processors/TTL/TTLDeleteAlgorithm.h +++ b/src/Processors/TTL/TTLDeleteAlgorithm.h @@ -10,7 +10,7 @@ namespace DB class TTLDeleteAlgorithm final : public ITTLAlgorithm { public: - TTLDeleteAlgorithm(const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_); + TTLDeleteAlgorithm(const TTlExpressions & ttl_expressions_, const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_); void execute(Block & block) override; void finalize(const MutableDataPartPtr & data_part) const override; diff --git a/src/Processors/TTL/TTLUpdateInfoAlgorithm.cpp b/src/Processors/TTL/TTLUpdateInfoAlgorithm.cpp index eba364aa2b8..34c0cad70ea 100644 --- a/src/Processors/TTL/TTLUpdateInfoAlgorithm.cpp +++ b/src/Processors/TTL/TTLUpdateInfoAlgorithm.cpp @@ -4,13 +4,14 @@ namespace DB { TTLUpdateInfoAlgorithm::TTLUpdateInfoAlgorithm( + const TTlExpressions & ttl_expressions_, const TTLDescription & description_, const TTLUpdateField ttl_update_field_, const String ttl_update_key_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_) - : ITTLAlgorithm(description_, old_ttl_info_, current_time_, force_) + : ITTLAlgorithm(ttl_expressions_, description_, old_ttl_info_, current_time_, force_) , ttl_update_field(ttl_update_field_) , ttl_update_key(ttl_update_key_) { @@ -21,7 +22,7 @@ void TTLUpdateInfoAlgorithm::execute(Block & block) if (!block) return; - auto ttl_column = executeExpressionAndGetColumn(description.expression, block, description.result_column); + auto ttl_column = executeExpressionAndGetColumn(ttl_expressions.expression, block, description.result_column); for (size_t i = 0; i < block.rows(); ++i) { UInt32 cur_ttl = ITTLAlgorithm::getTimestampByIndex(ttl_column.get(), i); diff --git a/src/Processors/TTL/TTLUpdateInfoAlgorithm.h b/src/Processors/TTL/TTLUpdateInfoAlgorithm.h index 45eecbde3d0..e9bcfcdec88 100644 --- a/src/Processors/TTL/TTLUpdateInfoAlgorithm.h +++ b/src/Processors/TTL/TTLUpdateInfoAlgorithm.h @@ -20,6 +20,7 @@ class TTLUpdateInfoAlgorithm : public ITTLAlgorithm { public: TTLUpdateInfoAlgorithm( + const TTlExpressions & ttl_expressions_, const TTLDescription & description_, const TTLUpdateField ttl_update_field_, const String ttl_update_key_, diff --git a/src/Processors/Transforms/TTLTransform.cpp b/src/Processors/Transforms/TTLTransform.cpp index 7cde86098c7..d3d45f68d46 100644 --- a/src/Processors/Transforms/TTLTransform.cpp +++ b/src/Processors/Transforms/TTLTransform.cpp @@ -36,7 +36,7 @@ TTLTransform::TTLTransform( rows_ttl, old_ttl_infos.table_ttl, current_time_, force_); /// Skip all data if table ttl is expired for part - if (algorithm->isMaxTTLExpired() && !rows_ttl.where_expression) + if (algorithm->isMaxTTLExpired() && !rows_ttl.where_expression_ast) all_data_dropped = true; delete_algorithm = algorithm.get(); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 2a381afa805..d080240b066 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -125,13 +125,18 @@ void buildScatterSelector( /// Computes ttls and updates ttl infos void updateTTL( + const ContextPtr context, const TTLDescription & ttl_entry, IMergeTreeDataPart::TTLInfos & ttl_infos, DB::MergeTreeDataPartTTLInfo & ttl_info, const Block & block, bool update_part_min_max_ttls) { - auto ttl_column = ITTLAlgorithm::executeExpressionAndGetColumn(ttl_entry.expression, block, ttl_entry.result_column); + auto expr_and_set = ttl_entry.buildExpression(); + for (auto & subquery : expr_and_set.sets->getSubqueries()) + subquery->buildSetInplace(context); + + auto ttl_column = ITTLAlgorithm::executeExpressionAndGetColumn(expr_and_set.expression, block, ttl_entry.result_column); if (const ColumnUInt16 * column_date = typeid_cast(ttl_column.get())) { @@ -488,7 +493,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( DB::IMergeTreeDataPart::TTLInfos move_ttl_infos; const auto & move_ttl_entries = metadata_snapshot->getMoveTTLs(); for (const auto & ttl_entry : move_ttl_entries) - updateTTL(ttl_entry, move_ttl_infos, move_ttl_infos.moves_ttl[ttl_entry.result_column], block, false); + updateTTL(context, ttl_entry, move_ttl_infos, move_ttl_infos.moves_ttl[ttl_entry.result_column], block, false); ReservationPtr reservation = data.reserveSpacePreferringTTLRules(metadata_snapshot, expected_size, move_ttl_infos, time(nullptr), 0, true); VolumePtr volume = data.getStoragePolicy()->getVolume(0); @@ -543,20 +548,20 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( } if (metadata_snapshot->hasRowsTTL()) - updateTTL(metadata_snapshot->getRowsTTL(), new_data_part->ttl_infos, new_data_part->ttl_infos.table_ttl, block, true); + updateTTL(context, metadata_snapshot->getRowsTTL(), new_data_part->ttl_infos, new_data_part->ttl_infos.table_ttl, block, true); for (const auto & ttl_entry : metadata_snapshot->getGroupByTTLs()) - updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.group_by_ttl[ttl_entry.result_column], block, true); + updateTTL(context, ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.group_by_ttl[ttl_entry.result_column], block, true); for (const auto & ttl_entry : metadata_snapshot->getRowsWhereTTLs()) - updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.rows_where_ttl[ttl_entry.result_column], block, true); + updateTTL(context, ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.rows_where_ttl[ttl_entry.result_column], block, true); for (const auto & [name, ttl_entry] : metadata_snapshot->getColumnTTLs()) - updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.columns_ttl[name], block, true); + updateTTL(context, ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.columns_ttl[name], block, true); const auto & recompression_ttl_entries = metadata_snapshot->getRecompressionTTLs(); for (const auto & ttl_entry : recompression_ttl_entries) - updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.recompression_ttl[ttl_entry.result_column], block, false); + updateTTL(context, ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.recompression_ttl[ttl_entry.result_column], block, false); new_data_part->ttl_infos.update(move_ttl_infos); diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index af285a953dc..7db5af82e0b 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -193,7 +193,7 @@ TTLDescription StorageInMemoryMetadata::getRowsTTL() const bool StorageInMemoryMetadata::hasRowsTTL() const { - return table_ttl.rows_ttl.expression != nullptr; + return table_ttl.rows_ttl.expression_ast != nullptr; } TTLDescriptions StorageInMemoryMetadata::getRowsWhereTTLs() const @@ -251,9 +251,8 @@ ColumnDependencies StorageInMemoryMetadata::getColumnDependencies( NameSet required_ttl_columns; NameSet updated_ttl_columns; - auto add_dependent_columns = [&updated_columns](const auto & expression, auto & to_set) + auto add_dependent_columns = [&updated_columns](const Names & required_columns, auto & to_set) { - auto required_columns = expression->getRequiredColumns(); for (const auto & dependency : required_columns) { if (updated_columns.contains(dependency)) @@ -269,13 +268,13 @@ ColumnDependencies StorageInMemoryMetadata::getColumnDependencies( for (const auto & index : getSecondaryIndices()) { if (has_dependency(index.name, ColumnDependency::SKIP_INDEX)) - add_dependent_columns(index.expression, indices_columns); + add_dependent_columns(index.expression->getRequiredColumns(), indices_columns); } for (const auto & projection : getProjections()) { if (has_dependency(projection.name, ColumnDependency::PROJECTION)) - add_dependent_columns(&projection, projections_columns); + add_dependent_columns(projection.getRequiredColumns(), projections_columns); } auto add_for_rows_ttl = [&](const auto & expression, auto & to_set) @@ -289,25 +288,25 @@ ColumnDependencies StorageInMemoryMetadata::getColumnDependencies( }; if (hasRowsTTL()) - add_for_rows_ttl(getRowsTTL().expression, required_ttl_columns); + add_for_rows_ttl(getRowsTTL().expression_columns, required_ttl_columns); for (const auto & entry : getRowsWhereTTLs()) - add_for_rows_ttl(entry.expression, required_ttl_columns); + add_for_rows_ttl(entry.expression_columns, required_ttl_columns); for (const auto & entry : getGroupByTTLs()) - add_for_rows_ttl(entry.expression, required_ttl_columns); + add_for_rows_ttl(entry.expression_columns, required_ttl_columns); for (const auto & entry : getRecompressionTTLs()) - add_dependent_columns(entry.expression, required_ttl_columns); + add_dependent_columns(entry.expression_columns, required_ttl_columns); for (const auto & [name, entry] : getColumnTTLs()) { - if (add_dependent_columns(entry.expression, required_ttl_columns) && include_ttl_target) + if (add_dependent_columns(entry.expression_columns, required_ttl_columns) && include_ttl_target) updated_ttl_columns.insert(name); } for (const auto & entry : getMoveTTLs()) - add_dependent_columns(entry.expression, required_ttl_columns); + add_dependent_columns(entry.expression_columns, required_ttl_columns); //TODO what about rows_where_ttl and group_by_ttl ?? diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index f601fed06ac..47138f30e4f 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -113,11 +113,11 @@ TTLDescription::TTLDescription(const TTLDescription & other) , if_exists(other.if_exists) , recompression_codec(other.recompression_codec) { - if (other.expression) - expression = other.expression->clone(); + // if (other.expression) + // expression = other.expression->clone(); - if (other.where_expression) - where_expression = other.where_expression->clone(); + // if (other.where_expression) + // where_expression = other.where_expression->clone(); } TTLDescription & TTLDescription::operator=(const TTLDescription & other) @@ -131,16 +131,16 @@ TTLDescription & TTLDescription::operator=(const TTLDescription & other) else expression_ast.reset(); - if (other.expression) - expression = other.expression->clone(); - else - expression.reset(); + // if (other.expression) + // expression = other.expression->clone(); + // else + // expression.reset(); result_column = other.result_column; - if (other.where_expression) - where_expression = other.where_expression->clone(); - else - where_expression.reset(); + // if (other.where_expression) + // where_expression = other.where_expression->clone(); + // else + // where_expression.reset(); where_result_column = other.where_result_column; group_by_keys = other.group_by_keys; @@ -158,6 +158,17 @@ TTLDescription & TTLDescription::operator=(const TTLDescription & other) return * this; } +static ExpressionAndSets buildExpressionAndSets(ASTPtr & ast, const NamesAndTypesList & columns, const ContextPtr & context) +{ + ExpressionAndSets result; + auto syntax_analyzer_result = TreeRewriter(context).analyze(ast, columns); + ExpressionAnalyzer analyzer(ast, syntax_analyzer_result, context); + result.expression = analyzer.getActions(false); + result.sets = analyzer.getPreparedSets(); + + return result; +} + TTLDescription TTLDescription::getTTLFromAST( const ASTPtr & definition_ast, const ColumnsDescription & columns, @@ -174,10 +185,15 @@ TTLDescription TTLDescription::getTTLFromAST( result.expression_ast = definition_ast->clone(); auto ttl_ast = result.expression_ast->clone(); - auto syntax_analyzer_result = TreeRewriter(context).analyze(ttl_ast, columns.getAllPhysical()); - result.expression = ExpressionAnalyzer(ttl_ast, syntax_analyzer_result, context).getActions(false); + auto expression = buildExpressionAndSets(ttl_ast, columns.getAllPhysical(), context).expression; + result.expression_columns = expression->getRequiredColumns(); + + // auto syntax_analyzer_result = TreeRewriter(context).analyze(ttl_ast, columns.getAllPhysical()); + // result.expression = ExpressionAnalyzer(ttl_ast, syntax_analyzer_result, context).getActions(false); result.result_column = ttl_ast->getColumnName(); + ExpressionActionsPtr where_expression; + if (ttl_element == nullptr) /// columns TTL { result.destination_type = DataDestinationType::DELETE; @@ -194,8 +210,10 @@ TTLDescription TTLDescription::getTTLFromAST( { if (ASTPtr where_expr_ast = ttl_element->where()) { - auto where_syntax_result = TreeRewriter(context).analyze(where_expr_ast, columns.getAllPhysical()); - result.where_expression = ExpressionAnalyzer(where_expr_ast, where_syntax_result, context).getActions(false); + result.where_expression_ast = where_expr_ast->clone(); + where_expression = buildExpressionAndSets(where_expr_ast, columns.getAllPhysical(), context).expression; + // auto where_syntax_result = TreeRewriter(context).analyze(where_expr_ast, columns.getAllPhysical()); + // result.where_expression = ExpressionAnalyzer(where_expr_ast, where_syntax_result, context).getActions(false); result.where_result_column = where_expr_ast->getColumnName(); } } @@ -221,17 +239,17 @@ TTLDescription TTLDescription::getTTLFromAST( for (const auto & ast : ttl_element->group_by_assignments) { const auto assignment = ast->as(); - auto expression = assignment.expression(); + auto ass_expression = assignment.expression(); FindAggregateFunctionVisitor::Data data{false}; - FindAggregateFunctionVisitor(data).visit(expression); + FindAggregateFunctionVisitor(data).visit(ass_expression); if (!data.has_aggregate_function) throw Exception(ErrorCodes::BAD_TTL_EXPRESSION, "Invalid expression for assignment of column {}. Should contain an aggregate function", assignment.column_name); - expression = addTypeConversionToAST(std::move(expression), columns.getPhysical(assignment.column_name).type->getName()); - aggregations.emplace_back(assignment.column_name, std::move(expression)); + ass_expression = addTypeConversionToAST(std::move(ass_expression), columns.getPhysical(assignment.column_name).type->getName()); + aggregations.emplace_back(assignment.column_name, std::move(ass_expression)); aggregation_columns_set.insert(assignment.column_name); } @@ -289,7 +307,7 @@ TTLDescription TTLDescription::getTTLFromAST( } } - checkTTLExpression(result.expression, result.result_column); + checkTTLExpression(expression, result.result_column); return result; } @@ -341,7 +359,7 @@ TTLTableDescription TTLTableDescription::getTTLForTableFromAST( auto ttl = TTLDescription::getTTLFromAST(ttl_element_ptr, columns, context, primary_key); if (ttl.mode == TTLMode::DELETE) { - if (!ttl.where_expression) + if (!ttl.where_expression_ast) { if (have_unconditional_delete_ttl) throw Exception(ErrorCodes::BAD_TTL_EXPRESSION, "More than one DELETE TTL expression without WHERE expression is not allowed"); diff --git a/src/Storages/TTLDescription.h b/src/Storages/TTLDescription.h index 8f60eb604b5..5ea243424cb 100644 --- a/src/Storages/TTLDescription.h +++ b/src/Storages/TTLDescription.h @@ -33,6 +33,15 @@ struct TTLAggregateDescription using TTLAggregateDescriptions = std::vector; +class PreparedSets; +using PreparedSetsPtr = std::shared_ptr; + +struct ExpressionAndSets +{ + ExpressionActionsPtr expression; + PreparedSetsPtr sets; +}; + /// Common struct for TTL record in storage struct TTLDescription { @@ -42,9 +51,10 @@ struct TTLDescription /// TTL d + INTERVAL 1 DAY /// ^~~~~~~~~~~~~~~~~~~^ ASTPtr expression_ast; + Names expression_columns; /// Expression actions evaluated from AST - ExpressionActionsPtr expression; + ExpressionAndSets buildExpression() const; /// Result column of this TTL expression String result_column; @@ -52,7 +62,8 @@ struct TTLDescription /// WHERE part in TTL expression /// TTL ... WHERE x % 10 == 0 and y > 5 /// ^~~~~~~~~~~~~~~~~~~~~~^ - ExpressionActionsPtr where_expression; + ASTPtr where_expression_ast; + ExpressionAndSets buildWhereExpression() const; /// Name of result column from WHERE expression String where_result_column; From 7ab4af06df0d78e6728e3cc5c727e5c9e4cc33ef Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 4 Dec 2023 18:04:42 +0000 Subject: [PATCH 006/276] Attempt to support subqueries in TTL. (2) --- src/Processors/QueryPlan/CreatingSetsStep.cpp | 29 +++++++++++ src/Processors/QueryPlan/CreatingSetsStep.h | 2 + src/Processors/TTL/ITTLAlgorithm.cpp | 2 +- src/Processors/TTL/ITTLAlgorithm.h | 6 +-- .../TTL/TTLAggregationAlgorithm.cpp | 2 +- src/Processors/TTL/TTLAggregationAlgorithm.h | 2 +- src/Processors/TTL/TTLColumnAlgorithm.cpp | 2 +- src/Processors/TTL/TTLColumnAlgorithm.h | 2 +- src/Processors/TTL/TTLDeleteAlgorithm.cpp | 2 +- src/Processors/TTL/TTLDeleteAlgorithm.h | 2 +- src/Processors/TTL/TTLUpdateInfoAlgorithm.cpp | 2 +- src/Processors/TTL/TTLUpdateInfoAlgorithm.h | 2 +- .../Transforms/TTLCalcTransform.cpp | 33 ++++++++++--- src/Processors/Transforms/TTLCalcTransform.h | 4 ++ src/Processors/Transforms/TTLTransform.cpp | 33 ++++++++++--- src/Processors/Transforms/TTLTransform.h | 5 ++ src/Storages/MergeTree/MergeTask.cpp | 36 +++++++++----- .../MergeTree/MergeTreeDataWriter.cpp | 2 +- src/Storages/MergeTree/MutateTask.cpp | 49 ++++++++++++++----- src/Storages/StorageInMemoryMetadata.cpp | 8 +-- src/Storages/TTLDescription.cpp | 21 +++++++- src/Storages/TTLDescription.h | 7 +-- 22 files changed, 197 insertions(+), 56 deletions(-) diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 3e4dfb0c7d1..11415e8d815 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -157,6 +157,35 @@ void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::Subqueries subque query_plan.unitePlans(std::move(creating_sets), std::move(plans)); } +QueryPipelineBuilderPtr addCreatingSetsTransform(QueryPipelineBuilderPtr pipeline, PreparedSets::Subqueries subqueries, ContextPtr context) +{ + DataStreams input_streams; + input_streams.emplace_back(DataStream{pipeline->getHeader()}); + + QueryPipelineBuilders pipelines; + pipelines.reserve(1 + subqueries.size()); + pipelines.push_back(std::move(pipeline)); + + auto plan_settings = QueryPlanOptimizationSettings::fromContext(context); + auto pipeline_settings = BuildQueryPipelineSettings::fromContext(context); + + for (auto & future_set : subqueries) + { + if (future_set->get()) + continue; + + auto plan = future_set->build(context); + if (!plan) + continue; + + input_streams.emplace_back(plan->getCurrentDataStream()); + pipelines.emplace_back(plan->buildQueryPipeline(plan_settings, pipeline_settings)); + } + + CreatingSetsStep(input_streams).updatePipeline(std::move(pipelines), pipeline_settings); + return std::move(pipelines.front()); +} + std::vector> DelayedCreatingSetsStep::makePlansForSets(DelayedCreatingSetsStep && step) { std::vector> plans; diff --git a/src/Processors/QueryPlan/CreatingSetsStep.h b/src/Processors/QueryPlan/CreatingSetsStep.h index a90b70a2fa4..292ec19914c 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.h +++ b/src/Processors/QueryPlan/CreatingSetsStep.h @@ -72,4 +72,6 @@ void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::Subqueries subque void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, ContextPtr context); +QueryPipelineBuilderPtr addCreatingSetsTransform(QueryPipelineBuilderPtr pipeline, PreparedSets::Subqueries subqueries, ContextPtr context); + } diff --git a/src/Processors/TTL/ITTLAlgorithm.cpp b/src/Processors/TTL/ITTLAlgorithm.cpp index af6c4e4ac35..761f43e2422 100644 --- a/src/Processors/TTL/ITTLAlgorithm.cpp +++ b/src/Processors/TTL/ITTLAlgorithm.cpp @@ -11,7 +11,7 @@ namespace ErrorCodes } ITTLAlgorithm::ITTLAlgorithm( - const TTlExpressions & ttl_expressions_, const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_) + const TTLExpressions & ttl_expressions_, const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_) : ttl_expressions(ttl_expressions_) , description(description_) , old_ttl_info(old_ttl_info_) diff --git a/src/Processors/TTL/ITTLAlgorithm.h b/src/Processors/TTL/ITTLAlgorithm.h index 6e73286b564..d79aa8a8dfc 100644 --- a/src/Processors/TTL/ITTLAlgorithm.h +++ b/src/Processors/TTL/ITTLAlgorithm.h @@ -8,7 +8,7 @@ namespace DB { -struct TTlExpressions +struct TTLExpressions { ExpressionActionsPtr expression; ExpressionActionsPtr where_expression; @@ -24,7 +24,7 @@ public: using TTLInfo = IMergeTreeDataPart::TTLInfo; using MutableDataPartPtr = MergeTreeMutableDataPartPtr; - ITTLAlgorithm(const TTlExpressions & ttl_expressions_, const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_); + ITTLAlgorithm(const TTLExpressions & ttl_expressions_, const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_); virtual ~ITTLAlgorithm() = default; virtual void execute(Block & block) = 0; @@ -45,7 +45,7 @@ protected: bool isTTLExpired(time_t ttl) const; UInt32 getTimestampByIndex(const IColumn * column, size_t index) const; - const TTlExpressions ttl_expressions; + const TTLExpressions ttl_expressions; const TTLDescription description; const TTLInfo old_ttl_info; const time_t current_time; diff --git a/src/Processors/TTL/TTLAggregationAlgorithm.cpp b/src/Processors/TTL/TTLAggregationAlgorithm.cpp index ab2ba5f58fc..0c6184a56e5 100644 --- a/src/Processors/TTL/TTLAggregationAlgorithm.cpp +++ b/src/Processors/TTL/TTLAggregationAlgorithm.cpp @@ -5,7 +5,7 @@ namespace DB { TTLAggregationAlgorithm::TTLAggregationAlgorithm( - const TTlExpressions & ttl_expressions_, + const TTLExpressions & ttl_expressions_, const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, diff --git a/src/Processors/TTL/TTLAggregationAlgorithm.h b/src/Processors/TTL/TTLAggregationAlgorithm.h index 9fd074efba8..f7bf19a202b 100644 --- a/src/Processors/TTL/TTLAggregationAlgorithm.h +++ b/src/Processors/TTL/TTLAggregationAlgorithm.h @@ -13,7 +13,7 @@ class TTLAggregationAlgorithm final : public ITTLAlgorithm { public: TTLAggregationAlgorithm( - const TTlExpressions & ttl_expressions_, + const TTLExpressions & ttl_expressions_, const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, diff --git a/src/Processors/TTL/TTLColumnAlgorithm.cpp b/src/Processors/TTL/TTLColumnAlgorithm.cpp index cb99dcf99b1..e27050564ce 100644 --- a/src/Processors/TTL/TTLColumnAlgorithm.cpp +++ b/src/Processors/TTL/TTLColumnAlgorithm.cpp @@ -4,7 +4,7 @@ namespace DB { TTLColumnAlgorithm::TTLColumnAlgorithm( - const TTlExpressions & ttl_expressions_, + const TTLExpressions & ttl_expressions_, const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, diff --git a/src/Processors/TTL/TTLColumnAlgorithm.h b/src/Processors/TTL/TTLColumnAlgorithm.h index efcd7c74454..f34dae952d1 100644 --- a/src/Processors/TTL/TTLColumnAlgorithm.h +++ b/src/Processors/TTL/TTLColumnAlgorithm.h @@ -11,7 +11,7 @@ class TTLColumnAlgorithm final : public ITTLAlgorithm { public: TTLColumnAlgorithm( - const TTlExpressions & ttl_expressions_, + const TTLExpressions & ttl_expressions_, const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, diff --git a/src/Processors/TTL/TTLDeleteAlgorithm.cpp b/src/Processors/TTL/TTLDeleteAlgorithm.cpp index 6a172e9c3c3..6f9bc315276 100644 --- a/src/Processors/TTL/TTLDeleteAlgorithm.cpp +++ b/src/Processors/TTL/TTLDeleteAlgorithm.cpp @@ -4,7 +4,7 @@ namespace DB { TTLDeleteAlgorithm::TTLDeleteAlgorithm( - const TTlExpressions & ttl_expressions_, const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_) + const TTLExpressions & ttl_expressions_, const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_) : ITTLAlgorithm(ttl_expressions_, description_, old_ttl_info_, current_time_, force_) { if (!isMinTTLExpired()) diff --git a/src/Processors/TTL/TTLDeleteAlgorithm.h b/src/Processors/TTL/TTLDeleteAlgorithm.h index 23389070774..622e45acecb 100644 --- a/src/Processors/TTL/TTLDeleteAlgorithm.h +++ b/src/Processors/TTL/TTLDeleteAlgorithm.h @@ -10,7 +10,7 @@ namespace DB class TTLDeleteAlgorithm final : public ITTLAlgorithm { public: - TTLDeleteAlgorithm(const TTlExpressions & ttl_expressions_, const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_); + TTLDeleteAlgorithm(const TTLExpressions & ttl_expressions_, const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_); void execute(Block & block) override; void finalize(const MutableDataPartPtr & data_part) const override; diff --git a/src/Processors/TTL/TTLUpdateInfoAlgorithm.cpp b/src/Processors/TTL/TTLUpdateInfoAlgorithm.cpp index 34c0cad70ea..b7cddf3c165 100644 --- a/src/Processors/TTL/TTLUpdateInfoAlgorithm.cpp +++ b/src/Processors/TTL/TTLUpdateInfoAlgorithm.cpp @@ -4,7 +4,7 @@ namespace DB { TTLUpdateInfoAlgorithm::TTLUpdateInfoAlgorithm( - const TTlExpressions & ttl_expressions_, + const TTLExpressions & ttl_expressions_, const TTLDescription & description_, const TTLUpdateField ttl_update_field_, const String ttl_update_key_, diff --git a/src/Processors/TTL/TTLUpdateInfoAlgorithm.h b/src/Processors/TTL/TTLUpdateInfoAlgorithm.h index e9bcfcdec88..0cf31765aef 100644 --- a/src/Processors/TTL/TTLUpdateInfoAlgorithm.h +++ b/src/Processors/TTL/TTLUpdateInfoAlgorithm.h @@ -20,7 +20,7 @@ class TTLUpdateInfoAlgorithm : public ITTLAlgorithm { public: TTLUpdateInfoAlgorithm( - const TTlExpressions & ttl_expressions_, + const TTLExpressions & ttl_expressions_, const TTLDescription & description_, const TTLUpdateField ttl_update_field_, const String ttl_update_key_, diff --git a/src/Processors/Transforms/TTLCalcTransform.cpp b/src/Processors/Transforms/TTLCalcTransform.cpp index 31fb61239ef..204dfe21733 100644 --- a/src/Processors/Transforms/TTLCalcTransform.cpp +++ b/src/Processors/Transforms/TTLCalcTransform.cpp @@ -4,7 +4,22 @@ namespace DB { +static TTLExpressions getExpressions(const TTLDescription & ttl_descr, PreparedSets::Subqueries & subqueries_for_sets, const ContextPtr & context) +{ + auto expr = ttl_descr.buildExpression(context); + auto where_expr = ttl_descr.buildWhereExpression(context); + + auto expr_queries = expr.sets->getSubqueries(); + auto where_expr_queries = expr.sets->getSubqueries(); + + subqueries_for_sets.insert(subqueries_for_sets.end(), expr_queries.begin(), expr_queries.end()); + subqueries_for_sets.insert(subqueries_for_sets.end(), where_expr_queries.begin(), where_expr_queries.end()); + + return {expr.expression, where_expr.expression}; +} + TTLCalcTransform::TTLCalcTransform( + const ContextPtr & context, const Block & header_, const MergeTreeData & storage_, const StorageMetadataPtr & metadata_snapshot_, @@ -21,33 +36,39 @@ TTLCalcTransform::TTLCalcTransform( { const auto & rows_ttl = metadata_snapshot_->getRowsTTL(); algorithms.emplace_back(std::make_unique( - rows_ttl, TTLUpdateField::TABLE_TTL, rows_ttl.result_column, old_ttl_infos.table_ttl, current_time_, force_)); + getExpressions(rows_ttl, subqueries_for_sets, context), rows_ttl, + TTLUpdateField::TABLE_TTL, rows_ttl.result_column, old_ttl_infos.table_ttl, current_time_, force_)); } for (const auto & where_ttl : metadata_snapshot_->getRowsWhereTTLs()) algorithms.emplace_back(std::make_unique( - where_ttl, TTLUpdateField::ROWS_WHERE_TTL, where_ttl.result_column, old_ttl_infos.rows_where_ttl[where_ttl.result_column], current_time_, force_)); + getExpressions(where_ttl, subqueries_for_sets, context), where_ttl, + TTLUpdateField::ROWS_WHERE_TTL, where_ttl.result_column, old_ttl_infos.rows_where_ttl[where_ttl.result_column], current_time_, force_)); for (const auto & group_by_ttl : metadata_snapshot_->getGroupByTTLs()) algorithms.emplace_back(std::make_unique( - group_by_ttl, TTLUpdateField::GROUP_BY_TTL, group_by_ttl.result_column, old_ttl_infos.group_by_ttl[group_by_ttl.result_column], current_time_, force_)); + getExpressions(group_by_ttl, subqueries_for_sets, context), group_by_ttl, + TTLUpdateField::GROUP_BY_TTL, group_by_ttl.result_column, old_ttl_infos.group_by_ttl[group_by_ttl.result_column], current_time_, force_)); if (metadata_snapshot_->hasAnyColumnTTL()) { for (const auto & [name, description] : metadata_snapshot_->getColumnTTLs()) { algorithms.emplace_back(std::make_unique( - description, TTLUpdateField::COLUMNS_TTL, name, old_ttl_infos.columns_ttl[name], current_time_, force_)); + getExpressions(description, subqueries_for_sets, context), description, + TTLUpdateField::COLUMNS_TTL, name, old_ttl_infos.columns_ttl[name], current_time_, force_)); } } for (const auto & move_ttl : metadata_snapshot_->getMoveTTLs()) algorithms.emplace_back(std::make_unique( - move_ttl, TTLUpdateField::MOVES_TTL, move_ttl.result_column, old_ttl_infos.moves_ttl[move_ttl.result_column], current_time_, force_)); + getExpressions(move_ttl, subqueries_for_sets, context), move_ttl, + TTLUpdateField::MOVES_TTL, move_ttl.result_column, old_ttl_infos.moves_ttl[move_ttl.result_column], current_time_, force_)); for (const auto & recompression_ttl : metadata_snapshot_->getRecompressionTTLs()) algorithms.emplace_back(std::make_unique( - recompression_ttl, TTLUpdateField::RECOMPRESSION_TTL, recompression_ttl.result_column, old_ttl_infos.recompression_ttl[recompression_ttl.result_column], current_time_, force_)); + getExpressions(recompression_ttl, subqueries_for_sets, context), recompression_ttl, + TTLUpdateField::RECOMPRESSION_TTL, recompression_ttl.result_column, old_ttl_infos.recompression_ttl[recompression_ttl.result_column], current_time_, force_)); } void TTLCalcTransform::consume(Chunk chunk) diff --git a/src/Processors/Transforms/TTLCalcTransform.h b/src/Processors/Transforms/TTLCalcTransform.h index 495879400dc..960438f5f2b 100644 --- a/src/Processors/Transforms/TTLCalcTransform.h +++ b/src/Processors/Transforms/TTLCalcTransform.h @@ -15,6 +15,7 @@ class TTLCalcTransform : public IAccumulatingTransform { public: TTLCalcTransform( + const ContextPtr & context, const Block & header_, const MergeTreeData & storage_, const StorageMetadataPtr & metadata_snapshot_, @@ -23,6 +24,8 @@ public: bool force_ ); + PreparedSets::Subqueries getSubqueries() { return std::move(subqueries_for_sets); } + String getName() const override { return "TTL_CALC"; } Status prepare() override; @@ -35,6 +38,7 @@ protected: private: std::vector algorithms; + PreparedSets::Subqueries subqueries_for_sets; /// ttl_infos and empty_columns are updating while reading const MergeTreeData::MutableDataPartPtr & data_part; diff --git a/src/Processors/Transforms/TTLTransform.cpp b/src/Processors/Transforms/TTLTransform.cpp index d3d45f68d46..69e2e6e5fc0 100644 --- a/src/Processors/Transforms/TTLTransform.cpp +++ b/src/Processors/Transforms/TTLTransform.cpp @@ -16,7 +16,22 @@ namespace DB { +static TTLExpressions getExpressions(const TTLDescription & ttl_descr, PreparedSets::Subqueries & subqueries_for_sets, const ContextPtr & context) +{ + auto expr = ttl_descr.buildExpression(context); + auto where_expr = ttl_descr.buildWhereExpression(context); + + auto expr_queries = expr.sets->getSubqueries(); + auto where_expr_queries = expr.sets->getSubqueries(); + + subqueries_for_sets.insert(subqueries_for_sets.end(), expr_queries.begin(), expr_queries.end()); + subqueries_for_sets.insert(subqueries_for_sets.end(), where_expr_queries.begin(), where_expr_queries.end()); + + return {expr.expression, where_expr.expression}; +} + TTLTransform::TTLTransform( + const ContextPtr & context, const Block & header_, const MergeTreeData & storage_, const StorageMetadataPtr & metadata_snapshot_, @@ -33,7 +48,8 @@ TTLTransform::TTLTransform( { const auto & rows_ttl = metadata_snapshot_->getRowsTTL(); auto algorithm = std::make_unique( - rows_ttl, old_ttl_infos.table_ttl, current_time_, force_); + getExpressions(rows_ttl, subqueries_for_sets, context), rows_ttl, + old_ttl_infos.table_ttl, current_time_, force_); /// Skip all data if table ttl is expired for part if (algorithm->isMaxTTLExpired() && !rows_ttl.where_expression_ast) @@ -45,11 +61,13 @@ TTLTransform::TTLTransform( for (const auto & where_ttl : metadata_snapshot_->getRowsWhereTTLs()) algorithms.emplace_back(std::make_unique( - where_ttl, old_ttl_infos.rows_where_ttl[where_ttl.result_column], current_time_, force_)); + getExpressions(where_ttl, subqueries_for_sets, context), where_ttl, + old_ttl_infos.rows_where_ttl[where_ttl.result_column], current_time_, force_)); for (const auto & group_by_ttl : metadata_snapshot_->getGroupByTTLs()) algorithms.emplace_back(std::make_unique( - group_by_ttl, old_ttl_infos.group_by_ttl[group_by_ttl.result_column], current_time_, force_, + getExpressions(group_by_ttl, subqueries_for_sets, context), group_by_ttl, + old_ttl_infos.group_by_ttl[group_by_ttl.result_column], current_time_, force_, getInputPort().getHeader(), storage_)); if (metadata_snapshot_->hasAnyColumnTTL()) @@ -75,18 +93,21 @@ TTLTransform::TTLTransform( } algorithms.emplace_back(std::make_unique( - description, old_ttl_infos.columns_ttl[name], current_time_, + getExpressions(description, subqueries_for_sets, context), description, + old_ttl_infos.columns_ttl[name], current_time_, force_, name, default_expression, default_column_name, isCompactPart(data_part))); } } for (const auto & move_ttl : metadata_snapshot_->getMoveTTLs()) algorithms.emplace_back(std::make_unique( - move_ttl, TTLUpdateField::MOVES_TTL, move_ttl.result_column, old_ttl_infos.moves_ttl[move_ttl.result_column], current_time_, force_)); + getExpressions(move_ttl, subqueries_for_sets, context), move_ttl, + TTLUpdateField::MOVES_TTL, move_ttl.result_column, old_ttl_infos.moves_ttl[move_ttl.result_column], current_time_, force_)); for (const auto & recompression_ttl : metadata_snapshot_->getRecompressionTTLs()) algorithms.emplace_back(std::make_unique( - recompression_ttl, TTLUpdateField::RECOMPRESSION_TTL, recompression_ttl.result_column, old_ttl_infos.recompression_ttl[recompression_ttl.result_column], current_time_, force_)); + getExpressions(recompression_ttl, subqueries_for_sets, context), recompression_ttl, + TTLUpdateField::RECOMPRESSION_TTL, recompression_ttl.result_column, old_ttl_infos.recompression_ttl[recompression_ttl.result_column], current_time_, force_)); } Block reorderColumns(Block block, const Block & header) diff --git a/src/Processors/Transforms/TTLTransform.h b/src/Processors/Transforms/TTLTransform.h index 3f0dffd1998..47da456a2e3 100644 --- a/src/Processors/Transforms/TTLTransform.h +++ b/src/Processors/Transforms/TTLTransform.h @@ -16,6 +16,7 @@ class TTLTransform : public IAccumulatingTransform { public: TTLTransform( + const ContextPtr & context, const Block & header_, const MergeTreeData & storage_, const StorageMetadataPtr & metadata_snapshot_, @@ -28,6 +29,8 @@ public: Status prepare() override; + PreparedSets::Subqueries getSubqueries() { return std::move(subqueries_for_sets); } + protected: void consume(Chunk chunk) override; Chunk generate() override; @@ -40,6 +43,8 @@ private: const TTLDeleteAlgorithm * delete_algorithm = nullptr; bool all_data_dropped = false; + PreparedSets::Subqueries subqueries_for_sets; + /// ttl_infos and empty_columns are updating while reading const MergeTreeData::MutableDataPartPtr & data_part; Poco::Logger * log; diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index e8e307bb148..26b290d33d5 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -31,6 +31,9 @@ #include #include #include +#include +#include +#include namespace DB { @@ -1004,8 +1007,9 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() break; } - auto res_pipe = Pipe::unitePipes(std::move(pipes)); - res_pipe.addTransform(std::move(merged_transform)); + auto builder = std::make_unique(); + builder->init(Pipe::unitePipes(std::move(pipes))); + builder->addTransform(std::move(merged_transform)); if (global_ctx->deduplicate) { @@ -1021,26 +1025,34 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() } if (DistinctSortedTransform::isApplicable(header, sort_description, global_ctx->deduplicate_by_columns)) - res_pipe.addTransform(std::make_shared( - res_pipe.getHeader(), sort_description, SizeLimits(), 0 /*limit_hint*/, global_ctx->deduplicate_by_columns)); + builder->addTransform(std::make_shared( + builder->getHeader(), sort_description, SizeLimits(), 0 /*limit_hint*/, global_ctx->deduplicate_by_columns)); else - res_pipe.addTransform(std::make_shared( - res_pipe.getHeader(), SizeLimits(), 0 /*limit_hint*/, global_ctx->deduplicate_by_columns)); + builder->addTransform(std::make_shared( + builder->getHeader(), SizeLimits(), 0 /*limit_hint*/, global_ctx->deduplicate_by_columns)); } + PreparedSets::Subqueries subqueries; + if (ctx->need_remove_expired_values) - res_pipe.addTransform(std::make_shared( - res_pipe.getHeader(), *global_ctx->data, global_ctx->metadata_snapshot, global_ctx->new_data_part, global_ctx->time_of_merge, ctx->force_ttl)); + { + auto transform = std::make_shared(global_ctx->context, builder->getHeader(), *global_ctx->data, global_ctx->metadata_snapshot, global_ctx->new_data_part, global_ctx->time_of_merge, ctx->force_ttl); + subqueries = transform->getSubqueries(); + builder->addTransform(std::move(transform)); + } if (global_ctx->metadata_snapshot->hasSecondaryIndices()) { const auto & indices = global_ctx->metadata_snapshot->getSecondaryIndices(); - res_pipe.addTransform(std::make_shared( - res_pipe.getHeader(), indices.getSingleExpressionForIndices(global_ctx->metadata_snapshot->getColumns(), global_ctx->data->getContext()))); - res_pipe.addTransform(std::make_shared(res_pipe.getHeader())); + builder->addTransform(std::make_shared( + builder->getHeader(), indices.getSingleExpressionForIndices(global_ctx->metadata_snapshot->getColumns(), global_ctx->data->getContext()))); + builder->addTransform(std::make_shared(builder->getHeader())); } - global_ctx->merged_pipeline = QueryPipeline(std::move(res_pipe)); + if (!subqueries.empty()) + builder = addCreatingSetsTransform(std::move(builder), std::move(subqueries), global_ctx->context); + + global_ctx->merged_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); /// Dereference unique_ptr and pass horizontal_stage_progress by reference global_ctx->merged_pipeline.setProgressCallback(MergeProgressCallback(global_ctx->merge_list_element_ptr, global_ctx->watch_prev_elapsed, *global_ctx->horizontal_stage_progress)); /// Is calculated inside MergeProgressCallback. diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index d080240b066..ce9e5762cb4 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -132,7 +132,7 @@ void updateTTL( const Block & block, bool update_part_min_max_ttls) { - auto expr_and_set = ttl_entry.buildExpression(); + auto expr_and_set = ttl_entry.buildExpression(context); for (auto & subquery : expr_and_set.sets->getSubqueries()) subquery->buildSetInplace(context); diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 6b6b5947581..61849f94e44 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -16,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -1507,21 +1509,34 @@ private: if (!ctx->mutating_pipeline_builder.initialized()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot mutate part columns with uninitialized mutations stream. It's a bug"); - QueryPipelineBuilder builder(std::move(ctx->mutating_pipeline_builder)); + auto builder = std::make_unique(std::move(ctx->mutating_pipeline_builder)); if (ctx->metadata_snapshot->hasPrimaryKey() || ctx->metadata_snapshot->hasSecondaryIndices()) { - builder.addTransform(std::make_shared( - builder.getHeader(), ctx->data->getPrimaryKeyAndSkipIndicesExpression(ctx->metadata_snapshot, skip_indices))); + builder->addTransform(std::make_shared( + builder->getHeader(), ctx->data->getPrimaryKeyAndSkipIndicesExpression(ctx->metadata_snapshot, skip_indices))); - builder.addTransform(std::make_shared(builder.getHeader())); + builder->addTransform(std::make_shared(builder->getHeader())); } + PreparedSets::Subqueries subqueries; + if (ctx->execute_ttl_type == ExecuteTTLType::NORMAL) - builder.addTransform(std::make_shared(builder.getHeader(), *ctx->data, ctx->metadata_snapshot, ctx->new_data_part, ctx->time_of_mutation, true)); + { + auto transform = std::make_shared(ctx->context, builder->getHeader(), *ctx->data, ctx->metadata_snapshot, ctx->new_data_part, ctx->time_of_mutation, true); + subqueries = transform->getSubqueries(); + builder->addTransform(std::move(transform)); + } if (ctx->execute_ttl_type == ExecuteTTLType::RECALCULATE) - builder.addTransform(std::make_shared(builder.getHeader(), *ctx->data, ctx->metadata_snapshot, ctx->new_data_part, ctx->time_of_mutation, true)); + { + auto transform = std::make_shared(ctx->context, builder->getHeader(), *ctx->data, ctx->metadata_snapshot, ctx->new_data_part, ctx->time_of_mutation, true); + subqueries = transform->getSubqueries(); + builder->addTransform(std::move(transform)); + } + + if (!subqueries.empty()) + builder = addCreatingSetsTransform(std::move(builder), std::move(subqueries), ctx->context); ctx->minmax_idx = std::make_shared(); @@ -1537,7 +1552,7 @@ private: /*blocks_are_granules_size=*/ false, ctx->context->getWriteSettings()); - ctx->mutating_pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); + ctx->mutating_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); ctx->mutating_pipeline.setProgressCallback(ctx->progress_callback); /// Is calculated inside MergeProgressCallback. ctx->mutating_pipeline.disableProfileEventUpdate(); @@ -1712,13 +1727,25 @@ private: if (ctx->mutating_pipeline_builder.initialized()) { - QueryPipelineBuilder builder(std::move(ctx->mutating_pipeline_builder)); + auto builder = std::make_unique(std::move(ctx->mutating_pipeline_builder)); + PreparedSets::Subqueries subqueries; if (ctx->execute_ttl_type == ExecuteTTLType::NORMAL) - builder.addTransform(std::make_shared(builder.getHeader(), *ctx->data, ctx->metadata_snapshot, ctx->new_data_part, ctx->time_of_mutation, true)); + { + auto transform = std::make_shared(ctx->context, builder->getHeader(), *ctx->data, ctx->metadata_snapshot, ctx->new_data_part, ctx->time_of_mutation, true); + subqueries = transform->getSubqueries(); + builder->addTransform(std::move(transform)); + } if (ctx->execute_ttl_type == ExecuteTTLType::RECALCULATE) - builder.addTransform(std::make_shared(builder.getHeader(), *ctx->data, ctx->metadata_snapshot, ctx->new_data_part, ctx->time_of_mutation, true)); + { + auto transform = std::make_shared(ctx->context, builder->getHeader(), *ctx->data, ctx->metadata_snapshot, ctx->new_data_part, ctx->time_of_mutation, true); + subqueries = transform->getSubqueries(); + builder->addTransform(std::move(transform)); + } + + if (!subqueries.empty()) + builder = addCreatingSetsTransform(std::move(builder), std::move(subqueries), ctx->context); ctx->out = std::make_shared( ctx->new_data_part, @@ -1732,7 +1759,7 @@ private: &ctx->source_part->index_granularity_info ); - ctx->mutating_pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); + ctx->mutating_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); ctx->mutating_pipeline.setProgressCallback(ctx->progress_callback); /// Is calculated inside MergeProgressCallback. ctx->mutating_pipeline.disableProfileEventUpdate(); diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 7db5af82e0b..158c13b653d 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -279,7 +279,7 @@ ColumnDependencies StorageInMemoryMetadata::getColumnDependencies( auto add_for_rows_ttl = [&](const auto & expression, auto & to_set) { - if (add_dependent_columns(expression, to_set) && include_ttl_target) + if (add_dependent_columns(expression.getNames(), to_set) && include_ttl_target) { /// Filter all columns, if rows TTL expression have to be recalculated. for (const auto & column : getColumns().getAllPhysical()) @@ -297,16 +297,16 @@ ColumnDependencies StorageInMemoryMetadata::getColumnDependencies( add_for_rows_ttl(entry.expression_columns, required_ttl_columns); for (const auto & entry : getRecompressionTTLs()) - add_dependent_columns(entry.expression_columns, required_ttl_columns); + add_dependent_columns(entry.expression_columns.getNames(), required_ttl_columns); for (const auto & [name, entry] : getColumnTTLs()) { - if (add_dependent_columns(entry.expression_columns, required_ttl_columns) && include_ttl_target) + if (add_dependent_columns(entry.expression_columns.getNames(), required_ttl_columns) && include_ttl_target) updated_ttl_columns.insert(name); } for (const auto & entry : getMoveTTLs()) - add_dependent_columns(entry.expression_columns, required_ttl_columns); + add_dependent_columns(entry.expression_columns.getNames(), required_ttl_columns); //TODO what about rows_where_ttl and group_by_ttl ?? diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 47138f30e4f..e02ac933028 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -169,6 +169,23 @@ static ExpressionAndSets buildExpressionAndSets(ASTPtr & ast, const NamesAndType return result; } +ExpressionAndSets TTLDescription::buildExpression(const ContextPtr & context) const +{ + auto ast = expression_ast->clone(); + return buildExpressionAndSets(ast, expression_columns, context); +} + +ExpressionAndSets TTLDescription::buildWhereExpression(const ContextPtr & context) const +{ + if (where_expression_ast) + { + auto ast = where_expression_ast->clone(); + return buildExpressionAndSets(ast, where_expression_columns, context); + } + + return {}; +} + TTLDescription TTLDescription::getTTLFromAST( const ASTPtr & definition_ast, const ColumnsDescription & columns, @@ -186,7 +203,7 @@ TTLDescription TTLDescription::getTTLFromAST( auto ttl_ast = result.expression_ast->clone(); auto expression = buildExpressionAndSets(ttl_ast, columns.getAllPhysical(), context).expression; - result.expression_columns = expression->getRequiredColumns(); + result.expression_columns = expression->getRequiredColumnsWithTypes(); // auto syntax_analyzer_result = TreeRewriter(context).analyze(ttl_ast, columns.getAllPhysical()); // result.expression = ExpressionAnalyzer(ttl_ast, syntax_analyzer_result, context).getActions(false); @@ -214,6 +231,8 @@ TTLDescription TTLDescription::getTTLFromAST( where_expression = buildExpressionAndSets(where_expr_ast, columns.getAllPhysical(), context).expression; // auto where_syntax_result = TreeRewriter(context).analyze(where_expr_ast, columns.getAllPhysical()); // result.where_expression = ExpressionAnalyzer(where_expr_ast, where_syntax_result, context).getActions(false); + + result.where_expression_columns = where_expression->getRequiredColumnsWithTypes(); result.where_result_column = where_expr_ast->getColumnName(); } } diff --git a/src/Storages/TTLDescription.h b/src/Storages/TTLDescription.h index 5ea243424cb..7dfc736ded2 100644 --- a/src/Storages/TTLDescription.h +++ b/src/Storages/TTLDescription.h @@ -51,10 +51,10 @@ struct TTLDescription /// TTL d + INTERVAL 1 DAY /// ^~~~~~~~~~~~~~~~~~~^ ASTPtr expression_ast; - Names expression_columns; + NamesAndTypesList expression_columns; /// Expression actions evaluated from AST - ExpressionAndSets buildExpression() const; + ExpressionAndSets buildExpression(const ContextPtr & context) const; /// Result column of this TTL expression String result_column; @@ -63,7 +63,8 @@ struct TTLDescription /// TTL ... WHERE x % 10 == 0 and y > 5 /// ^~~~~~~~~~~~~~~~~~~~~~^ ASTPtr where_expression_ast; - ExpressionAndSets buildWhereExpression() const; + NamesAndTypesList where_expression_columns; + ExpressionAndSets buildWhereExpression(const ContextPtr & context) const; /// Name of result column from WHERE expression String where_result_column; From 16558ccc840d7a15efb2ab0fe691a79c38dd5086 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 4 Dec 2023 18:13:34 +0000 Subject: [PATCH 007/276] Fix some tests --- src/Storages/TTLDescription.cpp | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index e02ac933028..e32ff11860b 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -103,7 +103,10 @@ using FindAggregateFunctionVisitor = InDepthNodeVisitorclone() : nullptr) + , expression_columns(other.expression_columns) , result_column(other.result_column) + , where_expression_ast(other.where_expression_ast ? other.where_expression_ast->clone() : nullptr) + , where_expression_columns(other.where_expression_columns) , where_result_column(other.where_result_column) , group_by_keys(other.group_by_keys) , set_parts(other.set_parts) @@ -136,12 +139,20 @@ TTLDescription & TTLDescription::operator=(const TTLDescription & other) // else // expression.reset(); + expression_columns = other.expression_columns; result_column = other.result_column; + + if (other.where_expression_ast) + where_expression_ast = other.where_expression_ast->clone(); + else + where_expression_ast.reset(); + // if (other.where_expression) // where_expression = other.where_expression->clone(); // else // where_expression.reset(); + where_expression_columns = other.where_expression_columns; where_result_column = other.where_result_column; group_by_keys = other.group_by_keys; set_parts = other.set_parts; From 6a821f9e737373b28bc98f25e10439dd04e7bdb8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 4 Dec 2023 19:24:27 +0000 Subject: [PATCH 008/276] Fix some staff --- src/Processors/QueryPlan/CreatingSetsStep.cpp | 3 +-- src/Processors/Transforms/TTLCalcTransform.cpp | 12 +++++++----- src/Processors/Transforms/TTLTransform.cpp | 12 +++++++----- 3 files changed, 15 insertions(+), 12 deletions(-) diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 11415e8d815..f13a717004f 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -182,8 +182,7 @@ QueryPipelineBuilderPtr addCreatingSetsTransform(QueryPipelineBuilderPtr pipelin pipelines.emplace_back(plan->buildQueryPipeline(plan_settings, pipeline_settings)); } - CreatingSetsStep(input_streams).updatePipeline(std::move(pipelines), pipeline_settings); - return std::move(pipelines.front()); + return CreatingSetsStep(input_streams).updatePipeline(std::move(pipelines), pipeline_settings); } std::vector> DelayedCreatingSetsStep::makePlansForSets(DelayedCreatingSetsStep && step) diff --git a/src/Processors/Transforms/TTLCalcTransform.cpp b/src/Processors/Transforms/TTLCalcTransform.cpp index 204dfe21733..0af9f38b20f 100644 --- a/src/Processors/Transforms/TTLCalcTransform.cpp +++ b/src/Processors/Transforms/TTLCalcTransform.cpp @@ -7,13 +7,15 @@ namespace DB static TTLExpressions getExpressions(const TTLDescription & ttl_descr, PreparedSets::Subqueries & subqueries_for_sets, const ContextPtr & context) { auto expr = ttl_descr.buildExpression(context); - auto where_expr = ttl_descr.buildWhereExpression(context); - auto expr_queries = expr.sets->getSubqueries(); - auto where_expr_queries = expr.sets->getSubqueries(); - subqueries_for_sets.insert(subqueries_for_sets.end(), expr_queries.begin(), expr_queries.end()); - subqueries_for_sets.insert(subqueries_for_sets.end(), where_expr_queries.begin(), where_expr_queries.end()); + + auto where_expr = ttl_descr.buildWhereExpression(context); + if (where_expr.sets) + { + auto where_expr_queries = where_expr.sets->getSubqueries(); + subqueries_for_sets.insert(subqueries_for_sets.end(), where_expr_queries.begin(), where_expr_queries.end()); + } return {expr.expression, where_expr.expression}; } diff --git a/src/Processors/Transforms/TTLTransform.cpp b/src/Processors/Transforms/TTLTransform.cpp index 69e2e6e5fc0..69b7d80c563 100644 --- a/src/Processors/Transforms/TTLTransform.cpp +++ b/src/Processors/Transforms/TTLTransform.cpp @@ -19,13 +19,15 @@ namespace DB static TTLExpressions getExpressions(const TTLDescription & ttl_descr, PreparedSets::Subqueries & subqueries_for_sets, const ContextPtr & context) { auto expr = ttl_descr.buildExpression(context); - auto where_expr = ttl_descr.buildWhereExpression(context); - auto expr_queries = expr.sets->getSubqueries(); - auto where_expr_queries = expr.sets->getSubqueries(); - subqueries_for_sets.insert(subqueries_for_sets.end(), expr_queries.begin(), expr_queries.end()); - subqueries_for_sets.insert(subqueries_for_sets.end(), where_expr_queries.begin(), where_expr_queries.end()); + + auto where_expr = ttl_descr.buildWhereExpression(context); + if (where_expr.sets) + { + auto where_expr_queries = where_expr.sets->getSubqueries(); + subqueries_for_sets.insert(subqueries_for_sets.end(), where_expr_queries.begin(), where_expr_queries.end()); + } return {expr.expression, where_expr.expression}; } From 0015ec28f9f70548c31e220f2dd826e4ac21f007 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 5 Dec 2023 12:45:25 +0000 Subject: [PATCH 009/276] Fixing test. --- src/Storages/TTLDescription.cpp | 22 +++++++++++++++++++--- 1 file changed, 19 insertions(+), 3 deletions(-) diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index e32ff11860b..bfd3afc30d8 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -18,6 +18,7 @@ #include #include #include +#include namespace DB @@ -172,11 +173,26 @@ TTLDescription & TTLDescription::operator=(const TTLDescription & other) static ExpressionAndSets buildExpressionAndSets(ASTPtr & ast, const NamesAndTypesList & columns, const ContextPtr & context) { ExpressionAndSets result; + auto ttl_string = queryToString(ast); auto syntax_analyzer_result = TreeRewriter(context).analyze(ast, columns); ExpressionAnalyzer analyzer(ast, syntax_analyzer_result, context); - result.expression = analyzer.getActions(false); + auto dag = analyzer.getActionsDAG(false); + + const auto * col = &dag->findInOutputs(ast->getColumnName()); + // std::cerr << "buildExpressionAndSets " << ttl_string << std::endl; + if (col->result_name != ttl_string) + col = &dag->addAlias(*col, ttl_string); + + dag->getOutputs() = {col}; + dag->removeUnusedActions(); + + result.expression = std::make_shared(dag, ExpressionActionsSettings::fromContext(context)); result.sets = analyzer.getPreparedSets(); + // std::cerr << "--------- buildExpressionAndSets\n"; + // std::cerr << result.expression->dumpActions() << std::endl; + // std::cerr << result.sets->getSubqueries().size() << std::endl; + return result; } @@ -218,7 +234,7 @@ TTLDescription TTLDescription::getTTLFromAST( // auto syntax_analyzer_result = TreeRewriter(context).analyze(ttl_ast, columns.getAllPhysical()); // result.expression = ExpressionAnalyzer(ttl_ast, syntax_analyzer_result, context).getActions(false); - result.result_column = ttl_ast->getColumnName(); + result.result_column = expression->getSampleBlock().safeGetByPosition(0).name; ExpressionActionsPtr where_expression; @@ -244,7 +260,7 @@ TTLDescription TTLDescription::getTTLFromAST( // result.where_expression = ExpressionAnalyzer(where_expr_ast, where_syntax_result, context).getActions(false); result.where_expression_columns = where_expression->getRequiredColumnsWithTypes(); - result.where_result_column = where_expr_ast->getColumnName(); + result.where_result_column = where_expression->getSampleBlock().safeGetByPosition(0).name; } } else if (ttl_element->mode == TTLMode::GROUP_BY) From 43a23898e0ddb71fe810dafd850cef911dace902 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 5 Dec 2023 14:20:07 +0000 Subject: [PATCH 010/276] Updating the tests. --- .../0_stateless/01465_ttl_recompression.reference | 6 +++--- .../queries/0_stateless/02932_set_ttl_where.reference | 3 +++ tests/queries/0_stateless/02932_set_ttl_where.sql | 10 +--------- 3 files changed, 7 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/01465_ttl_recompression.reference b/tests/queries/0_stateless/01465_ttl_recompression.reference index 108df565669..90661a5dc78 100644 --- a/tests/queries/0_stateless/01465_ttl_recompression.reference +++ b/tests/queries/0_stateless/01465_ttl_recompression.reference @@ -13,9 +13,9 @@ CREATE TABLE default.recompression_table\n(\n `dt` DateTime,\n `key` UInt6 1_1_1 LZ4 2_2_2 ZSTD(12) 3_3_3 ZSTD(12) -1_1_1 ['plus(dt, toIntervalDay(1))'] -2_2_2 ['plus(dt, toIntervalDay(1))'] -3_3_3 ['plus(dt, toIntervalDay(1))'] +1_1_1 ['dt + toIntervalDay(1)'] +2_2_2 ['dt + toIntervalDay(1)'] +3_3_3 ['dt + toIntervalDay(1)'] 1_1_1 LZ4 2_2_2 LZ4 3_3_3 LZ4 diff --git a/tests/queries/0_stateless/02932_set_ttl_where.reference b/tests/queries/0_stateless/02932_set_ttl_where.reference index e69de29bb2d..bb0b1cf658d 100644 --- a/tests/queries/0_stateless/02932_set_ttl_where.reference +++ b/tests/queries/0_stateless/02932_set_ttl_where.reference @@ -0,0 +1,3 @@ +0 +0 +0 diff --git a/tests/queries/0_stateless/02932_set_ttl_where.sql b/tests/queries/0_stateless/02932_set_ttl_where.sql index 85fddf613e8..bf2b317c4bf 100644 --- a/tests/queries/0_stateless/02932_set_ttl_where.sql +++ b/tests/queries/0_stateless/02932_set_ttl_where.sql @@ -1,18 +1,10 @@ -create or replace table temp ( - a UInt32 -) -engine = MergeTree -order by a; - -insert into temp select number from system.numbers limit 100_000; - create or replace table t_temp ( a UInt32, timestamp DateTime ) engine = MergeTree order by a -TTL timestamp + INTERVAL 2 SECOND WHERE a in (select a from temp); +TTL timestamp + INTERVAL 2 SECOND WHERE a in (select number from system.numbers limit 100_000); select sleep(1); insert into t_temp select rand(), now() from system.numbers limit 1_000_000; From 7dc7062dadd5ddf3bed3dea4364cabfa97bcd61a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 6 Dec 2023 12:53:14 +0000 Subject: [PATCH 011/276] Fixing test. --- src/Interpreters/PreparedSets.cpp | 3 ++- src/Interpreters/Set.h | 1 + 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index ea8d9a62b8b..9f646825d9f 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -189,7 +189,8 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) } } - set_and_key->set->fillSetElements(); + if (!set_and_key->set->hasSetElements()) + set_and_key->set->fillSetElements(); return buildSetInplace(context); } diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index 7136b090c42..7e8e0f2371b 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -77,6 +77,7 @@ public: const DataTypes & getElementsTypes() const { return set_elements_types; } bool hasExplicitSetElements() const { return fill_set_elements || (!set_elements.empty() && set_elements.front()->size() == data.getTotalRowCount()); } + bool hasSetElements() const { return !set_elements.empty(); } Columns getSetElements() const { checkIsCreated(); return { set_elements.begin(), set_elements.end() }; } void checkColumnsNumber(size_t num_key_columns) const; From a924b01a023512727d6a36fc12052f67438ba199 Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Tue, 19 Dec 2023 02:05:32 -0800 Subject: [PATCH 012/276] [Docs] Clarify to use query level settings in ClickHouse Cloud --- docs/en/operations/query-cache.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/operations/query-cache.md b/docs/en/operations/query-cache.md index def0f48b968..2f05599e666 100644 --- a/docs/en/operations/query-cache.md +++ b/docs/en/operations/query-cache.md @@ -31,6 +31,10 @@ This reduces maintenance effort and avoids redundancy. ## Configuration Settings and Usage +:::note +In ClickHouse Cloud, you must use [query level settings](/en/operations/settings/query-level) to edit query cache settings. Editing [config level settings](/en/operations/configuration-files) is currently not supported. +::: + Setting [use_query_cache](settings/settings.md#use-query-cache) can be used to control whether a specific query or all queries of the current session should utilize the query cache. For example, the first execution of query From bc757559c9f3fd1943bf338dc4fdac9e0e61240a Mon Sep 17 00:00:00 2001 From: una Date: Sat, 23 Dec 2023 18:10:42 +0800 Subject: [PATCH 013/276] feat:add InitialQuery event --- src/Common/ProfileEvents.cpp | 1 + src/Databases/DatabaseReplicatedWorker.cpp | 7 +++++-- src/Interpreters/DDLWorker.cpp | 2 +- .../queries/0_stateless/02950_initialquery_event.reference | 1 + tests/queries/0_stateless/02950_initialquery_event.sql | 1 + 5 files changed, 9 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02950_initialquery_event.reference create mode 100644 tests/queries/0_stateless/02950_initialquery_event.sql diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index f342a19b2aa..a2dc7f5ecd6 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -8,6 +8,7 @@ M(Query, "Number of queries to be interpreted and potentially executed. Does not include queries that failed to parse or were rejected due to AST size limits, quota limits or limits on the number of simultaneously running queries. May include internal queries initiated by ClickHouse itself. Does not count subqueries.") \ M(SelectQuery, "Same as Query, but only for SELECT queries.") \ M(InsertQuery, "Same as Query, but only for INSERT queries.") \ + M(InitialQuery, "Same as Query, but only counts initial queries (see is_initial_query).")\ M(QueriesWithSubqueries, "Count queries with all subqueries") \ M(SelectQueriesWithSubqueries, "Count SELECT queries with all subqueries") \ M(InsertQueriesWithSubqueries, "Count INSERT queries with all subqueries") \ diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index 2056b403ff6..c90af7d4ea8 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -6,7 +6,10 @@ #include namespace fs = std::filesystem; - +namespace ProfileEvents +{ + extern const Event InitialQuery; +} namespace DB { @@ -264,7 +267,7 @@ String DatabaseReplicatedDDLWorker::tryEnqueueAndExecuteEntry(DDLLogEntry & entr chassert(!task->entry.query.empty()); assert(!zookeeper->exists(task->getFinishedNodePath())); task->is_initial_query = true; - + ProfileEvents::increment(ProfileEvents::InitialQuery); LOG_DEBUG(log, "Waiting for worker thread to process all entries before {}", entry_name); UInt64 timeout = query_context->getSettingsRef().database_replicated_initial_query_timeout_sec; { diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index f08fd72ff7f..ac3af6e441c 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -490,7 +490,7 @@ bool DDLWorker::tryExecuteQuery(DDLTaskBase & task, const ZooKeeperPtr & zookeep if (!task.is_initial_query) query_scope.emplace(query_context); - + executeQuery(istr, ostr, !task.is_initial_query, query_context, {}, QueryFlags{ .internal = false, .distributed_backup_restore = task.entry.is_backup_restore }); if (auto txn = query_context->getZooKeeperMetadataTransaction()) diff --git a/tests/queries/0_stateless/02950_initialquery_event.reference b/tests/queries/0_stateless/02950_initialquery_event.reference new file mode 100644 index 00000000000..7ad67a1e7e4 --- /dev/null +++ b/tests/queries/0_stateless/02950_initialquery_event.reference @@ -0,0 +1 @@ +InitialQuery 6 Same as Query, but only counts initial queries (see is_initial_query). diff --git a/tests/queries/0_stateless/02950_initialquery_event.sql b/tests/queries/0_stateless/02950_initialquery_event.sql new file mode 100644 index 00000000000..2b03607c5c7 --- /dev/null +++ b/tests/queries/0_stateless/02950_initialquery_event.sql @@ -0,0 +1 @@ +SELECT * FROM system.events where event = 'InitialQuery' \ No newline at end of file From b38e7060ef455e6ae569d371203309a1ad992c66 Mon Sep 17 00:00:00 2001 From: una Date: Sat, 23 Dec 2023 18:36:23 +0800 Subject: [PATCH 014/276] feat:add InitialQuery event --- src/Common/ProfileEvents.cpp | 1 + src/Databases/DatabaseReplicatedWorker.cpp | 7 +++++-- src/Interpreters/DDLWorker.cpp | 2 +- .../queries/0_stateless/02950_initialquery_event.reference | 1 + tests/queries/0_stateless/02950_initialquery_event.sql | 1 + 5 files changed, 9 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02950_initialquery_event.reference create mode 100644 tests/queries/0_stateless/02950_initialquery_event.sql diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index f342a19b2aa..a2dc7f5ecd6 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -8,6 +8,7 @@ M(Query, "Number of queries to be interpreted and potentially executed. Does not include queries that failed to parse or were rejected due to AST size limits, quota limits or limits on the number of simultaneously running queries. May include internal queries initiated by ClickHouse itself. Does not count subqueries.") \ M(SelectQuery, "Same as Query, but only for SELECT queries.") \ M(InsertQuery, "Same as Query, but only for INSERT queries.") \ + M(InitialQuery, "Same as Query, but only counts initial queries (see is_initial_query).")\ M(QueriesWithSubqueries, "Count queries with all subqueries") \ M(SelectQueriesWithSubqueries, "Count SELECT queries with all subqueries") \ M(InsertQueriesWithSubqueries, "Count INSERT queries with all subqueries") \ diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index 2056b403ff6..c90af7d4ea8 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -6,7 +6,10 @@ #include namespace fs = std::filesystem; - +namespace ProfileEvents +{ + extern const Event InitialQuery; +} namespace DB { @@ -264,7 +267,7 @@ String DatabaseReplicatedDDLWorker::tryEnqueueAndExecuteEntry(DDLLogEntry & entr chassert(!task->entry.query.empty()); assert(!zookeeper->exists(task->getFinishedNodePath())); task->is_initial_query = true; - + ProfileEvents::increment(ProfileEvents::InitialQuery); LOG_DEBUG(log, "Waiting for worker thread to process all entries before {}", entry_name); UInt64 timeout = query_context->getSettingsRef().database_replicated_initial_query_timeout_sec; { diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index f08fd72ff7f..ac3af6e441c 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -490,7 +490,7 @@ bool DDLWorker::tryExecuteQuery(DDLTaskBase & task, const ZooKeeperPtr & zookeep if (!task.is_initial_query) query_scope.emplace(query_context); - + executeQuery(istr, ostr, !task.is_initial_query, query_context, {}, QueryFlags{ .internal = false, .distributed_backup_restore = task.entry.is_backup_restore }); if (auto txn = query_context->getZooKeeperMetadataTransaction()) diff --git a/tests/queries/0_stateless/02950_initialquery_event.reference b/tests/queries/0_stateless/02950_initialquery_event.reference new file mode 100644 index 00000000000..7ad67a1e7e4 --- /dev/null +++ b/tests/queries/0_stateless/02950_initialquery_event.reference @@ -0,0 +1 @@ +InitialQuery 6 Same as Query, but only counts initial queries (see is_initial_query). diff --git a/tests/queries/0_stateless/02950_initialquery_event.sql b/tests/queries/0_stateless/02950_initialquery_event.sql new file mode 100644 index 00000000000..2b03607c5c7 --- /dev/null +++ b/tests/queries/0_stateless/02950_initialquery_event.sql @@ -0,0 +1 @@ +SELECT * FROM system.events where event = 'InitialQuery' \ No newline at end of file From 3e22f29b4529b6fefd5e92616ce9ef1ac33966d0 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sat, 23 Dec 2023 11:40:58 +0100 Subject: [PATCH 015/276] Fixed parameters --- docs/en/operations/backup.md | 2 +- .../registerBackupEngineAzureBlobStorage.cpp | 25 +++++++++++++++---- .../test.py | 2 +- 3 files changed, 22 insertions(+), 7 deletions(-) diff --git a/docs/en/operations/backup.md b/docs/en/operations/backup.md index 15d953249a0..4871f97c270 100644 --- a/docs/en/operations/backup.md +++ b/docs/en/operations/backup.md @@ -463,7 +463,7 @@ To write backups to an AzureBlobStorage container you need the following pieces The destination for a backup will be specified like this: ``` -AzureBlobStorage('/', '', '', '', ') +AzureBlobStorage('/', '', '', '', '') ``` ```sql diff --git a/src/Backups/registerBackupEngineAzureBlobStorage.cpp b/src/Backups/registerBackupEngineAzureBlobStorage.cpp index 6f7b5f38c28..ef95206831f 100644 --- a/src/Backups/registerBackupEngineAzureBlobStorage.cpp +++ b/src/Backups/registerBackupEngineAzureBlobStorage.cpp @@ -22,6 +22,7 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int SUPPORT_IS_DISABLED; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } #if USE_AZURE_BLOB_STORAGE @@ -54,20 +55,34 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) StorageAzureBlob::Configuration configuration; - if (args.size() == 4) + if (args.size() == 3) { configuration.connection_url = args[0].safeGet(); configuration.is_connection_string = true; configuration.container = args[1].safeGet(); configuration.blob_path = args[2].safeGet(); - configuration.format = args[3].safeGet(); LOG_TRACE(&Poco::Logger::get("registerBackupEngineAzureBlobStorage"), "configuration.connection_url = {}" "configuration.container = {}" - "configuration.blob_path = {}" - "configuration.format = {}", - configuration.connection_url, configuration.container, configuration.blob_path, configuration.format); + "configuration.blob_path = {}", + configuration.connection_url, configuration.container, configuration.blob_path); + } + else if (args.size() == 5) + { + configuration.connection_url = args[0].safeGet(); + configuration.is_connection_string = false; + + configuration.container = args[1].safeGet(); + configuration.blob_path = args[2].safeGet(); + configuration.account_name = args[3].safeGet(); + configuration.account_key = args[4].safeGet(); + + } + else + { + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Backup AzureBlobStorage requires 3 or 5 arguments: connection string>/ Date: Sat, 23 Dec 2023 18:42:41 +0800 Subject: [PATCH 016/276] feat:add InitialQuery event --- src/Interpreters/DDLWorker.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index ac3af6e441c..f08fd72ff7f 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -490,7 +490,7 @@ bool DDLWorker::tryExecuteQuery(DDLTaskBase & task, const ZooKeeperPtr & zookeep if (!task.is_initial_query) query_scope.emplace(query_context); - + executeQuery(istr, ostr, !task.is_initial_query, query_context, {}, QueryFlags{ .internal = false, .distributed_backup_restore = task.entry.is_backup_restore }); if (auto txn = query_context->getZooKeeperMetadataTransaction()) From fa5dde0bff8f34ebe85e1cc6e929f834c5e6b496 Mon Sep 17 00:00:00 2001 From: una Date: Wed, 27 Dec 2023 12:37:06 +0800 Subject: [PATCH 017/276] feat: Add initial query event --- src/Databases/DatabaseReplicatedWorker.cpp | 6 +-- src/Interpreters/InterpreterFactory.cpp | 5 +- ..._distributed_initial_query_event.reference | 6 +++ .../02950_distributed_initial_query_event.sh | 54 +++++++++++++++++++ .../02950_initialquery_event.reference | 1 - .../0_stateless/02950_initialquery_event.sql | 1 - 6 files changed, 66 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/02950_distributed_initial_query_event.reference create mode 100644 tests/queries/0_stateless/02950_distributed_initial_query_event.sh delete mode 100644 tests/queries/0_stateless/02950_initialquery_event.reference delete mode 100644 tests/queries/0_stateless/02950_initialquery_event.sql diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index c90af7d4ea8..317cda3cd3d 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -7,9 +7,7 @@ namespace fs = std::filesystem; namespace ProfileEvents -{ - extern const Event InitialQuery; -} + namespace DB { @@ -267,7 +265,7 @@ String DatabaseReplicatedDDLWorker::tryEnqueueAndExecuteEntry(DDLLogEntry & entr chassert(!task->entry.query.empty()); assert(!zookeeper->exists(task->getFinishedNodePath())); task->is_initial_query = true; - ProfileEvents::increment(ProfileEvents::InitialQuery); + LOG_DEBUG(log, "Waiting for worker thread to process all entries before {}", entry_name); UInt64 timeout = query_context->getSettingsRef().database_replicated_initial_query_timeout_sec; { diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index e32cbe4ccad..fdf7e8ebfbb 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -120,6 +120,7 @@ namespace ProfileEvents { extern const Event Query; + extern const Event InitialQuery; extern const Event QueriesWithSubqueries; extern const Event SelectQuery; extern const Event InsertQuery; @@ -137,7 +138,9 @@ namespace ErrorCodes std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextMutablePtr context, const SelectQueryOptions & options) { ProfileEvents::increment(ProfileEvents::Query); - + + if (context->getClientInfo().query_kind == ClientInfo::QueryKind::INITIAL_QUERY) + ProfileEvents::increment(ProfileEvents::InitialQuery); /// SELECT and INSERT query will handle QueriesWithSubqueries on their own. if (!(query->as() || query->as() || diff --git a/tests/queries/0_stateless/02950_distributed_initial_query_event.reference b/tests/queries/0_stateless/02950_distributed_initial_query_event.reference new file mode 100644 index 00000000000..af8542c7204 --- /dev/null +++ b/tests/queries/0_stateless/02950_distributed_initial_query_event.reference @@ -0,0 +1,6 @@ +Local situation +Initial Query Difference: 1 +Query Difference: 1 +Distributed situation +Initial Query Difference: 1 +Query Difference: 2 diff --git a/tests/queries/0_stateless/02950_distributed_initial_query_event.sh b/tests/queries/0_stateless/02950_distributed_initial_query_event.sh new file mode 100644 index 00000000000..3a01aa63d87 --- /dev/null +++ b/tests/queries/0_stateless/02950_distributed_initial_query_event.sh @@ -0,0 +1,54 @@ +-- Tags: distributed + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh +# CREATE TABLE local (x UInt8) Engine=Memory; +# CREATE TABLE distributed ON CLUSTER cluster (p Date, i Int32) ENGINE = Distributed(test_cluster_two_shards, currentDatabase(), x) +$CLICKHOUSE_CLIENT -n -q " +DROP TABLE IF EXISTS local; +DROP TABLE IF EXISTS distributed; +CREATE TABLE local (x UInt8) Engine=Memory; +CREATE TABLE distributed AS local ENGINE = Distributed(test_cluster_two_shards, currentDatabase(), local, x); +INSERT INTO distributed SELECT number FROM numbers(10); +SYSTEM FLUSH DISTRIBUTED distributed; +" +echo "Local situation" +# before SELECT * FROM local +query_countI=$($CLICKHOUSE_CLIENT -q "SELECT value FROM system.events WHERE event = 'InitialQuery'") +query_countQ=$($CLICKHOUSE_CLIENT -q "SELECT value FROM system.events WHERE event = 'Query'") + +# Execute SELECT * FROM local +$CLICKHOUSE_CLIENT -q "SELECT * FROM local" > /dev/null + +# Counts after SELECT * FROM local +After_query_countI=$($CLICKHOUSE_CLIENT -q "SELECT value FROM system.events WHERE event = 'InitialQuery'") +After_query_countQ=$($CLICKHOUSE_CLIENT -q "SELECT value FROM system.events WHERE event = 'Query'") + +# Calculate the differences +Initial_query_diff=$(($After_query_countI-$query_countI-2)) +query_diff=$(($After_query_countQ-$query_countQ-2)) + +echo "Initial Query Difference: $Initial_query_diff" +echo "Query Difference: $query_diff" +echo "Distributed situation" + +# before SELECT * FROM distributed +query_countI=$($CLICKHOUSE_CLIENT -q "SELECT value FROM system.events WHERE event = 'InitialQuery'") +query_countQ=$($CLICKHOUSE_CLIENT -q "SELECT value FROM system.events WHERE event = 'Query'") + +# Execute SELECT * FROM distributed +$CLICKHOUSE_CLIENT -q "SELECT * FROM distributed" > /dev/null + +# Counts after SELECT * FROM distributed +After_query_countI=$($CLICKHOUSE_CLIENT -q "SELECT value FROM system.events WHERE event = 'InitialQuery'") +After_query_countQ=$($CLICKHOUSE_CLIENT -q "SELECT value FROM system.events WHERE event = 'Query'") + +# Calculate the differences +Initial_query_diff=$(($After_query_countI-$query_countI-2)) +query_diff=$(($After_query_countQ-$query_countQ-2)) + +echo "Initial Query Difference: $Initial_query_diff" +echo "Query Difference: $query_diff" + + diff --git a/tests/queries/0_stateless/02950_initialquery_event.reference b/tests/queries/0_stateless/02950_initialquery_event.reference deleted file mode 100644 index 7ad67a1e7e4..00000000000 --- a/tests/queries/0_stateless/02950_initialquery_event.reference +++ /dev/null @@ -1 +0,0 @@ -InitialQuery 6 Same as Query, but only counts initial queries (see is_initial_query). diff --git a/tests/queries/0_stateless/02950_initialquery_event.sql b/tests/queries/0_stateless/02950_initialquery_event.sql deleted file mode 100644 index 2b03607c5c7..00000000000 --- a/tests/queries/0_stateless/02950_initialquery_event.sql +++ /dev/null @@ -1 +0,0 @@ -SELECT * FROM system.events where event = 'InitialQuery' \ No newline at end of file From 1464c3d1aab8c6ecdc369facceb1b9f6cf4b36fb Mon Sep 17 00:00:00 2001 From: una Date: Wed, 27 Dec 2023 15:13:21 +0800 Subject: [PATCH 018/276] feat: Add initial query event --- src/Databases/DatabaseReplicatedWorker.cpp | 3 +-- .../02950_distributed_initial_query_event.reference | 2 +- .../0_stateless/02950_distributed_initial_query_event.sh | 7 +++---- 3 files changed, 5 insertions(+), 7 deletions(-) diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index 317cda3cd3d..2056b403ff6 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -6,7 +6,6 @@ #include namespace fs = std::filesystem; -namespace ProfileEvents namespace DB { @@ -265,7 +264,7 @@ String DatabaseReplicatedDDLWorker::tryEnqueueAndExecuteEntry(DDLLogEntry & entr chassert(!task->entry.query.empty()); assert(!zookeeper->exists(task->getFinishedNodePath())); task->is_initial_query = true; - + LOG_DEBUG(log, "Waiting for worker thread to process all entries before {}", entry_name); UInt64 timeout = query_context->getSettingsRef().database_replicated_initial_query_timeout_sec; { diff --git a/tests/queries/0_stateless/02950_distributed_initial_query_event.reference b/tests/queries/0_stateless/02950_distributed_initial_query_event.reference index af8542c7204..cf10427e9b3 100644 --- a/tests/queries/0_stateless/02950_distributed_initial_query_event.reference +++ b/tests/queries/0_stateless/02950_distributed_initial_query_event.reference @@ -3,4 +3,4 @@ Initial Query Difference: 1 Query Difference: 1 Distributed situation Initial Query Difference: 1 -Query Difference: 2 +Query Difference: 3 diff --git a/tests/queries/0_stateless/02950_distributed_initial_query_event.sh b/tests/queries/0_stateless/02950_distributed_initial_query_event.sh index 3a01aa63d87..c8a955c4fe5 100644 --- a/tests/queries/0_stateless/02950_distributed_initial_query_event.sh +++ b/tests/queries/0_stateless/02950_distributed_initial_query_event.sh @@ -1,4 +1,5 @@ --- Tags: distributed +#!/usr/bin/env bash +# Tags:no-parallel shard CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -38,7 +39,7 @@ query_countI=$($CLICKHOUSE_CLIENT -q "SELECT value FROM system.events WHERE even query_countQ=$($CLICKHOUSE_CLIENT -q "SELECT value FROM system.events WHERE event = 'Query'") # Execute SELECT * FROM distributed -$CLICKHOUSE_CLIENT -q "SELECT * FROM distributed" > /dev/null +$CLICKHOUSE_CLIENT -q "SELECT * FROM distributed SETTINGS prefer_localhost_replica = 0" > /dev/null # Counts after SELECT * FROM distributed After_query_countI=$($CLICKHOUSE_CLIENT -q "SELECT value FROM system.events WHERE event = 'InitialQuery'") @@ -50,5 +51,3 @@ query_diff=$(($After_query_countQ-$query_countQ-2)) echo "Initial Query Difference: $Initial_query_diff" echo "Query Difference: $query_diff" - - From 22e1bcb9d638d5df0c43585b1d78228beedb0dc8 Mon Sep 17 00:00:00 2001 From: una Date: Wed, 27 Dec 2023 16:12:10 +0800 Subject: [PATCH 019/276] feat:add InitialQuery event Signed-off-by: una --- .../0_stateless/02950_distributed_initial_query_event.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02950_distributed_initial_query_event.sh b/tests/queries/0_stateless/02950_distributed_initial_query_event.sh index c8a955c4fe5..ddd0fb1e408 100644 --- a/tests/queries/0_stateless/02950_distributed_initial_query_event.sh +++ b/tests/queries/0_stateless/02950_distributed_initial_query_event.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags:no-parallel shard +# Tags:no-parallel, shard CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From a6f2eaf5a6ba2a26943d0c1c53c7cf7460a7471d Mon Sep 17 00:00:00 2001 From: una Date: Wed, 27 Dec 2023 16:19:06 +0800 Subject: [PATCH 020/276] fix:use , to split tags Signed-off-by: una --- .../0_stateless/02950_distributed_initial_query_event.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02950_distributed_initial_query_event.sh b/tests/queries/0_stateless/02950_distributed_initial_query_event.sh index ddd0fb1e408..7f690a681c4 100644 --- a/tests/queries/0_stateless/02950_distributed_initial_query_event.sh +++ b/tests/queries/0_stateless/02950_distributed_initial_query_event.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags:no-parallel, shard +# Tags:no-parallel,shard CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From d46d91452176414426e40f598a7a1aa989f1a584 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 27 Dec 2023 10:28:52 +0100 Subject: [PATCH 021/276] Updated thread name --- src/Backups/BackupIO_AzureBlobStorage.cpp | 8 +- src/Backups/BackupIO_AzureBlobStorage.h | 81 +++++++++---------- .../copyAzureBlobStorageFile.cpp | 25 +++--- 3 files changed, 59 insertions(+), 55 deletions(-) diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index d41d23e3c36..a1fd5bd8327 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -35,7 +35,7 @@ BackupReaderAzureBlobStorage::BackupReaderAzureBlobStorage( const WriteSettings & write_settings_, const ContextPtr & context_) : BackupReaderDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupReaderAzureBlobStorage")) - , data_source_description{DataSourceType::AzureBlobStorage, "AzureBlobStorage", false, false} + , data_source_description{DataSourceType::AzureBlobStorage, configuration_.container, false, false} , configuration(configuration_) { client = StorageAzureBlob::createClient(configuration, /* is_read_only */ false); @@ -160,7 +160,7 @@ BackupWriterAzureBlobStorage::BackupWriterAzureBlobStorage( const WriteSettings & write_settings_, const ContextPtr & context_) : BackupWriterDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupWriterAzureBlobStorage")) - , data_source_description{DataSourceType::AzureBlobStorage, "AzureBlobStorage", false, false} + , data_source_description{DataSourceType::AzureBlobStorage,configuration_.container, false, false} , configuration(configuration_) { client = StorageAzureBlob::createClient(configuration, /* is_read_only */ false); @@ -209,7 +209,7 @@ void BackupWriterAzureBlobStorage::copyFileFromDisk(const String & path_in_backu settings, read_settings, {}, - threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupWriterAzureBlobStorage")); + threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupWriterS3")); return; /// copied! } } @@ -221,7 +221,7 @@ void BackupWriterAzureBlobStorage::copyFileFromDisk(const String & path_in_backu void BackupWriterAzureBlobStorage::copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) { copyDataToAzureBlobStorageFile(create_read_buffer, start_pos, length, client, configuration.container, path_in_backup, settings, {}, - threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupWriterAzureBlobStorage")); + threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupWriterS3")); } BackupWriterAzureBlobStorage::~BackupWriterAzureBlobStorage() = default; diff --git a/src/Backups/BackupIO_AzureBlobStorage.h b/src/Backups/BackupIO_AzureBlobStorage.h index 6ef66fc432d..65affb9f079 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.h +++ b/src/Backups/BackupIO_AzureBlobStorage.h @@ -12,57 +12,54 @@ namespace DB { -// using AzureClientPtr = std::shared_ptr; - /// Represents a backup stored to Azure - class BackupReaderAzureBlobStorage : public BackupReaderDefault - { - public: - BackupReaderAzureBlobStorage(StorageAzureBlob::Configuration configuration_, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_); - ~BackupReaderAzureBlobStorage() override; +class BackupReaderAzureBlobStorage : public BackupReaderDefault +{ +public: + BackupReaderAzureBlobStorage(StorageAzureBlob::Configuration configuration_, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_); + ~BackupReaderAzureBlobStorage() override; - bool fileExists(const String & file_name) override; - UInt64 getFileSize(const String & file_name) override; - std::unique_ptr readFile(const String & file_name) override; + bool fileExists(const String & file_name) override; + UInt64 getFileSize(const String & file_name) override; + std::unique_ptr readFile(const String & file_name) override; - void copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, - DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) override; + void copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, + DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) override; - private: - const DataSourceDescription data_source_description; - std::shared_ptr client; - StorageAzureBlob::Configuration configuration; - std::unique_ptr object_storage; - std::shared_ptr settings; - }; +private: + const DataSourceDescription data_source_description; + std::shared_ptr client; + StorageAzureBlob::Configuration configuration; + std::unique_ptr object_storage; + std::shared_ptr settings; +}; +class BackupWriterAzureBlobStorage : public BackupWriterDefault +{ +public: + BackupWriterAzureBlobStorage(StorageAzureBlob::Configuration configuration_, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_); + ~BackupWriterAzureBlobStorage() override; - class BackupWriterAzureBlobStorage : public BackupWriterDefault - { - public: - BackupWriterAzureBlobStorage(StorageAzureBlob::Configuration configuration_, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_); - ~BackupWriterAzureBlobStorage() override; + bool fileExists(const String & file_name) override; + UInt64 getFileSize(const String & file_name) override; + std::unique_ptr writeFile(const String & file_name) override; - bool fileExists(const String & file_name) override; - UInt64 getFileSize(const String & file_name) override; - std::unique_ptr writeFile(const String & file_name) override; + void copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) override; + void copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, + bool copy_encrypted, UInt64 start_pos, UInt64 length) override; - void copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) override; - void copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, - bool copy_encrypted, UInt64 start_pos, UInt64 length) override; + void removeFile(const String & file_name) override; + void removeFiles(const Strings & file_names) override; - void removeFile(const String & file_name) override; - void removeFiles(const Strings & file_names) override; - - private: - std::unique_ptr readFile(const String & file_name, size_t expected_file_size) override; - void removeFilesBatch(const Strings & file_names); - const DataSourceDescription data_source_description; - std::shared_ptr client; - StorageAzureBlob::Configuration configuration; - std::unique_ptr object_storage; - std::shared_ptr settings; - }; +private: + std::unique_ptr readFile(const String & file_name, size_t expected_file_size) override; + void removeFilesBatch(const Strings & file_names); + const DataSourceDescription data_source_description; + std::shared_ptr client; + StorageAzureBlob::Configuration configuration; + std::unique_ptr object_storage; + std::shared_ptr settings; +}; } diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp index bf0bcac664b..0a0a080b5cb 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp @@ -22,6 +22,11 @@ namespace ProfileEvents extern const Event DiskAzureUploadPart; } +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + namespace DB { @@ -44,7 +49,8 @@ namespace std::shared_ptr settings_, const std::optional> & object_metadata_, ThreadPoolCallbackRunner schedule_, - bool for_disk_azure_blob_storage_) + bool for_disk_azure_blob_storage_, + const Poco::Logger * log_) : create_read_buffer(create_read_buffer_) , client(client_) , offset (offset_) @@ -55,7 +61,7 @@ namespace , object_metadata(object_metadata_) , schedule(schedule_) , for_disk_azure_blob_storage(for_disk_azure_blob_storage_) - , log(&Poco::Logger::get("azureBlobStorageUploadHelper")) + , log(log_) , max_single_part_upload_size(settings_.get()->max_single_part_upload_size) { } @@ -179,11 +185,11 @@ namespace try { auto read_buffer = std::make_unique(create_read_buffer(), part_offset, part_size); - auto buffer = std::make_unique(std::move(read_buffer), part_size); task->data = new char[part_size]; task->size = part_size; - buffer->read(task->data,part_size); - task->block_id = getRandomASCIIString(64); + size_t n = read_buffer->read(task->data,part_size); + if (n != part_size) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected size"); schedule([this, task, task_finish_notify]() { @@ -208,9 +214,10 @@ namespace { UploadPartTask task; auto read_buffer = std::make_unique(create_read_buffer(), part_offset, part_size); - auto buffer = std::make_unique(std::move(read_buffer), part_size); task.data = new char[part_size]; - buffer->read(task.data,part_size); + size_t n = read_buffer->read(task.data,part_size); + if (n != part_size) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected size"); task.size = part_size; processUploadTask(task); block_ids.emplace_back(task.block_id); @@ -274,7 +281,7 @@ void copyDataToAzureBlobStorageFile( ThreadPoolCallbackRunner schedule, bool for_disk_azure_blob_storage) { - UploadHelper helper{create_read_buffer, dest_client, offset, size, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_azure_blob_storage}; + UploadHelper helper{create_read_buffer, dest_client, offset, size, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_azure_blob_storage, &Poco::Logger::get("copyDataToAzureBlobStorageFile")}; helper.performCopy(); } @@ -314,7 +321,7 @@ void copyAzureBlobStorageFile( settings->max_single_download_retries); }; - UploadHelper helper{create_read_buffer, dest_client, offset, size, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_azure_blob_storage}; + UploadHelper helper{create_read_buffer, dest_client, offset, size, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_azure_blob_storage, &Poco::Logger::get("copyAzureBlobStorageFile")}; helper.performCopy(); } } From 0181bab23c38c2d1c15f199d522a4743b11586d6 Mon Sep 17 00:00:00 2001 From: una Date: Wed, 27 Dec 2023 19:59:23 +0800 Subject: [PATCH 022/276] fix:style Signed-off-by: una --- src/Interpreters/InterpreterFactory.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index fdf7e8ebfbb..c5d7f0f891c 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -138,7 +138,6 @@ namespace ErrorCodes std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextMutablePtr context, const SelectQueryOptions & options) { ProfileEvents::increment(ProfileEvents::Query); - if (context->getClientInfo().query_kind == ClientInfo::QueryKind::INITIAL_QUERY) ProfileEvents::increment(ProfileEvents::InitialQuery); /// SELECT and INSERT query will handle QueriesWithSubqueries on their own. From 2c1513540768eaed34a13fd643c4ace491421c0e Mon Sep 17 00:00:00 2001 From: una Date: Wed, 27 Dec 2023 20:53:30 +0800 Subject: [PATCH 023/276] fix test-file permissions Signed-off-by: una --- .../queries/0_stateless/02950_distributed_initial_query_event.sh | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100644 => 100755 tests/queries/0_stateless/02950_distributed_initial_query_event.sh diff --git a/tests/queries/0_stateless/02950_distributed_initial_query_event.sh b/tests/queries/0_stateless/02950_distributed_initial_query_event.sh old mode 100644 new mode 100755 From 32ff152f2d7e4798a7bbc916808cc9ca883cf13e Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 28 Dec 2023 11:41:06 +0000 Subject: [PATCH 024/276] Support negtive position arguments --- .../replaceForPositionalArguments.cpp | 24 ++++- .../0_stateless/01798_having_push_down.sql | 3 +- .../02006_test_positional_arguments.reference | 94 +++++++++++++++++++ .../02006_test_positional_arguments.sql | 21 +++++ .../02932_group_by_null_fuzzer.sql | 1 + 5 files changed, 137 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/replaceForPositionalArguments.cpp b/src/Interpreters/replaceForPositionalArguments.cpp index 241dd7cf92c..bea87ad913a 100644 --- a/src/Interpreters/replaceForPositionalArguments.cpp +++ b/src/Interpreters/replaceForPositionalArguments.cpp @@ -27,14 +27,28 @@ bool replaceForPositionalArguments(ASTPtr & argument, const ASTSelectQuery * sel return false; auto which = ast_literal->value.getType(); - if (which != Field::Types::UInt64) + if (which != Field::Types::UInt64 && which != Field::Types::Int64) return false; - auto pos = ast_literal->value.get(); + UInt64 pos; + + if (which == Field::Types::UInt64) + { + pos = ast_literal->value.get(); + } + else if (which == Field::Types::Int64) + { + auto value = ast_literal->value.get(); + pos = value > 0 ? value : columns.size() + value + 1; + } + else + { + return false; + } + if (!pos || pos > columns.size()) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Positional argument out of bounds: {} (expected in range [1, {}]", - pos, columns.size()); + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Positional argument out of bounds: {} (expected in range [1, {}]", pos, columns.size()); const auto & column = columns[--pos]; if (typeid_cast(column.get()) || typeid_cast(column.get())) diff --git a/tests/queries/0_stateless/01798_having_push_down.sql b/tests/queries/0_stateless/01798_having_push_down.sql index b3a77c8f5b5..c0c3447f5ab 100644 --- a/tests/queries/0_stateless/01798_having_push_down.sql +++ b/tests/queries/0_stateless/01798_having_push_down.sql @@ -8,11 +8,12 @@ SELECT sum(c0 = 0), min(c0 + 1), sum(c0 + 2) FROM t_having GROUP BY c0 HAVING c0 = 0 SETTINGS enable_optimize_predicate_expression=0; +SET enable_positional_arguments=0; + SELECT c0 + -1, sum(intDivOrZero(intDivOrZero(NULL, NULL), '2'), intDivOrZero(10000000000., intDivOrZero(intDivOrZero(intDivOrZero(NULL, NULL), 10), NULL))) FROM t_having GROUP BY c0 = 2, c0 = 10, intDivOrZero(intDivOrZero(intDivOrZero(NULL, NULL), NULL), NULL), c0 HAVING c0 = 2 SETTINGS enable_optimize_predicate_expression = 0; SELECT sum(c0 + 257) FROM t_having GROUP BY c0 = -9223372036854775808, NULL, -2147483649, c0 HAVING c0 = -9223372036854775808 SETTINGS enable_optimize_predicate_expression = 0; -SET enable_positional_arguments=0; SELECT c0 + -2, c0 + -9223372036854775807, c0 = NULL FROM t_having GROUP BY c0 = 0.9998999834060669, 1023, c0 HAVING c0 = 0.9998999834060669 SETTINGS enable_optimize_predicate_expression = 0; DROP TABLE t_having; diff --git a/tests/queries/0_stateless/02006_test_positional_arguments.reference b/tests/queries/0_stateless/02006_test_positional_arguments.reference index 40100e8d5be..079bd071103 100644 --- a/tests/queries/0_stateless/02006_test_positional_arguments.reference +++ b/tests/queries/0_stateless/02006_test_positional_arguments.reference @@ -3,18 +3,50 @@ select x3, x2, x1 from test order by 1; 1 100 100 10 1 10 100 10 1 +select x3, x2, x1 from test order by -3; +1 100 100 +10 1 10 +100 10 1 select x3, x2, x1 from test order by x3; 1 100 100 10 1 10 100 10 1 +select x3, x2, x1 from test order by 3; +100 10 1 +10 1 10 +1 100 100 +select x3, x2, x1 from test order by -1; +100 10 1 +10 1 10 +1 100 100 +select x3, x2, x1 from test order by x1; +100 10 1 +10 1 10 +1 100 100 select x3, x2, x1 from test order by 1 desc; 100 10 1 10 1 10 1 100 100 +select x3, x2, x1 from test order by -3 desc; +100 10 1 +10 1 10 +1 100 100 select x3, x2, x1 from test order by x3 desc; 100 10 1 10 1 10 1 100 100 +select x3, x2, x1 from test order by 3 desc; +1 100 100 +10 1 10 +100 10 1 +select x3, x2, x1 from test order by -1 desc; +1 100 100 +10 1 10 +100 10 1 +select x3, x2, x1 from test order by x1 desc; +1 100 100 +10 1 10 +100 10 1 insert into test values (1, 10, 100), (10, 1, 10), (100, 100, 1); select x3, x2 from test group by x3, x2 order by x3; 1 100 @@ -54,6 +86,20 @@ SELECT x1 FROM test ORDER BY x3 + 1 ASC +explain syntax select x3, x2, x1 from test order by -1; +SELECT + x3, + x2, + x1 +FROM test +ORDER BY x1 ASC +explain syntax select x3 + 1, x2, x1 from test order by -1; +SELECT + x3 + 1, + x2, + x1 +FROM test +ORDER BY x1 ASC explain syntax select x3, x3 - x2, x2, x1 from test order by 2; SELECT x3, @@ -62,6 +108,14 @@ SELECT x1 FROM test ORDER BY x3 - x2 ASC +explain syntax select x3, x3 - x2, x2, x1 from test order by -2; +SELECT + x3, + x3 - x2, + x2, + x1 +FROM test +ORDER BY x2 ASC explain syntax select x3, if(x3 > 10, x3, plus(x1, x2)), x1 + x2 from test order by 2; SELECT x3, @@ -69,12 +123,28 @@ SELECT x1 + x2 FROM test ORDER BY if(x3 > 10, x3, x1 + x2) ASC +explain syntax select x3, if(x3 > 10, x3, plus(x1, x2)), x1 + x2 from test order by -2; +SELECT + x3, + if(x3 > 10, x3, x1 + x2), + x1 + x2 +FROM test +ORDER BY if(x3 > 10, x3, x1 + x2) ASC explain syntax select max(x1), x2 from test group by 2 order by 1, 2; SELECT max(x1), x2 FROM test GROUP BY x2 +ORDER BY + max(x1) ASC, + x2 ASC +explain syntax select max(x1), x2 from test group by -1 order by -2, -1; +SELECT + max(x1), + x2 +FROM test +GROUP BY x2 ORDER BY max(x1) ASC, x2 ASC @@ -83,16 +153,34 @@ SELECT 1 + greatest(x1, 1), x2 FROM test +GROUP BY + 1 + greatest(x1, 1), + x2 +explain syntax select 1 + greatest(x1, 1), x2 from test group by -2, -1; +SELECT + 1 + greatest(x1, 1), + x2 +FROM test GROUP BY 1 + greatest(x1, 1), x2 select max(x1), x2 from test group by 1, 2; -- { serverError 43, 184 } select 1 + max(x1), x2 from test group by 1, 2; -- { serverError 43, 184 } +select max(x1), x2 from test group by -2, -1; -- { serverError 43, 184 } +select 1 + max(x1), x2 from test group by -2, -1; -- { serverError 43, 184 } explain syntax select x1 + x3, x3 from test group by 1, 2; SELECT x1 + x3, x3 FROM test +GROUP BY + x1 + x3, + x3 +explain syntax select x1 + x3, x3 from test group by -2, -1; +SELECT + x1 + x3, + x3 +FROM test GROUP BY x1 + x3, x3 @@ -102,8 +190,14 @@ select x1, x1 * 2, max(x2), max(x3) from test2 group by 2, 1, x1 order by 1, 2, 1 2 10 100 10 20 1 10 100 200 100 1 +select x1, x1 * 2, max(x2), max(x3) from test2 group by 2, 1, x1 order by 1, 2, -1 desc, -2 asc; +1 2 10 100 +10 20 1 10 +100 200 100 1 select a, b, c, d, e, f from (select 44 a, 88 b, 13 c, 14 d, 15 e, 16 f) t group by 1,2,3,4,5,6 order by a; 44 88 13 14 15 16 +select a, b, c, d, e, f from (select 44 a, 88 b, 13 c, 14 d, 15 e, 16 f) t group by 1,2,3,-3,-2,-1 order by a; +44 88 13 14 15 16 explain syntax select plus(1, 1) as a group by a; SELECT 1 + 1 AS a GROUP BY a diff --git a/tests/queries/0_stateless/02006_test_positional_arguments.sql b/tests/queries/0_stateless/02006_test_positional_arguments.sql index 159ad6bd427..6f427e0298d 100644 --- a/tests/queries/0_stateless/02006_test_positional_arguments.sql +++ b/tests/queries/0_stateless/02006_test_positional_arguments.sql @@ -9,11 +9,21 @@ insert into test values (1, 10, 100), (10, 1, 10), (100, 100, 1); -- { echo } select x3, x2, x1 from test order by 1; +select x3, x2, x1 from test order by -3; select x3, x2, x1 from test order by x3; +select x3, x2, x1 from test order by 3; +select x3, x2, x1 from test order by -1; +select x3, x2, x1 from test order by x1; + select x3, x2, x1 from test order by 1 desc; +select x3, x2, x1 from test order by -3 desc; select x3, x2, x1 from test order by x3 desc; +select x3, x2, x1 from test order by 3 desc; +select x3, x2, x1 from test order by -1 desc; +select x3, x2, x1 from test order by x1 desc; + insert into test values (1, 10, 100), (10, 1, 10), (100, 100, 1); select x3, x2 from test group by x3, x2 order by x3; select x3, x2 from test group by 1, 2 order by x3; @@ -25,21 +35,32 @@ select x1, x2, x3 from test order by 3 limit 1 by 1; explain syntax select x3, x2, x1 from test order by 1; explain syntax select x3 + 1, x2, x1 from test order by 1; +explain syntax select x3, x2, x1 from test order by -1; +explain syntax select x3 + 1, x2, x1 from test order by -1; explain syntax select x3, x3 - x2, x2, x1 from test order by 2; +explain syntax select x3, x3 - x2, x2, x1 from test order by -2; explain syntax select x3, if(x3 > 10, x3, plus(x1, x2)), x1 + x2 from test order by 2; +explain syntax select x3, if(x3 > 10, x3, plus(x1, x2)), x1 + x2 from test order by -2; explain syntax select max(x1), x2 from test group by 2 order by 1, 2; +explain syntax select max(x1), x2 from test group by -1 order by -2, -1; explain syntax select 1 + greatest(x1, 1), x2 from test group by 1, 2; +explain syntax select 1 + greatest(x1, 1), x2 from test group by -2, -1; select max(x1), x2 from test group by 1, 2; -- { serverError 43, 184 } select 1 + max(x1), x2 from test group by 1, 2; -- { serverError 43, 184 } +select max(x1), x2 from test group by -2, -1; -- { serverError 43, 184 } +select 1 + max(x1), x2 from test group by -2, -1; -- { serverError 43, 184 } explain syntax select x1 + x3, x3 from test group by 1, 2; +explain syntax select x1 + x3, x3 from test group by -2, -1; create table test2(x1 Int, x2 Int, x3 Int) engine=Memory; insert into test2 values (1, 10, 100), (10, 1, 10), (100, 100, 1); select x1, x1 * 2, max(x2), max(x3) from test2 group by 2, 1, x1 order by 1, 2, 4 desc, 3 asc; +select x1, x1 * 2, max(x2), max(x3) from test2 group by 2, 1, x1 order by 1, 2, -1 desc, -2 asc; select a, b, c, d, e, f from (select 44 a, 88 b, 13 c, 14 d, 15 e, 16 f) t group by 1,2,3,4,5,6 order by a; +select a, b, c, d, e, f from (select 44 a, 88 b, 13 c, 14 d, 15 e, 16 f) t group by 1,2,3,-3,-2,-1 order by a; explain syntax select plus(1, 1) as a group by a; select substr('aaaaaaaaaaaaaa', 8) as a group by a order by a; diff --git a/tests/queries/0_stateless/02932_group_by_null_fuzzer.sql b/tests/queries/0_stateless/02932_group_by_null_fuzzer.sql index 0c28c120d40..603c7783ef8 100644 --- a/tests/queries/0_stateless/02932_group_by_null_fuzzer.sql +++ b/tests/queries/0_stateless/02932_group_by_null_fuzzer.sql @@ -1,5 +1,6 @@ -- https://github.com/ClickHouse/ClickHouse/issues/43202 -- Queries are generated by the fuzzer, so don't expect them to make sense +SET enable_positional_arguments=0; SELECT NULL, '' FROM (SELECT toNullable(''), NULL AS key GROUP BY GROUPING SETS ((NULL))) AS s1 ALL LEFT JOIN (SELECT '' AS key, NULL AS value GROUP BY GROUPING SETS (('')) WITH TOTALS UNION ALL SELECT NULL AS key, toNullable(NULL) AS value GROUP BY '', NULL, '' WITH TOTALS) AS s2 USING (key); SELECT NULL GROUP BY NULL WITH TOTALS; SELECT 1048575, NULL, b FROM (SELECT '25.5' AS a, NULL, NULL AS b GROUP BY GROUPING SETS ((0.0001)) WITH TOTALS) AS js1 ANY RIGHT JOIN (SELECT NULL AS a, NULL AS b WHERE NULL GROUP BY NULL, -9223372036854775807 WITH CUBE WITH TOTALS UNION ALL SELECT NULL AS a, NULL AS b GROUP BY 1, '21474836.46' WITH TOTALS) AS js2 USING (a, b) ORDER BY nan DESC NULLS LAST, '9223372036854775807' DESC NULLS LAST, a ASC NULLS LAST; From e2f4219c12c216ab32a267b153969b758126a077 Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 28 Dec 2023 12:22:30 +0000 Subject: [PATCH 025/276] Fix --- src/Interpreters/TreeOptimizer.cpp | 7 +++---- .../02943_positional_arguments_bugs.reference | 11 ++++++++++- .../0_stateless/02943_positional_arguments_bugs.sql | 13 +++++++------ 3 files changed, 20 insertions(+), 11 deletions(-) diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 729e2ed6007..57dba3eef89 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -76,11 +76,10 @@ const std::unordered_set possibly_injective_function_names */ void appendUnusedGroupByColumn(ASTSelectQuery * select_query) { - /// You must insert a constant that is not the name of the column in the table. Such a case is rare, but it happens. - /// Also start unused_column integer must not intersect with ([1, source_columns.size()]) - /// might be in positional GROUP BY. + /// Since ASTLiteral is different from ASTIdentifier, so we can use a special constant String Literal for this, + /// and do not need to worry about it conflict with the name of the column in the table. select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, std::make_shared()); - select_query->groupBy()->children.emplace_back(std::make_shared(static_cast(-1))); + select_query->groupBy()->children.emplace_back(std::make_shared("__unused_group_by_column")); } /// Eliminates injective function calls and constant expressions from group by statement. diff --git a/tests/queries/0_stateless/02943_positional_arguments_bugs.reference b/tests/queries/0_stateless/02943_positional_arguments_bugs.reference index 702e1261186..47e8df9e382 100644 --- a/tests/queries/0_stateless/02943_positional_arguments_bugs.reference +++ b/tests/queries/0_stateless/02943_positional_arguments_bugs.reference @@ -1,2 +1,11 @@ -45 1 +0 0 +4 4 +3 3 +2 2 +5 5 +1 1 +6 6 +7 7 +9 9 +8 8 processed 99 0 diff --git a/tests/queries/0_stateless/02943_positional_arguments_bugs.sql b/tests/queries/0_stateless/02943_positional_arguments_bugs.sql index b8cf73da42d..8cc3fb4b17d 100644 --- a/tests/queries/0_stateless/02943_positional_arguments_bugs.sql +++ b/tests/queries/0_stateless/02943_positional_arguments_bugs.sql @@ -2,18 +2,19 @@ DROP TABLE IF EXISTS t; CREATE TABLE t ( - `n` int + `n` int, + `__unused_group_by_column` int ) - ENGINE = MergeTree - ORDER BY n AS -SELECT * +ENGINE = MergeTree +ORDER BY n AS +SELECT number, number FROM numbers(10); SELECT sum(n), - 1 AS x + __unused_group_by_column FROM t -GROUP BY x; +GROUP BY __unused_group_by_column; SELECT 'processed' AS type, From 2e9cdd17ef136f064042b541dbc68ef64ba8194f Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 28 Dec 2023 14:08:14 +0000 Subject: [PATCH 026/276] Fix flaky test --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 26 ++++++++++++++----- .../02943_positional_arguments_bugs.reference | 11 ++++---- .../02943_positional_arguments_bugs.sql | 6 +++-- 3 files changed, 29 insertions(+), 14 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 3290d918a8b..9ec6d9e358c 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -2156,19 +2156,31 @@ void QueryAnalyzer::replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_ node_to_replace = &sort_node->getExpression(); auto * constant_node = (*node_to_replace)->as(); - if (!constant_node || constant_node->getValue().getType() != Field::Types::UInt64) + + if (!constant_node + || (constant_node->getValue().getType() != Field::Types::UInt64 && constant_node->getValue().getType() != Field::Types::Int64)) continue; - UInt64 positional_argument_number = constant_node->getValue().get(); - if (positional_argument_number == 0 || positional_argument_number > projection_nodes.size()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, + UInt64 pos; + if (constant_node->getValue().getType() == Field::Types::UInt64) + { + pos = constant_node->getValue().get(); + } + else // Int64 + { + auto value = constant_node->getValue().get(); + pos = value > 0 ? value : projection_nodes.size() + value + 1; + } + + if (!pos || pos > projection_nodes.size()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Positional argument number {} is out of bounds. Expected in range [1, {}]. In scope {}", - positional_argument_number, + pos, projection_nodes.size(), scope.scope_node->formatASTForErrorMessage()); - --positional_argument_number; - *node_to_replace = projection_nodes[positional_argument_number]; + *node_to_replace = projection_nodes[--pos]; } } diff --git a/tests/queries/0_stateless/02943_positional_arguments_bugs.reference b/tests/queries/0_stateless/02943_positional_arguments_bugs.reference index 47e8df9e382..08310b7cf27 100644 --- a/tests/queries/0_stateless/02943_positional_arguments_bugs.reference +++ b/tests/queries/0_stateless/02943_positional_arguments_bugs.reference @@ -1,11 +1,12 @@ 0 0 -4 4 -3 3 -2 2 -5 5 1 1 +2 2 +3 3 +4 4 +5 5 6 6 7 7 -9 9 8 8 +9 9 +45 1 processed 99 0 diff --git a/tests/queries/0_stateless/02943_positional_arguments_bugs.sql b/tests/queries/0_stateless/02943_positional_arguments_bugs.sql index 8cc3fb4b17d..9b1b872ae40 100644 --- a/tests/queries/0_stateless/02943_positional_arguments_bugs.sql +++ b/tests/queries/0_stateless/02943_positional_arguments_bugs.sql @@ -3,7 +3,7 @@ DROP TABLE IF EXISTS t; CREATE TABLE t ( `n` int, - `__unused_group_by_column` int + `__unused_group_by_column` int ) ENGINE = MergeTree ORDER BY n AS @@ -14,7 +14,9 @@ SELECT sum(n), __unused_group_by_column FROM t -GROUP BY __unused_group_by_column; +GROUP BY __unused_group_by_column ORDER BY __unused_group_by_column; + +SELECT sum(n), 1 as x from t group by x; SELECT 'processed' AS type, From b70ff6d8ea71d4633cdcdbe3ef486707e70c1abb Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 1 Jan 2024 11:02:57 +0100 Subject: [PATCH 027/276] Fix build --- src/Backups/BackupIO_AzureBlobStorage.cpp | 33 +++++++++++++++++++++-- src/Backups/BackupIO_AzureBlobStorage.h | 2 ++ 2 files changed, 33 insertions(+), 2 deletions(-) diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index a1fd5bd8327..bd4efcf63ae 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -218,10 +218,39 @@ void BackupWriterAzureBlobStorage::copyFileFromDisk(const String & path_in_backu BackupWriterDefault::copyFileFromDisk(path_in_backup, src_disk, src_path, copy_encrypted, start_pos, length); } +void BackupWriterAzureBlobStorage::copyFile(const String & destination, const String & source, size_t size) +{ + std::shared_ptr src_client; + std::shared_ptr dest_client; + StorageAzureBlob::Configuration src_configuration = configuration; + src_configuration.container = source; + src_client = StorageAzureBlob::createClient(src_configuration, /* is_read_only */ false); + + StorageAzureBlob::Configuration dest_configuration = configuration; + dest_configuration.container = destination; + dest_client = StorageAzureBlob::createClient(dest_configuration, /* is_read_only */ false); + + LOG_TRACE(log, "Copying file inside backup from {} to {} ", source, destination); + copyAzureBlobStorageFile( + src_client, + dest_client, + configuration.container, + fs::path(configuration.blob_path), + 0, + size, + /* dest_bucket= */ destination, + /* dest_key= */ configuration.blob_path, + settings, + read_settings, + {}, + threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupRDAzure"), + /* for_disk_azure_blob_storage= */ true); +} + void BackupWriterAzureBlobStorage::copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) { copyDataToAzureBlobStorageFile(create_read_buffer, start_pos, length, client, configuration.container, path_in_backup, settings, {}, - threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupWriterS3")); + threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupWRAzure")); } BackupWriterAzureBlobStorage::~BackupWriterAzureBlobStorage() = default; @@ -257,7 +286,7 @@ UInt64 BackupWriterAzureBlobStorage::getFileSize(const String & file_name) RelativePathsWithMetadata children; object_storage->listObjects(key,children,/*max_keys*/0); if (children.empty()) - throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Object {} must exist"); + throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Object must exist"); return children[0].metadata.size_bytes; } diff --git a/src/Backups/BackupIO_AzureBlobStorage.h b/src/Backups/BackupIO_AzureBlobStorage.h index 65affb9f079..87a6c3ef675 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.h +++ b/src/Backups/BackupIO_AzureBlobStorage.h @@ -48,6 +48,8 @@ public: void copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, bool copy_encrypted, UInt64 start_pos, UInt64 length) override; + void copyFile(const String & destination, const String & source, size_t size) override; + void removeFile(const String & file_name) override; void removeFiles(const Strings & file_names) override; From 4122de97213d835de5202d4ca741b4972973884b Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 2 Jan 2024 20:19:01 +0100 Subject: [PATCH 028/276] Updated tests and added settings --- src/Backups/BackupIO_AzureBlobStorage.cpp | 6 +- .../AzureBlobStorage/AzureBlobStorageAuth.cpp | 5 +- .../AzureBlobStorage/AzureObjectStorage.h | 11 ++- .../copyAzureBlobStorageFile.cpp | 68 +++++++++++++++++-- src/Storages/StorageAzureBlob.cpp | 2 +- .../configs/config.xml | 11 --- .../configs/disable_profilers.xml | 13 ---- .../configs/users.xml | 8 --- .../test.py | 2 - 9 files changed, 80 insertions(+), 46 deletions(-) delete mode 100644 tests/integration/test_backup_restore_azure_blob_storage/configs/config.xml delete mode 100644 tests/integration/test_backup_restore_azure_blob_storage/configs/disable_profilers.xml delete mode 100644 tests/integration/test_backup_restore_azure_blob_storage/configs/users.xml diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index bd4efcf63ae..15e8e92a85d 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -139,7 +139,7 @@ void BackupReaderAzureBlobStorage::copyFileToDisk(const String & path_in_backup, settings, read_settings, object_attributes, - threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupReaderAzureBlobStorage"), + threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupRDAzure"), /* for_disk_azure_blob_storage= */ true); return file_size; @@ -209,7 +209,7 @@ void BackupWriterAzureBlobStorage::copyFileFromDisk(const String & path_in_backu settings, read_settings, {}, - threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupWriterS3")); + threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupWRAzure")); return; /// copied! } } @@ -243,7 +243,7 @@ void BackupWriterAzureBlobStorage::copyFile(const String & destination, const St settings, read_settings, {}, - threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupRDAzure"), + threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupWRAzure"), /* for_disk_azure_blob_storage= */ true); } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp index 6075b385a6c..9e703d6fc5e 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -164,7 +164,10 @@ std::unique_ptr getAzureBlobStorageSettings(const Po config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024), config.getInt(config_prefix + ".max_single_read_retries", 3), config.getInt(config_prefix + ".max_single_download_retries", 3), - config.getInt(config_prefix + ".list_object_keys_size", 1000) + config.getInt(config_prefix + ".list_object_keys_size", 1000), + config.getUInt64(config_prefix + ".min_upload_part_size", 16 * 1024 * 1024), + config.getUInt64(config_prefix + ".max_upload_part_size", 5ULL * 1024 * 1024 * 1024), + config.getUInt64(config_prefix + ".max_part_number", 10000) ); } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 8e3d50418d3..55c81b4b7d9 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -24,12 +24,18 @@ struct AzureObjectStorageSettings uint64_t min_bytes_for_seek_, int max_single_read_retries_, int max_single_download_retries_, - int list_object_keys_size_) + int list_object_keys_size_, + size_t min_upload_part_size_, + size_t max_upload_part_size_, + size_t max_part_number_) : max_single_part_upload_size(max_single_part_upload_size_) , min_bytes_for_seek(min_bytes_for_seek_) , max_single_read_retries(max_single_read_retries_) , max_single_download_retries(max_single_download_retries_) , list_object_keys_size(list_object_keys_size_) + , min_upload_part_size(min_upload_part_size_) + , max_upload_part_size(max_upload_part_size_) + , max_part_number(max_part_number_) { } @@ -40,6 +46,9 @@ struct AzureObjectStorageSettings size_t max_single_read_retries = 3; size_t max_single_download_retries = 3; int list_object_keys_size = 1000; + size_t min_upload_part_size = 16 * 1024 * 1024; + size_t max_upload_part_size = 5ULL * 1024 * 1024 * 1024; + size_t max_part_number = 10000; }; using AzureClient = Azure::Storage::Blobs::BlobContainerClient; diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp index 0a0a080b5cb..5ca30fa8071 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp @@ -22,15 +22,17 @@ namespace ProfileEvents extern const Event DiskAzureUploadPart; } -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int INVALID_CONFIG_PARAMETER; +} + + size_t max_single_operation_copy_size = 256 * 1024 * 1024; @@ -106,6 +108,60 @@ namespace std::mutex bg_tasks_mutex; std::condition_variable bg_tasks_condvar; + void calculatePartSize() + { + if (!total_size) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Chosen multipart upload for an empty file. This must not happen"); + + auto max_part_number = settings.get()->max_part_number; + auto min_upload_part_size = settings.get()->min_upload_part_size; + auto max_upload_part_size = settings.get()->max_upload_part_size; + + if (!max_part_number) + throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "max_part_number must not be 0"); + else if (!min_upload_part_size) + throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "min_upload_part_size must not be 0"); + else if (max_upload_part_size < min_upload_part_size) + throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "max_upload_part_size must not be less than min_upload_part_size"); + + size_t part_size = min_upload_part_size; + size_t num_parts = (total_size + part_size - 1) / part_size; + + if (num_parts > max_part_number) + { + part_size = (total_size + max_part_number - 1) / max_part_number; + num_parts = (total_size + part_size - 1) / part_size; + } + + if (part_size > max_upload_part_size) + { + part_size = max_upload_part_size; + num_parts = (total_size + part_size - 1) / part_size; + } + + if (num_parts < 1 || num_parts > max_part_number || part_size < min_upload_part_size || part_size > max_upload_part_size) + { + String msg; + if (num_parts < 1) + msg = "Number of parts is zero"; + else if (num_parts > max_part_number) + msg = fmt::format("Number of parts exceeds {}", num_parts, max_part_number); + else if (part_size < min_upload_part_size) + msg = fmt::format("Size of a part is less than {}", part_size, min_upload_part_size); + else + msg = fmt::format("Size of a part exceeds {}", part_size, max_upload_part_size); + + throw Exception( + ErrorCodes::INVALID_CONFIG_PARAMETER, + "{} while writing {} bytes to AzureBlobStorage. Check max_part_number = {}, " + "min_upload_part_size = {}, max_upload_part_size = {}", + msg, total_size, max_part_number, min_upload_part_size, max_upload_part_size); + } + + /// We've calculated the size of a normal part (the final part can be smaller). + normal_part_size = part_size; + } + public: void performCopy() { @@ -120,7 +176,7 @@ namespace void performMultipartUpload() { - normal_part_size = 1024; + calculatePartSize(); size_t position = offset; size_t end_position = offset + total_size; diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 1b28a2c2fac..f1070c8c31e 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -1139,7 +1139,7 @@ StorageAzureBlobSource::ReaderHolder StorageAzureBlobSource::createReader() QueryPipelineBuilder builder; std::shared_ptr source; std::unique_ptr read_buf; - std::optional num_rows_from_cache = need_only_count && getContext()->getSettingsRef().use_cache_for_count_from_files + std::optional num_rows_from_cache = need_only_count && getContext()->getSettingsRef().use_cache_for_count_from_files ? tryGetNumRowsFromCache(path_with_metadata) : std::nullopt; if (num_rows_from_cache) { diff --git a/tests/integration/test_backup_restore_azure_blob_storage/configs/config.xml b/tests/integration/test_backup_restore_azure_blob_storage/configs/config.xml deleted file mode 100644 index 5725dce40cd..00000000000 --- a/tests/integration/test_backup_restore_azure_blob_storage/configs/config.xml +++ /dev/null @@ -1,11 +0,0 @@ - - 1 - 0 - 0.0 - 0 - 1 - 1 - 0 - 16 - 16 - \ No newline at end of file diff --git a/tests/integration/test_backup_restore_azure_blob_storage/configs/disable_profilers.xml b/tests/integration/test_backup_restore_azure_blob_storage/configs/disable_profilers.xml deleted file mode 100644 index b74bb1502ce..00000000000 --- a/tests/integration/test_backup_restore_azure_blob_storage/configs/disable_profilers.xml +++ /dev/null @@ -1,13 +0,0 @@ - - - - - 0 - 0 - 0 - 1000 - 1 - 1 - - - diff --git a/tests/integration/test_backup_restore_azure_blob_storage/configs/users.xml b/tests/integration/test_backup_restore_azure_blob_storage/configs/users.xml deleted file mode 100644 index c12eb2f79f4..00000000000 --- a/tests/integration/test_backup_restore_azure_blob_storage/configs/users.xml +++ /dev/null @@ -1,8 +0,0 @@ - - - - - default - - - diff --git a/tests/integration/test_backup_restore_azure_blob_storage/test.py b/tests/integration/test_backup_restore_azure_blob_storage/test.py index 0a48d3523f0..06c18d7468f 100644 --- a/tests/integration/test_backup_restore_azure_blob_storage/test.py +++ b/tests/integration/test_backup_restore_azure_blob_storage/test.py @@ -24,8 +24,6 @@ def cluster(): cluster = ClickHouseCluster(__file__) cluster.add_instance( "node", - main_configs=["configs/config.xml"], - user_configs=["configs/disable_profilers.xml", "configs/users.xml"], with_azurite=True, ) cluster.start() From df221f7db65fd17af6a71704f756e47ceec7a928 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 3 Jan 2024 11:35:06 +0100 Subject: [PATCH 029/276] Renamed Bucket-Key to Container-Blob --- src/Backups/BackupIO_AzureBlobStorage.cpp | 14 +++--- .../copyAzureBlobStorageFile.cpp | 44 +++++++++---------- .../copyAzureBlobStorageFile.h | 10 ++--- 3 files changed, 34 insertions(+), 34 deletions(-) diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index 15e8e92a85d..de40fc6b33b 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -134,8 +134,8 @@ void BackupReaderAzureBlobStorage::copyFileToDisk(const String & path_in_backup, fs::path(configuration.blob_path) / path_in_backup, 0, file_size, - /* dest_bucket= */ blob_path[1], - /* dest_key= */ blob_path[0], + /* dest_container */ blob_path[1], + /* dest_path */ blob_path[0], settings, read_settings, object_attributes, @@ -178,7 +178,7 @@ void BackupWriterAzureBlobStorage::copyFileFromDisk(const String & path_in_backu auto source_data_source_description = src_disk->getDataSourceDescription(); if (source_data_source_description.sameKind(data_source_description) && (source_data_source_description.is_encrypted == copy_encrypted)) { - /// getBlobPath() can return more than 3 elements if the file is stored as multiple objects in AzureBlobStorage bucket. + /// getBlobPath() can return more than 3 elements if the file is stored as multiple objects in AzureBlobStorage container. /// In this case we can't use the native copy. if (auto blob_path = src_disk->getBlobPath(src_path); blob_path.size() == 2) { @@ -200,8 +200,8 @@ void BackupWriterAzureBlobStorage::copyFileFromDisk(const String & path_in_backu copyAzureBlobStorageFile( src_client, client, - /* src_bucket */ blob_path[1], - /* src_key= */ blob_path[0], + /* src_container */ blob_path[1], + /* src_path */ blob_path[0], start_pos, length, configuration.container, @@ -238,8 +238,8 @@ void BackupWriterAzureBlobStorage::copyFile(const String & destination, const St fs::path(configuration.blob_path), 0, size, - /* dest_bucket= */ destination, - /* dest_key= */ configuration.blob_path, + /* dest_container */ destination, + /* dest_path */ configuration.blob_path, settings, read_settings, {}, diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp index 5ca30fa8071..df1341efdd1 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp @@ -46,8 +46,8 @@ namespace std::shared_ptr client_, size_t offset_, size_t total_size_, - const String & dest_bucket_, - const String & dest_key_, + const String & dest_container_, + const String & dest_blob_, std::shared_ptr settings_, const std::optional> & object_metadata_, ThreadPoolCallbackRunner schedule_, @@ -57,8 +57,8 @@ namespace , client(client_) , offset (offset_) , total_size (total_size_) - , dest_bucket(dest_bucket_) - , dest_key(dest_key_) + , dest_container(dest_container_) + , dest_blob(dest_blob_) , settings(settings_) , object_metadata(object_metadata_) , schedule(schedule_) @@ -75,8 +75,8 @@ namespace std::shared_ptr client; size_t offset; size_t total_size; - const String & dest_bucket; - const String & dest_key; + const String & dest_container; + const String & dest_blob; std::shared_ptr settings; const std::optional> & object_metadata; ThreadPoolCallbackRunner schedule; @@ -170,7 +170,7 @@ namespace void completeMultipartUpload() { - auto block_blob_client = client->GetBlockBlobClient(dest_key); + auto block_blob_client = client->GetBlockBlobClient(dest_blob); block_blob_client.CommitBlockList(block_ids); } @@ -207,7 +207,7 @@ namespace void uploadPart(size_t part_offset, size_t part_size) { - LOG_TRACE(log, "Writing part. Bucket: {}, Key: {}, Size: {}", dest_bucket, dest_key, part_size); + LOG_TRACE(log, "Writing part. Container: {}, Blob: {}, Size: {}", dest_container, dest_blob, part_size); if (!part_size) { @@ -286,7 +286,7 @@ namespace std::lock_guard lock(bg_tasks_mutex); /// Protect bg_tasks from race task.block_id = block_id; - LOG_TRACE(log, "Writing part finished. Bucket: {}, Key: {}, block_id: {}, Parts: {}", dest_bucket, dest_key, block_id, bg_tasks.size()); + LOG_TRACE(log, "Writing part finished. Container: {}, Blob: {}, block_id: {}, Parts: {}", dest_container, dest_blob, block_id, bg_tasks.size()); } String processUploadPartRequest(UploadPartTask & task) @@ -295,7 +295,7 @@ namespace if (for_disk_azure_blob_storage) ProfileEvents::increment(ProfileEvents::DiskAzureUploadPart); - auto block_blob_client = client->GetBlockBlobClient(dest_key); + auto block_blob_client = client->GetBlockBlobClient(dest_blob); task.block_id = getRandomASCIIString(64); Azure::Core::IO::MemoryBodyStream memory(reinterpret_cast(task.data), task.size); block_blob_client.StageBlock(task.block_id, memory); @@ -330,14 +330,14 @@ void copyDataToAzureBlobStorageFile( size_t offset, size_t size, std::shared_ptr & dest_client, - const String & dest_bucket, - const String & dest_key, + const String & dest_container, + const String & dest_blob, std::shared_ptr settings, const std::optional> & object_metadata, ThreadPoolCallbackRunner schedule, bool for_disk_azure_blob_storage) { - UploadHelper helper{create_read_buffer, dest_client, offset, size, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_azure_blob_storage, &Poco::Logger::get("copyDataToAzureBlobStorageFile")}; + UploadHelper helper{create_read_buffer, dest_client, offset, size, dest_container, dest_blob, settings, object_metadata, schedule, for_disk_azure_blob_storage, &Poco::Logger::get("copyDataToAzureBlobStorageFile")}; helper.performCopy(); } @@ -345,12 +345,12 @@ void copyDataToAzureBlobStorageFile( void copyAzureBlobStorageFile( std::shared_ptr src_client, std::shared_ptr dest_client, - const String & src_bucket, - const String & src_key, + const String & src_container, + const String & src_blob, size_t offset, size_t size, - const String & dest_bucket, - const String & dest_key, + const String & dest_container, + const String & dest_blob, std::shared_ptr settings, const ReadSettings & read_settings, const std::optional> & object_metadata, @@ -363,21 +363,21 @@ void copyAzureBlobStorageFile( ProfileEvents::increment(ProfileEvents::AzureCopyObject); if (for_disk_azure_blob_storage) ProfileEvents::increment(ProfileEvents::DiskAzureCopyObject); - auto block_blob_client_src = src_client->GetBlockBlobClient(src_key); - auto block_blob_client_dest = dest_client->GetBlockBlobClient(dest_key); + auto block_blob_client_src = src_client->GetBlockBlobClient(src_blob); + auto block_blob_client_dest = dest_client->GetBlockBlobClient(dest_blob); auto uri = block_blob_client_src.GetUrl(); block_blob_client_dest.CopyFromUri(uri); } else { - LOG_TRACE(&Poco::Logger::get("copyAzureBlobStorageFile"), "Reading from Bucket: {}, Key: {}", src_bucket, src_key); + LOG_TRACE(&Poco::Logger::get("copyAzureBlobStorageFile"), "Reading from Container: {}, Blob: {}", src_container, src_blob); auto create_read_buffer = [&] { - return std::make_unique(src_client, src_key, read_settings, settings->max_single_read_retries, + return std::make_unique(src_client, src_blob, read_settings, settings->max_single_read_retries, settings->max_single_download_retries); }; - UploadHelper helper{create_read_buffer, dest_client, offset, size, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_azure_blob_storage, &Poco::Logger::get("copyAzureBlobStorageFile")}; + UploadHelper helper{create_read_buffer, dest_client, offset, size, dest_container, dest_blob, settings, object_metadata, schedule, for_disk_azure_blob_storage, &Poco::Logger::get("copyAzureBlobStorageFile")}; helper.performCopy(); } } diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h index 31228fbcb23..059d0318f57 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h @@ -23,12 +23,12 @@ using CreateReadBuffer = std::function()>; void copyAzureBlobStorageFile( std::shared_ptr src_client, std::shared_ptr dest_client, - const String & src_bucket, - const String & src_key, + const String & src_container, + const String & src_path, size_t src_offset, size_t src_size, - const String & dest_bucket, - const String & dest_key, + const String & dest_container, + const String & dest_path, std::shared_ptr settings, const ReadSettings & read_settings, const std::optional> & object_metadata = std::nullopt, @@ -46,8 +46,8 @@ void copyDataToAzureBlobStorageFile( size_t offset, size_t size, std::shared_ptr & client, + const String & dest_container, const String & dest_bucket, - const String & dest_key, std::shared_ptr settings, const std::optional> & object_metadata = std::nullopt, ThreadPoolCallbackRunner schedule_ = {}, From 91bad5bc39963e9450f284dfc6b45fd69fa146de Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 4 Jan 2024 16:06:36 +0100 Subject: [PATCH 030/276] Updated to use MultiVersion for BlobContainerClient in Backups and updated to get client from disk --- src/Backups/BackupIO_AzureBlobStorage.cpp | 72 +++++-------------- src/Backups/BackupIO_AzureBlobStorage.h | 4 +- .../AzureBlobStorage/AzureObjectStorage.h | 5 ++ .../copyAzureBlobStorageFile.cpp | 20 +++--- .../copyAzureBlobStorageFile.h | 6 +- 5 files changed, 37 insertions(+), 70 deletions(-) diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index de40fc6b33b..968a60c566f 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -27,8 +27,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -//using AzureClientPtr = std::shared_ptr; - BackupReaderAzureBlobStorage::BackupReaderAzureBlobStorage( StorageAzureBlob::Configuration configuration_, const ReadSettings & read_settings_, @@ -38,12 +36,13 @@ BackupReaderAzureBlobStorage::BackupReaderAzureBlobStorage( , data_source_description{DataSourceType::AzureBlobStorage, configuration_.container, false, false} , configuration(configuration_) { - client = StorageAzureBlob::createClient(configuration, /* is_read_only */ false); + auto client_ptr = StorageAzureBlob::createClient(configuration, /* is_read_only */ false); settings = StorageAzureBlob::createSettingsAsSharedPtr(context_); auto settings_as_unique_ptr = StorageAzureBlob::createSettings(context_); object_storage = std::make_unique("BackupReaderAzureBlobStorage", - std::make_unique(*client.get()), + std::move(client_ptr), std::move(settings_as_unique_ptr)); + client = object_storage->getClient(); } BackupReaderAzureBlobStorage::~BackupReaderAzureBlobStorage() = default; @@ -89,7 +88,7 @@ std::unique_ptr BackupReaderAzureBlobStorage::readFile(const key = file_name; } return std::make_unique( - client, key, read_settings, settings->max_single_read_retries, + client.get(), key, read_settings, settings->max_single_read_retries, settings->max_single_download_retries); } @@ -113,23 +112,9 @@ void BackupReaderAzureBlobStorage::copyFileToDisk(const String & path_in_backup, "Blob writing function called with unexpected blob_path.size={} or mode={}", blob_path.size(), mode); - std::shared_ptr dest_client; - if (configuration.container == blob_path[1]) - { - dest_client = client; - } - else - { - StorageAzureBlob::Configuration dest_configuration = configuration; - dest_configuration.container = blob_path[1]; - dest_configuration.blob_path = blob_path[0]; - dest_client = StorageAzureBlob::createClient(dest_configuration, /* is_read_only */ false); - } - - copyAzureBlobStorageFile( client, - dest_client, + reinterpret_cast(destination_disk->getObjectStorage().get())->getClient(), configuration.container, fs::path(configuration.blob_path) / path_in_backup, 0, @@ -163,12 +148,13 @@ BackupWriterAzureBlobStorage::BackupWriterAzureBlobStorage( , data_source_description{DataSourceType::AzureBlobStorage,configuration_.container, false, false} , configuration(configuration_) { - client = StorageAzureBlob::createClient(configuration, /* is_read_only */ false); + auto client_ptr = StorageAzureBlob::createClient(configuration, /* is_read_only */ false); settings = StorageAzureBlob::createSettingsAsSharedPtr(context_); auto settings_as_unique_ptr = StorageAzureBlob::createSettings(context_); object_storage = std::make_unique("BackupWriterAzureBlobStorage", - std::make_unique(*client.get()), - std::move(settings_as_unique_ptr)); + std::move(client_ptr), + std::move(settings_as_unique_ptr)); + client = object_storage->getClient(); } void BackupWriterAzureBlobStorage::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, @@ -182,23 +168,9 @@ void BackupWriterAzureBlobStorage::copyFileFromDisk(const String & path_in_backu /// In this case we can't use the native copy. if (auto blob_path = src_disk->getBlobPath(src_path); blob_path.size() == 2) { - - std::shared_ptr src_client; - if (configuration.container == blob_path[1]) - { - src_client = client; - } - else - { - StorageAzureBlob::Configuration src_configuration = configuration; - src_configuration.container = blob_path[1]; - src_configuration.blob_path = blob_path[0]; - src_client = StorageAzureBlob::createClient(src_configuration, /* is_read_only */ false); - } - LOG_TRACE(log, "Copying file {} from disk {} to AzureBlobStorag", src_path, src_disk->getName()); copyAzureBlobStorageFile( - src_client, + reinterpret_cast(src_disk->getObjectStorage().get())->getClient(), client, /* src_container */ blob_path[1], /* src_path */ blob_path[0], @@ -220,26 +192,16 @@ void BackupWriterAzureBlobStorage::copyFileFromDisk(const String & path_in_backu void BackupWriterAzureBlobStorage::copyFile(const String & destination, const String & source, size_t size) { - std::shared_ptr src_client; - std::shared_ptr dest_client; - StorageAzureBlob::Configuration src_configuration = configuration; - src_configuration.container = source; - src_client = StorageAzureBlob::createClient(src_configuration, /* is_read_only */ false); - - StorageAzureBlob::Configuration dest_configuration = configuration; - dest_configuration.container = destination; - dest_client = StorageAzureBlob::createClient(dest_configuration, /* is_read_only */ false); - LOG_TRACE(log, "Copying file inside backup from {} to {} ", source, destination); copyAzureBlobStorageFile( - src_client, - dest_client, + client, + client, configuration.container, - fs::path(configuration.blob_path), + fs::path(source), 0, size, - /* dest_container */ destination, - /* dest_path */ configuration.blob_path, + /* dest_container */ configuration.container, + /* dest_path */ destination, settings, read_settings, {}, @@ -303,7 +265,7 @@ std::unique_ptr BackupWriterAzureBlobStorage::readFile(const String } return std::make_unique( - client, key, read_settings, settings->max_single_read_retries, + client.get(), key, read_settings, settings->max_single_read_retries, settings->max_single_download_retries); } @@ -319,7 +281,7 @@ std::unique_ptr BackupWriterAzureBlobStorage::writeFile(const Strin key = file_name; } return std::make_unique( - client, + client.get(), key, settings->max_single_part_upload_size, DBMS_DEFAULT_BUFFER_SIZE, diff --git a/src/Backups/BackupIO_AzureBlobStorage.h b/src/Backups/BackupIO_AzureBlobStorage.h index 87a6c3ef675..12bf073cd08 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.h +++ b/src/Backups/BackupIO_AzureBlobStorage.h @@ -28,7 +28,7 @@ public: private: const DataSourceDescription data_source_description; - std::shared_ptr client; + MultiVersion client; StorageAzureBlob::Configuration configuration; std::unique_ptr object_storage; std::shared_ptr settings; @@ -57,7 +57,7 @@ private: std::unique_ptr readFile(const String & file_name, size_t expected_file_size) override; void removeFilesBatch(const Strings & file_names); const DataSourceDescription data_source_description; - std::shared_ptr client; + MultiVersion client; StorageAzureBlob::Configuration configuration; std::unique_ptr object_storage; std::shared_ptr settings; diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 55c81b4b7d9..1ff4537742f 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -134,6 +134,11 @@ public: bool isRemote() const override { return true; } + MultiVersion & getClient() + { + return client; + } + private: const String name; /// client used to access the files in the Blob Storage cloud diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp index df1341efdd1..4ec90d2830e 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp @@ -43,7 +43,7 @@ namespace public: UploadHelper( const CreateReadBuffer & create_read_buffer_, - std::shared_ptr client_, + MultiVersion & client_, size_t offset_, size_t total_size_, const String & dest_container_, @@ -72,7 +72,7 @@ namespace protected: std::function()> create_read_buffer; - std::shared_ptr client; + MultiVersion & client; size_t offset; size_t total_size; const String & dest_container; @@ -170,7 +170,7 @@ namespace void completeMultipartUpload() { - auto block_blob_client = client->GetBlockBlobClient(dest_blob); + auto block_blob_client = client.get()->GetBlockBlobClient(dest_blob); block_blob_client.CommitBlockList(block_ids); } @@ -295,7 +295,7 @@ namespace if (for_disk_azure_blob_storage) ProfileEvents::increment(ProfileEvents::DiskAzureUploadPart); - auto block_blob_client = client->GetBlockBlobClient(dest_blob); + auto block_blob_client = client.get()->GetBlockBlobClient(dest_blob); task.block_id = getRandomASCIIString(64); Azure::Core::IO::MemoryBodyStream memory(reinterpret_cast(task.data), task.size); block_blob_client.StageBlock(task.block_id, memory); @@ -329,7 +329,7 @@ void copyDataToAzureBlobStorageFile( const std::function()> & create_read_buffer, size_t offset, size_t size, - std::shared_ptr & dest_client, + MultiVersion & dest_client, const String & dest_container, const String & dest_blob, std::shared_ptr settings, @@ -343,8 +343,8 @@ void copyDataToAzureBlobStorageFile( void copyAzureBlobStorageFile( - std::shared_ptr src_client, - std::shared_ptr dest_client, + MultiVersion & src_client, + MultiVersion & dest_client, const String & src_container, const String & src_blob, size_t offset, @@ -363,8 +363,8 @@ void copyAzureBlobStorageFile( ProfileEvents::increment(ProfileEvents::AzureCopyObject); if (for_disk_azure_blob_storage) ProfileEvents::increment(ProfileEvents::DiskAzureCopyObject); - auto block_blob_client_src = src_client->GetBlockBlobClient(src_blob); - auto block_blob_client_dest = dest_client->GetBlockBlobClient(dest_blob); + auto block_blob_client_src = src_client.get()->GetBlockBlobClient(src_blob); + auto block_blob_client_dest = dest_client.get()->GetBlockBlobClient(dest_blob); auto uri = block_blob_client_src.GetUrl(); block_blob_client_dest.CopyFromUri(uri); } @@ -373,7 +373,7 @@ void copyAzureBlobStorageFile( LOG_TRACE(&Poco::Logger::get("copyAzureBlobStorageFile"), "Reading from Container: {}, Blob: {}", src_container, src_blob); auto create_read_buffer = [&] { - return std::make_unique(src_client, src_blob, read_settings, settings->max_single_read_retries, + return std::make_unique(src_client.get(), src_blob, read_settings, settings->max_single_read_retries, settings->max_single_download_retries); }; diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h index 059d0318f57..a6502541db1 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h @@ -21,8 +21,8 @@ using CreateReadBuffer = std::function()>; /// Copies a file from AzureBlobStorage to AzureBlobStorage. /// The parameters `src_offset` and `src_size` specify a part in the source to copy. void copyAzureBlobStorageFile( - std::shared_ptr src_client, - std::shared_ptr dest_client, + MultiVersion & src_client, + MultiVersion & dest_client, const String & src_container, const String & src_path, size_t src_offset, @@ -45,7 +45,7 @@ void copyDataToAzureBlobStorageFile( const std::function()> & create_read_buffer, size_t offset, size_t size, - std::shared_ptr & client, + MultiVersion & client, const String & dest_container, const String & dest_bucket, std::shared_ptr settings, From c14605caa7f403531a6ff0663c242aa5d466ab07 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 4 Jan 2024 18:27:54 +0100 Subject: [PATCH 031/276] Added flag use_native_copy and updated to use StartCopyFromUri for native copy with large files --- .../AzureBlobStorage/AzureBlobStorageAuth.cpp | 3 ++- .../AzureBlobStorage/AzureObjectStorage.h | 10 +++---- .../copyAzureBlobStorageFile.cpp | 26 ++++++++++++++++--- 3 files changed, 30 insertions(+), 9 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp index 9e703d6fc5e..e29def06363 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -167,7 +167,8 @@ std::unique_ptr getAzureBlobStorageSettings(const Po config.getInt(config_prefix + ".list_object_keys_size", 1000), config.getUInt64(config_prefix + ".min_upload_part_size", 16 * 1024 * 1024), config.getUInt64(config_prefix + ".max_upload_part_size", 5ULL * 1024 * 1024 * 1024), - config.getUInt64(config_prefix + ".max_part_number", 10000) + config.getUInt64(config_prefix + ".max_part_number", 10000), + config.getBool(config_prefix + ".use_native_copy", false) ); } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 1ff4537742f..436b48c0ad4 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -27,7 +27,8 @@ struct AzureObjectStorageSettings int list_object_keys_size_, size_t min_upload_part_size_, size_t max_upload_part_size_, - size_t max_part_number_) + size_t max_part_number_, + bool use_native_copy_) : max_single_part_upload_size(max_single_part_upload_size_) , min_bytes_for_seek(min_bytes_for_seek_) , max_single_read_retries(max_single_read_retries_) @@ -36,6 +37,7 @@ struct AzureObjectStorageSettings , min_upload_part_size(min_upload_part_size_) , max_upload_part_size(max_upload_part_size_) , max_part_number(max_part_number_) + , use_native_copy(use_native_copy_) { } @@ -49,6 +51,7 @@ struct AzureObjectStorageSettings size_t min_upload_part_size = 16 * 1024 * 1024; size_t max_upload_part_size = 5ULL * 1024 * 1024 * 1024; size_t max_part_number = 10000; + bool use_native_copy = false; }; using AzureClient = Azure::Storage::Blobs::BlobContainerClient; @@ -134,10 +137,7 @@ public: bool isRemote() const override { return true; } - MultiVersion & getClient() - { - return client; - } + MultiVersion & getClient() { return client; } private: const String name; diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp index 4ec90d2830e..9db5ddb476a 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp @@ -30,6 +30,7 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int INVALID_CONFIG_PARAMETER; + extern const int AZURE_BLOB_STORAGE_ERROR; } @@ -358,15 +359,34 @@ void copyAzureBlobStorageFile( bool for_disk_azure_blob_storage) { - if (size < max_single_operation_copy_size) + if (settings->use_native_copy ) { ProfileEvents::increment(ProfileEvents::AzureCopyObject); if (for_disk_azure_blob_storage) ProfileEvents::increment(ProfileEvents::DiskAzureCopyObject); + auto block_blob_client_src = src_client.get()->GetBlockBlobClient(src_blob); auto block_blob_client_dest = dest_client.get()->GetBlockBlobClient(dest_blob); - auto uri = block_blob_client_src.GetUrl(); - block_blob_client_dest.CopyFromUri(uri); + auto source_uri = block_blob_client_src.GetUrl(); + + if (size < max_single_operation_copy_size) + { + block_blob_client_dest.CopyFromUri(source_uri); + } + else + { + Azure::Storage::Blobs::StartBlobCopyOperation operation = block_blob_client_dest.StartCopyFromUri(source_uri); + + // Wait for the operation to finish, checking for status every 100 second. + auto copy_response = operation.PollUntilDone(std::chrono::milliseconds(100)); + auto properties_model = copy_response.Value; + + if (properties_model.CopySource.HasValue()) + { + throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Copy failed"); + } + + } } else { From 2ee68933123583fe585093868e65c3562d36d66a Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 5 Jan 2024 10:58:04 +0100 Subject: [PATCH 032/276] Updated to return container for getObjectsNamespace --- src/Backups/BackupIO_AzureBlobStorage.cpp | 6 ++++-- .../ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp | 7 +++++-- .../ObjectStorages/AzureBlobStorage/AzureObjectStorage.h | 6 ++++-- .../AzureBlobStorage/registerDiskAzureBlobStorage.cpp | 4 +++- src/Storages/StorageAzureBlob.cpp | 2 +- src/TableFunctions/TableFunctionAzureBlobStorage.cpp | 4 ++-- .../TableFunctionAzureBlobStorageCluster.cpp | 4 ++-- 7 files changed, 21 insertions(+), 12 deletions(-) diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index 968a60c566f..5ddbb42e2c0 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -41,7 +41,8 @@ BackupReaderAzureBlobStorage::BackupReaderAzureBlobStorage( auto settings_as_unique_ptr = StorageAzureBlob::createSettings(context_); object_storage = std::make_unique("BackupReaderAzureBlobStorage", std::move(client_ptr), - std::move(settings_as_unique_ptr)); + std::move(settings_as_unique_ptr), + configuration_.container); client = object_storage->getClient(); } @@ -153,7 +154,8 @@ BackupWriterAzureBlobStorage::BackupWriterAzureBlobStorage( auto settings_as_unique_ptr = StorageAzureBlob::createSettings(context_); object_storage = std::make_unique("BackupWriterAzureBlobStorage", std::move(client_ptr), - std::move(settings_as_unique_ptr)); + std::move(settings_as_unique_ptr), + configuration_.container); client = object_storage->getClient(); } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 068e2aebab1..1f92ef48350 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -92,10 +92,12 @@ private: AzureObjectStorage::AzureObjectStorage( const String & name_, AzureClientPtr && client_, - SettingsPtr && settings_) + SettingsPtr && settings_, + const String & container_) : name(name_) , client(std::move(client_)) , settings(std::move(settings_)) + , container(container_) , log(&Poco::Logger::get("AzureObjectStorage")) { data_source_description.type = DataSourceType::AzureBlobStorage; @@ -379,7 +381,8 @@ std::unique_ptr AzureObjectStorage::cloneObjectStorage(const std return std::make_unique( name, getAzureBlobContainerClient(config, config_prefix), - getAzureBlobStorageSettings(config, config_prefix, context) + getAzureBlobStorageSettings(config, config_prefix, context), + container ); } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 436b48c0ad4..660d4a30889 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -66,7 +66,8 @@ public: AzureObjectStorage( const String & name_, AzureClientPtr && client_, - SettingsPtr && settings_); + SettingsPtr && settings_, + const String & container_); void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const override; @@ -125,7 +126,7 @@ public: const std::string & config_prefix, ContextPtr context) override; - String getObjectsNamespace() const override { return ""; } + String getObjectsNamespace() const override { return container ; } std::unique_ptr cloneObjectStorage( const std::string & new_namespace, @@ -144,6 +145,7 @@ private: /// client used to access the files in the Blob Storage cloud MultiVersion client; MultiVersion settings; + const String container; Poco::Logger * log; diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/registerDiskAzureBlobStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/registerDiskAzureBlobStorage.cpp index 7ba9d21db62..2ffd910f92a 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/registerDiskAzureBlobStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/registerDiskAzureBlobStorage.cpp @@ -26,10 +26,12 @@ void registerDiskAzureBlobStorage(DiskFactory & factory, bool global_skip_access { auto [metadata_path, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context); + String container_name = config.getString(config_prefix + ".container_name", "default-container"); ObjectStoragePtr azure_object_storage = std::make_unique( name, getAzureBlobContainerClient(config, config_prefix), - getAzureBlobStorageSettings(config, config_prefix, context)); + getAzureBlobStorageSettings(config, config_prefix, context), + container_name); String key_prefix; auto metadata_storage = std::make_shared(metadata_disk, key_prefix); diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index f1070c8c31e..fcd7074b9d2 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -314,7 +314,7 @@ void registerStorageAzureBlob(StorageFactory & factory) return std::make_shared( std::move(configuration), - std::make_unique("AzureBlobStorage", std::move(client), std::move(settings)), + std::make_unique("AzureBlobStorage", std::move(client), std::move(settings),configuration.container), args.getContext(), args.table_id, args.columns, diff --git a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp index d394c836369..b098cac5144 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp +++ b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp @@ -262,7 +262,7 @@ ColumnsDescription TableFunctionAzureBlobStorage::getActualTableStructure(Contex auto client = StorageAzureBlob::createClient(configuration, !is_insert_query); auto settings = StorageAzureBlob::createSettings(context); - auto object_storage = std::make_unique("AzureBlobStorageTableFunction", std::move(client), std::move(settings)); + auto object_storage = std::make_unique("AzureBlobStorageTableFunction", std::move(client), std::move(settings), configuration.container); return StorageAzureBlob::getTableStructureFromData(object_storage.get(), configuration, std::nullopt, context, false); } @@ -293,7 +293,7 @@ StoragePtr TableFunctionAzureBlobStorage::executeImpl(const ASTPtr & /*ast_funct StoragePtr storage = std::make_shared( configuration, - std::make_unique(table_name, std::move(client), std::move(settings)), + std::make_unique(table_name, std::move(client), std::move(settings), configuration.container), context, StorageID(getDatabaseName(), table_name), columns, diff --git a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp b/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp index eee585967c2..1c3b302a186 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp +++ b/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp @@ -40,7 +40,7 @@ StoragePtr TableFunctionAzureBlobStorageCluster::executeImpl( /// On worker node this filename won't contains globs storage = std::make_shared( configuration, - std::make_unique(table_name, std::move(client), std::move(settings)), + std::make_unique(table_name, std::move(client), std::move(settings), configuration.container), context, StorageID(getDatabaseName(), table_name), columns, @@ -55,7 +55,7 @@ StoragePtr TableFunctionAzureBlobStorageCluster::executeImpl( storage = std::make_shared( cluster_name, configuration, - std::make_unique(table_name, std::move(client), std::move(settings)), + std::make_unique(table_name, std::move(client), std::move(settings), configuration.container), StorageID(getDatabaseName(), table_name), columns, ConstraintsDescription{}, From b250acff789620be57e21977d8f3d4a3468070d5 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 5 Jan 2024 11:26:32 +0100 Subject: [PATCH 033/276] Fixed style check --- src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp index 9db5ddb476a..3399f1705f4 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp @@ -359,7 +359,7 @@ void copyAzureBlobStorageFile( bool for_disk_azure_blob_storage) { - if (settings->use_native_copy ) + if (settings->use_native_copy) { ProfileEvents::increment(ProfileEvents::AzureCopyObject); if (for_disk_azure_blob_storage) From fd92c1961e5f09411d83b21c4fe9f00b78be22ba Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sun, 7 Jan 2024 16:33:48 +0100 Subject: [PATCH 034/276] Fix clang tidy build --- src/Backups/BackupIO_AzureBlobStorage.cpp | 12 ++++++------ src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp | 10 +++++----- src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h | 6 +++--- 3 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index 5ddbb42e2c0..8c6c1040eec 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -305,21 +305,21 @@ void BackupWriterAzureBlobStorage::removeFile(const String & file_name) object_storage->removeObjectIfExists(object); } -void BackupWriterAzureBlobStorage::removeFiles(const Strings & keys) +void BackupWriterAzureBlobStorage::removeFiles(const Strings & file_names) { StoredObjects objects; - for (const auto & key : keys) - objects.emplace_back(key); + for (const auto & file_name : file_names) + objects.emplace_back(file_name); object_storage->removeObjectsIfExist(objects); } -void BackupWriterAzureBlobStorage::removeFilesBatch(const Strings & keys) +void BackupWriterAzureBlobStorage::removeFilesBatch(const Strings & file_names) { StoredObjects objects; - for (const auto & key : keys) - objects.emplace_back(key); + for (const auto & file_name : file_names) + objects.emplace_back(file_name); object_storage->removeObjectsIfExist(objects); } diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp index 3399f1705f4..272be914cc1 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp @@ -65,11 +65,11 @@ namespace , schedule(schedule_) , for_disk_azure_blob_storage(for_disk_azure_blob_storage_) , log(log_) - , max_single_part_upload_size(settings_.get()->max_single_part_upload_size) + , max_single_part_upload_size(settings_->max_single_part_upload_size) { } - ~UploadHelper() {} + virtual ~UploadHelper() = default; protected: std::function()> create_read_buffer; @@ -114,9 +114,9 @@ namespace if (!total_size) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chosen multipart upload for an empty file. This must not happen"); - auto max_part_number = settings.get()->max_part_number; - auto min_upload_part_size = settings.get()->min_upload_part_size; - auto max_upload_part_size = settings.get()->max_upload_part_size; + auto max_part_number = settings->max_part_number; + auto min_upload_part_size = settings->min_upload_part_size; + auto max_upload_part_size = settings->max_upload_part_size; if (!max_part_number) throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "max_part_number must not be 0"); diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h index a6502541db1..b022151d32d 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h @@ -24,11 +24,11 @@ void copyAzureBlobStorageFile( MultiVersion & src_client, MultiVersion & dest_client, const String & src_container, - const String & src_path, + const String & src_blob, size_t src_offset, size_t src_size, const String & dest_container, - const String & dest_path, + const String & dest_blob, std::shared_ptr settings, const ReadSettings & read_settings, const std::optional> & object_metadata = std::nullopt, @@ -47,7 +47,7 @@ void copyDataToAzureBlobStorageFile( size_t size, MultiVersion & client, const String & dest_container, - const String & dest_bucket, + const String & dest_blob, std::shared_ptr settings, const std::optional> & object_metadata = std::nullopt, ThreadPoolCallbackRunner schedule_ = {}, From f50f7f56949021d01ba692f6788e50d411ca8af9 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 8 Jan 2024 14:25:33 +0100 Subject: [PATCH 035/276] Removed unwanted includes --- .../registerBackupEngineAzureBlobStorage.cpp | 19 ------------------- 1 file changed, 19 deletions(-) diff --git a/src/Backups/registerBackupEngineAzureBlobStorage.cpp b/src/Backups/registerBackupEngineAzureBlobStorage.cpp index ef95206831f..810da5adb3f 100644 --- a/src/Backups/registerBackupEngineAzureBlobStorage.cpp +++ b/src/Backups/registerBackupEngineAzureBlobStorage.cpp @@ -10,13 +10,11 @@ #include #include #include -#include #endif namespace DB { -namespace fs = std::filesystem; namespace ErrorCodes { @@ -25,23 +23,6 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -#if USE_AZURE_BLOB_STORAGE -namespace -{ - String removeFileNameFromURL(String & url) - { - Poco::URI url2{url}; - String path = url2.getPath(); - size_t slash_pos = path.find_last_of('/'); - String file_name = path.substr(slash_pos + 1); - path.resize(slash_pos + 1); - url2.setPath(path); - url = url2.toString(); - return file_name; - } -} -#endif - void registerBackupEngineAzureBlobStorage(BackupFactory & factory) { From 2d914721e5101215c2c63c97151552cb7c8ff746 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 8 Jan 2024 15:10:37 +0100 Subject: [PATCH 036/276] Fix build --- .../registerBackupEngineAzureBlobStorage.cpp | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/src/Backups/registerBackupEngineAzureBlobStorage.cpp b/src/Backups/registerBackupEngineAzureBlobStorage.cpp index 810da5adb3f..3480ea75f1f 100644 --- a/src/Backups/registerBackupEngineAzureBlobStorage.cpp +++ b/src/Backups/registerBackupEngineAzureBlobStorage.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #endif @@ -23,6 +24,22 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } +#if USE_AZURE_BLOB_STORAGE +namespace +{ + String removeFileNameFromURL(String & url) + { + Poco::URI url2{url}; + String path = url2.getPath(); + size_t slash_pos = path.find_last_of('/'); + String file_name = path.substr(slash_pos + 1); + path.resize(slash_pos + 1); + url2.setPath(path); + url = url2.toString(); + return file_name; + } +} +#endif void registerBackupEngineAzureBlobStorage(BackupFactory & factory) { From c5bf722ee2d2b50d1b0691112b769e3e67612214 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 8 Jan 2024 21:24:44 +0300 Subject: [PATCH 037/276] Create ch/chc/chl symlinks by cmake as well (for develop mode) Before, they had been created only by install target. Follow-up for: #56634 Signed-off-by: Azat Khuzhin --- programs/CMakeLists.txt | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index b3a5af6d6c9..6e544bac81c 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -328,6 +328,10 @@ set (CLICKHOUSE_BUNDLE) if (ENABLE_CLICKHOUSE_SELF_EXTRACTING) list(APPEND CLICKHOUSE_BUNDLE self-extracting) endif () + +if (NOT BUILD_STANDALONE_KEEPER) + add_custom_target (ch ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse ch DEPENDS clickhouse) +endif() if (ENABLE_CLICKHOUSE_SERVER) add_custom_target (clickhouse-server ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-server DEPENDS clickhouse) install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-server" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) @@ -335,11 +339,13 @@ if (ENABLE_CLICKHOUSE_SERVER) endif () if (ENABLE_CLICKHOUSE_CLIENT) add_custom_target (clickhouse-client ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-client DEPENDS clickhouse) + add_custom_target (chc ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse chc DEPENDS clickhouse) install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-client" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) list(APPEND CLICKHOUSE_BUNDLE clickhouse-client) endif () if (ENABLE_CLICKHOUSE_LOCAL) add_custom_target (clickhouse-local ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-local DEPENDS clickhouse) + add_custom_target (chl ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse chl DEPENDS clickhouse) install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-local" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) list(APPEND CLICKHOUSE_BUNDLE clickhouse-local) endif () From ffde721f08359e0437c44026881e2514012a4966 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 13 Jan 2024 23:09:10 +0300 Subject: [PATCH 038/276] Update 02932_set_ttl_where.sql --- tests/queries/0_stateless/02932_set_ttl_where.sql | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/02932_set_ttl_where.sql b/tests/queries/0_stateless/02932_set_ttl_where.sql index bf2b317c4bf..ee8473e1af2 100644 --- a/tests/queries/0_stateless/02932_set_ttl_where.sql +++ b/tests/queries/0_stateless/02932_set_ttl_where.sql @@ -1,3 +1,5 @@ +-- Tags: no-ordinary-database + create or replace table t_temp ( a UInt32, timestamp DateTime @@ -12,3 +14,5 @@ select sleep(1); insert into t_temp select rand(), now() from system.numbers limit 1_000_000; select sleep(1); optimize table t_temp final; + +DROP TABLE t_temp; From 12585ea0e4cae1771ee6b51dd85a309e5923f12c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 13 Jan 2024 23:10:27 +0300 Subject: [PATCH 039/276] Update TTLDescription.cpp --- src/Storages/TTLDescription.cpp | 25 ------------------------- 1 file changed, 25 deletions(-) diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index bfd3afc30d8..3db5269b617 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -117,11 +117,6 @@ TTLDescription::TTLDescription(const TTLDescription & other) , if_exists(other.if_exists) , recompression_codec(other.recompression_codec) { - // if (other.expression) - // expression = other.expression->clone(); - - // if (other.where_expression) - // where_expression = other.where_expression->clone(); } TTLDescription & TTLDescription::operator=(const TTLDescription & other) @@ -135,11 +130,6 @@ TTLDescription & TTLDescription::operator=(const TTLDescription & other) else expression_ast.reset(); - // if (other.expression) - // expression = other.expression->clone(); - // else - // expression.reset(); - expression_columns = other.expression_columns; result_column = other.result_column; @@ -148,11 +138,6 @@ TTLDescription & TTLDescription::operator=(const TTLDescription & other) else where_expression_ast.reset(); - // if (other.where_expression) - // where_expression = other.where_expression->clone(); - // else - // where_expression.reset(); - where_expression_columns = other.where_expression_columns; where_result_column = other.where_result_column; group_by_keys = other.group_by_keys; @@ -179,7 +164,6 @@ static ExpressionAndSets buildExpressionAndSets(ASTPtr & ast, const NamesAndType auto dag = analyzer.getActionsDAG(false); const auto * col = &dag->findInOutputs(ast->getColumnName()); - // std::cerr << "buildExpressionAndSets " << ttl_string << std::endl; if (col->result_name != ttl_string) col = &dag->addAlias(*col, ttl_string); @@ -189,10 +173,6 @@ static ExpressionAndSets buildExpressionAndSets(ASTPtr & ast, const NamesAndType result.expression = std::make_shared(dag, ExpressionActionsSettings::fromContext(context)); result.sets = analyzer.getPreparedSets(); - // std::cerr << "--------- buildExpressionAndSets\n"; - // std::cerr << result.expression->dumpActions() << std::endl; - // std::cerr << result.sets->getSubqueries().size() << std::endl; - return result; } @@ -232,8 +212,6 @@ TTLDescription TTLDescription::getTTLFromAST( auto expression = buildExpressionAndSets(ttl_ast, columns.getAllPhysical(), context).expression; result.expression_columns = expression->getRequiredColumnsWithTypes(); - // auto syntax_analyzer_result = TreeRewriter(context).analyze(ttl_ast, columns.getAllPhysical()); - // result.expression = ExpressionAnalyzer(ttl_ast, syntax_analyzer_result, context).getActions(false); result.result_column = expression->getSampleBlock().safeGetByPosition(0).name; ExpressionActionsPtr where_expression; @@ -256,9 +234,6 @@ TTLDescription TTLDescription::getTTLFromAST( { result.where_expression_ast = where_expr_ast->clone(); where_expression = buildExpressionAndSets(where_expr_ast, columns.getAllPhysical(), context).expression; - // auto where_syntax_result = TreeRewriter(context).analyze(where_expr_ast, columns.getAllPhysical()); - // result.where_expression = ExpressionAnalyzer(where_expr_ast, where_syntax_result, context).getActions(false); - result.where_expression_columns = where_expression->getRequiredColumnsWithTypes(); result.where_result_column = where_expression->getSampleBlock().safeGetByPosition(0).name; } From 776ea26ce71287735897b00c65b47d73e8d9811c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jan 2024 02:45:51 +0300 Subject: [PATCH 040/276] Update PreparedSets.h --- src/Interpreters/PreparedSets.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index 30bfda4700d..4f5ca337c5b 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -115,7 +115,6 @@ public: SetPtr buildSetInplace(const ContextPtr & context); std::unique_ptr build(const ContextPtr & context); - void buildSetInplace(const ContextPtr & context); QueryTreeNodePtr detachQueryTree() { return std::move(query_tree); } void setQueryPlan(std::unique_ptr source_); From 1afc5e8c01685d1bb3e86b5a0fff55618db517b0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jan 2024 03:45:42 +0100 Subject: [PATCH 041/276] Enable coverage for debug build --- docker/packager/packager | 8 ++++++++ tests/ci/build_check.py | 2 ++ tests/ci/ci_config.py | 2 ++ 3 files changed, 12 insertions(+) diff --git a/docker/packager/packager b/docker/packager/packager index ade36a55591..4c443896f4a 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -121,6 +121,7 @@ def is_release_build(debug_build: bool, package_type: str, sanitizer: str) -> bo def parse_env_variables( debug_build: bool, + coverage: bool, compiler: str, sanitizer: str, package_type: str, @@ -287,6 +288,9 @@ def parse_env_variables( else: result.append("BUILD_TYPE=None") + if coverage: + result.append("SANITIZE_COVERAGE=1") + if not cache: cmake_flags.append("-DCOMPILER_CACHE=disabled") @@ -415,6 +419,9 @@ def parse_args() -> argparse.Namespace: choices=("address", "thread", "memory", "undefined", ""), default="", ) + parser.add_argument( + "--coverage", action="store_true", help="enable granular coverage with introspection" + ) parser.add_argument("--clang-tidy", action="store_true") parser.add_argument( @@ -507,6 +514,7 @@ def main() -> None: env_prepared = parse_env_variables( args.debug_build, + args.coverage, args.compiler, args.sanitizer, args.package_type, diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 27243aac4f1..fe4308f5933 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -73,6 +73,8 @@ def get_packager_cmd( cmd += " --debug-build" if build_config.sanitizer: cmd += f" --sanitizer={build_config.sanitizer}" + if build_config.coverage: + cmd += " --coverage" if build_config.tidy: cmd += " --clang-tidy" diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index e3319fe4a72..b8dff3f0a28 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -62,6 +62,7 @@ class BuildConfig: package_type: Literal["deb", "binary", "fuzzers"] additional_pkgs: bool = False debug_build: bool = False + coverage: bool = False sanitizer: str = "" tidy: bool = False sparse_checkout: bool = False @@ -473,6 +474,7 @@ CI_CONFIG = CiConfig( name="package_debug", compiler="clang-17", debug_build=True, + coverage=True, package_type="deb", sparse_checkout=True, ), From 0219d58d925bd3f7901f9251c2abca76c1ae00dc Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sun, 14 Jan 2024 02:56:50 +0000 Subject: [PATCH 042/276] Automatic style fix --- docker/packager/packager | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docker/packager/packager b/docker/packager/packager index 4c443896f4a..3e7f1ba447e 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -420,7 +420,9 @@ def parse_args() -> argparse.Namespace: default="", ) parser.add_argument( - "--coverage", action="store_true", help="enable granular coverage with introspection" + "--coverage", + action="store_true", + help="enable granular coverage with introspection", ) parser.add_argument("--clang-tidy", action="store_true") From 6405decbb0ad0e80fe20b22a9956481abbe3b479 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jan 2024 05:14:54 +0100 Subject: [PATCH 043/276] Fix Python --- docker/packager/packager | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/packager b/docker/packager/packager index 4c443896f4a..2e2b6550636 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -289,7 +289,7 @@ def parse_env_variables( result.append("BUILD_TYPE=None") if coverage: - result.append("SANITIZE_COVERAGE=1") + cmake_flags.append("-DSANITIZE_COVERAGE=1") if not cache: cmake_flags.append("-DCOMPILER_CACHE=disabled") From 9f5a7c51175dc3d4cfe46065b4912e7973a30983 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jan 2024 08:05:57 +0100 Subject: [PATCH 044/276] Fix error --- cmake/sanitize.cmake | 1 + contrib/jemalloc-cmake/CMakeLists.txt | 3 +++ 2 files changed, 4 insertions(+) diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index 3f7a8498059..3882b51227e 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -82,3 +82,4 @@ if (SANITIZE_COVERAGE) endif() set (WITHOUT_COVERAGE_FLAGS "-fno-profile-instr-generate -fno-coverage-mapping -fno-sanitize-coverage=trace-pc-guard,pc-table") +set (WITHOUT_COVERAGE_FLAGS_LIST -fno-profile-instr-generate -fno-coverage-mapping -fno-sanitize-coverage=trace-pc-guard,pc-table) diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index 15e965ed841..f85a38dcf8a 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -161,6 +161,9 @@ target_include_directories(_jemalloc SYSTEM PRIVATE target_compile_definitions(_jemalloc PRIVATE -DJEMALLOC_NO_PRIVATE_NAMESPACE) +# Because our coverage callbacks call malloc, and recursive call of malloc could not work. +target_compile_options(_jemalloc PRIVATE ${WITHOUT_COVERAGE_FLAGS_LIST}) + if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") target_compile_definitions(_jemalloc PRIVATE -DJEMALLOC_DEBUG=1 From 3d904cbf81eb6ce2472eabdcd0be5f6955984ce5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jan 2024 08:09:08 +0100 Subject: [PATCH 045/276] Slightly better --- base/base/coverage.cpp | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/base/base/coverage.cpp b/base/base/coverage.cpp index d70c3bcd82b..ac8055e836c 100644 --- a/base/base/coverage.cpp +++ b/base/base/coverage.cpp @@ -1,4 +1,5 @@ #include "coverage.h" +#include #pragma GCC diagnostic ignored "-Wreserved-identifier" @@ -57,6 +58,14 @@ namespace uintptr_t * all_addresses_array = nullptr; size_t all_addresses_array_size = 0; + + uintptr_t * allocate(size_t size) + { + void * map = mmap(nullptr, size, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0); + if (MAP_FAILED == map) + return nullptr; + return static_cast(map); + } } extern "C" @@ -79,7 +88,7 @@ void __sanitizer_cov_trace_pc_guard_init(uint32_t * start, uint32_t * stop) coverage_array_size = stop - start; /// Note: we will leak this. - coverage_array = static_cast(malloc(sizeof(uintptr_t) * coverage_array_size)); + coverage_array = allocate(sizeof(uintptr_t) * coverage_array_size); resetCoverage(); } @@ -92,7 +101,7 @@ void __sanitizer_cov_pcs_init(const uintptr_t * pcs_begin, const uintptr_t * pcs return; pc_table_initialized = true; - all_addresses_array = static_cast(malloc(sizeof(uintptr_t) * coverage_array_size)); + all_addresses_array = allocate(sizeof(uintptr_t) * coverage_array_size); all_addresses_array_size = pcs_end - pcs_begin; /// They are not a real pointers, but also contain a flag in the most significant bit, From 33d9a1d4e83d58f15e36ea6e88908c8410f03c40 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jan 2024 22:03:47 +0100 Subject: [PATCH 046/276] Documentation --- src/Functions/coverage.cpp | 48 ++++++++++++++++++++++++++++++++++++-- 1 file changed, 46 insertions(+), 2 deletions(-) diff --git a/src/Functions/coverage.cpp b/src/Functions/coverage.cpp index 8a62469fa54..86de047a76b 100644 --- a/src/Functions/coverage.cpp +++ b/src/Functions/coverage.cpp @@ -85,8 +85,52 @@ public: REGISTER_FUNCTION(Coverage) { - factory.registerFunction("coverage", [](ContextPtr){ return std::make_unique(std::make_shared(Kind::Current)); }); - factory.registerFunction("coverageAll", [](ContextPtr){ return std::make_unique(std::make_shared(Kind::All)); }); + factory.registerFunction("coverage", [](ContextPtr){ return std::make_unique(std::make_shared(Kind::Current)); }, + FunctionDocumentation + { + .description=R"( +This function is only available if ClickHouse was built with the SANITIZE_COVERAGE=1 option. + +It returns an array of unique addresses (a subset of the instrumented points in code) in the code +encountered at runtime after the previous coverage reset (with the `SYSTEM RESET COVERAGE` query) or after server startup. + +[example:functions] + +The order of array elements is undetermined. + +You can use another function, `coverageAll` to find all instrumented addresses in the code to compare and calculate the percentage. + +You can process the addresses with the `addressToSymbol` (possibly with `demangle`) and `addressToLine` functions +to calculate symbol-level, file-level, or line-level coverage. + +If you run multiple tests sequentially and reset the coverage with the `SYSTEM RESET COVERAGE` query between the tests, +you can obtain a coverage information for every test in isolation, to find which functions are covered by which tests and vise-versa. + +By default, every *basic block* in the code is covered, which roughly means - a sequence of instructions without jumps, +e.g. a body of for loop without ifs, or a single branch of if. + +See https://clang.llvm.org/docs/SanitizerCoverage.html for more information. +)", + .examples{ + {"functions", "SELECT DISTINCT demangle(addressToSymbol(arrayJoin(coverage())))", ""}}, + .categories{"Introspection"} + }); + + factory.registerFunction("coverageAll", [](ContextPtr){ return std::make_unique(std::make_shared(Kind::All)); }, + FunctionDocumentation + { + .description=R"( +This function is only available if ClickHouse was built with the SANITIZE_COVERAGE=1 option. + +It returns an array of all unique addresses in the code instrumented for coverage +- all possible addresses that can appear in the result of the `coverage` function. + +You can use this function, and the `coverage` function to compare and calculate the coverage percentage. + +See the `coverage` function for the details. +)", + .categories{"Introspection"} + }); } } From 3bd2c7e384d07d07da8768aa4708c7726b828db5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 14 Jan 2024 22:06:25 +0100 Subject: [PATCH 047/276] Report coverage if available --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index c7049b0e0c8..2d278f18176 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -2840,7 +2840,7 @@ def parse_args(): parser.add_argument( "--collect-per-test-coverage", action="store_true", - default=False, + default=True, help="Create `system.coverage` table on the server and collect information about low-level code coverage on a per test basis there", ) parser.add_argument( From 9141e1693f03f39d2eda37423918d2b2d873877a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jan 2024 00:24:17 +0100 Subject: [PATCH 048/276] Calculate cumulative coverage by default. --- tests/clickhouse-test | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 2d278f18176..f1b20a3a43e 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1322,6 +1322,7 @@ class TestCase: # We want to calculate per-test code coverage. That's why we reset it before each test. if ( args.collect_per_test_coverage + and args.reset_coverage_before_every_test and BuildFlags.SANITIZE_COVERAGE in args.build_flags ): clickhouse_execute( @@ -2843,6 +2844,12 @@ def parse_args(): default=True, help="Create `system.coverage` table on the server and collect information about low-level code coverage on a per test basis there", ) + parser.add_argument( + "--reset-coverage-before-every-test", + action="store_true", + default=False, + help="Collect isolated test coverage for every test instead of a cumulative. Useful only when tests are run sequentially.", + ) parser.add_argument( "--report-logs-stats", action="store_true", From f7abeff0857ec231a7107d2a006b5f98b60a689f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jan 2024 00:25:07 +0100 Subject: [PATCH 049/276] Slightly better reporting --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index f1b20a3a43e..e480957e5f4 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1259,7 +1259,7 @@ class TestCase: retry_error_codes=True, ).decode() - description_full += f" Coverage: {coverage}" + description_full += f" (coverage: {coverage})" description_full += "\n" From 3e09feda336a355173b46ec85a9cd86d640f3348 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jan 2024 00:36:07 +0100 Subject: [PATCH 050/276] More functions --- base/base/coverage.cpp | 21 +++++++++++++++------ base/base/coverage.h | 5 ++++- src/Functions/coverage.cpp | 33 ++++++++++++++++++++++++++++----- tests/clickhouse-test | 6 +++--- 4 files changed, 50 insertions(+), 15 deletions(-) diff --git a/base/base/coverage.cpp b/base/base/coverage.cpp index ac8055e836c..499e384d21f 100644 --- a/base/base/coverage.cpp +++ b/base/base/coverage.cpp @@ -53,7 +53,8 @@ namespace uint32_t * guards_start = nullptr; uint32_t * guards_end = nullptr; - uintptr_t * coverage_array = nullptr; + uintptr_t * current_coverage_array = nullptr; + uintptr_t * cumulative_coverage_array = nullptr; size_t coverage_array_size = 0; uintptr_t * all_addresses_array = nullptr; @@ -88,7 +89,8 @@ void __sanitizer_cov_trace_pc_guard_init(uint32_t * start, uint32_t * stop) coverage_array_size = stop - start; /// Note: we will leak this. - coverage_array = allocate(sizeof(uintptr_t) * coverage_array_size); + current_coverage_array = allocate(sizeof(uintptr_t) * coverage_array_size); + cumulative_coverage_array = allocate(sizeof(uintptr_t) * coverage_array_size); resetCoverage(); } @@ -126,15 +128,22 @@ void __sanitizer_cov_trace_pc_guard(uint32_t * guard) /// and use them to dereference an array or a bit vector. void * pc = __builtin_return_address(0); - coverage_array[guard - guards_start] = reinterpret_cast(pc); + current_coverage_array[guard - guards_start] = reinterpret_cast(pc); + cumulative_coverage_array[guard - guards_start] = reinterpret_cast(pc); } } __attribute__((no_sanitize("coverage"))) -std::span getCoverage() +std::span getCurrentCoverage() { - return {coverage_array, coverage_array_size}; + return {current_coverage_array, coverage_array_size}; +} + +__attribute__((no_sanitize("coverage"))) +std::span getCumulativeCoverage() +{ + return {cumulative_coverage_array, coverage_array_size}; } __attribute__((no_sanitize("coverage"))) @@ -146,7 +155,7 @@ std::span getAllInstrumentedAddresses() __attribute__((no_sanitize("coverage"))) void resetCoverage() { - memset(coverage_array, 0, coverage_array_size * sizeof(*coverage_array)); + memset(current_coverage_array, 0, coverage_array_size * sizeof(*current_coverage_array)); /// The guard defines whether the __sanitizer_cov_trace_pc_guard should be called. /// For example, you can unset it after first invocation to prevent excessive work. diff --git a/base/base/coverage.h b/base/base/coverage.h index f75ed2d3553..a6e5a6848d7 100644 --- a/base/base/coverage.h +++ b/base/base/coverage.h @@ -15,7 +15,10 @@ void dumpCoverageReportIfPossible(); /// Get accumulated unique program addresses of the instrumented parts of the code, /// seen so far after program startup or after previous reset. /// The returned span will be represented as a sparse map, containing mostly zeros, which you should filter away. -std::span getCoverage(); +std::span getCurrentCoverage(); + +/// Similar but not being reset. +std::span getCumulativeCoverage(); /// Get all instrumented addresses that could be in the coverage. std::span getAllInstrumentedAddresses(); diff --git a/src/Functions/coverage.cpp b/src/Functions/coverage.cpp index 86de047a76b..f4cac26df78 100644 --- a/src/Functions/coverage.cpp +++ b/src/Functions/coverage.cpp @@ -21,11 +21,14 @@ namespace enum class Kind { Current, + Cumulative, All }; /** If ClickHouse is build with coverage instrumentation, returns an array - * of currently accumulated (`coverage`) / all possible (`coverageAll`) unique code addresses. + * of currently accumulated (`coverageCurrent`) + * or accumulated since the startup (`coverageCumulative`) + * or all possible (`coverageAll`) unique code addresses. */ class FunctionCoverage : public IFunction { @@ -64,7 +67,11 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override { - auto coverage_table = kind == Kind::Current ? getCoverage() : getAllInstrumentedAddresses(); + auto coverage_table = kind == Kind::Current + ? getCurrentCoverage() + : (kind == Kind::Cumulative + ? getCumulativeCoverage() + : getAllInstrumentedAddresses()); auto column_addresses = ColumnUInt64::create(); auto & data = column_addresses->getData(); @@ -85,7 +92,7 @@ public: REGISTER_FUNCTION(Coverage) { - factory.registerFunction("coverage", [](ContextPtr){ return std::make_unique(std::make_shared(Kind::Current)); }, + factory.registerFunction("coverageCurrent", [](ContextPtr){ return std::make_unique(std::make_shared(Kind::Current)); }, FunctionDocumentation { .description=R"( @@ -112,7 +119,23 @@ e.g. a body of for loop without ifs, or a single branch of if. See https://clang.llvm.org/docs/SanitizerCoverage.html for more information. )", .examples{ - {"functions", "SELECT DISTINCT demangle(addressToSymbol(arrayJoin(coverage())))", ""}}, + {"functions", "SELECT DISTINCT demangle(addressToSymbol(arrayJoin(coverageCurrent())))", ""}}, + .categories{"Introspection"} + }); + + factory.registerFunction("coverageCumulative", [](ContextPtr){ return std::make_unique(std::make_shared(Kind::Cumulative)); }, + FunctionDocumentation + { + .description=R"( +This function is only available if ClickHouse was built with the SANITIZE_COVERAGE=1 option. + +It returns an array of unique addresses (a subset of the instrumented points in code) in the code +encountered at runtime after server startup. + +In contrast to `coverageCurrent` it cannot be reset with the `SYSTEM RESET COVERAGE`. + +See the `coverageCurrent` function for the details. +)", .categories{"Introspection"} }); @@ -127,7 +150,7 @@ It returns an array of all unique addresses in the code instrumented for coverag You can use this function, and the `coverage` function to compare and calculate the coverage percentage. -See the `coverage` function for the details. +See the `coverageCurrent` function for the details. )", .categories{"Introspection"} }); diff --git a/tests/clickhouse-test b/tests/clickhouse-test index e480957e5f4..a39c90947ba 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1249,13 +1249,13 @@ class TestCase: ): clickhouse_execute( args, - f"INSERT INTO system.coverage SELECT now(), '{self.case}', coverage()", + f"INSERT INTO system.coverage SELECT now(), '{self.case}', coverageCurrent()", retry_error_codes=True, ) coverage = clickhouse_execute( args, - "SELECT length(coverage())", + "SELECT length(coverageCurrent())", retry_error_codes=True, ).decode() @@ -2460,7 +2460,7 @@ def main(args): # Coverage collected at the system startup before running any tests: clickhouse_execute( args, - "INSERT INTO system.coverage SELECT now(), '', coverage()", + "INSERT INTO system.coverage SELECT now(), '', coverageCurrent()", ) total_tests_run = 0 From e4cd02ea39642dd9b8d519aee0426b752423c3bf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jan 2024 00:36:24 +0100 Subject: [PATCH 051/276] Fix typo --- src/IO/OpenedFile.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/IO/OpenedFile.h b/src/IO/OpenedFile.h index 10c36d9e1d3..4c4de2265bc 100644 --- a/src/IO/OpenedFile.h +++ b/src/IO/OpenedFile.h @@ -21,7 +21,7 @@ public: OpenedFile(const std::string & file_name_, int flags_); ~OpenedFile(); - /// Close prematurally. + /// Close prematurely. void close(); int getFD() const; @@ -40,4 +40,3 @@ private: }; } - From 30c362909089d6f7fe93b639dfdf1666d5bcfc7c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jan 2024 00:36:44 +0100 Subject: [PATCH 052/276] An option to dump coverage to a file at exit --- programs/main.cpp | 47 ++++++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 46 insertions(+), 1 deletion(-) diff --git a/programs/main.cpp b/programs/main.cpp index 7d07112de66..4852ed8990e 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include @@ -19,6 +20,7 @@ #include #include +#include /// Universal executable for various clickhouse applications @@ -512,6 +514,49 @@ int main(int argc_, char ** argv_) if (main_func == printHelp && !argv.empty() && (argv.size() == 1 || argv[1][0] == '-')) main_func = mainEntryClickHouseLocal; - return main_func(static_cast(argv.size()), argv.data()); + int exit_code = main_func(static_cast(argv.size()), argv.data()); + +#if defined(SANITIZE_COVERAGE) + /// A user can request to dump the coverage information into files at exit. + /// This is useful for non-server applications such as clickhouse-format or clickhouse-client, + /// that cannot introspect it with SQL functions at runtime. + + /// The CLICKHOUSE_WRITE_COVERAGE environment variable defines a prefix for two filenames: + /// 'prefix.covered' and 'prefix.all' which will contain + /// the list of addresses of covered and all instrumented addresses, respectively. + + /// The format is even simpler than Clang's "sancov": an array of 64-bit addresses, native byte order, no header. + + if (const char * coverage_filename_prefix = getenv("CLICKHOUSE_WRITE_COVERAGE")) // NOLINT(concurrency-mt-unsafe) + { + auto dumpCoverage = [](const std::string & name, auto span) + { + /// Write only non-zeros. + std::vector data; + data.reserve(span.size()); + for (auto addr : span) + if (addr) + data.push_back(addr); + + int fd = ::open(name.c_str(), O_WRONLY | O_CREAT | O_TRUNC | O_CLOEXEC, 0400); + if (-1 == fd) + { + writeError("Cannot open a file to write the coverage data\n"); + } + else + { + if (!writeRetry(fd, reinterpret_cast(data.data()), data.size() * sizeof(data[0]))) + writeError("Cannot write the coverage data to a file\n"); + if (0 != ::close(fd)) + writeError("Cannot close the file with coverage data\n"); + } + }; + + dumpCoverage(coverage_filename_prefix + std::string(".covered"), getCumulativeCoverage()); + dumpCoverage(coverage_filename_prefix + std::string(".all"), getAllInstrumentedAddresses()); + } +#endif + + return exit_code; } #endif From fe952fb64c460c260c77336142b5eb4bd05b46d8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jan 2024 00:38:42 +0100 Subject: [PATCH 053/276] Rename to system.coverage_log to simplify export --- tests/clickhouse-test | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a39c90947ba..eb85bdff0f5 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1249,7 +1249,7 @@ class TestCase: ): clickhouse_execute( args, - f"INSERT INTO system.coverage SELECT now(), '{self.case}', coverageCurrent()", + f"INSERT INTO system.coverage_log SELECT now(), '{self.case}', coverageCurrent()", retry_error_codes=True, ) @@ -2448,7 +2448,7 @@ def main(args): clickhouse_execute( args, """ - CREATE TABLE IF NOT EXISTS system.coverage + CREATE TABLE IF NOT EXISTS system.coverage_log ( time DateTime, test_name String, @@ -2460,7 +2460,7 @@ def main(args): # Coverage collected at the system startup before running any tests: clickhouse_execute( args, - "INSERT INTO system.coverage SELECT now(), '', coverageCurrent()", + "INSERT INTO system.coverage_log SELECT now(), '', coverageCurrent()", ) total_tests_run = 0 @@ -2842,7 +2842,7 @@ def parse_args(): "--collect-per-test-coverage", action="store_true", default=True, - help="Create `system.coverage` table on the server and collect information about low-level code coverage on a per test basis there", + help="Create `system.coverage_log` table on the server and collect information about low-level code coverage on a per test basis there", ) parser.add_argument( "--reset-coverage-before-every-test", From 7662628393f97dd1c094b3346cc55c71f10ad193 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jan 2024 00:41:14 +0100 Subject: [PATCH 054/276] Export coverage to the CI database --- docker/test/base/setup_export_logs.sh | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index ea82e071112..659bf29b057 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -129,6 +129,19 @@ function setup_logs_replication debug_or_sanitizer_build=$(clickhouse-client -q "WITH ((SELECT value FROM system.build_options WHERE name='BUILD_TYPE') AS build, (SELECT value FROM system.build_options WHERE name='CXX_FLAGS') as flags) SELECT build='Debug' OR flags LIKE '%fsanitize%'") echo "Build is debug or sanitizer: $debug_or_sanitizer_build" + # We will pre-create a table system.coverage_log. + # It is normally created by clickhouse-test rather than the server, + # so we will create it in advance to make it be picked up by the next commands: + + clickhouse-client --query " + CREATE TABLE IF NOT EXISTS system.coverage_log + ( + time DateTime, + test_name String, + coverage Array(UInt64) + ) ENGINE = MergeTree ORDER BY test_name + " + # For each system log table: echo 'Create %_log tables' clickhouse-client --query "SHOW TABLES FROM system LIKE '%\\_log'" | while read -r table From 97200e2c5d65693ad5d1e6a7c7dea3d5cac0e23d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jan 2024 00:46:01 +0100 Subject: [PATCH 055/276] Symbolization --- docker/test/base/setup_export_logs.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 659bf29b057..e141bc00a77 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -138,7 +138,8 @@ function setup_logs_replication ( time DateTime, test_name String, - coverage Array(UInt64) + coverage Array(UInt64), + symbols Array(LowCardinality(String)) MATERIALIZED arrayMap(x -> demangle(addressToSymbol(x)), coverage) ) ENGINE = MergeTree ORDER BY test_name " From bf2e5748575ad2eb74eb057e0ee242a149edecdb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jan 2024 00:48:47 +0100 Subject: [PATCH 056/276] Symbolization --- docker/test/base/setup_export_logs.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index e141bc00a77..20dd864318f 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -188,7 +188,7 @@ function setup_logs_replication echo "Creating table system.${table}_sender" >&2 # Create Distributed table and materialized view to watch on the original table: - clickhouse-client --query " + clickhouse-client --asterisk_include_materialized_columns 1 --query " CREATE TABLE system.${table}_sender ENGINE = Distributed(${CLICKHOUSE_CI_LOGS_CLUSTER}, default, ${table}_${hash}) SETTINGS flush_on_detach=0 @@ -199,7 +199,7 @@ function setup_logs_replication echo "Creating materialized view system.${table}_watcher" >&2 - clickhouse-client --query " + clickhouse-client --asterisk_include_materialized_columns 1 --query " CREATE MATERIALIZED VIEW system.${table}_watcher TO system.${table}_sender AS SELECT ${EXTRA_COLUMNS_EXPRESSION_FOR_TABLE}, * FROM system.${table} From c5dfae1bcade85289b78f0bb760c92bcee078743 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jan 2024 05:07:23 +0100 Subject: [PATCH 057/276] Fix error --- docker/test/base/setup_export_logs.sh | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 20dd864318f..26fcd10d666 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -23,6 +23,10 @@ EXTRA_ORDER_BY_COLUMNS=${EXTRA_ORDER_BY_COLUMNS:-"check_name, "} EXTRA_COLUMNS_TRACE_LOG="${EXTRA_COLUMNS} symbols Array(LowCardinality(String)), lines Array(LowCardinality(String)), " EXTRA_COLUMNS_EXPRESSION_TRACE_LOG="${EXTRA_COLUMNS_EXPRESSION}, arrayMap(x -> demangle(addressToSymbol(x)), trace)::Array(LowCardinality(String)) AS symbols, arrayMap(x -> addressToLine(x), trace)::Array(LowCardinality(String)) AS lines" +# coverage_log needs more columns for symbolization, but only symbol names (the line numbers are too heavy to calculate) +EXTRA_COLUMNS_COVERAGE_LOG="${EXTRA_COLUMNS} symbols Array(LowCardinality(String)), " +EXTRA_COLUMNS_EXPRESSION_COVERAGE_LOG="${EXTRA_COLUMNS_EXPRESSION}, arrayMap(x -> demangle(addressToSymbol(x)), coverage)::Array(LowCardinality(String)) AS symbols" + function __set_connection_args { @@ -138,8 +142,7 @@ function setup_logs_replication ( time DateTime, test_name String, - coverage Array(UInt64), - symbols Array(LowCardinality(String)) MATERIALIZED arrayMap(x -> demangle(addressToSymbol(x)), coverage) + coverage Array(UInt64) ) ENGINE = MergeTree ORDER BY test_name " @@ -158,7 +161,10 @@ function setup_logs_replication else EXTRA_COLUMNS_EXPRESSION_FOR_TABLE="${EXTRA_COLUMNS_EXPRESSION_TRACE_LOG}" fi - else + elif [[ "$table" = "coverage_log" ]] + EXTRA_COLUMNS_FOR_TABLE="${EXTRA_COLUMNS_COVERAGE_LOG}" + EXTRA_COLUMNS_EXPRESSION_FOR_TABLE="${EXTRA_COLUMNS_EXPRESSION_COVERAGE_LOG}" + then EXTRA_COLUMNS_FOR_TABLE="${EXTRA_COLUMNS}" EXTRA_COLUMNS_EXPRESSION_FOR_TABLE="${EXTRA_COLUMNS_EXPRESSION}" fi From e13ca48bce836a2534047e59a4e922395a8f6a87 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jan 2024 05:13:43 +0100 Subject: [PATCH 058/276] Better dump on exit --- programs/main.cpp | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/programs/main.cpp b/programs/main.cpp index 4852ed8990e..8958d84e243 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -521,9 +521,8 @@ int main(int argc_, char ** argv_) /// This is useful for non-server applications such as clickhouse-format or clickhouse-client, /// that cannot introspect it with SQL functions at runtime. - /// The CLICKHOUSE_WRITE_COVERAGE environment variable defines a prefix for two filenames: - /// 'prefix.covered' and 'prefix.all' which will contain - /// the list of addresses of covered and all instrumented addresses, respectively. + /// The CLICKHOUSE_WRITE_COVERAGE environment variable defines a prefix for a filename 'prefix.pid' + /// containing the list of addresses of covered . /// The format is even simpler than Clang's "sancov": an array of 64-bit addresses, native byte order, no header. @@ -552,8 +551,7 @@ int main(int argc_, char ** argv_) } }; - dumpCoverage(coverage_filename_prefix + std::string(".covered"), getCumulativeCoverage()); - dumpCoverage(coverage_filename_prefix + std::string(".all"), getAllInstrumentedAddresses()); + dumpCoverage(fmt::format("{}.{}", coverage_filename_prefix, getpid()), getCumulativeCoverage()); } #endif From e49cfbef089499a457c8793724629e2e94c8dc37 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jan 2024 05:40:03 +0100 Subject: [PATCH 059/276] Coverage for non-server tools --- tests/clickhouse-test | 23 +++++++++++++++++++++++ tests/queries/shell_config.sh | 4 ++++ 2 files changed, 27 insertions(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index eb85bdff0f5..bd796dbfdf2 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -12,6 +12,7 @@ import itertools import sys import os import os.path +import glob import platform import signal import re @@ -74,6 +75,10 @@ def stringhash(s): # only during process invocation https://stackoverflow.com/a/42089311 return zlib.crc32(s.encode("utf-8")) +def read_file_as_binary_string(file_path): + with open(file_path, 'rb') as file: + binary_data = file.read() + return binary_data # First and last lines of the log def trim_for_log(s): @@ -101,6 +106,7 @@ class HTTPError(Exception): def clickhouse_execute_http( base_args, query, + body=None, timeout=30, settings=None, default_format=None, @@ -140,6 +146,7 @@ def clickhouse_execute_http( client.request( "POST", f"/?{base_args.client_options_query_str}{urllib.parse.urlencode(params)}", + body=body ) res = client.getresponse() data = res.read() @@ -160,6 +167,7 @@ def clickhouse_execute_http( def clickhouse_execute( base_args, query, + body=None, timeout=30, settings=None, max_http_retries=5, @@ -168,6 +176,7 @@ def clickhouse_execute( return clickhouse_execute_http( base_args, query, + body, timeout, settings, max_http_retries=max_http_retries, @@ -181,6 +190,7 @@ def clickhouse_execute_json( data = clickhouse_execute_http( base_args, query, + None, timeout, settings, "JSONEachRow", @@ -1253,6 +1263,19 @@ class TestCase: retry_error_codes=True, ) + # Check for dumped coverage files + file_pattern = "coverage.*" + matching_files = glob.glob(file_pattern) + for file_path in matching_files: + body = read_file_as_binary_string(file_path) + clickhouse_execute( + args, + f"INSERT INTO system.coverage_log SELECT now(), '{self.case}', groupArray(data) FROM input('data UInt64') FORMAT RowBinary", + body=body, + retry_error_codes=True, + ) + os.remove(file_path) + coverage = clickhouse_execute( args, "SELECT length(coverageCurrent())", diff --git a/tests/queries/shell_config.sh b/tests/queries/shell_config.sh index c687a63623f..614bfcece8f 100644 --- a/tests/queries/shell_config.sh +++ b/tests/queries/shell_config.sh @@ -4,6 +4,10 @@ # Don't check for ODR violation, since we may test shared build with ASAN export ASAN_OPTIONS=detect_odr_violation=0 +# If ClickHouse was built with coverage - dump the coverage information at exit +# (in other cases this environment variable has no effect) +export CLICKHOUSE_WRITE_COVERAGE="coverage" + export CLICKHOUSE_DATABASE=${CLICKHOUSE_DATABASE:="test"} export CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL:="warning"} From 678a32cedee768b6c1a6748e96a0d103e853d8bc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jan 2024 06:12:58 +0100 Subject: [PATCH 060/276] Obey Python's quirky formatter --- tests/integration/ci-runner.py | 13 +- .../test_async_insert_memory/test.py | 2 +- tests/integration/test_check_table/test.py | 76 +++-- .../test_cluster_discovery/test.py | 2 +- .../test_ldap_external_user_directory/test.py | 26 +- tests/integration/test_mysql_protocol/test.py | 16 +- tests/integration/test_partition/test.py | 4 +- .../test_replicated_database/test.py | 9 +- .../test.py | 9 +- .../s3_mocks/unstable_server.py | 2 +- tests/integration/test_storage_s3/test.py | 17 +- tests/integration/test_storage_url/test.py | 22 +- tests/integration/test_system_merges/test.py | 45 ++- utils/grpc-client/pb2/clickhouse_grpc_pb2.py | 271 ++++++++++-------- .../pb2/clickhouse_grpc_pb2_grpc.py | 237 +++++++++------ 15 files changed, 433 insertions(+), 318 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 7c922e339fe..d54ed2bb767 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -305,14 +305,11 @@ class ClickhouseIntegrationTestsRunner: def _pre_pull_images(self, repo_path): image_cmd = self._get_runner_image_cmd(repo_path) - cmd = ( - "cd {repo_path}/tests/integration && " - "timeout --signal=KILL 1h ./runner {runner_opts} {image_cmd} --pre-pull --command '{command}' ".format( - repo_path=repo_path, - runner_opts=self._get_runner_opts(), - image_cmd=image_cmd, - command=r""" echo Pre Pull finished """, - ) + cmd = "cd {repo_path}/tests/integration && " "timeout --signal=KILL 1h ./runner {runner_opts} {image_cmd} --pre-pull --command '{command}' ".format( + repo_path=repo_path, + runner_opts=self._get_runner_opts(), + image_cmd=image_cmd, + command=r""" echo Pre Pull finished """, ) for i in range(5): diff --git a/tests/integration/test_async_insert_memory/test.py b/tests/integration/test_async_insert_memory/test.py index 5d2e5503680..f897007f7bb 100644 --- a/tests/integration/test_async_insert_memory/test.py +++ b/tests/integration/test_async_insert_memory/test.py @@ -43,7 +43,7 @@ def test_memory_usage(): response = node.get_query_request( "SELECT groupArray(number) FROM numbers(1000000) SETTINGS max_memory_usage_for_user={}".format( - 30 * (2**23) + 30 * (2 ** 23) ), user="A", ) diff --git a/tests/integration/test_check_table/test.py b/tests/integration/test_check_table/test.py index 021977fb6b6..ebf404e698b 100644 --- a/tests/integration/test_check_table/test.py +++ b/tests/integration/test_check_table/test.py @@ -95,15 +95,25 @@ def test_check_normal_table_corruption(started_cluster, merge_tree_settings): node1, "non_replicated_mt", "201902_1_1_0", database="default" ) - assert node1.query( - "CHECK TABLE non_replicated_mt", - settings={"check_query_single_value_result": 0, "max_threads": 1}, - ).strip().split("\t")[0:2] == ["201902_1_1_0", "0"] + assert ( + node1.query( + "CHECK TABLE non_replicated_mt", + settings={"check_query_single_value_result": 0, "max_threads": 1}, + ) + .strip() + .split("\t")[0:2] + == ["201902_1_1_0", "0"] + ) - assert node1.query( - "CHECK TABLE non_replicated_mt", - settings={"check_query_single_value_result": 0, "max_threads": 1}, - ).strip().split("\t")[0:2] == ["201902_1_1_0", "0"] + assert ( + node1.query( + "CHECK TABLE non_replicated_mt", + settings={"check_query_single_value_result": 0, "max_threads": 1}, + ) + .strip() + .split("\t")[0:2] + == ["201902_1_1_0", "0"] + ) node1.query( "INSERT INTO non_replicated_mt VALUES (toDate('2019-01-01'), 1, 10), (toDate('2019-01-01'), 2, 12)" @@ -123,10 +133,15 @@ def test_check_normal_table_corruption(started_cluster, merge_tree_settings): remove_checksums_on_disk(node1, "default", "non_replicated_mt", "201901_2_2_0") - assert node1.query( - "CHECK TABLE non_replicated_mt PARTITION 201901", - settings={"check_query_single_value_result": 0, "max_threads": 1}, - ).strip().split("\t")[0:2] == ["201901_2_2_0", "0"] + assert ( + node1.query( + "CHECK TABLE non_replicated_mt PARTITION 201901", + settings={"check_query_single_value_result": 0, "max_threads": 1}, + ) + .strip() + .split("\t")[0:2] + == ["201901_2_2_0", "0"] + ) @pytest.mark.parametrize("merge_tree_settings, zk_path_suffix", [("", "_0")]) @@ -194,12 +209,15 @@ def test_check_replicated_table_simple( == "201901_0_0_0\t1\t\n" ) - assert sorted( - node2.query( - "CHECK TABLE replicated_mt", - settings={"check_query_single_value_result": 0}, - ).split("\n") - ) == ["", "201901_0_0_0\t1\t", "201902_0_0_0\t1\t"] + assert ( + sorted( + node2.query( + "CHECK TABLE replicated_mt", + settings={"check_query_single_value_result": 0}, + ).split("\n") + ) + == ["", "201901_0_0_0\t1\t", "201902_0_0_0\t1\t"] + ) with pytest.raises(QueryRuntimeException) as exc: node2.query( @@ -273,10 +291,13 @@ def test_check_replicated_table_corruption( ) node1.query_with_retry("SYSTEM SYNC REPLICA replicated_mt_1") - assert node1.query( - "CHECK TABLE replicated_mt_1 PARTITION 201901", - settings={"check_query_single_value_result": 0, "max_threads": 1}, - ) == "{}\t1\t\n".format(part_name) + assert ( + node1.query( + "CHECK TABLE replicated_mt_1 PARTITION 201901", + settings={"check_query_single_value_result": 0, "max_threads": 1}, + ) + == "{}\t1\t\n".format(part_name) + ) assert node1.query("SELECT count() from replicated_mt_1") == "4\n" remove_part_from_disk(node2, "replicated_mt_1", part_name) @@ -288,10 +309,13 @@ def test_check_replicated_table_corruption( ) node1.query("SYSTEM SYNC REPLICA replicated_mt_1") - assert node1.query( - "CHECK TABLE replicated_mt_1 PARTITION 201901", - settings={"check_query_single_value_result": 0, "max_threads": 1}, - ) == "{}\t1\t\n".format(part_name) + assert ( + node1.query( + "CHECK TABLE replicated_mt_1 PARTITION 201901", + settings={"check_query_single_value_result": 0, "max_threads": 1}, + ) + == "{}\t1\t\n".format(part_name) + ) assert node1.query("SELECT count() from replicated_mt_1") == "4\n" diff --git a/tests/integration/test_cluster_discovery/test.py b/tests/integration/test_cluster_discovery/test.py index ad3deb5b142..a2e7e15b956 100644 --- a/tests/integration/test_cluster_discovery/test.py +++ b/tests/integration/test_cluster_discovery/test.py @@ -61,7 +61,7 @@ def check_on_cluster( print(f"Retry {retry}/{retries} unsuccessful, result: {node_results}") if retry != retries: - time.sleep(2**retry) + time.sleep(2 ** retry) else: msg = msg or f"Wrong '{what}' result" raise Exception( diff --git a/tests/integration/test_ldap_external_user_directory/test.py b/tests/integration/test_ldap_external_user_directory/test.py index 39753794d63..c9642c293ee 100644 --- a/tests/integration/test_ldap_external_user_directory/test.py +++ b/tests/integration/test_ldap_external_user_directory/test.py @@ -76,11 +76,14 @@ def test_role_mapping(ldap_cluster): "select currentUser()", user="johndoe", password="qwertz" ) == TSV([["johndoe"]]) - assert instance.query( - "select role_name from system.current_roles ORDER BY role_name", - user="johndoe", - password="qwertz", - ) == TSV([["role_1"], ["role_2"]]) + assert ( + instance.query( + "select role_name from system.current_roles ORDER BY role_name", + user="johndoe", + password="qwertz", + ) + == TSV([["role_1"], ["role_2"]]) + ) instance.query("CREATE ROLE role_3") add_ldap_group(ldap_cluster, group_cn="clickhouse-role_3", member_cn="johndoe") @@ -88,8 +91,11 @@ def test_role_mapping(ldap_cluster): # See https://github.com/ClickHouse/ClickHouse/issues/54318 add_ldap_group(ldap_cluster, group_cn="clickhouse-role_4", member_cn="johndoe") - assert instance.query( - "select role_name from system.current_roles ORDER BY role_name", - user="johndoe", - password="qwertz", - ) == TSV([["role_1"], ["role_2"], ["role_3"]]) + assert ( + instance.query( + "select role_name from system.current_roles ORDER BY role_name", + user="johndoe", + password="qwertz", + ) + == TSV([["role_1"], ["role_2"], ["role_3"]]) + ) diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 7a69d07633c..61e76c0dc97 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -854,14 +854,14 @@ def test_types(started_cluster): result = cursor.fetchall()[0] expected = [ - ("Int8_column", -(2**7)), - ("UInt8_column", 2**8 - 1), - ("Int16_column", -(2**15)), - ("UInt16_column", 2**16 - 1), - ("Int32_column", -(2**31)), - ("UInt32_column", 2**32 - 1), - ("Int64_column", -(2**63)), - ("UInt64_column", 2**64 - 1), + ("Int8_column", -(2 ** 7)), + ("UInt8_column", 2 ** 8 - 1), + ("Int16_column", -(2 ** 15)), + ("UInt16_column", 2 ** 16 - 1), + ("Int32_column", -(2 ** 31)), + ("UInt32_column", 2 ** 32 - 1), + ("Int64_column", -(2 ** 63)), + ("UInt64_column", 2 ** 64 - 1), ("String_column", "тест"), ("FixedString_column", "тест"), ("Float32_column", 1.5), diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index 054418a8ba9..d39787f8924 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -561,9 +561,7 @@ def test_make_clone_in_detached(started_cluster): ["cp", "-r", path + "all_0_0_0", path + "detached/broken_all_0_0_0"] ) assert_eq_with_retry(instance, "select * from clone_in_detached", "\n") - assert [ - "broken_all_0_0_0", - ] == sorted( + assert ["broken_all_0_0_0",] == sorted( instance.exec_in_container(["ls", path + "detached/"]).strip().split("\n") ) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 1fc3fe37044..16425c9bd9e 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -506,12 +506,9 @@ def test_alters_from_different_replicas(started_cluster): dummy_node.stop_clickhouse(kill=True) settings = {"distributed_ddl_task_timeout": 5} - assert ( - "There are 1 unfinished hosts (0 of them are currently executing the task" - in competing_node.query_and_get_error( - "ALTER TABLE alters_from_different_replicas.concurrent_test ADD COLUMN Added0 UInt32;", - settings=settings, - ) + assert "There are 1 unfinished hosts (0 of them are currently executing the task" in competing_node.query_and_get_error( + "ALTER TABLE alters_from_different_replicas.concurrent_test ADD COLUMN Added0 UInt32;", + settings=settings, ) settings = { "distributed_ddl_task_timeout": 5, diff --git a/tests/integration/test_replicated_database_cluster_groups/test.py b/tests/integration/test_replicated_database_cluster_groups/test.py index 647626d8014..5a315707efb 100644 --- a/tests/integration/test_replicated_database_cluster_groups/test.py +++ b/tests/integration/test_replicated_database_cluster_groups/test.py @@ -95,12 +95,9 @@ def test_cluster_groups(started_cluster): # Exception main_node_2.stop_clickhouse() settings = {"distributed_ddl_task_timeout": 5} - assert ( - "There are 1 unfinished hosts (0 of them are currently executing the task)" - in main_node_1.query_and_get_error( - "CREATE TABLE cluster_groups.table_2 (d Date, k UInt64) ENGINE=ReplicatedMergeTree ORDER BY k PARTITION BY toYYYYMM(d);", - settings=settings, - ) + assert "There are 1 unfinished hosts (0 of them are currently executing the task)" in main_node_1.query_and_get_error( + "CREATE TABLE cluster_groups.table_2 (d Date, k UInt64) ENGINE=ReplicatedMergeTree ORDER BY k PARTITION BY toYYYYMM(d);", + settings=settings, ) # 3. After start both groups are synced diff --git a/tests/integration/test_storage_s3/s3_mocks/unstable_server.py b/tests/integration/test_storage_s3/s3_mocks/unstable_server.py index 5ef781bdc9e..3632fa15d8a 100644 --- a/tests/integration/test_storage_s3/s3_mocks/unstable_server.py +++ b/tests/integration/test_storage_s3/s3_mocks/unstable_server.py @@ -9,7 +9,7 @@ import time def gen_n_digit_number(n): assert 0 < n < 19 - return random.randint(10 ** (n - 1), 10**n - 1) + return random.randint(10 ** (n - 1), 10 ** n - 1) sum_in_4_column = 0 diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 2549cb0d473..e941356261a 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -553,16 +553,13 @@ def test_multipart(started_cluster, maybe_auth, positive): assert csv_data == get_s3_file_content(started_cluster, bucket, filename) # select uploaded data from many threads - select_query = ( - "select sum(column1), sum(column2), sum(column3) " - "from s3('http://{host}:{port}/{bucket}/{filename}', {auth}'CSV', '{table_format}')".format( - host=started_cluster.minio_redirect_host, - port=started_cluster.minio_redirect_port, - bucket=bucket, - filename=filename, - auth=maybe_auth, - table_format=table_format, - ) + select_query = "select sum(column1), sum(column2), sum(column3) " "from s3('http://{host}:{port}/{bucket}/{filename}', {auth}'CSV', '{table_format}')".format( + host=started_cluster.minio_redirect_host, + port=started_cluster.minio_redirect_port, + bucket=bucket, + filename=filename, + auth=maybe_auth, + table_format=table_format, ) try: select_result = run_query( diff --git a/tests/integration/test_storage_url/test.py b/tests/integration/test_storage_url/test.py index 7ff7a871413..771df49cbac 100644 --- a/tests/integration/test_storage_url/test.py +++ b/tests/integration/test_storage_url/test.py @@ -79,15 +79,21 @@ def test_table_function_url_access_rights(): f"SELECT * FROM url('http://nginx:80/test_1', 'TSV')", user="u1" ) - assert node1.query( - f"DESCRIBE TABLE url('http://nginx:80/test_1', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32')", - user="u1", - ) == TSV([["column1", "UInt32"], ["column2", "UInt32"], ["column3", "UInt32"]]) + assert ( + node1.query( + f"DESCRIBE TABLE url('http://nginx:80/test_1', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32')", + user="u1", + ) + == TSV([["column1", "UInt32"], ["column2", "UInt32"], ["column3", "UInt32"]]) + ) - assert node1.query( - f"DESCRIBE TABLE url('http://nginx:80/not-exist', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32')", - user="u1", - ) == TSV([["column1", "UInt32"], ["column2", "UInt32"], ["column3", "UInt32"]]) + assert ( + node1.query( + f"DESCRIBE TABLE url('http://nginx:80/not-exist', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32')", + user="u1", + ) + == TSV([["column1", "UInt32"], ["column2", "UInt32"], ["column3", "UInt32"]]) + ) expected_error = "necessary to have the grant URL ON *.*" assert expected_error in node1.query_and_get_error( diff --git a/tests/integration/test_system_merges/test.py b/tests/integration/test_system_merges/test.py index 6dbe6c891f2..bacb0eb500d 100644 --- a/tests/integration/test_system_merges/test.py +++ b/tests/integration/test_system_merges/test.py @@ -204,36 +204,33 @@ def test_mutation_simple(started_cluster, replicated): sleep_time=0.1, ) - assert ( - split_tsv( - node_check.query( - """ + assert split_tsv( + node_check.query( + """ SELECT database, table, num_parts, source_part_names, source_part_paths, result_part_name, result_part_path, partition_id, is_mutation FROM system.merges WHERE table = '{name}' """.format( - name=table_name - ) + name=table_name ) ) - == [ - [ - db_name, - table_name, - "1", - "['{}']".format(part), - "['{clickhouse}/{table_path}/{}/']".format( - part, clickhouse=clickhouse_path, table_path=table_path - ), - result_part, - "{clickhouse}/{table_path}/{}/".format( - result_part, clickhouse=clickhouse_path, table_path=table_path - ), - "all", - "1", - ], - ] - ) + ) == [ + [ + db_name, + table_name, + "1", + "['{}']".format(part), + "['{clickhouse}/{table_path}/{}/']".format( + part, clickhouse=clickhouse_path, table_path=table_path + ), + result_part, + "{clickhouse}/{table_path}/{}/".format( + result_part, clickhouse=clickhouse_path, table_path=table_path + ), + "all", + "1", + ], + ] t.join() assert ( diff --git a/utils/grpc-client/pb2/clickhouse_grpc_pb2.py b/utils/grpc-client/pb2/clickhouse_grpc_pb2.py index 6218047af3c..9bf7817c7d3 100644 --- a/utils/grpc-client/pb2/clickhouse_grpc_pb2.py +++ b/utils/grpc-client/pb2/clickhouse_grpc_pb2.py @@ -8,16 +8,17 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import message as _message from google.protobuf import reflection as _reflection from google.protobuf import symbol_database as _symbol_database + # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( + b'\n\x15\x63lickhouse_grpc.proto\x12\x0f\x63lickhouse.grpc")\n\x0bNameAndType\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x0c\n\x04type\x18\x02 \x01(\t"\xf5\x01\n\rExternalTable\x12\x0c\n\x04name\x18\x01 \x01(\t\x12-\n\x07\x63olumns\x18\x02 \x03(\x0b\x32\x1c.clickhouse.grpc.NameAndType\x12\x0c\n\x04\x64\x61ta\x18\x03 \x01(\x0c\x12\x0e\n\x06\x66ormat\x18\x04 \x01(\t\x12\x18\n\x10\x63ompression_type\x18\x06 \x01(\t\x12>\n\x08settings\x18\x05 \x03(\x0b\x32,.clickhouse.grpc.ExternalTable.SettingsEntry\x1a/\n\rSettingsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01"\x85\x03\n\x1cObsoleteTransportCompression\x12U\n\talgorithm\x18\x01 \x01(\x0e\x32\x42.clickhouse.grpc.ObsoleteTransportCompression.CompressionAlgorithm\x12M\n\x05level\x18\x02 \x01(\x0e\x32>.clickhouse.grpc.ObsoleteTransportCompression.CompressionLevel"R\n\x14\x43ompressionAlgorithm\x12\x12\n\x0eNO_COMPRESSION\x10\x00\x12\x0b\n\x07\x44\x45\x46LATE\x10\x01\x12\x08\n\x04GZIP\x10\x02\x12\x0f\n\x0bSTREAM_GZIP\x10\x03"k\n\x10\x43ompressionLevel\x12\x14\n\x10\x43OMPRESSION_NONE\x10\x00\x12\x13\n\x0f\x43OMPRESSION_LOW\x10\x01\x12\x16\n\x12\x43OMPRESSION_MEDIUM\x10\x02\x12\x14\n\x10\x43OMPRESSION_HIGH\x10\x03"\x8e\x06\n\tQueryInfo\x12\r\n\x05query\x18\x01 \x01(\t\x12\x10\n\x08query_id\x18\x02 \x01(\t\x12:\n\x08settings\x18\x03 \x03(\x0b\x32(.clickhouse.grpc.QueryInfo.SettingsEntry\x12\x10\n\x08\x64\x61tabase\x18\x04 \x01(\t\x12\x12\n\ninput_data\x18\x05 \x01(\x0c\x12\x1c\n\x14input_data_delimiter\x18\x06 \x01(\x0c\x12\x15\n\routput_format\x18\x07 \x01(\t\x12\x1b\n\x13send_output_columns\x18\x18 \x01(\x08\x12\x37\n\x0f\x65xternal_tables\x18\x08 \x03(\x0b\x32\x1e.clickhouse.grpc.ExternalTable\x12\x11\n\tuser_name\x18\t \x01(\t\x12\x10\n\x08password\x18\n \x01(\t\x12\r\n\x05quota\x18\x0b \x01(\t\x12\x12\n\nsession_id\x18\x0c \x01(\t\x12\x15\n\rsession_check\x18\r \x01(\x08\x12\x17\n\x0fsession_timeout\x18\x0e \x01(\r\x12\x0e\n\x06\x63\x61ncel\x18\x0f \x01(\x08\x12\x17\n\x0fnext_query_info\x18\x10 \x01(\x08\x12\x1e\n\x16input_compression_type\x18\x14 \x01(\t\x12\x1f\n\x17output_compression_type\x18\x15 \x01(\t\x12 \n\x18output_compression_level\x18\x13 \x01(\x05\x12"\n\x1atransport_compression_type\x18\x16 \x01(\t\x12#\n\x1btransport_compression_level\x18\x17 \x01(\x05\x12R\n\x1bobsolete_result_compression\x18\x11 \x01(\x0b\x32-.clickhouse.grpc.ObsoleteTransportCompression\x12!\n\x19obsolete_compression_type\x18\x12 \x01(\t\x1a/\n\rSettingsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01"\xa1\x01\n\x08LogEntry\x12\x0c\n\x04time\x18\x01 \x01(\r\x12\x19\n\x11time_microseconds\x18\x02 \x01(\r\x12\x11\n\tthread_id\x18\x03 \x01(\x04\x12\x10\n\x08query_id\x18\x04 \x01(\t\x12)\n\x05level\x18\x05 \x01(\x0e\x32\x1a.clickhouse.grpc.LogsLevel\x12\x0e\n\x06source\x18\x06 \x01(\t\x12\x0c\n\x04text\x18\x07 \x01(\t"z\n\x08Progress\x12\x11\n\tread_rows\x18\x01 \x01(\x04\x12\x12\n\nread_bytes\x18\x02 \x01(\x04\x12\x1a\n\x12total_rows_to_read\x18\x03 \x01(\x04\x12\x14\n\x0cwritten_rows\x18\x04 \x01(\x04\x12\x15\n\rwritten_bytes\x18\x05 \x01(\x04"p\n\x05Stats\x12\x0c\n\x04rows\x18\x01 \x01(\x04\x12\x0e\n\x06\x62locks\x18\x02 \x01(\x04\x12\x17\n\x0f\x61llocated_bytes\x18\x03 \x01(\x04\x12\x15\n\rapplied_limit\x18\x04 \x01(\x08\x12\x19\n\x11rows_before_limit\x18\x05 \x01(\x04"R\n\tException\x12\x0c\n\x04\x63ode\x18\x01 \x01(\x05\x12\x0c\n\x04name\x18\x02 \x01(\t\x12\x14\n\x0c\x64isplay_text\x18\x03 \x01(\t\x12\x13\n\x0bstack_trace\x18\x04 \x01(\t"\xeb\x02\n\x06Result\x12\x10\n\x08query_id\x18\t \x01(\t\x12\x11\n\ttime_zone\x18\n \x01(\t\x12\x15\n\routput_format\x18\x0b \x01(\t\x12\x34\n\x0eoutput_columns\x18\x0c \x03(\x0b\x32\x1c.clickhouse.grpc.NameAndType\x12\x0e\n\x06output\x18\x01 \x01(\x0c\x12\x0e\n\x06totals\x18\x02 \x01(\x0c\x12\x10\n\x08\x65xtremes\x18\x03 \x01(\x0c\x12\'\n\x04logs\x18\x04 \x03(\x0b\x32\x19.clickhouse.grpc.LogEntry\x12+\n\x08progress\x18\x05 \x01(\x0b\x32\x19.clickhouse.grpc.Progress\x12%\n\x05stats\x18\x06 \x01(\x0b\x32\x16.clickhouse.grpc.Stats\x12-\n\texception\x18\x07 \x01(\x0b\x32\x1a.clickhouse.grpc.Exception\x12\x11\n\tcancelled\x18\x08 \x01(\x08*\x9d\x01\n\tLogsLevel\x12\x0c\n\x08LOG_NONE\x10\x00\x12\r\n\tLOG_FATAL\x10\x01\x12\x10\n\x0cLOG_CRITICAL\x10\x02\x12\r\n\tLOG_ERROR\x10\x03\x12\x0f\n\x0bLOG_WARNING\x10\x04\x12\x0e\n\nLOG_NOTICE\x10\x05\x12\x13\n\x0fLOG_INFORMATION\x10\x06\x12\r\n\tLOG_DEBUG\x10\x07\x12\r\n\tLOG_TRACE\x10\x08\x32\xdb\x02\n\nClickHouse\x12\x45\n\x0c\x45xecuteQuery\x12\x1a.clickhouse.grpc.QueryInfo\x1a\x17.clickhouse.grpc.Result"\x00\x12V\n\x1b\x45xecuteQueryWithStreamInput\x12\x1a.clickhouse.grpc.QueryInfo\x1a\x17.clickhouse.grpc.Result"\x00(\x01\x12W\n\x1c\x45xecuteQueryWithStreamOutput\x12\x1a.clickhouse.grpc.QueryInfo\x1a\x17.clickhouse.grpc.Result"\x00\x30\x01\x12U\n\x18\x45xecuteQueryWithStreamIO\x12\x1a.clickhouse.grpc.QueryInfo\x1a\x17.clickhouse.grpc.Result"\x00(\x01\x30\x01\x62\x06proto3' +) - -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x15\x63lickhouse_grpc.proto\x12\x0f\x63lickhouse.grpc\")\n\x0bNameAndType\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x0c\n\x04type\x18\x02 \x01(\t\"\xf5\x01\n\rExternalTable\x12\x0c\n\x04name\x18\x01 \x01(\t\x12-\n\x07\x63olumns\x18\x02 \x03(\x0b\x32\x1c.clickhouse.grpc.NameAndType\x12\x0c\n\x04\x64\x61ta\x18\x03 \x01(\x0c\x12\x0e\n\x06\x66ormat\x18\x04 \x01(\t\x12\x18\n\x10\x63ompression_type\x18\x06 \x01(\t\x12>\n\x08settings\x18\x05 \x03(\x0b\x32,.clickhouse.grpc.ExternalTable.SettingsEntry\x1a/\n\rSettingsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"\x85\x03\n\x1cObsoleteTransportCompression\x12U\n\talgorithm\x18\x01 \x01(\x0e\x32\x42.clickhouse.grpc.ObsoleteTransportCompression.CompressionAlgorithm\x12M\n\x05level\x18\x02 \x01(\x0e\x32>.clickhouse.grpc.ObsoleteTransportCompression.CompressionLevel\"R\n\x14\x43ompressionAlgorithm\x12\x12\n\x0eNO_COMPRESSION\x10\x00\x12\x0b\n\x07\x44\x45\x46LATE\x10\x01\x12\x08\n\x04GZIP\x10\x02\x12\x0f\n\x0bSTREAM_GZIP\x10\x03\"k\n\x10\x43ompressionLevel\x12\x14\n\x10\x43OMPRESSION_NONE\x10\x00\x12\x13\n\x0f\x43OMPRESSION_LOW\x10\x01\x12\x16\n\x12\x43OMPRESSION_MEDIUM\x10\x02\x12\x14\n\x10\x43OMPRESSION_HIGH\x10\x03\"\x8e\x06\n\tQueryInfo\x12\r\n\x05query\x18\x01 \x01(\t\x12\x10\n\x08query_id\x18\x02 \x01(\t\x12:\n\x08settings\x18\x03 \x03(\x0b\x32(.clickhouse.grpc.QueryInfo.SettingsEntry\x12\x10\n\x08\x64\x61tabase\x18\x04 \x01(\t\x12\x12\n\ninput_data\x18\x05 \x01(\x0c\x12\x1c\n\x14input_data_delimiter\x18\x06 \x01(\x0c\x12\x15\n\routput_format\x18\x07 \x01(\t\x12\x1b\n\x13send_output_columns\x18\x18 \x01(\x08\x12\x37\n\x0f\x65xternal_tables\x18\x08 \x03(\x0b\x32\x1e.clickhouse.grpc.ExternalTable\x12\x11\n\tuser_name\x18\t \x01(\t\x12\x10\n\x08password\x18\n \x01(\t\x12\r\n\x05quota\x18\x0b \x01(\t\x12\x12\n\nsession_id\x18\x0c \x01(\t\x12\x15\n\rsession_check\x18\r \x01(\x08\x12\x17\n\x0fsession_timeout\x18\x0e \x01(\r\x12\x0e\n\x06\x63\x61ncel\x18\x0f \x01(\x08\x12\x17\n\x0fnext_query_info\x18\x10 \x01(\x08\x12\x1e\n\x16input_compression_type\x18\x14 \x01(\t\x12\x1f\n\x17output_compression_type\x18\x15 \x01(\t\x12 \n\x18output_compression_level\x18\x13 \x01(\x05\x12\"\n\x1atransport_compression_type\x18\x16 \x01(\t\x12#\n\x1btransport_compression_level\x18\x17 \x01(\x05\x12R\n\x1bobsolete_result_compression\x18\x11 \x01(\x0b\x32-.clickhouse.grpc.ObsoleteTransportCompression\x12!\n\x19obsolete_compression_type\x18\x12 \x01(\t\x1a/\n\rSettingsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"\xa1\x01\n\x08LogEntry\x12\x0c\n\x04time\x18\x01 \x01(\r\x12\x19\n\x11time_microseconds\x18\x02 \x01(\r\x12\x11\n\tthread_id\x18\x03 \x01(\x04\x12\x10\n\x08query_id\x18\x04 \x01(\t\x12)\n\x05level\x18\x05 \x01(\x0e\x32\x1a.clickhouse.grpc.LogsLevel\x12\x0e\n\x06source\x18\x06 \x01(\t\x12\x0c\n\x04text\x18\x07 \x01(\t\"z\n\x08Progress\x12\x11\n\tread_rows\x18\x01 \x01(\x04\x12\x12\n\nread_bytes\x18\x02 \x01(\x04\x12\x1a\n\x12total_rows_to_read\x18\x03 \x01(\x04\x12\x14\n\x0cwritten_rows\x18\x04 \x01(\x04\x12\x15\n\rwritten_bytes\x18\x05 \x01(\x04\"p\n\x05Stats\x12\x0c\n\x04rows\x18\x01 \x01(\x04\x12\x0e\n\x06\x62locks\x18\x02 \x01(\x04\x12\x17\n\x0f\x61llocated_bytes\x18\x03 \x01(\x04\x12\x15\n\rapplied_limit\x18\x04 \x01(\x08\x12\x19\n\x11rows_before_limit\x18\x05 \x01(\x04\"R\n\tException\x12\x0c\n\x04\x63ode\x18\x01 \x01(\x05\x12\x0c\n\x04name\x18\x02 \x01(\t\x12\x14\n\x0c\x64isplay_text\x18\x03 \x01(\t\x12\x13\n\x0bstack_trace\x18\x04 \x01(\t\"\xeb\x02\n\x06Result\x12\x10\n\x08query_id\x18\t \x01(\t\x12\x11\n\ttime_zone\x18\n \x01(\t\x12\x15\n\routput_format\x18\x0b \x01(\t\x12\x34\n\x0eoutput_columns\x18\x0c \x03(\x0b\x32\x1c.clickhouse.grpc.NameAndType\x12\x0e\n\x06output\x18\x01 \x01(\x0c\x12\x0e\n\x06totals\x18\x02 \x01(\x0c\x12\x10\n\x08\x65xtremes\x18\x03 \x01(\x0c\x12\'\n\x04logs\x18\x04 \x03(\x0b\x32\x19.clickhouse.grpc.LogEntry\x12+\n\x08progress\x18\x05 \x01(\x0b\x32\x19.clickhouse.grpc.Progress\x12%\n\x05stats\x18\x06 \x01(\x0b\x32\x16.clickhouse.grpc.Stats\x12-\n\texception\x18\x07 \x01(\x0b\x32\x1a.clickhouse.grpc.Exception\x12\x11\n\tcancelled\x18\x08 \x01(\x08*\x9d\x01\n\tLogsLevel\x12\x0c\n\x08LOG_NONE\x10\x00\x12\r\n\tLOG_FATAL\x10\x01\x12\x10\n\x0cLOG_CRITICAL\x10\x02\x12\r\n\tLOG_ERROR\x10\x03\x12\x0f\n\x0bLOG_WARNING\x10\x04\x12\x0e\n\nLOG_NOTICE\x10\x05\x12\x13\n\x0fLOG_INFORMATION\x10\x06\x12\r\n\tLOG_DEBUG\x10\x07\x12\r\n\tLOG_TRACE\x10\x08\x32\xdb\x02\n\nClickHouse\x12\x45\n\x0c\x45xecuteQuery\x12\x1a.clickhouse.grpc.QueryInfo\x1a\x17.clickhouse.grpc.Result\"\x00\x12V\n\x1b\x45xecuteQueryWithStreamInput\x12\x1a.clickhouse.grpc.QueryInfo\x1a\x17.clickhouse.grpc.Result\"\x00(\x01\x12W\n\x1c\x45xecuteQueryWithStreamOutput\x12\x1a.clickhouse.grpc.QueryInfo\x1a\x17.clickhouse.grpc.Result\"\x00\x30\x01\x12U\n\x18\x45xecuteQueryWithStreamIO\x12\x1a.clickhouse.grpc.QueryInfo\x1a\x17.clickhouse.grpc.Result\"\x00(\x01\x30\x01\x62\x06proto3') - -_LOGSLEVEL = DESCRIPTOR.enum_types_by_name['LogsLevel'] +_LOGSLEVEL = DESCRIPTOR.enum_types_by_name["LogsLevel"] LogsLevel = enum_type_wrapper.EnumTypeWrapper(_LOGSLEVEL) LOG_NONE = 0 LOG_FATAL = 1 @@ -30,134 +31,180 @@ LOG_DEBUG = 7 LOG_TRACE = 8 -_NAMEANDTYPE = DESCRIPTOR.message_types_by_name['NameAndType'] -_EXTERNALTABLE = DESCRIPTOR.message_types_by_name['ExternalTable'] -_EXTERNALTABLE_SETTINGSENTRY = _EXTERNALTABLE.nested_types_by_name['SettingsEntry'] -_OBSOLETETRANSPORTCOMPRESSION = DESCRIPTOR.message_types_by_name['ObsoleteTransportCompression'] -_QUERYINFO = DESCRIPTOR.message_types_by_name['QueryInfo'] -_QUERYINFO_SETTINGSENTRY = _QUERYINFO.nested_types_by_name['SettingsEntry'] -_LOGENTRY = DESCRIPTOR.message_types_by_name['LogEntry'] -_PROGRESS = DESCRIPTOR.message_types_by_name['Progress'] -_STATS = DESCRIPTOR.message_types_by_name['Stats'] -_EXCEPTION = DESCRIPTOR.message_types_by_name['Exception'] -_RESULT = DESCRIPTOR.message_types_by_name['Result'] -_OBSOLETETRANSPORTCOMPRESSION_COMPRESSIONALGORITHM = _OBSOLETETRANSPORTCOMPRESSION.enum_types_by_name['CompressionAlgorithm'] -_OBSOLETETRANSPORTCOMPRESSION_COMPRESSIONLEVEL = _OBSOLETETRANSPORTCOMPRESSION.enum_types_by_name['CompressionLevel'] -NameAndType = _reflection.GeneratedProtocolMessageType('NameAndType', (_message.Message,), { - 'DESCRIPTOR' : _NAMEANDTYPE, - '__module__' : 'clickhouse_grpc_pb2' - # @@protoc_insertion_point(class_scope:clickhouse.grpc.NameAndType) - }) +_NAMEANDTYPE = DESCRIPTOR.message_types_by_name["NameAndType"] +_EXTERNALTABLE = DESCRIPTOR.message_types_by_name["ExternalTable"] +_EXTERNALTABLE_SETTINGSENTRY = _EXTERNALTABLE.nested_types_by_name["SettingsEntry"] +_OBSOLETETRANSPORTCOMPRESSION = DESCRIPTOR.message_types_by_name[ + "ObsoleteTransportCompression" +] +_QUERYINFO = DESCRIPTOR.message_types_by_name["QueryInfo"] +_QUERYINFO_SETTINGSENTRY = _QUERYINFO.nested_types_by_name["SettingsEntry"] +_LOGENTRY = DESCRIPTOR.message_types_by_name["LogEntry"] +_PROGRESS = DESCRIPTOR.message_types_by_name["Progress"] +_STATS = DESCRIPTOR.message_types_by_name["Stats"] +_EXCEPTION = DESCRIPTOR.message_types_by_name["Exception"] +_RESULT = DESCRIPTOR.message_types_by_name["Result"] +_OBSOLETETRANSPORTCOMPRESSION_COMPRESSIONALGORITHM = ( + _OBSOLETETRANSPORTCOMPRESSION.enum_types_by_name["CompressionAlgorithm"] +) +_OBSOLETETRANSPORTCOMPRESSION_COMPRESSIONLEVEL = ( + _OBSOLETETRANSPORTCOMPRESSION.enum_types_by_name["CompressionLevel"] +) +NameAndType = _reflection.GeneratedProtocolMessageType( + "NameAndType", + (_message.Message,), + { + "DESCRIPTOR": _NAMEANDTYPE, + "__module__": "clickhouse_grpc_pb2" + # @@protoc_insertion_point(class_scope:clickhouse.grpc.NameAndType) + }, +) _sym_db.RegisterMessage(NameAndType) -ExternalTable = _reflection.GeneratedProtocolMessageType('ExternalTable', (_message.Message,), { - - 'SettingsEntry' : _reflection.GeneratedProtocolMessageType('SettingsEntry', (_message.Message,), { - 'DESCRIPTOR' : _EXTERNALTABLE_SETTINGSENTRY, - '__module__' : 'clickhouse_grpc_pb2' - # @@protoc_insertion_point(class_scope:clickhouse.grpc.ExternalTable.SettingsEntry) - }) - , - 'DESCRIPTOR' : _EXTERNALTABLE, - '__module__' : 'clickhouse_grpc_pb2' - # @@protoc_insertion_point(class_scope:clickhouse.grpc.ExternalTable) - }) +ExternalTable = _reflection.GeneratedProtocolMessageType( + "ExternalTable", + (_message.Message,), + { + "SettingsEntry": _reflection.GeneratedProtocolMessageType( + "SettingsEntry", + (_message.Message,), + { + "DESCRIPTOR": _EXTERNALTABLE_SETTINGSENTRY, + "__module__": "clickhouse_grpc_pb2" + # @@protoc_insertion_point(class_scope:clickhouse.grpc.ExternalTable.SettingsEntry) + }, + ), + "DESCRIPTOR": _EXTERNALTABLE, + "__module__": "clickhouse_grpc_pb2" + # @@protoc_insertion_point(class_scope:clickhouse.grpc.ExternalTable) + }, +) _sym_db.RegisterMessage(ExternalTable) _sym_db.RegisterMessage(ExternalTable.SettingsEntry) -ObsoleteTransportCompression = _reflection.GeneratedProtocolMessageType('ObsoleteTransportCompression', (_message.Message,), { - 'DESCRIPTOR' : _OBSOLETETRANSPORTCOMPRESSION, - '__module__' : 'clickhouse_grpc_pb2' - # @@protoc_insertion_point(class_scope:clickhouse.grpc.ObsoleteTransportCompression) - }) +ObsoleteTransportCompression = _reflection.GeneratedProtocolMessageType( + "ObsoleteTransportCompression", + (_message.Message,), + { + "DESCRIPTOR": _OBSOLETETRANSPORTCOMPRESSION, + "__module__": "clickhouse_grpc_pb2" + # @@protoc_insertion_point(class_scope:clickhouse.grpc.ObsoleteTransportCompression) + }, +) _sym_db.RegisterMessage(ObsoleteTransportCompression) -QueryInfo = _reflection.GeneratedProtocolMessageType('QueryInfo', (_message.Message,), { - - 'SettingsEntry' : _reflection.GeneratedProtocolMessageType('SettingsEntry', (_message.Message,), { - 'DESCRIPTOR' : _QUERYINFO_SETTINGSENTRY, - '__module__' : 'clickhouse_grpc_pb2' - # @@protoc_insertion_point(class_scope:clickhouse.grpc.QueryInfo.SettingsEntry) - }) - , - 'DESCRIPTOR' : _QUERYINFO, - '__module__' : 'clickhouse_grpc_pb2' - # @@protoc_insertion_point(class_scope:clickhouse.grpc.QueryInfo) - }) +QueryInfo = _reflection.GeneratedProtocolMessageType( + "QueryInfo", + (_message.Message,), + { + "SettingsEntry": _reflection.GeneratedProtocolMessageType( + "SettingsEntry", + (_message.Message,), + { + "DESCRIPTOR": _QUERYINFO_SETTINGSENTRY, + "__module__": "clickhouse_grpc_pb2" + # @@protoc_insertion_point(class_scope:clickhouse.grpc.QueryInfo.SettingsEntry) + }, + ), + "DESCRIPTOR": _QUERYINFO, + "__module__": "clickhouse_grpc_pb2" + # @@protoc_insertion_point(class_scope:clickhouse.grpc.QueryInfo) + }, +) _sym_db.RegisterMessage(QueryInfo) _sym_db.RegisterMessage(QueryInfo.SettingsEntry) -LogEntry = _reflection.GeneratedProtocolMessageType('LogEntry', (_message.Message,), { - 'DESCRIPTOR' : _LOGENTRY, - '__module__' : 'clickhouse_grpc_pb2' - # @@protoc_insertion_point(class_scope:clickhouse.grpc.LogEntry) - }) +LogEntry = _reflection.GeneratedProtocolMessageType( + "LogEntry", + (_message.Message,), + { + "DESCRIPTOR": _LOGENTRY, + "__module__": "clickhouse_grpc_pb2" + # @@protoc_insertion_point(class_scope:clickhouse.grpc.LogEntry) + }, +) _sym_db.RegisterMessage(LogEntry) -Progress = _reflection.GeneratedProtocolMessageType('Progress', (_message.Message,), { - 'DESCRIPTOR' : _PROGRESS, - '__module__' : 'clickhouse_grpc_pb2' - # @@protoc_insertion_point(class_scope:clickhouse.grpc.Progress) - }) +Progress = _reflection.GeneratedProtocolMessageType( + "Progress", + (_message.Message,), + { + "DESCRIPTOR": _PROGRESS, + "__module__": "clickhouse_grpc_pb2" + # @@protoc_insertion_point(class_scope:clickhouse.grpc.Progress) + }, +) _sym_db.RegisterMessage(Progress) -Stats = _reflection.GeneratedProtocolMessageType('Stats', (_message.Message,), { - 'DESCRIPTOR' : _STATS, - '__module__' : 'clickhouse_grpc_pb2' - # @@protoc_insertion_point(class_scope:clickhouse.grpc.Stats) - }) +Stats = _reflection.GeneratedProtocolMessageType( + "Stats", + (_message.Message,), + { + "DESCRIPTOR": _STATS, + "__module__": "clickhouse_grpc_pb2" + # @@protoc_insertion_point(class_scope:clickhouse.grpc.Stats) + }, +) _sym_db.RegisterMessage(Stats) -Exception = _reflection.GeneratedProtocolMessageType('Exception', (_message.Message,), { - 'DESCRIPTOR' : _EXCEPTION, - '__module__' : 'clickhouse_grpc_pb2' - # @@protoc_insertion_point(class_scope:clickhouse.grpc.Exception) - }) +Exception = _reflection.GeneratedProtocolMessageType( + "Exception", + (_message.Message,), + { + "DESCRIPTOR": _EXCEPTION, + "__module__": "clickhouse_grpc_pb2" + # @@protoc_insertion_point(class_scope:clickhouse.grpc.Exception) + }, +) _sym_db.RegisterMessage(Exception) -Result = _reflection.GeneratedProtocolMessageType('Result', (_message.Message,), { - 'DESCRIPTOR' : _RESULT, - '__module__' : 'clickhouse_grpc_pb2' - # @@protoc_insertion_point(class_scope:clickhouse.grpc.Result) - }) +Result = _reflection.GeneratedProtocolMessageType( + "Result", + (_message.Message,), + { + "DESCRIPTOR": _RESULT, + "__module__": "clickhouse_grpc_pb2" + # @@protoc_insertion_point(class_scope:clickhouse.grpc.Result) + }, +) _sym_db.RegisterMessage(Result) -_CLICKHOUSE = DESCRIPTOR.services_by_name['ClickHouse'] +_CLICKHOUSE = DESCRIPTOR.services_by_name["ClickHouse"] if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None - _EXTERNALTABLE_SETTINGSENTRY._options = None - _EXTERNALTABLE_SETTINGSENTRY._serialized_options = b'8\001' - _QUERYINFO_SETTINGSENTRY._options = None - _QUERYINFO_SETTINGSENTRY._serialized_options = b'8\001' - _LOGSLEVEL._serialized_start=2363 - _LOGSLEVEL._serialized_end=2520 - _NAMEANDTYPE._serialized_start=42 - _NAMEANDTYPE._serialized_end=83 - _EXTERNALTABLE._serialized_start=86 - _EXTERNALTABLE._serialized_end=331 - _EXTERNALTABLE_SETTINGSENTRY._serialized_start=284 - _EXTERNALTABLE_SETTINGSENTRY._serialized_end=331 - _OBSOLETETRANSPORTCOMPRESSION._serialized_start=334 - _OBSOLETETRANSPORTCOMPRESSION._serialized_end=723 - _OBSOLETETRANSPORTCOMPRESSION_COMPRESSIONALGORITHM._serialized_start=532 - _OBSOLETETRANSPORTCOMPRESSION_COMPRESSIONALGORITHM._serialized_end=614 - _OBSOLETETRANSPORTCOMPRESSION_COMPRESSIONLEVEL._serialized_start=616 - _OBSOLETETRANSPORTCOMPRESSION_COMPRESSIONLEVEL._serialized_end=723 - _QUERYINFO._serialized_start=726 - _QUERYINFO._serialized_end=1508 - _QUERYINFO_SETTINGSENTRY._serialized_start=284 - _QUERYINFO_SETTINGSENTRY._serialized_end=331 - _LOGENTRY._serialized_start=1511 - _LOGENTRY._serialized_end=1672 - _PROGRESS._serialized_start=1674 - _PROGRESS._serialized_end=1796 - _STATS._serialized_start=1798 - _STATS._serialized_end=1910 - _EXCEPTION._serialized_start=1912 - _EXCEPTION._serialized_end=1994 - _RESULT._serialized_start=1997 - _RESULT._serialized_end=2360 - _CLICKHOUSE._serialized_start=2523 - _CLICKHOUSE._serialized_end=2870 + DESCRIPTOR._options = None + _EXTERNALTABLE_SETTINGSENTRY._options = None + _EXTERNALTABLE_SETTINGSENTRY._serialized_options = b"8\001" + _QUERYINFO_SETTINGSENTRY._options = None + _QUERYINFO_SETTINGSENTRY._serialized_options = b"8\001" + _LOGSLEVEL._serialized_start = 2363 + _LOGSLEVEL._serialized_end = 2520 + _NAMEANDTYPE._serialized_start = 42 + _NAMEANDTYPE._serialized_end = 83 + _EXTERNALTABLE._serialized_start = 86 + _EXTERNALTABLE._serialized_end = 331 + _EXTERNALTABLE_SETTINGSENTRY._serialized_start = 284 + _EXTERNALTABLE_SETTINGSENTRY._serialized_end = 331 + _OBSOLETETRANSPORTCOMPRESSION._serialized_start = 334 + _OBSOLETETRANSPORTCOMPRESSION._serialized_end = 723 + _OBSOLETETRANSPORTCOMPRESSION_COMPRESSIONALGORITHM._serialized_start = 532 + _OBSOLETETRANSPORTCOMPRESSION_COMPRESSIONALGORITHM._serialized_end = 614 + _OBSOLETETRANSPORTCOMPRESSION_COMPRESSIONLEVEL._serialized_start = 616 + _OBSOLETETRANSPORTCOMPRESSION_COMPRESSIONLEVEL._serialized_end = 723 + _QUERYINFO._serialized_start = 726 + _QUERYINFO._serialized_end = 1508 + _QUERYINFO_SETTINGSENTRY._serialized_start = 284 + _QUERYINFO_SETTINGSENTRY._serialized_end = 331 + _LOGENTRY._serialized_start = 1511 + _LOGENTRY._serialized_end = 1672 + _PROGRESS._serialized_start = 1674 + _PROGRESS._serialized_end = 1796 + _STATS._serialized_start = 1798 + _STATS._serialized_end = 1910 + _EXCEPTION._serialized_start = 1912 + _EXCEPTION._serialized_end = 1994 + _RESULT._serialized_start = 1997 + _RESULT._serialized_end = 2360 + _CLICKHOUSE._serialized_start = 2523 + _CLICKHOUSE._serialized_end = 2870 # @@protoc_insertion_point(module_scope) diff --git a/utils/grpc-client/pb2/clickhouse_grpc_pb2_grpc.py b/utils/grpc-client/pb2/clickhouse_grpc_pb2_grpc.py index 1c71218bbe5..25643a243b3 100644 --- a/utils/grpc-client/pb2/clickhouse_grpc_pb2_grpc.py +++ b/utils/grpc-client/pb2/clickhouse_grpc_pb2_grpc.py @@ -15,25 +15,25 @@ class ClickHouseStub(object): channel: A grpc.Channel. """ self.ExecuteQuery = channel.unary_unary( - '/clickhouse.grpc.ClickHouse/ExecuteQuery', - request_serializer=clickhouse__grpc__pb2.QueryInfo.SerializeToString, - response_deserializer=clickhouse__grpc__pb2.Result.FromString, - ) + "/clickhouse.grpc.ClickHouse/ExecuteQuery", + request_serializer=clickhouse__grpc__pb2.QueryInfo.SerializeToString, + response_deserializer=clickhouse__grpc__pb2.Result.FromString, + ) self.ExecuteQueryWithStreamInput = channel.stream_unary( - '/clickhouse.grpc.ClickHouse/ExecuteQueryWithStreamInput', - request_serializer=clickhouse__grpc__pb2.QueryInfo.SerializeToString, - response_deserializer=clickhouse__grpc__pb2.Result.FromString, - ) + "/clickhouse.grpc.ClickHouse/ExecuteQueryWithStreamInput", + request_serializer=clickhouse__grpc__pb2.QueryInfo.SerializeToString, + response_deserializer=clickhouse__grpc__pb2.Result.FromString, + ) self.ExecuteQueryWithStreamOutput = channel.unary_stream( - '/clickhouse.grpc.ClickHouse/ExecuteQueryWithStreamOutput', - request_serializer=clickhouse__grpc__pb2.QueryInfo.SerializeToString, - response_deserializer=clickhouse__grpc__pb2.Result.FromString, - ) + "/clickhouse.grpc.ClickHouse/ExecuteQueryWithStreamOutput", + request_serializer=clickhouse__grpc__pb2.QueryInfo.SerializeToString, + response_deserializer=clickhouse__grpc__pb2.Result.FromString, + ) self.ExecuteQueryWithStreamIO = channel.stream_stream( - '/clickhouse.grpc.ClickHouse/ExecuteQueryWithStreamIO', - request_serializer=clickhouse__grpc__pb2.QueryInfo.SerializeToString, - response_deserializer=clickhouse__grpc__pb2.Result.FromString, - ) + "/clickhouse.grpc.ClickHouse/ExecuteQueryWithStreamIO", + request_serializer=clickhouse__grpc__pb2.QueryInfo.SerializeToString, + response_deserializer=clickhouse__grpc__pb2.Result.FromString, + ) class ClickHouseServicer(object): @@ -42,124 +42,173 @@ class ClickHouseServicer(object): def ExecuteQuery(self, request, context): """Missing associated documentation comment in .proto file.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def ExecuteQueryWithStreamInput(self, request_iterator, context): """Missing associated documentation comment in .proto file.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def ExecuteQueryWithStreamOutput(self, request, context): """Missing associated documentation comment in .proto file.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def ExecuteQueryWithStreamIO(self, request_iterator, context): """Missing associated documentation comment in .proto file.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details('Method not implemented!') - raise NotImplementedError('Method not implemented!') + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") def add_ClickHouseServicer_to_server(servicer, server): rpc_method_handlers = { - 'ExecuteQuery': grpc.unary_unary_rpc_method_handler( - servicer.ExecuteQuery, - request_deserializer=clickhouse__grpc__pb2.QueryInfo.FromString, - response_serializer=clickhouse__grpc__pb2.Result.SerializeToString, - ), - 'ExecuteQueryWithStreamInput': grpc.stream_unary_rpc_method_handler( - servicer.ExecuteQueryWithStreamInput, - request_deserializer=clickhouse__grpc__pb2.QueryInfo.FromString, - response_serializer=clickhouse__grpc__pb2.Result.SerializeToString, - ), - 'ExecuteQueryWithStreamOutput': grpc.unary_stream_rpc_method_handler( - servicer.ExecuteQueryWithStreamOutput, - request_deserializer=clickhouse__grpc__pb2.QueryInfo.FromString, - response_serializer=clickhouse__grpc__pb2.Result.SerializeToString, - ), - 'ExecuteQueryWithStreamIO': grpc.stream_stream_rpc_method_handler( - servicer.ExecuteQueryWithStreamIO, - request_deserializer=clickhouse__grpc__pb2.QueryInfo.FromString, - response_serializer=clickhouse__grpc__pb2.Result.SerializeToString, - ), + "ExecuteQuery": grpc.unary_unary_rpc_method_handler( + servicer.ExecuteQuery, + request_deserializer=clickhouse__grpc__pb2.QueryInfo.FromString, + response_serializer=clickhouse__grpc__pb2.Result.SerializeToString, + ), + "ExecuteQueryWithStreamInput": grpc.stream_unary_rpc_method_handler( + servicer.ExecuteQueryWithStreamInput, + request_deserializer=clickhouse__grpc__pb2.QueryInfo.FromString, + response_serializer=clickhouse__grpc__pb2.Result.SerializeToString, + ), + "ExecuteQueryWithStreamOutput": grpc.unary_stream_rpc_method_handler( + servicer.ExecuteQueryWithStreamOutput, + request_deserializer=clickhouse__grpc__pb2.QueryInfo.FromString, + response_serializer=clickhouse__grpc__pb2.Result.SerializeToString, + ), + "ExecuteQueryWithStreamIO": grpc.stream_stream_rpc_method_handler( + servicer.ExecuteQueryWithStreamIO, + request_deserializer=clickhouse__grpc__pb2.QueryInfo.FromString, + response_serializer=clickhouse__grpc__pb2.Result.SerializeToString, + ), } generic_handler = grpc.method_handlers_generic_handler( - 'clickhouse.grpc.ClickHouse', rpc_method_handlers) + "clickhouse.grpc.ClickHouse", rpc_method_handlers + ) server.add_generic_rpc_handlers((generic_handler,)) - # This class is part of an EXPERIMENTAL API. +# This class is part of an EXPERIMENTAL API. class ClickHouse(object): """Missing associated documentation comment in .proto file.""" @staticmethod - def ExecuteQuery(request, + def ExecuteQuery( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.unary_unary(request, target, '/clickhouse.grpc.ClickHouse/ExecuteQuery', + "/clickhouse.grpc.ClickHouse/ExecuteQuery", clickhouse__grpc__pb2.QueryInfo.SerializeToString, clickhouse__grpc__pb2.Result.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) @staticmethod - def ExecuteQueryWithStreamInput(request_iterator, + def ExecuteQueryWithStreamInput( + request_iterator, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.stream_unary( + request_iterator, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.stream_unary(request_iterator, target, '/clickhouse.grpc.ClickHouse/ExecuteQueryWithStreamInput', + "/clickhouse.grpc.ClickHouse/ExecuteQueryWithStreamInput", clickhouse__grpc__pb2.QueryInfo.SerializeToString, clickhouse__grpc__pb2.Result.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) @staticmethod - def ExecuteQueryWithStreamOutput(request, + def ExecuteQueryWithStreamOutput( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_stream( + request, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.unary_stream(request, target, '/clickhouse.grpc.ClickHouse/ExecuteQueryWithStreamOutput', + "/clickhouse.grpc.ClickHouse/ExecuteQueryWithStreamOutput", clickhouse__grpc__pb2.QueryInfo.SerializeToString, clickhouse__grpc__pb2.Result.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) @staticmethod - def ExecuteQueryWithStreamIO(request_iterator, + def ExecuteQueryWithStreamIO( + request_iterator, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.stream_stream( + request_iterator, target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None): - return grpc.experimental.stream_stream(request_iterator, target, '/clickhouse.grpc.ClickHouse/ExecuteQueryWithStreamIO', + "/clickhouse.grpc.ClickHouse/ExecuteQueryWithStreamIO", clickhouse__grpc__pb2.QueryInfo.SerializeToString, clickhouse__grpc__pb2.Result.FromString, - options, channel_credentials, - insecure, call_credentials, compression, wait_for_ready, timeout, metadata) + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + ) From e42d10fa9ccf4296732941e9f1b333d692e83384 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jan 2024 06:25:20 +0100 Subject: [PATCH 061/276] Revert "Obey Python's quirky formatter" This reverts commit 678a32cedee768b6c1a6748e96a0d103e853d8bc. --- tests/integration/ci-runner.py | 13 +- .../test_async_insert_memory/test.py | 2 +- tests/integration/test_check_table/test.py | 76 ++--- .../test_cluster_discovery/test.py | 2 +- .../test_ldap_external_user_directory/test.py | 26 +- tests/integration/test_mysql_protocol/test.py | 16 +- tests/integration/test_partition/test.py | 4 +- .../test_replicated_database/test.py | 9 +- .../test.py | 9 +- .../s3_mocks/unstable_server.py | 2 +- tests/integration/test_storage_s3/test.py | 17 +- tests/integration/test_storage_url/test.py | 22 +- tests/integration/test_system_merges/test.py | 45 +-- utils/grpc-client/pb2/clickhouse_grpc_pb2.py | 271 ++++++++---------- .../pb2/clickhouse_grpc_pb2_grpc.py | 237 ++++++--------- 15 files changed, 318 insertions(+), 433 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index d54ed2bb767..7c922e339fe 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -305,11 +305,14 @@ class ClickhouseIntegrationTestsRunner: def _pre_pull_images(self, repo_path): image_cmd = self._get_runner_image_cmd(repo_path) - cmd = "cd {repo_path}/tests/integration && " "timeout --signal=KILL 1h ./runner {runner_opts} {image_cmd} --pre-pull --command '{command}' ".format( - repo_path=repo_path, - runner_opts=self._get_runner_opts(), - image_cmd=image_cmd, - command=r""" echo Pre Pull finished """, + cmd = ( + "cd {repo_path}/tests/integration && " + "timeout --signal=KILL 1h ./runner {runner_opts} {image_cmd} --pre-pull --command '{command}' ".format( + repo_path=repo_path, + runner_opts=self._get_runner_opts(), + image_cmd=image_cmd, + command=r""" echo Pre Pull finished """, + ) ) for i in range(5): diff --git a/tests/integration/test_async_insert_memory/test.py b/tests/integration/test_async_insert_memory/test.py index f897007f7bb..5d2e5503680 100644 --- a/tests/integration/test_async_insert_memory/test.py +++ b/tests/integration/test_async_insert_memory/test.py @@ -43,7 +43,7 @@ def test_memory_usage(): response = node.get_query_request( "SELECT groupArray(number) FROM numbers(1000000) SETTINGS max_memory_usage_for_user={}".format( - 30 * (2 ** 23) + 30 * (2**23) ), user="A", ) diff --git a/tests/integration/test_check_table/test.py b/tests/integration/test_check_table/test.py index ebf404e698b..021977fb6b6 100644 --- a/tests/integration/test_check_table/test.py +++ b/tests/integration/test_check_table/test.py @@ -95,25 +95,15 @@ def test_check_normal_table_corruption(started_cluster, merge_tree_settings): node1, "non_replicated_mt", "201902_1_1_0", database="default" ) - assert ( - node1.query( - "CHECK TABLE non_replicated_mt", - settings={"check_query_single_value_result": 0, "max_threads": 1}, - ) - .strip() - .split("\t")[0:2] - == ["201902_1_1_0", "0"] - ) + assert node1.query( + "CHECK TABLE non_replicated_mt", + settings={"check_query_single_value_result": 0, "max_threads": 1}, + ).strip().split("\t")[0:2] == ["201902_1_1_0", "0"] - assert ( - node1.query( - "CHECK TABLE non_replicated_mt", - settings={"check_query_single_value_result": 0, "max_threads": 1}, - ) - .strip() - .split("\t")[0:2] - == ["201902_1_1_0", "0"] - ) + assert node1.query( + "CHECK TABLE non_replicated_mt", + settings={"check_query_single_value_result": 0, "max_threads": 1}, + ).strip().split("\t")[0:2] == ["201902_1_1_0", "0"] node1.query( "INSERT INTO non_replicated_mt VALUES (toDate('2019-01-01'), 1, 10), (toDate('2019-01-01'), 2, 12)" @@ -133,15 +123,10 @@ def test_check_normal_table_corruption(started_cluster, merge_tree_settings): remove_checksums_on_disk(node1, "default", "non_replicated_mt", "201901_2_2_0") - assert ( - node1.query( - "CHECK TABLE non_replicated_mt PARTITION 201901", - settings={"check_query_single_value_result": 0, "max_threads": 1}, - ) - .strip() - .split("\t")[0:2] - == ["201901_2_2_0", "0"] - ) + assert node1.query( + "CHECK TABLE non_replicated_mt PARTITION 201901", + settings={"check_query_single_value_result": 0, "max_threads": 1}, + ).strip().split("\t")[0:2] == ["201901_2_2_0", "0"] @pytest.mark.parametrize("merge_tree_settings, zk_path_suffix", [("", "_0")]) @@ -209,15 +194,12 @@ def test_check_replicated_table_simple( == "201901_0_0_0\t1\t\n" ) - assert ( - sorted( - node2.query( - "CHECK TABLE replicated_mt", - settings={"check_query_single_value_result": 0}, - ).split("\n") - ) - == ["", "201901_0_0_0\t1\t", "201902_0_0_0\t1\t"] - ) + assert sorted( + node2.query( + "CHECK TABLE replicated_mt", + settings={"check_query_single_value_result": 0}, + ).split("\n") + ) == ["", "201901_0_0_0\t1\t", "201902_0_0_0\t1\t"] with pytest.raises(QueryRuntimeException) as exc: node2.query( @@ -291,13 +273,10 @@ def test_check_replicated_table_corruption( ) node1.query_with_retry("SYSTEM SYNC REPLICA replicated_mt_1") - assert ( - node1.query( - "CHECK TABLE replicated_mt_1 PARTITION 201901", - settings={"check_query_single_value_result": 0, "max_threads": 1}, - ) - == "{}\t1\t\n".format(part_name) - ) + assert node1.query( + "CHECK TABLE replicated_mt_1 PARTITION 201901", + settings={"check_query_single_value_result": 0, "max_threads": 1}, + ) == "{}\t1\t\n".format(part_name) assert node1.query("SELECT count() from replicated_mt_1") == "4\n" remove_part_from_disk(node2, "replicated_mt_1", part_name) @@ -309,13 +288,10 @@ def test_check_replicated_table_corruption( ) node1.query("SYSTEM SYNC REPLICA replicated_mt_1") - assert ( - node1.query( - "CHECK TABLE replicated_mt_1 PARTITION 201901", - settings={"check_query_single_value_result": 0, "max_threads": 1}, - ) - == "{}\t1\t\n".format(part_name) - ) + assert node1.query( + "CHECK TABLE replicated_mt_1 PARTITION 201901", + settings={"check_query_single_value_result": 0, "max_threads": 1}, + ) == "{}\t1\t\n".format(part_name) assert node1.query("SELECT count() from replicated_mt_1") == "4\n" diff --git a/tests/integration/test_cluster_discovery/test.py b/tests/integration/test_cluster_discovery/test.py index a2e7e15b956..ad3deb5b142 100644 --- a/tests/integration/test_cluster_discovery/test.py +++ b/tests/integration/test_cluster_discovery/test.py @@ -61,7 +61,7 @@ def check_on_cluster( print(f"Retry {retry}/{retries} unsuccessful, result: {node_results}") if retry != retries: - time.sleep(2 ** retry) + time.sleep(2**retry) else: msg = msg or f"Wrong '{what}' result" raise Exception( diff --git a/tests/integration/test_ldap_external_user_directory/test.py b/tests/integration/test_ldap_external_user_directory/test.py index c9642c293ee..39753794d63 100644 --- a/tests/integration/test_ldap_external_user_directory/test.py +++ b/tests/integration/test_ldap_external_user_directory/test.py @@ -76,14 +76,11 @@ def test_role_mapping(ldap_cluster): "select currentUser()", user="johndoe", password="qwertz" ) == TSV([["johndoe"]]) - assert ( - instance.query( - "select role_name from system.current_roles ORDER BY role_name", - user="johndoe", - password="qwertz", - ) - == TSV([["role_1"], ["role_2"]]) - ) + assert instance.query( + "select role_name from system.current_roles ORDER BY role_name", + user="johndoe", + password="qwertz", + ) == TSV([["role_1"], ["role_2"]]) instance.query("CREATE ROLE role_3") add_ldap_group(ldap_cluster, group_cn="clickhouse-role_3", member_cn="johndoe") @@ -91,11 +88,8 @@ def test_role_mapping(ldap_cluster): # See https://github.com/ClickHouse/ClickHouse/issues/54318 add_ldap_group(ldap_cluster, group_cn="clickhouse-role_4", member_cn="johndoe") - assert ( - instance.query( - "select role_name from system.current_roles ORDER BY role_name", - user="johndoe", - password="qwertz", - ) - == TSV([["role_1"], ["role_2"], ["role_3"]]) - ) + assert instance.query( + "select role_name from system.current_roles ORDER BY role_name", + user="johndoe", + password="qwertz", + ) == TSV([["role_1"], ["role_2"], ["role_3"]]) diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 61e76c0dc97..7a69d07633c 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -854,14 +854,14 @@ def test_types(started_cluster): result = cursor.fetchall()[0] expected = [ - ("Int8_column", -(2 ** 7)), - ("UInt8_column", 2 ** 8 - 1), - ("Int16_column", -(2 ** 15)), - ("UInt16_column", 2 ** 16 - 1), - ("Int32_column", -(2 ** 31)), - ("UInt32_column", 2 ** 32 - 1), - ("Int64_column", -(2 ** 63)), - ("UInt64_column", 2 ** 64 - 1), + ("Int8_column", -(2**7)), + ("UInt8_column", 2**8 - 1), + ("Int16_column", -(2**15)), + ("UInt16_column", 2**16 - 1), + ("Int32_column", -(2**31)), + ("UInt32_column", 2**32 - 1), + ("Int64_column", -(2**63)), + ("UInt64_column", 2**64 - 1), ("String_column", "тест"), ("FixedString_column", "тест"), ("Float32_column", 1.5), diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index d39787f8924..054418a8ba9 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -561,7 +561,9 @@ def test_make_clone_in_detached(started_cluster): ["cp", "-r", path + "all_0_0_0", path + "detached/broken_all_0_0_0"] ) assert_eq_with_retry(instance, "select * from clone_in_detached", "\n") - assert ["broken_all_0_0_0",] == sorted( + assert [ + "broken_all_0_0_0", + ] == sorted( instance.exec_in_container(["ls", path + "detached/"]).strip().split("\n") ) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 16425c9bd9e..1fc3fe37044 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -506,9 +506,12 @@ def test_alters_from_different_replicas(started_cluster): dummy_node.stop_clickhouse(kill=True) settings = {"distributed_ddl_task_timeout": 5} - assert "There are 1 unfinished hosts (0 of them are currently executing the task" in competing_node.query_and_get_error( - "ALTER TABLE alters_from_different_replicas.concurrent_test ADD COLUMN Added0 UInt32;", - settings=settings, + assert ( + "There are 1 unfinished hosts (0 of them are currently executing the task" + in competing_node.query_and_get_error( + "ALTER TABLE alters_from_different_replicas.concurrent_test ADD COLUMN Added0 UInt32;", + settings=settings, + ) ) settings = { "distributed_ddl_task_timeout": 5, diff --git a/tests/integration/test_replicated_database_cluster_groups/test.py b/tests/integration/test_replicated_database_cluster_groups/test.py index 5a315707efb..647626d8014 100644 --- a/tests/integration/test_replicated_database_cluster_groups/test.py +++ b/tests/integration/test_replicated_database_cluster_groups/test.py @@ -95,9 +95,12 @@ def test_cluster_groups(started_cluster): # Exception main_node_2.stop_clickhouse() settings = {"distributed_ddl_task_timeout": 5} - assert "There are 1 unfinished hosts (0 of them are currently executing the task)" in main_node_1.query_and_get_error( - "CREATE TABLE cluster_groups.table_2 (d Date, k UInt64) ENGINE=ReplicatedMergeTree ORDER BY k PARTITION BY toYYYYMM(d);", - settings=settings, + assert ( + "There are 1 unfinished hosts (0 of them are currently executing the task)" + in main_node_1.query_and_get_error( + "CREATE TABLE cluster_groups.table_2 (d Date, k UInt64) ENGINE=ReplicatedMergeTree ORDER BY k PARTITION BY toYYYYMM(d);", + settings=settings, + ) ) # 3. After start both groups are synced diff --git a/tests/integration/test_storage_s3/s3_mocks/unstable_server.py b/tests/integration/test_storage_s3/s3_mocks/unstable_server.py index 3632fa15d8a..5ef781bdc9e 100644 --- a/tests/integration/test_storage_s3/s3_mocks/unstable_server.py +++ b/tests/integration/test_storage_s3/s3_mocks/unstable_server.py @@ -9,7 +9,7 @@ import time def gen_n_digit_number(n): assert 0 < n < 19 - return random.randint(10 ** (n - 1), 10 ** n - 1) + return random.randint(10 ** (n - 1), 10**n - 1) sum_in_4_column = 0 diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index e941356261a..2549cb0d473 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -553,13 +553,16 @@ def test_multipart(started_cluster, maybe_auth, positive): assert csv_data == get_s3_file_content(started_cluster, bucket, filename) # select uploaded data from many threads - select_query = "select sum(column1), sum(column2), sum(column3) " "from s3('http://{host}:{port}/{bucket}/{filename}', {auth}'CSV', '{table_format}')".format( - host=started_cluster.minio_redirect_host, - port=started_cluster.minio_redirect_port, - bucket=bucket, - filename=filename, - auth=maybe_auth, - table_format=table_format, + select_query = ( + "select sum(column1), sum(column2), sum(column3) " + "from s3('http://{host}:{port}/{bucket}/{filename}', {auth}'CSV', '{table_format}')".format( + host=started_cluster.minio_redirect_host, + port=started_cluster.minio_redirect_port, + bucket=bucket, + filename=filename, + auth=maybe_auth, + table_format=table_format, + ) ) try: select_result = run_query( diff --git a/tests/integration/test_storage_url/test.py b/tests/integration/test_storage_url/test.py index 771df49cbac..7ff7a871413 100644 --- a/tests/integration/test_storage_url/test.py +++ b/tests/integration/test_storage_url/test.py @@ -79,21 +79,15 @@ def test_table_function_url_access_rights(): f"SELECT * FROM url('http://nginx:80/test_1', 'TSV')", user="u1" ) - assert ( - node1.query( - f"DESCRIBE TABLE url('http://nginx:80/test_1', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32')", - user="u1", - ) - == TSV([["column1", "UInt32"], ["column2", "UInt32"], ["column3", "UInt32"]]) - ) + assert node1.query( + f"DESCRIBE TABLE url('http://nginx:80/test_1', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32')", + user="u1", + ) == TSV([["column1", "UInt32"], ["column2", "UInt32"], ["column3", "UInt32"]]) - assert ( - node1.query( - f"DESCRIBE TABLE url('http://nginx:80/not-exist', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32')", - user="u1", - ) - == TSV([["column1", "UInt32"], ["column2", "UInt32"], ["column3", "UInt32"]]) - ) + assert node1.query( + f"DESCRIBE TABLE url('http://nginx:80/not-exist', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32')", + user="u1", + ) == TSV([["column1", "UInt32"], ["column2", "UInt32"], ["column3", "UInt32"]]) expected_error = "necessary to have the grant URL ON *.*" assert expected_error in node1.query_and_get_error( diff --git a/tests/integration/test_system_merges/test.py b/tests/integration/test_system_merges/test.py index bacb0eb500d..6dbe6c891f2 100644 --- a/tests/integration/test_system_merges/test.py +++ b/tests/integration/test_system_merges/test.py @@ -204,33 +204,36 @@ def test_mutation_simple(started_cluster, replicated): sleep_time=0.1, ) - assert split_tsv( - node_check.query( - """ + assert ( + split_tsv( + node_check.query( + """ SELECT database, table, num_parts, source_part_names, source_part_paths, result_part_name, result_part_path, partition_id, is_mutation FROM system.merges WHERE table = '{name}' """.format( - name=table_name + name=table_name + ) ) ) - ) == [ - [ - db_name, - table_name, - "1", - "['{}']".format(part), - "['{clickhouse}/{table_path}/{}/']".format( - part, clickhouse=clickhouse_path, table_path=table_path - ), - result_part, - "{clickhouse}/{table_path}/{}/".format( - result_part, clickhouse=clickhouse_path, table_path=table_path - ), - "all", - "1", - ], - ] + == [ + [ + db_name, + table_name, + "1", + "['{}']".format(part), + "['{clickhouse}/{table_path}/{}/']".format( + part, clickhouse=clickhouse_path, table_path=table_path + ), + result_part, + "{clickhouse}/{table_path}/{}/".format( + result_part, clickhouse=clickhouse_path, table_path=table_path + ), + "all", + "1", + ], + ] + ) t.join() assert ( diff --git a/utils/grpc-client/pb2/clickhouse_grpc_pb2.py b/utils/grpc-client/pb2/clickhouse_grpc_pb2.py index 9bf7817c7d3..6218047af3c 100644 --- a/utils/grpc-client/pb2/clickhouse_grpc_pb2.py +++ b/utils/grpc-client/pb2/clickhouse_grpc_pb2.py @@ -8,17 +8,16 @@ from google.protobuf import descriptor_pool as _descriptor_pool from google.protobuf import message as _message from google.protobuf import reflection as _reflection from google.protobuf import symbol_database as _symbol_database - # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x15\x63lickhouse_grpc.proto\x12\x0f\x63lickhouse.grpc")\n\x0bNameAndType\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x0c\n\x04type\x18\x02 \x01(\t"\xf5\x01\n\rExternalTable\x12\x0c\n\x04name\x18\x01 \x01(\t\x12-\n\x07\x63olumns\x18\x02 \x03(\x0b\x32\x1c.clickhouse.grpc.NameAndType\x12\x0c\n\x04\x64\x61ta\x18\x03 \x01(\x0c\x12\x0e\n\x06\x66ormat\x18\x04 \x01(\t\x12\x18\n\x10\x63ompression_type\x18\x06 \x01(\t\x12>\n\x08settings\x18\x05 \x03(\x0b\x32,.clickhouse.grpc.ExternalTable.SettingsEntry\x1a/\n\rSettingsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01"\x85\x03\n\x1cObsoleteTransportCompression\x12U\n\talgorithm\x18\x01 \x01(\x0e\x32\x42.clickhouse.grpc.ObsoleteTransportCompression.CompressionAlgorithm\x12M\n\x05level\x18\x02 \x01(\x0e\x32>.clickhouse.grpc.ObsoleteTransportCompression.CompressionLevel"R\n\x14\x43ompressionAlgorithm\x12\x12\n\x0eNO_COMPRESSION\x10\x00\x12\x0b\n\x07\x44\x45\x46LATE\x10\x01\x12\x08\n\x04GZIP\x10\x02\x12\x0f\n\x0bSTREAM_GZIP\x10\x03"k\n\x10\x43ompressionLevel\x12\x14\n\x10\x43OMPRESSION_NONE\x10\x00\x12\x13\n\x0f\x43OMPRESSION_LOW\x10\x01\x12\x16\n\x12\x43OMPRESSION_MEDIUM\x10\x02\x12\x14\n\x10\x43OMPRESSION_HIGH\x10\x03"\x8e\x06\n\tQueryInfo\x12\r\n\x05query\x18\x01 \x01(\t\x12\x10\n\x08query_id\x18\x02 \x01(\t\x12:\n\x08settings\x18\x03 \x03(\x0b\x32(.clickhouse.grpc.QueryInfo.SettingsEntry\x12\x10\n\x08\x64\x61tabase\x18\x04 \x01(\t\x12\x12\n\ninput_data\x18\x05 \x01(\x0c\x12\x1c\n\x14input_data_delimiter\x18\x06 \x01(\x0c\x12\x15\n\routput_format\x18\x07 \x01(\t\x12\x1b\n\x13send_output_columns\x18\x18 \x01(\x08\x12\x37\n\x0f\x65xternal_tables\x18\x08 \x03(\x0b\x32\x1e.clickhouse.grpc.ExternalTable\x12\x11\n\tuser_name\x18\t \x01(\t\x12\x10\n\x08password\x18\n \x01(\t\x12\r\n\x05quota\x18\x0b \x01(\t\x12\x12\n\nsession_id\x18\x0c \x01(\t\x12\x15\n\rsession_check\x18\r \x01(\x08\x12\x17\n\x0fsession_timeout\x18\x0e \x01(\r\x12\x0e\n\x06\x63\x61ncel\x18\x0f \x01(\x08\x12\x17\n\x0fnext_query_info\x18\x10 \x01(\x08\x12\x1e\n\x16input_compression_type\x18\x14 \x01(\t\x12\x1f\n\x17output_compression_type\x18\x15 \x01(\t\x12 \n\x18output_compression_level\x18\x13 \x01(\x05\x12"\n\x1atransport_compression_type\x18\x16 \x01(\t\x12#\n\x1btransport_compression_level\x18\x17 \x01(\x05\x12R\n\x1bobsolete_result_compression\x18\x11 \x01(\x0b\x32-.clickhouse.grpc.ObsoleteTransportCompression\x12!\n\x19obsolete_compression_type\x18\x12 \x01(\t\x1a/\n\rSettingsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01"\xa1\x01\n\x08LogEntry\x12\x0c\n\x04time\x18\x01 \x01(\r\x12\x19\n\x11time_microseconds\x18\x02 \x01(\r\x12\x11\n\tthread_id\x18\x03 \x01(\x04\x12\x10\n\x08query_id\x18\x04 \x01(\t\x12)\n\x05level\x18\x05 \x01(\x0e\x32\x1a.clickhouse.grpc.LogsLevel\x12\x0e\n\x06source\x18\x06 \x01(\t\x12\x0c\n\x04text\x18\x07 \x01(\t"z\n\x08Progress\x12\x11\n\tread_rows\x18\x01 \x01(\x04\x12\x12\n\nread_bytes\x18\x02 \x01(\x04\x12\x1a\n\x12total_rows_to_read\x18\x03 \x01(\x04\x12\x14\n\x0cwritten_rows\x18\x04 \x01(\x04\x12\x15\n\rwritten_bytes\x18\x05 \x01(\x04"p\n\x05Stats\x12\x0c\n\x04rows\x18\x01 \x01(\x04\x12\x0e\n\x06\x62locks\x18\x02 \x01(\x04\x12\x17\n\x0f\x61llocated_bytes\x18\x03 \x01(\x04\x12\x15\n\rapplied_limit\x18\x04 \x01(\x08\x12\x19\n\x11rows_before_limit\x18\x05 \x01(\x04"R\n\tException\x12\x0c\n\x04\x63ode\x18\x01 \x01(\x05\x12\x0c\n\x04name\x18\x02 \x01(\t\x12\x14\n\x0c\x64isplay_text\x18\x03 \x01(\t\x12\x13\n\x0bstack_trace\x18\x04 \x01(\t"\xeb\x02\n\x06Result\x12\x10\n\x08query_id\x18\t \x01(\t\x12\x11\n\ttime_zone\x18\n \x01(\t\x12\x15\n\routput_format\x18\x0b \x01(\t\x12\x34\n\x0eoutput_columns\x18\x0c \x03(\x0b\x32\x1c.clickhouse.grpc.NameAndType\x12\x0e\n\x06output\x18\x01 \x01(\x0c\x12\x0e\n\x06totals\x18\x02 \x01(\x0c\x12\x10\n\x08\x65xtremes\x18\x03 \x01(\x0c\x12\'\n\x04logs\x18\x04 \x03(\x0b\x32\x19.clickhouse.grpc.LogEntry\x12+\n\x08progress\x18\x05 \x01(\x0b\x32\x19.clickhouse.grpc.Progress\x12%\n\x05stats\x18\x06 \x01(\x0b\x32\x16.clickhouse.grpc.Stats\x12-\n\texception\x18\x07 \x01(\x0b\x32\x1a.clickhouse.grpc.Exception\x12\x11\n\tcancelled\x18\x08 \x01(\x08*\x9d\x01\n\tLogsLevel\x12\x0c\n\x08LOG_NONE\x10\x00\x12\r\n\tLOG_FATAL\x10\x01\x12\x10\n\x0cLOG_CRITICAL\x10\x02\x12\r\n\tLOG_ERROR\x10\x03\x12\x0f\n\x0bLOG_WARNING\x10\x04\x12\x0e\n\nLOG_NOTICE\x10\x05\x12\x13\n\x0fLOG_INFORMATION\x10\x06\x12\r\n\tLOG_DEBUG\x10\x07\x12\r\n\tLOG_TRACE\x10\x08\x32\xdb\x02\n\nClickHouse\x12\x45\n\x0c\x45xecuteQuery\x12\x1a.clickhouse.grpc.QueryInfo\x1a\x17.clickhouse.grpc.Result"\x00\x12V\n\x1b\x45xecuteQueryWithStreamInput\x12\x1a.clickhouse.grpc.QueryInfo\x1a\x17.clickhouse.grpc.Result"\x00(\x01\x12W\n\x1c\x45xecuteQueryWithStreamOutput\x12\x1a.clickhouse.grpc.QueryInfo\x1a\x17.clickhouse.grpc.Result"\x00\x30\x01\x12U\n\x18\x45xecuteQueryWithStreamIO\x12\x1a.clickhouse.grpc.QueryInfo\x1a\x17.clickhouse.grpc.Result"\x00(\x01\x30\x01\x62\x06proto3' -) -_LOGSLEVEL = DESCRIPTOR.enum_types_by_name["LogsLevel"] + +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x15\x63lickhouse_grpc.proto\x12\x0f\x63lickhouse.grpc\")\n\x0bNameAndType\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x0c\n\x04type\x18\x02 \x01(\t\"\xf5\x01\n\rExternalTable\x12\x0c\n\x04name\x18\x01 \x01(\t\x12-\n\x07\x63olumns\x18\x02 \x03(\x0b\x32\x1c.clickhouse.grpc.NameAndType\x12\x0c\n\x04\x64\x61ta\x18\x03 \x01(\x0c\x12\x0e\n\x06\x66ormat\x18\x04 \x01(\t\x12\x18\n\x10\x63ompression_type\x18\x06 \x01(\t\x12>\n\x08settings\x18\x05 \x03(\x0b\x32,.clickhouse.grpc.ExternalTable.SettingsEntry\x1a/\n\rSettingsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"\x85\x03\n\x1cObsoleteTransportCompression\x12U\n\talgorithm\x18\x01 \x01(\x0e\x32\x42.clickhouse.grpc.ObsoleteTransportCompression.CompressionAlgorithm\x12M\n\x05level\x18\x02 \x01(\x0e\x32>.clickhouse.grpc.ObsoleteTransportCompression.CompressionLevel\"R\n\x14\x43ompressionAlgorithm\x12\x12\n\x0eNO_COMPRESSION\x10\x00\x12\x0b\n\x07\x44\x45\x46LATE\x10\x01\x12\x08\n\x04GZIP\x10\x02\x12\x0f\n\x0bSTREAM_GZIP\x10\x03\"k\n\x10\x43ompressionLevel\x12\x14\n\x10\x43OMPRESSION_NONE\x10\x00\x12\x13\n\x0f\x43OMPRESSION_LOW\x10\x01\x12\x16\n\x12\x43OMPRESSION_MEDIUM\x10\x02\x12\x14\n\x10\x43OMPRESSION_HIGH\x10\x03\"\x8e\x06\n\tQueryInfo\x12\r\n\x05query\x18\x01 \x01(\t\x12\x10\n\x08query_id\x18\x02 \x01(\t\x12:\n\x08settings\x18\x03 \x03(\x0b\x32(.clickhouse.grpc.QueryInfo.SettingsEntry\x12\x10\n\x08\x64\x61tabase\x18\x04 \x01(\t\x12\x12\n\ninput_data\x18\x05 \x01(\x0c\x12\x1c\n\x14input_data_delimiter\x18\x06 \x01(\x0c\x12\x15\n\routput_format\x18\x07 \x01(\t\x12\x1b\n\x13send_output_columns\x18\x18 \x01(\x08\x12\x37\n\x0f\x65xternal_tables\x18\x08 \x03(\x0b\x32\x1e.clickhouse.grpc.ExternalTable\x12\x11\n\tuser_name\x18\t \x01(\t\x12\x10\n\x08password\x18\n \x01(\t\x12\r\n\x05quota\x18\x0b \x01(\t\x12\x12\n\nsession_id\x18\x0c \x01(\t\x12\x15\n\rsession_check\x18\r \x01(\x08\x12\x17\n\x0fsession_timeout\x18\x0e \x01(\r\x12\x0e\n\x06\x63\x61ncel\x18\x0f \x01(\x08\x12\x17\n\x0fnext_query_info\x18\x10 \x01(\x08\x12\x1e\n\x16input_compression_type\x18\x14 \x01(\t\x12\x1f\n\x17output_compression_type\x18\x15 \x01(\t\x12 \n\x18output_compression_level\x18\x13 \x01(\x05\x12\"\n\x1atransport_compression_type\x18\x16 \x01(\t\x12#\n\x1btransport_compression_level\x18\x17 \x01(\x05\x12R\n\x1bobsolete_result_compression\x18\x11 \x01(\x0b\x32-.clickhouse.grpc.ObsoleteTransportCompression\x12!\n\x19obsolete_compression_type\x18\x12 \x01(\t\x1a/\n\rSettingsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"\xa1\x01\n\x08LogEntry\x12\x0c\n\x04time\x18\x01 \x01(\r\x12\x19\n\x11time_microseconds\x18\x02 \x01(\r\x12\x11\n\tthread_id\x18\x03 \x01(\x04\x12\x10\n\x08query_id\x18\x04 \x01(\t\x12)\n\x05level\x18\x05 \x01(\x0e\x32\x1a.clickhouse.grpc.LogsLevel\x12\x0e\n\x06source\x18\x06 \x01(\t\x12\x0c\n\x04text\x18\x07 \x01(\t\"z\n\x08Progress\x12\x11\n\tread_rows\x18\x01 \x01(\x04\x12\x12\n\nread_bytes\x18\x02 \x01(\x04\x12\x1a\n\x12total_rows_to_read\x18\x03 \x01(\x04\x12\x14\n\x0cwritten_rows\x18\x04 \x01(\x04\x12\x15\n\rwritten_bytes\x18\x05 \x01(\x04\"p\n\x05Stats\x12\x0c\n\x04rows\x18\x01 \x01(\x04\x12\x0e\n\x06\x62locks\x18\x02 \x01(\x04\x12\x17\n\x0f\x61llocated_bytes\x18\x03 \x01(\x04\x12\x15\n\rapplied_limit\x18\x04 \x01(\x08\x12\x19\n\x11rows_before_limit\x18\x05 \x01(\x04\"R\n\tException\x12\x0c\n\x04\x63ode\x18\x01 \x01(\x05\x12\x0c\n\x04name\x18\x02 \x01(\t\x12\x14\n\x0c\x64isplay_text\x18\x03 \x01(\t\x12\x13\n\x0bstack_trace\x18\x04 \x01(\t\"\xeb\x02\n\x06Result\x12\x10\n\x08query_id\x18\t \x01(\t\x12\x11\n\ttime_zone\x18\n \x01(\t\x12\x15\n\routput_format\x18\x0b \x01(\t\x12\x34\n\x0eoutput_columns\x18\x0c \x03(\x0b\x32\x1c.clickhouse.grpc.NameAndType\x12\x0e\n\x06output\x18\x01 \x01(\x0c\x12\x0e\n\x06totals\x18\x02 \x01(\x0c\x12\x10\n\x08\x65xtremes\x18\x03 \x01(\x0c\x12\'\n\x04logs\x18\x04 \x03(\x0b\x32\x19.clickhouse.grpc.LogEntry\x12+\n\x08progress\x18\x05 \x01(\x0b\x32\x19.clickhouse.grpc.Progress\x12%\n\x05stats\x18\x06 \x01(\x0b\x32\x16.clickhouse.grpc.Stats\x12-\n\texception\x18\x07 \x01(\x0b\x32\x1a.clickhouse.grpc.Exception\x12\x11\n\tcancelled\x18\x08 \x01(\x08*\x9d\x01\n\tLogsLevel\x12\x0c\n\x08LOG_NONE\x10\x00\x12\r\n\tLOG_FATAL\x10\x01\x12\x10\n\x0cLOG_CRITICAL\x10\x02\x12\r\n\tLOG_ERROR\x10\x03\x12\x0f\n\x0bLOG_WARNING\x10\x04\x12\x0e\n\nLOG_NOTICE\x10\x05\x12\x13\n\x0fLOG_INFORMATION\x10\x06\x12\r\n\tLOG_DEBUG\x10\x07\x12\r\n\tLOG_TRACE\x10\x08\x32\xdb\x02\n\nClickHouse\x12\x45\n\x0c\x45xecuteQuery\x12\x1a.clickhouse.grpc.QueryInfo\x1a\x17.clickhouse.grpc.Result\"\x00\x12V\n\x1b\x45xecuteQueryWithStreamInput\x12\x1a.clickhouse.grpc.QueryInfo\x1a\x17.clickhouse.grpc.Result\"\x00(\x01\x12W\n\x1c\x45xecuteQueryWithStreamOutput\x12\x1a.clickhouse.grpc.QueryInfo\x1a\x17.clickhouse.grpc.Result\"\x00\x30\x01\x12U\n\x18\x45xecuteQueryWithStreamIO\x12\x1a.clickhouse.grpc.QueryInfo\x1a\x17.clickhouse.grpc.Result\"\x00(\x01\x30\x01\x62\x06proto3') + +_LOGSLEVEL = DESCRIPTOR.enum_types_by_name['LogsLevel'] LogsLevel = enum_type_wrapper.EnumTypeWrapper(_LOGSLEVEL) LOG_NONE = 0 LOG_FATAL = 1 @@ -31,180 +30,134 @@ LOG_DEBUG = 7 LOG_TRACE = 8 -_NAMEANDTYPE = DESCRIPTOR.message_types_by_name["NameAndType"] -_EXTERNALTABLE = DESCRIPTOR.message_types_by_name["ExternalTable"] -_EXTERNALTABLE_SETTINGSENTRY = _EXTERNALTABLE.nested_types_by_name["SettingsEntry"] -_OBSOLETETRANSPORTCOMPRESSION = DESCRIPTOR.message_types_by_name[ - "ObsoleteTransportCompression" -] -_QUERYINFO = DESCRIPTOR.message_types_by_name["QueryInfo"] -_QUERYINFO_SETTINGSENTRY = _QUERYINFO.nested_types_by_name["SettingsEntry"] -_LOGENTRY = DESCRIPTOR.message_types_by_name["LogEntry"] -_PROGRESS = DESCRIPTOR.message_types_by_name["Progress"] -_STATS = DESCRIPTOR.message_types_by_name["Stats"] -_EXCEPTION = DESCRIPTOR.message_types_by_name["Exception"] -_RESULT = DESCRIPTOR.message_types_by_name["Result"] -_OBSOLETETRANSPORTCOMPRESSION_COMPRESSIONALGORITHM = ( - _OBSOLETETRANSPORTCOMPRESSION.enum_types_by_name["CompressionAlgorithm"] -) -_OBSOLETETRANSPORTCOMPRESSION_COMPRESSIONLEVEL = ( - _OBSOLETETRANSPORTCOMPRESSION.enum_types_by_name["CompressionLevel"] -) -NameAndType = _reflection.GeneratedProtocolMessageType( - "NameAndType", - (_message.Message,), - { - "DESCRIPTOR": _NAMEANDTYPE, - "__module__": "clickhouse_grpc_pb2" - # @@protoc_insertion_point(class_scope:clickhouse.grpc.NameAndType) - }, -) +_NAMEANDTYPE = DESCRIPTOR.message_types_by_name['NameAndType'] +_EXTERNALTABLE = DESCRIPTOR.message_types_by_name['ExternalTable'] +_EXTERNALTABLE_SETTINGSENTRY = _EXTERNALTABLE.nested_types_by_name['SettingsEntry'] +_OBSOLETETRANSPORTCOMPRESSION = DESCRIPTOR.message_types_by_name['ObsoleteTransportCompression'] +_QUERYINFO = DESCRIPTOR.message_types_by_name['QueryInfo'] +_QUERYINFO_SETTINGSENTRY = _QUERYINFO.nested_types_by_name['SettingsEntry'] +_LOGENTRY = DESCRIPTOR.message_types_by_name['LogEntry'] +_PROGRESS = DESCRIPTOR.message_types_by_name['Progress'] +_STATS = DESCRIPTOR.message_types_by_name['Stats'] +_EXCEPTION = DESCRIPTOR.message_types_by_name['Exception'] +_RESULT = DESCRIPTOR.message_types_by_name['Result'] +_OBSOLETETRANSPORTCOMPRESSION_COMPRESSIONALGORITHM = _OBSOLETETRANSPORTCOMPRESSION.enum_types_by_name['CompressionAlgorithm'] +_OBSOLETETRANSPORTCOMPRESSION_COMPRESSIONLEVEL = _OBSOLETETRANSPORTCOMPRESSION.enum_types_by_name['CompressionLevel'] +NameAndType = _reflection.GeneratedProtocolMessageType('NameAndType', (_message.Message,), { + 'DESCRIPTOR' : _NAMEANDTYPE, + '__module__' : 'clickhouse_grpc_pb2' + # @@protoc_insertion_point(class_scope:clickhouse.grpc.NameAndType) + }) _sym_db.RegisterMessage(NameAndType) -ExternalTable = _reflection.GeneratedProtocolMessageType( - "ExternalTable", - (_message.Message,), - { - "SettingsEntry": _reflection.GeneratedProtocolMessageType( - "SettingsEntry", - (_message.Message,), - { - "DESCRIPTOR": _EXTERNALTABLE_SETTINGSENTRY, - "__module__": "clickhouse_grpc_pb2" - # @@protoc_insertion_point(class_scope:clickhouse.grpc.ExternalTable.SettingsEntry) - }, - ), - "DESCRIPTOR": _EXTERNALTABLE, - "__module__": "clickhouse_grpc_pb2" - # @@protoc_insertion_point(class_scope:clickhouse.grpc.ExternalTable) - }, -) +ExternalTable = _reflection.GeneratedProtocolMessageType('ExternalTable', (_message.Message,), { + + 'SettingsEntry' : _reflection.GeneratedProtocolMessageType('SettingsEntry', (_message.Message,), { + 'DESCRIPTOR' : _EXTERNALTABLE_SETTINGSENTRY, + '__module__' : 'clickhouse_grpc_pb2' + # @@protoc_insertion_point(class_scope:clickhouse.grpc.ExternalTable.SettingsEntry) + }) + , + 'DESCRIPTOR' : _EXTERNALTABLE, + '__module__' : 'clickhouse_grpc_pb2' + # @@protoc_insertion_point(class_scope:clickhouse.grpc.ExternalTable) + }) _sym_db.RegisterMessage(ExternalTable) _sym_db.RegisterMessage(ExternalTable.SettingsEntry) -ObsoleteTransportCompression = _reflection.GeneratedProtocolMessageType( - "ObsoleteTransportCompression", - (_message.Message,), - { - "DESCRIPTOR": _OBSOLETETRANSPORTCOMPRESSION, - "__module__": "clickhouse_grpc_pb2" - # @@protoc_insertion_point(class_scope:clickhouse.grpc.ObsoleteTransportCompression) - }, -) +ObsoleteTransportCompression = _reflection.GeneratedProtocolMessageType('ObsoleteTransportCompression', (_message.Message,), { + 'DESCRIPTOR' : _OBSOLETETRANSPORTCOMPRESSION, + '__module__' : 'clickhouse_grpc_pb2' + # @@protoc_insertion_point(class_scope:clickhouse.grpc.ObsoleteTransportCompression) + }) _sym_db.RegisterMessage(ObsoleteTransportCompression) -QueryInfo = _reflection.GeneratedProtocolMessageType( - "QueryInfo", - (_message.Message,), - { - "SettingsEntry": _reflection.GeneratedProtocolMessageType( - "SettingsEntry", - (_message.Message,), - { - "DESCRIPTOR": _QUERYINFO_SETTINGSENTRY, - "__module__": "clickhouse_grpc_pb2" - # @@protoc_insertion_point(class_scope:clickhouse.grpc.QueryInfo.SettingsEntry) - }, - ), - "DESCRIPTOR": _QUERYINFO, - "__module__": "clickhouse_grpc_pb2" - # @@protoc_insertion_point(class_scope:clickhouse.grpc.QueryInfo) - }, -) +QueryInfo = _reflection.GeneratedProtocolMessageType('QueryInfo', (_message.Message,), { + + 'SettingsEntry' : _reflection.GeneratedProtocolMessageType('SettingsEntry', (_message.Message,), { + 'DESCRIPTOR' : _QUERYINFO_SETTINGSENTRY, + '__module__' : 'clickhouse_grpc_pb2' + # @@protoc_insertion_point(class_scope:clickhouse.grpc.QueryInfo.SettingsEntry) + }) + , + 'DESCRIPTOR' : _QUERYINFO, + '__module__' : 'clickhouse_grpc_pb2' + # @@protoc_insertion_point(class_scope:clickhouse.grpc.QueryInfo) + }) _sym_db.RegisterMessage(QueryInfo) _sym_db.RegisterMessage(QueryInfo.SettingsEntry) -LogEntry = _reflection.GeneratedProtocolMessageType( - "LogEntry", - (_message.Message,), - { - "DESCRIPTOR": _LOGENTRY, - "__module__": "clickhouse_grpc_pb2" - # @@protoc_insertion_point(class_scope:clickhouse.grpc.LogEntry) - }, -) +LogEntry = _reflection.GeneratedProtocolMessageType('LogEntry', (_message.Message,), { + 'DESCRIPTOR' : _LOGENTRY, + '__module__' : 'clickhouse_grpc_pb2' + # @@protoc_insertion_point(class_scope:clickhouse.grpc.LogEntry) + }) _sym_db.RegisterMessage(LogEntry) -Progress = _reflection.GeneratedProtocolMessageType( - "Progress", - (_message.Message,), - { - "DESCRIPTOR": _PROGRESS, - "__module__": "clickhouse_grpc_pb2" - # @@protoc_insertion_point(class_scope:clickhouse.grpc.Progress) - }, -) +Progress = _reflection.GeneratedProtocolMessageType('Progress', (_message.Message,), { + 'DESCRIPTOR' : _PROGRESS, + '__module__' : 'clickhouse_grpc_pb2' + # @@protoc_insertion_point(class_scope:clickhouse.grpc.Progress) + }) _sym_db.RegisterMessage(Progress) -Stats = _reflection.GeneratedProtocolMessageType( - "Stats", - (_message.Message,), - { - "DESCRIPTOR": _STATS, - "__module__": "clickhouse_grpc_pb2" - # @@protoc_insertion_point(class_scope:clickhouse.grpc.Stats) - }, -) +Stats = _reflection.GeneratedProtocolMessageType('Stats', (_message.Message,), { + 'DESCRIPTOR' : _STATS, + '__module__' : 'clickhouse_grpc_pb2' + # @@protoc_insertion_point(class_scope:clickhouse.grpc.Stats) + }) _sym_db.RegisterMessage(Stats) -Exception = _reflection.GeneratedProtocolMessageType( - "Exception", - (_message.Message,), - { - "DESCRIPTOR": _EXCEPTION, - "__module__": "clickhouse_grpc_pb2" - # @@protoc_insertion_point(class_scope:clickhouse.grpc.Exception) - }, -) +Exception = _reflection.GeneratedProtocolMessageType('Exception', (_message.Message,), { + 'DESCRIPTOR' : _EXCEPTION, + '__module__' : 'clickhouse_grpc_pb2' + # @@protoc_insertion_point(class_scope:clickhouse.grpc.Exception) + }) _sym_db.RegisterMessage(Exception) -Result = _reflection.GeneratedProtocolMessageType( - "Result", - (_message.Message,), - { - "DESCRIPTOR": _RESULT, - "__module__": "clickhouse_grpc_pb2" - # @@protoc_insertion_point(class_scope:clickhouse.grpc.Result) - }, -) +Result = _reflection.GeneratedProtocolMessageType('Result', (_message.Message,), { + 'DESCRIPTOR' : _RESULT, + '__module__' : 'clickhouse_grpc_pb2' + # @@protoc_insertion_point(class_scope:clickhouse.grpc.Result) + }) _sym_db.RegisterMessage(Result) -_CLICKHOUSE = DESCRIPTOR.services_by_name["ClickHouse"] +_CLICKHOUSE = DESCRIPTOR.services_by_name['ClickHouse'] if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None - _EXTERNALTABLE_SETTINGSENTRY._options = None - _EXTERNALTABLE_SETTINGSENTRY._serialized_options = b"8\001" - _QUERYINFO_SETTINGSENTRY._options = None - _QUERYINFO_SETTINGSENTRY._serialized_options = b"8\001" - _LOGSLEVEL._serialized_start = 2363 - _LOGSLEVEL._serialized_end = 2520 - _NAMEANDTYPE._serialized_start = 42 - _NAMEANDTYPE._serialized_end = 83 - _EXTERNALTABLE._serialized_start = 86 - _EXTERNALTABLE._serialized_end = 331 - _EXTERNALTABLE_SETTINGSENTRY._serialized_start = 284 - _EXTERNALTABLE_SETTINGSENTRY._serialized_end = 331 - _OBSOLETETRANSPORTCOMPRESSION._serialized_start = 334 - _OBSOLETETRANSPORTCOMPRESSION._serialized_end = 723 - _OBSOLETETRANSPORTCOMPRESSION_COMPRESSIONALGORITHM._serialized_start = 532 - _OBSOLETETRANSPORTCOMPRESSION_COMPRESSIONALGORITHM._serialized_end = 614 - _OBSOLETETRANSPORTCOMPRESSION_COMPRESSIONLEVEL._serialized_start = 616 - _OBSOLETETRANSPORTCOMPRESSION_COMPRESSIONLEVEL._serialized_end = 723 - _QUERYINFO._serialized_start = 726 - _QUERYINFO._serialized_end = 1508 - _QUERYINFO_SETTINGSENTRY._serialized_start = 284 - _QUERYINFO_SETTINGSENTRY._serialized_end = 331 - _LOGENTRY._serialized_start = 1511 - _LOGENTRY._serialized_end = 1672 - _PROGRESS._serialized_start = 1674 - _PROGRESS._serialized_end = 1796 - _STATS._serialized_start = 1798 - _STATS._serialized_end = 1910 - _EXCEPTION._serialized_start = 1912 - _EXCEPTION._serialized_end = 1994 - _RESULT._serialized_start = 1997 - _RESULT._serialized_end = 2360 - _CLICKHOUSE._serialized_start = 2523 - _CLICKHOUSE._serialized_end = 2870 + DESCRIPTOR._options = None + _EXTERNALTABLE_SETTINGSENTRY._options = None + _EXTERNALTABLE_SETTINGSENTRY._serialized_options = b'8\001' + _QUERYINFO_SETTINGSENTRY._options = None + _QUERYINFO_SETTINGSENTRY._serialized_options = b'8\001' + _LOGSLEVEL._serialized_start=2363 + _LOGSLEVEL._serialized_end=2520 + _NAMEANDTYPE._serialized_start=42 + _NAMEANDTYPE._serialized_end=83 + _EXTERNALTABLE._serialized_start=86 + _EXTERNALTABLE._serialized_end=331 + _EXTERNALTABLE_SETTINGSENTRY._serialized_start=284 + _EXTERNALTABLE_SETTINGSENTRY._serialized_end=331 + _OBSOLETETRANSPORTCOMPRESSION._serialized_start=334 + _OBSOLETETRANSPORTCOMPRESSION._serialized_end=723 + _OBSOLETETRANSPORTCOMPRESSION_COMPRESSIONALGORITHM._serialized_start=532 + _OBSOLETETRANSPORTCOMPRESSION_COMPRESSIONALGORITHM._serialized_end=614 + _OBSOLETETRANSPORTCOMPRESSION_COMPRESSIONLEVEL._serialized_start=616 + _OBSOLETETRANSPORTCOMPRESSION_COMPRESSIONLEVEL._serialized_end=723 + _QUERYINFO._serialized_start=726 + _QUERYINFO._serialized_end=1508 + _QUERYINFO_SETTINGSENTRY._serialized_start=284 + _QUERYINFO_SETTINGSENTRY._serialized_end=331 + _LOGENTRY._serialized_start=1511 + _LOGENTRY._serialized_end=1672 + _PROGRESS._serialized_start=1674 + _PROGRESS._serialized_end=1796 + _STATS._serialized_start=1798 + _STATS._serialized_end=1910 + _EXCEPTION._serialized_start=1912 + _EXCEPTION._serialized_end=1994 + _RESULT._serialized_start=1997 + _RESULT._serialized_end=2360 + _CLICKHOUSE._serialized_start=2523 + _CLICKHOUSE._serialized_end=2870 # @@protoc_insertion_point(module_scope) diff --git a/utils/grpc-client/pb2/clickhouse_grpc_pb2_grpc.py b/utils/grpc-client/pb2/clickhouse_grpc_pb2_grpc.py index 25643a243b3..1c71218bbe5 100644 --- a/utils/grpc-client/pb2/clickhouse_grpc_pb2_grpc.py +++ b/utils/grpc-client/pb2/clickhouse_grpc_pb2_grpc.py @@ -15,25 +15,25 @@ class ClickHouseStub(object): channel: A grpc.Channel. """ self.ExecuteQuery = channel.unary_unary( - "/clickhouse.grpc.ClickHouse/ExecuteQuery", - request_serializer=clickhouse__grpc__pb2.QueryInfo.SerializeToString, - response_deserializer=clickhouse__grpc__pb2.Result.FromString, - ) + '/clickhouse.grpc.ClickHouse/ExecuteQuery', + request_serializer=clickhouse__grpc__pb2.QueryInfo.SerializeToString, + response_deserializer=clickhouse__grpc__pb2.Result.FromString, + ) self.ExecuteQueryWithStreamInput = channel.stream_unary( - "/clickhouse.grpc.ClickHouse/ExecuteQueryWithStreamInput", - request_serializer=clickhouse__grpc__pb2.QueryInfo.SerializeToString, - response_deserializer=clickhouse__grpc__pb2.Result.FromString, - ) + '/clickhouse.grpc.ClickHouse/ExecuteQueryWithStreamInput', + request_serializer=clickhouse__grpc__pb2.QueryInfo.SerializeToString, + response_deserializer=clickhouse__grpc__pb2.Result.FromString, + ) self.ExecuteQueryWithStreamOutput = channel.unary_stream( - "/clickhouse.grpc.ClickHouse/ExecuteQueryWithStreamOutput", - request_serializer=clickhouse__grpc__pb2.QueryInfo.SerializeToString, - response_deserializer=clickhouse__grpc__pb2.Result.FromString, - ) + '/clickhouse.grpc.ClickHouse/ExecuteQueryWithStreamOutput', + request_serializer=clickhouse__grpc__pb2.QueryInfo.SerializeToString, + response_deserializer=clickhouse__grpc__pb2.Result.FromString, + ) self.ExecuteQueryWithStreamIO = channel.stream_stream( - "/clickhouse.grpc.ClickHouse/ExecuteQueryWithStreamIO", - request_serializer=clickhouse__grpc__pb2.QueryInfo.SerializeToString, - response_deserializer=clickhouse__grpc__pb2.Result.FromString, - ) + '/clickhouse.grpc.ClickHouse/ExecuteQueryWithStreamIO', + request_serializer=clickhouse__grpc__pb2.QueryInfo.SerializeToString, + response_deserializer=clickhouse__grpc__pb2.Result.FromString, + ) class ClickHouseServicer(object): @@ -42,173 +42,124 @@ class ClickHouseServicer(object): def ExecuteQuery(self, request, context): """Missing associated documentation comment in .proto file.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def ExecuteQueryWithStreamInput(self, request_iterator, context): """Missing associated documentation comment in .proto file.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def ExecuteQueryWithStreamOutput(self, request, context): """Missing associated documentation comment in .proto file.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def ExecuteQueryWithStreamIO(self, request_iterator, context): """Missing associated documentation comment in .proto file.""" context.set_code(grpc.StatusCode.UNIMPLEMENTED) - context.set_details("Method not implemented!") - raise NotImplementedError("Method not implemented!") + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') def add_ClickHouseServicer_to_server(servicer, server): rpc_method_handlers = { - "ExecuteQuery": grpc.unary_unary_rpc_method_handler( - servicer.ExecuteQuery, - request_deserializer=clickhouse__grpc__pb2.QueryInfo.FromString, - response_serializer=clickhouse__grpc__pb2.Result.SerializeToString, - ), - "ExecuteQueryWithStreamInput": grpc.stream_unary_rpc_method_handler( - servicer.ExecuteQueryWithStreamInput, - request_deserializer=clickhouse__grpc__pb2.QueryInfo.FromString, - response_serializer=clickhouse__grpc__pb2.Result.SerializeToString, - ), - "ExecuteQueryWithStreamOutput": grpc.unary_stream_rpc_method_handler( - servicer.ExecuteQueryWithStreamOutput, - request_deserializer=clickhouse__grpc__pb2.QueryInfo.FromString, - response_serializer=clickhouse__grpc__pb2.Result.SerializeToString, - ), - "ExecuteQueryWithStreamIO": grpc.stream_stream_rpc_method_handler( - servicer.ExecuteQueryWithStreamIO, - request_deserializer=clickhouse__grpc__pb2.QueryInfo.FromString, - response_serializer=clickhouse__grpc__pb2.Result.SerializeToString, - ), + 'ExecuteQuery': grpc.unary_unary_rpc_method_handler( + servicer.ExecuteQuery, + request_deserializer=clickhouse__grpc__pb2.QueryInfo.FromString, + response_serializer=clickhouse__grpc__pb2.Result.SerializeToString, + ), + 'ExecuteQueryWithStreamInput': grpc.stream_unary_rpc_method_handler( + servicer.ExecuteQueryWithStreamInput, + request_deserializer=clickhouse__grpc__pb2.QueryInfo.FromString, + response_serializer=clickhouse__grpc__pb2.Result.SerializeToString, + ), + 'ExecuteQueryWithStreamOutput': grpc.unary_stream_rpc_method_handler( + servicer.ExecuteQueryWithStreamOutput, + request_deserializer=clickhouse__grpc__pb2.QueryInfo.FromString, + response_serializer=clickhouse__grpc__pb2.Result.SerializeToString, + ), + 'ExecuteQueryWithStreamIO': grpc.stream_stream_rpc_method_handler( + servicer.ExecuteQueryWithStreamIO, + request_deserializer=clickhouse__grpc__pb2.QueryInfo.FromString, + response_serializer=clickhouse__grpc__pb2.Result.SerializeToString, + ), } generic_handler = grpc.method_handlers_generic_handler( - "clickhouse.grpc.ClickHouse", rpc_method_handlers - ) + 'clickhouse.grpc.ClickHouse', rpc_method_handlers) server.add_generic_rpc_handlers((generic_handler,)) -# This class is part of an EXPERIMENTAL API. + # This class is part of an EXPERIMENTAL API. class ClickHouse(object): """Missing associated documentation comment in .proto file.""" @staticmethod - def ExecuteQuery( - request, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.unary_unary( - request, + def ExecuteQuery(request, target, - "/clickhouse.grpc.ClickHouse/ExecuteQuery", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.unary_unary(request, target, '/clickhouse.grpc.ClickHouse/ExecuteQuery', clickhouse__grpc__pb2.QueryInfo.SerializeToString, clickhouse__grpc__pb2.Result.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) @staticmethod - def ExecuteQueryWithStreamInput( - request_iterator, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.stream_unary( - request_iterator, + def ExecuteQueryWithStreamInput(request_iterator, target, - "/clickhouse.grpc.ClickHouse/ExecuteQueryWithStreamInput", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.stream_unary(request_iterator, target, '/clickhouse.grpc.ClickHouse/ExecuteQueryWithStreamInput', clickhouse__grpc__pb2.QueryInfo.SerializeToString, clickhouse__grpc__pb2.Result.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) @staticmethod - def ExecuteQueryWithStreamOutput( - request, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.unary_stream( - request, + def ExecuteQueryWithStreamOutput(request, target, - "/clickhouse.grpc.ClickHouse/ExecuteQueryWithStreamOutput", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.unary_stream(request, target, '/clickhouse.grpc.ClickHouse/ExecuteQueryWithStreamOutput', clickhouse__grpc__pb2.QueryInfo.SerializeToString, clickhouse__grpc__pb2.Result.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) @staticmethod - def ExecuteQueryWithStreamIO( - request_iterator, - target, - options=(), - channel_credentials=None, - call_credentials=None, - insecure=False, - compression=None, - wait_for_ready=None, - timeout=None, - metadata=None, - ): - return grpc.experimental.stream_stream( - request_iterator, + def ExecuteQueryWithStreamIO(request_iterator, target, - "/clickhouse.grpc.ClickHouse/ExecuteQueryWithStreamIO", + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None): + return grpc.experimental.stream_stream(request_iterator, target, '/clickhouse.grpc.ClickHouse/ExecuteQueryWithStreamIO', clickhouse__grpc__pb2.QueryInfo.SerializeToString, clickhouse__grpc__pb2.Result.FromString, - options, - channel_credentials, - insecure, - call_credentials, - compression, - wait_for_ready, - timeout, - metadata, - ) + options, channel_credentials, + insecure, call_credentials, compression, wait_for_ready, timeout, metadata) From 5cda358e62c90a2345a60a249b6d7e8430f6454d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jan 2024 06:26:04 +0100 Subject: [PATCH 062/276] Obey Python's quirky formatter --- tests/clickhouse-test | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index bd796dbfdf2..dd9047c293f 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -75,11 +75,13 @@ def stringhash(s): # only during process invocation https://stackoverflow.com/a/42089311 return zlib.crc32(s.encode("utf-8")) + def read_file_as_binary_string(file_path): - with open(file_path, 'rb') as file: + with open(file_path, "rb") as file: binary_data = file.read() return binary_data + # First and last lines of the log def trim_for_log(s): if not s: @@ -146,7 +148,7 @@ def clickhouse_execute_http( client.request( "POST", f"/?{base_args.client_options_query_str}{urllib.parse.urlencode(params)}", - body=body + body=body, ) res = client.getresponse() data = res.read() From 71bef27abfa9cd64a318306ddd11b21b907a37ac Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jan 2024 06:26:54 +0100 Subject: [PATCH 063/276] Follow-up --- docker/test/base/setup_export_logs.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 26fcd10d666..96a15c33674 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -194,7 +194,7 @@ function setup_logs_replication echo "Creating table system.${table}_sender" >&2 # Create Distributed table and materialized view to watch on the original table: - clickhouse-client --asterisk_include_materialized_columns 1 --query " + clickhouse-client --query " CREATE TABLE system.${table}_sender ENGINE = Distributed(${CLICKHOUSE_CI_LOGS_CLUSTER}, default, ${table}_${hash}) SETTINGS flush_on_detach=0 @@ -205,7 +205,7 @@ function setup_logs_replication echo "Creating materialized view system.${table}_watcher" >&2 - clickhouse-client --asterisk_include_materialized_columns 1 --query " + clickhouse-client --query " CREATE MATERIALIZED VIEW system.${table}_watcher TO system.${table}_sender AS SELECT ${EXTRA_COLUMNS_EXPRESSION_FOR_TABLE}, * FROM system.${table} From c5024a5f6d7f88f0fd8dc2af2c52eb1c1d57d2c2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 15 Jan 2024 23:36:48 +0100 Subject: [PATCH 064/276] Fix typo --- docker/test/base/setup_export_logs.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 96a15c33674..416281c2aa3 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -30,7 +30,7 @@ EXTRA_COLUMNS_EXPRESSION_COVERAGE_LOG="${EXTRA_COLUMNS_EXPRESSION}, arrayMap(x - function __set_connection_args { - # It's impossible to use generous $CONNECTION_ARGS string, it's unsafe from word splitting perspective. + # It's impossible to use a generic $CONNECTION_ARGS string, it's unsafe from word splitting perspective. # That's why we must stick to the generated option CONNECTION_ARGS=( --receive_timeout=45 --send_timeout=45 --secure From 1caef191436fc05856be3b85b19cfcd97d0dc804 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 16 Jan 2024 09:44:52 +0100 Subject: [PATCH 065/276] Maybe better --- programs/main.cpp | 81 ++++++++++++++++++++++++++--------------------- 1 file changed, 45 insertions(+), 36 deletions(-) diff --git a/programs/main.cpp b/programs/main.cpp index 8958d84e243..1ff7e5db560 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -394,6 +394,50 @@ void checkHarmfulEnvironmentVariables(char ** argv) } #endif + +#if defined(SANITIZE_COVERAGE) +__attribute__((no_sanitize("coverage"))) +void dumpCoverage() +{ + /// A user can request to dump the coverage information into files at exit. + /// This is useful for non-server applications such as clickhouse-format or clickhouse-client, + /// that cannot introspect it with SQL functions at runtime. + + /// The CLICKHOUSE_WRITE_COVERAGE environment variable defines a prefix for a filename 'prefix.pid' + /// containing the list of addresses of covered . + + /// The format is even simpler than Clang's "sancov": an array of 64-bit addresses, native byte order, no header. + + if (const char * coverage_filename_prefix = getenv("CLICKHOUSE_WRITE_COVERAGE")) // NOLINT(concurrency-mt-unsafe) + { + auto dump = [](const std::string & name, auto span) + { + /// Write only non-zeros. + std::vector data; + data.reserve(span.size()); + for (auto addr : span) + if (addr) + data.push_back(addr); + + int fd = ::open(name.c_str(), O_WRONLY | O_CREAT | O_TRUNC | O_CLOEXEC, 0400); + if (-1 == fd) + { + writeError("Cannot open a file to write the coverage data\n"); + } + else + { + if (!writeRetry(fd, reinterpret_cast(data.data()), data.size() * sizeof(data[0]))) + writeError("Cannot write the coverage data to a file\n"); + if (0 != ::close(fd)) + writeError("Cannot close the file with coverage data\n"); + } + }; + + dump(fmt::format("{}.{}", coverage_filename_prefix, getpid()), getCumulativeCoverage()); + } +} +#endif + } bool isClickhouseApp(std::string_view app_suffix, std::vector & argv) @@ -517,42 +561,7 @@ int main(int argc_, char ** argv_) int exit_code = main_func(static_cast(argv.size()), argv.data()); #if defined(SANITIZE_COVERAGE) - /// A user can request to dump the coverage information into files at exit. - /// This is useful for non-server applications such as clickhouse-format or clickhouse-client, - /// that cannot introspect it with SQL functions at runtime. - - /// The CLICKHOUSE_WRITE_COVERAGE environment variable defines a prefix for a filename 'prefix.pid' - /// containing the list of addresses of covered . - - /// The format is even simpler than Clang's "sancov": an array of 64-bit addresses, native byte order, no header. - - if (const char * coverage_filename_prefix = getenv("CLICKHOUSE_WRITE_COVERAGE")) // NOLINT(concurrency-mt-unsafe) - { - auto dumpCoverage = [](const std::string & name, auto span) - { - /// Write only non-zeros. - std::vector data; - data.reserve(span.size()); - for (auto addr : span) - if (addr) - data.push_back(addr); - - int fd = ::open(name.c_str(), O_WRONLY | O_CREAT | O_TRUNC | O_CLOEXEC, 0400); - if (-1 == fd) - { - writeError("Cannot open a file to write the coverage data\n"); - } - else - { - if (!writeRetry(fd, reinterpret_cast(data.data()), data.size() * sizeof(data[0]))) - writeError("Cannot write the coverage data to a file\n"); - if (0 != ::close(fd)) - writeError("Cannot close the file with coverage data\n"); - } - }; - - dumpCoverage(fmt::format("{}.{}", coverage_filename_prefix, getpid()), getCumulativeCoverage()); - } + dumpCoverage(); #endif return exit_code; From 21082be9a681166b5585445c8aed62e705063081 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 16 Jan 2024 10:41:13 +0100 Subject: [PATCH 066/276] Better test --- tests/clickhouse-test | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index dd9047c293f..6d398115d43 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1269,13 +1269,17 @@ class TestCase: file_pattern = "coverage.*" matching_files = glob.glob(file_pattern) for file_path in matching_files: - body = read_file_as_binary_string(file_path) - clickhouse_execute( - args, - f"INSERT INTO system.coverage_log SELECT now(), '{self.case}', groupArray(data) FROM input('data UInt64') FORMAT RowBinary", - body=body, - retry_error_codes=True, - ) + try: + body = read_file_as_binary_string(file_path) + clickhouse_execute( + args, + f"INSERT INTO system.coverage_log SELECT now(), '{self.case}', groupArray(data) FROM input('data UInt64') FORMAT RowBinary", + body=body, + retry_error_codes=True, + ) + except Exception as e: + print("Cannot insert coverage data: ", str(e)) + # Remove the file even in case of exception to avoid accumulation and quadratic complexity. os.remove(file_path) coverage = clickhouse_execute( From 6b8d53a9fa54e53c766c431201ea8dfd742630ea Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 16 Jan 2024 11:07:30 +0100 Subject: [PATCH 067/276] Remove obsolete comment --- tests/clickhouse-test | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 6d398115d43..02693b997b4 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -412,7 +412,6 @@ def get_stacktraces_from_gdb(server_pid): # collect server stacktraces from system.stack_trace table -# it does not work in Sandbox def get_stacktraces_from_clickhouse(args): settings_str = " ".join( [ From e3f5058f6129badab2e0071e86f51ffb77e57ce5 Mon Sep 17 00:00:00 2001 From: MyroTk <44327070+MyroTk@users.noreply.github.com> Date: Wed, 17 Jan 2024 12:13:15 -0800 Subject: [PATCH 068/276] Update Dockerfile --- docker/test/integration/runner/Dockerfile | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index c795fbf0672..2a81db78a3d 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -94,6 +94,7 @@ RUN python3 -m pip install --no-cache-dir \ pytest-repeat \ pytest-timeout \ pytest-xdist \ + pytest-reportlog==0.4.0 \ pytz \ pyyaml==5.3.1 \ redis \ From f89803ebf65d7590e73816052b7ac2de81e04864 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Jan 2024 23:17:53 +0100 Subject: [PATCH 069/276] Slightly better --- docker/test/base/setup_export_logs.sh | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 416281c2aa3..043adf99ffc 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -156,7 +156,8 @@ function setup_logs_replication # Do not try to resolve stack traces in case of debug/sanitizers # build, since it is too slow (flushing of trace_log can take ~1min # with such MV attached) - if [[ "$debug_or_sanitizer_build" = 1 ]]; then + if [[ "$debug_or_sanitizer_build" = 1 ]] + then EXTRA_COLUMNS_EXPRESSION_FOR_TABLE="${EXTRA_COLUMNS_EXPRESSION}" else EXTRA_COLUMNS_EXPRESSION_FOR_TABLE="${EXTRA_COLUMNS_EXPRESSION_TRACE_LOG}" @@ -180,7 +181,7 @@ function setup_logs_replication # Create the destination table with adapted name and structure: statement=$(clickhouse-client --format TSVRaw --query "SHOW CREATE TABLE system.${table}" | sed -r -e ' s/^\($/('"$EXTRA_COLUMNS_FOR_TABLE"'/; - s/ORDER BY \(/ORDER BY ('"$EXTRA_ORDER_BY_COLUMNS"'/; + s/ORDER BY \(?(.+?)\)?/ORDER BY ('"$EXTRA_ORDER_BY_COLUMNS"', \1)'/; s/^CREATE TABLE system\.\w+_log$/CREATE TABLE IF NOT EXISTS '"$table"'_'"$hash"'/; /^TTL /d ') From f9ca4e3b8541d7db85effa3f9be286f7ad916965 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 18 Jan 2024 00:52:05 +0100 Subject: [PATCH 070/276] Slightly better --- docker/test/base/setup_export_logs.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 043adf99ffc..7033d4b52e2 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -181,7 +181,7 @@ function setup_logs_replication # Create the destination table with adapted name and structure: statement=$(clickhouse-client --format TSVRaw --query "SHOW CREATE TABLE system.${table}" | sed -r -e ' s/^\($/('"$EXTRA_COLUMNS_FOR_TABLE"'/; - s/ORDER BY \(?(.+?)\)?/ORDER BY ('"$EXTRA_ORDER_BY_COLUMNS"', \1)'/; + s/ORDER BY \(?(.+?)\)?/ORDER BY ('"$EXTRA_ORDER_BY_COLUMNS"', \1)/; s/^CREATE TABLE system\.\w+_log$/CREATE TABLE IF NOT EXISTS '"$table"'_'"$hash"'/; /^TTL /d ') From 61101d1a577b441931ef74b24d449b085d0f0ec3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 18 Jan 2024 01:16:50 +0100 Subject: [PATCH 071/276] Add a release build with coverage, turn off coverage in the debug build --- .github/workflows/master.yml | 8 ++++++++ .github/workflows/pull_request.yml | 8 ++++++++ tests/ci/ci_config.py | 17 +++++++++++++---- 3 files changed, 29 insertions(+), 4 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index d2865eb737d..50d3eb4a062 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -98,6 +98,14 @@ jobs: build_name: package_release checkout_depth: 0 data: ${{ needs.RunConfig.outputs.data }} + BuilderDebReleaseCoverage: + needs: [ RunConfig, BuildDockers ] + if: ${{ !failure() && !cancelled() }} + uses: ./.github/workflows/reusable_build.yml + with: + build_name: package_release_coverage + checkout_depth: 0 + data: ${{ needs.RunConfig.outputs.data }} BuilderDebAarch64: needs: [RunConfig, BuildDockers] if: ${{ !failure() && !cancelled() }} diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index bd2b2b60904..7f843f82c01 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -146,6 +146,14 @@ jobs: build_name: package_release checkout_depth: 0 data: ${{ needs.RunConfig.outputs.data }} + BuilderDebReleaseCoverage: + needs: [ RunConfig, FastTest ] + if: ${{ !failure() && !cancelled() }} + uses: ./.github/workflows/reusable_build.yml + with: + build_name: package_release_coverage + checkout_depth: 0 + data: ${{ needs.RunConfig.outputs.data }} BuilderDebAarch64: needs: [RunConfig, FastTest] if: ${{ !failure() && !cancelled() }} diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index b8dff3f0a28..1ca4e06bc8c 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -474,6 +474,12 @@ CI_CONFIG = CiConfig( name="package_debug", compiler="clang-17", debug_build=True, + package_type="deb", + sparse_checkout=True, + ), + "package_release_coverage": BuildConfig( + name="package_release_coverage", + compiler="clang-17", coverage=True, package_type="deb", sparse_checkout=True, @@ -571,6 +577,7 @@ CI_CONFIG = CiConfig( "package_tsan", "package_msan", "package_debug", + "package_release_coverage", "binary_release", "fuzzers", ] @@ -660,16 +667,15 @@ CI_CONFIG = CiConfig( "Stateful tests (release)": TestConfig( "package_release", job_config=JobConfig(**stateful_test_common_params) # type: ignore ), + "Stateful tests (coverage)": TestConfig( + "package_release_coverage", job_config=JobConfig(**stateful_test_common_params) # type: ignore + ), "Stateful tests (aarch64)": TestConfig( "package_aarch64", job_config=JobConfig(**stateful_test_common_params) # type: ignore ), "Stateful tests (release, DatabaseOrdinary)": TestConfig( "package_release", job_config=JobConfig(**stateful_test_common_params) # type: ignore ), - # "Stateful tests (release, DatabaseReplicated)": TestConfig( - # "package_release", job_config=JobConfig(**stateful_test_common_params) # type: ignore - # ), - # Stateful tests for parallel replicas "Stateful tests (release, ParallelReplicas)": TestConfig( "package_release", job_config=JobConfig(**stateful_test_common_params) # type: ignore ), @@ -712,6 +718,9 @@ CI_CONFIG = CiConfig( "Stateless tests (release)": TestConfig( "package_release", job_config=JobConfig(**statless_test_common_params) # type: ignore ), + "Stateless tests (coverage)": TestConfig( + "package_release_coverage", job_config=JobConfig(**statless_test_common_params) # type: ignore + ), "Stateless tests (aarch64)": TestConfig( "package_aarch64", job_config=JobConfig(**statless_test_common_params) # type: ignore ), From b9f8fff623448e7013bbe604b39d0f72b81032f9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 18 Jan 2024 01:36:48 +0100 Subject: [PATCH 072/276] Fix YAML --- .github/workflows/master.yml | 2 +- .github/workflows/pull_request.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 50d3eb4a062..1920f3a2a56 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -99,7 +99,7 @@ jobs: checkout_depth: 0 data: ${{ needs.RunConfig.outputs.data }} BuilderDebReleaseCoverage: - needs: [ RunConfig, BuildDockers ] + needs: [RunConfig, BuildDockers] if: ${{ !failure() && !cancelled() }} uses: ./.github/workflows/reusable_build.yml with: diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 7f843f82c01..57199e6b9d9 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -147,7 +147,7 @@ jobs: checkout_depth: 0 data: ${{ needs.RunConfig.outputs.data }} BuilderDebReleaseCoverage: - needs: [ RunConfig, FastTest ] + needs: [RunConfig, FastTest] if: ${{ !failure() && !cancelled() }} uses: ./.github/workflows/reusable_build.yml with: From cc5cc361ef561993bc7bbea6f1588562f7d3deae Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 18 Jan 2024 12:35:49 +0100 Subject: [PATCH 073/276] Fix error --- docker/packager/packager | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/packager b/docker/packager/packager index c310185b071..8efd3b8f302 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -289,7 +289,7 @@ def parse_env_variables( result.append("BUILD_TYPE=None") if coverage: - cmake_flags.append("-DSANITIZE_COVERAGE=1") + cmake_flags.append("-DSANITIZE_COVERAGE=1 -DBUILD_STANDALONE_KEEPER=0") if not cache: cmake_flags.append("-DCOMPILER_CACHE=disabled") From 6c0445f36584a60724f7d616f47c7b953621997c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 18 Jan 2024 12:47:18 +0100 Subject: [PATCH 074/276] Fix CMake --- cmake/sanitize.cmake | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index 3882b51227e..23e9cc34fec 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -63,14 +63,14 @@ endif() option(WITH_COVERAGE "Instrumentation for code coverage with default implementation" OFF) if (WITH_COVERAGE) - message (INFORMATION "Enabled instrumentation for code coverage") + message (STATUS "Enabled instrumentation for code coverage") set(COVERAGE_FLAGS "-fprofile-instr-generate -fcoverage-mapping") endif() option (SANITIZE_COVERAGE "Instrumentation for code coverage with custom callbacks" OFF) if (SANITIZE_COVERAGE) - message (INFORMATION "Enabled instrumentation for code coverage") + message (STATUS "Enabled instrumentation for code coverage") # We set this define for whole build to indicate that at least some parts are compiled with coverage. # And to expose it in system.build_options. From 6d6b8fcf8e988d78fc983ed4043ed556e36b833b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 18 Jan 2024 12:58:50 +0100 Subject: [PATCH 075/276] Add missing comments in code --- tests/ci/ci_config.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 1ca4e06bc8c..45bdfbecb0c 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -55,6 +55,13 @@ class JobConfig: run_always: bool = False +# About the "sparse_checkout" option: +# +# Misha f. Shiryaev +# :facepalm: +# we have this feature, it's used by devs, we need to test it in CI +# It's not useful for the CI itself + @dataclass class BuildConfig: name: str From db3ffa5c86dba79ca7052abe8d53799ac3e4afb9 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 18 Jan 2024 12:11:03 +0000 Subject: [PATCH 076/276] Automatic style fix --- tests/ci/ci_config.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 45bdfbecb0c..ab37659e65b 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -62,6 +62,7 @@ class JobConfig: # we have this feature, it's used by devs, we need to test it in CI # It's not useful for the CI itself + @dataclass class BuildConfig: name: str From c966674c242552584540dc2e28026894c39f9b16 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 19 Jan 2024 03:59:05 +0100 Subject: [PATCH 077/276] Disable LTO with Coverage --- CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 063cfc77302..6e984ddd864 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -348,7 +348,7 @@ if (COMPILER_CLANG) set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fdiagnostics-absolute-paths") set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fdiagnostics-absolute-paths") - if (NOT ENABLE_TESTS AND NOT SANITIZE AND OS_LINUX) + if (NOT ENABLE_TESTS AND NOT SANITIZE AND NOT SANITIZE_COVERAGE AND OS_LINUX) # https://clang.llvm.org/docs/ThinLTO.html # Applies to clang and linux only. # Disabled when building with tests or sanitizers. From c6afbe522cae20ee6041534bf7ee7e31e3acb51c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 19 Jan 2024 05:26:14 +0100 Subject: [PATCH 078/276] Do not check for large translation units with coverage --- CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 6e984ddd864..d0f44f6f3ad 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -546,7 +546,7 @@ if (ENABLE_RUST) endif() endif() -if (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" AND NOT SANITIZE AND OS_LINUX AND (ARCH_AMD64 OR ARCH_AARCH64)) +if (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" AND NOT SANITIZE AND NOT SANITIZE_COVERAGE AND OS_LINUX AND (ARCH_AMD64 OR ARCH_AARCH64)) set(CHECK_LARGE_OBJECT_SIZES_DEFAULT ON) else () set(CHECK_LARGE_OBJECT_SIZES_DEFAULT OFF) From 482229cd27c7ddf4218af2ea5d9b087e51876ab0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 19 Jan 2024 08:04:34 +0100 Subject: [PATCH 079/276] Add tests with coverage --- .github/workflows/master.yml | 16 ++++++++++++++++ .github/workflows/pull_request.yml | 16 ++++++++++++++++ 2 files changed, 32 insertions(+) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 1920f3a2a56..5f683fa6c59 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -371,6 +371,14 @@ jobs: test_name: Stateless tests (release) runner_type: func-tester data: ${{ needs.RunConfig.outputs.data }} + FunctionalStatelessTestCoverage: + needs: [RunConfig, BuilderDebReleaseCoverage] + if: ${{ !failure() && !cancelled() }} + uses: ./.github/workflows/reusable_test.yml + with: + test_name: Stateless tests (coverage) + runner_type: func-tester + data: ${{ needs.RunConfig.outputs.data }} FunctionalStatelessTestReleaseDatabaseOrdinary: needs: [RunConfig, BuilderDebRelease] if: ${{ !failure() && !cancelled() }} @@ -463,6 +471,14 @@ jobs: test_name: Stateful tests (release) runner_type: func-tester data: ${{ needs.RunConfig.outputs.data }} + FunctionalStatefulTestCoverage: + needs: [RunConfig, BuilderDebReleaseCoverage] + if: ${{ !failure() && !cancelled() }} + uses: ./.github/workflows/reusable_test.yml + with: + test_name: Stateful tests (coverage) + runner_type: func-tester + data: ${{ needs.RunConfig.outputs.data }} FunctionalStatefulTestAarch64: needs: [RunConfig, BuilderDebAarch64] if: ${{ !failure() && !cancelled() }} diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 28617695ad5..235c8042657 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -382,6 +382,14 @@ jobs: test_name: Stateless tests (release) runner_type: func-tester data: ${{ needs.RunConfig.outputs.data }} + FunctionalStatelessTestCoverage: + needs: [RunConfig, BuilderDebReleaseCoverage] + if: ${{ !failure() && !cancelled() }} + uses: ./.github/workflows/reusable_test.yml + with: + test_name: Stateless tests (coverage) + runner_type: func-tester + data: ${{ needs.RunConfig.outputs.data }} FunctionalStatelessTestReleaseDatabaseReplicated: needs: [RunConfig, BuilderDebRelease] if: ${{ !failure() && !cancelled() }} @@ -509,6 +517,14 @@ jobs: test_name: Stateful tests (release) runner_type: func-tester data: ${{ needs.RunConfig.outputs.data }} + FunctionalStatefulTestCoverage: + needs: [RunConfig, BuilderDebReleaseCoverage] + if: ${{ !failure() && !cancelled() }} + uses: ./.github/workflows/reusable_test.yml + with: + test_name: Stateful tests (coverage) + runner_type: func-tester + data: ${{ needs.RunConfig.outputs.data }} FunctionalStatefulTestAarch64: needs: [RunConfig, BuilderDebAarch64] if: ${{ !failure() && !cancelled() }} From 05609cf75d5048fbd62508fcf6454cec1855943d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 19 Jan 2024 17:02:17 +0100 Subject: [PATCH 080/276] Ci to CI --- tests/ci/ci_config.py | 6 +++--- tests/ci/test_ci_config.py | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 1d94f4fc1cc..611767be2e4 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -270,9 +270,9 @@ sql_test_params = { @dataclass -class CiConfig: +class CIConfig: """ - Contains configs for ALL jobs in CI pipeline + Contains configs for all jobs in the CI pipeline each config item in the below dicts should be an instance of JobConfig class or inherited from it """ @@ -435,7 +435,7 @@ class CiConfig: raise KeyError("config contains errors", errors) -CI_CONFIG = CiConfig( +CI_CONFIG = CIConfig( label_configs={ Labels.DO_NOT_TEST_LABEL.value: LabelConfig(run_jobs=["Style check"]), }, diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index d22ed16748e..49d49d9c328 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -3,7 +3,7 @@ import unittest -class TestCiConfig(unittest.TestCase): +class TestCIConfig(unittest.TestCase): def test_no_errors_in_ci_config(self): raised = None try: From 639d7745d450073234405d0725cbd64884d4f8c5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 19 Jan 2024 17:02:23 +0100 Subject: [PATCH 081/276] Fix error --- docker/test/base/setup_export_logs.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 7033d4b52e2..d3721108426 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -181,7 +181,7 @@ function setup_logs_replication # Create the destination table with adapted name and structure: statement=$(clickhouse-client --format TSVRaw --query "SHOW CREATE TABLE system.${table}" | sed -r -e ' s/^\($/('"$EXTRA_COLUMNS_FOR_TABLE"'/; - s/ORDER BY \(?(.+?)\)?/ORDER BY ('"$EXTRA_ORDER_BY_COLUMNS"', \1)/; + s/^ORDER BY \(?(.+?)\)?$/ORDER BY ('"$EXTRA_ORDER_BY_COLUMNS"', \1)/; s/^CREATE TABLE system\.\w+_log$/CREATE TABLE IF NOT EXISTS '"$table"'_'"$hash"'/; /^TTL /d ') From 8c54a09e6652b491764abeddf3a0e8e6800374ef Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 20 Jan 2024 08:59:29 +0100 Subject: [PATCH 082/276] Fix error --- docker/test/base/setup_export_logs.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index d3721108426..156adb1d1e4 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -17,7 +17,7 @@ CLICKHOUSE_CI_LOGS_CLUSTER=${CLICKHOUSE_CI_LOGS_CLUSTER:-system_logs_export} EXTRA_COLUMNS=${EXTRA_COLUMNS:-"pull_request_number UInt32, commit_sha String, check_start_time DateTime('UTC'), check_name LowCardinality(String), instance_type LowCardinality(String), instance_id String, INDEX ix_pr (pull_request_number) TYPE set(100), INDEX ix_commit (commit_sha) TYPE set(100), INDEX ix_check_time (check_start_time) TYPE minmax, "} EXTRA_COLUMNS_EXPRESSION=${EXTRA_COLUMNS_EXPRESSION:-"CAST(0 AS UInt32) AS pull_request_number, '' AS commit_sha, now() AS check_start_time, toLowCardinality('') AS check_name, toLowCardinality('') AS instance_type, '' AS instance_id"} -EXTRA_ORDER_BY_COLUMNS=${EXTRA_ORDER_BY_COLUMNS:-"check_name, "} +EXTRA_ORDER_BY_COLUMNS=${EXTRA_ORDER_BY_COLUMNS:-"check_name"} # trace_log needs more columns for symbolization EXTRA_COLUMNS_TRACE_LOG="${EXTRA_COLUMNS} symbols Array(LowCardinality(String)), lines Array(LowCardinality(String)), " @@ -181,7 +181,7 @@ function setup_logs_replication # Create the destination table with adapted name and structure: statement=$(clickhouse-client --format TSVRaw --query "SHOW CREATE TABLE system.${table}" | sed -r -e ' s/^\($/('"$EXTRA_COLUMNS_FOR_TABLE"'/; - s/^ORDER BY \(?(.+?)\)?$/ORDER BY ('"$EXTRA_ORDER_BY_COLUMNS"', \1)/; + s/^ORDER BY (([^\(].+?)|\((.+?)\))$/ORDER BY ('"$EXTRA_ORDER_BY_COLUMNS"', \2\3)/; s/^CREATE TABLE system\.\w+_log$/CREATE TABLE IF NOT EXISTS '"$table"'_'"$hash"'/; /^TTL /d ') From cfe60586c007a230df68771b3f914d9a66414b7d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 20 Jan 2024 21:45:11 +0100 Subject: [PATCH 083/276] Reset coverage after each test --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index b62bd5975ea..49c517852a6 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -2894,7 +2894,7 @@ def parse_args(): parser.add_argument( "--reset-coverage-before-every-test", action="store_true", - default=False, + default=True, help="Collect isolated test coverage for every test instead of a cumulative. Useful only when tests are run sequentially.", ) parser.add_argument( From 51cc01f8be8fea1fcaea0af9c85ca2930536e593 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jan 2024 14:36:03 +0100 Subject: [PATCH 084/276] Minor change --- base/base/coverage.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/base/base/coverage.cpp b/base/base/coverage.cpp index 499e384d21f..05bef21049b 100644 --- a/base/base/coverage.cpp +++ b/base/base/coverage.cpp @@ -62,6 +62,7 @@ namespace uintptr_t * allocate(size_t size) { + /// Note: mmap return zero-initialized memory, and we count on that. void * map = mmap(nullptr, size, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0); if (MAP_FAILED == map) return nullptr; @@ -91,8 +92,6 @@ void __sanitizer_cov_trace_pc_guard_init(uint32_t * start, uint32_t * stop) /// Note: we will leak this. current_coverage_array = allocate(sizeof(uintptr_t) * coverage_array_size); cumulative_coverage_array = allocate(sizeof(uintptr_t) * coverage_array_size); - - resetCoverage(); } /// This is called at least once for every DSO for initialization From b967cc6af9deac20eff318e3433fc5b09fd6314a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jan 2024 15:30:50 +0100 Subject: [PATCH 085/276] Fix error --- base/base/coverage.cpp | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/base/base/coverage.cpp b/base/base/coverage.cpp index 05bef21049b..b85f1a16d32 100644 --- a/base/base/coverage.cpp +++ b/base/base/coverage.cpp @@ -92,6 +92,8 @@ void __sanitizer_cov_trace_pc_guard_init(uint32_t * start, uint32_t * stop) /// Note: we will leak this. current_coverage_array = allocate(sizeof(uintptr_t) * coverage_array_size); cumulative_coverage_array = allocate(sizeof(uintptr_t) * coverage_array_size); + + resetCoverage(); } /// This is called at least once for every DSO for initialization @@ -102,8 +104,8 @@ void __sanitizer_cov_pcs_init(const uintptr_t * pcs_begin, const uintptr_t * pcs return; pc_table_initialized = true; - all_addresses_array = allocate(sizeof(uintptr_t) * coverage_array_size); all_addresses_array_size = pcs_end - pcs_begin; + all_addresses_array = allocate(sizeof(uintptr_t) * all_addresses_array_size); /// They are not a real pointers, but also contain a flag in the most significant bit, /// in which we are not interested for now. Reset it. @@ -125,10 +127,10 @@ void __sanitizer_cov_trace_pc_guard(uint32_t * guard) /// The values of `*guard` are as you set them in /// __sanitizer_cov_trace_pc_guard_init and so you can make them consecutive /// and use them to dereference an array or a bit vector. - void * pc = __builtin_return_address(0); + intptr_t pc = reinterpret_cast(__builtin_return_address(0)); - current_coverage_array[guard - guards_start] = reinterpret_cast(pc); - cumulative_coverage_array[guard - guards_start] = reinterpret_cast(pc); + current_coverage_array[guard - guards_start] = pc; + cumulative_coverage_array[guard - guards_start] = pc; } } From 2e7ce5b0e208c91874d44eb0c828a1e01544a387 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 22 Jan 2024 16:24:43 +0100 Subject: [PATCH 086/276] Updated settings ptr and fetching of client from Disk & ObjectStorage --- src/Backups/BackupIO_AzureBlobStorage.cpp | 32 ++++++++----------- src/Backups/BackupIO_AzureBlobStorage.h | 4 +-- .../AzureBlobStorage/AzureObjectStorage.h | 7 +++- .../Cached/CachedObjectStorage.h | 8 +++++ src/Disks/ObjectStorages/IObjectStorage.h | 13 ++++++++ .../copyAzureBlobStorageFile.cpp | 22 ++++++------- .../copyAzureBlobStorageFile.h | 4 +-- src/Storages/StorageAzureBlob.cpp | 2 +- 8 files changed, 57 insertions(+), 35 deletions(-) diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index 8c6c1040eec..fca324869ae 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -37,13 +37,12 @@ BackupReaderAzureBlobStorage::BackupReaderAzureBlobStorage( , configuration(configuration_) { auto client_ptr = StorageAzureBlob::createClient(configuration, /* is_read_only */ false); - settings = StorageAzureBlob::createSettingsAsSharedPtr(context_); - auto settings_as_unique_ptr = StorageAzureBlob::createSettings(context_); object_storage = std::make_unique("BackupReaderAzureBlobStorage", std::move(client_ptr), - std::move(settings_as_unique_ptr), + StorageAzureBlob::createSettings(context_), configuration_.container); - client = object_storage->getClient(); + client = object_storage->getAzureBlobStorageClient(); + settings = object_storage->getSettings(); } BackupReaderAzureBlobStorage::~BackupReaderAzureBlobStorage() = default; @@ -89,8 +88,8 @@ std::unique_ptr BackupReaderAzureBlobStorage::readFile(const key = file_name; } return std::make_unique( - client.get(), key, read_settings, settings->max_single_read_retries, - settings->max_single_download_retries); + client.get(), key, read_settings, settings.get()->max_single_read_retries, + settings.get()->max_single_download_retries); } void BackupReaderAzureBlobStorage::copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, @@ -98,10 +97,8 @@ void BackupReaderAzureBlobStorage::copyFileToDisk(const String & path_in_backup, { LOG_INFO(&Poco::Logger::get("BackupReaderAzureBlobStorage"), "Enter copyFileToDisk"); - /// Use the native copy as a more optimal way to copy a file from AzureBlobStorage to AzureBlobStorage if it's possible. - /// We don't check for `has_throttling` here because the native copy almost doesn't use network. auto destination_data_source_description = destination_disk->getDataSourceDescription(); - if (destination_data_source_description.sameKind(data_source_description) + if ((destination_data_source_description.type == DataSourceType::AzureBlobStorage) && (destination_data_source_description.is_encrypted == encrypted_in_backup)) { LOG_TRACE(log, "Copying {} from AzureBlobStorage to disk {}", path_in_backup, destination_disk->getName()); @@ -115,7 +112,7 @@ void BackupReaderAzureBlobStorage::copyFileToDisk(const String & path_in_backup, copyAzureBlobStorageFile( client, - reinterpret_cast(destination_disk->getObjectStorage().get())->getClient(), + destination_disk->getObjectStorage()->getAzureBlobStorageClient(), configuration.container, fs::path(configuration.blob_path) / path_in_backup, 0, @@ -150,13 +147,12 @@ BackupWriterAzureBlobStorage::BackupWriterAzureBlobStorage( , configuration(configuration_) { auto client_ptr = StorageAzureBlob::createClient(configuration, /* is_read_only */ false); - settings = StorageAzureBlob::createSettingsAsSharedPtr(context_); - auto settings_as_unique_ptr = StorageAzureBlob::createSettings(context_); object_storage = std::make_unique("BackupWriterAzureBlobStorage", std::move(client_ptr), - std::move(settings_as_unique_ptr), + StorageAzureBlob::createSettings(context_), configuration_.container); - client = object_storage->getClient(); + client = object_storage->getAzureBlobStorageClient(); + settings = object_storage->getSettings(); } void BackupWriterAzureBlobStorage::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, @@ -172,7 +168,7 @@ void BackupWriterAzureBlobStorage::copyFileFromDisk(const String & path_in_backu { LOG_TRACE(log, "Copying file {} from disk {} to AzureBlobStorag", src_path, src_disk->getName()); copyAzureBlobStorageFile( - reinterpret_cast(src_disk->getObjectStorage().get())->getClient(), + src_disk->getObjectStorage()->getAzureBlobStorageClient(), client, /* src_container */ blob_path[1], /* src_path */ blob_path[0], @@ -267,8 +263,8 @@ std::unique_ptr BackupWriterAzureBlobStorage::readFile(const String } return std::make_unique( - client.get(), key, read_settings, settings->max_single_read_retries, - settings->max_single_download_retries); + client.get(), key, read_settings, settings.get()->max_single_read_retries, + settings.get()->max_single_download_retries); } std::unique_ptr BackupWriterAzureBlobStorage::writeFile(const String & file_name) @@ -285,7 +281,7 @@ std::unique_ptr BackupWriterAzureBlobStorage::writeFile(const Strin return std::make_unique( client.get(), key, - settings->max_single_part_upload_size, + settings.get()->max_single_part_upload_size, DBMS_DEFAULT_BUFFER_SIZE, write_settings); } diff --git a/src/Backups/BackupIO_AzureBlobStorage.h b/src/Backups/BackupIO_AzureBlobStorage.h index 12bf073cd08..87dc470cdb3 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.h +++ b/src/Backups/BackupIO_AzureBlobStorage.h @@ -31,7 +31,7 @@ private: MultiVersion client; StorageAzureBlob::Configuration configuration; std::unique_ptr object_storage; - std::shared_ptr settings; + MultiVersion settings; }; class BackupWriterAzureBlobStorage : public BackupWriterDefault @@ -60,7 +60,7 @@ private: MultiVersion client; StorageAzureBlob::Configuration configuration; std::unique_ptr object_storage; - std::shared_ptr settings; + MultiVersion settings; }; } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 52d535054ff..a9d082539e6 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -139,7 +139,12 @@ public: bool isRemote() const override { return true; } - MultiVersion & getClient() { return client; } + MultiVersion & getSettings() { return settings; } + + MultiVersion & getAzureBlobStorageClient() override + { + return client; + } private: const String name; diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index 4c185db051d..6b0ff8be58a 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -3,6 +3,7 @@ #include #include #include +#include "config.h" namespace Poco { @@ -118,6 +119,13 @@ public: static bool canUseReadThroughCache(const ReadSettings & settings); +#if USE_AZURE_BLOB_STORAGE + MultiVersion & getAzureBlobStorageClient() override + { + return object_storage->getAzureBlobStorageClient(); + } +#endif + private: FileCache::Key getCacheKey(const std::string & path) const; diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index f405be72287..cf113586ddf 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -23,7 +23,12 @@ #include #include #include +#include "config.h" +#if USE_AZURE_BLOB_STORAGE +#include +#include +#endif namespace DB { @@ -212,6 +217,14 @@ public: virtual WriteSettings patchSettings(const WriteSettings & write_settings) const; +#if USE_AZURE_BLOB_STORAGE + virtual MultiVersion & getAzureBlobStorageClient() + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "This function is only implemented for AzureBlobStorage"); + } +#endif + + private: mutable std::mutex throttlers_mutex; ThrottlerPtr remote_read_throttler; diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp index 272be914cc1..bb8702e9b41 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp @@ -49,7 +49,7 @@ namespace size_t total_size_, const String & dest_container_, const String & dest_blob_, - std::shared_ptr settings_, + MultiVersion settings_, const std::optional> & object_metadata_, ThreadPoolCallbackRunner schedule_, bool for_disk_azure_blob_storage_, @@ -65,7 +65,7 @@ namespace , schedule(schedule_) , for_disk_azure_blob_storage(for_disk_azure_blob_storage_) , log(log_) - , max_single_part_upload_size(settings_->max_single_part_upload_size) + , max_single_part_upload_size(settings_.get()->max_single_part_upload_size) { } @@ -78,7 +78,7 @@ namespace size_t total_size; const String & dest_container; const String & dest_blob; - std::shared_ptr settings; + MultiVersion settings; const std::optional> & object_metadata; ThreadPoolCallbackRunner schedule; bool for_disk_azure_blob_storage; @@ -114,9 +114,9 @@ namespace if (!total_size) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chosen multipart upload for an empty file. This must not happen"); - auto max_part_number = settings->max_part_number; - auto min_upload_part_size = settings->min_upload_part_size; - auto max_upload_part_size = settings->max_upload_part_size; + auto max_part_number = settings.get()->max_part_number; + auto min_upload_part_size = settings.get()->min_upload_part_size; + auto max_upload_part_size = settings.get()->max_upload_part_size; if (!max_part_number) throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "max_part_number must not be 0"); @@ -333,7 +333,7 @@ void copyDataToAzureBlobStorageFile( MultiVersion & dest_client, const String & dest_container, const String & dest_blob, - std::shared_ptr settings, + MultiVersion settings, const std::optional> & object_metadata, ThreadPoolCallbackRunner schedule, bool for_disk_azure_blob_storage) @@ -352,14 +352,14 @@ void copyAzureBlobStorageFile( size_t size, const String & dest_container, const String & dest_blob, - std::shared_ptr settings, + MultiVersion settings, const ReadSettings & read_settings, const std::optional> & object_metadata, ThreadPoolCallbackRunner schedule, bool for_disk_azure_blob_storage) { - if (settings->use_native_copy) + if (settings.get()->use_native_copy) { ProfileEvents::increment(ProfileEvents::AzureCopyObject); if (for_disk_azure_blob_storage) @@ -393,8 +393,8 @@ void copyAzureBlobStorageFile( LOG_TRACE(&Poco::Logger::get("copyAzureBlobStorageFile"), "Reading from Container: {}, Blob: {}", src_container, src_blob); auto create_read_buffer = [&] { - return std::make_unique(src_client.get(), src_blob, read_settings, settings->max_single_read_retries, - settings->max_single_download_retries); + return std::make_unique(src_client.get(), src_blob, read_settings, settings.get()->max_single_read_retries, + settings.get()->max_single_download_retries); }; UploadHelper helper{create_read_buffer, dest_client, offset, size, dest_container, dest_blob, settings, object_metadata, schedule, for_disk_azure_blob_storage, &Poco::Logger::get("copyAzureBlobStorageFile")}; diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h index b022151d32d..491f7cd7176 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h @@ -29,7 +29,7 @@ void copyAzureBlobStorageFile( size_t src_size, const String & dest_container, const String & dest_blob, - std::shared_ptr settings, + MultiVersion settings, const ReadSettings & read_settings, const std::optional> & object_metadata = std::nullopt, ThreadPoolCallbackRunner schedule_ = {}, @@ -48,7 +48,7 @@ void copyDataToAzureBlobStorageFile( MultiVersion & client, const String & dest_container, const String & dest_blob, - std::shared_ptr settings, + MultiVersion settings, const std::optional> & object_metadata = std::nullopt, ThreadPoolCallbackRunner schedule_ = {}, bool for_disk_azure_blob_storage = false); diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 7a40d2dcb73..e54838c7a61 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -1214,7 +1214,7 @@ StorageAzureBlobSource::ReaderHolder StorageAzureBlobSource::createReader() QueryPipelineBuilder builder; std::shared_ptr source; std::unique_ptr read_buf; - std::optional num_rows_from_cache = need_only_count && getContext()->getSettingsRef().use_cache_for_count_from_files + std::optional num_rows_from_cache = need_only_count && getContext()->getSettingsRef().use_cache_for_count_from_files ? tryGetNumRowsFromCache(path_with_metadata) : std::nullopt; if (num_rows_from_cache) { From e78eb41264ebb37d3fd813850a3e55ce7690ecea Mon Sep 17 00:00:00 2001 From: MyroTk <44327070+MyroTk@users.noreply.github.com> Date: Mon, 22 Jan 2024 15:19:31 -0800 Subject: [PATCH 087/276] Update Dockerfile --- docker/test/integration/runner/Dockerfile | 57 +++++++++++------------ 1 file changed, 28 insertions(+), 29 deletions(-) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index 2a81db78a3d..dbf90f9b810 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -63,47 +63,46 @@ RUN curl -fsSL https://download.docker.com/linux/ubuntu/gpg | apt-key add - \ RUN python3 -m pip install --no-cache-dir \ - PyMySQL \ - aerospike==11.1.0 \ - asyncio \ + PyMySQL==1.1.0 \ + asyncio==3.4.3 \ avro==1.10.2 \ - azure-storage-blob \ - boto3 \ - cassandra-driver \ + azure-storage-blob==12.19.0 \ + boto3==1.34.24 \ + cassandra-driver==3.29.0 \ confluent-kafka==1.9.2 \ delta-spark==2.3.0 \ - dict2xml \ - dicttoxml \ + dict2xml==1.7.4 \ + dicttoxml==1.7.16 \ docker==6.1.3 \ docker-compose==1.29.2 \ - grpcio \ - grpcio-tools \ - kafka-python \ - kazoo \ - lz4 \ - minio \ - nats-py \ - protobuf \ + grpcio==1.60.0 \ + grpcio-tools==1.60.0 \ + kafka-python==2.0.2 \ + kazoo==2.9.0 \ + lz4==4.3.3 \ + minio==7.2.3 \ + nats-py==2.6.0 \ + protobuf==4.25.2 \ psycopg2-binary==2.9.6 \ - pyhdfs \ + pyhdfs==0.3.1 \ pymongo==3.11.0 \ pyspark==3.3.2 \ - pytest \ + pytest==7.4.4 \ pytest-order==1.0.0 \ - pytest-random \ - pytest-repeat \ - pytest-timeout \ - pytest-xdist \ + pytest-random==0.2 \ + pytest-repeat==0.9.3 \ + pytest-timeout==2.2.0 \ + pytest-xdist==3.5.0 \ pytest-reportlog==0.4.0 \ - pytz \ + pytz==2023.3.post1 \ pyyaml==5.3.1 \ - redis \ - requests-kerberos \ + redis==5.0.1 \ + requests-kerberos==0.14.0 \ tzlocal==2.1 \ - retry \ - bs4 \ - lxml \ - urllib3 + retry==0.9.2 \ + bs4==0.0.2 \ + lxml==5.1.0 \ + urllib3==2.0.7 # bs4, lxml are for cloud tests, do not delete # Hudi supports only spark 3.3.*, not 3.4 From 992d859e726895dadc9fbab1ebf99acd4b29881c Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 23 Jan 2024 14:16:14 +0100 Subject: [PATCH 088/276] Fix style check --- src/Disks/ObjectStorages/IObjectStorage.h | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index cf113586ddf..b7db353fb6a 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -23,6 +23,7 @@ #include #include #include +#include #include "config.h" #if USE_AZURE_BLOB_STORAGE @@ -33,6 +34,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + class ReadBufferFromFileBase; class WriteBufferFromFileBase; From 8e0aea301ee4b416d6bb4bcfdf664756ebff55ec Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 23 Jan 2024 14:29:26 +0000 Subject: [PATCH 089/276] Analyzer: Add cast for ConstantNode from constant folding --- src/Analyzer/ConstantNode.cpp | 5 ++++- tests/analyzer_tech_debt.txt | 1 - 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Analyzer/ConstantNode.cpp b/src/Analyzer/ConstantNode.cpp index cb05e6ed4e3..69bed3dbe90 100644 --- a/src/Analyzer/ConstantNode.cpp +++ b/src/Analyzer/ConstantNode.cpp @@ -128,7 +128,10 @@ ASTPtr ConstantNode::toASTImpl(const ConvertToASTOptions & options) const } } - if (need_to_add_cast_function) + // Add cast if constant was created as a result of constant folding. + // Constant folding may lead to type transformation and literal on shard + // may have a different type. + if (need_to_add_cast_function || source_expression != nullptr) { auto constant_type_name_ast = std::make_shared(constant_value->getType()->getName()); return makeASTFunction("_CAST", std::move(constant_value_ast), std::move(constant_type_name_ast)); diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 4643d109c3d..dd747fff7df 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -6,7 +6,6 @@ 01155_rename_move_materialized_view 01214_test_storage_merge_aliases_with_where 01244_optimize_distributed_group_by_sharding_key -01268_shard_avgweighted 01495_subqueries_in_with_statement 01560_merge_distributed_join 01584_distributed_buffer_cannot_find_column From 849858017237d9752f3efb801bcc2267288cb8c8 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 24 Jan 2024 10:01:06 +0100 Subject: [PATCH 090/276] Fixing build --- src/Backups/BackupIO_AzureBlobStorage.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index fca324869ae..34be110cd42 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include @@ -143,7 +144,7 @@ BackupWriterAzureBlobStorage::BackupWriterAzureBlobStorage( const WriteSettings & write_settings_, const ContextPtr & context_) : BackupWriterDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupWriterAzureBlobStorage")) - , data_source_description{DataSourceType::AzureBlobStorage,configuration_.container, false, false} + , data_source_description{DataSourceType::AzureBlobStorage, configuration_.container, false, false} , configuration(configuration_) { auto client_ptr = StorageAzureBlob::createClient(configuration, /* is_read_only */ false); From 788eb487075fe770097759edfd46544134e11116 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 24 Jan 2024 11:51:02 +0100 Subject: [PATCH 091/276] Fix build after merging master --- src/Backups/BackupIO_AzureBlobStorage.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index 34be110cd42..2c2396e9c0a 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -34,7 +34,7 @@ BackupReaderAzureBlobStorage::BackupReaderAzureBlobStorage( const WriteSettings & write_settings_, const ContextPtr & context_) : BackupReaderDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupReaderAzureBlobStorage")) - , data_source_description{DataSourceType::AzureBlobStorage, configuration_.container, false, false} + , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, configuration_.container, false, false} , configuration(configuration_) { auto client_ptr = StorageAzureBlob::createClient(configuration, /* is_read_only */ false); @@ -99,7 +99,8 @@ void BackupReaderAzureBlobStorage::copyFileToDisk(const String & path_in_backup, LOG_INFO(&Poco::Logger::get("BackupReaderAzureBlobStorage"), "Enter copyFileToDisk"); auto destination_data_source_description = destination_disk->getDataSourceDescription(); - if ((destination_data_source_description.type == DataSourceType::AzureBlobStorage) + if ((destination_data_source_description.type == DataSourceType::ObjectStorage) + && (destination_data_source_description.object_storage_type == ObjectStorageType::Azure) && (destination_data_source_description.is_encrypted == encrypted_in_backup)) { LOG_TRACE(log, "Copying {} from AzureBlobStorage to disk {}", path_in_backup, destination_disk->getName()); @@ -144,7 +145,7 @@ BackupWriterAzureBlobStorage::BackupWriterAzureBlobStorage( const WriteSettings & write_settings_, const ContextPtr & context_) : BackupWriterDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupWriterAzureBlobStorage")) - , data_source_description{DataSourceType::AzureBlobStorage, configuration_.container, false, false} + , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, configuration_.container, false, false} , configuration(configuration_) { auto client_ptr = StorageAzureBlob::createClient(configuration, /* is_read_only */ false); From f551081dd4c38ac014f554c7ee4efc4e18777f9a Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 24 Jan 2024 21:10:50 +0100 Subject: [PATCH 092/276] Addressed review comments --- src/Backups/BackupIO_AzureBlobStorage.cpp | 7 ++--- .../copyAzureBlobStorageFile.cpp | 27 ++++++++----------- .../copyAzureBlobStorageFile.h | 8 +++--- 3 files changed, 16 insertions(+), 26 deletions(-) diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index 2c2396e9c0a..1b4c10ad0cb 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -104,7 +104,7 @@ void BackupReaderAzureBlobStorage::copyFileToDisk(const String & path_in_backup, && (destination_data_source_description.is_encrypted == encrypted_in_backup)) { LOG_TRACE(log, "Copying {} from AzureBlobStorage to disk {}", path_in_backup, destination_disk->getName()); - auto write_blob_function = [&](const Strings & blob_path, WriteMode mode, const std::optional & object_attributes) -> size_t + auto write_blob_function = [&](const Strings & blob_path, WriteMode mode, const std::optional &) -> size_t { /// Object storage always uses mode `Rewrite` because it simulates append using metadata and different files. if (blob_path.size() != 2 || mode != WriteMode::Rewrite) @@ -123,7 +123,6 @@ void BackupReaderAzureBlobStorage::copyFileToDisk(const String & path_in_backup, /* dest_path */ blob_path[0], settings, read_settings, - object_attributes, threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupRDAzure"), /* for_disk_azure_blob_storage= */ true); @@ -180,7 +179,6 @@ void BackupWriterAzureBlobStorage::copyFileFromDisk(const String & path_in_backu fs::path(configuration.blob_path) / path_in_backup, settings, read_settings, - {}, threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupWRAzure")); return; /// copied! } @@ -204,14 +202,13 @@ void BackupWriterAzureBlobStorage::copyFile(const String & destination, const St /* dest_path */ destination, settings, read_settings, - {}, threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupWRAzure"), /* for_disk_azure_blob_storage= */ true); } void BackupWriterAzureBlobStorage::copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) { - copyDataToAzureBlobStorageFile(create_read_buffer, start_pos, length, client, configuration.container, path_in_backup, settings, {}, + copyDataToAzureBlobStorageFile(create_read_buffer, start_pos, length, client, configuration.container, path_in_backup, settings, threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupWRAzure")); } diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp index bb8702e9b41..350d2d1d34e 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp @@ -47,10 +47,9 @@ namespace MultiVersion & client_, size_t offset_, size_t total_size_, - const String & dest_container_, + const String & dest_container_for_logging_, const String & dest_blob_, MultiVersion settings_, - const std::optional> & object_metadata_, ThreadPoolCallbackRunner schedule_, bool for_disk_azure_blob_storage_, const Poco::Logger * log_) @@ -58,10 +57,9 @@ namespace , client(client_) , offset (offset_) , total_size (total_size_) - , dest_container(dest_container_) + , dest_container_for_logging(dest_container_for_logging_) , dest_blob(dest_blob_) , settings(settings_) - , object_metadata(object_metadata_) , schedule(schedule_) , for_disk_azure_blob_storage(for_disk_azure_blob_storage_) , log(log_) @@ -76,10 +74,9 @@ namespace MultiVersion & client; size_t offset; size_t total_size; - const String & dest_container; + const String & dest_container_for_logging; const String & dest_blob; MultiVersion settings; - const std::optional> & object_metadata; ThreadPoolCallbackRunner schedule; bool for_disk_azure_blob_storage; const Poco::Logger * log; @@ -208,7 +205,7 @@ namespace void uploadPart(size_t part_offset, size_t part_size) { - LOG_TRACE(log, "Writing part. Container: {}, Blob: {}, Size: {}", dest_container, dest_blob, part_size); + LOG_TRACE(log, "Writing part. Container: {}, Blob: {}, Size: {}", dest_container_for_logging, dest_blob, part_size); if (!part_size) { @@ -287,7 +284,7 @@ namespace std::lock_guard lock(bg_tasks_mutex); /// Protect bg_tasks from race task.block_id = block_id; - LOG_TRACE(log, "Writing part finished. Container: {}, Blob: {}, block_id: {}, Parts: {}", dest_container, dest_blob, block_id, bg_tasks.size()); + LOG_TRACE(log, "Writing part finished. Container: {}, Blob: {}, block_id: {}, Parts: {}", dest_container_for_logging, dest_blob, block_id, bg_tasks.size()); } String processUploadPartRequest(UploadPartTask & task) @@ -331,14 +328,13 @@ void copyDataToAzureBlobStorageFile( size_t offset, size_t size, MultiVersion & dest_client, - const String & dest_container, + const String & dest_container_for_logging, const String & dest_blob, MultiVersion settings, - const std::optional> & object_metadata, ThreadPoolCallbackRunner schedule, bool for_disk_azure_blob_storage) { - UploadHelper helper{create_read_buffer, dest_client, offset, size, dest_container, dest_blob, settings, object_metadata, schedule, for_disk_azure_blob_storage, &Poco::Logger::get("copyDataToAzureBlobStorageFile")}; + UploadHelper helper{create_read_buffer, dest_client, offset, size, dest_container_for_logging, dest_blob, settings, schedule, for_disk_azure_blob_storage, &Poco::Logger::get("copyDataToAzureBlobStorageFile")}; helper.performCopy(); } @@ -346,15 +342,14 @@ void copyDataToAzureBlobStorageFile( void copyAzureBlobStorageFile( MultiVersion & src_client, MultiVersion & dest_client, - const String & src_container, + const String & src_container_for_logging, const String & src_blob, size_t offset, size_t size, - const String & dest_container, + const String & dest_container_for_logging, const String & dest_blob, MultiVersion settings, const ReadSettings & read_settings, - const std::optional> & object_metadata, ThreadPoolCallbackRunner schedule, bool for_disk_azure_blob_storage) { @@ -390,14 +385,14 @@ void copyAzureBlobStorageFile( } else { - LOG_TRACE(&Poco::Logger::get("copyAzureBlobStorageFile"), "Reading from Container: {}, Blob: {}", src_container, src_blob); + LOG_TRACE(&Poco::Logger::get("copyAzureBlobStorageFile"), "Reading from Container: {}, Blob: {}", src_container_for_logging, src_blob); auto create_read_buffer = [&] { return std::make_unique(src_client.get(), src_blob, read_settings, settings.get()->max_single_read_retries, settings.get()->max_single_download_retries); }; - UploadHelper helper{create_read_buffer, dest_client, offset, size, dest_container, dest_blob, settings, object_metadata, schedule, for_disk_azure_blob_storage, &Poco::Logger::get("copyAzureBlobStorageFile")}; + UploadHelper helper{create_read_buffer, dest_client, offset, size, dest_container_for_logging, dest_blob, settings, schedule, for_disk_azure_blob_storage, &Poco::Logger::get("copyAzureBlobStorageFile")}; helper.performCopy(); } } diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h index 491f7cd7176..15a31031f63 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h @@ -23,15 +23,14 @@ using CreateReadBuffer = std::function()>; void copyAzureBlobStorageFile( MultiVersion & src_client, MultiVersion & dest_client, - const String & src_container, + const String & src_container_for_logging, const String & src_blob, size_t src_offset, size_t src_size, - const String & dest_container, + const String & dest_container_for_logging, const String & dest_blob, MultiVersion settings, const ReadSettings & read_settings, - const std::optional> & object_metadata = std::nullopt, ThreadPoolCallbackRunner schedule_ = {}, bool for_disk_azure_blob_storage = false); @@ -46,10 +45,9 @@ void copyDataToAzureBlobStorageFile( size_t offset, size_t size, MultiVersion & client, - const String & dest_container, + const String & dest_container_for_logging, const String & dest_blob, MultiVersion settings, - const std::optional> & object_metadata = std::nullopt, ThreadPoolCallbackRunner schedule_ = {}, bool for_disk_azure_blob_storage = false); From 3e3ae52acaedc57b9470f5b59c45307a7e048068 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 24 Jan 2024 14:35:35 +0000 Subject: [PATCH 093/276] Reduce even more memory --- programs/keeper-converter/KeeperConverter.cpp | 42 ++-- programs/keeper/CMakeLists.txt | 2 +- src/Coordination/KeeperSnapshotManager.cpp | 82 ++++--- src/Coordination/KeeperStorage.cpp | 223 +++++++++++------- src/Coordination/KeeperStorage.h | 153 ++++++++++-- src/Coordination/SnapshotableHashTable.h | 34 +-- src/Coordination/ZooKeeperDataReader.cpp | 2 +- 7 files changed, 353 insertions(+), 185 deletions(-) diff --git a/programs/keeper-converter/KeeperConverter.cpp b/programs/keeper-converter/KeeperConverter.cpp index 20448aafa2f..99f8bab3403 100644 --- a/programs/keeper-converter/KeeperConverter.cpp +++ b/programs/keeper-converter/KeeperConverter.cpp @@ -38,31 +38,31 @@ int mainEntryClickHouseKeeperConverter(int argc, char ** argv) return 0; } - try - { - auto keeper_context = std::make_shared(true); - keeper_context->setDigestEnabled(true); - keeper_context->setSnapshotDisk(std::make_shared("Keeper-snapshots", options["output-dir"].as())); + //try + //{ + // auto keeper_context = std::make_shared(true); + // keeper_context->setDigestEnabled(true); + // keeper_context->setSnapshotDisk(std::make_shared("Keeper-snapshots", options["output-dir"].as())); - DB::KeeperStorage storage(/* tick_time_ms */ 500, /* superdigest */ "", keeper_context, /* initialize_system_nodes */ false); + // DB::KeeperStorage storage(/* tick_time_ms */ 500, /* superdigest */ "", keeper_context, /* initialize_system_nodes */ false); - DB::deserializeKeeperStorageFromSnapshotsDir(storage, options["zookeeper-snapshots-dir"].as(), logger); - storage.initializeSystemNodes(); + // DB::deserializeKeeperStorageFromSnapshotsDir(storage, options["zookeeper-snapshots-dir"].as(), logger); + // storage.initializeSystemNodes(); - DB::deserializeLogsAndApplyToStorage(storage, options["zookeeper-logs-dir"].as(), logger); - DB::SnapshotMetadataPtr snapshot_meta = std::make_shared(storage.getZXID(), 1, std::make_shared()); - DB::KeeperStorageSnapshot snapshot(&storage, snapshot_meta); + // DB::deserializeLogsAndApplyToStorage(storage, options["zookeeper-logs-dir"].as(), logger); + // DB::SnapshotMetadataPtr snapshot_meta = std::make_shared(storage.getZXID(), 1, std::make_shared()); + // DB::KeeperStorageSnapshot snapshot(&storage, snapshot_meta); - DB::KeeperSnapshotManager manager(1, keeper_context); - auto snp = manager.serializeSnapshotToBuffer(snapshot); - auto file_info = manager.serializeSnapshotBufferToDisk(*snp, storage.getZXID()); - std::cout << "Snapshot serialized to path:" << fs::path(file_info.disk->getPath()) / file_info.path << std::endl; - } - catch (...) - { - std::cerr << getCurrentExceptionMessage(true) << '\n'; - return getCurrentExceptionCode(); - } + // DB::KeeperSnapshotManager manager(1, keeper_context); + // auto snp = manager.serializeSnapshotToBuffer(snapshot); + // auto file_info = manager.serializeSnapshotBufferToDisk(*snp, storage.getZXID()); + // std::cout << "Snapshot serialized to path:" << fs::path(file_info.disk->getPath()) / file_info.path << std::endl; + //} + //catch (...) + //{ + // std::cerr << getCurrentExceptionMessage(true) << '\n'; + // return getCurrentExceptionCode(); + //} return 0; } diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 143ded0ee85..fba9b3e4d86 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -44,7 +44,7 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/SessionExpiryQueue.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/SummingStateMachine.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/WriteBufferFromNuraftBuffer.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/ZooKeeperDataReader.cpp + #${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/ZooKeeperDataReader.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Core/SettingsFields.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Core/BaseSettings.cpp diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index ee5935015e4..2f51e855763 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -79,20 +79,21 @@ namespace writeBinary(false, out); /// Serialize stat - writeBinary(node.stat.czxid, out); - writeBinary(node.stat.mzxid, out); - writeBinary(node.stat.ctime, out); - writeBinary(node.stat.mtime, out); - writeBinary(node.stat.version, out); - writeBinary(node.stat.cversion, out); - writeBinary(node.stat.aversion, out); - writeBinary(node.stat.ephemeralOwner, out); + writeBinary(node.czxid, out); + writeBinary(node.mzxid, out); + writeBinary(node.ctime(), out); + writeBinary(node.mtime(), out); + writeBinary(node.version, out); + writeBinary(node.cversion, out); + writeBinary(node.aversion, out); + const bool is_ephemeral = node.isEphemeral(); + writeBinary(is_ephemeral ? node.ephemeralOwner() : 0, out); if (version < SnapshotVersion::V6) - writeBinary(static_cast(node.getData().size()), out); - writeBinary(node.stat.numChildren, out); - writeBinary(node.stat.pzxid, out); + writeBinary(static_cast(node.data_size), out); + writeBinary(is_ephemeral ? 0 : node.numChildren(), out); + writeBinary(node.pzxid, out); - writeBinary(node.seq_num, out); + writeBinary(node.seqNum(), out); if (version >= SnapshotVersion::V4 && version <= SnapshotVersion::V5) writeBinary(node.sizeInBytes(), out); @@ -102,7 +103,7 @@ namespace { String new_data; readBinary(new_data, in); - node.setData(std::move(new_data)); + node.setData(new_data); if (version >= SnapshotVersion::V1) { @@ -138,22 +139,41 @@ namespace } /// Deserialize stat - readBinary(node.stat.czxid, in); - readBinary(node.stat.mzxid, in); - readBinary(node.stat.ctime, in); - readBinary(node.stat.mtime, in); - readBinary(node.stat.version, in); - readBinary(node.stat.cversion, in); - readBinary(node.stat.aversion, in); - readBinary(node.stat.ephemeralOwner, in); + readBinary(node.czxid, in); + readBinary(node.mzxid, in); + int64_t ctime; + readBinary(ctime, in); + node.setCtime(ctime); + int64_t mtime; + readBinary(mtime, in); + node.setMtime(mtime); + readBinary(node.version, in); + readBinary(node.cversion, in); + readBinary(node.aversion, in); + int64_t ephemeral_owner = 0; + readBinary(ephemeral_owner, in); + if (ephemeral_owner != 0) + { + node.is_ephemeral_and_mtime.is_ephemeral = true; + node.ephemeral_or_children_data.ephemeral_owner = ephemeral_owner; + } + if (version < SnapshotVersion::V6) { int32_t data_length = 0; readBinary(data_length, in); } - readBinary(node.stat.numChildren, in); - readBinary(node.stat.pzxid, in); - readBinary(node.seq_num, in); + int32_t num_children; + readBinary(num_children, in); + if (num_children) + node.ephemeral_or_children_data.children_info.num_children = num_children; + + readBinary(node.pzxid, in); + + int32_t seq_num; + readBinary(seq_num, in); + if (seq_num) + node.ephemeral_or_children_data.children_info.seq_num = seq_num; if (version >= SnapshotVersion::V4 && version <= SnapshotVersion::V5) { @@ -238,7 +258,7 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr /// Benign race condition possible while taking snapshot: NuRaft decide to create snapshot at some log id /// and only after some time we lock storage and enable snapshot mode. So snapshot_container_size can be /// slightly bigger than required. - if (node.stat.mzxid > snapshot.zxid) + if (node.mzxid > snapshot.zxid) break; writeBinary(path, out); @@ -363,9 +383,9 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial if (recalculate_digest) storage.nodes_digest = 0; - const auto is_node_empty = [](const auto & node) + const auto is_node_empty = [](const auto & /*node*/) { - return node.getData().empty() && node.stat == KeeperStorage::Node::Stat{}; + return false; //node.getData().empty() && node == KeeperStorage::Node{}; }; for (size_t nodes_read = 0; nodes_read < snapshot_container_size; ++nodes_read) @@ -412,8 +432,8 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial } storage.container.insertOrReplace(path, node); - if (node.stat.ephemeralOwner != 0) - storage.ephemerals[node.stat.ephemeralOwner].insert(path); + if (node.isEphemeral()) + storage.ephemerals[node.ephemeralOwner()].insert(path); if (recalculate_digest) storage.nodes_digest += node.getDigest(path); @@ -433,12 +453,12 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial { if (itr.key != "/") { - if (itr.value.stat.numChildren != static_cast(itr.value.getChildren().size())) + if (itr.value.numChildren() != static_cast(itr.value.getChildren().size())) { #ifdef NDEBUG /// TODO (alesapin) remove this, it should be always CORRUPTED_DATA. LOG_ERROR(&Poco::Logger::get("KeeperSnapshotManager"), "Children counter in stat.numChildren {}" - " is different from actual children size {} for node {}", itr.value.stat.numChildren, itr.value.getChildren().size(), itr.key); + " is different from actual children size {} for node {}", itr.value.numChildren(), itr.value.getChildren().size(), itr.key); #else throw Exception(ErrorCodes::LOGICAL_ERROR, "Children counter in stat.numChildren {}" " is different from actual children size {} for node {}", diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index c128d7c2f98..6cdbedc2dc6 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -166,56 +166,88 @@ KeeperStorage::ResponsesForSessions processWatchesImpl( } // When this function is updated, update CURRENT_DIGEST_VERSION!! -uint64_t calculateDigest(std::string_view path, std::string_view data, const KeeperStorage::Node::Stat & stat) +uint64_t calculateDigest(std::string_view path, const KeeperStorage::Node & node) { SipHash hash; hash.update(path); - hash.update(data); + hash.update(node.data, node.data_size); - hash.update(stat.czxid); - hash.update(stat.czxid); - hash.update(stat.mzxid); - hash.update(stat.ctime); - hash.update(stat.mtime); - hash.update(stat.version); - hash.update(stat.cversion); - hash.update(stat.aversion); - hash.update(stat.ephemeralOwner); - hash.update(data.length()); - hash.update(stat.numChildren); - hash.update(stat.pzxid); + hash.update(node.czxid); + hash.update(node.czxid); + hash.update(node.mzxid); + hash.update(node.ctime()); + hash.update(node.mtime()); + hash.update(node.version); + hash.update(node.cversion); + hash.update(node.aversion); + bool is_ephemeral = node.isEphemeral(); + hash.update(is_ephemeral ? node.ephemeral_or_children_data.ephemeral_owner : 0); + hash.update(node.data_size); + hash.update(is_ephemeral ? 0 : node.ephemeral_or_children_data.children_info.num_children); + hash.update(node.pzxid); return hash.get64(); } } +void KeeperStorage::Node::copyStats(const Coordination::Stat & stat) +{ + czxid = stat.czxid; + mzxid = stat.mzxid; + pzxid = stat.pzxid; + + setCtime(stat.ctime); + setMtime(stat.mtime); + + version = stat.version; + cversion = stat.cversion; + aversion = stat.aversion; + + if (stat.ephemeralOwner == 0) + { + is_ephemeral_and_mtime.is_ephemeral = false; + ephemeral_or_children_data.children_info.num_children = stat.numChildren; + } + else + { + is_ephemeral_and_mtime.is_ephemeral = true; + ephemeral_or_children_data.ephemeral_owner = stat.ephemeralOwner; + } +} + void KeeperStorage::Node::setResponseStat(Coordination::Stat & response_stat) const { - response_stat.czxid = stat.czxid; - response_stat.mzxid = stat.mzxid; - response_stat.ctime = stat.ctime; - response_stat.mtime = stat.mtime; - response_stat.version = stat.version; - response_stat.cversion = stat.cversion; - response_stat.aversion = stat.aversion; - response_stat.ephemeralOwner = stat.ephemeralOwner; - response_stat.dataLength = static_cast(data.size()); - response_stat.numChildren = stat.numChildren; - response_stat.pzxid = stat.pzxid; + response_stat.czxid = czxid; + response_stat.mzxid = mzxid; + response_stat.ctime = ctime(); + response_stat.mtime = mtime(); + response_stat.version = version; + response_stat.cversion = cversion; + response_stat.aversion = aversion; + bool is_ephemeral = isEphemeral(); + response_stat.ephemeralOwner = is_ephemeral ? ephemeral_or_children_data.ephemeral_owner : 0; + response_stat.dataLength = static_cast(data_size); + response_stat.numChildren = is_ephemeral ? 0 : numChildren(); + response_stat.pzxid = pzxid; } uint64_t KeeperStorage::Node::sizeInBytes() const { - return sizeof(Node) + children.size() * sizeof(StringRef) + data.size(); + return sizeof(Node) + children.size() * sizeof(StringRef) + data_size; } -void KeeperStorage::Node::setData(String new_data) +void KeeperStorage::Node::setData(const String & new_data) { - data = std::move(new_data); + data_size = static_cast(new_data.size()); + if (data_size != 0) + { + data = new char[new_data.size()]; + memcpy(data, new_data.data(), data_size); + } } void KeeperStorage::Node::addChild(StringRef child_path) @@ -230,15 +262,15 @@ void KeeperStorage::Node::removeChild(StringRef child_path) void KeeperStorage::Node::invalidateDigestCache() const { - has_cached_digest = false; + has_cached_digest_and_ctime.has_cached_digest = false; } UInt64 KeeperStorage::Node::getDigest(const std::string_view path) const { - if (!has_cached_digest) + if (!has_cached_digest_and_ctime.has_cached_digest) { - cached_digest = calculateDigest(path, data, stat); - has_cached_digest = true; + cached_digest = calculateDigest(path, *this); + has_cached_digest_and_ctime.has_cached_digest = true; } return cached_digest; @@ -246,9 +278,28 @@ UInt64 KeeperStorage::Node::getDigest(const std::string_view path) const void KeeperStorage::Node::shallowCopy(const KeeperStorage::Node & other) { - stat = other.stat; - seq_num = other.seq_num; - setData(other.getData()); + czxid = other.czxid; + mzxid = other.mzxid; + pzxid = other.pzxid; + acl_id = other.acl_id; /// 0 -- no ACL by default + + has_cached_digest_and_ctime = other.has_cached_digest_and_ctime; + + is_ephemeral_and_mtime = other.is_ephemeral_and_mtime; + + ephemeral_or_children_data = other.ephemeral_or_children_data; + + data_size = other.data_size; + if (data_size != 0) + { + data = new char[data_size]; + memcpy(data, other.data, data_size); + } + + version = other.version; + cversion = other.cversion; + aversion = other.aversion; + cached_digest = other.cached_digest; } @@ -284,9 +335,9 @@ void KeeperStorage::initializeSystemNodes() removeDigest(current_root_it->value, "/"); auto updated_root_it = container.updateValue( "/", - [](auto & node) + [](KeeperStorage::Node & node) { - ++node.stat.numChildren; + node.increaseNumChildren(); node.addChild(getBaseNodeName(keeper_system_path)); } ); @@ -359,7 +410,7 @@ void KeeperStorage::UncommittedState::applyDelta(const Delta & delta) { assert(!node); node = std::make_shared(); - node->stat = operation.stat; + node->copyStats(operation.stat); node->setData(operation.data); acls = operation.acls; last_applied_zxid = delta.zxid; @@ -673,7 +724,7 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) if (node_it == container.end()) onStorageInconsistency(); - if (operation.version != -1 && operation.version != node_it->value.stat.version) + if (operation.version != -1 && operation.version != node_it->value.version) onStorageInconsistency(); removeDigest(node_it->value, path); @@ -695,7 +746,7 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) if (node_it == container.end()) onStorageInconsistency(); - if (operation.version != -1 && operation.version != node_it->value.stat.aversion) + if (operation.version != -1 && operation.version != node_it->value.aversion) onStorageInconsistency(); acl_map.removeUsage(node_it->value.acl_id); @@ -740,7 +791,7 @@ Coordination::Error KeeperStorage::commit(int64_t commit_zxid) bool KeeperStorage::createNode( const std::string & path, String data, - const KeeperStorage::Node::Stat & stat, + const Coordination::Stat & stat, Coordination::ACLs node_acls) { auto parent_path = parentNodePath(path); @@ -749,7 +800,7 @@ bool KeeperStorage::createNode( if (node_it == container.end()) return false; - if (node_it->value.stat.ephemeralOwner != 0) + if (node_it->value.isEphemeral()) return false; if (container.contains(path)) @@ -761,8 +812,8 @@ bool KeeperStorage::createNode( acl_map.addUsage(acl_id); created_node.acl_id = acl_id; - created_node.stat = stat; - created_node.setData(std::move(data)); + created_node.copyStats(stat); + created_node.setData(data); auto [map_key, _] = container.insert(path, created_node); /// Take child path from key owned by map. auto child_path = getBaseNodeName(map_key->getKey()); @@ -771,7 +822,7 @@ bool KeeperStorage::createNode( [child_path](KeeperStorage::Node & parent) { parent.addChild(child_path); - chassert(parent.stat.numChildren == static_cast(parent.getChildren().size())); + chassert(parent.numChildren() == static_cast(parent.getChildren().size())); } ); @@ -785,21 +836,22 @@ bool KeeperStorage::removeNode(const std::string & path, int32_t version) if (node_it == container.end()) return false; - if (version != -1 && version != node_it->value.stat.version) + if (version != -1 && version != node_it->value.version) return false; - if (node_it->value.stat.numChildren) + if (node_it->value.numChildren()) return false; - auto prev_node = node_it->value; - acl_map.removeUsage(prev_node.acl_id); + KeeperStorage::Node prev_node; + prev_node.shallowCopy(node_it->value); + acl_map.removeUsage(node_it->value.acl_id); container.updateValue( parentNodePath(path), [child_basename = getBaseNodeName(node_it->key)](KeeperStorage::Node & parent) { parent.removeChild(child_basename); - chassert(parent.stat.numChildren == static_cast(parent.getChildren().size())); + chassert(parent.numChildren() == static_cast(parent.getChildren().size())); } ); @@ -959,7 +1011,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr if (parent_node == nullptr) return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}}; - else if (parent_node->stat.ephemeralOwner != 0) + else if (parent_node->isEphemeral()) return {KeeperStorage::Delta{zxid, Coordination::Error::ZNOCHILDRENFOREPHEMERALS}}; std::string path_created = request.path; @@ -968,7 +1020,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr if (request.not_exists) return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; - auto seq_num = parent_node->seq_num; + auto seq_num = parent_node->seqNum(); std::stringstream seq_num_str; // STYLE_CHECK_ALLOW_STD_STRING_STREAM seq_num_str.exceptions(std::ios::failbit); @@ -1008,20 +1060,20 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr auto parent_update = [parent_cversion, zxid](KeeperStorage::Node & node) { /// Increment sequential number even if node is not sequential - ++node.seq_num; + node.increaseSeqNum(); if (parent_cversion == -1) - ++node.stat.cversion; - else if (parent_cversion > node.stat.cversion) - node.stat.cversion = parent_cversion; + ++node.cversion; + else if (parent_cversion > node.cversion) + node.cversion = parent_cversion; - if (zxid > node.stat.pzxid) - node.stat.pzxid = zxid; - ++node.stat.numChildren; + if (zxid > node.pzxid) + node.pzxid = zxid; + node.increaseNumChildren(); }; new_deltas.emplace_back(std::string{parent_path}, zxid, KeeperStorage::UpdateNodeDelta{std::move(parent_update)}); - KeeperStorage::Node::Stat stat; + Coordination::Stat stat; stat.czxid = zxid; stat.mzxid = zxid; stat.pzxid = zxid; @@ -1135,7 +1187,8 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce else { node_it->value.setResponseStat(response.stat); - response.data = node_it->value.getData(); + auto data = node_it->value.getData(); + response.data = std::string(data.data, data.size); response.error = Coordination::Error::ZOK; } @@ -1192,8 +1245,8 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr { [zxid](KeeperStorage::Node & parent) { - if (parent.stat.pzxid < zxid) - parent.stat.pzxid = zxid; + if (parent.pzxid < zxid) + parent.pzxid = zxid; } } ); @@ -1207,9 +1260,9 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr update_parent_pzxid(); return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}}; } - else if (request.version != -1 && request.version != node->stat.version) + else if (request.version != -1 && request.version != node->version) return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADVERSION}}; - else if (node->stat.numChildren != 0) + else if (node->numChildren() != 0) return {KeeperStorage::Delta{zxid, Coordination::Error::ZNOTEMPTY}}; if (request.restored_from_zookeeper_log) @@ -1220,14 +1273,14 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr zxid, KeeperStorage::UpdateNodeDelta{[](KeeperStorage::Node & parent) { - ++parent.stat.cversion; - --parent.stat.numChildren; + ++parent.cversion; + --parent.ephemeral_or_children_data.children_info.num_children; }}); - new_deltas.emplace_back(request.path, zxid, KeeperStorage::RemoveNodeDelta{request.version, node->stat.ephemeralOwner}); + new_deltas.emplace_back(request.path, zxid, KeeperStorage::RemoveNodeDelta{request.version, node->ephemeralOwner()}); - if (node->stat.ephemeralOwner != 0) - storage.unregisterEphemeralPath(node->stat.ephemeralOwner, request.path); + if (node->isEphemeral()) + storage.unregisterEphemeralPath(node->ephemeralOwner(), request.path); digest = storage.calculateNodesDigest(digest, new_deltas); @@ -1341,7 +1394,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce auto node = storage.uncommitted_state.getNode(request.path); - if (request.version != -1 && request.version != node->stat.version) + if (request.version != -1 && request.version != node->version) return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADVERSION}}; new_deltas.emplace_back( @@ -1350,9 +1403,9 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce KeeperStorage::UpdateNodeDelta{ [zxid, data = request.data, time](KeeperStorage::Node & value) { - value.stat.version++; - value.stat.mzxid = zxid; - value.stat.mtime = time; + value.version++; + value.mzxid = zxid; + value.setMtime(time); value.setData(data); }, request.version}); @@ -1364,7 +1417,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce { [](KeeperStorage::Node & parent) { - parent.stat.cversion++; + parent.cversion++; } } ); @@ -1478,7 +1531,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc if (child_it == container.end()) onStorageInconsistency(); - const auto is_ephemeral = child_it->value.stat.ephemeralOwner != 0; + const auto is_ephemeral = child_it->value.isEphemeral(); return (is_ephemeral && list_request_type == EPHEMERAL_ONLY) || (!is_ephemeral && list_request_type == PERSISTENT_ONLY); }; @@ -1531,7 +1584,7 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro auto node = storage.uncommitted_state.getNode(request.path); if (check_not_exists) { - if (node && (request.version == -1 || request.version == node->stat.version)) + if (node && (request.version == -1 || request.version == node->version)) return {KeeperStorage::Delta{zxid, Coordination::Error::ZNODEEXISTS}}; } else @@ -1539,7 +1592,7 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro if (!node) return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}}; - if (request.version != -1 && request.version != node->stat.version) + if (request.version != -1 && request.version != node->version) return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADVERSION}}; } @@ -1575,7 +1628,7 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro if (check_not_exists) { - if (node_it != container.end() && (request.version == -1 || request.version == node_it->value.stat.version)) + if (node_it != container.end() && (request.version == -1 || request.version == node_it->value.version)) on_error(Coordination::Error::ZNODEEXISTS); else response.error = Coordination::Error::ZOK; @@ -1584,7 +1637,7 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro { if (node_it == container.end()) on_error(Coordination::Error::ZNONODE); - else if (request.version != -1 && request.version != node_it->value.stat.version) + else if (request.version != -1 && request.version != node_it->value.version) on_error(Coordination::Error::ZBADVERSION); else response.error = Coordination::Error::ZOK; @@ -1637,7 +1690,7 @@ struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestPr auto node = uncommitted_state.getNode(request.path); - if (request.version != -1 && request.version != node->stat.aversion) + if (request.version != -1 && request.version != node->aversion) return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADVERSION}}; @@ -1657,7 +1710,7 @@ struct KeeperStorageSetACLRequestProcessor final : public KeeperStorageRequestPr zxid, KeeperStorage::UpdateNodeDelta { - [](KeeperStorage::Node & n) { ++n.stat.aversion; } + [](KeeperStorage::Node & n) { ++n.aversion; } } } }; @@ -2075,7 +2128,7 @@ UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, const std::vec [&](const CreateNodeDelta & create_delta) { auto node = std::make_shared(); - node->stat = create_delta.stat; + node->copyStats(create_delta.stat); node->setData(create_delta.data); updated_nodes.emplace(delta.path, node); }, @@ -2198,8 +2251,8 @@ void KeeperStorage::preprocessRequest( { [ephemeral_path](Node & parent) { - ++parent.stat.cversion; - --parent.stat.numChildren; + ++parent.cversion; + --parent.ephemeral_or_children_data.children_info.num_children; } } ); diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 01c1413a884..f14a6ed772c 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -35,40 +35,144 @@ public: /// New fields should be added to the struct only if it's really necessary struct Node { - /// to reduce size of the Node struct we use a custom Stat without dataLength - struct Stat - { - int64_t czxid{0}; - int64_t mzxid{0}; - int64_t ctime{0}; - int64_t mtime{0}; - int32_t version{0}; - int32_t cversion{0}; - int32_t aversion{0}; - int32_t numChildren{0}; /// NOLINT - int64_t ephemeralOwner{0}; /// NOLINT - int64_t pzxid{0}; - - bool operator==(const Stat &) const = default; - }; - + int64_t czxid{0}; + int64_t mzxid{0}; + int64_t pzxid{0}; uint64_t acl_id = 0; /// 0 -- no ACL by default - Stat stat{}; - int32_t seq_num = 0; + + mutable struct + { + bool has_cached_digest : 1; + int64_t ctime : 7; + } has_cached_digest_and_ctime{false, 0}; + + struct + { + bool is_ephemeral : 1; + int64_t mtime : 7; + } is_ephemeral_and_mtime{false, 0}; + + + union + { + int64_t ephemeral_owner; + struct + { + int32_t seq_num; + int32_t num_children; + } children_info; + } ephemeral_or_children_data{0}; + + char * data{nullptr}; + uint32_t data_size{0}; + + int32_t version{0}; + int32_t cversion{0}; + int32_t aversion{0}; /// we cannot use `std::optional because we want to /// pack the boolean with seq_num above - mutable bool has_cached_digest = false; mutable uint64_t cached_digest = 0; + ~Node() + { + if (data_size) + delete [] data; + } + + Node() = default; + + Node & operator=(const Node & other) + { + if (this == &other) + return *this; + + czxid = other.czxid; + mzxid = other.mzxid; + pzxid = other.pzxid; + acl_id = other.acl_id; + has_cached_digest_and_ctime = other.has_cached_digest_and_ctime; + is_ephemeral_and_mtime = other.is_ephemeral_and_mtime; + ephemeral_or_children_data = other.ephemeral_or_children_data; + data_size = other.data_size; + version = other.version; + cversion = other.cversion; + aversion = other.aversion; + + if (data_size != 0) + { + data = new char[data_size]; + memcpy(data, other.data, data_size); + } + return *this; + } + + Node(const Node & other) + { + *this = other; + } + + bool isEphemeral() const + { + + return is_ephemeral_and_mtime.is_ephemeral; + } + + int64_t ephemeralOwner() const + { + return isEphemeral() ? ephemeral_or_children_data.ephemeral_owner : 0; + } + + int32_t numChildren() const + { + return ephemeral_or_children_data.children_info.num_children; + } + + void increaseNumChildren() + { + ++ephemeral_or_children_data.children_info.num_children; + } + + int32_t seqNum() const + { + return ephemeral_or_children_data.children_info.seq_num; + } + + void increaseSeqNum() + { + ++ephemeral_or_children_data.children_info.seq_num; + } + + int64_t ctime() const + { + return has_cached_digest_and_ctime.ctime; + } + + void setCtime(uint64_t ctime) + { + has_cached_digest_and_ctime.ctime = ctime; + } + + int64_t mtime() const + { + return is_ephemeral_and_mtime.mtime; + } + + void setMtime(uint64_t mtime) + { + is_ephemeral_and_mtime.mtime = mtime; + } + + void copyStats(const Coordination::Stat & stat); + void setResponseStat(Coordination::Stat & response_stat) const; /// Object memory size uint64_t sizeInBytes() const; - void setData(String new_data); + void setData(const String & new_data); - const auto & getData() const noexcept { return data; } + StringRef getData() const noexcept { return {data, data_size}; } void addChild(StringRef child_path); @@ -87,7 +191,6 @@ public: // (e.g. we don't need to copy list of children) void shallowCopy(const Node & other); private: - String data; ChildrenSet children{}; }; @@ -177,7 +280,7 @@ public: // - quickly commit the changes to the storage struct CreateNodeDelta { - KeeperStorage::Node::Stat stat; + Coordination::Stat stat; Coordination::ACLs acls; String data; }; @@ -342,7 +445,7 @@ public: bool createNode( const std::string & path, String data, - const KeeperStorage::Node::Stat & stat, + const Coordination::Stat & stat, Coordination::ACLs node_acls); // Remove node in the storage diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index ac8d36745c2..716f08faf64 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -19,55 +19,47 @@ struct ListNode StringRef key; V value; - /// |* * ****** | - /// ^ ^ ^ - /// active_in_map free_key version - /// (1 byte) (1 byte) (6 bytes) - uint64_t node_metadata = 0; + struct + { + bool active_in_map : 1; + bool free_key : 1; + uint64_t version : 6; + } node_metadata{false, false, 0}; void setInactiveInMap() { - node_metadata &= ~active_in_map_mask; + node_metadata.active_in_map = false; } void setActiveInMap() { - node_metadata |= active_in_map_mask; + node_metadata.active_in_map = true; } bool isActiveInMap() { - return node_metadata & active_in_map_mask; + return node_metadata.active_in_map; } void setFreeKey() { - node_metadata |= free_key_mask; + node_metadata.free_key = true; } bool getFreeKey() { - return node_metadata & free_key_mask; + return node_metadata.free_key; } uint64_t getVersion() { - return node_metadata & version_mask; + return node_metadata.version; } void setVersion(uint64_t version) { - if (version > version_mask) - throw Exception( - ErrorCodes::LOGICAL_ERROR, "Snapshot version {} is larger than maximum allowed value {}", version, version_mask); - - node_metadata &= ~version_mask; - node_metadata |= version; + node_metadata.version = version; } - - static constexpr uint64_t active_in_map_mask = static_cast(1) << 63; - static constexpr uint64_t free_key_mask = static_cast(1) << 62; - static constexpr uint64_t version_mask = ~(static_cast(3) << 62); }; template diff --git a/src/Coordination/ZooKeeperDataReader.cpp b/src/Coordination/ZooKeeperDataReader.cpp index b55ebef327f..b4334893849 100644 --- a/src/Coordination/ZooKeeperDataReader.cpp +++ b/src/Coordination/ZooKeeperDataReader.cpp @@ -105,7 +105,7 @@ int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, Poco::L Coordination::read(node.acl_id, in); /// Deserialize stat - Coordination::read(node.stat.czxid, in); + Coordination::read(node.src/Coordination/ZooKeeperDataReader.cppstat.czxid, in); Coordination::read(node.stat.mzxid, in); /// For some reason ZXID specified in filename can be smaller /// then actual zxid from nodes. In this case we will use zxid from nodes. From 09f1e2840c2859a517c9f76183a7abd51c488b6f Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 25 Jan 2024 10:06:05 +0100 Subject: [PATCH 094/276] Simplified calculatePartSize and upload task --- .../AzureBlobStorage/AzureBlobStorageAuth.cpp | 2 - .../AzureBlobStorage/AzureObjectStorage.h | 4 - .../copyAzureBlobStorageFile.cpp | 114 +++++------------- 3 files changed, 27 insertions(+), 93 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp index cbc2996f5c1..02b0d5bb599 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -165,9 +165,7 @@ std::unique_ptr getAzureBlobStorageSettings(const Po config.getInt(config_prefix + ".max_single_read_retries", 3), config.getInt(config_prefix + ".max_single_download_retries", 3), config.getInt(config_prefix + ".list_object_keys_size", 1000), - config.getUInt64(config_prefix + ".min_upload_part_size", 16 * 1024 * 1024), config.getUInt64(config_prefix + ".max_upload_part_size", 5ULL * 1024 * 1024 * 1024), - config.getUInt64(config_prefix + ".max_part_number", 10000), config.getBool(config_prefix + ".use_native_copy", false) ); } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 3be4989d4f2..30fedb601dc 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -24,18 +24,14 @@ struct AzureObjectStorageSettings int max_single_read_retries_, int max_single_download_retries_, int list_object_keys_size_, - size_t min_upload_part_size_, size_t max_upload_part_size_, - size_t max_part_number_, bool use_native_copy_) : max_single_part_upload_size(max_single_part_upload_size_) , min_bytes_for_seek(min_bytes_for_seek_) , max_single_read_retries(max_single_read_retries_) , max_single_download_retries(max_single_download_retries_) , list_object_keys_size(list_object_keys_size_) - , min_upload_part_size(min_upload_part_size_) , max_upload_part_size(max_upload_part_size_) - , max_part_number(max_part_number_) , use_native_copy(use_native_copy_) { } diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp index 350d2d1d34e..e5517a1a021 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp @@ -84,17 +84,10 @@ namespace struct UploadPartTask { - char *data = nullptr; - size_t size = 0; - std::string block_id; + std::unique_ptr read_buffer = nullptr; + std::vector block_ids; bool is_finished = false; std::exception_ptr exception; - - ~UploadPartTask() - { - if (data != nullptr) - free(data); - } }; size_t normal_part_size; @@ -108,56 +101,11 @@ namespace void calculatePartSize() { - if (!total_size) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Chosen multipart upload for an empty file. This must not happen"); - - auto max_part_number = settings.get()->max_part_number; - auto min_upload_part_size = settings.get()->min_upload_part_size; auto max_upload_part_size = settings.get()->max_upload_part_size; - - if (!max_part_number) - throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "max_part_number must not be 0"); - else if (!min_upload_part_size) - throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "min_upload_part_size must not be 0"); - else if (max_upload_part_size < min_upload_part_size) - throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "max_upload_part_size must not be less than min_upload_part_size"); - - size_t part_size = min_upload_part_size; - size_t num_parts = (total_size + part_size - 1) / part_size; - - if (num_parts > max_part_number) - { - part_size = (total_size + max_part_number - 1) / max_part_number; - num_parts = (total_size + part_size - 1) / part_size; - } - - if (part_size > max_upload_part_size) - { - part_size = max_upload_part_size; - num_parts = (total_size + part_size - 1) / part_size; - } - - if (num_parts < 1 || num_parts > max_part_number || part_size < min_upload_part_size || part_size > max_upload_part_size) - { - String msg; - if (num_parts < 1) - msg = "Number of parts is zero"; - else if (num_parts > max_part_number) - msg = fmt::format("Number of parts exceeds {}", num_parts, max_part_number); - else if (part_size < min_upload_part_size) - msg = fmt::format("Size of a part is less than {}", part_size, min_upload_part_size); - else - msg = fmt::format("Size of a part exceeds {}", part_size, max_upload_part_size); - - throw Exception( - ErrorCodes::INVALID_CONFIG_PARAMETER, - "{} while writing {} bytes to AzureBlobStorage. Check max_part_number = {}, " - "min_upload_part_size = {}, max_upload_part_size = {}", - msg, total_size, max_part_number, min_upload_part_size, max_upload_part_size); - } - + if (!max_upload_part_size) + throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "max_upload_part_size must not be 0"); /// We've calculated the size of a normal part (the final part can be smaller). - normal_part_size = part_size; + normal_part_size = max_upload_part_size; } public: @@ -238,18 +186,13 @@ namespace try { - auto read_buffer = std::make_unique(create_read_buffer(), part_offset, part_size); - task->data = new char[part_size]; - task->size = part_size; - size_t n = read_buffer->read(task->data,part_size); - if (n != part_size) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected size"); + task->read_buffer = std::make_unique(create_read_buffer(), part_offset, part_size); schedule([this, task, task_finish_notify]() { try { - processUploadTask(*task); + processUploadPartRequest(*task); } catch (...) { @@ -267,38 +210,35 @@ namespace else { UploadPartTask task; - auto read_buffer = std::make_unique(create_read_buffer(), part_offset, part_size); - task.data = new char[part_size]; - size_t n = read_buffer->read(task.data,part_size); - if (n != part_size) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected size"); - task.size = part_size; - processUploadTask(task); - block_ids.emplace_back(task.block_id); + task.read_buffer = std::make_unique(create_read_buffer(), part_offset, part_size); + processUploadPartRequest(task); + block_ids.insert(block_ids.end(),task.block_ids.begin(), task.block_ids.end()); } } - void processUploadTask(UploadPartTask & task) - { - auto block_id = processUploadPartRequest(task); - - std::lock_guard lock(bg_tasks_mutex); /// Protect bg_tasks from race - task.block_id = block_id; - LOG_TRACE(log, "Writing part finished. Container: {}, Blob: {}, block_id: {}, Parts: {}", dest_container_for_logging, dest_blob, block_id, bg_tasks.size()); - } - - String processUploadPartRequest(UploadPartTask & task) + void processUploadPartRequest(UploadPartTask & task) { ProfileEvents::increment(ProfileEvents::AzureUploadPart); if (for_disk_azure_blob_storage) ProfileEvents::increment(ProfileEvents::DiskAzureUploadPart); auto block_blob_client = client.get()->GetBlockBlobClient(dest_blob); - task.block_id = getRandomASCIIString(64); - Azure::Core::IO::MemoryBodyStream memory(reinterpret_cast(task.data), task.size); - block_blob_client.StageBlock(task.block_id, memory); - return task.block_id; + while (!task.read_buffer->eof()) + { + auto size = task.read_buffer->available(); + if (size > 0) + { + auto block_id = getRandomASCIIString(64); + Azure::Core::IO::MemoryBodyStream memory(reinterpret_cast(task.read_buffer->position()), size); + block_blob_client.StageBlock(block_id, memory); + task.block_ids.emplace_back(block_id); + task.read_buffer->ignore(size); + LOG_TRACE(log, "Writing part. Container: {}, Blob: {}, block_id: {}", dest_container_for_logging, dest_blob, block_id); + } + } + std::lock_guard lock(bg_tasks_mutex); /// Protect bg_tasks from race + LOG_TRACE(log, "Writing part finished. Container: {}, Blob: {}, Parts: {}", dest_container_for_logging, dest_blob, bg_tasks.size()); } @@ -316,7 +256,7 @@ namespace { if (task.exception) std::rethrow_exception(task.exception); - block_ids.emplace_back(task.block_id); + block_ids.insert(block_ids.end(),task.block_ids.begin(), task.block_ids.end()); } } }; From d264a5a148c577ab046dc4bbef50b5a4e0c32db9 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sun, 28 Jan 2024 12:06:52 +0100 Subject: [PATCH 095/276] Update client & settings to std::shared_ptr --- src/Backups/BackupIO_AzureBlobStorage.cpp | 6 ++-- src/Backups/BackupIO_AzureBlobStorage.h | 8 +++--- .../AzureBlobStorage/AzureObjectStorage.h | 6 ++-- .../Cached/CachedObjectStorage.h | 2 +- src/Disks/ObjectStorages/IObjectStorage.h | 2 +- .../copyAzureBlobStorageFile.cpp | 28 +++++++++---------- .../copyAzureBlobStorageFile.h | 10 +++---- 7 files changed, 31 insertions(+), 31 deletions(-) diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index 1b4c10ad0cb..d99f296cca1 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -89,7 +89,7 @@ std::unique_ptr BackupReaderAzureBlobStorage::readFile(const key = file_name; } return std::make_unique( - client.get(), key, read_settings, settings.get()->max_single_read_retries, + client, key, read_settings, settings.get()->max_single_read_retries, settings.get()->max_single_download_retries); } @@ -262,7 +262,7 @@ std::unique_ptr BackupWriterAzureBlobStorage::readFile(const String } return std::make_unique( - client.get(), key, read_settings, settings.get()->max_single_read_retries, + client, key, read_settings, settings.get()->max_single_read_retries, settings.get()->max_single_download_retries); } @@ -278,7 +278,7 @@ std::unique_ptr BackupWriterAzureBlobStorage::writeFile(const Strin key = file_name; } return std::make_unique( - client.get(), + client, key, settings.get()->max_single_part_upload_size, DBMS_DEFAULT_BUFFER_SIZE, diff --git a/src/Backups/BackupIO_AzureBlobStorage.h b/src/Backups/BackupIO_AzureBlobStorage.h index 87dc470cdb3..95325044a62 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.h +++ b/src/Backups/BackupIO_AzureBlobStorage.h @@ -28,10 +28,10 @@ public: private: const DataSourceDescription data_source_description; - MultiVersion client; + std::shared_ptr client; StorageAzureBlob::Configuration configuration; std::unique_ptr object_storage; - MultiVersion settings; + std::shared_ptr settings; }; class BackupWriterAzureBlobStorage : public BackupWriterDefault @@ -57,10 +57,10 @@ private: std::unique_ptr readFile(const String & file_name, size_t expected_file_size) override; void removeFilesBatch(const Strings & file_names); const DataSourceDescription data_source_description; - MultiVersion client; + std::shared_ptr client; StorageAzureBlob::Configuration configuration; std::unique_ptr object_storage; - MultiVersion settings; + std::shared_ptr settings; }; } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 30fedb601dc..0ae12fb205f 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -137,11 +137,11 @@ public: bool isRemote() const override { return true; } - MultiVersion & getSettings() { return settings; } + std::shared_ptr getSettings() { return settings.get(); } - MultiVersion & getAzureBlobStorageClient() override + std::shared_ptr getAzureBlobStorageClient() override { - return client; + return client.get(); } private: diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index 2ed8990515f..1f293e5857e 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -122,7 +122,7 @@ public: static bool canUseReadThroughCache(const ReadSettings & settings); #if USE_AZURE_BLOB_STORAGE - MultiVersion & getAzureBlobStorageClient() override + std::shared_ptr getAzureBlobStorageClient() override { return object_storage->getAzureBlobStorageClient(); } diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index e066beaefcc..049935ad60c 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -226,7 +226,7 @@ public: virtual WriteSettings patchSettings(const WriteSettings & write_settings) const; #if USE_AZURE_BLOB_STORAGE - virtual MultiVersion & getAzureBlobStorageClient() + virtual std::shared_ptr getAzureBlobStorageClient() { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "This function is only implemented for AzureBlobStorage"); } diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp index e5517a1a021..537a5a191e7 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp @@ -44,12 +44,12 @@ namespace public: UploadHelper( const CreateReadBuffer & create_read_buffer_, - MultiVersion & client_, + std::shared_ptr client_, size_t offset_, size_t total_size_, const String & dest_container_for_logging_, const String & dest_blob_, - MultiVersion settings_, + std::shared_ptr settings_, ThreadPoolCallbackRunner schedule_, bool for_disk_azure_blob_storage_, const Poco::Logger * log_) @@ -71,12 +71,12 @@ namespace protected: std::function()> create_read_buffer; - MultiVersion & client; + std::shared_ptr client; size_t offset; size_t total_size; const String & dest_container_for_logging; const String & dest_blob; - MultiVersion settings; + std::shared_ptr settings; ThreadPoolCallbackRunner schedule; bool for_disk_azure_blob_storage; const Poco::Logger * log; @@ -116,7 +116,7 @@ namespace void completeMultipartUpload() { - auto block_blob_client = client.get()->GetBlockBlobClient(dest_blob); + auto block_blob_client = client->GetBlockBlobClient(dest_blob); block_blob_client.CommitBlockList(block_ids); } @@ -222,7 +222,7 @@ namespace if (for_disk_azure_blob_storage) ProfileEvents::increment(ProfileEvents::DiskAzureUploadPart); - auto block_blob_client = client.get()->GetBlockBlobClient(dest_blob); + auto block_blob_client = client->GetBlockBlobClient(dest_blob); while (!task.read_buffer->eof()) { @@ -267,10 +267,10 @@ void copyDataToAzureBlobStorageFile( const std::function()> & create_read_buffer, size_t offset, size_t size, - MultiVersion & dest_client, + std::shared_ptr dest_client, const String & dest_container_for_logging, const String & dest_blob, - MultiVersion settings, + std::shared_ptr settings, ThreadPoolCallbackRunner schedule, bool for_disk_azure_blob_storage) { @@ -280,15 +280,15 @@ void copyDataToAzureBlobStorageFile( void copyAzureBlobStorageFile( - MultiVersion & src_client, - MultiVersion & dest_client, + std::shared_ptr src_client, + std::shared_ptr dest_client, const String & src_container_for_logging, const String & src_blob, size_t offset, size_t size, const String & dest_container_for_logging, const String & dest_blob, - MultiVersion settings, + std::shared_ptr settings, const ReadSettings & read_settings, ThreadPoolCallbackRunner schedule, bool for_disk_azure_blob_storage) @@ -300,8 +300,8 @@ void copyAzureBlobStorageFile( if (for_disk_azure_blob_storage) ProfileEvents::increment(ProfileEvents::DiskAzureCopyObject); - auto block_blob_client_src = src_client.get()->GetBlockBlobClient(src_blob); - auto block_blob_client_dest = dest_client.get()->GetBlockBlobClient(dest_blob); + auto block_blob_client_src = src_client->GetBlockBlobClient(src_blob); + auto block_blob_client_dest = dest_client->GetBlockBlobClient(dest_blob); auto source_uri = block_blob_client_src.GetUrl(); if (size < max_single_operation_copy_size) @@ -328,7 +328,7 @@ void copyAzureBlobStorageFile( LOG_TRACE(&Poco::Logger::get("copyAzureBlobStorageFile"), "Reading from Container: {}, Blob: {}", src_container_for_logging, src_blob); auto create_read_buffer = [&] { - return std::make_unique(src_client.get(), src_blob, read_settings, settings.get()->max_single_read_retries, + return std::make_unique(src_client, src_blob, read_settings, settings.get()->max_single_read_retries, settings.get()->max_single_download_retries); }; diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h index 15a31031f63..83814f42693 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h @@ -21,15 +21,15 @@ using CreateReadBuffer = std::function()>; /// Copies a file from AzureBlobStorage to AzureBlobStorage. /// The parameters `src_offset` and `src_size` specify a part in the source to copy. void copyAzureBlobStorageFile( - MultiVersion & src_client, - MultiVersion & dest_client, + std::shared_ptr src_client, + std::shared_ptr dest_client, const String & src_container_for_logging, const String & src_blob, size_t src_offset, size_t src_size, const String & dest_container_for_logging, const String & dest_blob, - MultiVersion settings, + std::shared_ptr settings, const ReadSettings & read_settings, ThreadPoolCallbackRunner schedule_ = {}, bool for_disk_azure_blob_storage = false); @@ -44,10 +44,10 @@ void copyDataToAzureBlobStorageFile( const std::function()> & create_read_buffer, size_t offset, size_t size, - MultiVersion & client, + std::shared_ptr client, const String & dest_container_for_logging, const String & dest_blob, - MultiVersion settings, + std::shared_ptr settings, ThreadPoolCallbackRunner schedule_ = {}, bool for_disk_azure_blob_storage = false); From 8798f469b3a0e38341e759f5cc98ca86b8220069 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jan 2024 22:27:21 +0100 Subject: [PATCH 096/276] Fix conflicts. --- tests/ci/ci_config.py | 47 +++++++------------------------------------ 1 file changed, 7 insertions(+), 40 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 0fc4b3505ce..8c8c45b877c 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -50,6 +50,7 @@ class JobNames(metaclass=WithIter): STATELESS_TEST_DEBUG = "Stateless tests (debug)" STATELESS_TEST_RELEASE = "Stateless tests (release)" + STATELESS_TEST_RELEASE_COVERAGE = "Stateless tests (coverage)" STATELESS_TEST_AARCH64 = "Stateless tests (aarch64)" STATELESS_TEST_ASAN = "Stateless tests (asan)" STATELESS_TEST_TSAN = "Stateless tests (tsan)" @@ -64,6 +65,7 @@ class JobNames(metaclass=WithIter): STATEFUL_TEST_DEBUG = "Stateful tests (debug)" STATEFUL_TEST_RELEASE = "Stateful tests (release)" + STATEFUL_TEST_RELEASE_COVERAGE = "Stateful tests (coverage)" STATEFUL_TEST_AARCH64 = "Stateful tests (aarch64)" STATEFUL_TEST_ASAN = "Stateful tests (asan)" STATEFUL_TEST_TSAN = "Stateful tests (tsan)" @@ -763,18 +765,6 @@ CI_CONFIG = CIConfig( builds_report_config={ JobNames.BUILD_CHECK: BuildReportConfig( builds=[ -<<<<<<< HEAD - "package_release", - "package_aarch64", - "package_asan", - "package_ubsan", - "package_tsan", - "package_msan", - "package_debug", - "package_release_coverage", - "binary_release", - "fuzzers", -======= Build.PACKAGE_RELEASE, Build.PACKAGE_AARCH64, Build.PACKAGE_ASAN, @@ -782,9 +772,9 @@ CI_CONFIG = CIConfig( Build.PACKAGE_TSAN, Build.PACKAGE_MSAN, Build.PACKAGE_DEBUG, + Build.PACKAGE_RELEASE_COVERAGE, Build.BINARY_RELEASE, Build.FUZZERS, ->>>>>>> master ] ), JobNames.BUILD_CHECK_SPECIAL: BuildReportConfig( @@ -872,33 +862,15 @@ CI_CONFIG = CIConfig( JobNames.STATEFUL_TEST_RELEASE: TestConfig( Build.PACKAGE_RELEASE, job_config=JobConfig(**stateful_test_common_params) # type: ignore ), -<<<<<<< HEAD - "Stateful tests (coverage)": TestConfig( - "package_release_coverage", job_config=JobConfig(**stateful_test_common_params) # type: ignore + JobNames.STATEFUL_TEST_RELEASE_COVERAGE: TestConfig( + Build.PACKAGE_RELEASE_COVERAGE, job_config=JobConfig(**stateful_test_common_params) # type: ignore ), - "Stateful tests (aarch64)": TestConfig( - "package_aarch64", job_config=JobConfig(**stateful_test_common_params) # type: ignore - ), - "Stateful tests (release, DatabaseOrdinary)": TestConfig( - "package_release", job_config=JobConfig(**stateful_test_common_params) # type: ignore - ), - "Stateful tests (release, ParallelReplicas)": TestConfig( - "package_release", job_config=JobConfig(**stateful_test_common_params) # type: ignore -======= JobNames.STATEFUL_TEST_AARCH64: TestConfig( Build.PACKAGE_AARCH64, job_config=JobConfig(**stateful_test_common_params) # type: ignore ), - # FIXME: delete? - # "Stateful tests (release, DatabaseOrdinary)": TestConfig( - # Build.PACKAGE_RELEASE, job_config=JobConfig(**stateful_test_common_params) # type: ignore - # ), - # "Stateful tests (release, DatabaseReplicated)": TestConfig( - # Build.PACKAGE_RELEASE, job_config=JobConfig(**stateful_test_common_params) # type: ignore - # ), # Stateful tests for parallel replicas JobNames.STATEFUL_TEST_PARALLEL_REPL_RELEASE: TestConfig( Build.PACKAGE_RELEASE, job_config=JobConfig(**stateful_test_common_params) # type: ignore ->>>>>>> master ), JobNames.STATEFUL_TEST_PARALLEL_REPL_DEBUG: TestConfig( Build.PACKAGE_DEBUG, job_config=JobConfig(**stateful_test_common_params) # type: ignore @@ -939,16 +911,11 @@ CI_CONFIG = CIConfig( JobNames.STATELESS_TEST_RELEASE: TestConfig( Build.PACKAGE_RELEASE, job_config=JobConfig(**statless_test_common_params) # type: ignore ), -<<<<<<< HEAD - "Stateless tests (coverage)": TestConfig( - "package_release_coverage", job_config=JobConfig(**statless_test_common_params) # type: ignore + JobNames.STATELESS_TEST_RELEASE_COVERAGE: TestConfig( + Build.PACKAGE_RELEASE_COVERAGE, job_config=JobConfig(**statless_test_common_params) # type: ignore ), - "Stateless tests (aarch64)": TestConfig( - "package_aarch64", job_config=JobConfig(**statless_test_common_params) # type: ignore -======= JobNames.STATELESS_TEST_AARCH64: TestConfig( Build.PACKAGE_AARCH64, job_config=JobConfig(**statless_test_common_params) # type: ignore ->>>>>>> master ), JobNames.STATELESS_TEST_ANALYZER_RELEASE: TestConfig( Build.PACKAGE_RELEASE, job_config=JobConfig(**statless_test_common_params) # type: ignore From 1ef8062c7701c1788abda4d5da7ee56a5b9de372 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jan 2024 22:28:09 +0100 Subject: [PATCH 097/276] Split by batches --- tests/ci/ci_config.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 8c8c45b877c..0cfddbe0435 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -912,7 +912,8 @@ CI_CONFIG = CIConfig( Build.PACKAGE_RELEASE, job_config=JobConfig(**statless_test_common_params) # type: ignore ), JobNames.STATELESS_TEST_RELEASE_COVERAGE: TestConfig( - Build.PACKAGE_RELEASE_COVERAGE, job_config=JobConfig(**statless_test_common_params) # type: ignore + Build.PACKAGE_RELEASE_COVERAGE, + job_config=JobConfig(num_batches=6, **statless_test_common_params) # type: ignore ), JobNames.STATELESS_TEST_AARCH64: TestConfig( Build.PACKAGE_AARCH64, job_config=JobConfig(**statless_test_common_params) # type: ignore From 982e3ddbddb0baaa2ac11f9dec74a8be8c8e8545 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Jan 2024 23:11:03 +0100 Subject: [PATCH 098/276] Fix Python --- tests/ci/ci_config.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 0cfddbe0435..7458f25805f 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -913,7 +913,7 @@ CI_CONFIG = CIConfig( ), JobNames.STATELESS_TEST_RELEASE_COVERAGE: TestConfig( Build.PACKAGE_RELEASE_COVERAGE, - job_config=JobConfig(num_batches=6, **statless_test_common_params) # type: ignore + job_config=JobConfig(num_batches=6, **statless_test_common_params), # type: ignore ), JobNames.STATELESS_TEST_AARCH64: TestConfig( Build.PACKAGE_AARCH64, job_config=JobConfig(**statless_test_common_params) # type: ignore From a22b68f46fec54f98fc3c3cb9a9c1f597bae7ffd Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 29 Jan 2024 10:49:36 +0100 Subject: [PATCH 099/276] Added setting azure_max_single_part_copy_size --- src/Core/Settings.h | 3 ++- .../AzureBlobStorage/AzureBlobStorageAuth.cpp | 4 +++- .../ObjectStorages/AzureBlobStorage/AzureObjectStorage.h | 3 +++ src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp | 6 +----- 4 files changed, 9 insertions(+), 7 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 305d6466658..4ae5d1585f3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -82,7 +82,8 @@ class IColumn; M(UInt64, s3_upload_part_size_multiply_parts_count_threshold, 500, "Each time this number of parts was uploaded to S3, s3_min_upload_part_size is multiplied by s3_upload_part_size_multiply_factor.", 0) \ M(UInt64, s3_max_inflight_parts_for_one_file, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited. You ", 0) \ M(UInt64, s3_max_single_part_upload_size, 32*1024*1024, "The maximum size of object to upload using singlepart upload to S3.", 0) \ - M(UInt64, azure_max_single_part_upload_size, 100*1024*1024, "The maximum size of object to upload using singlepart upload to Azure blob storage.", 0) \ + M(UInt64, azure_max_single_part_upload_size, 100*1024*1024, "The maximum size of object to upload using singlepart upload to Azure blob storage.", 0) \ + M(UInt64, azure_max_single_part_copy_size, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage.", 0) \ M(UInt64, s3_max_single_read_retries, 4, "The maximum number of retries during single S3 read.", 0) \ M(UInt64, azure_max_single_read_retries, 4, "The maximum number of retries during single Azure blob storage read.", 0) \ M(UInt64, s3_max_unexpected_write_error_retries, 4, "The maximum number of retries in case of unexpected errors during S3 write.", 0) \ diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp index 02b0d5bb599..9da84d430e4 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -7,6 +7,7 @@ #include #include #include +#include using namespace Azure::Storage::Blobs; @@ -157,7 +158,7 @@ std::unique_ptr getAzureBlobContainerClient( } } -std::unique_ptr getAzureBlobStorageSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr /*context*/) +std::unique_ptr getAzureBlobStorageSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context) { return std::make_unique( config.getUInt64(config_prefix + ".max_single_part_upload_size", 100 * 1024 * 1024), @@ -166,6 +167,7 @@ std::unique_ptr getAzureBlobStorageSettings(const Po config.getInt(config_prefix + ".max_single_download_retries", 3), config.getInt(config_prefix + ".list_object_keys_size", 1000), config.getUInt64(config_prefix + ".max_upload_part_size", 5ULL * 1024 * 1024 * 1024), + config.getUInt64(config_prefix + ".max_single_part_copy_size", context->getSettings().azure_max_single_part_copy_size), config.getBool(config_prefix + ".use_native_copy", false) ); } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 0ae12fb205f..18b1a70defe 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -25,6 +25,7 @@ struct AzureObjectStorageSettings int max_single_download_retries_, int list_object_keys_size_, size_t max_upload_part_size_, + size_t max_single_part_copy_size_, bool use_native_copy_) : max_single_part_upload_size(max_single_part_upload_size_) , min_bytes_for_seek(min_bytes_for_seek_) @@ -32,6 +33,7 @@ struct AzureObjectStorageSettings , max_single_download_retries(max_single_download_retries_) , list_object_keys_size(list_object_keys_size_) , max_upload_part_size(max_upload_part_size_) + , max_single_part_copy_size(max_single_part_copy_size_) , use_native_copy(use_native_copy_) { } @@ -46,6 +48,7 @@ struct AzureObjectStorageSettings size_t min_upload_part_size = 16 * 1024 * 1024; size_t max_upload_part_size = 5ULL * 1024 * 1024 * 1024; size_t max_part_number = 10000; + size_t max_single_part_copy_size = 256 * 1024 * 1024; bool use_native_copy = false; }; diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp index 537a5a191e7..ff4cfe62feb 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp @@ -33,10 +33,6 @@ namespace ErrorCodes extern const int AZURE_BLOB_STORAGE_ERROR; } - -size_t max_single_operation_copy_size = 256 * 1024 * 1024; - - namespace { class UploadHelper @@ -304,7 +300,7 @@ void copyAzureBlobStorageFile( auto block_blob_client_dest = dest_client->GetBlockBlobClient(dest_blob); auto source_uri = block_blob_client_src.GetUrl(); - if (size < max_single_operation_copy_size) + if (size < settings.get()->max_single_part_copy_size) { block_blob_client_dest.CopyFromUri(source_uri); } From 99a1b269d71054a1d4d1e59a55b229469652435c Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 29 Jan 2024 11:00:59 +0100 Subject: [PATCH 100/276] Removed unwanted setting --- src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 18b1a70defe..7d5c8f07a75 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -47,7 +47,6 @@ struct AzureObjectStorageSettings int list_object_keys_size = 1000; size_t min_upload_part_size = 16 * 1024 * 1024; size_t max_upload_part_size = 5ULL * 1024 * 1024 * 1024; - size_t max_part_number = 10000; size_t max_single_part_copy_size = 256 * 1024 * 1024; bool use_native_copy = false; }; From ce0ebd964519d0961d92318e8a171d5338365213 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 29 Jan 2024 11:14:19 +0100 Subject: [PATCH 101/276] Removed unwanted log lines --- src/Backups/BackupImpl.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index 0fb0d8cbda9..28a7d60b52c 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -535,7 +535,6 @@ void BackupImpl::checkBackupDoesntExist() const else file_name_to_check_existence = ".backup"; - LOG_INFO(&Poco::Logger::get("BackupImpl"), "checkBackupDoesntExist 1"); if (writer->fileExists(file_name_to_check_existence)) throw Exception(ErrorCodes::BACKUP_ALREADY_EXISTS, "Backup {} already exists", backup_name_for_logging); @@ -567,8 +566,6 @@ bool BackupImpl::checkLockFile(bool throw_if_failed) const if (throw_if_failed) { - LOG_INFO(&Poco::Logger::get("BackupImpl"), "checkLockFile"); - if (!writer->fileExists(lock_file_name)) { throw Exception( From 6bfa910d9ea403e91fb9be04573c73bfae77b4c4 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 29 Jan 2024 16:47:02 +0100 Subject: [PATCH 102/276] Fix merge --- src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 78a67f3e59a..8556f0237e3 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -47,12 +47,10 @@ struct AzureObjectStorageSettings size_t max_single_read_retries = 3; size_t max_single_download_retries = 3; int list_object_keys_size = 1000; -<<<<<<< HEAD size_t min_upload_part_size = 16 * 1024 * 1024; size_t max_upload_part_size = 5ULL * 1024 * 1024 * 1024; size_t max_single_part_copy_size = 256 * 1024 * 1024; bool use_native_copy = false; -======= size_t max_unexpected_write_error_retries = 4; >>>>>>> master }; From 0ded5800112f95f7b13ca8d060e743559ce787e6 Mon Sep 17 00:00:00 2001 From: flynn Date: Tue, 30 Jan 2024 04:03:27 +0000 Subject: [PATCH 103/276] Fix --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 14 +++++++++++++- .../replaceForPositionalArguments.cpp | 19 +++++++++++++++---- 2 files changed, 28 insertions(+), 5 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index c683214840b..fbabef87112 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -2170,7 +2170,19 @@ void QueryAnalyzer::replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_ else // Int64 { auto value = constant_node->getValue().get(); - pos = value > 0 ? value : projection_nodes.size() + value + 1; + if (value > 0) + pos = value; + else + { + if (static_cast(std::abs(value)) > projection_nodes.size()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Negtive positional argument number {} is out of bounds. Expected in range [-{}, -1]. In scope {}", + value, + projection_nodes.size(), + scope.scope_node->formatASTForErrorMessage()); + pos = projection_nodes.size() + value + 1; + } } if (!pos || pos > projection_nodes.size()) diff --git a/src/Interpreters/replaceForPositionalArguments.cpp b/src/Interpreters/replaceForPositionalArguments.cpp index bea87ad913a..c72cac25c9d 100644 --- a/src/Interpreters/replaceForPositionalArguments.cpp +++ b/src/Interpreters/replaceForPositionalArguments.cpp @@ -10,7 +10,8 @@ namespace DB namespace ErrorCodes { - extern const int ILLEGAL_TYPE_OF_ARGUMENT; +extern const int BAD_ARGUMENTS; +extern const int ILLEGAL_TYPE_OF_ARGUMENT; } bool replaceForPositionalArguments(ASTPtr & argument, const ASTSelectQuery * select_query, ASTSelectQuery::Expression expression) @@ -39,7 +40,18 @@ bool replaceForPositionalArguments(ASTPtr & argument, const ASTSelectQuery * sel else if (which == Field::Types::Int64) { auto value = ast_literal->value.get(); - pos = value > 0 ? value : columns.size() + value + 1; + if (value > 0) + pos = value; + else + { + if (static_cast(std::abs(value)) > columns.size()) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Negtive positional argument number {} is out of bounds. Expected in range [-{}, -1]", + value, + columns.size()); + pos = columns.size() + value + 1; + } } else { @@ -47,8 +59,7 @@ bool replaceForPositionalArguments(ASTPtr & argument, const ASTSelectQuery * sel } if (!pos || pos > columns.size()) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Positional argument out of bounds: {} (expected in range [1, {}]", pos, columns.size()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional argument out of bounds: {} (expected in range [1, {}]", pos, columns.size()); const auto & column = columns[--pos]; if (typeid_cast(column.get()) || typeid_cast(column.get())) From 4f12ca249d4d728c403f52c1d68edda68a9af286 Mon Sep 17 00:00:00 2001 From: flynn Date: Tue, 30 Jan 2024 07:01:07 +0000 Subject: [PATCH 104/276] Fix typo --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 +- src/Interpreters/replaceForPositionalArguments.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 54767c88993..d9434c878d2 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -2180,7 +2180,7 @@ void QueryAnalyzer::replaceNodesWithPositionalArguments(QueryTreeNodePtr & node_ if (static_cast(std::abs(value)) > projection_nodes.size()) throw Exception( ErrorCodes::BAD_ARGUMENTS, - "Negtive positional argument number {} is out of bounds. Expected in range [-{}, -1]. In scope {}", + "Negative positional argument number {} is out of bounds. Expected in range [-{}, -1]. In scope {}", value, projection_nodes.size(), scope.scope_node->formatASTForErrorMessage()); diff --git a/src/Interpreters/replaceForPositionalArguments.cpp b/src/Interpreters/replaceForPositionalArguments.cpp index c72cac25c9d..cceb0650fcd 100644 --- a/src/Interpreters/replaceForPositionalArguments.cpp +++ b/src/Interpreters/replaceForPositionalArguments.cpp @@ -47,7 +47,7 @@ bool replaceForPositionalArguments(ASTPtr & argument, const ASTSelectQuery * sel if (static_cast(std::abs(value)) > columns.size()) throw Exception( ErrorCodes::BAD_ARGUMENTS, - "Negtive positional argument number {} is out of bounds. Expected in range [-{}, -1]", + "Negative positional argument number {} is out of bounds. Expected in range [-{}, -1]", value, columns.size()); pos = columns.size() + value + 1; From b6aecc11498eef490eddfcbec2f02da738e06b6f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 30 Jan 2024 19:35:26 +0000 Subject: [PATCH 105/276] Update reference file --- tests/queries/0_stateless/02479_mysql_connect_to_self.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02479_mysql_connect_to_self.reference b/tests/queries/0_stateless/02479_mysql_connect_to_self.reference index 6838dacc3b3..8057b945c5a 100644 --- a/tests/queries/0_stateless/02479_mysql_connect_to_self.reference +++ b/tests/queries/0_stateless/02479_mysql_connect_to_self.reference @@ -67,6 +67,6 @@ SELECT __table1.a AS a, __table1.b AS b, __table1.c AS c -FROM mysql(\'127.0.0.1:9004\', \'default\', foo, \'default\', \'\', SETTINGS connection_wait_timeout = 123, connect_timeout = 40123002, read_write_timeout = 40123001, connection_pool_size = 3) AS __table1 +FROM mysql(\'127.0.0.1:9004\', _CAST(\'default\', \'String\'), foo, \'default\', \'\', SETTINGS connection_wait_timeout = 123, connect_timeout = 40123002, read_write_timeout = 40123001, connection_pool_size = 3) AS __table1 --- 5 From 3108a988dea5807a177bb0e02489bbf9da2c0aa7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 1 Feb 2024 15:53:13 +0100 Subject: [PATCH 106/276] Fix possible uncaught exception during distributed query cancellation Cancellation of distributed queries may throw (i.e. some timeout), and in case of pipeline had not been properly created properly (EXCEPTION_BEFORE_START) cancel will not be sent, so cancellation will be done from dtor and will throw.
stacktrace ``` BaseDaemon: (version 23.9.2.56 (official build), build id: 76109A79FA62B9BC630A6C39438DEA7D28147B68, git hash: a1bf3f1de55abf2354dc498ffbee270be043d633) (from thread 51895) Terminate called for uncaught exception: BaseDaemon: Code: 209. DB::NetException: Timeout exceeded while reading from socket (socket (10.61.1.50:9000), receive timeout 20000 ms). (SOCKET_TIMEOUT), Stack trace (when copying this message, always include the lines below): BaseDaemon: BaseDaemon: 0. ./build_docker/./src/Common/Exception.cpp:98: DB::Exception::Exception(DB::Exception::MessageMasked&&, int, bool) @ 0x000000000c741d97 in /usr/lib/debug/usr/bin/clickhouse.debug BaseDaemon: 1. ./build_docker/./contrib/llvm-project/libcxx/include/string:1499: DB::NetException::NetException(int, String const&) @ 0x000000001148ace7 in /usr/lib/debug/usr/bin/clickhouse.debug BaseDaemon: 2. ./build_docker/./src/QueryPipeline/RemoteQueryExecutorReadContext.cpp:101: DB::RemoteQueryExecutorReadContext::checkTimeout(bool) @ 0x000000001148a70a in /usr/lib/debug/usr/bin/clickhouse.debug BaseDaemon: 3. ./build_docker/./src/Common/Fiber.h:42: DB::RemoteQueryExecutorReadContext::cancelBefore() @ 0x000000001148aded in /usr/lib/debug/usr/bin/clickhouse.debug BaseDaemon: 4. ./build_docker/./src/Common/Fiber.h:27: DB::AsyncTaskExecutor::cancel() @ 0x000000001148969d in /usr/lib/debug/usr/bin/clickhouse.debug BaseDaemon: 5. ./build_docker/./src/QueryPipeline/RemoteQueryExecutor.cpp:169: DB::RemoteQueryExecutor::~RemoteQueryExecutor() @ 0x000000001147a9d0 in /usr/lib/debug/usr/bin/clickhouse.debug BaseDaemon: 6. ./build_docker/./contrib/llvm-project/libcxx/src/include/atomic_support.h:74: std::__shared_ptr_emplace>::__on_zero_shared() @ 0x000000000c94a4a1 in /usr/lib/debug/usr/bin/clickhou BaseDaemon: 7. ./build_docker/./contrib/llvm-project/libcxx/src/include/atomic_support.h:74: std::__shared_ptr_emplace, std::allocator>>, std::allocator>>>>::__on_zero_shared() @ 0x00000000114603a1 in /usr/lib/debug/usr/bin/clickhouse.debug BaseDaemon: 8. ./build_docker/./contrib/llvm-project/libcxx/src/include/atomic_support.h:74: DB::QueryPipeline::~QueryPipeline() @ 0x000000001146347a in /usr/lib/debug/usr/bin/clickhouse.debug BaseDaemon: 9. ./build_docker/./src/QueryPipeline/QueryPipeline.cpp:709: DB::QueryPipeline::reset() @ 0x0000000011468f78 in /usr/lib/debug/usr/bin/clickhouse.debug BaseDaemon: 10. ./build_docker/./src/Interpreters/executeQuery.cpp:1427: DB::executeQuery(DB::ReadBuffer&, DB::WriteBuffer&, bool, std::shared_ptr, std::function, std::optional const&, std::function) @ 0x000000001249aa8b in /usr/lib/debug/usr/bin/clickhouse.debug BaseDaemon: 11. ./build_docker/./contrib/llvm-project/libcxx/include/__functional/function.h:818: DB::HTTPHandler::processQuery(DB::HTTPServerRequest&, DB::HTMLForm&, DB::HTTPServerResponse&, DB::HTTPHandler::Output&, std::optional&) @ 0x00000000132bddb4 in /usr/lib/debug/usr/bin/clickhouse.debug BaseDaemon: 12. ./build_docker/./contrib/llvm-project/libcxx/include/__memory/unique_ptr.h:290: DB::HTTPHandler::handleRequest(DB::HTTPServerRequest&, DB::HTTPServerResponse&) @ 0x00000000132c23a4 in /usr/lib/debug/usr/bin/clickhouse.debug ```
Signed-off-by: Azat Khuzhin --- src/QueryPipeline/RemoteQueryExecutor.cpp | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 136a3bb09c6..e053bd2703a 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -182,7 +182,19 @@ RemoteQueryExecutor::~RemoteQueryExecutor() { /// Set was_cancelled, so the query won't be sent after creating connections. was_cancelled = true; - read_context->cancel(); + + /// Cancellation may throw (i.e. some timeout), and in case of pipeline + /// had not been properly created properly (EXCEPTION_BEFORE_START) + /// cancel will not be sent, so cancellation will be done from dtor and + /// will throw. + try + { + read_context->cancel(); + } + catch (...) + { + tryLogCurrentException(log ? log : getLogger("RemoteQueryExecutor")); + } } /** If interrupted in the middle of the loop of communication with replicas, then interrupt From 2b786f7c6de22ca5ee01613a529a35306b391ab0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 1 Feb 2024 20:56:31 +0100 Subject: [PATCH 107/276] Wrap disconnect into try/catch in RemoteQueryExecutor dtor as well Signed-off-by: Azat Khuzhin --- src/QueryPipeline/RemoteQueryExecutor.cpp | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index e053bd2703a..ab52dee2cf9 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -202,7 +202,17 @@ RemoteQueryExecutor::~RemoteQueryExecutor() * these connections did not remain hanging in the out-of-sync state. */ if (established || (isQueryPending() && connections)) - connections->disconnect(); + { + /// May also throw (so as cancel() above) + try + { + connections->disconnect(); + } + catch (...) + { + tryLogCurrentException(log ? log : getLogger("RemoteQueryExecutor")); + } + } } /** If we receive a block with slightly different column types, or with excessive columns, From 82d7b2214407c68b96334d002f01ed68937e07f4 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 1 Feb 2024 21:06:29 +0000 Subject: [PATCH 108/276] Parallel replicas: better replicas failover --- .../ClusterProxy/executeQuery.cpp | 15 +++--- src/Processors/QueryPlan/ReadFromRemote.cpp | 52 ++++++------------- src/Processors/QueryPlan/ReadFromRemote.h | 5 +- src/QueryPipeline/RemoteQueryExecutor.cpp | 44 ++++++++++++++++ src/QueryPipeline/RemoteQueryExecutor.h | 13 ++++- 5 files changed, 82 insertions(+), 47 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 35451e1d774..023ed6c7b61 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -374,12 +374,12 @@ void executeQueryWithParallelReplicas( shard_num = column->getUInt(0); } - ClusterPtr new_cluster; + const auto shard_count = not_optimized_cluster->getShardCount(); + ClusterPtr new_cluster = not_optimized_cluster; /// if got valid shard_num from query initiator, then parallel replicas scope is the specified shard /// shards are numbered in order of appearance in the cluster config if (shard_num > 0) { - const auto shard_count = not_optimized_cluster->getShardCount(); if (shard_num > shard_count) throw Exception( ErrorCodes::LOGICAL_ERROR, @@ -395,17 +395,16 @@ void executeQueryWithParallelReplicas( // get cluster for shard specified by shard_num // shard_num is 1-based, but getClusterWithSingleShard expects 0-based index - auto single_shard_cluster = not_optimized_cluster->getClusterWithSingleShard(shard_num - 1); - // convert cluster to representation expected by parallel replicas - new_cluster = single_shard_cluster->getClusterWithReplicasAsShards(settings, settings.max_parallel_replicas); + new_cluster = not_optimized_cluster->getClusterWithSingleShard(shard_num - 1); } else { - new_cluster = not_optimized_cluster->getClusterWithReplicasAsShards(settings, settings.max_parallel_replicas); + // todo: add error and exception for this case + chassert(not_optimized_cluster->getShardCount() == 1); } - auto coordinator - = std::make_shared(new_cluster->getShardCount(), settings.parallel_replicas_mark_segment_size); + auto coordinator = std::make_shared( + new_cluster->getShardsInfo().begin()->getAllNodeCount(), settings.parallel_replicas_mark_segment_size); auto external_tables = new_context->getExternalTables(); auto read_from_remote = std::make_unique( query_ast, diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 4dd79903965..fcdb7cd4a70 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -399,51 +399,33 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder const Settings & current_settings = context->getSettingsRef(); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings); + const auto & shard = cluster->getShardsInfo().at(0); size_t all_replicas_count = current_settings.max_parallel_replicas; - if (all_replicas_count > cluster->getShardsInfo().size()) + if (all_replicas_count > shard.getAllNodeCount()) { - LOG_INFO(getLogger("ReadFromParallelRemoteReplicasStep"), - "The number of replicas requested ({}) is bigger than the real number available in the cluster ({}). "\ - "Will use the latter number to execute the query.", current_settings.max_parallel_replicas, cluster->getShardsInfo().size()); - all_replicas_count = cluster->getShardsInfo().size(); + LOG_INFO( + getLogger("ReadFromParallelRemoteReplicasStep"), + "The number of replicas requested ({}) is bigger than the real number available in the cluster ({}). " + "Will use the latter number to execute the query.", + current_settings.max_parallel_replicas, + shard.getAllNodeCount()); + all_replicas_count = shard.getAllNodeCount(); } - /// Find local shard. It might happen that there is no local shard, but that's fine - for (const auto & shard: cluster->getShardsInfo()) + chassert(cluster->getShardCount() == 1); + auto shuffled_pool = shard.pool->getShuffledPools(current_settings); + shuffled_pool.resize(all_replicas_count); + + for (size_t i=0; i < all_replicas_count; ++i) { - if (shard.isLocal()) - { - IConnections::ReplicaInfo replica_info - { - .all_replicas_count = all_replicas_count, - /// `shard_num` will be equal to the number of the given replica in the cluster (set by `Cluster::getClusterWithReplicasAsShards`). - /// we should use this number specifically because efficiency of data distribution by consistent hash depends on it. - .number_of_current_replica = shard.shard_num - 1, - }; - - addPipeForSingeReplica(pipes, shard.pool, replica_info); - } - } - - auto current_shard = cluster->getShardsInfo().begin(); - while (pipes.size() != all_replicas_count) - { - if (current_shard->isLocal()) - { - ++current_shard; - continue; - } - IConnections::ReplicaInfo replica_info { .all_replicas_count = all_replicas_count, - /// `shard_num` will be equal to the number of the given replica in the cluster (set by `Cluster::getClusterWithReplicasAsShards`). /// we should use this number specifically because efficiency of data distribution by consistent hash depends on it. - .number_of_current_replica = current_shard->shard_num - 1, + .number_of_current_replica = i, }; - addPipeForSingeReplica(pipes, current_shard->pool, replica_info); - ++current_shard; + addPipeForSingeReplica(pipes, shuffled_pool[i].pool, replica_info); } auto pipe = Pipe::unitePipes(std::move(pipes)); @@ -456,7 +438,7 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder } -void ReadFromParallelRemoteReplicasStep::addPipeForSingeReplica(Pipes & pipes, std::shared_ptr pool, IConnections::ReplicaInfo replica_info) +void ReadFromParallelRemoteReplicasStep::addPipeForSingeReplica(Pipes & pipes, IConnectionPool* pool, IConnections::ReplicaInfo replica_info) { bool add_agg_info = stage == QueryProcessingStage::WithMergeableState; bool add_totals = false; diff --git a/src/Processors/QueryPlan/ReadFromRemote.h b/src/Processors/QueryPlan/ReadFromRemote.h index f853a12910b..07443220c8d 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.h +++ b/src/Processors/QueryPlan/ReadFromRemote.h @@ -10,8 +10,7 @@ namespace DB { -class ConnectionPoolWithFailover; -using ConnectionPoolWithFailoverPtr = std::shared_ptr; +class IConnectionPool; class Throttler; using ThrottlerPtr = std::shared_ptr; @@ -92,7 +91,7 @@ public: private: - void addPipeForSingeReplica(Pipes & pipes, std::shared_ptr pool, IConnections::ReplicaInfo replica_info); + void addPipeForSingeReplica(Pipes & pipes, IConnectionPool* pool, IConnections::ReplicaInfo replica_info); ClusterPtr cluster; ASTPtr query_ast; diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 136a3bb09c6..7f25c2331c3 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -62,6 +62,50 @@ RemoteQueryExecutor::RemoteQueryExecutor( { } +RemoteQueryExecutor::RemoteQueryExecutor( + IConnectionPool * pool, + const String & query_, + const Block & header_, + ContextPtr context_, + ThrottlerPtr throttler, + const Scalars & scalars_, + const Tables & external_tables_, + QueryProcessingStage::Enum stage_, + std::optional extension_) + : RemoteQueryExecutor(query_, header_, context_, scalars_, external_tables_, stage_, extension_) +{ + create_connections = [this, pool, throttler, extension_](AsyncCallback) + { + const Settings & current_settings = context->getSettingsRef(); + auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings); + + ConnectionPoolWithFailover::TryResult result; + std::string fail_message; + if (main_table) + { + auto table_name = main_table.getQualifiedName(); + + ConnectionEstablisher connection_establisher(pool, &timeouts, current_settings, log, &table_name); + connection_establisher.run(result, fail_message); + } + else + { + ConnectionEstablisher connection_establisher(pool, &timeouts, current_settings, log, nullptr); + connection_establisher.run(result, fail_message); + } + + std::vector connection_entries; + if (!result.entry.isNull() && result.is_usable) + connection_entries.emplace_back(std::move(result.entry)); + + auto res = std::make_unique(std::move(connection_entries), context->getSettingsRef(), throttler); + if (extension_ && extension_->replica_info) + res->setReplicaInfo(*extension_->replica_info); + + return res; + }; +} + RemoteQueryExecutor::RemoteQueryExecutor( Connection & connection, const String & query_, const Block & header_, ContextPtr context_, diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index 444f1258f3e..cc3291313a8 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -50,9 +50,20 @@ public: std::shared_ptr task_iterator = nullptr; std::shared_ptr parallel_reading_coordinator = nullptr; std::optional replica_info = {}; - GetPriorityForLoadBalancing::Func priority_func; }; + /// Takes a connection pool to a node (not cluster) + RemoteQueryExecutor( + IConnectionPool * pool, + const String & query_, + const Block & header_, + ContextPtr context_, + ThrottlerPtr throttler = nullptr, + const Scalars & scalars_ = Scalars(), + const Tables & external_tables_ = Tables(), + QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete, + std::optional extension_ = std::nullopt); + /// Takes already set connection. RemoteQueryExecutor( Connection & connection, From 4b77258341cfc35bfeb2ad63a6747f34cfb0c54b Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 2 Feb 2024 11:28:35 +0000 Subject: [PATCH 109/276] Fix test --- src/QueryPipeline/RemoteQueryExecutor.h | 2 +- .../02769_parallel_replicas_unavailable_shards.sql | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index cc3291313a8..1eee0ac664f 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -52,7 +52,7 @@ public: std::optional replica_info = {}; }; - /// Takes a connection pool to a node (not cluster) + /// Takes a connection pool for a node (not cluster) RemoteQueryExecutor( IConnectionPool * pool, const String & query_, diff --git a/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql b/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql index 38d592201e3..be200353f06 100644 --- a/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql +++ b/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql @@ -6,10 +6,11 @@ SYSTEM FLUSH LOGS; SET allow_experimental_parallel_reading_from_replicas=2, max_parallel_replicas=11, cluster_for_parallel_replicas='parallel_replicas', parallel_replicas_for_non_replicated_merge_tree=1; SET send_logs_level='error'; -SELECT count() FROM test_parallel_replicas_unavailable_shards WHERE NOT ignore(*); +SELECT count() FROM test_parallel_replicas_unavailable_shards WHERE NOT ignore(*) SETTINGS log_comment = '02769_7b513191-5082-4073-8568-53b86a49da79'; SYSTEM FLUSH LOGS; -SELECT count() > 0 FROM system.text_log WHERE yesterday() <= event_date AND message LIKE '%Replica number 10 is unavailable%'; +SET allow_experimental_parallel_reading_from_replicas=0; +SELECT count() FROM system.text_log WHERE yesterday() <= event_date AND query_id in (select query_id from system.query_log where log_comment = '02769_7b513191-5082-4073-8568-53b86a49da79' and current_database = currentDatabase()) and message LIKE '%Replica number % is unavailable%'; DROP TABLE test_parallel_replicas_unavailable_shards; From 8ebd7a7952dcace13bf71cf29431f992869ad909 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 2 Feb 2024 13:39:06 +0000 Subject: [PATCH 110/276] Formatting --- src/Processors/QueryPlan/ReadFromRemote.cpp | 12 ++++++++++-- src/Processors/QueryPlan/ReadFromRemote.h | 3 +-- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index fcdb7cd4a70..cab3ae72678 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -438,7 +438,8 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder } -void ReadFromParallelRemoteReplicasStep::addPipeForSingeReplica(Pipes & pipes, IConnectionPool* pool, IConnections::ReplicaInfo replica_info) +void ReadFromParallelRemoteReplicasStep::addPipeForSingeReplica( + Pipes & pipes, IConnectionPool * pool, IConnections::ReplicaInfo replica_info) { bool add_agg_info = stage == QueryProcessingStage::WithMergeableState; bool add_totals = false; @@ -458,7 +459,14 @@ void ReadFromParallelRemoteReplicasStep::addPipeForSingeReplica(Pipes & pipes, I assert(output_stream); auto remote_query_executor = std::make_shared( - pool, query_string, output_stream->header, context, throttler, scalars, external_tables, stage, + pool, + query_string, + output_stream->header, + context, + throttler, + scalars, + external_tables, + stage, RemoteQueryExecutor::Extension{.parallel_reading_coordinator = coordinator, .replica_info = std::move(replica_info)}); remote_query_executor->setLogger(log); diff --git a/src/Processors/QueryPlan/ReadFromRemote.h b/src/Processors/QueryPlan/ReadFromRemote.h index 07443220c8d..eb3bcd12cc3 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.h +++ b/src/Processors/QueryPlan/ReadFromRemote.h @@ -90,8 +90,7 @@ public: void enforceAggregationInOrder(); private: - - void addPipeForSingeReplica(Pipes & pipes, IConnectionPool* pool, IConnections::ReplicaInfo replica_info); + void addPipeForSingeReplica(Pipes & pipes, IConnectionPool * pool, IConnections::ReplicaInfo replica_info); ClusterPtr cluster; ASTPtr query_ast; From ea2dad181d5c91bddd63fc1ca64636123053fb8a Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 2 Feb 2024 15:05:14 +0000 Subject: [PATCH 111/276] Fix non linux builds --- src/QueryPipeline/RemoteQueryExecutor.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 7f25c2331c3..90bd4bfdfdf 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -4,7 +4,7 @@ #include #include -#include "Core/Protocol.h" +#include #include #include #include @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include From 736314015003f999303c9e025fbf624f973af9d8 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 2 Feb 2024 22:28:06 +0100 Subject: [PATCH 112/276] remove unneeded field --- src/Processors/Transforms/AggregatingTransform.h | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index e05528afdc7..e167acde067 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -71,16 +71,12 @@ struct AggregatingTransformParams struct ManyAggregatedData { ManyAggregatedDataVariants variants; - std::vector> mutexes; std::atomic num_finished = 0; - explicit ManyAggregatedData(size_t num_threads = 0) : variants(num_threads), mutexes(num_threads) + explicit ManyAggregatedData(size_t num_threads = 0) : variants(num_threads) { for (auto & elem : variants) elem = std::make_shared(); - - for (auto & mut : mutexes) - mut = std::make_unique(); } ~ManyAggregatedData() From b0994c5fa79ec6ba8dd14e028a6e2731d4e5f031 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sun, 4 Feb 2024 11:28:20 +0100 Subject: [PATCH 113/276] Addressed comments, added test for named collection --- src/Backups/BackupIO_AzureBlobStorage.cpp | 10 +- src/Backups/BackupImpl.cpp | 1 - .../registerBackupEngineAzureBlobStorage.cpp | 75 +++++++---- .../AzureBlobStorage/AzureObjectStorage.cpp | 1 + .../AzureBlobStorage/AzureObjectStorage.h | 1 - .../copyAzureBlobStorageFile.cpp | 19 +-- src/Storages/StorageAzureBlob.cpp | 11 -- src/Storages/StorageAzureBlob.h | 1 - .../test.py | 123 ++++++++++++++++-- 9 files changed, 179 insertions(+), 63 deletions(-) diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index d99f296cca1..27928e871ce 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -33,7 +33,7 @@ BackupReaderAzureBlobStorage::BackupReaderAzureBlobStorage( const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_) - : BackupReaderDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupReaderAzureBlobStorage")) + : BackupReaderDefault(read_settings_, write_settings_, getLogger("BackupReaderAzureBlobStorage")) , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, configuration_.container, false, false} , configuration(configuration_) { @@ -96,8 +96,6 @@ std::unique_ptr BackupReaderAzureBlobStorage::readFile(const void BackupReaderAzureBlobStorage::copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) { - LOG_INFO(&Poco::Logger::get("BackupReaderAzureBlobStorage"), "Enter copyFileToDisk"); - auto destination_data_source_description = destination_disk->getDataSourceDescription(); if ((destination_data_source_description.type == DataSourceType::ObjectStorage) && (destination_data_source_description.object_storage_type == ObjectStorageType::Azure) @@ -143,7 +141,7 @@ BackupWriterAzureBlobStorage::BackupWriterAzureBlobStorage( const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_) - : BackupWriterDefault(read_settings_, write_settings_, &Poco::Logger::get("BackupWriterAzureBlobStorage")) + : BackupWriterDefault(read_settings_, write_settings_, getLogger("BackupWriterAzureBlobStorage")) , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, configuration_.container, false, false} , configuration(configuration_) { @@ -225,14 +223,11 @@ bool BackupWriterAzureBlobStorage::fileExists(const String & file_name) { key = file_name; } - LOG_INFO(&Poco::Logger::get("BackupWriterAzureBlobStorage"), "Result fileExists {} ", object_storage->exists(StoredObject(key))); - return object_storage->exists(StoredObject(key)); } UInt64 BackupWriterAzureBlobStorage::getFileSize(const String & file_name) { - LOG_INFO(&Poco::Logger::get("BackupWriterAzureBlobStorage"), "Enter getFileSize"); String key; if (startsWith(file_name, ".")) { @@ -281,6 +276,7 @@ std::unique_ptr BackupWriterAzureBlobStorage::writeFile(const Strin client, key, settings.get()->max_single_part_upload_size, + settings.get()->max_unexpected_write_error_retries, DBMS_DEFAULT_BUFFER_SIZE, write_settings); } diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index 28a7d60b52c..8a4ed31bfd7 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -542,7 +542,6 @@ void BackupImpl::checkBackupDoesntExist() const if (!is_internal_backup) { assert(!lock_file_name.empty()); - LOG_INFO(&Poco::Logger::get("BackupImpl"), "checkBackupDoesntExist 2"); if (writer->fileExists(lock_file_name)) throw Exception(ErrorCodes::BACKUP_ALREADY_EXISTS, "Backup {} is being written already", backup_name_for_logging); } diff --git a/src/Backups/registerBackupEngineAzureBlobStorage.cpp b/src/Backups/registerBackupEngineAzureBlobStorage.cpp index 3480ea75f1f..48f66569304 100644 --- a/src/Backups/registerBackupEngineAzureBlobStorage.cpp +++ b/src/Backups/registerBackupEngineAzureBlobStorage.cpp @@ -49,40 +49,65 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) const String & id_arg = params.backup_info.id_arg; const auto & args = params.backup_info.args; - LOG_INFO(&Poco::Logger::get("registerBackupEngineAzureBlobStorage"), "Begin id_arg={} args.size={}", id_arg, args.size()); - StorageAzureBlob::Configuration configuration; - if (args.size() == 3) + if (!id_arg.empty()) { - configuration.connection_url = args[0].safeGet(); - configuration.is_connection_string = true; + const auto & config = params.context->getConfigRef(); + auto config_prefix = "named_collections." + id_arg; - configuration.container = args[1].safeGet(); - configuration.blob_path = args[2].safeGet(); + if (!config.has(config_prefix)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no collection named `{}` in config", id_arg); - LOG_TRACE(&Poco::Logger::get("registerBackupEngineAzureBlobStorage"), "configuration.connection_url = {}" - "configuration.container = {}" - "configuration.blob_path = {}", - configuration.connection_url, configuration.container, configuration.blob_path); - } - else if (args.size() == 5) - { - configuration.connection_url = args[0].safeGet(); - configuration.is_connection_string = false; + if (config.has(config_prefix + ".connection_string")) + { + configuration.connection_url = config.getString(config_prefix + ".connection_string"); + configuration.is_connection_string = true; + configuration.container = config.getString(config_prefix + ".container"); + } + else + { + configuration.connection_url = config.getString(config_prefix + ".storage_account_url"); + configuration.is_connection_string = false; + configuration.container = config.getString(config_prefix + ".container"); + configuration.account_name = config.getString(config_prefix + ".account_name"); + configuration.account_key = config.getString(config_prefix + ".account_key"); + } - configuration.container = args[1].safeGet(); - configuration.blob_path = args[2].safeGet(); - configuration.account_name = args[3].safeGet(); - configuration.account_key = args[4].safeGet(); + if (args.size() > 1) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Backup AzureBlobStorage requires 1 or 2 arguments: named_collection, [filename]"); + + if (args.size() == 1) + configuration.blob_path = args[0].safeGet(); } else { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Backup AzureBlobStorage requires 3 or 5 arguments: connection string>/(); + configuration.is_connection_string = true; + configuration.container = args[1].safeGet(); + configuration.blob_path = args[2].safeGet(); + } + else if (args.size() == 5) + { + configuration.connection_url = args[0].safeGet(); + configuration.is_connection_string = false; + + configuration.container = args[1].safeGet(); + configuration.blob_path = args[2].safeGet(); + configuration.account_name = args[3].safeGet(); + configuration.account_key = args[4].safeGet(); + + } + else + { + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Backup AzureBlobStorage requires 3 or 5 arguments: connection string>/>>>>>> master }; using AzureClient = Azure::Storage::Blobs::BlobContainerClient; diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp index ff4cfe62feb..114a970384b 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp @@ -80,7 +80,8 @@ namespace struct UploadPartTask { - std::unique_ptr read_buffer = nullptr; + size_t part_offset; + size_t part_size; std::vector block_ids; bool is_finished = false; std::exception_ptr exception; @@ -182,7 +183,8 @@ namespace try { - task->read_buffer = std::make_unique(create_read_buffer(), part_offset, part_size); + task->part_offset = part_offset; + task->part_size = part_size; schedule([this, task, task_finish_notify]() { @@ -206,7 +208,8 @@ namespace else { UploadPartTask task; - task.read_buffer = std::make_unique(create_read_buffer(), part_offset, part_size); + task.part_offset = part_offset; + task.part_size = part_size; processUploadPartRequest(task); block_ids.insert(block_ids.end(),task.block_ids.begin(), task.block_ids.end()); } @@ -219,17 +222,17 @@ namespace ProfileEvents::increment(ProfileEvents::DiskAzureUploadPart); auto block_blob_client = client->GetBlockBlobClient(dest_blob); - - while (!task.read_buffer->eof()) + auto read_buffer = std::make_unique(create_read_buffer(), task.part_offset, task.part_size); + while (!read_buffer->eof()) { - auto size = task.read_buffer->available(); + auto size = read_buffer->available(); if (size > 0) { auto block_id = getRandomASCIIString(64); - Azure::Core::IO::MemoryBodyStream memory(reinterpret_cast(task.read_buffer->position()), size); + Azure::Core::IO::MemoryBodyStream memory(reinterpret_cast(read_buffer->position()), size); block_blob_client.StageBlock(block_id, memory); task.block_ids.emplace_back(block_id); - task.read_buffer->ignore(size); + read_buffer->ignore(size); LOG_TRACE(log, "Writing part. Container: {}, Blob: {}, block_id: {}", dest_container_for_logging, dest_blob, block_id); } } diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 67d67ea3fae..c09db0bfb7b 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -253,17 +253,6 @@ AzureObjectStorage::SettingsPtr StorageAzureBlob::createSettings(ContextPtr loca return settings_ptr; } -std::shared_ptr StorageAzureBlob::createSettingsAsSharedPtr(ContextPtr local_context) -{ - const auto & context_settings = local_context->getSettingsRef(); - auto settings_ptr = std::make_shared(); - settings_ptr->max_single_part_upload_size = context_settings.azure_max_single_part_upload_size; - settings_ptr->max_single_read_retries = context_settings.azure_max_single_read_retries; - settings_ptr->list_object_keys_size = static_cast(context_settings.azure_list_object_keys_size); - - return settings_ptr; -} - void registerStorageAzureBlob(StorageFactory & factory) { factory.registerStorage("AzureBlobStorage", [](const StorageFactory::Arguments & args) diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h index 196983522bf..6fc3c5ce592 100644 --- a/src/Storages/StorageAzureBlob.h +++ b/src/Storages/StorageAzureBlob.h @@ -72,7 +72,6 @@ public: static AzureClientPtr createClient(StorageAzureBlob::Configuration configuration, bool is_read_only); static AzureObjectStorage::SettingsPtr createSettings(ContextPtr local_context); - static std::shared_ptr createSettingsAsSharedPtr(ContextPtr local_context); static void processNamedCollectionResult(StorageAzureBlob::Configuration & configuration, const NamedCollection & collection); diff --git a/tests/integration/test_backup_restore_azure_blob_storage/test.py b/tests/integration/test_backup_restore_azure_blob_storage/test.py index 06c18d7468f..22aff39ce87 100644 --- a/tests/integration/test_backup_restore_azure_blob_storage/test.py +++ b/tests/integration/test_backup_restore_azure_blob_storage/test.py @@ -18,12 +18,45 @@ from helpers.mock_servers import start_mock_servers from helpers.test_tools import exec_query_with_retry +def generate_cluster_def(port): + path = os.path.join( + os.path.dirname(os.path.realpath(__file__)), + "./_gen/named_collections.xml", + ) + os.makedirs(os.path.dirname(path), exist_ok=True) + with open(path, "w") as f: + f.write( + f""" + + + DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:{port}/devstoreaccount1; + cont + CSV + + + http://azurite1:{port}/devstoreaccount1 + cont + CSV + devstoreaccount1 + Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw== + + + +""" + ) + return path + + + @pytest.fixture(scope="module") def cluster(): try: cluster = ClickHouseCluster(__file__) + port = cluster.azurite_port + path = generate_cluster_def(port) cluster.add_instance( "node", + main_configs=[path], with_azurite=True, ) cluster.start() @@ -123,15 +156,6 @@ def delete_all_files(cluster): yield - -def test_create_table_connection_string(cluster): - node = cluster.instances["node"] - azure_query( - node, - f"CREATE TABLE test_create_table_conn_string (key UInt64, data String) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont', 'test_create_connection_string', 'CSV')", - ) - - def test_backup_restore(cluster): node = cluster.instances["node"] port = cluster.env_variables["AZURITE_PORT"] @@ -159,3 +183,84 @@ def test_backup_restore(cluster): azure_query(node, f"SELECT * from test_simple_write_connection_string_restored") == "1\ta\n" ) + +def test_backup_restore_diff_container(cluster): + node = cluster.instances["node"] + port = cluster.env_variables["AZURITE_PORT"] + azure_query( + node, + f"CREATE TABLE test_simple_write_connection_string_cont1 (key UInt64, data String) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont', 'test_simple_write_c_cont1.csv', 'CSV')", + ) + azure_query( + node, f"INSERT INTO test_simple_write_connection_string_cont1 VALUES (1, 'a')" + ) + backup_destination = f"AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont1', 'test_simple_write_c_backup_cont1.csv')" + azure_query( + node, + f"BACKUP TABLE test_simple_write_connection_string_cont1 TO {backup_destination}", + ) + azure_query( + node, + f"RESTORE TABLE test_simple_write_connection_string_cont1 AS test_simple_write_connection_string_restored_cont1 FROM {backup_destination};", + ) + assert ( + azure_query(node, f"SELECT * from test_simple_write_connection_string_restored_cont1") + == "1\ta\n" + ) + + +def test_backup_restore_with_named_collection_azure_conf1(cluster): + node = cluster.instances["node"] + port = cluster.env_variables["AZURITE_PORT"] + azure_query( + node, + f"CREATE TABLE test_write_connection_string (key UInt64, data String) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont', 'test_simple_write.csv', 'CSV')", + ) + azure_query( + node, f"INSERT INTO test_write_connection_string VALUES (1, 'a')" + ) + print(get_azure_file_content("test_simple_write.csv", port)) + assert get_azure_file_content("test_simple_write.csv", port) == '1,"a"\n' + + backup_destination = f"AzureBlobStorage(azure_conf1, 'test_simple_write_nc_backup.csv')" + azure_query( + node, + f"BACKUP TABLE test_write_connection_string TO {backup_destination}", + ) + print(get_azure_file_content("test_simple_write_nc_backup.csv.backup", port)) + azure_query( + node, + f"RESTORE TABLE test_write_connection_string AS test_write_connection_string_restored FROM {backup_destination};", + ) + assert ( + azure_query(node, f"SELECT * from test_write_connection_string_restored") + == "1\ta\n" + ) + +def test_backup_restore_with_named_collection_azure_conf2(cluster): + node = cluster.instances["node"] + port = cluster.env_variables["AZURITE_PORT"] + azure_query( + node, + f"CREATE TABLE test_write_connection_string_2 (key UInt64, data String) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont', 'test_simple_write_2.csv', 'CSV')", + ) + azure_query( + node, f"INSERT INTO test_write_connection_string_2 VALUES (1, 'a')" + ) + print(get_azure_file_content("test_simple_write_2.csv", port)) + assert get_azure_file_content("test_simple_write_2.csv", port) == '1,"a"\n' + + backup_destination = f"AzureBlobStorage(azure_conf2, 'test_simple_write_nc_backup_2.csv')" + azure_query( + node, + f"BACKUP TABLE test_write_connection_string_2 TO {backup_destination}", + ) + print(get_azure_file_content("test_simple_write_nc_backup_2.csv.backup", port)) + azure_query( + node, + f"RESTORE TABLE test_write_connection_string_2 AS test_write_connection_string_restored_2 FROM {backup_destination};", + ) + assert ( + azure_query(node, f"SELECT * from test_write_connection_string_restored_2") + == "1\ta\n" + ) From ce6df0fb137a16c574f5be561205bd171e9ce3a5 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sun, 4 Feb 2024 10:37:05 +0000 Subject: [PATCH 114/276] Automatic style fix --- .../test.py | 34 +++++++++++-------- 1 file changed, 19 insertions(+), 15 deletions(-) diff --git a/tests/integration/test_backup_restore_azure_blob_storage/test.py b/tests/integration/test_backup_restore_azure_blob_storage/test.py index 22aff39ce87..a7c7b439560 100644 --- a/tests/integration/test_backup_restore_azure_blob_storage/test.py +++ b/tests/integration/test_backup_restore_azure_blob_storage/test.py @@ -47,7 +47,6 @@ def generate_cluster_def(port): return path - @pytest.fixture(scope="module") def cluster(): try: @@ -156,6 +155,7 @@ def delete_all_files(cluster): yield + def test_backup_restore(cluster): node = cluster.instances["node"] port = cluster.env_variables["AZURITE_PORT"] @@ -184,6 +184,7 @@ def test_backup_restore(cluster): == "1\ta\n" ) + def test_backup_restore_diff_container(cluster): node = cluster.instances["node"] port = cluster.env_variables["AZURITE_PORT"] @@ -204,8 +205,10 @@ def test_backup_restore_diff_container(cluster): f"RESTORE TABLE test_simple_write_connection_string_cont1 AS test_simple_write_connection_string_restored_cont1 FROM {backup_destination};", ) assert ( - azure_query(node, f"SELECT * from test_simple_write_connection_string_restored_cont1") - == "1\ta\n" + azure_query( + node, f"SELECT * from test_simple_write_connection_string_restored_cont1" + ) + == "1\ta\n" ) @@ -216,13 +219,13 @@ def test_backup_restore_with_named_collection_azure_conf1(cluster): node, f"CREATE TABLE test_write_connection_string (key UInt64, data String) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont', 'test_simple_write.csv', 'CSV')", ) - azure_query( - node, f"INSERT INTO test_write_connection_string VALUES (1, 'a')" - ) + azure_query(node, f"INSERT INTO test_write_connection_string VALUES (1, 'a')") print(get_azure_file_content("test_simple_write.csv", port)) assert get_azure_file_content("test_simple_write.csv", port) == '1,"a"\n' - backup_destination = f"AzureBlobStorage(azure_conf1, 'test_simple_write_nc_backup.csv')" + backup_destination = ( + f"AzureBlobStorage(azure_conf1, 'test_simple_write_nc_backup.csv')" + ) azure_query( node, f"BACKUP TABLE test_write_connection_string TO {backup_destination}", @@ -233,10 +236,11 @@ def test_backup_restore_with_named_collection_azure_conf1(cluster): f"RESTORE TABLE test_write_connection_string AS test_write_connection_string_restored FROM {backup_destination};", ) assert ( - azure_query(node, f"SELECT * from test_write_connection_string_restored") - == "1\ta\n" + azure_query(node, f"SELECT * from test_write_connection_string_restored") + == "1\ta\n" ) + def test_backup_restore_with_named_collection_azure_conf2(cluster): node = cluster.instances["node"] port = cluster.env_variables["AZURITE_PORT"] @@ -244,13 +248,13 @@ def test_backup_restore_with_named_collection_azure_conf2(cluster): node, f"CREATE TABLE test_write_connection_string_2 (key UInt64, data String) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont', 'test_simple_write_2.csv', 'CSV')", ) - azure_query( - node, f"INSERT INTO test_write_connection_string_2 VALUES (1, 'a')" - ) + azure_query(node, f"INSERT INTO test_write_connection_string_2 VALUES (1, 'a')") print(get_azure_file_content("test_simple_write_2.csv", port)) assert get_azure_file_content("test_simple_write_2.csv", port) == '1,"a"\n' - backup_destination = f"AzureBlobStorage(azure_conf2, 'test_simple_write_nc_backup_2.csv')" + backup_destination = ( + f"AzureBlobStorage(azure_conf2, 'test_simple_write_nc_backup_2.csv')" + ) azure_query( node, f"BACKUP TABLE test_write_connection_string_2 TO {backup_destination}", @@ -261,6 +265,6 @@ def test_backup_restore_with_named_collection_azure_conf2(cluster): f"RESTORE TABLE test_write_connection_string_2 AS test_write_connection_string_restored_2 FROM {backup_destination};", ) assert ( - azure_query(node, f"SELECT * from test_write_connection_string_restored_2") - == "1\ta\n" + azure_query(node, f"SELECT * from test_write_connection_string_restored_2") + == "1\ta\n" ) From 23b9f43d4f56a99e6c6b324910e9c69a665ce92d Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sun, 4 Feb 2024 16:18:17 +0100 Subject: [PATCH 115/276] Fix style --- src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp index 114a970384b..2f4c9374def 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp @@ -28,7 +28,6 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; extern const int INVALID_CONFIG_PARAMETER; extern const int AZURE_BLOB_STORAGE_ERROR; } From f036948f91882f6a9b594fe1393f82f2122f7da4 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 5 Feb 2024 10:09:48 +0100 Subject: [PATCH 116/276] Fix clang tidy build --- src/Backups/BackupIO_AzureBlobStorage.cpp | 12 ++++++------ src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp | 10 +++++----- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index 27928e871ce..52ce20d5108 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -89,8 +89,8 @@ std::unique_ptr BackupReaderAzureBlobStorage::readFile(const key = file_name; } return std::make_unique( - client, key, read_settings, settings.get()->max_single_read_retries, - settings.get()->max_single_download_retries); + client, key, read_settings, settings->max_single_read_retries, + settings->max_single_download_retries); } void BackupReaderAzureBlobStorage::copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, @@ -257,8 +257,8 @@ std::unique_ptr BackupWriterAzureBlobStorage::readFile(const String } return std::make_unique( - client, key, read_settings, settings.get()->max_single_read_retries, - settings.get()->max_single_download_retries); + client, key, read_settings, settings->max_single_read_retries, + settings->max_single_download_retries); } std::unique_ptr BackupWriterAzureBlobStorage::writeFile(const String & file_name) @@ -275,8 +275,8 @@ std::unique_ptr BackupWriterAzureBlobStorage::writeFile(const Strin return std::make_unique( client, key, - settings.get()->max_single_part_upload_size, - settings.get()->max_unexpected_write_error_retries, + settings->max_single_part_upload_size, + settings->max_unexpected_write_error_retries, DBMS_DEFAULT_BUFFER_SIZE, write_settings); } diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp index 2f4c9374def..9162f371b5b 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp @@ -97,7 +97,7 @@ namespace void calculatePartSize() { - auto max_upload_part_size = settings.get()->max_upload_part_size; + auto max_upload_part_size = settings->max_upload_part_size; if (!max_upload_part_size) throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "max_upload_part_size must not be 0"); /// We've calculated the size of a normal part (the final part can be smaller). @@ -292,7 +292,7 @@ void copyAzureBlobStorageFile( bool for_disk_azure_blob_storage) { - if (settings.get()->use_native_copy) + if (settings->use_native_copy) { ProfileEvents::increment(ProfileEvents::AzureCopyObject); if (for_disk_azure_blob_storage) @@ -302,7 +302,7 @@ void copyAzureBlobStorageFile( auto block_blob_client_dest = dest_client->GetBlockBlobClient(dest_blob); auto source_uri = block_blob_client_src.GetUrl(); - if (size < settings.get()->max_single_part_copy_size) + if (size < settings->max_single_part_copy_size) { block_blob_client_dest.CopyFromUri(source_uri); } @@ -326,8 +326,8 @@ void copyAzureBlobStorageFile( LOG_TRACE(&Poco::Logger::get("copyAzureBlobStorageFile"), "Reading from Container: {}, Blob: {}", src_container_for_logging, src_blob); auto create_read_buffer = [&] { - return std::make_unique(src_client, src_blob, read_settings, settings.get()->max_single_read_retries, - settings.get()->max_single_download_retries); + return std::make_unique(src_client, src_blob, read_settings, settings->max_single_read_retries, + settings->max_single_download_retries); }; UploadHelper helper{create_read_buffer, dest_client, offset, size, dest_container_for_logging, dest_blob, settings, schedule, for_disk_azure_blob_storage, &Poco::Logger::get("copyAzureBlobStorageFile")}; From fd2fdcdb4b33fe91227d7d623aea24a0356f29f0 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 5 Feb 2024 11:23:37 +0100 Subject: [PATCH 117/276] Fixed unwanted dereferencing --- src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp index 9162f371b5b..4714c795927 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp @@ -58,7 +58,7 @@ namespace , schedule(schedule_) , for_disk_azure_blob_storage(for_disk_azure_blob_storage_) , log(log_) - , max_single_part_upload_size(settings_.get()->max_single_part_upload_size) + , max_single_part_upload_size(settings_->max_single_part_upload_size) { } From 45bcc04d98ebaaf511ccf2920c1eaca98ee3a10c Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 5 Feb 2024 12:56:03 +0000 Subject: [PATCH 118/276] Use ConnectionPoolPtr instead of raw pointer --- src/Processors/QueryPlan/ReadFromRemote.cpp | 2 +- src/Processors/QueryPlan/ReadFromRemote.h | 2 +- src/QueryPipeline/RemoteQueryExecutor.cpp | 2 +- src/QueryPipeline/RemoteQueryExecutor.h | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 379ea950081..1edb672aa38 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -439,7 +439,7 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder void ReadFromParallelRemoteReplicasStep::addPipeForSingeReplica( - Pipes & pipes, IConnectionPool * pool, IConnections::ReplicaInfo replica_info) + Pipes & pipes, const ConnectionPoolPtr & pool, IConnections::ReplicaInfo replica_info) { bool add_agg_info = stage == QueryProcessingStage::WithMergeableState; bool add_totals = false; diff --git a/src/Processors/QueryPlan/ReadFromRemote.h b/src/Processors/QueryPlan/ReadFromRemote.h index eb3bcd12cc3..67fd9d24261 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.h +++ b/src/Processors/QueryPlan/ReadFromRemote.h @@ -90,7 +90,7 @@ public: void enforceAggregationInOrder(); private: - void addPipeForSingeReplica(Pipes & pipes, IConnectionPool * pool, IConnections::ReplicaInfo replica_info); + void addPipeForSingeReplica(Pipes & pipes, const ConnectionPoolPtr & pool, IConnections::ReplicaInfo replica_info); ClusterPtr cluster; ASTPtr query_ast; diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 8a4bee1d8af..1a68c9d4471 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -64,7 +64,7 @@ RemoteQueryExecutor::RemoteQueryExecutor( } RemoteQueryExecutor::RemoteQueryExecutor( - IConnectionPool * pool, + ConnectionPoolPtr pool, const String & query_, const Block & header_, ContextPtr context_, diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index e617deef7e8..6b1539bd08e 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -54,7 +54,7 @@ public: /// Takes a connection pool for a node (not cluster) RemoteQueryExecutor( - IConnectionPool * pool, + ConnectionPoolPtr pool, const String & query_, const Block & header_, ContextPtr context_, From 576cfdbf5cb4428868a03f1b78da760f776010fd Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 5 Feb 2024 14:27:07 +0000 Subject: [PATCH 119/276] Includes cleanup --- src/Client/Connection.h | 1 - src/Client/IConnections.h | 2 -- src/Processors/QueryPlan/ReadFromRemote.cpp | 2 +- src/Processors/QueryPlan/ReadFromRemote.h | 3 --- 4 files changed, 1 insertion(+), 7 deletions(-) diff --git a/src/Client/Connection.h b/src/Client/Connection.h index e93a7539d15..5d0411027a1 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -19,7 +19,6 @@ #include -#include #include #include "config.h" diff --git a/src/Client/IConnections.h b/src/Client/IConnections.h index ee17d198fc3..ebc71511834 100644 --- a/src/Client/IConnections.h +++ b/src/Client/IConnections.h @@ -1,7 +1,5 @@ #pragma once -#include - #include #include diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 1edb672aa38..6764e095088 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -12,7 +12,7 @@ #include #include #include -#include "Common/logger_useful.h" +#include #include #include #include diff --git a/src/Processors/QueryPlan/ReadFromRemote.h b/src/Processors/QueryPlan/ReadFromRemote.h index 67fd9d24261..498d584e85a 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.h +++ b/src/Processors/QueryPlan/ReadFromRemote.h @@ -9,9 +9,6 @@ namespace DB { - -class IConnectionPool; - class Throttler; using ThrottlerPtr = std::shared_ptr; From 8748d8c537c96dccb31df76099425b3321ff9e02 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 5 Feb 2024 15:10:51 +0000 Subject: [PATCH 120/276] UNEXPECTED_CLUSTER error for cluster with more than 1 shard --- src/Common/ErrorCodes.cpp | 1 + src/Interpreters/ClusterProxy/executeQuery.cpp | 7 +++++-- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 2e154ddb32d..8e81a626b41 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -593,6 +593,7 @@ M(711, FILECACHE_ACCESS_DENIED) \ M(712, TOO_MANY_MATERIALIZED_VIEWS) \ M(713, BROKEN_PROJECTION) \ + M(714, UNEXPECTED_CLUSTER) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 023ed6c7b61..33b86854ba9 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -32,6 +32,7 @@ namespace ErrorCodes extern const int TOO_LARGE_DISTRIBUTED_DEPTH; extern const int LOGICAL_ERROR; extern const int CLUSTER_DOESNT_EXIST; + extern const int UNEXPECTED_CLUSTER; } namespace ClusterProxy @@ -399,8 +400,10 @@ void executeQueryWithParallelReplicas( } else { - // todo: add error and exception for this case - chassert(not_optimized_cluster->getShardCount() == 1); + if (not_optimized_cluster->getShardCount() > 1) + throw DB::Exception( + ErrorCodes::UNEXPECTED_CLUSTER, + "`cluster_for_parallel_replicas` setting refers to cluster with several shards. Expected a cluster with one shard"); } auto coordinator = std::make_shared( From a50a14062621733b938cb9fb986c18d698ee3c36 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 5 Feb 2024 15:29:08 +0000 Subject: [PATCH 121/276] Test for UNEXPECTED_CLUSTER --- .../02982_parallel_replicas_unexpected_cluster.reference | 0 .../02982_parallel_replicas_unexpected_cluster.sql | 8 ++++++++ 2 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/02982_parallel_replicas_unexpected_cluster.reference create mode 100644 tests/queries/0_stateless/02982_parallel_replicas_unexpected_cluster.sql diff --git a/tests/queries/0_stateless/02982_parallel_replicas_unexpected_cluster.reference b/tests/queries/0_stateless/02982_parallel_replicas_unexpected_cluster.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02982_parallel_replicas_unexpected_cluster.sql b/tests/queries/0_stateless/02982_parallel_replicas_unexpected_cluster.sql new file mode 100644 index 00000000000..210b7d2a18a --- /dev/null +++ b/tests/queries/0_stateless/02982_parallel_replicas_unexpected_cluster.sql @@ -0,0 +1,8 @@ +DROP TABLE IF EXISTS test_unexpected_cluster; +CREATE TABLE test_unexpected_cluster (n UInt64) ENGINE=MergeTree() ORDER BY tuple(); +INSERT INTO test_unexpected_cluster SELECT * FROM numbers(10); + +SET allow_experimental_parallel_reading_from_replicas=2, max_parallel_replicas=2, cluster_for_parallel_replicas='test_cluster_two_shards', parallel_replicas_for_non_replicated_merge_tree=1; +SELECT count() FROM test_unexpected_cluster WHERE NOT ignore(*); -- { serverError UNEXPECTED_CLUSTER } + +DROP TABLE test_unexpected_cluster; From a0170bcbd5a7f9fbc956a6667a0929a72c1f189e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 5 Feb 2024 15:30:00 +0100 Subject: [PATCH 122/276] Fix utils --- programs/keeper-converter/KeeperConverter.cpp | 43 +++++++------ programs/keeper/CMakeLists.txt | 2 +- src/Coordination/KeeperSnapshotManager.cpp | 24 +++----- src/Coordination/KeeperStorage.cpp | 44 ++++++++++++- src/Coordination/KeeperStorage.h | 61 +++++++------------ src/Coordination/SnapshotableHashTable.h | 4 +- src/Coordination/ZooKeeperDataReader.cpp | 37 ++++++----- src/Coordination/tests/gtest_coordination.cpp | 42 ++++++------- utils/keeper-bench/Generator.cpp | 1 + utils/keeper-data-dumper/main.cpp | 14 ++--- 10 files changed, 146 insertions(+), 126 deletions(-) diff --git a/programs/keeper-converter/KeeperConverter.cpp b/programs/keeper-converter/KeeperConverter.cpp index f39a7bd5eb8..92bdea28738 100644 --- a/programs/keeper-converter/KeeperConverter.cpp +++ b/programs/keeper-converter/KeeperConverter.cpp @@ -1,5 +1,4 @@ #include -#include #include #include @@ -38,31 +37,31 @@ int mainEntryClickHouseKeeperConverter(int argc, char ** argv) return 0; } - //try - //{ - // auto keeper_context = std::make_shared(true); - // keeper_context->setDigestEnabled(true); - // keeper_context->setSnapshotDisk(std::make_shared("Keeper-snapshots", options["output-dir"].as())); + try + { + auto keeper_context = std::make_shared(true); + keeper_context->setDigestEnabled(true); + keeper_context->setSnapshotDisk(std::make_shared("Keeper-snapshots", options["output-dir"].as())); - // DB::KeeperStorage storage(/* tick_time_ms */ 500, /* superdigest */ "", keeper_context, /* initialize_system_nodes */ false); + DB::KeeperStorage storage(/* tick_time_ms */ 500, /* superdigest */ "", keeper_context, /* initialize_system_nodes */ false); - // DB::deserializeKeeperStorageFromSnapshotsDir(storage, options["zookeeper-snapshots-dir"].as(), logger); - // storage.initializeSystemNodes(); + DB::deserializeKeeperStorageFromSnapshotsDir(storage, options["zookeeper-snapshots-dir"].as(), logger); + storage.initializeSystemNodes(); - // DB::deserializeLogsAndApplyToStorage(storage, options["zookeeper-logs-dir"].as(), logger); - // DB::SnapshotMetadataPtr snapshot_meta = std::make_shared(storage.getZXID(), 1, std::make_shared()); - // DB::KeeperStorageSnapshot snapshot(&storage, snapshot_meta); + DB::deserializeLogsAndApplyToStorage(storage, options["zookeeper-logs-dir"].as(), logger); + DB::SnapshotMetadataPtr snapshot_meta = std::make_shared(storage.getZXID(), 1, std::make_shared()); + DB::KeeperStorageSnapshot snapshot(&storage, snapshot_meta); - // DB::KeeperSnapshotManager manager(1, keeper_context); - // auto snp = manager.serializeSnapshotToBuffer(snapshot); - // auto file_info = manager.serializeSnapshotBufferToDisk(*snp, storage.getZXID()); - // std::cout << "Snapshot serialized to path:" << fs::path(file_info.disk->getPath()) / file_info.path << std::endl; - //} - //catch (...) - //{ - // std::cerr << getCurrentExceptionMessage(true) << '\n'; - // return getCurrentExceptionCode(); - //} + DB::KeeperSnapshotManager manager(1, keeper_context); + auto snp = manager.serializeSnapshotToBuffer(snapshot); + auto file_info = manager.serializeSnapshotBufferToDisk(*snp, storage.getZXID()); + std::cout << "Snapshot serialized to path:" << fs::path(file_info.disk->getPath()) / file_info.path << std::endl; + } + catch (...) + { + std::cerr << getCurrentExceptionMessage(true) << '\n'; + return getCurrentExceptionCode(); + } return 0; } diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index fba9b3e4d86..143ded0ee85 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -44,7 +44,7 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/SessionExpiryQueue.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/SummingStateMachine.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/WriteBufferFromNuraftBuffer.cpp - #${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/ZooKeeperDataReader.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/ZooKeeperDataReader.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Core/SettingsFields.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Core/BaseSettings.cpp diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index e1466172fd9..fe6c7e3a389 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -86,14 +86,14 @@ namespace writeBinary(node.version, out); writeBinary(node.cversion, out); writeBinary(node.aversion, out); - const bool is_ephemeral = node.isEphemeral(); + const bool is_ephemeral = node.isEphemeral(); writeBinary(is_ephemeral ? node.ephemeralOwner() : 0, out); if (version < SnapshotVersion::V6) writeBinary(static_cast(node.data_size), out); writeBinary(is_ephemeral ? 0 : node.numChildren(), out); writeBinary(node.pzxid, out); - writeBinary(node.seqNum(), out); + writeBinary(is_ephemeral ? 0 : node.seqNum(), out); if (version >= SnapshotVersion::V4 && version <= SnapshotVersion::V5) writeBinary(node.sizeInBytes(), out); @@ -153,26 +153,23 @@ namespace int64_t ephemeral_owner = 0; readBinary(ephemeral_owner, in); if (ephemeral_owner != 0) - { - node.is_ephemeral_and_mtime.is_ephemeral = true; - node.ephemeral_or_children_data.ephemeral_owner = ephemeral_owner; - } + node.setEphemeralOwner(ephemeral_owner); if (version < SnapshotVersion::V6) { int32_t data_length = 0; readBinary(data_length, in); } - int32_t num_children; + int32_t num_children = 0; readBinary(num_children, in); - if (num_children) + if (num_children != 0) node.ephemeral_or_children_data.children_info.num_children = num_children; readBinary(node.pzxid, in); - int32_t seq_num; + int32_t seq_num = 0; readBinary(seq_num, in); - if (seq_num) + if (seq_num != 0) node.ephemeral_or_children_data.children_info.seq_num = seq_num; if (version >= SnapshotVersion::V4 && version <= SnapshotVersion::V5) @@ -383,11 +380,6 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial if (recalculate_digest) storage.nodes_digest = 0; - const auto is_node_empty = [](const auto & /*node*/) - { - return false; //node.getData().empty() && node == KeeperStorage::Node{}; - }; - for (size_t nodes_read = 0; nodes_read < snapshot_container_size; ++nodes_read) { std::string path; @@ -415,7 +407,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial } else if (match_result == EXACT) { - if (!is_node_empty(node)) + if (!node.empty()) { if (keeper_context->ignoreSystemPathOnStartup() || keeper_context->getServerState() != KeeperContext::Phase::INIT) { diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index b06f87fc054..ac4a9433e30 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -191,6 +191,47 @@ uint64_t calculateDigest(std::string_view path, const KeeperStorage::Node & node } +KeeperStorage::Node::~Node() +{ + if (data_size) + delete[] data; +} + +KeeperStorage::Node & KeeperStorage::Node::operator=(const Node & other) +{ + if (this == &other) + return *this; + + czxid = other.czxid; + mzxid = other.mzxid; + pzxid = other.pzxid; + acl_id = other.acl_id; + has_cached_digest_and_ctime = other.has_cached_digest_and_ctime; + is_ephemeral_and_mtime = other.is_ephemeral_and_mtime; + ephemeral_or_children_data = other.ephemeral_or_children_data; + data_size = other.data_size; + version = other.version; + cversion = other.cversion; + aversion = other.aversion; + + if (data_size != 0) + { + data = new char[data_size]; + memcpy(data, other.data, data_size); + } + return *this; +} + +KeeperStorage::Node::Node(const Node & other) +{ + *this = other; +} + +bool KeeperStorage::Node::empty() const +{ + return data_size == 0 && mzxid == 0; +} + void KeeperStorage::Node::copyStats(const Coordination::Stat & stat) { czxid = stat.czxid; @@ -211,8 +252,7 @@ void KeeperStorage::Node::copyStats(const Coordination::Stat & stat) } else { - is_ephemeral_and_mtime.is_ephemeral = true; - ephemeral_or_children_data.ephemeral_owner = stat.ephemeralOwner; + setEphemeralOwner(stat.ephemeralOwner); } } diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 76b6ab43c50..55d2e20d44c 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -5,17 +5,15 @@ #include #include #include -#include -#include -#include -#include -#include #include namespace DB { +class KeeperContext; +using KeeperContextPtr = std::shared_ptr; + struct KeeperStorageRequestProcessor; using KeeperStorageRequestProcessorPtr = std::shared_ptr; using ResponseCallback = std::function; @@ -43,13 +41,13 @@ public: mutable struct { bool has_cached_digest : 1; - int64_t ctime : 7; + int64_t ctime : 63; } has_cached_digest_and_ctime{false, 0}; struct { bool is_ephemeral : 1; - int64_t mtime : 7; + int64_t mtime : 63; } is_ephemeral_and_mtime{false, 0}; @@ -74,47 +72,18 @@ public: /// pack the boolean with seq_num above mutable uint64_t cached_digest = 0; - ~Node() - { - if (data_size) - delete [] data; - } + ~Node(); Node() = default; - Node & operator=(const Node & other) - { - if (this == &other) - return *this; + Node & operator=(const Node & other); - czxid = other.czxid; - mzxid = other.mzxid; - pzxid = other.pzxid; - acl_id = other.acl_id; - has_cached_digest_and_ctime = other.has_cached_digest_and_ctime; - is_ephemeral_and_mtime = other.is_ephemeral_and_mtime; - ephemeral_or_children_data = other.ephemeral_or_children_data; - data_size = other.data_size; - version = other.version; - cversion = other.cversion; - aversion = other.aversion; + Node(const Node & other); - if (data_size != 0) - { - data = new char[data_size]; - memcpy(data, other.data, data_size); - } - return *this; - } - - Node(const Node & other) - { - *this = other; - } + bool empty() const; bool isEphemeral() const { - return is_ephemeral_and_mtime.is_ephemeral; } @@ -123,6 +92,12 @@ public: return isEphemeral() ? ephemeral_or_children_data.ephemeral_owner : 0; } + void setEphemeralOwner(int64_t ephemeral_owner) + { + is_ephemeral_and_mtime.is_ephemeral = true; + ephemeral_or_children_data.ephemeral_owner = ephemeral_owner; + } + int32_t numChildren() const { return ephemeral_or_children_data.children_info.num_children; @@ -130,6 +105,7 @@ public: void increaseNumChildren() { + chassert(!isEphemeral()); ++ephemeral_or_children_data.children_info.num_children; } @@ -138,6 +114,11 @@ public: return ephemeral_or_children_data.children_info.seq_num; } + void setSeqNum(int32_t seq_num) + { + ephemeral_or_children_data.children_info.seq_num = seq_num; + } + void increaseSeqNum() { ++ephemeral_or_children_data.children_info.seq_num; diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index 6a763a93210..5533913b3b8 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -18,11 +18,11 @@ struct ListNode StringRef key; V value; - struct + struct { bool active_in_map : 1; bool free_key : 1; - uint64_t version : 6; + uint64_t version : 62; } node_metadata{false, false, 0}; void setInactiveInMap() diff --git a/src/Coordination/ZooKeeperDataReader.cpp b/src/Coordination/ZooKeeperDataReader.cpp index 11a33e198a9..2e173667e28 100644 --- a/src/Coordination/ZooKeeperDataReader.cpp +++ b/src/Coordination/ZooKeeperDataReader.cpp @@ -101,30 +101,37 @@ int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, LoggerP KeeperStorage::Node node{}; String data; Coordination::read(data, in); - node.setData(std::move(data)); + node.setData(data); Coordination::read(node.acl_id, in); /// Deserialize stat - Coordination::read(node.src/Coordination/ZooKeeperDataReader.cppstat.czxid, in); - Coordination::read(node.stat.mzxid, in); + Coordination::read(node.czxid, in); + Coordination::read(node.mzxid, in); /// For some reason ZXID specified in filename can be smaller /// then actual zxid from nodes. In this case we will use zxid from nodes. - max_zxid = std::max(max_zxid, node.stat.mzxid); + max_zxid = std::max(max_zxid, node.mzxid); - Coordination::read(node.stat.ctime, in); - Coordination::read(node.stat.mtime, in); - Coordination::read(node.stat.version, in); - Coordination::read(node.stat.cversion, in); - Coordination::read(node.stat.aversion, in); - Coordination::read(node.stat.ephemeralOwner, in); - Coordination::read(node.stat.pzxid, in); + int64_t ctime; + Coordination::read(ctime, in); + node.setCtime(ctime); + int64_t mtime; + Coordination::read(mtime, in); + node.setMtime(mtime); + Coordination::read(node.version, in); + Coordination::read(node.cversion, in); + Coordination::read(node.aversion, in); + int64_t ephemeral_owner; + Coordination::read(ephemeral_owner, in); + if (ephemeral_owner != 0) + node.setEphemeralOwner(ephemeral_owner); + Coordination::read(node.pzxid, in); if (!path.empty()) { - node.seq_num = node.stat.cversion; + node.setSeqNum(node.cversion); storage.container.insertOrReplace(path, node); - if (node.stat.ephemeralOwner != 0) - storage.ephemerals[node.stat.ephemeralOwner].insert(path); + if (ephemeral_owner != 0) + storage.ephemerals[ephemeral_owner].insert(path); storage.acl_map.addUsage(node.acl_id); } @@ -139,7 +146,7 @@ int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, LoggerP if (itr.key != "/") { auto parent_path = parentNodePath(itr.key); - storage.container.updateValue(parent_path, [my_path = itr.key] (KeeperStorage::Node & value) { value.addChild(getBaseNodeName(my_path)); ++value.stat.numChildren; }); + storage.container.updateValue(parent_path, [my_path = itr.key] (KeeperStorage::Node & value) { value.addChild(getBaseNodeName(my_path)); value.increaseNumChildren(); }); } } diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 59a550177a4..bd9dc4c3fd3 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1508,7 +1508,7 @@ void addNode(DB::KeeperStorage & storage, const std::string & path, const std::s using Node = DB::KeeperStorage::Node; Node node{}; node.setData(data); - node.stat.ephemeralOwner = ephemeral_owner; + node.setEphemeralOwner(ephemeral_owner); storage.container.insertOrReplace(path, node); auto child_it = storage.container.find(path); auto child_path = DB::getBaseNodeName(child_it->key); @@ -1517,7 +1517,7 @@ void addNode(DB::KeeperStorage & storage, const std::string & path, const std::s [&](auto & parent) { parent.addChild(child_path); - parent.stat.numChildren++; + parent.increaseNumChildren(); }); } @@ -1530,12 +1530,12 @@ TEST_P(CoordinationTest, TestStorageSnapshotSimple) DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); DB::KeeperStorage storage(500, "", keeper_context); - addNode(storage, "/hello", "world", 1); - addNode(storage, "/hello/somepath", "somedata", 3); + addNode(storage, "/hello1", "world", 1); + addNode(storage, "/hello2", "somedata", 3); storage.session_id_counter = 5; storage.zxid = 2; - storage.ephemerals[3] = {"/hello"}; - storage.ephemerals[1] = {"/hello/somepath"}; + storage.ephemerals[3] = {"/hello2"}; + storage.ephemerals[1] = {"/hello1"}; storage.getSessionID(130); storage.getSessionID(130); @@ -1556,13 +1556,13 @@ TEST_P(CoordinationTest, TestStorageSnapshotSimple) auto [restored_storage, snapshot_meta, _] = manager.deserializeSnapshotFromBuffer(debuf); EXPECT_EQ(restored_storage->container.size(), 6); - EXPECT_EQ(restored_storage->container.getValue("/").getChildren().size(), 2); - EXPECT_EQ(restored_storage->container.getValue("/hello").getChildren().size(), 1); - EXPECT_EQ(restored_storage->container.getValue("/hello/somepath").getChildren().size(), 0); + EXPECT_EQ(restored_storage->container.getValue("/").getChildren().size(), 3); + EXPECT_EQ(restored_storage->container.getValue("/hello1").getChildren().size(), 0); + EXPECT_EQ(restored_storage->container.getValue("/hello2").getChildren().size(), 0); EXPECT_EQ(restored_storage->container.getValue("/").getData(), ""); - EXPECT_EQ(restored_storage->container.getValue("/hello").getData(), "world"); - EXPECT_EQ(restored_storage->container.getValue("/hello/somepath").getData(), "somedata"); + EXPECT_EQ(restored_storage->container.getValue("/hello1").getData(), "world"); + EXPECT_EQ(restored_storage->container.getValue("/hello2").getData(), "somedata"); EXPECT_EQ(restored_storage->session_id_counter, 7); EXPECT_EQ(restored_storage->zxid, 2); EXPECT_EQ(restored_storage->ephemerals.size(), 2); @@ -2251,12 +2251,12 @@ TEST_P(CoordinationTest, TestStorageSnapshotDifferentCompressions) DB::KeeperSnapshotManager manager(3, keeper_context, params.enable_compression); DB::KeeperStorage storage(500, "", keeper_context); - addNode(storage, "/hello", "world", 1); - addNode(storage, "/hello/somepath", "somedata", 3); + addNode(storage, "/hello1", "world", 1); + addNode(storage, "/hello2", "somedata", 3); storage.session_id_counter = 5; storage.zxid = 2; - storage.ephemerals[3] = {"/hello"}; - storage.ephemerals[1] = {"/hello/somepath"}; + storage.ephemerals[3] = {"/hello2"}; + storage.ephemerals[1] = {"/hello1"}; storage.getSessionID(130); storage.getSessionID(130); @@ -2273,13 +2273,13 @@ TEST_P(CoordinationTest, TestStorageSnapshotDifferentCompressions) auto [restored_storage, snapshot_meta, _] = new_manager.deserializeSnapshotFromBuffer(debuf); EXPECT_EQ(restored_storage->container.size(), 6); - EXPECT_EQ(restored_storage->container.getValue("/").getChildren().size(), 2); - EXPECT_EQ(restored_storage->container.getValue("/hello").getChildren().size(), 1); - EXPECT_EQ(restored_storage->container.getValue("/hello/somepath").getChildren().size(), 0); + EXPECT_EQ(restored_storage->container.getValue("/").getChildren().size(), 3); + EXPECT_EQ(restored_storage->container.getValue("/hello1").getChildren().size(), 0); + EXPECT_EQ(restored_storage->container.getValue("/hello2").getChildren().size(), 0); EXPECT_EQ(restored_storage->container.getValue("/").getData(), ""); - EXPECT_EQ(restored_storage->container.getValue("/hello").getData(), "world"); - EXPECT_EQ(restored_storage->container.getValue("/hello/somepath").getData(), "somedata"); + EXPECT_EQ(restored_storage->container.getValue("/hello1").getData(), "world"); + EXPECT_EQ(restored_storage->container.getValue("/hello2").getData(), "somedata"); EXPECT_EQ(restored_storage->session_id_counter, 7); EXPECT_EQ(restored_storage->zxid, 2); EXPECT_EQ(restored_storage->ephemerals.size(), 2); @@ -2948,7 +2948,7 @@ TEST_P(CoordinationTest, TestCheckNotExistsRequest) create_path("/test_node"); auto node_it = storage.container.find("/test_node"); ASSERT_NE(node_it, storage.container.end()); - auto node_version = node_it->value.stat.version; + auto node_version = node_it->value.version; { SCOPED_TRACE("CheckNotExists returns ZNODEEXISTS"); diff --git a/utils/keeper-bench/Generator.cpp b/utils/keeper-bench/Generator.cpp index 2212f7158ae..a3e85f670d6 100644 --- a/utils/keeper-bench/Generator.cpp +++ b/utils/keeper-bench/Generator.cpp @@ -455,6 +455,7 @@ Coordination::ZooKeeperRequestPtr CreateRequestGenerator::generateImpl(const Coo auto request = std::make_shared(); request->acls = acls; + request->is_sequential = true; std::string path_candidate = std::filesystem::path(parent_path.getPath()) / name.getString(); diff --git a/utils/keeper-data-dumper/main.cpp b/utils/keeper-data-dumper/main.cpp index e06b301edbf..12acd0a8db9 100644 --- a/utils/keeper-data-dumper/main.cpp +++ b/utils/keeper-data-dumper/main.cpp @@ -25,13 +25,13 @@ void dumpMachine(std::shared_ptr machine) keys.pop(); std::cout << key << "\n"; auto value = storage.container.getValue(key); - std::cout << "\tStat: {version: " << value.stat.version << - ", mtime: " << value.stat.mtime << - ", emphemeralOwner: " << value.stat.ephemeralOwner << - ", czxid: " << value.stat.czxid << - ", mzxid: " << value.stat.mzxid << - ", numChildren: " << value.stat.numChildren << - ", dataLength: " << value.getData().size() << + std::cout << "\tStat: {version: " << value.version << + ", mtime: " << value.mtime() << + ", emphemeralOwner: " << value.ephemeralOwner() << + ", czxid: " << value.czxid << + ", mzxid: " << value.mzxid << + ", numChildren: " << value.numChildren() << + ", dataLength: " << value.data_size << "}" << std::endl; std::cout << "\tData: " << storage.container.getValue(key).getData() << std::endl; From dab078f7d24bca61a1cf05dc0294fdee48103202 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 5 Feb 2024 20:21:48 +0000 Subject: [PATCH 123/276] Profile events --- src/Common/ProfileEvents.cpp | 2 ++ src/QueryPipeline/RemoteQueryExecutor.cpp | 6 ++++++ .../MergeTree/ParallelReplicasReadingCoordinator.cpp | 8 ++++---- .../02769_parallel_replicas_unavailable_shards.sql | 4 +--- 4 files changed, 13 insertions(+), 7 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 8782f895f3f..0218545c3a4 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -616,6 +616,8 @@ The server successfully detected this situation and will download merged part fr M(InterfacePostgreSQLReceiveBytes, "Number of bytes received through PostgreSQL interfaces") \ \ M(ParallelReplicasUsedCount, "Number of replicas used to execute a query with task-based parallel replicas") \ + M(ParallelReplicasAvailableCount, "Number of replicas available to execute a query with task-based parallel replicas") \ + M(ParallelReplicasUnavailableCount, "Number of replicas which was chosen, but unavailable, to execute a query with task-based parallel replicas") \ #ifdef APPLY_FOR_EXTERNAL_EVENTS #define APPLY_FOR_EVENTS(M) APPLY_FOR_BUILTIN_EVENTS(M) APPLY_FOR_EXTERNAL_EVENTS(M) diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 1a68c9d4471..e44749dfb97 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -30,6 +30,7 @@ namespace ProfileEvents extern const Event SuspendSendingQueryToShard; extern const Event ReadTaskRequestsReceived; extern const Event MergeTreeReadTaskRequestsReceived; + extern const Event ParallelReplicasAvailableCount; } namespace DB @@ -97,7 +98,12 @@ RemoteQueryExecutor::RemoteQueryExecutor( std::vector connection_entries; if (!result.entry.isNull() && result.is_usable) + { + if (extension_ && extension_->parallel_reading_coordinator) + ProfileEvents::increment(ProfileEvents::ParallelReplicasAvailableCount); + connection_entries.emplace_back(std::move(result.entry)); + } auto res = std::make_unique(std::move(connection_entries), context->getSettingsRef(), throttler); if (extension_ && extension_->replica_info) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index abc51bde3fb..2fe237efdc7 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -97,11 +97,9 @@ extern const Event ParallelReplicasCollectingOwnedSegmentsMicroseconds; extern const Event ParallelReplicasReadAssignedMarks; extern const Event ParallelReplicasReadUnassignedMarks; extern const Event ParallelReplicasReadAssignedForStealingMarks; -} -namespace ProfileEvents -{ - extern const Event ParallelReplicasUsedCount; +extern const Event ParallelReplicasUsedCount; +extern const Event ParallelReplicasUnavailableCount; } namespace DB @@ -1025,6 +1023,8 @@ ParallelReadResponse ParallelReplicasReadingCoordinator::handleRequest(ParallelR void ParallelReplicasReadingCoordinator::markReplicaAsUnavailable(size_t replica_number) { + ProfileEvents::increment(ProfileEvents::ParallelReplicasUnavailableCount); + std::lock_guard lock(mutex); if (!pimpl) diff --git a/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql b/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql index be200353f06..1a75e000349 100644 --- a/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql +++ b/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql @@ -2,8 +2,6 @@ DROP TABLE IF EXISTS test_parallel_replicas_unavailable_shards; CREATE TABLE test_parallel_replicas_unavailable_shards (n UInt64) ENGINE=MergeTree() ORDER BY tuple(); INSERT INTO test_parallel_replicas_unavailable_shards SELECT * FROM numbers(10); -SYSTEM FLUSH LOGS; - SET allow_experimental_parallel_reading_from_replicas=2, max_parallel_replicas=11, cluster_for_parallel_replicas='parallel_replicas', parallel_replicas_for_non_replicated_merge_tree=1; SET send_logs_level='error'; SELECT count() FROM test_parallel_replicas_unavailable_shards WHERE NOT ignore(*) SETTINGS log_comment = '02769_7b513191-5082-4073-8568-53b86a49da79'; @@ -11,6 +9,6 @@ SELECT count() FROM test_parallel_replicas_unavailable_shards WHERE NOT ignore(* SYSTEM FLUSH LOGS; SET allow_experimental_parallel_reading_from_replicas=0; -SELECT count() FROM system.text_log WHERE yesterday() <= event_date AND query_id in (select query_id from system.query_log where log_comment = '02769_7b513191-5082-4073-8568-53b86a49da79' and current_database = currentDatabase()) and message LIKE '%Replica number % is unavailable%'; +SELECT ProfileEvents['ParallelReplicasUnavailableCount'] FROM system.query_log WHERE yesterday() <= event_date AND query_id in (select query_id from system.query_log where log_comment = '02769_7b513191-5082-4073-8568-53b86a49da79' and current_database = currentDatabase()) and type = 'QueryFinish' and query_id == initial_query_id; DROP TABLE test_parallel_replicas_unavailable_shards; From b9bb04ebf03f6677d6f81ac628ea94ca3b9b43bd Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Mon, 5 Feb 2024 13:23:54 -0800 Subject: [PATCH 124/276] [Docs] Specify Memory table engine usage on cloud --- docs/en/engines/table-engines/special/memory.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/engines/table-engines/special/memory.md b/docs/en/engines/table-engines/special/memory.md index 5cd766a318a..a30f620f5c5 100644 --- a/docs/en/engines/table-engines/special/memory.md +++ b/docs/en/engines/table-engines/special/memory.md @@ -6,6 +6,10 @@ sidebar_label: Memory # Memory Table Engine +:::note +When using the Memory table engine on ClickHouse Cloud, you must use a client that uses TCP (such as [clickhouse-client](/en/interfaces/cli)) or a native interface, and not one that uses HTTP (such as [clickhouse-connect](/en/integrations/python)). If you use HTTP, all queries must be submitted and executed at once using a multi-statement query. +::: + The Memory engine stores data in RAM, in uncompressed form. Data is stored in exactly the same form as it is received when read. In other words, reading from this table is completely free. Concurrent data access is synchronized. Locks are short: read and write operations do not block each other. Indexes are not supported. Reading is parallelized. From 5de010ddda8311ff178519c0161b6d8314e2641e Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Mon, 5 Feb 2024 13:29:02 -0800 Subject: [PATCH 125/276] [Docs] Add detail on using temporary tables in Cloud --- docs/en/sql-reference/statements/create/table.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index 7322bc17b76..067761e760b 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -514,6 +514,10 @@ ENGINE = MergeTree ORDER BY x; ## Temporary Tables +:::note +Please note that on ClickHouse Cloud, temporary tables are not replicated. As a result, there is no guarantee that data inserted into a temporary table will be available in other replicas. The primary use case where temporary tables can be useful in ClickHouse Cloud is for querying or joining small external datasets during a single session. +::: + ClickHouse supports temporary tables which have the following characteristics: - Temporary tables disappear when the session ends, including if the connection is lost. From 754b73a38bd8743a8f4ec87c543f2e56a508e2d8 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 6 Feb 2024 09:35:48 +0100 Subject: [PATCH 126/276] remove unused --- src/Coordination/SnapshotableHashTable.h | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index 5533913b3b8..0c6af29d24a 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -7,11 +7,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - template struct ListNode { From 127996486602e9ecf46ff8e3ca8cd59cfefcbce6 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 6 Feb 2024 10:18:02 +0100 Subject: [PATCH 127/276] fix build --- src/Coordination/KeeperSnapshotManager.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index fe6c7e3a389..c176536dfbf 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -454,7 +454,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial #else throw Exception(ErrorCodes::LOGICAL_ERROR, "Children counter in stat.numChildren {}" " is different from actual children size {} for node {}", - itr.value.stat.numChildren, itr.value.getChildren().size(), itr.key); + itr.value.numChildren(), itr.value.getChildren().size(), itr.key); #endif } } From ba94cd2ef70e3263aea35fa6c4687582ca1864f5 Mon Sep 17 00:00:00 2001 From: Val Doroshchuk Date: Tue, 30 Jan 2024 10:09:58 +0100 Subject: [PATCH 128/276] MaterializedMySQL: Fix gtid_after_attach_test to retry on detach --- .../materialized_with_ddl.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py index 97c8b65f15d..57f2ccd720d 100644 --- a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py @@ -3379,7 +3379,7 @@ def gtid_after_attach_test(clickhouse_node, mysql_node, replication): f"CREATE TABLE {db}.t(id INT PRIMARY KEY AUTO_INCREMENT, score int, create_time DATETIME DEFAULT NOW())" ) - db_count = 6 + db_count = 4 for i in range(db_count): replication.create_db_ch( f"{db}{i}", @@ -3392,7 +3392,11 @@ def gtid_after_attach_test(clickhouse_node, mysql_node, replication): "t\n", ) for i in range(int(db_count / 2)): - clickhouse_node.query(f"DETACH DATABASE {db}{i}") + check_query( + clickhouse_node, + f"DETACH DATABASE {db}{i}", + "", + ) mysql_node.query(f"USE {db}") rows = 10000 From b43f90fce209351972fafd420e1b62d58159ba81 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 6 Feb 2024 19:55:01 +0100 Subject: [PATCH 129/276] Fix query start time on non initial queries --- src/Interpreters/executeQuery.cpp | 18 ++++-------- .../02985_shard_query_start_time.reference | 2 ++ .../02985_shard_query_start_time.sql | 29 +++++++++++++++++++ 3 files changed, 37 insertions(+), 12 deletions(-) create mode 100644 tests/queries/0_stateless/02985_shard_query_start_time.reference create mode 100644 tests/queries/0_stateless/02985_shard_query_start_time.sql diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 1787f627c2e..5c16a5d800f 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -663,15 +663,17 @@ static std::tuple executeQueryImpl( if (query_span && query_span->trace_id != UUID{}) LOG_TRACE(getLogger("executeQuery"), "Query span trace_id for opentelemetry log: {}", query_span->trace_id); + /// Used for logging query start time in system.query_log auto query_start_time = std::chrono::system_clock::now(); - /// Used to set the watch in QueryStatus and the output formats. It is not based on query_start_time as that might be based on - /// the value passed by the client + /// Used for: + /// * Setting the watch in QueryStatus (controls timeouts and progress) and the output formats + /// * Logging query duration (system.query_log) Stopwatch start_watch{CLOCK_MONOTONIC}; const auto & client_info = context->getClientInfo(); - if (!internal) + if (!internal && client_info.initial_query_start_time == 0) { // If it's not an internal query and we don't see an initial_query_start_time yet, initialize it // to current time. Internal queries are those executed without an independent client context, @@ -679,15 +681,7 @@ static std::tuple executeQueryImpl( // possible to have unset initial_query_start_time for non-internal and non-initial queries. For // example, the query is from an initiator that is running an old version of clickhouse. // On the other hand, if it's initialized then take it as the start of the query - if (client_info.initial_query_start_time == 0) - { - context->setInitialQueryStartTime(query_start_time); - } - else - { - query_start_time = std::chrono::time_point( - std::chrono::microseconds{client_info.initial_query_start_time_microseconds}); - } + context->setInitialQueryStartTime(query_start_time); } assert(internal || CurrentThread::get().getQueryContext()); diff --git a/tests/queries/0_stateless/02985_shard_query_start_time.reference b/tests/queries/0_stateless/02985_shard_query_start_time.reference new file mode 100644 index 00000000000..1957f3a9604 --- /dev/null +++ b/tests/queries/0_stateless/02985_shard_query_start_time.reference @@ -0,0 +1,2 @@ +1 1 +1 1 diff --git a/tests/queries/0_stateless/02985_shard_query_start_time.sql b/tests/queries/0_stateless/02985_shard_query_start_time.sql new file mode 100644 index 00000000000..b0d8d2b6e53 --- /dev/null +++ b/tests/queries/0_stateless/02985_shard_query_start_time.sql @@ -0,0 +1,29 @@ +DROP TABLE IF EXISTS sharded_table; +CREATE TABLE sharded_table (dummy UInt8) ENGINE = Distributed('test_cluster_two_shards', 'system', 'one'); + +SELECT * FROM sharded_table FORMAT Null SETTINGS log_comment='02985_shard_query_start_time_query_1'; + +SYSTEM FLUSH LOGS; + +-- We do not test for query_start_time because that would conflict pretty easily +WITH +( + SELECT + (query_id, query_start_time_microseconds) + FROM + system.query_log + WHERE + event_date >= yesterday() + AND current_database = currentDatabase() + AND log_comment = '02985_shard_query_start_time_query_1' + AND type = 'QueryFinish' + ORDER BY query_start_time_microseconds DESC + LIMIT 1 +) AS id_and_start_tuple +SELECT + query_start_time_microseconds > initial_query_start_time_microseconds, + initial_query_start_time_microseconds = id_and_start_tuple.2 +FROM + system.query_log +WHERE + NOT is_initial_query AND initial_query_id = id_and_start_tuple.1; From 10082399d540a1ab5cee3ff381856193afc9ad22 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 7 Feb 2024 00:52:25 +0000 Subject: [PATCH 130/276] Minor review fixes --- src/Client/ConnectionPool.h | 22 ++++++++++++--------- src/Common/ProfileEvents.cpp | 2 +- src/Processors/QueryPlan/ReadFromRemote.cpp | 8 ++++---- src/QueryPipeline/RemoteQueryExecutor.cpp | 2 +- 4 files changed, 19 insertions(+), 15 deletions(-) diff --git a/src/Client/ConnectionPool.h b/src/Client/ConnectionPool.h index 1886a0431a5..8e707e8190f 100644 --- a/src/Client/ConnectionPool.h +++ b/src/Client/ConnectionPool.h @@ -27,6 +27,9 @@ class IConnectionPool : private boost::noncopyable public: using Entry = PoolBase::Entry; + IConnectionPool() = default; + IConnectionPool(String host_, UInt16 port_) : host(host_), port(port_), address(host + ":" + toString(port_)) {} + virtual ~IConnectionPool() = default; /// Selects the connection to work. @@ -36,7 +39,15 @@ public: const Settings & settings, bool force_connected = true) = 0; + const std::string & getHost() const { return host; } + UInt16 getPort() const { return port; } + const String & getAddress() const { return address; } virtual Priority getPriority() const { return Priority{1}; } + +protected: + const String host; + const UInt16 port = 0; + const String address; }; using ConnectionPoolPtr = std::shared_ptr; @@ -63,10 +74,9 @@ public: Protocol::Compression compression_, Protocol::Secure secure_, Priority priority_ = Priority{1}) - : Base(max_connections_, + : IConnectionPool(host_, port_), + Base(max_connections_, getLogger("ConnectionPool (" + host_ + ":" + toString(port_) + ")")), - host(host_), - port(port_), default_database(default_database_), user(user_), password(password_), @@ -99,10 +109,6 @@ public: return entry; } - const std::string & getHost() const - { - return host; - } std::string getDescription() const { return host + ":" + toString(port); @@ -125,8 +131,6 @@ protected: } private: - String host; - UInt16 port; String default_database; String user; String password; diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 0218545c3a4..fc30a4e0794 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -617,7 +617,7 @@ The server successfully detected this situation and will download merged part fr \ M(ParallelReplicasUsedCount, "Number of replicas used to execute a query with task-based parallel replicas") \ M(ParallelReplicasAvailableCount, "Number of replicas available to execute a query with task-based parallel replicas") \ - M(ParallelReplicasUnavailableCount, "Number of replicas which was chosen, but unavailable, to execute a query with task-based parallel replicas") \ + M(ParallelReplicasUnavailableCount, "Number of replicas which was chosen, but found to be unavailable during query execution with task-based parallel replicas") \ #ifdef APPLY_FOR_EXTERNAL_EVENTS #define APPLY_FOR_EVENTS(M) APPLY_FOR_BUILTIN_EVENTS(M) APPLY_FOR_EXTERNAL_EVENTS(M) diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 6764e095088..5707eb2e9c6 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -375,10 +375,11 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( , storage_limits(std::move(storage_limits_)) , log(log_) { - std::vector description; + chassert(cluster->getShardCount() == 1); - for (const auto & address : cluster->getShardsAddresses()) - description.push_back(fmt::format("Replica: {}", address[0].host_name)); + std::vector description; + for (const auto & pool : cluster->getShardsInfo().front().per_replica_pools) + description.push_back(fmt::format("Replica: {}", pool->getHost())); setStepDescription(boost::algorithm::join(description, ", ")); } @@ -412,7 +413,6 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder all_replicas_count = shard.getAllNodeCount(); } - chassert(cluster->getShardCount() == 1); auto shuffled_pool = shard.pool->getShuffledPools(current_settings); shuffled_pool.resize(all_replicas_count); diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index e44749dfb97..46616905bcb 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -105,7 +105,7 @@ RemoteQueryExecutor::RemoteQueryExecutor( connection_entries.emplace_back(std::move(result.entry)); } - auto res = std::make_unique(std::move(connection_entries), context->getSettingsRef(), throttler); + auto res = std::make_unique(std::move(connection_entries), current_settings, throttler); if (extension_ && extension_->replica_info) res->setReplicaInfo(*extension_->replica_info); From 755298838fba555fa5dba644277e151841d1cf67 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 7 Feb 2024 01:28:27 +0000 Subject: [PATCH 131/276] Preserve replicas order for data locality --- src/Processors/QueryPlan/ReadFromRemote.cpp | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 5707eb2e9c6..91c81d619a7 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -413,7 +413,17 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder all_replicas_count = shard.getAllNodeCount(); } - auto shuffled_pool = shard.pool->getShuffledPools(current_settings); + + std::vector shuffled_pool; + if (all_replicas_count < shard.getAllNodeCount()) + shuffled_pool = shard.pool->getShuffledPools(current_settings); + else + { + /// try to preserve replicas order if all replicas in cluster are used for query execution + /// it's important for data locality during query execution + auto priority_func = [](size_t i) { return Priority{static_cast(i)}; }; + shuffled_pool = shard.pool->getShuffledPools(current_settings, priority_func); + } shuffled_pool.resize(all_replicas_count); for (size_t i=0; i < all_replicas_count; ++i) From 4f153b59c06afa2c75e9f43b1279900d594359cd Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 7 Feb 2024 01:30:48 +0000 Subject: [PATCH 132/276] Minor fix --- src/Processors/QueryPlan/ReadFromRemote.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 91c81d619a7..93c73a66b78 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -416,7 +416,10 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder std::vector shuffled_pool; if (all_replicas_count < shard.getAllNodeCount()) + { shuffled_pool = shard.pool->getShuffledPools(current_settings); + shuffled_pool.resize(all_replicas_count); + } else { /// try to preserve replicas order if all replicas in cluster are used for query execution @@ -424,7 +427,6 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder auto priority_func = [](size_t i) { return Priority{static_cast(i)}; }; shuffled_pool = shard.pool->getShuffledPools(current_settings, priority_func); } - shuffled_pool.resize(all_replicas_count); for (size_t i=0; i < all_replicas_count; ++i) { From 06d112135ec0156155510653abdef429d7cb6283 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 5 Jan 2024 16:31:10 +0000 Subject: [PATCH 133/276] Simplify prewhere push down from query plan. Try to always use it. --- src/Interpreters/ActionsDAG.cpp | 11 +- src/Interpreters/ActionsDAG.h | 9 +- src/Interpreters/InterpreterSelectQuery.cpp | 56 +- .../Optimizations/liftUpFunctions.cpp | 2 +- .../Optimizations/optimizePrewhere.cpp | 509 ++++++++++-------- .../MergeTree/MergeTreeWhereOptimizer.cpp | 27 +- .../MergeTree/MergeTreeWhereOptimizer.h | 9 +- 7 files changed, 334 insertions(+), 289 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 6512def9202..03d7e620541 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1631,7 +1631,7 @@ void ActionsDAG::mergeNodes(ActionsDAG && second) } } -ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split_nodes) const +ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split_nodes, bool create_split_nodes_mapping) const { /// Split DAG into two parts. /// (first_nodes, first_outputs) is a part which will have split_list in result. @@ -1830,7 +1830,14 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split second_actions->outputs.swap(second_outputs); second_actions->inputs.swap(second_inputs); - return {std::move(first_actions), std::move(second_actions)}; + std::unordered_map split_nodes_mapping; + if (create_split_nodes_mapping) + { + for (const auto * node : split_nodes) + split_nodes_mapping[node] = data[node].to_first; + } + + return {std::move(first_actions), std::move(second_actions), std::move(split_nodes_mapping)}; } ActionsDAG::SplitResult ActionsDAG::splitActionsBeforeArrayJoin(const NameSet & array_joined_columns) const diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 45f6e5cc717..04683832c6d 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -326,13 +326,18 @@ public: /// Merge current nodes with specified dag nodes void mergeNodes(ActionsDAG && second); - using SplitResult = std::pair; + struct SplitResult + { + ActionsDAGPtr first; + ActionsDAGPtr second; + std::unordered_map split_nodes_mapping; + }; /// Split ActionsDAG into two DAGs, where first part contains all nodes from split_nodes and their children. /// Execution of first then second parts on block is equivalent to execution of initial DAG. /// First DAG and initial DAG have equal inputs, second DAG and initial DAG has equal outputs. /// Second DAG inputs may contain less inputs then first DAG (but also include other columns). - SplitResult split(std::unordered_set split_nodes) const; + SplitResult split(std::unordered_set split_nodes, bool create_split_nodes_mapping = false) const; /// Splits actions into two parts. Returned first half may be swapped with ARRAY JOIN. SplitResult splitActionsBeforeArrayJoin(const NameSet & array_joined_columns) const; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d0cf9f1160c..187518b9f6c 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -600,7 +600,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( query.setFinal(); } - auto analyze = [&] (bool try_move_to_prewhere) + auto analyze = [&] (bool) { /// Allow push down and other optimizations for VIEW: replace with subquery and rewrite it. ASTPtr view_table; @@ -632,37 +632,37 @@ InterpreterSelectQuery::InterpreterSelectQuery( view = nullptr; } - if (try_move_to_prewhere - && storage && storage->canMoveConditionsToPrewhere() - && query.where() && !query.prewhere() - && !query.hasJoin()) /// Join may produce rows with nulls or default values, it's difficult to analyze if they affected or not. - { - /// PREWHERE optimization: transfer some condition from WHERE to PREWHERE if enabled and viable - if (const auto & column_sizes = storage->getColumnSizes(); !column_sizes.empty()) - { - /// Extract column compressed sizes. - std::unordered_map column_compressed_sizes; - for (const auto & [name, sizes] : column_sizes) - column_compressed_sizes[name] = sizes.data_compressed; + // if (try_move_to_prewhere + // && storage && storage->canMoveConditionsToPrewhere() + // && query.where() && !query.prewhere() + // && !query.hasJoin()) /// Join may produce rows with nulls or default values, it's difficult to analyze if they affected or not. + // { + // /// PREWHERE optimization: transfer some condition from WHERE to PREWHERE if enabled and viable + // if (const auto & column_sizes = storage->getColumnSizes(); !column_sizes.empty()) + // { + // /// Extract column compressed sizes. + // std::unordered_map column_compressed_sizes; + // for (const auto & [name, sizes] : column_sizes) + // column_compressed_sizes[name] = sizes.data_compressed; - SelectQueryInfo current_info; - current_info.query = query_ptr; - current_info.syntax_analyzer_result = syntax_analyzer_result; + // SelectQueryInfo current_info; + // current_info.query = query_ptr; + // current_info.syntax_analyzer_result = syntax_analyzer_result; - Names queried_columns = syntax_analyzer_result->requiredSourceColumns(); - const auto & supported_prewhere_columns = storage->supportedPrewhereColumns(); + // Names queried_columns = syntax_analyzer_result->requiredSourceColumns(); + // const auto & supported_prewhere_columns = storage->supportedPrewhereColumns(); - MergeTreeWhereOptimizer where_optimizer{ - std::move(column_compressed_sizes), - metadata_snapshot, - storage->getConditionEstimatorByPredicate(query_info, storage_snapshot, context), - queried_columns, - supported_prewhere_columns, - log}; + // MergeTreeWhereOptimizer where_optimizer{ + // std::move(column_compressed_sizes), + // metadata_snapshot, + // storage->getConditionEstimatorByPredicate(query_info, storage_snapshot, context), + // queried_columns, + // supported_prewhere_columns, + // log}; - where_optimizer.optimize(current_info, context); - } - } + // where_optimizer.optimize(current_info, context); + // } + // } if (query.prewhere() && query.where()) { diff --git a/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp b/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp index 34a1fc2bb88..3fc2d64b11f 100644 --- a/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp @@ -66,7 +66,7 @@ size_t tryExecuteFunctionsAfterSorting(QueryPlan::Node * parent_node, QueryPlan: NameSet sort_columns; for (const auto & col : sorting_step->getSortDescription()) sort_columns.insert(col.column_name); - auto [needed_for_sorting, unneeded_for_sorting] = expression_step->getExpression()->splitActionsBySortingDescription(sort_columns); + auto [needed_for_sorting, unneeded_for_sorting, _] = expression_step->getExpression()->splitActionsBySortingDescription(sort_columns); // No calculations can be postponed. if (unneeded_for_sorting->trivial()) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index 7902b36f80e..b2ac34b4b24 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -5,7 +5,8 @@ #include #include #include -#include +#include "Functions/FunctionsLogical.h" +#include "Functions/IFunctionAdaptors.h" namespace DB { @@ -15,58 +16,58 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -namespace -{ +// namespace +// { -void matchDAGOutputNodesOrderWithHeader(ActionsDAGPtr & actions_dag, const Block & expected_header) -{ - std::unordered_map output_name_to_node; - for (const auto * output_node : actions_dag->getOutputs()) - output_name_to_node.emplace(output_node->result_name, output_node); +// void matchDAGOutputNodesOrderWithHeader(ActionsDAGPtr & actions_dag, const Block & expected_header) +// { +// std::unordered_map output_name_to_node; +// for (const auto * output_node : actions_dag->getOutputs()) +// output_name_to_node.emplace(output_node->result_name, output_node); - std::unordered_set used_output_nodes; +// std::unordered_set used_output_nodes; - ActionsDAG::NodeRawConstPtrs updated_outputs; - updated_outputs.reserve(expected_header.columns()); +// ActionsDAG::NodeRawConstPtrs updated_outputs; +// updated_outputs.reserve(expected_header.columns()); - for (const auto & column : expected_header) - { - auto output_node_it = output_name_to_node.find(column.name); - if (output_node_it == output_name_to_node.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Invalid move to PREWHERE optimization. Cannot find column {} in output", - column.name); +// for (const auto & column : expected_header) +// { +// auto output_node_it = output_name_to_node.find(column.name); +// if (output_node_it == output_name_to_node.end()) +// throw Exception(ErrorCodes::LOGICAL_ERROR, +// "Invalid move to PREWHERE optimization. Cannot find column {} in output", +// column.name); - updated_outputs.push_back(output_node_it->second); - used_output_nodes.insert(output_node_it->second); - } +// updated_outputs.push_back(output_node_it->second); +// used_output_nodes.insert(output_node_it->second); +// } - ActionsDAG::NodeRawConstPtrs unused_outputs; - for (const auto * output_node : actions_dag->getOutputs()) - { - if (used_output_nodes.contains(output_node)) - continue; +// ActionsDAG::NodeRawConstPtrs unused_outputs; +// for (const auto * output_node : actions_dag->getOutputs()) +// { +// if (used_output_nodes.contains(output_node)) +// continue; - unused_outputs.push_back(output_node); - } +// unused_outputs.push_back(output_node); +// } - auto & actions_dag_outputs = actions_dag->getOutputs(); - actions_dag_outputs = std::move(updated_outputs); - actions_dag_outputs.insert(actions_dag_outputs.end(), unused_outputs.begin(), unused_outputs.end()); -} +// auto & actions_dag_outputs = actions_dag->getOutputs(); +// actions_dag_outputs = std::move(updated_outputs); +// actions_dag_outputs.insert(actions_dag_outputs.end(), unused_outputs.begin(), unused_outputs.end()); +// } -} +// } namespace QueryPlanOptimizations { -void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) +void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) { if (stack.size() < 3) return; - const auto & frame = stack.back(); + auto & frame = stack.back(); /** Assume that on stack there are at least 3 nodes: * @@ -82,7 +83,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) if (storage_prewhere_info && storage_prewhere_info->prewhere_actions) return; - const QueryPlan::Node * filter_node = (stack.rbegin() + 1)->node; + QueryPlan::Node * filter_node = (stack.rbegin() + 1)->node; const auto * filter_step = typeid_cast(filter_node->step.get()); if (!filter_step) return; @@ -92,40 +93,40 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) * Collect input node to output nodes mapping. */ ColumnsWithTypeAndName required_columns_after_filter; - std::unordered_set output_nodes_mapped_to_input; - std::unordered_map> input_node_to_output_names; + // std::unordered_set output_nodes_mapped_to_input; + // std::unordered_map> input_node_to_output_names; - for (const auto * output_node : filter_step->getExpression()->getOutputs()) - { - const auto * node_without_alias = output_node; - while (node_without_alias->type == ActionsDAG::ActionType::ALIAS) - node_without_alias = node_without_alias->children[0]; + // for (const auto * output_node : filter_step->getExpression()->getOutputs()) + // { + // const auto * node_without_alias = output_node; + // while (node_without_alias->type == ActionsDAG::ActionType::ALIAS) + // node_without_alias = node_without_alias->children[0]; - if (node_without_alias->type == ActionsDAG::ActionType::INPUT) - { - output_nodes_mapped_to_input.emplace(output_node->result_name); + // if (node_without_alias->type == ActionsDAG::ActionType::INPUT) + // { + // output_nodes_mapped_to_input.emplace(output_node->result_name); - auto output_names_it = input_node_to_output_names.find(node_without_alias->result_name); - if (output_names_it == input_node_to_output_names.end()) - { - auto [insert_it, _] = input_node_to_output_names.emplace(node_without_alias->result_name, std::vector()); - output_names_it = insert_it; - } + // auto output_names_it = input_node_to_output_names.find(node_without_alias->result_name); + // if (output_names_it == input_node_to_output_names.end()) + // { + // auto [insert_it, _] = input_node_to_output_names.emplace(node_without_alias->result_name, std::vector()); + // output_names_it = insert_it; + // } - output_names_it->second.push_back(output_node->result_name); - } + // output_names_it->second.push_back(output_node->result_name); + // } - if (output_node->result_name == filter_step->getFilterColumnName() && filter_step->removesFilterColumn()) - continue; + // if (output_node->result_name == filter_step->getFilterColumnName() && filter_step->removesFilterColumn()) + // continue; - required_columns_after_filter.push_back(ColumnWithTypeAndName(output_node->result_type, output_node->result_name)); - } + // required_columns_after_filter.push_back(ColumnWithTypeAndName(output_node->result_type, output_node->result_name)); + // } const auto & context = read_from_merge_tree->getContext(); const auto & settings = context->getSettingsRef(); - if (!settings.allow_experimental_analyzer) - return; + // if (!settings.allow_experimental_analyzer) + // return; const auto & table_expression_modifiers = read_from_merge_tree->getQueryInfo().table_expression_modifiers; bool is_final = table_expression_modifiers && table_expression_modifiers->hasFinal(); @@ -170,7 +171,8 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) filter_step->getFilterColumnName(), read_from_merge_tree->getContext(), is_final); - if (!optimize_result.has_value()) + + if (!optimize_result.fully_moved_to_prewhere && optimize_result.prewhere_nodes.empty()) return; PrewhereInfoPtr prewhere_info; @@ -181,198 +183,243 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes) prewhere_info->need_filter = true; - auto & prewhere_filter_actions = optimize_result->prewhere_filter_actions; + // QueryPlan::Node * replace_old_filter_node = nullptr; + // bool remove_filter_node = false; - ActionsChain actions_chain; - - std::string prewere_filter_node_name = prewhere_filter_actions->getOutputs().at(0)->result_name; - actions_chain.addStep(std::make_unique(prewhere_filter_actions)); - - auto & filter_actions = optimize_result->filter_actions; - - /** Merge tree where optimizer splits conjunctions in filter expression into 2 parts: - * 1. Filter expressions. - * 2. Prewhere filter expressions. - * - * There can be cases when all expressions are moved to PREWHERE, but it is not - * enough to produce required filter output columns. - * - * Example: SELECT (a AND b) AS cond FROM test_table WHERE cond AND c; - * In this example condition expressions `a`, `b`, `c` can move to PREWHERE, but PREWHERE will not contain expression `and(a, b)`. - * It will contain only `a`, `b`, `c`, `and(a, b, c)` expressions. - * - * In such scenario we need to create additional step to calculate `and(a, b)` expression after PREWHERE. - */ - bool need_additional_filter_after_prewhere = false; - - if (!filter_actions) + if (!optimize_result.fully_moved_to_prewhere) { - /// Any node from PREWHERE filter actions can be used as possible output node - std::unordered_set possible_prewhere_output_nodes; - for (const auto & node : prewhere_filter_actions->getNodes()) - possible_prewhere_output_nodes.insert(node.result_name); + auto split_result = filter_step->getExpression()->split(optimize_result.prewhere_nodes, true); + ActionsDAG::NodeRawConstPtrs conditions; + conditions.reserve(split_result.split_nodes_mapping.size()); + for (const auto * condition : optimize_result.prewhere_nodes) + conditions.push_back(split_result.split_nodes_mapping.at(condition)); - for (auto & required_column : required_columns_after_filter) + prewhere_info->prewhere_actions = std::move(split_result.first); + prewhere_info->remove_prewhere_column = true; + + if (conditions.size() == 1) { - if (!possible_prewhere_output_nodes.contains(required_column.name) && - !output_nodes_mapped_to_input.contains(required_column.name)) + for (const auto * output : prewhere_info->prewhere_actions->getOutputs()) { - need_additional_filter_after_prewhere = true; - break; - } - } - } - - /** If there are additional filter actions after PREWHERE filter actions, we create filter actions dag using PREWHERE filter - * actions output columns as filter actions dag input columns. - * Then we merge this filter actions dag nodes with old filter step actions dag nodes, to reuse some expressions from - * PREWHERE filter actions. - */ - if (need_additional_filter_after_prewhere || filter_actions) - { - auto merged_filter_actions = std::make_shared(actions_chain.getLastStepAvailableOutputColumns()); - merged_filter_actions->getOutputs().clear(); - merged_filter_actions->mergeNodes(std::move(*filter_step->getExpression()->clone())); - - /// Add old filter step filter column to outputs - for (const auto & node : merged_filter_actions->getNodes()) - { - if (node.result_name == filter_step->getFilterColumnName()) - { - merged_filter_actions->getOutputs().push_back(&node); - break; - } - } - - filter_actions = std::move(merged_filter_actions); - - /// If there is filter after PREWHERE, we can ignore filtering during PREWHERE stage - prewhere_info->need_filter = false; - - actions_chain.addStep(std::make_unique(filter_actions)); - } - - auto required_output_actions = std::make_shared(required_columns_after_filter); - actions_chain.addStep(std::make_unique(required_output_actions)); - - actions_chain.finalize(); - - prewhere_filter_actions->projectInput(false); - - auto & prewhere_actions_chain_node = actions_chain[0]; - prewhere_info->prewhere_actions = std::move(prewhere_filter_actions); - prewhere_info->prewhere_column_name = prewere_filter_node_name; - prewhere_info->remove_prewhere_column = !prewhere_actions_chain_node->getChildRequiredOutputColumnsNames().contains(prewere_filter_node_name); - - read_from_merge_tree->updatePrewhereInfo(prewhere_info); - - QueryPlan::Node * replace_old_filter_node = nullptr; - bool remove_filter_node = false; - - if (filter_actions) - { - filter_actions->projectInput(false); - - /// Match dag output nodes with old filter step header - matchDAGOutputNodesOrderWithHeader(filter_actions, filter_step->getOutputStream().header); - - auto & filter_actions_chain_node = actions_chain[1]; - bool remove_filter_column = !filter_actions_chain_node->getChildRequiredOutputColumnsNames().contains(filter_step->getFilterColumnName()); - auto after_prewhere_filter_step = std::make_unique(read_from_merge_tree->getOutputStream(), - filter_actions, - filter_step->getFilterColumnName(), - remove_filter_column); - - auto & node = nodes.emplace_back(); - node.children.emplace_back(frame.node); - node.step = std::move(after_prewhere_filter_step); - - replace_old_filter_node = &node; - } - else - { - auto rename_actions_dag = std::make_shared(read_from_merge_tree->getOutputStream().header.getColumnsWithTypeAndName()); - bool apply_rename_step = false; - - ActionsDAG::NodeRawConstPtrs updated_outputs; - - /** If in output after read from merge tree there are column names without aliases, - * apply old filter step aliases to them. - */ - for (const auto * output_node : rename_actions_dag->getOutputs()) - { - const auto alias_it = input_node_to_output_names.find(output_node->result_name); - if (alias_it == input_node_to_output_names.end()) - { - updated_outputs.push_back(output_node); - continue; + if (output == conditions.front()) + prewhere_info->remove_prewhere_column = false; } - for (auto & output_name : alias_it->second) - { - if (output_name == output_node->result_name) - { - updated_outputs.push_back(output_node); - continue; - } - - updated_outputs.push_back(&rename_actions_dag->addAlias(*output_node, output_name)); - apply_rename_step = true; - } - } - - rename_actions_dag->getOutputs() = std::move(updated_outputs); - - bool apply_match_step = false; - - /// If column order does not match old filter step column order, match dag output nodes with header - if (!blocksHaveEqualStructure(read_from_merge_tree->getOutputStream().header, filter_step->getOutputStream().header)) - { - apply_match_step = true; - matchDAGOutputNodesOrderWithHeader(rename_actions_dag, filter_step->getOutputStream().header); - } - - if (apply_rename_step || apply_match_step) - { - auto rename_step = std::make_unique(read_from_merge_tree->getOutputStream(), rename_actions_dag); - if (apply_rename_step) - rename_step->setStepDescription("Change column names to column identifiers"); - - auto & node = nodes.emplace_back(); - node.children.emplace_back(frame.node); - node.step = std::move(rename_step); - - replace_old_filter_node = &node; + prewhere_info->prewhere_column_name = conditions.front()->result_name; } else { - replace_old_filter_node = frame.node; - remove_filter_node = true; + + FunctionOverloadResolverPtr func_builder_and = std::make_unique(std::make_shared()); + const auto * node = &prewhere_info->prewhere_actions->addFunction(func_builder_and, std::move(conditions), {}); + prewhere_info->prewhere_column_name = node->result_name; + prewhere_info->prewhere_actions->getOutputs().push_back(node); } + + read_from_merge_tree->updatePrewhereInfo(prewhere_info); + filter_node->step = std::make_unique( + read_from_merge_tree->getOutputStream(), + std::move(split_result.second), + filter_step->getFilterColumnName(), + filter_step->removesFilterColumn()); + + return; } + prewhere_info->prewhere_actions = filter_step->getExpression(); + prewhere_info->prewhere_column_name = filter_step->getFilterColumnName(); + prewhere_info->remove_prewhere_column = filter_step->removesFilterColumn(); + + read_from_merge_tree->updatePrewhereInfo(prewhere_info); + + // replace_old_filter_node = frame.node; + // remove_filter_node = true; + + + + // auto & prewhere_filter_actions = optimize_result->prewhere_filter_actions; + + // ActionsChain actions_chain; + + // std::string prewere_filter_node_name = prewhere_filter_actions->getOutputs().at(0)->result_name; + // actions_chain.addStep(std::make_unique(prewhere_filter_actions)); + + // auto & filter_actions = optimize_result->filter_actions; + + // /** Merge tree where optimizer splits conjunctions in filter expression into 2 parts: + // * 1. Filter expressions. + // * 2. Prewhere filter expressions. + // * + // * There can be cases when all expressions are moved to PREWHERE, but it is not + // * enough to produce required filter output columns. + // * + // * Example: SELECT (a AND b) AS cond FROM test_table WHERE cond AND c; + // * In this example condition expressions `a`, `b`, `c` can move to PREWHERE, but PREWHERE will not contain expression `and(a, b)`. + // * It will contain only `a`, `b`, `c`, `and(a, b, c)` expressions. + // * + // * In such scenario we need to create additional step to calculate `and(a, b)` expression after PREWHERE. + // */ + // bool need_additional_filter_after_prewhere = false; + + // if (!filter_actions) + // { + // /// Any node from PREWHERE filter actions can be used as possible output node + // std::unordered_set possible_prewhere_output_nodes; + // for (const auto & node : prewhere_filter_actions->getNodes()) + // possible_prewhere_output_nodes.insert(node.result_name); + + // for (auto & required_column : required_columns_after_filter) + // { + // if (!possible_prewhere_output_nodes.contains(required_column.name) && + // !output_nodes_mapped_to_input.contains(required_column.name)) + // { + // need_additional_filter_after_prewhere = true; + // break; + // } + // } + // } + + // /** If there are additional filter actions after PREWHERE filter actions, we create filter actions dag using PREWHERE filter + // * actions output columns as filter actions dag input columns. + // * Then we merge this filter actions dag nodes with old filter step actions dag nodes, to reuse some expressions from + // * PREWHERE filter actions. + // */ + // if (need_additional_filter_after_prewhere || filter_actions) + // { + // auto merged_filter_actions = std::make_shared(actions_chain.getLastStepAvailableOutputColumns()); + // merged_filter_actions->getOutputs().clear(); + // merged_filter_actions->mergeNodes(std::move(*filter_step->getExpression()->clone())); + + // /// Add old filter step filter column to outputs + // for (const auto & node : merged_filter_actions->getNodes()) + // { + // if (node.result_name == filter_step->getFilterColumnName()) + // { + // merged_filter_actions->getOutputs().push_back(&node); + // break; + // } + // } + + // filter_actions = std::move(merged_filter_actions); + + // /// If there is filter after PREWHERE, we can ignore filtering during PREWHERE stage + // prewhere_info->need_filter = false; + + // actions_chain.addStep(std::make_unique(filter_actions)); + // } + + // auto required_output_actions = std::make_shared(required_columns_after_filter); + // actions_chain.addStep(std::make_unique(required_output_actions)); + + // actions_chain.finalize(); + + // prewhere_filter_actions->projectInput(false); + + // auto & prewhere_actions_chain_node = actions_chain[0]; + // prewhere_info->prewhere_actions = std::move(prewhere_filter_actions); + // prewhere_info->prewhere_column_name = prewere_filter_node_name; + // prewhere_info->remove_prewhere_column = !prewhere_actions_chain_node->getChildRequiredOutputColumnsNames().contains(prewere_filter_node_name); + + // read_from_merge_tree->updatePrewhereInfo(prewhere_info); + + // QueryPlan::Node * replace_old_filter_node = nullptr; + // bool remove_filter_node = false; + + // if (filter_actions) + // { + // filter_actions->projectInput(false); + + // /// Match dag output nodes with old filter step header + // matchDAGOutputNodesOrderWithHeader(filter_actions, filter_step->getOutputStream().header); + + // auto & filter_actions_chain_node = actions_chain[1]; + // bool remove_filter_column = !filter_actions_chain_node->getChildRequiredOutputColumnsNames().contains(filter_step->getFilterColumnName()); + // auto after_prewhere_filter_step = std::make_unique(read_from_merge_tree->getOutputStream(), + // filter_actions, + // filter_step->getFilterColumnName(), + // remove_filter_column); + + // auto & node = nodes.emplace_back(); + // node.children.emplace_back(frame.node); + // node.step = std::move(after_prewhere_filter_step); + + // replace_old_filter_node = &node; + // } + // else + // { + // auto rename_actions_dag = std::make_shared(read_from_merge_tree->getOutputStream().header.getColumnsWithTypeAndName()); + // bool apply_rename_step = false; + + // ActionsDAG::NodeRawConstPtrs updated_outputs; + + // /** If in output after read from merge tree there are column names without aliases, + // * apply old filter step aliases to them. + // */ + // for (const auto * output_node : rename_actions_dag->getOutputs()) + // { + // const auto alias_it = input_node_to_output_names.find(output_node->result_name); + // if (alias_it == input_node_to_output_names.end()) + // { + // updated_outputs.push_back(output_node); + // continue; + // } + + // for (auto & output_name : alias_it->second) + // { + // if (output_name == output_node->result_name) + // { + // updated_outputs.push_back(output_node); + // continue; + // } + + // updated_outputs.push_back(&rename_actions_dag->addAlias(*output_node, output_name)); + // apply_rename_step = true; + // } + // } + + // rename_actions_dag->getOutputs() = std::move(updated_outputs); + + // bool apply_match_step = false; + + // /// If column order does not match old filter step column order, match dag output nodes with header + // if (!blocksHaveEqualStructure(read_from_merge_tree->getOutputStream().header, filter_step->getOutputStream().header)) + // { + // apply_match_step = true; + // matchDAGOutputNodesOrderWithHeader(rename_actions_dag, filter_step->getOutputStream().header); + // } + + // if (apply_rename_step || apply_match_step) + // { + // auto rename_step = std::make_unique(read_from_merge_tree->getOutputStream(), rename_actions_dag); + // if (apply_rename_step) + // rename_step->setStepDescription("Change column names to column identifiers"); + + // auto & node = nodes.emplace_back(); + // node.children.emplace_back(frame.node); + // node.step = std::move(rename_step); + + // replace_old_filter_node = &node; + // } + // else + // { + // replace_old_filter_node = frame.node; + // remove_filter_node = true; + // } + // } + QueryPlan::Node * filter_parent_node = (stack.rbegin() + 2)->node; for (auto & filter_parent_child : filter_parent_node->children) { if (filter_parent_child == filter_node) { - filter_parent_child = replace_old_filter_node; + filter_parent_child = frame.node; size_t stack_size = stack.size(); - /** If filter step is completely replaced with PREWHERE filter actions, remove it from stack. - * Otherwise replace old filter step with new filter step after PREWHERE. - */ - if (remove_filter_node) - { - std::swap(stack[stack_size - 1], stack[stack_size - 2]); - stack.pop_back(); - } - else - { - stack[stack_size - 2] = Frame{.node = replace_old_filter_node, .next_child = 1}; - } + /// Step is completely replaced with PREWHERE filter actions, remove it from stack. + std::swap(stack[stack_size - 1], stack[stack_size - 2]); + stack.pop_back(); break; } diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 4aecf85ac2a..151ce7635b2 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -112,7 +112,7 @@ void MergeTreeWhereOptimizer::optimize(SelectQueryInfo & select_query_info, cons LOG_DEBUG(log, "MergeTreeWhereOptimizer: condition \"{}\" moved to PREWHERE", select.prewhere()->formatForLogging(log_queries_cut_to_length)); } -std::optional MergeTreeWhereOptimizer::optimize(const ActionsDAGPtr & filter_dag, +MergeTreeWhereOptimizer::FilterActionsOptimizeResult MergeTreeWhereOptimizer::optimize(const ActionsDAGPtr & filter_dag, const std::string & filter_column_name, const ContextPtr & context, bool is_final) @@ -132,11 +132,14 @@ std::optional MergeTreeWhe if (!optimize_result) return {}; - auto filter_actions = reconstructDAG(optimize_result->where_conditions); - auto prewhere_filter_actions = reconstructDAG(optimize_result->prewhere_conditions); + if (optimize_result->where_conditions.empty()) + return {.prewhere_nodes = {}, .fully_moved_to_prewhere = true}; - FilterActionsOptimizeResult result = { std::move(filter_actions), std::move(prewhere_filter_actions) }; - return result; + std::unordered_set prewhere_conditions; + for (const auto & condition : optimize_result->prewhere_conditions) + prewhere_conditions.insert(condition.node.getDAGNode()); + + return {.prewhere_nodes = std::move(prewhere_conditions), .fully_moved_to_prewhere = false}; } static void collectColumns(const RPNBuilderTreeNode & node, const NameSet & columns_names, NameSet & result_set, bool & has_invalid_column) @@ -343,20 +346,6 @@ ASTPtr MergeTreeWhereOptimizer::reconstructAST(const Conditions & conditions) return function; } -ActionsDAGPtr MergeTreeWhereOptimizer::reconstructDAG(const Conditions & conditions) -{ - if (conditions.empty()) - return {}; - - ActionsDAG::NodeRawConstPtrs filter_nodes; - filter_nodes.reserve(conditions.size()); - - for (const auto & condition : conditions) - filter_nodes.push_back(condition.node.getDAGNode()); - - return ActionsDAG::buildFilterActionsDAG(filter_nodes); -} - std::optional MergeTreeWhereOptimizer::optimizeImpl(const RPNBuilderTreeNode & node, const WhereOptimizerContext & where_optimizer_context) const { diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h index b56219e3c59..84afa4cda17 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h @@ -47,11 +47,11 @@ public: struct FilterActionsOptimizeResult { - ActionsDAGPtr filter_actions; - ActionsDAGPtr prewhere_filter_actions; + std::unordered_set prewhere_nodes; + bool fully_moved_to_prewhere = false; }; - std::optional optimize(const ActionsDAGPtr & filter_dag, + FilterActionsOptimizeResult optimize(const ActionsDAGPtr & filter_dag, const std::string & filter_column_name, const ContextPtr & context, bool is_final); @@ -122,9 +122,6 @@ private: /// Reconstruct AST from conditions static ASTPtr reconstructAST(const Conditions & conditions); - /// Reconstruct DAG from conditions - static ActionsDAGPtr reconstructDAG(const Conditions & conditions); - void optimizeArbitrary(ASTSelectQuery & select) const; UInt64 getColumnsSize(const NameSet & columns) const; From fbd71ee15e22f68d4155437201a83d5c8133c203 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 5 Jan 2024 20:46:13 +0000 Subject: [PATCH 134/276] Fixing style. --- src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index b2ac34b4b24..4cea74b9b12 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -235,8 +235,6 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) // replace_old_filter_node = frame.node; // remove_filter_node = true; - - // auto & prewhere_filter_actions = optimize_result->prewhere_filter_actions; // ActionsChain actions_chain; From d1902cdba0b9fce3c621e1266e9e004fe2a21daf Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 9 Jan 2024 16:31:16 +0000 Subject: [PATCH 135/276] Fix some tests. --- src/Interpreters/ActionsDAG.cpp | 42 +++-- .../Optimizations/optimizePrewhere.cpp | 151 ++++++++++++------ .../QueryPlan/ReadFromMergeTree.cpp | 79 +++++++-- .../MergeTree/MergeTreeSelectProcessor.cpp | 2 +- .../MergeTree/MergeTreeWhereOptimizer.cpp | 6 +- .../02235_add_part_offset_virtual_column.sql | 4 +- 6 files changed, 208 insertions(+), 76 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 03d7e620541..5a1f9a87974 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1765,13 +1765,13 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split } /// Input from second DAG should also be in the first. - if (copy.type == ActionType::INPUT) - { - auto & input_copy = first_nodes.emplace_back(*cur.node); - assert(cur_data.to_first == nullptr); - cur_data.to_first = &input_copy; - new_inputs.push_back(cur.node); - } + // if (copy.type == ActionType::INPUT) + // { + // auto & input_copy = first_nodes.emplace_back(*cur.node); + // assert(cur_data.to_first == nullptr); + // cur_data.to_first = &input_copy; + // new_inputs.push_back(cur.node); + // } } else { @@ -1790,11 +1790,12 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split /// If this node is needed in result, add it as input. Node input_node; input_node.type = ActionType::INPUT; - input_node.result_type = node.result_type; - input_node.result_name = node.result_name; + input_node.result_type = cur.node->result_type; + input_node.result_name = cur.node->result_name; cur_data.to_second = &second_nodes.emplace_back(std::move(input_node)); - new_inputs.push_back(cur.node); + if (cur.node->type != ActionType::INPUT) + new_inputs.push_back(cur.node); } } } @@ -1810,14 +1811,29 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split for (const auto * input_node : inputs) { const auto & cur = data[input_node]; - first_inputs.push_back(cur.to_first); + if (cur.to_first) + { + first_inputs.push_back(cur.to_first); + + if (cur.to_second) + first_outputs.push_back(cur.to_first); + } } for (const auto * input : new_inputs) { const auto & cur = data[input]; - second_inputs.push_back(cur.to_second); - first_outputs.push_back(cur.to_first); + if (cur.to_second) + second_inputs.push_back(cur.to_second); + if (cur.to_first) + first_outputs.push_back(cur.to_first); + } + + for (const auto * input_node : inputs) + { + const auto & cur = data[input_node]; + if (cur.to_second) + second_inputs.push_back(cur.to_second); } auto first_actions = std::make_shared(); diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index 4cea74b9b12..a9405d0cbdb 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -62,6 +62,20 @@ namespace ErrorCodes namespace QueryPlanOptimizations { +static void removeFromOutput(ActionsDAG & dag, const std::string name) +{ + const auto * node = &dag.findInOutputs(name); + auto & outputs = dag.getOutputs(); + for (size_t i = 0; i < outputs.size(); ++i) + { + if (node == outputs[i]) + { + outputs.erase(outputs.begin() + i); + return; + } + } +} + void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) { if (stack.size() < 3) @@ -172,7 +186,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) read_from_merge_tree->getContext(), is_final); - if (!optimize_result.fully_moved_to_prewhere && optimize_result.prewhere_nodes.empty()) + if (optimize_result.prewhere_nodes.empty()) return; PrewhereInfoPtr prewhere_info; @@ -182,55 +196,102 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) prewhere_info = std::make_shared(); prewhere_info->need_filter = true; + // std::cerr << filter_step->getExpression()->dumpDAG() << std::endl; // QueryPlan::Node * replace_old_filter_node = nullptr; // bool remove_filter_node = false; - if (!optimize_result.fully_moved_to_prewhere) - { - auto split_result = filter_step->getExpression()->split(optimize_result.prewhere_nodes, true); - ActionsDAG::NodeRawConstPtrs conditions; - conditions.reserve(split_result.split_nodes_mapping.size()); - for (const auto * condition : optimize_result.prewhere_nodes) - conditions.push_back(split_result.split_nodes_mapping.at(condition)); + auto filter_expression = filter_step->getExpression(); + const auto & filter_column_name = filter_step->getFilterColumnName(); - prewhere_info->prewhere_actions = std::move(split_result.first); + if (optimize_result.fully_moved_to_prewhere && filter_step->removesFilterColumn()) + { + removeFromOutput(*filter_expression, filter_column_name); + auto & outputs = filter_expression->getOutputs(); + size_t size = outputs.size(); + outputs.insert(outputs.end(), optimize_result.prewhere_nodes.begin(), optimize_result.prewhere_nodes.end()); + filter_expression->removeUnusedActions(false); + outputs.resize(size); + } + + // std::cerr << "!!!!!!!!!!!!!!!!\n"; + + // if (!optimize_result.fully_moved_to_prewhere) + // { + auto split_result = filter_step->getExpression()->split(optimize_result.prewhere_nodes, true); + + // std::cerr << split_result.first->dumpDAG() << std::endl; + // std::cerr << split_result.second->dumpDAG() << std::endl; + + // for (const auto * input : split_result.first->getInputs()) + // std::cerr << "in 1" << input->result_name << std::endl; + // for (const auto * input : split_result.second->getInputs()) + // std::cerr << "in 2" << input->result_name << std::endl; + + ActionsDAG::NodeRawConstPtrs conditions; + conditions.reserve(split_result.split_nodes_mapping.size()); + for (const auto * condition : optimize_result.prewhere_nodes) + { + // std::cerr << ".. " << condition->result_name << std::endl; + conditions.push_back(split_result.split_nodes_mapping.at(condition)); + } + + prewhere_info->prewhere_actions = std::move(split_result.first); + prewhere_info->remove_prewhere_column = optimize_result.fully_moved_to_prewhere && filter_step->removesFilterColumn(); + + if (conditions.size() == 1) + { + prewhere_info->prewhere_column_name = conditions.front()->result_name; + prewhere_info->prewhere_actions->getOutputs().push_back(conditions.front()); + } + else + { prewhere_info->remove_prewhere_column = true; - if (conditions.size() == 1) - { - for (const auto * output : prewhere_info->prewhere_actions->getOutputs()) - { - if (output == conditions.front()) - prewhere_info->remove_prewhere_column = false; - } + FunctionOverloadResolverPtr func_builder_and = std::make_unique(std::make_shared()); + const auto * node = &prewhere_info->prewhere_actions->addFunction(func_builder_and, std::move(conditions), {}); + prewhere_info->prewhere_column_name = node->result_name; + prewhere_info->prewhere_actions->getOutputs().push_back(node); + } - prewhere_info->prewhere_column_name = conditions.front()->result_name; - } - else - { + // std::cerr << read_from_merge_tree->getOutputStream().header.dumpStructure() << std::endl; + // std::cerr << read_from_merge_tree->getOutputStream().header.dumpIndex() << std::endl; - FunctionOverloadResolverPtr func_builder_and = std::make_unique(std::make_shared()); - const auto * node = &prewhere_info->prewhere_actions->addFunction(func_builder_and, std::move(conditions), {}); - prewhere_info->prewhere_column_name = node->result_name; - prewhere_info->prewhere_actions->getOutputs().push_back(node); - } + read_from_merge_tree->updatePrewhereInfo(prewhere_info); - read_from_merge_tree->updatePrewhereInfo(prewhere_info); + // std::cerr << read_from_merge_tree->getOutputStream().header.dumpStructure() << std::endl; + // std::cerr << read_from_merge_tree->getOutputStream().header.dumpIndex() << std::endl; + + if (!optimize_result.fully_moved_to_prewhere) + { filter_node->step = std::make_unique( read_from_merge_tree->getOutputStream(), std::move(split_result.second), filter_step->getFilterColumnName(), filter_step->removesFilterColumn()); - - return; } + else + { + // std::cerr << split_result.second->dumpDAG() << std::endl; + // std::cerr << read_from_merge_tree->getOutputStream().header.dumpStructure() << std::endl; + // std::cerr << read_from_merge_tree->getOutputStream().header.dumpIndex() << std::endl; - prewhere_info->prewhere_actions = filter_step->getExpression(); - prewhere_info->prewhere_column_name = filter_step->getFilterColumnName(); - prewhere_info->remove_prewhere_column = filter_step->removesFilterColumn(); + filter_node->step = std::make_unique( + read_from_merge_tree->getOutputStream(), + std::move(split_result.second)); + } + // return; + // } - read_from_merge_tree->updatePrewhereInfo(prewhere_info); + // std::cerr << "!!!!!!!!!!!!!!!!\n"; + + // prewhere_info->prewhere_actions = filter_step->getExpression(); + // prewhere_info->prewhere_actions->projectInput(false); + // std::cerr << prewhere_info->prewhere_actions->dumpDAG() << std::endl; + // prewhere_info->prewhere_column_name = filter_step->getFilterColumnName(); + // prewhere_info->remove_prewhere_column = filter_step->removesFilterColumn(); + + // read_from_merge_tree->updatePrewhereInfo(prewhere_info); // replace_old_filter_node = frame.node; // remove_filter_node = true; @@ -405,23 +466,23 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) // } // } - QueryPlan::Node * filter_parent_node = (stack.rbegin() + 2)->node; + // QueryPlan::Node * filter_parent_node = (stack.rbegin() + 2)->node; - for (auto & filter_parent_child : filter_parent_node->children) - { - if (filter_parent_child == filter_node) - { - filter_parent_child = frame.node; + // for (auto & filter_parent_child : filter_parent_node->children) + // { + // if (filter_parent_child == filter_node) + // { + // filter_parent_child = frame.node; - size_t stack_size = stack.size(); + // size_t stack_size = stack.size(); - /// Step is completely replaced with PREWHERE filter actions, remove it from stack. - std::swap(stack[stack_size - 1], stack[stack_size - 2]); - stack.pop_back(); + // /// Step is completely replaced with PREWHERE filter actions, remove it from stack. + // std::swap(stack[stack_size - 1], stack[stack_size - 2]); + // stack.pop_back(); - break; - } - } + // break; + // } + // } } } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 5ed56f59fc1..6adc48d87d2 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -89,6 +89,34 @@ size_t countPartitions(const MergeTreeData::DataPartsVector & prepared_parts) return countPartitions(prepared_parts, get_partition_id); } +bool restoreDAGInputs(ActionsDAG & dag, const NameSet & inputs) +{ + std::unordered_set outputs(dag.getOutputs().begin(), dag.getOutputs().end()); + bool added = false; + for (const auto * input : dag.getInputs()) + { + if (inputs.contains(input->result_name) && !outputs.contains(input)) + { + dag.getOutputs().push_back(input); + added = true; + } + } + + return added; +} + +bool restorePrewhereInputs(PrewhereInfo & info, const NameSet & inputs) +{ + bool added = false; + if (info.row_level_filter) + added = added || restoreDAGInputs(*info.row_level_filter, inputs); + + if (info.prewhere_actions) + added = added || restoreDAGInputs(*info.prewhere_actions, inputs); + + return added; +} + } namespace ProfileEvents @@ -786,18 +814,13 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( /// To fix this, we prohibit removing any input in prewhere actions. Instead, projection actions will be added after sorting. /// See 02354_read_in_order_prewhere.sql as an example. bool have_input_columns_removed_after_prewhere = false; - if (prewhere_info && prewhere_info->prewhere_actions) + if (prewhere_info) { - auto & outputs = prewhere_info->prewhere_actions->getOutputs(); - std::unordered_set outputs_set(outputs.begin(), outputs.end()); - for (const auto * input : prewhere_info->prewhere_actions->getInputs()) - { - if (!outputs_set.contains(input)) - { - outputs.push_back(input); - have_input_columns_removed_after_prewhere = true; - } - } + NameSet sorting_columns; + for (const auto & column : metadata_for_reading->getSortingKey().expression->getRequiredColumnsWithTypes()) + sorting_columns.insert(column.name); + + have_input_columns_removed_after_prewhere = restorePrewhereInputs(*prewhere_info, sorting_columns); } /// Let's split ranges to avoid reading much data. @@ -984,7 +1007,6 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( /// Thus we need to merge all partition parts into a single sorted stream. Pipe pipe = Pipe::unitePipes(std::move(pipes)); merge_streams(pipe); - out_projection = createProjection(pipe_header); return pipe; } @@ -1133,6 +1155,14 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal( auto sorting_expr = std::make_shared(metadata_for_reading->getSortingKey().expression->getActionsDAG().clone()); + if (prewhere_info) + { + NameSet sorting_columns; + for (const auto & column : metadata_for_reading->getSortingKey().expression->getRequiredColumnsWithTypes()) + sorting_columns.insert(column.name); + restorePrewhereInputs(*prewhere_info, sorting_columns); + } + for (size_t range_index = 0; range_index < parts_to_merge_ranges.size() - 1; ++range_index) { /// If do_not_merge_across_partitions_select_final is true and there is only one part in partition @@ -1802,13 +1832,20 @@ Pipe ReadFromMergeTree::spreadMarkRanges( if (!final && result.sampling.use_sampling) { + NameSet sampling_columns; + /// Add columns needed for `sample_by_ast` to `column_names_to_read`. /// Skip this if final was used, because such columns were already added from PK. for (const auto & column : result.sampling.filter_expression->getRequiredColumns().getNames()) { if (!names.contains(column)) column_names_to_read.push_back(column); + + sampling_columns.insert(column); } + + if (prewhere_info) + restorePrewhereInputs(*prewhere_info, sampling_columns); } if (final) @@ -2002,6 +2039,24 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons }); } + /// Some extra columns could be added by sample/final/in-order/etc + /// Remove them from header if not needed. + if (!blocksHaveEqualStructure(pipe.getHeader(), getOutputStream().header)) + { + auto convert_actions_dag = ActionsDAG::makeConvertingActions( + pipe.getHeader().getColumnsWithTypeAndName(), + getOutputStream().header.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name, + true); + + auto converting_dag_expr = std::make_shared(convert_actions_dag); + + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, converting_dag_expr); + }); + } + for (const auto & processor : pipe.getProcessors()) processors.emplace_back(processor); diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index aeff438f509..4e93bd267ec 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -449,8 +449,8 @@ Block MergeTreeSelectProcessor::applyPrewhereActions(Block block, const Prewhere Block MergeTreeSelectProcessor::transformHeader( Block block, const PrewhereInfoPtr & prewhere_info, const DataTypePtr & partition_value_type, const Names & virtual_columns) { + injectVirtualColumns(block, 0, nullptr, partition_value_type, virtual_columns); auto transformed = applyPrewhereActions(std::move(block), prewhere_info); - injectVirtualColumns(transformed, 0, nullptr, partition_value_type, virtual_columns); return transformed; } diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 151ce7635b2..c52a2fee051 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -132,14 +132,14 @@ MergeTreeWhereOptimizer::FilterActionsOptimizeResult MergeTreeWhereOptimizer::op if (!optimize_result) return {}; - if (optimize_result->where_conditions.empty()) - return {.prewhere_nodes = {}, .fully_moved_to_prewhere = true}; + // if (optimize_result->where_conditions.empty()) + // return {.prewhere_nodes = {}, .fully_moved_to_prewhere = true}; std::unordered_set prewhere_conditions; for (const auto & condition : optimize_result->prewhere_conditions) prewhere_conditions.insert(condition.node.getDAGNode()); - return {.prewhere_nodes = std::move(prewhere_conditions), .fully_moved_to_prewhere = false}; + return {.prewhere_nodes = std::move(prewhere_conditions), .fully_moved_to_prewhere = optimize_result->where_conditions.empty()}; } static void collectColumns(const RPNBuilderTreeNode & node, const NameSet & columns_names, NameSet & result_set, bool & has_invalid_column) diff --git a/tests/queries/0_stateless/02235_add_part_offset_virtual_column.sql b/tests/queries/0_stateless/02235_add_part_offset_virtual_column.sql index dc8fceddc52..73ae6eb499f 100644 --- a/tests/queries/0_stateless/02235_add_part_offset_virtual_column.sql +++ b/tests/queries/0_stateless/02235_add_part_offset_virtual_column.sql @@ -52,7 +52,7 @@ SELECT _part_offset, foo FROM t_1 where granule == 0 AND _part_offset >= 100000 SELECT 'PREWHERE'; SELECT count(*), sum(_part_offset), sum(order_0) from t_1 prewhere granule == 0 where _part_offset >= 100000; -SELECT count(*), sum(_part_offset), sum(order_0) from t_1 prewhere _part != '' where granule == 0; -- { serverError 10 } -SELECT count(*), sum(_part_offset), sum(order_0) from t_1 prewhere _part_offset > 100000 where granule == 0; -- { serverError 10 } +SELECT count(*), sum(_part_offset), sum(order_0) from t_1 prewhere _part != '' where granule == 0; -- { serverError 10, 16 } +SELECT count(*), sum(_part_offset), sum(order_0) from t_1 prewhere _part_offset > 100000 where granule == 0; -- { serverError 10, 16 } SELECT _part_offset FROM t_1 PREWHERE order_0 % 10000 == 42 ORDER BY order_0 LIMIT 3; SELECT _part_offset, foo FROM t_1 PREWHERE order_0 % 10000 == 42 ORDER BY order_0 LIMIT 3; From df0c30878eaca90c3eb4426e93727481b9ed4f0b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 19 Jan 2024 10:44:58 +0000 Subject: [PATCH 136/276] Fixing style --- src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index a9405d0cbdb..6872141951e 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -214,8 +214,6 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) outputs.resize(size); } - // std::cerr << "!!!!!!!!!!!!!!!!\n"; - // if (!optimize_result.fully_moved_to_prewhere) // { auto split_result = filter_step->getExpression()->split(optimize_result.prewhere_nodes, true); @@ -283,8 +281,6 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) // return; // } - // std::cerr << "!!!!!!!!!!!!!!!!\n"; - // prewhere_info->prewhere_actions = filter_step->getExpression(); // prewhere_info->prewhere_actions->projectInput(false); // std::cerr << prewhere_info->prewhere_actions->dumpDAG() << std::endl; From 41d624317064bc2a2784b6afaf3b3f22548c2f2b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 7 Feb 2024 12:43:26 +0000 Subject: [PATCH 137/276] Fix PreparedSets --- src/Interpreters/PreparedSets.cpp | 9 +-------- src/Interpreters/PreparedSets.h | 2 +- 2 files changed, 2 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 946eef08ce3..76f75cde1dc 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -205,18 +205,11 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) } } - if (!set_and_key->set->hasSetElements()) - set_and_key->set->fillSetElements(); - - return buildSetInplace(context); -} - -SetPtr FutureSetFromSubquery::buildSetInplace(const ContextPtr & context) -{ auto plan = build(context); if (!plan) return nullptr; + set_and_key->set->fillSetElements(); auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); pipeline.complete(std::make_shared(Block())); diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index a65f30351db..3419d3b6839 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -111,9 +111,9 @@ public: SetPtr get() const override; DataTypes getTypes() const override; SetPtr buildOrderedSetInplace(const ContextPtr & context) override; - SetPtr buildSetInplace(const ContextPtr & context); std::unique_ptr build(const ContextPtr & context); + void buildSetInplace(const ContextPtr & context); QueryTreeNodePtr detachQueryTree() { return std::move(query_tree); } void setQueryPlan(std::unique_ptr source_); From e8c0637fe2eebb2a369a79ac2c2e8b381e7a726c Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 2 Feb 2024 22:53:28 +0100 Subject: [PATCH 138/276] fix --- src/Processors/Transforms/AggregatingTransform.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 2fd9f102159..74da97f2199 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -600,6 +600,12 @@ IProcessor::Status AggregatingTransform::prepare() if (is_consume_finished) { output.finish(); + /// input.isFinished() means that merging is done. Now we can release our reference to aggregation states. + /// TODO: there is another case, when output port is getting closed first. + /// E.g. `select ... group by x limit 10`, if it was two-level aggregation and first few buckets contained already enough rows + /// limit will stop merging. It turned out to be not trivial to both release aggregation states and ensure that + /// ManyAggregatedData holds the last references to them to trigger parallel destruction in its dtor. Will work on that. + many_data.reset(); return Status::Finished; } else @@ -828,8 +834,6 @@ void AggregatingTransform::initGenerate() processors = Pipe::detachProcessors(std::move(pipe)); } - - many_data.reset(); } } From a64d9cf1c0ab4294a8ea9c83fa7d03bd6b7fe4c6 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 7 Feb 2024 14:15:49 +0100 Subject: [PATCH 139/276] add test --- .../02982_aggregation_states_destruction.reference | 1 + .../02982_aggregation_states_destruction.sh | 14 ++++++++++++++ 2 files changed, 15 insertions(+) create mode 100644 tests/queries/0_stateless/02982_aggregation_states_destruction.reference create mode 100755 tests/queries/0_stateless/02982_aggregation_states_destruction.sh diff --git a/tests/queries/0_stateless/02982_aggregation_states_destruction.reference b/tests/queries/0_stateless/02982_aggregation_states_destruction.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02982_aggregation_states_destruction.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02982_aggregation_states_destruction.sh b/tests/queries/0_stateless/02982_aggregation_states_destruction.sh new file mode 100755 index 00000000000..1c72cf2b8c1 --- /dev/null +++ b/tests/queries/0_stateless/02982_aggregation_states_destruction.sh @@ -0,0 +1,14 @@ +#!/usr/bin/env bash +# Tags: no-random-settings + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +query_id="02982_$RANDOM" +$CLICKHOUSE_CLIENT --query_id $query_id --log_query_threads 1 --query="select number, uniq(number) from numbers_mt(1e7) group by number limit 100 format Null;" + +$CLICKHOUSE_CLIENT -q "system flush logs;" + +$CLICKHOUSE_CLIENT -q "select count() > 1 from system.query_thread_log where query_id = '$query_id' and current_database = currentDatabase() and thread_name = 'AggregDestruct';" From 99be5b5cb1d49222dc85c184e51cc9f83aa8de15 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 7 Feb 2024 15:57:52 +0100 Subject: [PATCH 140/276] Fix --- src/Coordination/KeeperSnapshotManager.cpp | 14 +++++----- src/Coordination/KeeperStorage.cpp | 32 ++++++++++++---------- src/Coordination/KeeperStorage.h | 19 +++++++++++-- src/Coordination/ZooKeeperDataReader.cpp | 6 ++-- 4 files changed, 46 insertions(+), 25 deletions(-) diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index c176536dfbf..07116fedfab 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -16,7 +16,7 @@ #include #include #include -#include "Core/Field.h" +#include #include @@ -86,10 +86,10 @@ namespace writeBinary(node.version, out); writeBinary(node.cversion, out); writeBinary(node.aversion, out); - const bool is_ephemeral = node.isEphemeral(); - writeBinary(is_ephemeral ? node.ephemeralOwner() : 0, out); + writeBinary(node.ephemeralOwner(), out); if (version < SnapshotVersion::V6) writeBinary(static_cast(node.data_size), out); + const bool is_ephemeral = node.isEphemeral(); writeBinary(is_ephemeral ? 0 : node.numChildren(), out); writeBinary(node.pzxid, out); @@ -162,15 +162,15 @@ namespace } int32_t num_children = 0; readBinary(num_children, in); - if (num_children != 0) - node.ephemeral_or_children_data.children_info.num_children = num_children; + if (ephemeral_owner == 0) + node.setNumChildren(num_children); readBinary(node.pzxid, in); int32_t seq_num = 0; readBinary(seq_num, in); - if (seq_num != 0) - node.ephemeral_or_children_data.children_info.seq_num = seq_num; + if (ephemeral_owner == 0) + node.setSeqNum(seq_num); if (version >= SnapshotVersion::V4 && version <= SnapshotVersion::V5) { diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index ac4a9433e30..e855274f08b 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -172,7 +172,11 @@ uint64_t calculateDigest(std::string_view path, const KeeperStorage::Node & node hash.update(path); - hash.update(node.data, node.data_size); + if (node.data_size != 0) + { + chassert(node.data != nullptr); + hash.update(node.data, node.data_size); + } hash.update(node.czxid); hash.update(node.mzxid); @@ -369,7 +373,7 @@ void KeeperStorage::initializeSystemNodes() // update root and the digest based on it auto current_root_it = container.find("/"); - assert(current_root_it != container.end()); + chassert(current_root_it != container.end()); removeDigest(current_root_it->value, "/"); auto updated_root_it = container.updateValue( "/", @@ -385,7 +389,7 @@ void KeeperStorage::initializeSystemNodes() // insert child system nodes for (const auto & [path, data] : keeper_context->getSystemNodesWithData()) { - assert(path.starts_with(keeper_system_path)); + chassert(path.starts_with(keeper_system_path)); Node child_system_node; child_system_node.setData(data); auto [map_key, _] = container.insert(std::string{path}, child_system_node); @@ -430,7 +434,7 @@ std::shared_ptr KeeperStorage::UncommittedState::tryGetNode void KeeperStorage::UncommittedState::applyDelta(const Delta & delta) { - assert(!delta.path.empty()); + chassert(!delta.path.empty()); if (!nodes.contains(delta.path)) { if (auto storage_node = tryGetNodeFromStorage(delta.path)) @@ -446,7 +450,7 @@ void KeeperStorage::UncommittedState::applyDelta(const Delta & delta) if constexpr (std::same_as) { - assert(!node); + chassert(!node); node = std::make_shared(); node->copyStats(operation.stat); node->setData(operation.data); @@ -455,13 +459,13 @@ void KeeperStorage::UncommittedState::applyDelta(const Delta & delta) } else if constexpr (std::same_as) { - assert(node); + chassert(node); node = nullptr; last_applied_zxid = delta.zxid; } else if constexpr (std::same_as) { - assert(node); + chassert(node); node->invalidateDigestCache(); operation.update_fn(*node); last_applied_zxid = delta.zxid; @@ -499,7 +503,7 @@ void KeeperStorage::UncommittedState::addDeltas(std::vector new_deltas) void KeeperStorage::UncommittedState::commit(int64_t commit_zxid) { - assert(deltas.empty() || deltas.front().zxid >= commit_zxid); + chassert(deltas.empty() || deltas.front().zxid >= commit_zxid); // collect nodes that have no further modification in the current transaction std::unordered_set modified_nodes; @@ -517,7 +521,7 @@ void KeeperStorage::UncommittedState::commit(int64_t commit_zxid) if (!front_delta.path.empty()) { auto & path_deltas = deltas_for_path.at(front_delta.path); - assert(path_deltas.front() == &front_delta); + chassert(path_deltas.front() == &front_delta); path_deltas.pop_front(); if (path_deltas.empty()) { @@ -535,7 +539,7 @@ void KeeperStorage::UncommittedState::commit(int64_t commit_zxid) else if (auto * add_auth = std::get_if(&front_delta.operation)) { auto & uncommitted_auth = session_and_auth[add_auth->session_id]; - assert(!uncommitted_auth.empty() && uncommitted_auth.front() == &add_auth->auth_id); + chassert(!uncommitted_auth.empty() && uncommitted_auth.front() == &add_auth->auth_id); uncommitted_auth.pop_front(); if (uncommitted_auth.empty()) session_and_auth.erase(add_auth->session_id); @@ -575,7 +579,7 @@ void KeeperStorage::UncommittedState::rollback(int64_t rollback_zxid) if (delta_it->zxid < rollback_zxid) break; - assert(delta_it->zxid == rollback_zxid); + chassert(delta_it->zxid == rollback_zxid); if (!delta_it->path.empty()) { std::visit( @@ -1917,7 +1921,7 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro } } - assert(request.requests.empty() || operation_type.has_value()); + chassert(request.requests.empty() || operation_type.has_value()); } std::vector @@ -1966,7 +1970,7 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro auto & deltas = storage.uncommitted_state.deltas; // the deltas will have at least SubDeltaEnd or FailedMultiDelta - assert(!deltas.empty()); + chassert(!deltas.empty()); if (auto * failed_multi = std::get_if(&deltas.front().operation)) { for (size_t i = 0; i < concrete_requests.size(); ++i) @@ -2393,7 +2397,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest( if (is_local) { - assert(zk_request->isReadRequest()); + chassert(zk_request->isReadRequest()); if (check_acl && !request_processor->checkAuth(*this, session_id, true)) { response = zk_request->makeResponse(); diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 55d2e20d44c..c3350275d2f 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -50,7 +50,6 @@ public: int64_t mtime : 63; } is_ephemeral_and_mtime{false, 0}; - union { int64_t ephemeral_owner; @@ -89,7 +88,9 @@ public: int64_t ephemeralOwner() const { - return isEphemeral() ? ephemeral_or_children_data.ephemeral_owner : 0; + if (isEphemeral()) + return ephemeral_or_children_data.ephemeral_owner; + return 0; } void setEphemeralOwner(int64_t ephemeral_owner) @@ -100,9 +101,17 @@ public: int32_t numChildren() const { + if (isEphemeral()) + return 0; + return ephemeral_or_children_data.children_info.num_children; } + void setNumChildren(int32_t num_children) + { + ephemeral_or_children_data.children_info.num_children = num_children; + } + void increaseNumChildren() { chassert(!isEphemeral()); @@ -111,6 +120,9 @@ public: int32_t seqNum() const { + if (isEphemeral()) + return 0; + return ephemeral_or_children_data.children_info.seq_num; } @@ -121,6 +133,7 @@ public: void increaseSeqNum() { + chassert(!isEphemeral()); ++ephemeral_or_children_data.children_info.seq_num; } @@ -175,8 +188,10 @@ public: ChildrenSet children{}; }; +#if !defined(ADDRESS_SANITIZER) && !defined(MEMORY_SANITIZER) static_assert( sizeof(ListNode) <= 144, "std::list node containing ListNode is > 160 bytes which will increase memory consumption"); +#endif enum DigestVersion : uint8_t { diff --git a/src/Coordination/ZooKeeperDataReader.cpp b/src/Coordination/ZooKeeperDataReader.cpp index 2e173667e28..e027b44b3a2 100644 --- a/src/Coordination/ZooKeeperDataReader.cpp +++ b/src/Coordination/ZooKeeperDataReader.cpp @@ -123,11 +123,13 @@ int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, LoggerP int64_t ephemeral_owner; Coordination::read(ephemeral_owner, in); if (ephemeral_owner != 0) - node.setEphemeralOwner(ephemeral_owner); + node.setEphemeralOwner(ephemeral_owner); Coordination::read(node.pzxid, in); if (!path.empty()) { - node.setSeqNum(node.cversion); + if (ephemeral_owner == 0) + node.setSeqNum(node.cversion); + storage.container.insertOrReplace(path, node); if (ephemeral_owner != 0) From 328359dcd3b97ace008750e7ab40002c1c051e2a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 7 Feb 2024 18:51:19 +0100 Subject: [PATCH 141/276] Update ZooKeeperImpl.cpp --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 1fbadbd7616..dcfadff536a 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -401,6 +401,9 @@ ZooKeeper::ZooKeeper( keeper_feature_flags.logFlags(log); ProfileEvents::increment(ProfileEvents::ZooKeeperInit); + + /// Avoid stale reads after connecting + sync("/", [](const SyncResponse &){}); } catch (...) { From 9a4dbc843ac26165a876395ae61983b3dcc32ae0 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 7 Feb 2024 17:28:09 +0000 Subject: [PATCH 142/276] validate type of arguments for minmax secondary index --- .../MergeTree/MergeTreeIndexMinMax.cpp | 16 ++++++++- ..._minmax_index_aggregate_function.reference | 6 ++++ .../02985_minmax_index_aggregate_function.sql | 36 +++++++++++++++++++ 3 files changed, 57 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02985_minmax_index_aggregate_function.reference create mode 100644 tests/queries/0_stateless/02985_minmax_index_aggregate_function.sql diff --git a/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp b/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp index b1f8e09be9f..20dfed8cf8f 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp @@ -15,6 +15,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; } @@ -217,7 +218,20 @@ MergeTreeIndexPtr minmaxIndexCreator( return std::make_shared(index); } -void minmaxIndexValidator(const IndexDescription & /* index */, bool /* attach */) +void minmaxIndexValidator(const IndexDescription & index, bool attach) { + if (attach) + return; + + for (const auto & column : index.sample_block) + { + if (!column.type->isComparable()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Data type of argument for minmax index must be comparable, got {} type for column {} instead", + column.type->getName(), column.name); + } + } } + } diff --git a/tests/queries/0_stateless/02985_minmax_index_aggregate_function.reference b/tests/queries/0_stateless/02985_minmax_index_aggregate_function.reference new file mode 100644 index 00000000000..e71eb4f0d57 --- /dev/null +++ b/tests/queries/0_stateless/02985_minmax_index_aggregate_function.reference @@ -0,0 +1,6 @@ +1 +5 10 +6 11 +7 12 +8 13 +9 14 diff --git a/tests/queries/0_stateless/02985_minmax_index_aggregate_function.sql b/tests/queries/0_stateless/02985_minmax_index_aggregate_function.sql new file mode 100644 index 00000000000..7d35c1b310b --- /dev/null +++ b/tests/queries/0_stateless/02985_minmax_index_aggregate_function.sql @@ -0,0 +1,36 @@ +DROP TABLE IF EXISTS t_index_agg_func; + +CREATE TABLE t_index_agg_func +( + id UInt64, + v AggregateFunction(avg, UInt64), + INDEX idx_v v TYPE minmax GRANULARITY 1 +) +ENGINE = AggregatingMergeTree ORDER BY id +SETTINGS index_granularity = 4; -- { serverError BAD_ARGUMENTS } + +CREATE TABLE t_index_agg_func +( + id UInt64, + v AggregateFunction(avg, UInt64), +) +ENGINE = AggregatingMergeTree ORDER BY id +SETTINGS index_granularity = 4; + +ALTER TABLE t_index_agg_func ADD INDEX idx_v v TYPE minmax GRANULARITY 1; -- { serverError BAD_ARGUMENTS } + +ALTER TABLE t_index_agg_func ADD INDEX idx_v finalizeAggregation(v) TYPE minmax GRANULARITY 1; + +INSERT INTO t_index_agg_func SELECT number % 10, initializeAggregation('avgState', toUInt64(number % 20)) FROM numbers(1000); +INSERT INTO t_index_agg_func SELECT number % 10, initializeAggregation('avgState', toUInt64(number % 20)) FROM numbers(1000, 1000); + +OPTIMIZE TABLE t_index_agg_func FINAL; + +SELECT count() FROM system.parts WHERE table = 't_index_agg_func' AND database = currentDatabase() AND active; + +SET force_data_skipping_indices = 'idx_v'; +SET use_skip_indexes_if_final = 1; + +SELECT id, finalizeAggregation(v) AS vv FROM t_index_agg_func FINAL WHERE vv >= 10 ORDER BY id; + +DROP TABLE t_index_agg_func; From 7f8449f6d8868005acee00ac9aad11aecdddb692 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 7 Feb 2024 18:52:41 +0100 Subject: [PATCH 143/276] FixedString: Assert columns are of the same size --- src/Columns/ColumnFixedString.cpp | 1 + src/Columns/ColumnFixedString.h | 16 +++++++++++----- 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index 1ba59ce4f4b..7ff2716baf8 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -203,6 +203,7 @@ void ColumnFixedString::updatePermutation(IColumn::PermutationSortDirection dire void ColumnFixedString::insertRangeFrom(const IColumn & src, size_t start, size_t length) { const ColumnFixedString & src_concrete = assert_cast(src); + chassert(this->n == src_concrete.n); if (start + length > src_concrete.size()) throw Exception(ErrorCodes::PARAMETER_OUT_OF_BOUND, "Parameters start = {}, length = {} are out of bound " diff --git a/src/Columns/ColumnFixedString.h b/src/Columns/ColumnFixedString.h index 70c26ba8496..b5a9038e5de 100644 --- a/src/Columns/ColumnFixedString.h +++ b/src/Columns/ColumnFixedString.h @@ -130,15 +130,21 @@ public: int compareAt(size_t p1, size_t p2, const IColumn & rhs_, int /*nan_direction_hint*/) const override { const ColumnFixedString & rhs = assert_cast(rhs_); + chassert(this->n == rhs.n); return memcmpSmallAllowOverflow15(chars.data() + p1 * n, rhs.chars.data() + p2 * n, n); } - void compareColumn(const IColumn & rhs, size_t rhs_row_num, - PaddedPODArray * row_indexes, PaddedPODArray & compare_results, - int direction, int nan_direction_hint) const override + void compareColumn( + const IColumn & rhs_, + size_t rhs_row_num, + PaddedPODArray * row_indexes, + PaddedPODArray & compare_results, + int direction, + int nan_direction_hint) const override { - return doCompareColumn(assert_cast(rhs), rhs_row_num, row_indexes, - compare_results, direction, nan_direction_hint); + const ColumnFixedString & rhs = assert_cast(rhs_); + chassert(this->n == rhs.n); + return doCompareColumn(rhs, rhs_row_num, row_indexes, compare_results, direction, nan_direction_hint); } bool hasEqualValues() const override From 39da54cd508adb7a8d071d5ba4a0f118a8a1af23 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 7 Feb 2024 19:15:59 +0000 Subject: [PATCH 144/276] Fixing more tests. --- .../Optimizations/optimizePrewhere.cpp | 48 +++++++++++++++++-- 1 file changed, 45 insertions(+), 3 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index 6872141951e..9d997584a28 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -142,15 +142,17 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) // if (!settings.allow_experimental_analyzer) // return; - const auto & table_expression_modifiers = read_from_merge_tree->getQueryInfo().table_expression_modifiers; - bool is_final = table_expression_modifiers && table_expression_modifiers->hasFinal(); + //const auto & table_expression_modifiers = read_from_merge_tree->getQueryInfo().table_expression_modifiers; + bool is_final = read_from_merge_tree->isQueryWithFinal(); //table_expression_modifiers && table_expression_modifiers->hasFinal(); bool optimize_move_to_prewhere = settings.optimize_move_to_prewhere && (!is_final || settings.optimize_move_to_prewhere_if_final); + // std::cerr << "============ !!! << " << is_final << ' ' << settings.optimize_move_to_prewhere_if_final << std::endl; if (!optimize_move_to_prewhere) return; const auto & storage_snapshot = read_from_merge_tree->getStorageSnapshot(); - if (table_expression_modifiers && table_expression_modifiers->hasSampleSizeRatio()) + //if (table_expression_modifiers && table_expression_modifiers->hasSampleSizeRatio()) + if (read_from_merge_tree->isQueryWithSampling()) { const auto & sampling_key = storage_snapshot->getMetadataForQuery()->getSamplingKey(); const auto & sampling_source_columns = sampling_key.expression->getRequiredColumnsWithTypes(); @@ -226,6 +228,46 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) // for (const auto * input : split_result.second->getInputs()) // std::cerr << "in 2" << input->result_name << std::endl; + + /// This is the leak of abstraction. + /// Splited actions may have inputs which are needed only for PREWHERE. + /// This is fine for ActionsDAG to have such a split, but it breakes defaults calculation. + /// + /// See 00950_default_prewhere for example. + /// Table has structure `APIKey UInt8, SessionType UInt8` and default `OperatingSystem = SessionType+1` + /// For a query with `SELECT OperatingSystem WHERE APIKey = 42 AND SessionType = 42` we push everything to PREWHERE + /// and columns APIKey, SessionType are removed from inputs (cause only OperatingSystem is needed). + /// However, column OperatingSystem is calculated after PREWHERE stage, based on SessionType value. + /// If column SessionType is removed by PREWHERE actions, we use zero as defaut, and get a wrong result. + /// + /// So, here we restore removed inputs for PREWHERE actions + { + // const auto & virtuals = read_from_merge_tree->getVirtualColumnNames(); + // NameSet virtual_names(virtuals.begin(), virtuals.end()); + + //std::unordered_set first_inputs(split_result.first->getInputs().begin(), split_result.first->getInputs().end()); + std::unordered_set first_outputs(split_result.first->getOutputs().begin(), split_result.first->getOutputs().end()); + ///std::unordered_set second_inputs(split_result.second->getInputs().begin(), split_result.second->getInputs().end()); + + for (const auto * input : split_result.first->getInputs()) + { + if (!first_outputs.contains(input)) + { + split_result.first->getOutputs().push_back(input); + /// Add column to second actions as input. + /// Do not add it to result, so it would be removed. + split_result.second->addInput(input->result_name, input->result_type); + } + } + + // NameSet input_columns; + // for (const auto * input : split_result.first->getInputs()) + // input_columns.insert(input->result_name); + + // auto header = read_from_merge_tree->getStorageSnapshot()->getSampleBlockForColumns(read_from_merge_tree->getRealColumnNames()); + // header = MergeTreeSelectProcessor::transformHeader(std::move(header), prewhere_info, {}, {}); + } + ActionsDAG::NodeRawConstPtrs conditions; conditions.reserve(split_result.split_nodes_mapping.size()); for (const auto * condition : optimize_result.prewhere_nodes) From 10c09b5817cd13b34d662728c95d9098e7dc7f32 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 7 Feb 2024 20:23:52 +0100 Subject: [PATCH 145/276] Enable MSAN in mem*Overflow15 functions --- src/Common/MemorySanitizer.h | 5 ++++ src/Common/memcmpSmall.h | 48 +++++++++++++++++++++++++++++++++--- src/Common/memcpySmall.h | 4 +++ 3 files changed, 53 insertions(+), 4 deletions(-) diff --git a/src/Common/MemorySanitizer.h b/src/Common/MemorySanitizer.h index d1b8fd07f33..2c4ae3a6cfb 100644 --- a/src/Common/MemorySanitizer.h +++ b/src/Common/MemorySanitizer.h @@ -13,6 +13,9 @@ #undef __msan_unpoison_string #define __msan_unpoison(X, Y) /// NOLINT +/// Given a pointer and **its size**, unpoisons up to 15 bytes **at the end** +/// See memcmpSmall.h / memcpySmall.h +#define __msan_unpoison_overflow_15(X, Y) /// NOLINT #define __msan_test_shadow(X, Y) (false) /// NOLINT #define __msan_print_shadow(X, Y) /// NOLINT #define __msan_unpoison_string(X) /// NOLINT @@ -24,6 +27,8 @@ # undef __msan_print_shadow # undef __msan_unpoison_string # include +# undef __msan_unpoison_overflow_15 +# define __msan_unpoison_overflow_15(PTR, PTR_SIZE) __msan_unpoison(&(PTR)[(PTR_SIZE) - ((PTR_SIZE) % 16)], ((PTR_SIZE) % 16)) # endif #endif diff --git a/src/Common/memcmpSmall.h b/src/Common/memcmpSmall.h index 36d5d7efab8..0b05b4a53bd 100644 --- a/src/Common/memcmpSmall.h +++ b/src/Common/memcmpSmall.h @@ -7,6 +7,7 @@ #include #include +#include namespace detail @@ -26,9 +27,8 @@ inline int cmp(T a, T b) /// We can process uninitialized memory in the functions below. -/// Results don't depend on the values inside uninitialized memory but Memory Sanitizer cannot see it. -/// Disable optimized functions if compile with Memory Sanitizer. -#if defined(__AVX512BW__) && defined(__AVX512VL__) && !defined(MEMORY_SANITIZER) +/// Results don't depend on the values inside uninitialized memory +#if defined(__AVX512BW__) && defined(__AVX512VL__) # include @@ -42,6 +42,9 @@ inline int cmp(T a, T b) template inline int memcmpSmallAllowOverflow15(const Char * a, size_t a_size, const Char * b, size_t b_size) { + __msan_unpoison_overflow_15(a, a_size); + __msan_unpoison_overflow_15(b, b_size); + size_t min_size = std::min(a_size, b_size); for (size_t offset = 0; offset < min_size; offset += 16) @@ -74,6 +77,9 @@ inline int memcmpSmallAllowOverflow15(const Char * a, size_t a_size, const Char template inline int memcmpSmallLikeZeroPaddedAllowOverflow15(const Char * a, size_t a_size, const Char * b, size_t b_size) { + __msan_unpoison_overflow_15(a, a_size); + __msan_unpoison_overflow_15(b, b_size); + size_t min_size = std::min(a_size, b_size); for (size_t offset = 0; offset < min_size; offset += 16) @@ -144,6 +150,9 @@ inline int memcmpSmallLikeZeroPaddedAllowOverflow15(const Char * a, size_t a_siz template inline int memcmpSmallAllowOverflow15(const Char * a, const Char * b, size_t size) { + __msan_unpoison_overflow_15(a, size); + __msan_unpoison_overflow_15(b, size); + for (size_t offset = 0; offset < size; offset += 16) { uint16_t mask = _mm_cmp_epi8_mask( @@ -174,6 +183,9 @@ inline bool memequalSmallAllowOverflow15(const Char * a, size_t a_size, const Ch if (a_size != b_size) return false; + __msan_unpoison_overflow_15(a, a_size); + __msan_unpoison_overflow_15(b, b_size); + for (size_t offset = 0; offset < a_size; offset += 16) { uint16_t mask = _mm_cmp_epi8_mask( @@ -246,6 +258,7 @@ inline bool memequal16(const void * a, const void * b) /** Compare memory region to zero */ inline bool memoryIsZeroSmallAllowOverflow15(const void * data, size_t size) { + __msan_unpoison_overflow_15(reinterpret_cast(data), size); const __m128i zero16 = _mm_setzero_si128(); for (size_t offset = 0; offset < size; offset += 16) @@ -263,7 +276,7 @@ inline bool memoryIsZeroSmallAllowOverflow15(const void * data, size_t size) return true; } -#elif defined(__SSE2__) && !defined(MEMORY_SANITIZER) +#elif defined(__SSE2__) # include @@ -277,6 +290,9 @@ inline bool memoryIsZeroSmallAllowOverflow15(const void * data, size_t size) template inline int memcmpSmallAllowOverflow15(const Char * a, size_t a_size, const Char * b, size_t b_size) { + __msan_unpoison_overflow_15(a, a_size); + __msan_unpoison_overflow_15(b, b_size); + size_t min_size = std::min(a_size, b_size); for (size_t offset = 0; offset < min_size; offset += 16) @@ -309,6 +325,9 @@ inline int memcmpSmallAllowOverflow15(const Char * a, size_t a_size, const Char template inline int memcmpSmallLikeZeroPaddedAllowOverflow15(const Char * a, size_t a_size, const Char * b, size_t b_size) { + __msan_unpoison_overflow_15(a, a_size); + __msan_unpoison_overflow_15(b, b_size); + size_t min_size = std::min(a_size, b_size); for (size_t offset = 0; offset < min_size; offset += 16) @@ -380,6 +399,9 @@ inline int memcmpSmallLikeZeroPaddedAllowOverflow15(const Char * a, size_t a_siz template inline int memcmpSmallAllowOverflow15(const Char * a, const Char * b, size_t size) { + __msan_unpoison_overflow_15(a, size); + __msan_unpoison_overflow_15(b, size); + for (size_t offset = 0; offset < size; offset += 16) { uint16_t mask = _mm_movemask_epi8(_mm_cmpeq_epi8( @@ -410,6 +432,9 @@ inline bool memequalSmallAllowOverflow15(const Char * a, size_t a_size, const Ch if (a_size != b_size) return false; + __msan_unpoison_overflow_15(a, a_size); + __msan_unpoison_overflow_15(b, b_size); + for (size_t offset = 0; offset < a_size; offset += 16) { uint16_t mask = _mm_movemask_epi8(_mm_cmpeq_epi8( @@ -483,6 +508,8 @@ inline bool memequal16(const void * a, const void * b) /** Compare memory region to zero */ inline bool memoryIsZeroSmallAllowOverflow15(const void * data, size_t size) { + __msan_unpoison_overflow_15(reinterpret_cast(data), size); + const __m128i zero16 = _mm_setzero_si128(); for (size_t offset = 0; offset < size; offset += 16) @@ -509,6 +536,9 @@ inline bool memoryIsZeroSmallAllowOverflow15(const void * data, size_t size) template inline int memcmpSmallAllowOverflow15(const Char * a, size_t a_size, const Char * b, size_t b_size) { + __msan_unpoison_overflow_15(a, a_size); + __msan_unpoison_overflow_15(b, b_size); + size_t min_size = std::min(a_size, b_size); for (size_t offset = 0; offset < min_size; offset += 16) @@ -534,6 +564,9 @@ inline int memcmpSmallAllowOverflow15(const Char * a, size_t a_size, const Char template inline int memcmpSmallLikeZeroPaddedAllowOverflow15(const Char * a, size_t a_size, const Char * b, size_t b_size) { + __msan_unpoison_overflow_15(a, a_size); + __msan_unpoison_overflow_15(b, b_size); + size_t min_size = std::min(a_size, b_size); for (size_t offset = 0; offset < min_size; offset += 16) @@ -599,6 +632,9 @@ inline int memcmpSmallLikeZeroPaddedAllowOverflow15(const Char * a, size_t a_siz template inline int memcmpSmallAllowOverflow15(const Char * a, const Char * b, size_t size) { + __msan_unpoison_overflow_15(a, size); + __msan_unpoison_overflow_15(b, size); + for (size_t offset = 0; offset < size; offset += 16) { uint64_t mask = getNibbleMask(vceqq_u8( @@ -625,6 +661,9 @@ inline bool memequalSmallAllowOverflow15(const Char * a, size_t a_size, const Ch if (a_size != b_size) return false; + __msan_unpoison(&a[a_size - a_size % 16], a_size % 16); + __msan_unpoison(&b[b_size - b_size % 16], b_size % 16); + for (size_t offset = 0; offset < a_size; offset += 16) { uint64_t mask = getNibbleMask(vceqq_u8( @@ -683,6 +722,7 @@ inline bool memequal16(const void * a, const void * b) inline bool memoryIsZeroSmallAllowOverflow15(const void * data, size_t size) { + __msan_unpoison_overflow_15(reinterpret_cast(data), size); for (size_t offset = 0; offset < size; offset += 16) { uint64_t mask = getNibbleMask(vceqzq_u8(vld1q_u8(reinterpret_cast(data) + offset))); diff --git a/src/Common/memcpySmall.h b/src/Common/memcpySmall.h index 0c2aee96250..f3d26c60380 100644 --- a/src/Common/memcpySmall.h +++ b/src/Common/memcpySmall.h @@ -1,5 +1,7 @@ #pragma once +#include + #include #include /// ssize_t @@ -38,6 +40,7 @@ namespace detail { inline void memcpySmallAllowReadWriteOverflow15Impl(char * __restrict dst, const char * __restrict src, ssize_t n) { + __msan_unpoison_overflow_15(src, n); while (n > 0) { _mm_storeu_si128(reinterpret_cast<__m128i *>(dst), @@ -64,6 +67,7 @@ namespace detail { inline void memcpySmallAllowReadWriteOverflow15Impl(char * __restrict dst, const char * __restrict src, ssize_t n) { + __msan_unpoison_overflow_15(src, n); while (n > 0) { vst1q_s8(reinterpret_cast(dst), vld1q_s8(reinterpret_cast(src))); From 33401643a409e28c7bfafff6e15bcee0579dbaed Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 7 Feb 2024 20:28:37 +0100 Subject: [PATCH 146/276] Fix error --- docker/test/base/setup_export_logs.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 156adb1d1e4..6209336ab4e 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -163,9 +163,10 @@ function setup_logs_replication EXTRA_COLUMNS_EXPRESSION_FOR_TABLE="${EXTRA_COLUMNS_EXPRESSION_TRACE_LOG}" fi elif [[ "$table" = "coverage_log" ]] + then EXTRA_COLUMNS_FOR_TABLE="${EXTRA_COLUMNS_COVERAGE_LOG}" EXTRA_COLUMNS_EXPRESSION_FOR_TABLE="${EXTRA_COLUMNS_EXPRESSION_COVERAGE_LOG}" - then + else EXTRA_COLUMNS_FOR_TABLE="${EXTRA_COLUMNS}" EXTRA_COLUMNS_EXPRESSION_FOR_TABLE="${EXTRA_COLUMNS_EXPRESSION}" fi From 4d25929d74b221a29436f26cd4abdc5ba692059b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 7 Feb 2024 20:58:40 +0100 Subject: [PATCH 147/276] A few missing changes --- docker/packager/packager | 6 +++--- tests/ci/report.py | 9 +++++++++ 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/docker/packager/packager b/docker/packager/packager index 8efd3b8f302..6af5f97e6ce 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -115,8 +115,8 @@ def run_docker_image_with_env( subprocess.check_call(cmd, shell=True) -def is_release_build(debug_build: bool, package_type: str, sanitizer: str) -> bool: - return not debug_build and package_type == "deb" and sanitizer == "" +def is_release_build(debug_build: bool, package_type: str, sanitizer: str, coverage: bool) -> bool: + return not debug_build and package_type == "deb" and sanitizer == "" and not coverage def parse_env_variables( @@ -262,7 +262,7 @@ def parse_env_variables( build_target = ( f"{build_target} clickhouse-odbc-bridge clickhouse-library-bridge" ) - if is_release_build(debug_build, package_type, sanitizer): + if is_release_build(debug_build, package_type, sanitizer, coverage): cmake_flags.append("-DSPLIT_DEBUG_SYMBOLS=ON") result.append("WITH_PERFORMANCE=1") if is_cross_arm: diff --git a/tests/ci/report.py b/tests/ci/report.py index ce20c7293f9..9d43830a755 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -449,6 +449,12 @@ class BuildResult: return self._wrong_config_message return self.build_config.sanitizer + @property + def coverage(self) -> str: + if self.build_config is None: + return self._wrong_config_message + return self.build_config.coverage + @property def grouped_urls(self) -> List[List[str]]: "Combine and preserve build_urls by artifact types" @@ -775,6 +781,7 @@ HTML_BASE_BUILD_TEMPLATE = ( Build type Version Sanitizer +Coverage Status Build log Build time @@ -816,6 +823,8 @@ def create_build_html_report( else: row.append("none") + row.append(f"{build_result.coverage}") + if build_result.status: style = _get_status_style(build_result.status) row.append(f'{build_result.status}') From 1c71a27527262db3034f067aef0e159d85306c6b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 7 Feb 2024 20:05:39 +0000 Subject: [PATCH 148/276] Remove some comments --- src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index 9d997584a28..eb5f7a42819 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -142,16 +142,13 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) // if (!settings.allow_experimental_analyzer) // return; - //const auto & table_expression_modifiers = read_from_merge_tree->getQueryInfo().table_expression_modifiers; - bool is_final = read_from_merge_tree->isQueryWithFinal(); //table_expression_modifiers && table_expression_modifiers->hasFinal(); + bool is_final = read_from_merge_tree->isQueryWithFinal(); bool optimize_move_to_prewhere = settings.optimize_move_to_prewhere && (!is_final || settings.optimize_move_to_prewhere_if_final); - // std::cerr << "============ !!! << " << is_final << ' ' << settings.optimize_move_to_prewhere_if_final << std::endl; if (!optimize_move_to_prewhere) return; const auto & storage_snapshot = read_from_merge_tree->getStorageSnapshot(); - //if (table_expression_modifiers && table_expression_modifiers->hasSampleSizeRatio()) if (read_from_merge_tree->isQueryWithSampling()) { const auto & sampling_key = storage_snapshot->getMetadataForQuery()->getSamplingKey(); From 64d9d2d467dd187647c88fde23e21c3af92d8439 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 7 Feb 2024 20:44:35 +0000 Subject: [PATCH 149/276] Automatic style fix --- docker/packager/packager | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/docker/packager/packager b/docker/packager/packager index 6af5f97e6ce..ca0ae8358f3 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -115,8 +115,12 @@ def run_docker_image_with_env( subprocess.check_call(cmd, shell=True) -def is_release_build(debug_build: bool, package_type: str, sanitizer: str, coverage: bool) -> bool: - return not debug_build and package_type == "deb" and sanitizer == "" and not coverage +def is_release_build( + debug_build: bool, package_type: str, sanitizer: str, coverage: bool +) -> bool: + return ( + not debug_build and package_type == "deb" and sanitizer == "" and not coverage + ) def parse_env_variables( From 62721fef781cfde424701b2148c9595d2705f088 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 7 Feb 2024 22:45:35 +0100 Subject: [PATCH 150/276] Fix your Py --- tests/ci/report.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/report.py b/tests/ci/report.py index 9d43830a755..8b83566eb57 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -450,7 +450,7 @@ class BuildResult: return self.build_config.sanitizer @property - def coverage(self) -> str: + def coverage(self) -> bool: if self.build_config is None: return self._wrong_config_message return self.build_config.coverage From 85bd8d3cf0fe71963db819b2bbb2b93728353f87 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 8 Feb 2024 00:23:04 +0100 Subject: [PATCH 151/276] Fix your Py --- tests/ci/report.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/report.py b/tests/ci/report.py index 8b83566eb57..de94c23bc2c 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -450,10 +450,10 @@ class BuildResult: return self.build_config.sanitizer @property - def coverage(self) -> bool: + def coverage(self) -> str: if self.build_config is None: return self._wrong_config_message - return self.build_config.coverage + return str(self.build_config.coverage) @property def grouped_urls(self) -> List[List[str]]: From 560e78cf418ada54d23e34f25b88e8b93bcf3411 Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Wed, 7 Feb 2024 21:08:30 -0800 Subject: [PATCH 152/276] [Docs] More details for Memory table engine in Cloud --- docs/en/engines/table-engines/special/memory.md | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/special/memory.md b/docs/en/engines/table-engines/special/memory.md index a30f620f5c5..54547b1bc69 100644 --- a/docs/en/engines/table-engines/special/memory.md +++ b/docs/en/engines/table-engines/special/memory.md @@ -7,7 +7,10 @@ sidebar_label: Memory # Memory Table Engine :::note -When using the Memory table engine on ClickHouse Cloud, you must use a client that uses TCP (such as [clickhouse-client](/en/interfaces/cli)) or a native interface, and not one that uses HTTP (such as [clickhouse-connect](/en/integrations/python)). If you use HTTP, all queries must be submitted and executed at once using a multi-statement query. +When using the Memory table engine on ClickHouse Cloud, data is not replicated across all nodes (by design). To guarantee that all queries are routed to the same node and that the Memory table engine works as expected, you can do one of the following: +- Execute all operations in the same session +- Use a client that uses TCP or the native interface (which enables support for sticky connections) such as [clickhouse-client](/en/interfaces/cli) +- Submit and execute all queries at once using a multi-statement query (required with clients using the HTTP interface such as [clickhouse-connect](/en/integrations/python)) ::: The Memory engine stores data in RAM, in uncompressed form. Data is stored in exactly the same form as it is received when read. In other words, reading from this table is completely free. From c2019b3b1eb3b09803ecbbce8bd9bfe0560faf36 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 8 Feb 2024 09:08:44 +0000 Subject: [PATCH 153/276] Fix test --- tests/queries/0_stateless/02972_parallel_replicas_cte.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02972_parallel_replicas_cte.sql b/tests/queries/0_stateless/02972_parallel_replicas_cte.sql index c39ad172a27..3702184e336 100644 --- a/tests/queries/0_stateless/02972_parallel_replicas_cte.sql +++ b/tests/queries/0_stateless/02972_parallel_replicas_cte.sql @@ -12,16 +12,16 @@ SELECT count() FROM pr_2 INNER JOIN filtered_groups ON pr_2.a = filtered_groups. WITH filtered_groups AS (SELECT a FROM pr_1 WHERE a >= 10000) SELECT count() FROM pr_2 INNER JOIN filtered_groups ON pr_2.a = filtered_groups.a -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_two_shards', max_parallel_replicas = 3; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3; -- Testing that it is disabled for allow_experimental_analyzer=0. With analyzer it will be supported (with correct result) WITH filtered_groups AS (SELECT a FROM pr_1 WHERE a >= 10000) SELECT count() FROM pr_2 INNER JOIN filtered_groups ON pr_2.a = filtered_groups.a -SETTINGS allow_experimental_analyzer = 0, allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_two_shards', max_parallel_replicas = 3; -- { serverError SUPPORT_IS_DISABLED } +SETTINGS allow_experimental_analyzer = 0, allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3; -- { serverError SUPPORT_IS_DISABLED } -- Sanitizer SELECT count() FROM pr_2 JOIN numbers(10) as pr_1 ON pr_2.a = pr_1.number -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_two_shards', max_parallel_replicas = 3; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3; DROP TABLE IF EXISTS pr_1; DROP TABLE IF EXISTS pr_2; From 3167bfe423d48a99aed758c7b032d360d682b8f0 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 8 Feb 2024 11:35:23 +0100 Subject: [PATCH 154/276] Bump server version --- tests/analyzer_integration_broken_tests.txt | 1 - tests/integration/test_replicating_constants/test.py | 3 +-- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/analyzer_integration_broken_tests.txt b/tests/analyzer_integration_broken_tests.txt index c04ed440c18..89580771bc9 100644 --- a/tests/analyzer_integration_broken_tests.txt +++ b/tests/analyzer_integration_broken_tests.txt @@ -10,7 +10,6 @@ test_merge_table_over_distributed/test.py::test_select_table_name_from_merge_ove test_mutations_with_merge_tree/test.py::test_mutations_with_merge_background_task test_mysql_database_engine/test.py::test_mysql_ddl_for_mysql_database test_passing_max_partitions_to_read_remotely/test.py::test_default_database_on_cluster -test_replicating_constants/test.py::test_different_versions test_select_access_rights/test_main.py::test_alias_columns test_settings_profile/test.py::test_show_profiles test_shard_level_const_function/test.py::test_remote diff --git a/tests/integration/test_replicating_constants/test.py b/tests/integration/test_replicating_constants/test.py index 9669e890cd3..fbf7450577f 100644 --- a/tests/integration/test_replicating_constants/test.py +++ b/tests/integration/test_replicating_constants/test.py @@ -9,9 +9,8 @@ node2 = cluster.add_instance( "node2", with_zookeeper=True, image="yandex/clickhouse-server", - tag="19.16.9.37", + tag="22.3", with_installed_binary=True, - allow_analyzer=False, ) From fafd8005a05412db754a6ea595472ba59fda6f29 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 8 Feb 2024 13:51:29 +0000 Subject: [PATCH 155/276] Fixing style. --- src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index eb5f7a42819..2f790d9892f 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -228,14 +228,14 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) /// This is the leak of abstraction. /// Splited actions may have inputs which are needed only for PREWHERE. - /// This is fine for ActionsDAG to have such a split, but it breakes defaults calculation. + /// This is fine for ActionsDAG to have such a split, but it breaks defaults calculation. /// /// See 00950_default_prewhere for example. /// Table has structure `APIKey UInt8, SessionType UInt8` and default `OperatingSystem = SessionType+1` /// For a query with `SELECT OperatingSystem WHERE APIKey = 42 AND SessionType = 42` we push everything to PREWHERE /// and columns APIKey, SessionType are removed from inputs (cause only OperatingSystem is needed). /// However, column OperatingSystem is calculated after PREWHERE stage, based on SessionType value. - /// If column SessionType is removed by PREWHERE actions, we use zero as defaut, and get a wrong result. + /// If column SessionType is removed by PREWHERE actions, we use zero as default, and get a wrong result. /// /// So, here we restore removed inputs for PREWHERE actions { From 73d2ff3933ca60133c33ed61af277d616e10bf9c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 8 Feb 2024 14:55:35 +0100 Subject: [PATCH 156/276] Update MergeTask.cpp --- src/Storages/MergeTree/MergeTask.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index f7cd721b8dd..9cbcdbaaaaa 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -1066,7 +1066,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() #ifndef NDEBUG if (!sort_description.empty()) { - res_pipe.addSimpleTransform([&](const Block & header_) + builder->addSimpleTransform([&](const Block & header_) { auto transform = std::make_shared(header_, sort_description); return transform; From fa0acaf39ba3c9d78b80c6af6f22663ff78d79ff Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 8 Feb 2024 15:18:17 +0100 Subject: [PATCH 157/276] Fix image name --- tests/integration/test_replicating_constants/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_replicating_constants/test.py b/tests/integration/test_replicating_constants/test.py index fbf7450577f..90132b71a64 100644 --- a/tests/integration/test_replicating_constants/test.py +++ b/tests/integration/test_replicating_constants/test.py @@ -8,7 +8,7 @@ node1 = cluster.add_instance("node1", with_zookeeper=True) node2 = cluster.add_instance( "node2", with_zookeeper=True, - image="yandex/clickhouse-server", + image="clickhouse/clickhouse-server", tag="22.3", with_installed_binary=True, ) From 97d1eb109190b0bff8fdf3c61cf44d5c1fbafd7d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 8 Feb 2024 14:31:24 +0000 Subject: [PATCH 158/276] Fixing test. --- tests/integration/test_recompression_ttl/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_recompression_ttl/test.py b/tests/integration/test_recompression_ttl/test.py index 851e3bb4eb8..9d7b09eacdf 100644 --- a/tests/integration/test_recompression_ttl/test.py +++ b/tests/integration/test_recompression_ttl/test.py @@ -155,7 +155,7 @@ def test_recompression_multiple_ttls(started_cluster): node2.query( "SELECT recompression_ttl_info.expression FROM system.parts where name = 'all_1_1_4'" ) - == "['plus(d, toIntervalSecond(10))','plus(d, toIntervalSecond(15))','plus(d, toIntervalSecond(5))']\n" + == "['d + toIntervalSecond(10)','d + toIntervalSecond(15)','d + toIntervalSecond(5)']\n" ) From 5f750871786772e1135084c34b5a20aa8c108c74 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 8 Feb 2024 15:48:20 +0000 Subject: [PATCH 159/276] Add comments --- src/Storages/StorageMerge.cpp | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index d3b8f30b1c5..09c38996b22 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -756,16 +756,23 @@ QueryTreeNodePtr replaceTableExpressionAndRemoveJoin( auto join_tree_type = query_node->getJoinTree()->getNodeType(); auto modified_query = query_node->cloneAndReplace(original_table_expression, replacement_table_expression); + // For the case when join tree is just a table or a table function we don't need to do anything more. if (join_tree_type == QueryTreeNodeType::TABLE || join_tree_type == QueryTreeNodeType::TABLE_FUNCTION) return modified_query; + // JOIN needs to be removed because StorageMerge should produce not joined data. + // GROUP BY should be removed as well. + auto * modified_query_node = modified_query->as(); + // Remove the JOIN statement. As a result query will have a form like: SELECT * FROM ... modified_query = modified_query->cloneAndReplace(modified_query_node->getJoinTree(), replacement_table_expression); modified_query_node = modified_query->as(); query_node = modified_query->as(); + // For backward compatibility we need to leave all filters related to this table. + // It may lead to some incorrect result. if (query_node->hasPrewhere()) replaceFilterExpression(query_node->getPrewhere(), replacement_table_expression, context); if (query_node->hasWhere()) @@ -779,6 +786,9 @@ QueryTreeNodePtr replaceTableExpressionAndRemoveJoin( projection.clear(); NamesAndTypes projection_columns; + // Select only required columns from the table, because prjection list may contain: + // 1. aggregate functions + // 2. expressions referencing other tables of JOIN for (auto const & column_name : required_column_names) { QueryTreeNodePtr fake_node = std::make_shared(Identifier{column_name}); @@ -791,6 +801,8 @@ QueryTreeNodePtr replaceTableExpressionAndRemoveJoin( throw Exception(ErrorCodes::LOGICAL_ERROR, "Required column '{}' is not resolved", column_name); auto fake_column = resolved_column->getColumn(); + // Identifier is resolved to ColumnNode, but we need to get rid of ALIAS columns + // and also fix references to source expression (now column is referencing original table expression). ApplyAliasColumnExpressionsVisitor visitor(replacement_table_expression); visitor.visit(fake_node); @@ -860,7 +872,7 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const ContextPtr & modified_ QueryTreeNodePtr column_node; - + // Replace all references to ALIAS columns in the query by expressions. if (is_alias) { QueryTreeNodePtr fake_node = std::make_shared(Identifier{column}); From dfb3a8f7be0c8f7aebb52d9cd3f7ebabd7f83af5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 8 Feb 2024 13:52:46 +0100 Subject: [PATCH 160/276] Another fix --- src/Coordination/KeeperSnapshotManager.cpp | 5 ++--- src/Coordination/KeeperStorage.cpp | 21 ++++++++++----------- src/Coordination/KeeperStorage.h | 9 ++++++++- utils/keeper-bench/Generator.cpp | 1 - 4 files changed, 20 insertions(+), 16 deletions(-) diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 07116fedfab..d7c9acae07a 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -89,11 +89,10 @@ namespace writeBinary(node.ephemeralOwner(), out); if (version < SnapshotVersion::V6) writeBinary(static_cast(node.data_size), out); - const bool is_ephemeral = node.isEphemeral(); - writeBinary(is_ephemeral ? 0 : node.numChildren(), out); + writeBinary(node.numChildren(), out); writeBinary(node.pzxid, out); - writeBinary(is_ephemeral ? 0 : node.seqNum(), out); + writeBinary(node.seqNum(), out); if (version >= SnapshotVersion::V4 && version <= SnapshotVersion::V5) writeBinary(node.sizeInBytes(), out); diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index e855274f08b..9d4c22164d3 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -185,9 +185,8 @@ uint64_t calculateDigest(std::string_view path, const KeeperStorage::Node & node hash.update(node.version); hash.update(node.cversion); hash.update(node.aversion); - bool is_ephemeral = node.isEphemeral(); - hash.update(is_ephemeral ? node.ephemeralOwner() : 0); - hash.update(is_ephemeral ? 0 : node.numChildren()); + hash.update(node.ephemeralOwner()); + hash.update(node.numChildren()); hash.update(node.pzxid); return hash.get64(); @@ -223,6 +222,9 @@ KeeperStorage::Node & KeeperStorage::Node::operator=(const Node & other) data = new char[data_size]; memcpy(data, other.data, data_size); } + + children = other.children; + return *this; } @@ -252,7 +254,7 @@ void KeeperStorage::Node::copyStats(const Coordination::Stat & stat) if (stat.ephemeralOwner == 0) { is_ephemeral_and_mtime.is_ephemeral = false; - ephemeral_or_children_data.children_info.num_children = stat.numChildren; + setNumChildren(stat.numChildren); } else { @@ -269,10 +271,9 @@ void KeeperStorage::Node::setResponseStat(Coordination::Stat & response_stat) co response_stat.version = version; response_stat.cversion = cversion; response_stat.aversion = aversion; - bool is_ephemeral = isEphemeral(); - response_stat.ephemeralOwner = is_ephemeral ? ephemeral_or_children_data.ephemeral_owner : 0; + response_stat.ephemeralOwner = ephemeralOwner(); response_stat.dataLength = static_cast(data_size); - response_stat.numChildren = is_ephemeral ? 0 : numChildren(); + response_stat.numChildren = numChildren(); response_stat.pzxid = pzxid; } @@ -1316,7 +1317,7 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr KeeperStorage::UpdateNodeDelta{[](KeeperStorage::Node & parent) { ++parent.cversion; - --parent.ephemeral_or_children_data.children_info.num_children; + parent.decreaseNumChildren(); }}); new_deltas.emplace_back(request.path, zxid, KeeperStorage::RemoveNodeDelta{request.version, node->ephemeralOwner()}); @@ -1561,9 +1562,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc auto list_request_type = ALL; if (auto * filtered_list = dynamic_cast(&request)) - { list_request_type = filtered_list->list_request_type; - } if (list_request_type == ALL) return true; @@ -2294,7 +2293,7 @@ void KeeperStorage::preprocessRequest( [ephemeral_path](Node & parent) { ++parent.cversion; - --parent.ephemeral_or_children_data.children_info.num_children; + parent.decreaseNumChildren(); } } ); diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index c3350275d2f..cf9c1710d3d 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -90,12 +90,13 @@ public: { if (isEphemeral()) return ephemeral_or_children_data.ephemeral_owner; + return 0; } void setEphemeralOwner(int64_t ephemeral_owner) { - is_ephemeral_and_mtime.is_ephemeral = true; + is_ephemeral_and_mtime.is_ephemeral = ephemeral_owner != 0; ephemeral_or_children_data.ephemeral_owner = ephemeral_owner; } @@ -118,6 +119,12 @@ public: ++ephemeral_or_children_data.children_info.num_children; } + void decreaseNumChildren() + { + chassert(!isEphemeral()); + --ephemeral_or_children_data.children_info.num_children; + } + int32_t seqNum() const { if (isEphemeral()) diff --git a/utils/keeper-bench/Generator.cpp b/utils/keeper-bench/Generator.cpp index a3e85f670d6..2212f7158ae 100644 --- a/utils/keeper-bench/Generator.cpp +++ b/utils/keeper-bench/Generator.cpp @@ -455,7 +455,6 @@ Coordination::ZooKeeperRequestPtr CreateRequestGenerator::generateImpl(const Coo auto request = std::make_shared(); request->acls = acls; - request->is_sequential = true; std::string path_candidate = std::filesystem::path(parent_path.getPath()) / name.getString(); From d97c5496f54ea157796aae309bbe980db831c8d9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 8 Feb 2024 18:37:07 +0100 Subject: [PATCH 161/276] Fix unpoison --- src/Common/MemorySanitizer.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/MemorySanitizer.h b/src/Common/MemorySanitizer.h index 2c4ae3a6cfb..cc6b0d5171f 100644 --- a/src/Common/MemorySanitizer.h +++ b/src/Common/MemorySanitizer.h @@ -28,7 +28,7 @@ # undef __msan_unpoison_string # include # undef __msan_unpoison_overflow_15 -# define __msan_unpoison_overflow_15(PTR, PTR_SIZE) __msan_unpoison(&(PTR)[(PTR_SIZE) - ((PTR_SIZE) % 16)], ((PTR_SIZE) % 16)) +# define __msan_unpoison_overflow_15(PTR, PTR_SIZE) __msan_unpoison(&(PTR)[(PTR_SIZE) - ((PTR_SIZE) % 16)], 16) # endif #endif From 16e2b275c26eb0916b92b98f9500745ff2e72934 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 8 Feb 2024 19:08:46 +0100 Subject: [PATCH 162/276] Fix unpoison --- src/Common/MemorySanitizer.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/MemorySanitizer.h b/src/Common/MemorySanitizer.h index cc6b0d5171f..02747d1e130 100644 --- a/src/Common/MemorySanitizer.h +++ b/src/Common/MemorySanitizer.h @@ -28,7 +28,7 @@ # undef __msan_unpoison_string # include # undef __msan_unpoison_overflow_15 -# define __msan_unpoison_overflow_15(PTR, PTR_SIZE) __msan_unpoison(&(PTR)[(PTR_SIZE) - ((PTR_SIZE) % 16)], 16) +# define __msan_unpoison_overflow_15(PTR, PTR_SIZE) __msan_unpoison(&(PTR)[(PTR_SIZE)], ((16 - ((PTR_SIZE) % 16)) % 16)) # endif #endif From a2beae80f5077e0a477df9a136f8f32224e67246 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 8 Feb 2024 20:57:33 +0100 Subject: [PATCH 163/276] Improve test --- .../02985_shard_query_start_time.reference | 4 ++-- .../02985_shard_query_start_time.sql | 21 ++++++++++++------- 2 files changed, 15 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/02985_shard_query_start_time.reference b/tests/queries/0_stateless/02985_shard_query_start_time.reference index 1957f3a9604..ff563ea1d53 100644 --- a/tests/queries/0_stateless/02985_shard_query_start_time.reference +++ b/tests/queries/0_stateless/02985_shard_query_start_time.reference @@ -1,2 +1,2 @@ -1 1 -1 1 +QueryStart 2 2 2 2 +QueryFinish 2 2 2 2 diff --git a/tests/queries/0_stateless/02985_shard_query_start_time.sql b/tests/queries/0_stateless/02985_shard_query_start_time.sql index b0d8d2b6e53..c31d81e58ae 100644 --- a/tests/queries/0_stateless/02985_shard_query_start_time.sql +++ b/tests/queries/0_stateless/02985_shard_query_start_time.sql @@ -1,29 +1,34 @@ DROP TABLE IF EXISTS sharded_table; CREATE TABLE sharded_table (dummy UInt8) ENGINE = Distributed('test_cluster_two_shards', 'system', 'one'); +SET prefer_localhost_replica=0; SELECT * FROM sharded_table FORMAT Null SETTINGS log_comment='02985_shard_query_start_time_query_1'; SYSTEM FLUSH LOGS; --- We do not test for query_start_time because that would conflict pretty easily +-- Check that there are 2 queries to shards and for each one query_start_time_microseconds is more recent +-- than initial_query_start_time_microseconds, and initial_query_start_time_microseconds matches the original query +-- query_start_time_microseconds WITH ( SELECT - (query_id, query_start_time_microseconds) + (query_id, query_start_time, query_start_time_microseconds) FROM system.query_log WHERE - event_date >= yesterday() + event_date >= yesterday() AND current_database = currentDatabase() AND log_comment = '02985_shard_query_start_time_query_1' AND type = 'QueryFinish' - ORDER BY query_start_time_microseconds DESC - LIMIT 1 ) AS id_and_start_tuple SELECT - query_start_time_microseconds > initial_query_start_time_microseconds, - initial_query_start_time_microseconds = id_and_start_tuple.2 + type, + countIf(query_start_time >= initial_query_start_time), -- Using >= because it's comparing seconds + countIf(query_start_time_microseconds > initial_query_start_time_microseconds), + countIf(initial_query_start_time = id_and_start_tuple.2), + countIf(initial_query_start_time_microseconds = id_and_start_tuple.3) FROM system.query_log WHERE - NOT is_initial_query AND initial_query_id = id_and_start_tuple.1; + NOT is_initial_query AND initial_query_id = id_and_start_tuple.1 +GROUP BY type; From 075da5602fdc03d4d5b15cd8d769704259b168a9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 8 Feb 2024 21:10:49 +0100 Subject: [PATCH 164/276] Remove incorrect test --- ...nt_info_initial_query_start_time.reference | 8 --- ...de_client_info_initial_query_start_time.sh | 67 ------------------- 2 files changed, 75 deletions(-) delete mode 100644 tests/queries/0_stateless/02590_interserver_mode_client_info_initial_query_start_time.reference delete mode 100755 tests/queries/0_stateless/02590_interserver_mode_client_info_initial_query_start_time.sh diff --git a/tests/queries/0_stateless/02590_interserver_mode_client_info_initial_query_start_time.reference b/tests/queries/0_stateless/02590_interserver_mode_client_info_initial_query_start_time.reference deleted file mode 100644 index fbce8ae2026..00000000000 --- a/tests/queries/0_stateless/02590_interserver_mode_client_info_initial_query_start_time.reference +++ /dev/null @@ -1,8 +0,0 @@ -SELECT -3 0 0 -3 0 0 -INSERT -CHECK -1 -2 -6 0 2 diff --git a/tests/queries/0_stateless/02590_interserver_mode_client_info_initial_query_start_time.sh b/tests/queries/0_stateless/02590_interserver_mode_client_info_initial_query_start_time.sh deleted file mode 100755 index 5da643bd17b..00000000000 --- a/tests/queries/0_stateless/02590_interserver_mode_client_info_initial_query_start_time.sh +++ /dev/null @@ -1,67 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-fasttest -# Tag no-fasttest: interserver mode requires SSL -# -# Test that checks that some of ClientInfo correctly passed in inter-server mode. -# NOTE: we need .sh test (.sql is not enough) because queries on remote nodes does not have current_database = currentDatabase() -# -# Check-style suppression: select * from system.query_log where current_database = currentDatabase(); - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CUR_DIR"/../shell_config.sh - -function get_query_id() { random_str 10; } - -$CLICKHOUSE_CLIENT -nm -q " - drop table if exists buf; - drop table if exists dist; - drop table if exists data; - - create table data (key Int) engine=Memory(); - create table dist as data engine=Distributed(test_cluster_interserver_secret, currentDatabase(), data, key); - create table dist_dist as data engine=Distributed(test_cluster_interserver_secret, currentDatabase(), dist, key); - system stop distributed sends dist; -" - -echo "SELECT" -query_id="$(get_query_id)" -# initialize connection, but actually if there are other tables that uses this -# cluster then, it will be created long time ago, but this is OK for this -# test, since we care about the difference between NOW() and there should -# not be any significant difference. -$CLICKHOUSE_CLIENT --prefer_localhost_replica=0 --query_id "$query_id" -q "select * from dist" -$CLICKHOUSE_CLIENT -nm --param_query_id "$query_id" -q " - system flush logs; - select count(), countIf(initial_query_start_time_microseconds != query_start_time_microseconds), countIf(event_time - initial_query_start_time > 3) from system.query_log where type = 'QueryFinish' and initial_query_id = {query_id:String}; -" - -sleep 6 - -query_id="$(get_query_id)" -# this query (and all subsequent) should reuse the previous connection (at least most of the time) -$CLICKHOUSE_CLIENT --prefer_localhost_replica=0 --query_id "$query_id" -q "select * from dist" - -$CLICKHOUSE_CLIENT -nm --param_query_id "$query_id" -q " - system flush logs; - select count(), countIf(initial_query_start_time_microseconds != query_start_time_microseconds), countIf(event_time - initial_query_start_time > 3) from system.query_log where type = 'QueryFinish' and initial_query_id = {query_id:String}; -" - -echo "INSERT" -query_id="$(get_query_id)" -$CLICKHOUSE_CLIENT --prefer_localhost_replica=0 --query_id "$query_id" -nm -q " - insert into dist_dist values (1),(2); - select * from data; -" - -sleep 3 -$CLICKHOUSE_CLIENT -nm --param_query_id "$query_id" -q "system flush distributed dist_dist" -sleep 1 -$CLICKHOUSE_CLIENT -nm --param_query_id "$query_id" -q "system flush distributed dist" - -echo "CHECK" -$CLICKHOUSE_CLIENT -nm --param_query_id "$query_id" -q " - select * from data order by key; - system flush logs; - select count(), countIf(initial_query_start_time_microseconds != query_start_time_microseconds), countIf(event_time - initial_query_start_time > 3) from system.query_log where type = 'QueryFinish' and initial_query_id = {query_id:String}; -" From 91c607f249fe6320c5c6a3bbffd61a6dcc44ff49 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 8 Feb 2024 22:11:37 +0100 Subject: [PATCH 165/276] Fix a test --- docker/test/base/setup_export_logs.sh | 8 ++++---- src/Dictionaries/CacheDictionary.cpp | 2 -- .../0_stateless/02982_comments_in_system_tables.sh | 4 ++-- 3 files changed, 6 insertions(+), 8 deletions(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 6209336ab4e..917701b8d35 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -140,10 +140,10 @@ function setup_logs_replication clickhouse-client --query " CREATE TABLE IF NOT EXISTS system.coverage_log ( - time DateTime, - test_name String, - coverage Array(UInt64) - ) ENGINE = MergeTree ORDER BY test_name + time DateTime COMMENT 'The time of test run', + test_name String COMMENT 'The name of the test', + coverage Array(UInt64) COMMENT 'An array of addresses of the code (a subset of addresses instrumented for coverage) that were encountered during the test run' + ) ENGINE = MergeTree ORDER BY test_name COMMENT 'Contains information about per-test coverage from the CI' " # For each system log table: diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index 000f0ef5b4c..bf3d5a5cd12 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -3,9 +3,7 @@ #include #include -#include #include -#include #include #include #include diff --git a/tests/queries/0_stateless/02982_comments_in_system_tables.sh b/tests/queries/0_stateless/02982_comments_in_system_tables.sh index cc04592bc27..0825b1be2d6 100755 --- a/tests/queries/0_stateless/02982_comments_in_system_tables.sh +++ b/tests/queries/0_stateless/02982_comments_in_system_tables.sh @@ -4,5 +4,5 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_LOCAL} --query "SELECT 'Table ' || database || '.' || name || ' doesnt have a comment' FROM system.tables WHERE name NOT LIKE '%\_log\_%' AND database='system' AND comment==''" -${CLICKHOUSE_CLIENT} --query "SELECT 'Table ' || database || '.' || name || ' doesnt have a comment' FROM system.tables WHERE name NOT LIKE '%\_log\_%' AND database='system' AND comment==''" +${CLICKHOUSE_LOCAL} --query "SELECT 'Table ' || database || '.' || name || ' doesn\'t have a comment' FROM system.tables WHERE name NOT LIKE '%\_log\_%' AND database='system' AND comment==''" +${CLICKHOUSE_CLIENT} --query "SELECT 'Table ' || database || '.' || name || ' doesn\'t have a comment' FROM system.tables WHERE name NOT LIKE '%\_log\_%' AND database='system' AND comment==''" From dbc288cd51affc3f6758a66ac51130aa8b5b565f Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Thu, 8 Feb 2024 23:13:44 +0100 Subject: [PATCH 166/276] Update ne-tormozit.md Fix broken youtube video embedding --- docs/ru/faq/general/ne-tormozit.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/faq/general/ne-tormozit.md b/docs/ru/faq/general/ne-tormozit.md index 0f888de839f..6d0803680a8 100644 --- a/docs/ru/faq/general/ne-tormozit.md +++ b/docs/ru/faq/general/ne-tormozit.md @@ -20,6 +20,6 @@ sidebar_position: 11 Если вы не видели наших футболок, посмотрите видео о ClickHouse. Например, вот это: -![iframe](https://www.youtube.com/embed/bSyQahMVZ7w) + P.S. Эти футболки не продаются, а распространяются бесплатно на большинстве митапов [ClickHouse](https://clickhouse.com/#meet), обычно в награду за самые интересные вопросы или другие виды активного участия. From c12e34df199c0c30764dfa865cc30da427635aca Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 8 Feb 2024 23:28:29 +0100 Subject: [PATCH 167/276] Fix report --- .github/workflows/master.yml | 1 + .github/workflows/pull_request.yml | 1 + tests/ci/ci_config.py | 9 +++------ 3 files changed, 5 insertions(+), 6 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 653b46300ac..1bebe80ca7e 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -285,6 +285,7 @@ jobs: - BuilderDebDebug - BuilderDebMsan - BuilderDebRelease + - BuilderDebReleaseCoverage - BuilderDebTsan - BuilderDebUBsan uses: ./.github/workflows/reusable_test.yml diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 6a5bb851447..405e1ec1502 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -317,6 +317,7 @@ jobs: - BuilderDebDebug - BuilderDebMsan - BuilderDebRelease + - BuilderDebReleaseCoverage - BuilderDebTsan - BuilderDebUBsan uses: ./.github/workflows/reusable_test.yml diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index b40584ff1a8..817018f044c 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -204,10 +204,9 @@ class JobConfig: # About the "sparse_checkout" option: # -# Misha f. Shiryaev -# :facepalm: -# we have this feature, it's used by devs, we need to test it in CI -# It's not useful for the CI itself +# It's needed only to test the option itself. +# No particular sense to use it in the build, +# and it is even slows down the job. @dataclass @@ -730,7 +729,6 @@ CI_CONFIG = CIConfig( compiler="clang-17", coverage=True, package_type="deb", - sparse_checkout=True, ), Build.BINARY_RELEASE: BuildConfig( name=Build.BINARY_RELEASE, @@ -751,7 +749,6 @@ CI_CONFIG = CIConfig( compiler="clang-17-darwin", package_type="binary", static_binary_name="macos", - sparse_checkout=True, # Check that it works with at least one build, see also update-submodules.sh ), Build.BINARY_AARCH64: BuildConfig( name=Build.BINARY_AARCH64, From 41ff8b3217595a6d24eaaf973e73b04f09242ae6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Feb 2024 00:15:52 +0100 Subject: [PATCH 168/276] Reading Docker files --- docker/test/stateless/Dockerfile | 1 - docker/test/util/Dockerfile | 8 ++------ 2 files changed, 2 insertions(+), 7 deletions(-) diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index f09ba46de1e..7f4bad3d4e6 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -74,7 +74,6 @@ RUN arch=${TARGETARCH:-amd64} \ && wget "https://dl.min.io/client/mc/release/linux-${arch}/archive/mc.RELEASE.${MINIO_CLIENT_VERSION}" -O ./mc \ && chmod +x ./mc ./minio - RUN wget --no-verbose 'https://archive.apache.org/dist/hadoop/common/hadoop-3.3.1/hadoop-3.3.1.tar.gz' \ && tar -xvf hadoop-3.3.1.tar.gz \ && rm -rf hadoop-3.3.1.tar.gz diff --git a/docker/test/util/Dockerfile b/docker/test/util/Dockerfile index 054eac5f764..396d5801be9 100644 --- a/docker/test/util/Dockerfile +++ b/docker/test/util/Dockerfile @@ -5,7 +5,6 @@ FROM ubuntu:22.04 ARG apt_archive="http://archive.ubuntu.com" RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list -# 15.0.2 ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=17 RUN apt-get update \ @@ -30,8 +29,7 @@ RUN apt-get update \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* - -# Install cmake 3.20+ for rust support +# Install cmake 3.20+ for Rust support # Used https://askubuntu.com/a/1157132 as reference RUN curl -s https://apt.kitware.com/keys/kitware-archive-latest.asc | \ gpg --dearmor - > /etc/apt/trusted.gpg.d/kitware.gpg && \ @@ -65,8 +63,7 @@ RUN apt-get update \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* - -# This symlink required by gcc to find lld compiler +# This symlink is required by gcc to find the lld linker RUN ln -s /usr/bin/lld-${LLVM_VERSION} /usr/bin/ld.lld # for external_symbolizer_path RUN ln -s /usr/bin/llvm-symbolizer-${LLVM_VERSION} /usr/bin/llvm-symbolizer @@ -111,5 +108,4 @@ RUN arch=${TARGETARCH:-amd64} \ && mv "/tmp/sccache-$SCCACHE_VERSION-$rarch-unknown-linux-musl/sccache" /usr/bin \ && rm "/tmp/sccache-$SCCACHE_VERSION-$rarch-unknown-linux-musl" -r - COPY process_functional_tests_result.py / From 1b91b7b999e8681d972e3a5a93a4a166bbfd4e2a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Feb 2024 00:16:15 +0100 Subject: [PATCH 169/276] Better logs in CI and non-interactive install --- programs/install/Install.cpp | 30 ++++++++++++++++++------------ 1 file changed, 18 insertions(+), 12 deletions(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 52f30098b38..978b5cc38ba 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -79,10 +79,6 @@ namespace ErrorCodes } -/// ANSI escape sequence for intense color in terminal. -#define HILITE "\033[1m" -#define END_HILITE "\033[0m" - #if defined(OS_DARWIN) /// Until createUser() and createGroup() are implemented, only sudo-less installations are supported/default for macOS. static constexpr auto DEFAULT_CLICKHOUSE_SERVER_USER = ""; @@ -216,6 +212,16 @@ int mainEntryClickHouseInstall(int argc, char ** argv) { try { + const char * start_hilite = ""; + const char * end_hilite = ""; + + if (isatty(STDOUT_FILENO)) + { + /// ANSI escape sequence for intense color in terminal. + start_hilite = "\033[1m"; + end_hilite = "\033[0m"; + } + po::options_description desc; desc.add_options() ("help,h", "produce help message") @@ -799,13 +805,13 @@ int mainEntryClickHouseInstall(int argc, char ** argv) /// Set up password for default user. if (has_password_for_default_user) { - fmt::print(HILITE "Password for default user is already specified. To remind or reset, see {} and {}." END_HILITE "\n", - users_config_file.string(), users_d.string()); + fmt::print("{}Password for default user is already specified. To remind or reset, see {} and {}.{}\n", + start_hilite, users_config_file.string(), users_d.string(), end_hilite); } else if (!can_ask_password) { - fmt::print(HILITE "Password for default user is empty string. See {} and {} to change it." END_HILITE "\n", - users_config_file.string(), users_d.string()); + fmt::print("{}Password for default user is empty string. See {} and {} to change it.{}\n", + start_hilite, users_config_file.string(), users_d.string(), end_hilite); } else { @@ -839,7 +845,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) "\n"; out.sync(); out.finalize(); - fmt::print(HILITE "Password for default user is saved in file {}." END_HILITE "\n", password_file); + fmt::print("{}Password for default user is saved in file {}.{}\n", start_hilite, password_file, end_hilite); #else out << "\n" " \n" @@ -850,13 +856,13 @@ int mainEntryClickHouseInstall(int argc, char ** argv) "\n"; out.sync(); out.finalize(); - fmt::print(HILITE "Password for default user is saved in plaintext in file {}." END_HILITE "\n", password_file); + fmt::print("{}Password for default user is saved in plaintext in file {}.{}\n", start_hilite, password_file, end_hilite); #endif has_password_for_default_user = true; } else - fmt::print(HILITE "Password for default user is empty string. See {} and {} to change it." END_HILITE "\n", - users_config_file.string(), users_d.string()); + fmt::print("{}Password for default user is empty string. See {} and {} to change it.{}\n", + start_hilite, users_config_file.string(), users_d.string(), end_hilite); } /** Set capabilities for the binary. From a585ae60a85aa22bfee4f3e2a47685c915cbb770 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Feb 2024 00:30:29 +0100 Subject: [PATCH 170/276] English (somewhat incomplete) --- programs/install/Install.cpp | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 978b5cc38ba..a4a4672f5c4 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -713,7 +713,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) { fmt::print("Users config file {} already exists, will keep it and extract users info from it.\n", users_config_file.string()); - /// Check if password for default user already specified. + /// Check if password for the default user already specified. ConfigProcessor processor(users_config_file.string(), /* throw_on_bad_incl = */ false, /* log_to_console = */ false); ConfigurationPtr configuration(new Poco::Util::XMLConfiguration(processor.processConfig())); @@ -805,12 +805,12 @@ int mainEntryClickHouseInstall(int argc, char ** argv) /// Set up password for default user. if (has_password_for_default_user) { - fmt::print("{}Password for default user is already specified. To remind or reset, see {} and {}.{}\n", + fmt::print("{}Password for the default user is already specified. To remind or reset, see {} and {}.{}\n", start_hilite, users_config_file.string(), users_d.string(), end_hilite); } else if (!can_ask_password) { - fmt::print("{}Password for default user is empty string. See {} and {} to change it.{}\n", + fmt::print("{}Password for the default user is an empty string. See {} and {} to change it.{}\n", start_hilite, users_config_file.string(), users_d.string(), end_hilite); } else @@ -820,7 +820,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) char buf[1000] = {}; std::string password; - if (auto * result = readpassphrase("Enter password for default user: ", buf, sizeof(buf), 0)) + if (auto * result = readpassphrase("Enter password for the default user: ", buf, sizeof(buf), 0)) password = result; if (!password.empty()) @@ -845,7 +845,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) "\n"; out.sync(); out.finalize(); - fmt::print("{}Password for default user is saved in file {}.{}\n", start_hilite, password_file, end_hilite); + fmt::print("{}Password for the default user is saved in file {}.{}\n", start_hilite, password_file, end_hilite); #else out << "\n" " \n" @@ -856,12 +856,12 @@ int mainEntryClickHouseInstall(int argc, char ** argv) "\n"; out.sync(); out.finalize(); - fmt::print("{}Password for default user is saved in plaintext in file {}.{}\n", start_hilite, password_file, end_hilite); + fmt::print("{}Password for the default user is saved in plaintext in file {}.{}\n", start_hilite, password_file, end_hilite); #endif has_password_for_default_user = true; } else - fmt::print("{}Password for default user is empty string. See {} and {} to change it.{}\n", + fmt::print("{}Password for the default user is an empty string. See {} and {} to change it.{}\n", start_hilite, users_config_file.string(), users_d.string(), end_hilite); } From f386e34b912d3ba2b4af4abff1632b6f4985f42d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Feb 2024 00:45:16 +0100 Subject: [PATCH 171/276] Fix an error by rewriting the CI from Perl to Python --- tests/ci/build_download_helper.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index 21012f6337d..99044e786cc 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -113,7 +113,7 @@ def get_build_name_for_check(check_name: str) -> str: def read_build_urls(build_name: str, reports_path: Union[Path, str]) -> List[str]: for root, _, files in os.walk(reports_path): for f in files: - if build_name in f: + if f.endswith(f"_{build_name}.json"): logging.info("Found build report json %s", f) with open(os.path.join(root, f), "r", encoding="utf-8") as file_handler: build_report = json.load(file_handler) From ad665f9b063f49903c74771bcb92091906926411 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Feb 2024 00:46:12 +0100 Subject: [PATCH 172/276] Better --- tests/ci/build_download_helper.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index 99044e786cc..74478fd8038 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -112,10 +112,10 @@ def get_build_name_for_check(check_name: str) -> str: def read_build_urls(build_name: str, reports_path: Union[Path, str]) -> List[str]: for root, _, files in os.walk(reports_path): - for f in files: - if f.endswith(f"_{build_name}.json"): - logging.info("Found build report json %s", f) - with open(os.path.join(root, f), "r", encoding="utf-8") as file_handler: + for file in files: + if file.endswith(f"_{build_name}.json"): + logging.info("Found build report json %s", file) + with open(os.path.join(root, file), "r", encoding="utf-8") as file_handler: build_report = json.load(file_handler) return build_report["build_urls"] # type: ignore return [] From b58a2387ef12e5b18c10b22827063412b0df56e5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Feb 2024 00:49:48 +0100 Subject: [PATCH 173/276] Probably better --- packages/build | 2 ++ 1 file changed, 2 insertions(+) diff --git a/packages/build b/packages/build index c2285b8ee7c..b2dd085d9dd 100755 --- a/packages/build +++ b/packages/build @@ -130,6 +130,8 @@ if [ -n "$SANITIZER" ]; then fi elif [[ $BUILD_TYPE == 'debug' ]]; then VERSION_POSTFIX+="+debug" +elif [[ $BUILD_TYPE =~ 'coverage' ]]; then + VERSION_POSTFIX+="+coverage" fi if [[ "$PKG_ROOT" != "$SOURCE" ]]; then From 07ae125a52c60370d74f4fe44c5fd29b0bc328c2 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 9 Feb 2024 00:34:00 +0000 Subject: [PATCH 174/276] Automatic style fix --- tests/ci/build_download_helper.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index 74478fd8038..0d24cb80021 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -115,7 +115,9 @@ def read_build_urls(build_name: str, reports_path: Union[Path, str]) -> List[str for file in files: if file.endswith(f"_{build_name}.json"): logging.info("Found build report json %s", file) - with open(os.path.join(root, file), "r", encoding="utf-8") as file_handler: + with open( + os.path.join(root, file), "r", encoding="utf-8" + ) as file_handler: build_report = json.load(file_handler) return build_report["build_urls"] # type: ignore return [] From fe6d8316a35ce5ca93759a3eac2e36193476af22 Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Thu, 8 Feb 2024 17:34:43 -0800 Subject: [PATCH 175/276] [Docs] Add default cloud core settings --- docs/en/operations/settings/settings.md | 44 ++++++++++++++++++++----- 1 file changed, 36 insertions(+), 8 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 3a826b095d2..a552616b3ec 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -508,7 +508,9 @@ Possible values: - Any positive integer number of hops. - 0 — No hops allowed. -Default value: 0. +Default value: `0`. + +Cloud default value: `10`. ## insert_null_as_default {#insert_null_as_default} @@ -1126,7 +1128,9 @@ Possible values: - 0 (or 1) — `INSERT SELECT` no parallel execution. - Positive integer. Bigger than 1. -Default value: 0. +Default value: `0`. + +Cloud default value: `2`. Parallel `INSERT SELECT` has effect only if the `SELECT` part is executed in parallel, see [max_threads](#max_threads) setting. Higher values will lead to higher memory usage. @@ -1207,7 +1211,9 @@ Default value: 10000. Cancels HTTP read-only queries (e.g. SELECT) when a client closes the connection without waiting for the response. -Default value: 0 +Default value: `0`. + +Cloud default value: `1`. ## poll_interval {#poll-interval} @@ -1946,6 +1952,8 @@ Possible values: Default value: `200`. +Cloud default value: `1000`. + ### async_insert_poll_timeout_ms {#async-insert-poll-timeout-ms} Timeout in milliseconds for polling data from asynchronous insert queue. @@ -2132,6 +2140,8 @@ Possible values: Default value: 0 +Cloud default value: `20`. + Keeper request retries are done after some timeout. The timeout is controlled by the following settings: `insert_keeper_retry_initial_backoff_ms`, `insert_keeper_retry_max_backoff_ms`. The first retry is done after `insert_keeper_retry_initial_backoff_ms` timeout. The consequent timeouts will be calculated as follows: ``` @@ -2660,6 +2670,8 @@ Type: [UInt64](../../sql-reference/data-types/int-uint.md). Default value: 1000000000 nanoseconds (once a second). +Cloud default value: `0`. + See also: - System table [trace_log](../../operations/system-tables/trace_log.md/#system_tables-trace_log) @@ -2683,6 +2695,8 @@ Type: [UInt64](../../sql-reference/data-types/int-uint.md). Default value: 1000000000 nanoseconds. +Cloud default value: `0`. + See also: - System table [trace_log](../../operations/system-tables/trace_log.md/#system_tables-trace_log) @@ -2804,6 +2818,8 @@ Possible values: Default value: `0`. +Cloud default value: `1`. + **See Also** - [Distributed Table Engine](../../engines/table-engines/special/distributed.md/#distributed) @@ -3319,7 +3335,9 @@ Possible values: - a string representing any valid table engine name -Default value: `None` +Default value: `None`. + +Cloud default value: `SharedMergeTree`. **Example** @@ -3689,6 +3707,8 @@ Possible values: Default value: `0`. +Cloud default value: `1`. + ## live_view_heartbeat_interval {#live-view-heartbeat-interval} Sets the heartbeat interval in seconds to indicate [live view](../../sql-reference/statements/create/view.md/#live-view) is alive . @@ -3933,6 +3953,8 @@ Possible values: Default value: `throw`. +Cloud default value: `none`. + ## flatten_nested {#flatten-nested} Sets the data format of a [nested](../../sql-reference/data-types/nested-data-structures/index.md) columns. @@ -4068,6 +4090,8 @@ Possible values: Default value: `1`. +Cloud default value: `0`. + :::note `alter_sync` is applicable to `Replicated` tables only, it does nothing to alters of not `Replicated` tables. ::: @@ -4723,6 +4747,8 @@ other connections are cancelled. Queries with `max_parallel_replicas > 1` are su Enabled by default. +Disabled by default on Cloud. + ## hedged_connection_timeout {#hedged_connection_timeout} If we can't establish connection with replica after this timeout in hedged requests, we start working with the next replica without cancelling connection to the previous. @@ -5348,10 +5374,11 @@ Default value: `false`. ## max_partition_size_to_drop -Restriction on dropping partitions in query time. +Restriction on dropping partitions in query time. The value 0 means that you can drop partitions without any restrictions. Default value: 50 GB. -The value 0 means that you can drop partitions without any restrictions. + +Cloud default value: 1 TB. :::note This query setting overwrites its server setting equivalent, see [max_partition_size_to_drop](/docs/en/operations/server-configuration-parameters/settings.md/#max-partition-size-to-drop) @@ -5359,10 +5386,11 @@ This query setting overwrites its server setting equivalent, see [max_partition_ ## max_table_size_to_drop -Restriction on deleting tables in query time. +Restriction on deleting tables in query time. The value 0 means that you can delete all tables without any restrictions. Default value: 50 GB. -The value 0 means that you can delete all tables without any restrictions. + +Cloud default value: 1 TB. :::note This query setting overwrites its server setting equivalent, see [max_table_size_to_drop](/docs/en/operations/server-configuration-parameters/settings.md/#max-table-size-to-drop) From d17a12fe6451e02dc13101cbdacb4e81951c239e Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Thu, 8 Feb 2024 17:46:37 -0800 Subject: [PATCH 176/276] [Docs] Fix for default cloud core settings --- docs/en/operations/settings/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index a552616b3ec..b6db3ccc197 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3707,8 +3707,6 @@ Possible values: Default value: `0`. -Cloud default value: `1`. - ## live_view_heartbeat_interval {#live-view-heartbeat-interval} Sets the heartbeat interval in seconds to indicate [live view](../../sql-reference/statements/create/view.md/#live-view) is alive . @@ -3915,6 +3913,8 @@ Possible values: Default value: `0`. +Cloud default value: `1`. + ## database_replicated_initial_query_timeout_sec {#database_replicated_initial_query_timeout_sec} Sets how long initial DDL query should wait for Replicated database to process previous DDL queue entries in seconds. From 2954cdf200f9f03dc1fa4c1a5fff37454edba3b7 Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Thu, 8 Feb 2024 18:06:34 -0800 Subject: [PATCH 177/276] [Docs] Add cloud default values for query complexity --- .../operations/settings/query-complexity.md | 22 +++++++++++++++---- 1 file changed, 18 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/settings/query-complexity.md b/docs/en/operations/settings/query-complexity.md index 9a80f977ed1..f6d5a2a5017 100644 --- a/docs/en/operations/settings/query-complexity.md +++ b/docs/en/operations/settings/query-complexity.md @@ -1,4 +1,4 @@ ---- +![image](https://github.com/ClickHouse/ClickHouse/assets/3936029/17039d32-4699-4498-bcff-947079345a66)--- slug: /en/operations/settings/query-complexity sidebar_position: 59 sidebar_label: Restrictions on Query Complexity @@ -28,6 +28,8 @@ The maximum amount of RAM to use for running a query on a single server. The default setting is unlimited (set to `0`). +Cloud default value: 17 GB. + The setting does not consider the volume of available memory or the total volume of memory on the machine. The restriction applies to a single query within a single server. You can use `SHOW PROCESSLIST` to see the current memory consumption for each query. @@ -104,7 +106,9 @@ Possible values: - Maximum volume of RAM (in bytes) that can be used by the single [GROUP BY](../../sql-reference/statements/select/group-by.md#select-group-by-clause) operation. - 0 — `GROUP BY` in external memory disabled. -Default value: 0. +Default value: `0`. + +Cloud default value: 8 GB. ## max_bytes_before_external_sort {#settings-max_bytes_before_external_sort} @@ -115,6 +119,8 @@ Enables or disables execution of `ORDER BY` clauses in external memory. See [ORD Default value: 0. +Cloud default value: 8 GB. + ## max_rows_to_sort {#max-rows-to-sort} A maximum number of rows before sorting. This allows you to limit memory consumption when sorting. @@ -129,7 +135,11 @@ What to do if the number of rows received before sorting exceeds one of the limi ## max_result_rows {#setting-max_result_rows} -Limit on the number of rows in the result. Also checked for subqueries, and on remote servers when running parts of a distributed query. +Limit on the number of rows in the result. Also checked for subqueries, and on remote servers when running parts of a distributed query. No limit is applied when value is `0`. + +Default value: `0`. + +Cloud default value: `500000`. ## max_result_bytes {#max-result-bytes} @@ -137,10 +147,14 @@ Limit on the number of bytes in the result. The same as the previous setting. ## result_overflow_mode {#result-overflow-mode} -What to do if the volume of the result exceeds one of the limits: ‘throw’ or ‘break’. By default, throw. +What to do if the volume of the result exceeds one of the limits: ‘throw’ or ‘break’. Using ‘break’ is similar to using LIMIT. `Break` interrupts execution only at the block level. This means that amount of returned rows is greater than [max_result_rows](#setting-max_result_rows), multiple of [max_block_size](../../operations/settings/settings.md#setting-max_block_size) and depends on [max_threads](../../operations/settings/settings.md#max_threads). +Default value: `throw`. + +Cloud default value: `break`. + Example: ``` sql From 32305adb0865f67f589565f8e2416dcdf991c693 Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Thu, 8 Feb 2024 18:10:01 -0800 Subject: [PATCH 178/276] [Docs] Remove accidental link on query complexity page --- docs/en/operations/settings/query-complexity.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/query-complexity.md b/docs/en/operations/settings/query-complexity.md index f6d5a2a5017..9c54e22b01d 100644 --- a/docs/en/operations/settings/query-complexity.md +++ b/docs/en/operations/settings/query-complexity.md @@ -1,4 +1,4 @@ -![image](https://github.com/ClickHouse/ClickHouse/assets/3936029/17039d32-4699-4498-bcff-947079345a66)--- +--- slug: /en/operations/settings/query-complexity sidebar_position: 59 sidebar_label: Restrictions on Query Complexity From d466e8d61ef7b4813fd4e632ed0ff7486d8e312f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Feb 2024 03:55:39 +0100 Subject: [PATCH 179/276] Run coverage collection sequentially --- docker/test/stateless/run.sh | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index ea76447aef2..55349c6c406 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -185,11 +185,15 @@ function run_tests() if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then ADDITIONAL_OPTIONS+=('--replicated-database') + # Too many tests fail for DatabaseReplicated in parallel. ADDITIONAL_OPTIONS+=('--jobs') ADDITIONAL_OPTIONS+=('2') + elif [[ 1 == $(clickhouse-client --query "SELECT value LIKE '%sanitize-coverage%' FROM system.build_options WHERE name = 'CXX_FLAGS'") ]]; then + # Coverage on a per-test basis could only be collected sequentially. + # Do not set the --jobs parameter. + echo "Running tests with coverage collection." else - # Too many tests fail for DatabaseReplicated in parallel. All other - # configurations are OK. + # All other configurations are OK. ADDITIONAL_OPTIONS+=('--jobs') ADDITIONAL_OPTIONS+=('8') fi From 2dfd310cf26740344ecc633a5ae438f834649ce7 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 9 Feb 2024 03:31:30 +0000 Subject: [PATCH 180/276] Hide URL/S3 'headers' argument in SHOW CREATE --- src/Parsers/ASTFunction.cpp | 121 +++++++++++++++--- .../0_stateless/02968_url_args.reference | 9 +- tests/queries/0_stateless/02968_url_args.sql | 17 ++- 3 files changed, 126 insertions(+), 21 deletions(-) diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index e309dec2131..e7f7b48091a 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -71,6 +71,13 @@ namespace size_t count = 0; /// Mostly it's either 0 or 1. There are only a few cases where `count` can be greater than 1 (e.g. see `encrypt`). /// In all known cases secret arguments are consecutive bool are_named = false; /// Arguments like `password = 'password'` are considered as named arguments. + /// E.g. "headers" in `url('..', headers('foo' = '[HIDDEN]'))` + std::vector nested_maps; + + bool hasSecrets() const + { + return count != 0 || !nested_maps.empty(); + } }; Result getResult() const { return result; } @@ -127,6 +134,10 @@ namespace /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) findEncryptionFunctionSecretArguments(); } + else if (function.name == "url") + { + findURLSecretArguments(); + } } void findMySQLFunctionSecretArguments() @@ -143,6 +154,25 @@ namespace } } + /// Returns the number of arguments excluding "headers" and "extra_credentials" (which should + /// always be at the end). Marks "headers" as secret, if found. + size_t excludeS3OrURLNestedMaps() + { + size_t count = arguments->size(); + while (count > 0) + { + const ASTFunction * f = arguments->at(count - 1)->as(); + if (!f) + break; + if (f->name == "headers") + result.nested_maps.push_back(f->name); + else if (f->name != "extra_credentials") + break; + count -= 1; + } + return count; + } + void findS3FunctionSecretArguments(bool is_cluster_function) { /// s3Cluster('cluster_name', 'url', ...) has 'url' as its second argument. @@ -156,9 +186,10 @@ namespace } /// We should check other arguments first because we don't need to do any replacement in case of - /// s3('url', NOSIGN, 'format' [, 'compression']) - /// s3('url', 'format', 'structure' [, 'compression']) - if ((url_arg_idx + 3 <= arguments->size()) && (arguments->size() <= url_arg_idx + 4)) + /// s3('url', NOSIGN, 'format' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) + /// s3('url', 'format', 'structure' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) + size_t count = excludeS3OrURLNestedMaps(); + if ((url_arg_idx + 3 <= count) && (count <= url_arg_idx + 4)) { String second_arg; if (tryGetStringFromArgument(url_arg_idx + 1, &second_arg)) @@ -174,7 +205,14 @@ namespace /// We're going to replace 'aws_secret_access_key' with '[HIDDEN]' for the following signatures: /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') - markSecretArgument(url_arg_idx + 2); + if (url_arg_idx + 2 < count) + markSecretArgument(url_arg_idx + 2); + } + + void findURLSecretArguments() + { + if (!isNamedCollectionName(0)) + excludeS3OrURLNestedMaps(); } bool tryGetStringFromArgument(size_t arg_idx, String * res, bool allow_identifier = true) const @@ -347,6 +385,10 @@ namespace /// S3('url', ['aws_access_key_id', 'aws_secret_access_key',] ...) findS3TableEngineSecretArguments(); } + else if (engine_name == "URL") + { + findURLSecretArguments(); + } } void findExternalDistributedTableEngineSecretArguments() @@ -373,9 +415,10 @@ namespace } /// We should check other arguments first because we don't need to do any replacement in case of - /// S3('url', NOSIGN, 'format' [, 'compression']) - /// S3('url', 'format', 'compression') - if ((3 <= arguments->size()) && (arguments->size() <= 4)) + /// S3('url', NOSIGN, 'format' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) + /// S3('url', 'format', 'compression' [, extra_credentials(..)] [, headers(..)]) + size_t count = excludeS3OrURLNestedMaps(); + if ((3 <= count) && (count <= 4)) { String second_arg; if (tryGetStringFromArgument(1, &second_arg)) @@ -383,7 +426,7 @@ namespace if (boost::iequals(second_arg, "NOSIGN")) return; /// The argument after 'url' is "NOSIGN". - if (arguments->size() == 3) + if (count == 3) { if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) return; /// The argument after 'url' is a format: S3('url', 'format', ...) @@ -391,11 +434,12 @@ namespace } } - /// We replace 'aws_secret_access_key' with '[HIDDEN'] for the following signatures: + /// We replace 'aws_secret_access_key' with '[HIDDEN]' for the following signatures: /// S3('url', 'aws_access_key_id', 'aws_secret_access_key') /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format') /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') - markSecretArgument(2); + if (2 < count) + markSecretArgument(2); } void findDatabaseEngineSecretArguments() @@ -724,6 +768,25 @@ ASTSelectWithUnionQuery * ASTFunction::tryGetQueryArgument() const } +static bool formatNamedArgWithHiddenValue(IAST * arg, const IAST::FormatSettings & settings, IAST::FormatState & state, IAST::FormatStateStacked frame) +{ + const auto * equals_func = arg->as(); + if (!equals_func || (equals_func->name != "equals")) + return false; + const auto * expr_list = equals_func->arguments->as(); + if (!expr_list) + return false; + const auto & equal_args = expr_list->children; + if (equal_args.size() != 2) + return false; + + equal_args[0]->formatImpl(settings, state, frame); + settings.ostr << (settings.hilite ? IAST::hilite_operator : "") << " = " << (settings.hilite ? IAST::hilite_none : ""); + settings.ostr << "'[HIDDEN]'"; + + return true; +} + void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { frame.expression_list_prepend_whitespace = false; @@ -1133,17 +1196,37 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format if (argument->as()) settings.ostr << "SETTINGS "; - if (!settings.show_secrets && (secret_arguments.start <= i) && (i < secret_arguments.start + secret_arguments.count)) + if (!settings.show_secrets) { - if (secret_arguments.are_named) + if (secret_arguments.start <= i && i < secret_arguments.start + secret_arguments.count) { - assert_cast(argument.get())->arguments->children[0]->formatImpl(settings, state, nested_dont_need_parens); - settings.ostr << (settings.hilite ? hilite_operator : "") << " = " << (settings.hilite ? hilite_none : ""); + if (secret_arguments.are_named) + { + assert_cast(argument.get())->arguments->children[0]->formatImpl(settings, state, nested_dont_need_parens); + settings.ostr << (settings.hilite ? hilite_operator : "") << " = " << (settings.hilite ? hilite_none : ""); + } + settings.ostr << "'[HIDDEN]'"; + if (size <= secret_arguments.start + secret_arguments.count && !secret_arguments.are_named) + break; /// All other arguments should also be hidden. + continue; + } + + const ASTFunction * function = argument->as(); + if (function && function->arguments && std::count(secret_arguments.nested_maps.begin(), secret_arguments.nested_maps.end(), function->name) != 0) + { + /// headers('foo' = '[HIDDEN]', 'bar' = '[HIDDEN]') + settings.ostr << (settings.hilite ? hilite_function : "") << function->name << (settings.hilite ? hilite_none : "") << "("; + for (size_t j = 0; j < function->arguments->children.size(); ++j) + { + if (j != 0) + settings.ostr << ", "; + auto inner_arg = function->arguments->children[j]; + if (!formatNamedArgWithHiddenValue(inner_arg.get(), settings, state, nested_dont_need_parens)) + inner_arg->formatImpl(settings, state, nested_dont_need_parens); + } + settings.ostr << ")"; + continue; } - settings.ostr << "'[HIDDEN]'"; - if (size <= secret_arguments.start + secret_arguments.count && !secret_arguments.are_named) - break; /// All other arguments should also be hidden. - continue; } if ((i == 1) && special_hilite_regexp @@ -1166,7 +1249,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format bool ASTFunction::hasSecretParts() const { - return (FunctionSecretArgumentsFinder{*this}.getResult().count > 0) || childrenHaveSecretParts(); + return (FunctionSecretArgumentsFinder{*this}.getResult().hasSecrets()) || childrenHaveSecretParts(); } String getFunctionName(const IAST * ast) diff --git a/tests/queries/0_stateless/02968_url_args.reference b/tests/queries/0_stateless/02968_url_args.reference index aa19e45301c..1c3693e4a66 100644 --- a/tests/queries/0_stateless/02968_url_args.reference +++ b/tests/queries/0_stateless/02968_url_args.reference @@ -1 +1,8 @@ -CREATE TABLE default.a\n(\n `x` Int64\n)\nENGINE = URL(\'https://example.com/\', \'CSV\', headers(\'foo\' = \'bar\')) +CREATE TABLE default.a\n(\n `x` Int64\n)\nENGINE = URL(\'https://example.com/\', \'CSV\', headers(\'foo\' = \'[HIDDEN]\', \'a\' = \'[HIDDEN]\')) +CREATE TABLE default.b\n(\n `x` Int64\n)\nENGINE = URL(\'https://example.com/\', \'CSV\', headers()) +CREATE TABLE default.c\n(\n `x` Int64\n)\nENGINE = S3(\'https://example.s3.amazonaws.com/a.csv\', \'NOSIGN\', \'CSV\', headers(\'foo\' = \'[HIDDEN]\')) +CREATE TABLE default.d\n(\n `x` Int64\n)\nENGINE = S3(\'https://example.s3.amazonaws.com/a.csv\', \'NOSIGN\', headers(\'foo\' = \'[HIDDEN]\')) +CREATE VIEW default.e\n(\n `x` Int64\n) AS\nSELECT count()\nFROM url(\'https://example.com/\', CSV, headers(\'foo\' = \'[HIDDEN]\', \'a\' = \'[HIDDEN]\')) +CREATE VIEW default.f\n(\n `x` Int64\n) AS\nSELECT count()\nFROM url(\'https://example.com/\', CSV, headers()) +CREATE VIEW default.g\n(\n `x` Int64\n) AS\nSELECT count()\nFROM s3(\'https://example.s3.amazonaws.com/a.csv\', CSV, headers(\'foo\' = \'[HIDDEN]\')) +CREATE VIEW default.h\n(\n `x` Int64\n) AS\nSELECT count()\nFROM s3(\'https://example.s3.amazonaws.com/a.csv\', headers(\'foo\' = \'[HIDDEN]\')) diff --git a/tests/queries/0_stateless/02968_url_args.sql b/tests/queries/0_stateless/02968_url_args.sql index 8bee9fec0ac..e97ea381ea5 100644 --- a/tests/queries/0_stateless/02968_url_args.sql +++ b/tests/queries/0_stateless/02968_url_args.sql @@ -1,2 +1,17 @@ -create table a (x Int64) engine URL('https://example.com/', CSV, headers('foo' = 'bar')); +create table a (x Int64) engine URL('https://example.com/', CSV, headers('foo' = 'bar', 'a' = '13')); show create a; +create table b (x Int64) engine URL('https://example.com/', CSV, headers()); +show create b; +create table c (x Int64) engine S3('https://example.s3.amazonaws.com/a.csv', NOSIGN, CSV, headers('foo' = 'bar')); +show create c; +create table d (x Int64) engine S3('https://example.s3.amazonaws.com/a.csv', NOSIGN, headers('foo' = 'bar')); +show create d; + +create view e (x Int64) as select count() from url('https://example.com/', CSV, headers('foo' = 'bar', 'a' = '13')); +show create e; +create view f (x Int64) as select count() from url('https://example.com/', CSV, headers()); +show create f; +create view g (x Int64) as select count() from s3('https://example.s3.amazonaws.com/a.csv', CSV, headers('foo' = 'bar')); +show create g; +create view h (x Int64) as select count() from s3('https://example.s3.amazonaws.com/a.csv', headers('foo' = 'bar')); +show create h; From 2a34bbb0e027f40c65da7db475496c20a455e0a0 Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Thu, 8 Feb 2024 21:36:45 -0800 Subject: [PATCH 181/276] [Docs] Cloud core settings corrections Co-authored-by: Alexey Milovidov --- docs/en/operations/settings/settings.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index b6db3ccc197..9e259ea43b4 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1130,7 +1130,7 @@ Possible values: Default value: `0`. -Cloud default value: `2`. +Cloud default value: from `2` to `4`, depending on the service size. Parallel `INSERT SELECT` has effect only if the `SELECT` part is executed in parallel, see [max_threads](#max_threads) setting. Higher values will lead to higher memory usage. @@ -2138,7 +2138,7 @@ Possible values: - Positive integer. - 0 — Retries are disabled -Default value: 0 +Default value: 20 Cloud default value: `20`. @@ -3335,7 +3335,7 @@ Possible values: - a string representing any valid table engine name -Default value: `None`. +Default value: `MergeTree`. Cloud default value: `SharedMergeTree`. From 89d4b1e77b545c18d4ad6017aa04ebe2d66da192 Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Thu, 8 Feb 2024 21:39:08 -0800 Subject: [PATCH 182/276] [Docs] Specify that some query profiler settings are disabled in Cloud --- docs/en/operations/settings/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 9e259ea43b4..a275878f32e 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -2670,7 +2670,7 @@ Type: [UInt64](../../sql-reference/data-types/int-uint.md). Default value: 1000000000 nanoseconds (once a second). -Cloud default value: `0`. +**Temporarily disabled in ClickHouse Cloud.** See also: @@ -2695,7 +2695,7 @@ Type: [UInt64](../../sql-reference/data-types/int-uint.md). Default value: 1000000000 nanoseconds. -Cloud default value: `0`. +**Temporarily disabled in ClickHouse Cloud.** See also: From aa05f07130cb2fe6070a32198138348df4bf1a03 Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Thu, 8 Feb 2024 21:41:11 -0800 Subject: [PATCH 183/276] [Docs] Corrections for default cloud values for query complexity Co-authored-by: Alexey Milovidov --- docs/en/operations/settings/query-complexity.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/operations/settings/query-complexity.md b/docs/en/operations/settings/query-complexity.md index 9c54e22b01d..d86f18ff982 100644 --- a/docs/en/operations/settings/query-complexity.md +++ b/docs/en/operations/settings/query-complexity.md @@ -28,7 +28,7 @@ The maximum amount of RAM to use for running a query on a single server. The default setting is unlimited (set to `0`). -Cloud default value: 17 GB. +Cloud default value: depends on the amount of RAM on the replica. The setting does not consider the volume of available memory or the total volume of memory on the machine. The restriction applies to a single query within a single server. @@ -108,7 +108,7 @@ Possible values: Default value: `0`. -Cloud default value: 8 GB. +Cloud default value: half the memory amount per replica. ## max_bytes_before_external_sort {#settings-max_bytes_before_external_sort} @@ -119,7 +119,7 @@ Enables or disables execution of `ORDER BY` clauses in external memory. See [ORD Default value: 0. -Cloud default value: 8 GB. +Cloud default value: half the memory amount per replica. ## max_rows_to_sort {#max-rows-to-sort} @@ -139,7 +139,7 @@ Limit on the number of rows in the result. Also checked for subqueries, and on r Default value: `0`. -Cloud default value: `500000`. +Cloud default value: `0`. ## max_result_bytes {#max-result-bytes} @@ -153,7 +153,7 @@ Using ‘break’ is similar to using LIMIT. `Break` interrupts execution only a Default value: `throw`. -Cloud default value: `break`. +Cloud default value: `throw`. Example: From e8fa8c3f7525c1507263b35222060dae81ae94f8 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 9 Feb 2024 06:36:45 +0000 Subject: [PATCH 184/276] no-fasttest --- tests/queries/0_stateless/02968_url_args.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02968_url_args.sql b/tests/queries/0_stateless/02968_url_args.sql index e97ea381ea5..a9ac96970e0 100644 --- a/tests/queries/0_stateless/02968_url_args.sql +++ b/tests/queries/0_stateless/02968_url_args.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest + create table a (x Int64) engine URL('https://example.com/', CSV, headers('foo' = 'bar', 'a' = '13')); show create a; create table b (x Int64) engine URL('https://example.com/', CSV, headers()); From 42f2fefeab9941b11436dafcdcd62cfd31346980 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Feb 2024 10:40:43 +0100 Subject: [PATCH 185/276] Fix data race and leak --- src/Coordination/KeeperSnapshotManager.cpp | 6 +-- src/Coordination/KeeperStorage.cpp | 51 ++++++++++------------ src/Coordination/KeeperStorage.h | 41 +++++------------ src/Coordination/ZooKeeperDataReader.cpp | 4 +- 4 files changed, 39 insertions(+), 63 deletions(-) diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index d7c9acae07a..091571b4a1a 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -82,7 +82,7 @@ namespace writeBinary(node.czxid, out); writeBinary(node.mzxid, out); writeBinary(node.ctime(), out); - writeBinary(node.mtime(), out); + writeBinary(node.mtime, out); writeBinary(node.version, out); writeBinary(node.cversion, out); writeBinary(node.aversion, out); @@ -143,9 +143,7 @@ namespace int64_t ctime; readBinary(ctime, in); node.setCtime(ctime); - int64_t mtime; - readBinary(mtime, in); - node.setMtime(mtime); + readBinary(node.mtime, in); readBinary(node.version, in); readBinary(node.cversion, in); readBinary(node.aversion, in); diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 9d4c22164d3..446d3877c1c 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -175,13 +175,13 @@ uint64_t calculateDigest(std::string_view path, const KeeperStorage::Node & node if (node.data_size != 0) { chassert(node.data != nullptr); - hash.update(node.data, node.data_size); + hash.update(node.getData()); } hash.update(node.czxid); hash.update(node.mzxid); hash.update(node.ctime()); - hash.update(node.mtime()); + hash.update(node.mtime); hash.update(node.version); hash.update(node.cversion); hash.update(node.aversion); @@ -189,17 +189,17 @@ uint64_t calculateDigest(std::string_view path, const KeeperStorage::Node & node hash.update(node.numChildren()); hash.update(node.pzxid); + auto digest = hash.get64(); + + /// 0 means no cached digest + if (digest == 0) + return 1; + return hash.get64(); } } -KeeperStorage::Node::~Node() -{ - if (data_size) - delete[] data; -} - KeeperStorage::Node & KeeperStorage::Node::operator=(const Node & other) { if (this == &other) @@ -209,8 +209,8 @@ KeeperStorage::Node & KeeperStorage::Node::operator=(const Node & other) mzxid = other.mzxid; pzxid = other.pzxid; acl_id = other.acl_id; - has_cached_digest_and_ctime = other.has_cached_digest_and_ctime; - is_ephemeral_and_mtime = other.is_ephemeral_and_mtime; + mtime = other.mtime; + is_ephemeral_and_ctime = other.is_ephemeral_and_ctime; ephemeral_or_children_data = other.ephemeral_or_children_data; data_size = other.data_size; version = other.version; @@ -219,8 +219,8 @@ KeeperStorage::Node & KeeperStorage::Node::operator=(const Node & other) if (data_size != 0) { - data = new char[data_size]; - memcpy(data, other.data, data_size); + data = std::unique_ptr(new char[data_size]); + memcpy(data.get(), other.data.get(), data_size); } children = other.children; @@ -244,8 +244,8 @@ void KeeperStorage::Node::copyStats(const Coordination::Stat & stat) mzxid = stat.mzxid; pzxid = stat.pzxid; + mtime = stat.mtime; setCtime(stat.ctime); - setMtime(stat.mtime); version = stat.version; cversion = stat.cversion; @@ -253,7 +253,7 @@ void KeeperStorage::Node::copyStats(const Coordination::Stat & stat) if (stat.ephemeralOwner == 0) { - is_ephemeral_and_mtime.is_ephemeral = false; + is_ephemeral_and_ctime.is_ephemeral = false; setNumChildren(stat.numChildren); } else @@ -267,7 +267,7 @@ void KeeperStorage::Node::setResponseStat(Coordination::Stat & response_stat) co response_stat.czxid = czxid; response_stat.mzxid = mzxid; response_stat.ctime = ctime(); - response_stat.mtime = mtime(); + response_stat.mtime = mtime; response_stat.version = version; response_stat.cversion = cversion; response_stat.aversion = aversion; @@ -288,8 +288,8 @@ void KeeperStorage::Node::setData(const String & new_data) data_size = static_cast(new_data.size()); if (data_size != 0) { - data = new char[new_data.size()]; - memcpy(data, new_data.data(), data_size); + data = std::unique_ptr(new char[new_data.size()]); + memcpy(data.get(), new_data.data(), data_size); } } @@ -305,16 +305,13 @@ void KeeperStorage::Node::removeChild(StringRef child_path) void KeeperStorage::Node::invalidateDigestCache() const { - has_cached_digest_and_ctime.has_cached_digest = false; + cached_digest = 0; } UInt64 KeeperStorage::Node::getDigest(const std::string_view path) const { - if (!has_cached_digest_and_ctime.has_cached_digest) - { + if (cached_digest == 0) cached_digest = calculateDigest(path, *this); - has_cached_digest_and_ctime.has_cached_digest = true; - } return cached_digest; }; @@ -326,17 +323,17 @@ void KeeperStorage::Node::shallowCopy(const KeeperStorage::Node & other) pzxid = other.pzxid; acl_id = other.acl_id; /// 0 -- no ACL by default - has_cached_digest_and_ctime = other.has_cached_digest_and_ctime; + mtime = other.mtime; - is_ephemeral_and_mtime = other.is_ephemeral_and_mtime; + is_ephemeral_and_ctime = other.is_ephemeral_and_ctime; ephemeral_or_children_data = other.ephemeral_or_children_data; data_size = other.data_size; if (data_size != 0) { - data = new char[data_size]; - memcpy(data, other.data, data_size); + data = std::unique_ptr(new char[data_size]); + memcpy(data.get(), other.data.get(), data_size); } version = other.version; @@ -1448,7 +1445,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce { value.version++; value.mzxid = zxid; - value.setMtime(time); + value.mtime = time; value.setData(data); }, request.version}); diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index cf9c1710d3d..222433dc5d4 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -38,17 +38,13 @@ public: int64_t pzxid{0}; uint64_t acl_id = 0; /// 0 -- no ACL by default - mutable struct - { - bool has_cached_digest : 1; - int64_t ctime : 63; - } has_cached_digest_and_ctime{false, 0}; + int64_t mtime; struct { bool is_ephemeral : 1; - int64_t mtime : 63; - } is_ephemeral_and_mtime{false, 0}; + int64_t ctime : 63; + } is_ephemeral_and_ctime{false, 0}; union { @@ -60,19 +56,15 @@ public: } children_info; } ephemeral_or_children_data{0}; - char * data{nullptr}; + std::unique_ptr data{nullptr}; uint32_t data_size{0}; int32_t version{0}; int32_t cversion{0}; int32_t aversion{0}; - /// we cannot use `std::optional because we want to - /// pack the boolean with seq_num above mutable uint64_t cached_digest = 0; - ~Node(); - Node() = default; Node & operator=(const Node & other); @@ -83,7 +75,7 @@ public: bool isEphemeral() const { - return is_ephemeral_and_mtime.is_ephemeral; + return is_ephemeral_and_ctime.is_ephemeral; } int64_t ephemeralOwner() const @@ -96,7 +88,7 @@ public: void setEphemeralOwner(int64_t ephemeral_owner) { - is_ephemeral_and_mtime.is_ephemeral = ephemeral_owner != 0; + is_ephemeral_and_ctime.is_ephemeral = ephemeral_owner != 0; ephemeral_or_children_data.ephemeral_owner = ephemeral_owner; } @@ -146,22 +138,12 @@ public: int64_t ctime() const { - return has_cached_digest_and_ctime.ctime; + return is_ephemeral_and_ctime.ctime; } void setCtime(uint64_t ctime) { - has_cached_digest_and_ctime.ctime = ctime; - } - - int64_t mtime() const - { - return is_ephemeral_and_mtime.mtime; - } - - void setMtime(uint64_t mtime) - { - is_ephemeral_and_mtime.mtime = mtime; + is_ephemeral_and_ctime.ctime = ctime; } void copyStats(const Coordination::Stat & stat); @@ -173,7 +155,7 @@ public: void setData(const String & new_data); - StringRef getData() const noexcept { return {data, data_size}; } + StringRef getData() const noexcept { return {data.get(), data_size}; } void addChild(StringRef child_path); @@ -205,10 +187,11 @@ public: NO_DIGEST = 0, V1 = 1, V2 = 2, // added system nodes that modify the digest on startup so digest from V0 is invalid - V3 = 3 // fixed bug with casting, removed duplicate czxid usage + V3 = 3, // fixed bug with casting, removed duplicate czxid usage + V4 = 4 // 0 is not a valid digest value }; - static constexpr auto CURRENT_DIGEST_VERSION = DigestVersion::V3; + static constexpr auto CURRENT_DIGEST_VERSION = DigestVersion::V4; struct ResponseForSession { diff --git a/src/Coordination/ZooKeeperDataReader.cpp b/src/Coordination/ZooKeeperDataReader.cpp index e027b44b3a2..c7b1abf1d83 100644 --- a/src/Coordination/ZooKeeperDataReader.cpp +++ b/src/Coordination/ZooKeeperDataReader.cpp @@ -114,9 +114,7 @@ int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, LoggerP int64_t ctime; Coordination::read(ctime, in); node.setCtime(ctime); - int64_t mtime; - Coordination::read(mtime, in); - node.setMtime(mtime); + Coordination::read(node.mtime, in); Coordination::read(node.version, in); Coordination::read(node.cversion, in); Coordination::read(node.aversion, in); From 1ff0a5430e63600f65b006513ac16d15b577d5b8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 9 Feb 2024 11:08:54 +0100 Subject: [PATCH 186/276] Fix unpoison call --- src/Common/memcmpSmall.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/memcmpSmall.h b/src/Common/memcmpSmall.h index 0b05b4a53bd..103eabb5b8d 100644 --- a/src/Common/memcmpSmall.h +++ b/src/Common/memcmpSmall.h @@ -661,8 +661,8 @@ inline bool memequalSmallAllowOverflow15(const Char * a, size_t a_size, const Ch if (a_size != b_size) return false; - __msan_unpoison(&a[a_size - a_size % 16], a_size % 16); - __msan_unpoison(&b[b_size - b_size % 16], b_size % 16); + __msan_unpoison_overflow_15(a, a_size); + __msan_unpoison_overflow_15(b, b_size); for (size_t offset = 0; offset < a_size; offset += 16) { From 8655c11280c154f267b8f36fd24dc21c7b786aec Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 9 Feb 2024 11:45:15 +0100 Subject: [PATCH 187/276] Fix typo --- src/Storages/StorageMerge.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 09c38996b22..79d7b83cada 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -786,7 +786,7 @@ QueryTreeNodePtr replaceTableExpressionAndRemoveJoin( projection.clear(); NamesAndTypes projection_columns; - // Select only required columns from the table, because prjection list may contain: + // Select only required columns from the table, because projection list may contain: // 1. aggregate functions // 2. expressions referencing other tables of JOIN for (auto const & column_name : required_column_names) From d2eac53c86ccfbd809f45511dfc087c11bc92f9a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 9 Feb 2024 12:06:01 +0100 Subject: [PATCH 188/276] Fix --- src/Coordination/KeeperStorage.cpp | 45 ++++++++++++++++++++++++++---- src/Coordination/KeeperStorage.h | 38 ++----------------------- utils/keeper-data-dumper/main.cpp | 2 +- 3 files changed, 44 insertions(+), 41 deletions(-) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 446d3877c1c..c62ed90e378 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -172,10 +172,11 @@ uint64_t calculateDigest(std::string_view path, const KeeperStorage::Node & node hash.update(path); - if (node.data_size != 0) + auto data = node.getData(); + if (!data.empty()) { - chassert(node.data != nullptr); - hash.update(node.getData()); + chassert(data.data() != nullptr); + hash.update(data); } hash.update(node.czxid); @@ -195,7 +196,7 @@ uint64_t calculateDigest(std::string_view path, const KeeperStorage::Node & node if (digest == 0) return 1; - return hash.get64(); + return digest; } } @@ -477,6 +478,40 @@ void KeeperStorage::UncommittedState::applyDelta(const Delta & delta) delta.operation); } +bool KeeperStorage::UncommittedState::hasACL(int64_t session_id, bool is_local, std::function predicate) const +{ + const auto check_auth = [&](const auto & auth_ids) + { + for (const auto & auth : auth_ids) + { + using TAuth = std::remove_reference_t; + + const AuthID * auth_ptr = nullptr; + if constexpr (std::is_pointer_v) + auth_ptr = auth; + else + auth_ptr = &auth; + + if (predicate(*auth_ptr)) + return true; + } + return false; + }; + + if (is_local) + return check_auth(storage.session_and_auth[session_id]); + + if (check_auth(storage.session_and_auth[session_id])) + return true; + + // check if there are uncommitted + const auto auth_it = session_and_auth.find(session_id); + if (auth_it == session_and_auth.end()) + return false; + + return check_auth(auth_it->second); +} + void KeeperStorage::UncommittedState::addDelta(Delta new_delta) { const auto & added_delta = deltas.emplace_back(std::move(new_delta)); @@ -1228,7 +1263,7 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce { node_it->value.setResponseStat(response.stat); auto data = node_it->value.getData(); - response.data = std::string(data.data, data.size); + response.data = std::string(data); response.error = Coordination::Error::ZOK; } diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 222433dc5d4..9743e051422 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -38,7 +38,7 @@ public: int64_t pzxid{0}; uint64_t acl_id = 0; /// 0 -- no ACL by default - int64_t mtime; + int64_t mtime{0}; struct { @@ -155,7 +155,7 @@ public: void setData(const String & new_data); - StringRef getData() const noexcept { return {data.get(), data_size}; } + std::string_view getData() const noexcept { return {data.get(), data_size}; } void addChild(StringRef child_path); @@ -335,39 +335,7 @@ public: void applyDelta(const Delta & delta); - bool hasACL(int64_t session_id, bool is_local, std::function predicate) - { - const auto check_auth = [&](const auto & auth_ids) - { - for (const auto & auth : auth_ids) - { - using TAuth = std::remove_reference_t; - - const AuthID * auth_ptr = nullptr; - if constexpr (std::is_pointer_v) - auth_ptr = auth; - else - auth_ptr = &auth; - - if (predicate(*auth_ptr)) - return true; - } - return false; - }; - - if (is_local) - return check_auth(storage.session_and_auth[session_id]); - - if (check_auth(storage.session_and_auth[session_id])) - return true; - - // check if there are uncommitted - const auto auth_it = session_and_auth.find(session_id); - if (auth_it == session_and_auth.end()) - return false; - - return check_auth(auth_it->second); - } + bool hasACL(int64_t session_id, bool is_local, std::function predicate) const; void forEachAuthInSession(int64_t session_id, std::function func) const; diff --git a/utils/keeper-data-dumper/main.cpp b/utils/keeper-data-dumper/main.cpp index 12acd0a8db9..bae44bb1628 100644 --- a/utils/keeper-data-dumper/main.cpp +++ b/utils/keeper-data-dumper/main.cpp @@ -26,7 +26,7 @@ void dumpMachine(std::shared_ptr machine) std::cout << key << "\n"; auto value = storage.container.getValue(key); std::cout << "\tStat: {version: " << value.version << - ", mtime: " << value.mtime() << + ", mtime: " << value.mtime << ", emphemeralOwner: " << value.ephemeralOwner() << ", czxid: " << value.czxid << ", mzxid: " << value.mzxid << From e522e23ce86b92c0b9765687cda8153a1f8ccd42 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 9 Feb 2024 12:41:25 +0100 Subject: [PATCH 189/276] Update reference file --- .../02731_parallel_replicas_join_subquery.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference index ec4928bc325..48ea1154bc3 100644 --- a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference +++ b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference @@ -31,7 +31,7 @@ 29 2j&S)ba?XG QuQj 17163829389637435056 3 UlI+1 14144472852965836438 =============== QUERIES EXECUTED BY PARALLEL INNER QUERY ALONE =============== -0 3 SELECT `__table1`.`key` AS `key`, `__table1`.`value1` AS `value1`, `__table1`.`value2` AS `value2`, toUInt64(min(`__table1`.`time`)) AS `start_ts` FROM `default`.`join_inner_table` AS `__table1` PREWHERE (`__table1`.`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`__table1`.`number` > 1610517366120) GROUP BY `__table1`.`key`, `__table1`.`value1`, `__table1`.`value2` ORDER BY `__table1`.`key` ASC, `__table1`.`value1` ASC, `__table1`.`value2` ASC LIMIT _CAST(10, \'UInt64\') SETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer = 1 +0 3 SELECT `__table1`.`key` AS `key`, `__table1`.`value1` AS `value1`, `__table1`.`value2` AS `value2`, toUInt64(min(`__table1`.`time`)) AS `start_ts` FROM `default`.`join_inner_table` AS `__table1` PREWHERE (`__table1`.`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`__table1`.`number` > _CAST(1610517366120, \'UInt64\')) GROUP BY `__table1`.`key`, `__table1`.`value1`, `__table1`.`value2` ORDER BY `__table1`.`key` ASC, `__table1`.`value1` ASC, `__table1`.`value2` ASC LIMIT _CAST(10, \'UInt64\') SETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer = 1 0 3 SELECT `key`, `value1`, `value2`, toUInt64(min(`time`)) AS `start_ts` FROM `default`.`join_inner_table` PREWHERE (`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`number` > toUInt64(\'1610517366120\')) GROUP BY `key`, `value1`, `value2` ORDER BY `key` ASC, `value1` ASC, `value2` ASC LIMIT 10 1 1 -- Parallel inner query alone\nSELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\nFROM join_inner_table\nPREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\nGROUP BY key, value1, value2\nORDER BY key, value1, value2\nLIMIT 10\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=0; 1 1 -- Parallel inner query alone\nSELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\nFROM join_inner_table\nPREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\nGROUP BY key, value1, value2\nORDER BY key, value1, value2\nLIMIT 10\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=1; @@ -58,7 +58,7 @@ U c 10 UlI+1 10 bX?}ix [ Ny]2 G 10 t 1610517366120) GROUP BY `__table1`.`key`, `__table1`.`value1`, `__table1`.`value2` +0 3 SELECT `__table1`.`key` AS `key`, `__table1`.`value1` AS `value1`, `__table1`.`value2` AS `value2` FROM `default`.`join_inner_table` AS `__table1` PREWHERE (`__table1`.`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`__table1`.`number` > _CAST(1610517366120, \'UInt64\')) GROUP BY `__table1`.`key`, `__table1`.`value1`, `__table1`.`value2` 0 3 SELECT `__table2`.`value1` AS `value1`, `__table2`.`value2` AS `value2`, count() AS `count` FROM `default`.`join_outer_table` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` USING (`key`) GROUP BY `__table1`.`key`, `__table2`.`value1`, `__table2`.`value2` 0 3 SELECT `key`, `value1`, `value2` FROM `default`.`join_inner_table` PREWHERE (`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`number` > toUInt64(\'1610517366120\')) GROUP BY `key`, `value1`, `value2` 0 3 SELECT `value1`, `value2`, count() AS `count` FROM `default`.`join_outer_table` ALL INNER JOIN `_data_` USING (`key`) GROUP BY `key`, `value1`, `value2` From fc34507a94019465ce6aa23ae084fd21714c9aec Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 9 Feb 2024 12:48:07 +0100 Subject: [PATCH 190/276] Bump version --- tests/integration/test_replicating_constants/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_replicating_constants/test.py b/tests/integration/test_replicating_constants/test.py index 90132b71a64..af8916dd625 100644 --- a/tests/integration/test_replicating_constants/test.py +++ b/tests/integration/test_replicating_constants/test.py @@ -9,7 +9,7 @@ node2 = cluster.add_instance( "node2", with_zookeeper=True, image="clickhouse/clickhouse-server", - tag="22.3", + tag="23.3", with_installed_binary=True, ) From 3ee2dda51142c5321837eeaf7a00404ac46c74f4 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 9 Feb 2024 12:59:06 +0100 Subject: [PATCH 191/276] Fix special build reports in release branches --- .github/workflows/release_branches.yml | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index c5d8294b999..87fbf363f0b 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -206,13 +206,8 @@ jobs: if: ${{ !cancelled() }} needs: - RunConfig - - BuilderDebRelease - - BuilderDebAarch64 - - BuilderDebAsan - - BuilderDebTsan - - BuilderDebUBsan - - BuilderDebMsan - - BuilderDebDebug + - BuilderBinDarwin + - BuilderBinDarwinAarch64 uses: ./.github/workflows/reusable_test.yml with: test_name: ClickHouse special build check From 54b7fdb8329160cee44b47af980dd0499eeed253 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 9 Feb 2024 12:24:33 +0000 Subject: [PATCH 192/276] Update version_date.tsv and changelogs after v24.1.3.31-stable --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v24.1.3.31-stable.md | 34 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 2 ++ 5 files changed, 39 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v24.1.3.31-stable.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index fe33bf9e0ea..e8293869a56 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -34,7 +34,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.1.2.5" +ARG VERSION="24.1.3.31" ARG PACKAGES="clickhouse-keeper" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index f0adadd2d59..3cf89f1b841 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.1.2.5" +ARG VERSION="24.1.3.31" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index dd00684cc24..d477d8aaca1 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -27,7 +27,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="24.1.2.5" +ARG VERSION="24.1.3.31" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v24.1.3.31-stable.md b/docs/changelogs/v24.1.3.31-stable.md new file mode 100644 index 00000000000..046ca451fbc --- /dev/null +++ b/docs/changelogs/v24.1.3.31-stable.md @@ -0,0 +1,34 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.1.3.31-stable (135b08cbd28) FIXME as compared to v24.1.2.5-stable (b2605dd4a5a) + +#### Improvement +* Backported in [#59569](https://github.com/ClickHouse/ClickHouse/issues/59569): Now dashboard understands both compressed and uncompressed state of URL's #hash (backward compatibility). Continuation of [#59124](https://github.com/ClickHouse/ClickHouse/issues/59124) . [#59548](https://github.com/ClickHouse/ClickHouse/pull/59548) ([Amos Bird](https://github.com/amosbird)). +* Backported in [#59776](https://github.com/ClickHouse/ClickHouse/issues/59776): Added settings `split_parts_ranges_into_intersecting_and_non_intersecting_final` and `split_intersecting_parts_ranges_into_layers_final`. This settings are needed to disable optimizations for queries with `FINAL` and needed for debug only. [#59705](https://github.com/ClickHouse/ClickHouse/pull/59705) ([Maksim Kita](https://github.com/kitaisreal)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix `ASTAlterCommand::formatImpl` in case of column specific settings… [#59445](https://github.com/ClickHouse/ClickHouse/pull/59445) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Make MAX use the same rules as permutation for complex types [#59498](https://github.com/ClickHouse/ClickHouse/pull/59498) ([Raúl Marín](https://github.com/Algunenano)). +* Fix corner case when passing `update_insert_deduplication_token_in_dependent_materialized_views` [#59544](https://github.com/ClickHouse/ClickHouse/pull/59544) ([Jordi Villar](https://github.com/jrdi)). +* Fix incorrect result of arrayElement / map[] on empty value [#59594](https://github.com/ClickHouse/ClickHouse/pull/59594) ([Raúl Marín](https://github.com/Algunenano)). +* Fix crash in topK when merging empty states [#59603](https://github.com/ClickHouse/ClickHouse/pull/59603) ([Raúl Marín](https://github.com/Algunenano)). +* Maintain function alias in RewriteSumFunctionWithSumAndCountVisitor [#59658](https://github.com/ClickHouse/ClickHouse/pull/59658) ([Raúl Marín](https://github.com/Algunenano)). +* Fix leftPad / rightPad function with FixedString input [#59739](https://github.com/ClickHouse/ClickHouse/pull/59739) ([Raúl Marín](https://github.com/Algunenano)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Revert "Backport [#59650](https://github.com/ClickHouse/ClickHouse/issues/59650) to 24.1: MergeTree FINAL optimization diagnostics and settings"'. [#59701](https://github.com/ClickHouse/ClickHouse/pull/59701) ([Raúl Marín](https://github.com/Algunenano)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Fix 02720_row_policy_column_with_dots [#59453](https://github.com/ClickHouse/ClickHouse/pull/59453) ([Duc Canh Le](https://github.com/canhld94)). +* Refactoring of dashboard state encoding [#59554](https://github.com/ClickHouse/ClickHouse/pull/59554) ([Sergei Trifonov](https://github.com/serxa)). +* MergeTree FINAL optimization diagnostics and settings [#59650](https://github.com/ClickHouse/ClickHouse/pull/59650) ([Maksim Kita](https://github.com/kitaisreal)). +* Pin python dependencies in stateless tests [#59663](https://github.com/ClickHouse/ClickHouse/pull/59663) ([Raúl Marín](https://github.com/Algunenano)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 36b1db583a8..1e931b95b41 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,5 +1,7 @@ +v24.1.3.31-stable 2024-02-09 v24.1.2.5-stable 2024-02-02 v24.1.1.2048-stable 2024-01-30 +v23.12.4.15-stable 2024-02-09 v23.12.3.40-stable 2024-02-02 v23.12.2.59-stable 2024-01-05 v23.12.1.1368-stable 2023-12-28 From 582abd40fdba6cca9a645271034fc75a292ae3a2 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 9 Feb 2024 12:25:38 +0000 Subject: [PATCH 193/276] Update version_date.tsv and changelogs after v23.12.4.15-stable --- docker/keeper/Dockerfile | 2 +- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v23.12.4.15-stable.md | 21 +++++++++++++++++++++ utils/list-versions/version_date.tsv | 2 ++ 5 files changed, 26 insertions(+), 3 deletions(-) create mode 100644 docs/changelogs/v23.12.4.15-stable.md diff --git a/docker/keeper/Dockerfile b/docker/keeper/Dockerfile index fe33bf9e0ea..e8293869a56 100644 --- a/docker/keeper/Dockerfile +++ b/docker/keeper/Dockerfile @@ -34,7 +34,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.1.2.5" +ARG VERSION="24.1.3.31" ARG PACKAGES="clickhouse-keeper" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index f0adadd2d59..3cf89f1b841 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -32,7 +32,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="24.1.2.5" +ARG VERSION="24.1.3.31" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" ARG DIRECT_DOWNLOAD_URLS="" diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index dd00684cc24..d477d8aaca1 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -27,7 +27,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="24.1.2.5" +ARG VERSION="24.1.3.31" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v23.12.4.15-stable.md b/docs/changelogs/v23.12.4.15-stable.md new file mode 100644 index 00000000000..a67b5aee312 --- /dev/null +++ b/docs/changelogs/v23.12.4.15-stable.md @@ -0,0 +1,21 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v23.12.4.15-stable (4233d111d20) FIXME as compared to v23.12.3.40-stable (a594704ae75) + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix incorrect result of arrayElement / map[] on empty value [#59594](https://github.com/ClickHouse/ClickHouse/pull/59594) ([Raúl Marín](https://github.com/Algunenano)). +* Fix crash in topK when merging empty states [#59603](https://github.com/ClickHouse/ClickHouse/pull/59603) ([Raúl Marín](https://github.com/Algunenano)). +* Fix distributed table with a constant sharding key [#59606](https://github.com/ClickHouse/ClickHouse/pull/59606) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix leftPad / rightPad function with FixedString input [#59739](https://github.com/ClickHouse/ClickHouse/pull/59739) ([Raúl Marín](https://github.com/Algunenano)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Fix 02720_row_policy_column_with_dots [#59453](https://github.com/ClickHouse/ClickHouse/pull/59453) ([Duc Canh Le](https://github.com/canhld94)). +* Pin python dependencies in stateless tests [#59663](https://github.com/ClickHouse/ClickHouse/pull/59663) ([Raúl Marín](https://github.com/Algunenano)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 36b1db583a8..1e931b95b41 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,5 +1,7 @@ +v24.1.3.31-stable 2024-02-09 v24.1.2.5-stable 2024-02-02 v24.1.1.2048-stable 2024-01-30 +v23.12.4.15-stable 2024-02-09 v23.12.3.40-stable 2024-02-02 v23.12.2.59-stable 2024-01-05 v23.12.1.1368-stable 2023-12-28 From 361dd64374c843293ae6cccbcebcb04388c21ff9 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 9 Feb 2024 13:41:49 +0100 Subject: [PATCH 194/276] Fix expected error message --- tests/integration/test_access_for_functions/test.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_access_for_functions/test.py b/tests/integration/test_access_for_functions/test.py index 5069468110c..3e58c961421 100644 --- a/tests/integration/test_access_for_functions/test.py +++ b/tests/integration/test_access_for_functions/test.py @@ -38,8 +38,11 @@ def test_access_rights_for_function(): instance.query("GRANT DROP FUNCTION ON *.* TO B") instance.query("DROP FUNCTION MySum", user="B") - assert "Unknown function MySum" in instance.query_and_get_error( - "SELECT MySum(1, 2)" + + function_resolution_error = instance.query_and_get_error("SELECT MySum(1, 2)") + assert ( + "Unknown function MySum" in function_resolution_error + or "Function with name 'MySum' does not exists." in function_resolution_error ) instance.query("REVOKE CREATE FUNCTION ON *.* FROM A") From 19fcffb69949b1aa2e7da5231f8c65f79c3bdf36 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 9 Feb 2024 13:42:46 +0100 Subject: [PATCH 195/276] Update tests/analyzer_integration_broken_tests.txt --- tests/analyzer_integration_broken_tests.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/analyzer_integration_broken_tests.txt b/tests/analyzer_integration_broken_tests.txt index c04ed440c18..725b746f1a0 100644 --- a/tests/analyzer_integration_broken_tests.txt +++ b/tests/analyzer_integration_broken_tests.txt @@ -1,4 +1,3 @@ -test_access_for_functions/test.py::test_access_rights_for_function test_build_sets_from_multiple_threads/test.py::test_set test_concurrent_backups_s3/test.py::test_concurrent_backups test_distributed_backward_compatability/test.py::test_distributed_in_tuple From 9dfe3f55590d3f094b6bb7dffda6e4d8f7b44739 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 9 Feb 2024 12:21:11 +0000 Subject: [PATCH 196/276] CI: do not reuse builds on release branches #job_package_debug #job_style_check --- .github/workflows/master.yml | 22 +++++++++++++--------- .github/workflows/release_branches.yml | 11 ++++++++--- .github/workflows/reusable_build.yml | 9 +++++++-- tests/ci/ci.py | 8 +++++++- tests/ci/ci_config.py | 5 ----- 5 files changed, 35 insertions(+), 20 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index dac1332adc6..33f98e492b5 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -318,15 +318,19 @@ jobs: run_command: | python3 build_report_check.py "$CHECK_NAME" MarkReleaseReady: - needs: [RunConfig, BuilderBinDarwin, BuilderBinDarwinAarch64, BuilderDebRelease, BuilderDebAarch64] - if: ${{ !failure() && !cancelled() }} - uses: ./.github/workflows/reusable_test.yml - with: - test_name: Mark Commit Release Ready - runner_type: style-checker - data: ${{ needs.RunConfig.outputs.data }} - run_command: | - python3 mark_release_ready.py + needs: + - BuilderBinDarwin + - BuilderBinDarwinAarch64 + - BuilderDebRelease + - BuilderDebAarch64 + runs-on: [self-hosted, style-checker] + steps: + - name: Check out repository code + uses: ClickHouse/checkout@v1 + - name: Mark Commit Release Ready + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 mark_release_ready.py ############################################################################################ #################################### INSTALL PACKAGES ###################################### ############################################################################################ diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index c5d8294b999..c1e4a1800c7 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -91,6 +91,8 @@ jobs: build_name: package_release checkout_depth: 0 data: ${{ needs.RunConfig.outputs.data }} + # always rebuild on release branches to be able to publish from any commit + force: true BuilderDebAarch64: needs: [RunConfig, BuildDockers] if: ${{ !failure() && !cancelled() }} @@ -99,6 +101,8 @@ jobs: build_name: package_aarch64 checkout_depth: 0 data: ${{ needs.RunConfig.outputs.data }} + # always rebuild on release branches to be able to publish from any commit + force: true BuilderDebAsan: needs: [RunConfig, BuildDockers] if: ${{ !failure() && !cancelled() }} @@ -142,6 +146,8 @@ jobs: build_name: binary_darwin checkout_depth: 0 data: ${{ needs.RunConfig.outputs.data }} + # always rebuild on release branches to be able to publish from any commit + force: true BuilderBinDarwinAarch64: needs: [RunConfig, BuildDockers] if: ${{ !failure() && !cancelled() }} @@ -150,6 +156,8 @@ jobs: build_name: binary_darwin_aarch64 checkout_depth: 0 data: ${{ needs.RunConfig.outputs.data }} + # always rebuild on release branches to be able to publish from any commit + force: true ############################################################################################ ##################################### Docker images ####################################### ############################################################################################ @@ -225,7 +233,6 @@ jobs: run_command: | python3 build_report_check.py "$CHECK_NAME" MarkReleaseReady: - if: ${{ !failure() && !cancelled() }} needs: - BuilderBinDarwin - BuilderBinDarwinAarch64 @@ -235,8 +242,6 @@ jobs: steps: - name: Check out repository code uses: ClickHouse/checkout@v1 - with: - clear-repository: true - name: Mark Commit Release Ready run: | cd "$GITHUB_WORKSPACE/tests/ci" diff --git a/.github/workflows/reusable_build.yml b/.github/workflows/reusable_build.yml index 6be9d30175e..80d78d93e1b 100644 --- a/.github/workflows/reusable_build.yml +++ b/.github/workflows/reusable_build.yml @@ -26,6 +26,10 @@ name: Build ClickHouse description: json ci data type: string required: true + force: + description: disallow job skipping + type: boolean + default: false additional_envs: description: additional ENV variables to setup the job type: string @@ -33,7 +37,7 @@ name: Build ClickHouse jobs: Build: name: Build-${{inputs.build_name}} - if: contains(fromJson(inputs.data).jobs_data.jobs_to_do, inputs.build_name) + if: ${{ contains(fromJson(inputs.data).jobs_data.jobs_to_do, inputs.build_name) || inputs.force }} env: GITHUB_JOB_OVERRIDDEN: Build-${{inputs.build_name}} runs-on: [self-hosted, '${{inputs.runner_type}}'] @@ -78,7 +82,8 @@ jobs: python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" \ --infile ${{ toJson(inputs.data) }} \ --job-name "$BUILD_NAME" \ - --run + --run \ + ${{ inputs.force && '--force' || '' }} - name: Post # it still be build report to upload for failed build job if: ${{ !cancelled() }} diff --git a/tests/ci/ci.py b/tests/ci/ci.py index ce8d1c8c20e..213225d0ea8 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -816,6 +816,12 @@ def parse_args(parser: argparse.ArgumentParser) -> argparse.Namespace: default=False, help="skip fetching data about job runs, used in --configure action (for debugging and nigthly ci)", ) + parser.add_argument( + "--force", + action="store_true", + default=False, + help="Used with --run, force the job to run, omitting the ci cache", + ) # FIXME: remove, not used parser.add_argument( "--rebuild-all-binaries", @@ -1762,7 +1768,7 @@ def main() -> int: previous_status = job_status.status GHActions.print_in_group("Commit Status Data", job_status) - if previous_status: + if previous_status and not args.force: print( f"Commit status or Build Report is already present - job will be skipped with status: [{previous_status}]" ) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 7c8990e8d16..998d0876527 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -140,8 +140,6 @@ class JobNames(metaclass=WithIter): DOCS_CHECK = "Docs check" BUGFIX_VALIDATE = "tests bugfix validate check" - MARK_RELEASE_READY = "Mark Commit Release Ready" - # dynamically update JobName with Build jobs for attr_name in dir(Build): @@ -828,9 +826,6 @@ CI_CONFIG = CiConfig( ), }, other_jobs_configs={ - JobNames.MARK_RELEASE_READY: TestConfig( - "", job_config=JobConfig(release_only=True) - ), JobNames.DOCKER_SERVER: TestConfig( "", job_config=JobConfig( From c2320c2d160b3a04a03a3bff9fa738af09066c92 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 9 Feb 2024 13:17:49 +0000 Subject: [PATCH 197/276] CI: await tune ups #no_ci_cache #job_package_debug --- tests/ci/ci.py | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 213225d0ea8..47e20b3ec09 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -646,14 +646,14 @@ class CiCache: TIMEOUT = 3600 await_finished: Dict[str, List[int]] = {} round_cnt = 0 - while len(jobs_with_params) > 5 and round_cnt < 3: + while len(jobs_with_params) > 4 and round_cnt < 5: round_cnt += 1 GHActions.print_in_group( f"Wait pending jobs, round [{round_cnt}]:", list(jobs_with_params) ) # this is initial approach to wait pending jobs: - # start waiting for the next TIMEOUT seconds if there are more than X(=5) jobs to wait - # wait TIMEOUT seconds in rounds. Y(=3) is the max number of rounds + # start waiting for the next TIMEOUT seconds if there are more than X(=4) jobs to wait + # wait TIMEOUT seconds in rounds. Y(=5) is the max number of rounds expired_sec = 0 start_at = int(time.time()) while expired_sec < TIMEOUT and jobs_with_params: @@ -701,13 +701,11 @@ class CiCache: print( f"...awaiting continues... seconds left [{TIMEOUT - expired_sec}]" ) - if await_finished: - GHActions.print_in_group( - "Finished jobs:", - [f"{job}:{batches}" for job, batches in await_finished.items()], - ) - else: - print("Awaiting FAILED. No job has finished successfully.") + if await_finished: + GHActions.print_in_group( + f"Finished jobs, round [{round_cnt}]:", + [f"{job}:{batches}" for job, batches in await_finished.items()], + ) GHActions.print_in_group( "Remaining jobs:", [f"{job}:{params['batches']}" for job, params in jobs_with_params.items()], From 68df20114508c838cff0dbc7bc0d7f81025a6bdf Mon Sep 17 00:00:00 2001 From: Alexander Nikolaev Date: Fri, 9 Feb 2024 16:50:50 +0300 Subject: [PATCH 198/276] Add CLICKHOUSE_ALWAYS_RUN_INITDB_SCRIPTS envoronment variable. --- docker/server/entrypoint.sh | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/docker/server/entrypoint.sh b/docker/server/entrypoint.sh index 78f18f376f4..69c0fdda351 100755 --- a/docker/server/entrypoint.sh +++ b/docker/server/entrypoint.sh @@ -118,13 +118,19 @@ if [ -n "$CLICKHOUSE_USER" ] && [ "$CLICKHOUSE_USER" != "default" ] || [ -n "$CL EOT fi +CLICKHOUSE_ALWAYS_RUN_INITDB_SCRIPTS="${CLICKHOUSE_ALWAYS_RUN_INITDB_SCRIPTS:-}" + # checking $DATA_DIR for initialization if [ -d "${DATA_DIR%/}/data" ]; then DATABASE_ALREADY_EXISTS='true' fi -# only run initialization on an empty data directory -if [ -z "${DATABASE_ALREADY_EXISTS}" ]; then +# run initialization if flag CLICKHOUSE_ALWAYS_RUN_INITDB_SCRIPTS is not empty or data directory is empty +if [[ -n "${CLICKHOUSE_ALWAYS_RUN_INITDB_SCRIPTS}" || -z "${DATABASE_ALREADY_EXISTS}" ]]; then + RUN_INITDB_SCRIPTS='true' +fi + +if [ -z "${RUN_INITDB_SCRIPTS}" ]; then if [ -n "$(ls /docker-entrypoint-initdb.d/)" ] || [ -n "$CLICKHOUSE_DB" ]; then # port is needed to check if clickhouse-server is ready for connections HTTP_PORT="$(clickhouse extract-from-config --config-file "$CLICKHOUSE_CONFIG" --key=http_port --try)" From 50828dade25fa0f431907f9a466c305ca3d6e1b2 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 9 Feb 2024 14:53:03 +0100 Subject: [PATCH 199/276] Fix stacktraces on MacOS (#59690) --- CMakeLists.txt | 15 +++++--- contrib/libunwind-cmake/unwind-override.c | 4 +++ contrib/llvm-project | 2 +- src/Common/StackTrace.cpp | 43 ++++++++++++++++++++++- 4 files changed, 58 insertions(+), 6 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 063cfc77302..8b232aa12ef 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -254,10 +254,17 @@ endif() include(cmake/cpu_features.cmake) -# Asynchronous unwind tables are needed for Query Profiler. -# They are already by default on some platforms but possibly not on all platforms. -# Enable it explicitly. -set (COMPILER_FLAGS "${COMPILER_FLAGS} -fasynchronous-unwind-tables") + +# Query Profiler doesn't work on MacOS for several reasons +# - PHDR cache is not available +# - We use native functionality to get stacktraces which is not async signal safe +# and thus we don't need to generate asynchronous unwind tables +if (NOT OS_DARWIN) + # Asynchronous unwind tables are needed for Query Profiler. + # They are already by default on some platforms but possibly not on all platforms. + # Enable it explicitly. + set (COMPILER_FLAGS "${COMPILER_FLAGS} -fasynchronous-unwind-tables") +endif() # Reproducible builds. if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") diff --git a/contrib/libunwind-cmake/unwind-override.c b/contrib/libunwind-cmake/unwind-override.c index 616bab6ae4b..57928d817eb 100644 --- a/contrib/libunwind-cmake/unwind-override.c +++ b/contrib/libunwind-cmake/unwind-override.c @@ -1,6 +1,10 @@ #include +/// On MacOS this function will be replaced with a dynamic symbol +/// from the system library. +#if !defined(OS_DARWIN) int backtrace(void ** buffer, int size) { return unw_backtrace(buffer, size); } +#endif diff --git a/contrib/llvm-project b/contrib/llvm-project index 2568a7cd129..d2142eed980 160000 --- a/contrib/llvm-project +++ b/contrib/llvm-project @@ -1 +1 @@ -Subproject commit 2568a7cd1297c7c3044b0f3cc0c23a6f6444d856 +Subproject commit d2142eed98046a47ff7112e3cc1e197c8a5cd80f diff --git a/src/Common/StackTrace.cpp b/src/Common/StackTrace.cpp index 8431630b16c..7e683ae91de 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -4,6 +4,7 @@ #include #include +#include #include #include #include @@ -24,6 +25,15 @@ #include "config.h" +#include + +#if defined(OS_DARWIN) +/// This header contains functions like `backtrace` and `backtrace_symbols` +/// Which will be used for stack unwinding on Mac. +/// Read: https://developer.apple.com/library/archive/documentation/System/Conceptual/ManPages_iPhoneOS/man3/backtrace.3.html +#include "execinfo.h" +#endif + namespace { /// Currently this variable is set up once on server startup. @@ -262,6 +272,33 @@ void StackTrace::forEachFrame( callback(current_inline_frame); } + callback(current_frame); + } +#elif defined(OS_DARWIN) + UNUSED(fatal); + + /// This function returns an array of string in a special (a little bit weird format) + /// The frame number, library name, address in hex, mangled symbol name, `+` sign, the offset. + char** strs = ::backtrace_symbols(frame_pointers.data(), static_cast(size)); + SCOPE_EXIT_SAFE({free(strs);}); + + for (size_t i = offset; i < size; ++i) + { + StackTrace::Frame current_frame; + + std::vector split; + boost::split(split, strs[i], isWhitespaceASCII); + split.erase( + std::remove_if( + split.begin(), split.end(), + [](const std::string & x) { return x.empty(); }), + split.end()); + assert(split.size() == 6); + + current_frame.virtual_addr = frame_pointers[i]; + current_frame.physical_addr = frame_pointers[i]; + current_frame.object = split[1]; + current_frame.symbol = split[3]; callback(current_frame); } #else @@ -306,7 +343,11 @@ StackTrace::StackTrace(const ucontext_t & signal_context) void StackTrace::tryCapture() { +#if defined(OS_DARWIN) + size = backtrace(frame_pointers.data(), capacity); +#else size = unw_backtrace(frame_pointers.data(), capacity); +#endif __msan_unpoison(frame_pointers.data(), size * sizeof(frame_pointers[0])); } @@ -376,7 +417,7 @@ toStringEveryLineImpl([[maybe_unused]] bool fatal, const StackTraceRefTriple & s return callback(""); size_t frame_index = stack_trace.offset; -#if defined(__ELF__) && !defined(OS_FREEBSD) +#if (defined(__ELF__) && !defined(OS_FREEBSD)) || defined(OS_DARWIN) size_t inline_frame_index = 0; auto callback_wrapper = [&](const StackTrace::Frame & frame) { From 3cca665f667bbda8b8c1a25b35e2684813ea61a3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 9 Feb 2024 14:06:41 +0000 Subject: [PATCH 200/276] Attempt to fix more tests. --- .../QueryPlan/Optimizations/optimizeTree.cpp | 16 +++++++++ .../QueryPlan/Optimizations/splitFilter.cpp | 35 +++++++++++++++++-- 2 files changed, 48 insertions(+), 3 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index c8c95e7443f..93430e072bb 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -118,6 +118,22 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s optimizePrewhere(stack, nodes); optimizePrimaryKeyCondition(stack); + auto & frame = stack.back(); + + /// Traverse all children first. + if (frame.next_child < frame.node->children.size()) + { + auto next_frame = Frame{.node = frame.node->children[frame.next_child]}; + ++frame.next_child; + stack.push_back(next_frame); + continue; + } + + stack.pop_back(); + } + + while (!stack.empty()) + { { /// NOTE: frame cannot be safely used after stack was modified. auto & frame = stack.back(); diff --git a/src/Processors/QueryPlan/Optimizations/splitFilter.cpp b/src/Processors/QueryPlan/Optimizations/splitFilter.cpp index 8c212936195..561ad7302c6 100644 --- a/src/Processors/QueryPlan/Optimizations/splitFilter.cpp +++ b/src/Processors/QueryPlan/Optimizations/splitFilter.cpp @@ -14,19 +14,33 @@ size_t trySplitFilter(QueryPlan::Node * node, QueryPlan::Nodes & nodes) return 0; const auto & expr = filter_step->getExpression(); + const std::string & filter_column_name = filter_step->getFilterColumnName(); /// Do not split if there are function like runningDifference. if (expr->hasStatefulFunctions()) return 0; - auto split = expr->splitActionsForFilter(filter_step->getFilterColumnName()); + bool filter_name_clashs_with_input = false; + if (filter_step->removesFilterColumn()) + { + for (const auto * input : expr->getInputs()) + { + if (input->result_name == filter_column_name) + { + filter_name_clashs_with_input = true; + break; + } + } + } + + auto split = expr->splitActionsForFilter(filter_column_name); if (split.second->trivial()) return 0; bool remove_filter = false; if (filter_step->removesFilterColumn()) - remove_filter = split.second->removeUnusedResult(filter_step->getFilterColumnName()); + remove_filter = split.second->removeUnusedResult(filter_column_name); auto description = filter_step->getStepDescription(); @@ -34,10 +48,25 @@ size_t trySplitFilter(QueryPlan::Node * node, QueryPlan::Nodes & nodes) node->children.swap(filter_node.children); node->children.push_back(&filter_node); + std::string split_filter_name = filter_column_name; + if (filter_name_clashs_with_input) + { + split_filter_name = "__split_filter"; + + for (auto & filter_output : split.first->getOutputs()) + { + if (filter_output->result_name == filter_column_name) + { + filter_output = &split.first->addAlias(*filter_output, split_filter_name); + break; + } + } + } + filter_node.step = std::make_unique( filter_node.children.at(0)->step->getOutputStream(), std::move(split.first), - filter_step->getFilterColumnName(), + std::move(split_filter_name), remove_filter); node->step = std::make_unique(filter_node.step->getOutputStream(), std::move(split.second)); From 7575f0db84dc01fb1a6e5b5404d0f3d298e220f1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 9 Feb 2024 14:36:56 +0000 Subject: [PATCH 201/276] Fix some tests. --- ...771_ignore_data_skipping_indices.reference | 111 +++++++++----- .../02771_ignore_data_skipping_indices.sql | 8 ++ ...ndex_in_function_different_types.reference | 136 +++++++----------- ..._key_index_in_function_different_types.sql | 17 +-- ...f_indexes_support_match_function.reference | 12 -- ...ngrambf_indexes_support_match_function.sql | 89 +----------- 6 files changed, 140 insertions(+), 233 deletions(-) diff --git a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference index fcede2caf2a..e23e3094ca3 100644 --- a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference +++ b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference @@ -1,40 +1,77 @@ 1 2 3 1 2 3 1 2 3 - ReadFromMergeTree (default.data_02771) - Indexes: - PrimaryKey - Condition: true - Parts: 1/1 - Granules: 1/1 - Skip - Name: x_idx - Description: minmax GRANULARITY 1 - Parts: 0/1 - Granules: 0/1 - Skip - Name: y_idx - Description: minmax GRANULARITY 1 - Parts: 0/0 - Granules: 0/0 - Skip - Name: xy_idx - Description: minmax GRANULARITY 1 - Parts: 0/0 - Granules: 0/0 - ReadFromMergeTree (default.data_02771) - Indexes: - PrimaryKey - Condition: true - Parts: 1/1 - Granules: 1/1 - Skip - Name: x_idx - Description: minmax GRANULARITY 1 - Parts: 0/1 - Granules: 0/1 - Skip - Name: y_idx - Description: minmax GRANULARITY 1 - Parts: 0/0 - Granules: 0/0 + ReadFromMergeTree (default.data_02771) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 + Skip + Name: x_idx + Description: minmax GRANULARITY 1 + Parts: 0/1 + Granules: 0/1 + Skip + Name: y_idx + Description: minmax GRANULARITY 1 + Parts: 0/0 + Granules: 0/0 + Skip + Name: xy_idx + Description: minmax GRANULARITY 1 + Parts: 0/0 + Granules: 0/0 + ReadFromMergeTree (default.data_02771) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 + Skip + Name: x_idx + Description: minmax GRANULARITY 1 + Parts: 0/1 + Granules: 0/1 + Skip + Name: y_idx + Description: minmax GRANULARITY 1 + Parts: 0/0 + Granules: 0/0 + ReadFromMergeTree (default.data_02771) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 + Skip + Name: x_idx + Description: minmax GRANULARITY 1 + Parts: 0/1 + Granules: 0/1 + Skip + Name: y_idx + Description: minmax GRANULARITY 1 + Parts: 0/0 + Granules: 0/0 + Skip + Name: xy_idx + Description: minmax GRANULARITY 1 + Parts: 0/0 + Granules: 0/0 + ReadFromMergeTree (default.data_02771) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 + Skip + Name: x_idx + Description: minmax GRANULARITY 1 + Parts: 0/1 + Granules: 0/1 + Skip + Name: y_idx + Description: minmax GRANULARITY 1 + Parts: 0/0 + Granules: 0/0 diff --git a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql index a49239e9de2..716421b7342 100644 --- a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql +++ b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql @@ -24,6 +24,14 @@ SELECT * FROM data_02771 SETTINGS ignore_data_skipping_indices='na_idx'; SELECT * FROM data_02771 WHERE x = 1 AND y = 1 SETTINGS ignore_data_skipping_indices='xy_idx',force_data_skipping_indices='xy_idx' ; -- { serverError 277 } SELECT * FROM data_02771 WHERE x = 1 AND y = 2 SETTINGS ignore_data_skipping_indices='xy_idx'; + +SET allow_experimental_analyzer = 0; + +SELECT * from ( EXPLAIN indexes = 1 SELECT * FROM data_02771 WHERE x = 1 AND y = 2 ) WHERE explain NOT LIKE '%Expression%' AND explain NOT LIKE '%Filter%'; +SELECT * from ( EXPLAIN indexes = 1 SELECT * FROM data_02771 WHERE x = 1 AND y = 2 SETTINGS ignore_data_skipping_indices='xy_idx' ) WHERE explain NOT LIKE '%Expression%' AND explain NOT LIKE '%Filter%'; + +SET allow_experimental_analyzer = 1; + SELECT * from ( EXPLAIN indexes = 1 SELECT * FROM data_02771 WHERE x = 1 AND y = 2 ) WHERE explain NOT LIKE '%Expression%' AND explain NOT LIKE '%Filter%'; SELECT * from ( EXPLAIN indexes = 1 SELECT * FROM data_02771 WHERE x = 1 AND y = 2 SETTINGS ignore_data_skipping_indices='xy_idx' ) WHERE explain NOT LIKE '%Expression%' AND explain NOT LIKE '%Filter%'; diff --git a/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.reference b/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.reference index f34aad737d4..6338d048186 100644 --- a/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.reference +++ b/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.reference @@ -1,88 +1,48 @@ -CreatingSets (Create sets before main query execution) - Expression ((Projection + Before ORDER BY)) - ReadFromMergeTree (default.test_table) - Indexes: - PrimaryKey - Keys: - id - value - Condition: and((id in (-Inf, 10]), (value in 1-element set)) - Parts: 1/1 - Granules: 1/1 -CreatingSets (Create sets before main query execution) - Expression ((Projection + Before ORDER BY)) - ReadFromMergeTree (default.test_table) - Indexes: - PrimaryKey - Keys: - id - value - Condition: and((id in (-Inf, 10]), (value in 1-element set)) - Parts: 1/1 - Granules: 1/1 -CreatingSets (Create sets before main query execution) - Expression ((Projection + Before ORDER BY)) - ReadFromMergeTree (default.test_table) - Indexes: - PrimaryKey - Keys: - id - value - Condition: and((id in (-Inf, 10]), (value in 5-element set)) - Parts: 1/1 - Granules: 1/1 -CreatingSets (Create sets before main query execution) - Expression ((Projection + Before ORDER BY)) - ReadFromMergeTree (default.test_table) - Indexes: - PrimaryKey - Keys: - id - value - Condition: and((id in (-Inf, 10]), (value in 5-element set)) - Parts: 1/1 - Granules: 1/1 -CreatingSets (Create sets before main query execution) - Expression ((Project names + Projection)) - ReadFromMergeTree (default.test_table) - Indexes: - PrimaryKey - Keys: - id - value - Condition: and((id in (-Inf, 10]), (value in 1-element set)) - Parts: 1/1 - Granules: 1/1 -CreatingSets (Create sets before main query execution) - Expression ((Project names + Projection)) - ReadFromMergeTree (default.test_table) - Indexes: - PrimaryKey - Keys: - id - value - Condition: and((id in (-Inf, 10]), (value in 1-element set)) - Parts: 1/1 - Granules: 1/1 -CreatingSets (Create sets before main query execution) - Expression ((Project names + Projection)) - ReadFromMergeTree (default.test_table) - Indexes: - PrimaryKey - Keys: - id - value - Condition: and((id in (-Inf, 10]), (value in 5-element set)) - Parts: 1/1 - Granules: 1/1 -CreatingSets (Create sets before main query execution) - Expression ((Project names + Projection)) - ReadFromMergeTree (default.test_table) - Indexes: - PrimaryKey - Keys: - id - value - Condition: and((id in (-Inf, 10]), (value in 5-element set)) - Parts: 1/1 - Granules: 1/1 +CreatingSets + Expression + Expression + ReadFromMergeTree + Indexes: + PrimaryKey + Keys: + id + value + Condition: and((value in 1-element set), (id in (-Inf, 10])) + Parts: 1/1 + Granules: 1/1 +CreatingSets + Expression + Expression + ReadFromMergeTree + Indexes: + PrimaryKey + Keys: + id + value + Condition: and((value in 1-element set), (id in (-Inf, 10])) + Parts: 1/1 + Granules: 1/1 +CreatingSets + Expression + Expression + ReadFromMergeTree + Indexes: + PrimaryKey + Keys: + id + value + Condition: and((value in 5-element set), (id in (-Inf, 10])) + Parts: 1/1 + Granules: 1/1 +CreatingSets + Expression + Expression + ReadFromMergeTree + Indexes: + PrimaryKey + Keys: + id + value + Condition: and((value in 5-element set), (id in (-Inf, 10])) + Parts: 1/1 + Granules: 1/1 diff --git a/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.sql b/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.sql index 077c49fb22e..585c2635970 100644 --- a/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.sql +++ b/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.sql @@ -7,18 +7,9 @@ CREATE TABLE test_table INSERT INTO test_table SELECT number, number FROM numbers(10); -SET allow_experimental_analyzer = 0; - -EXPLAIN indexes = 1 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT 5); -EXPLAIN indexes = 1 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT '5'); -EXPLAIN indexes = 1 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT toUInt8(number) FROM numbers(5)); -EXPLAIN indexes = 1 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT toString(number) FROM numbers(5)); - -SET allow_experimental_analyzer = 1; - -EXPLAIN indexes = 1 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT 5); -EXPLAIN indexes = 1 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT '5'); -EXPLAIN indexes = 1 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT toUInt8(number) FROM numbers(5)); -EXPLAIN indexes = 1 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT toString(number) FROM numbers(5)); +EXPLAIN indexes = 1, description=0 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT 5); +EXPLAIN indexes = 1, description=0 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT '5'); +EXPLAIN indexes = 1, description=0 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT toUInt8(number) FROM numbers(5)); +EXPLAIN indexes = 1, description=0 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT toString(number) FROM numbers(5)); DROP TABLE test_table; diff --git a/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.reference b/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.reference index 1cf1644fe0a..5c6a213a03f 100644 --- a/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.reference +++ b/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.reference @@ -2,12 +2,8 @@ 2 Hello World 1 Hello ClickHouse 2 Hello World - Granules: 6/6 - Granules: 2/6 Granules: 6/6 Granules: 2/6 - Granules: 6/6 - Granules: 2/6 Granules: 6/6 Granules: 2/6 --- @@ -17,22 +13,14 @@ 1 Hello ClickHouse 2 Hello World 6 World Champion - Granules: 6/6 - Granules: 3/6 Granules: 6/6 Granules: 3/6 - Granules: 6/6 - Granules: 3/6 Granules: 6/6 Granules: 3/6 --- 5 OLAP Database 5 OLAP Database - Granules: 6/6 - Granules: 1/6 Granules: 6/6 Granules: 1/6 - Granules: 6/6 - Granules: 1/6 Granules: 6/6 Granules: 1/6 diff --git a/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.sql b/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.sql index 49d39c601ef..5db9697d018 100644 --- a/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.sql +++ b/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.sql @@ -38,20 +38,7 @@ FROM SELECT * FROM tokenbf_tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id ) WHERE - explain LIKE '%Granules: %' -SETTINGS - allow_experimental_analyzer = 0; - -SELECT * -FROM -( - EXPLAIN PLAN indexes=1 - SELECT * FROM tokenbf_tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id -) -WHERE - explain LIKE '%Granules: %' -SETTINGS - allow_experimental_analyzer = 1; + explain LIKE '%Granules: %'; SELECT * FROM @@ -60,20 +47,7 @@ FROM SELECT * FROM ngrambf_tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id ) WHERE - explain LIKE '%Granules: %' -SETTINGS - allow_experimental_analyzer = 0; - -SELECT * -FROM -( - EXPLAIN PLAN indexes=1 - SELECT * FROM ngrambf_tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id -) -WHERE - explain LIKE '%Granules: %' -SETTINGS - allow_experimental_analyzer = 1; + explain LIKE '%Granules: %'; SELECT '---'; @@ -92,20 +66,7 @@ FROM SELECT * FROM tokenbf_tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id ) WHERE - explain LIKE '%Granules: %' -SETTINGS - allow_experimental_analyzer = 0; - -SELECT * -FROM -( - EXPLAIN PLAN indexes = 1 - SELECT * FROM tokenbf_tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id -) -WHERE - explain LIKE '%Granules: %' -SETTINGS - allow_experimental_analyzer = 1; + explain LIKE '%Granules: %'; SELECT * FROM @@ -114,20 +75,7 @@ FROM SELECT * FROM ngrambf_tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id ) WHERE - explain LIKE '%Granules: %' -SETTINGS - allow_experimental_analyzer = 0; - -SELECT * -FROM -( - EXPLAIN PLAN indexes = 1 - SELECT * FROM ngrambf_tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id -) -WHERE - explain LIKE '%Granules: %' -SETTINGS - allow_experimental_analyzer = 1; + explain LIKE '%Granules: %'; SELECT '---'; @@ -145,19 +93,7 @@ FROM SELECT * FROM tokenbf_tab WHERE match(str, 'OLAP (.*?)*') ORDER BY id ) WHERE - explain LIKE '%Granules: %' -SETTINGS - allow_experimental_analyzer = 0; -SELECT * -FROM -( - EXPLAIN PLAN indexes = 1 - SELECT * FROM tokenbf_tab WHERE match(str, 'OLAP (.*?)*') ORDER BY id -) -WHERE - explain LIKE '%Granules: %' -SETTINGS - allow_experimental_analyzer = 1; + explain LIKE '%Granules: %'; SELECT * FROM @@ -166,20 +102,7 @@ FROM SELECT * FROM ngrambf_tab WHERE match(str, 'OLAP (.*?)*') ORDER BY id ) WHERE - explain LIKE '%Granules: %' -SETTINGS - allow_experimental_analyzer = 0; - -SELECT * -FROM -( - EXPLAIN PLAN indexes = 1 - SELECT * FROM ngrambf_tab WHERE match(str, 'OLAP (.*?)*') ORDER BY id -) -WHERE - explain LIKE '%Granules: %' -SETTINGS - allow_experimental_analyzer = 1; + explain LIKE '%Granules: %'; DROP TABLE tokenbf_tab; DROP TABLE ngrambf_tab; From 1063d9ca5005c317cc6ce5dfaac6cbcfd271e9b7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 9 Feb 2024 14:47:24 +0000 Subject: [PATCH 202/276] Use opd prewhere optimizaer for old analyzer. --- src/Interpreters/InterpreterSelectQuery.cpp | 56 ++++++++++----------- 1 file changed, 28 insertions(+), 28 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 187518b9f6c..d0cf9f1160c 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -600,7 +600,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( query.setFinal(); } - auto analyze = [&] (bool) + auto analyze = [&] (bool try_move_to_prewhere) { /// Allow push down and other optimizations for VIEW: replace with subquery and rewrite it. ASTPtr view_table; @@ -632,37 +632,37 @@ InterpreterSelectQuery::InterpreterSelectQuery( view = nullptr; } - // if (try_move_to_prewhere - // && storage && storage->canMoveConditionsToPrewhere() - // && query.where() && !query.prewhere() - // && !query.hasJoin()) /// Join may produce rows with nulls or default values, it's difficult to analyze if they affected or not. - // { - // /// PREWHERE optimization: transfer some condition from WHERE to PREWHERE if enabled and viable - // if (const auto & column_sizes = storage->getColumnSizes(); !column_sizes.empty()) - // { - // /// Extract column compressed sizes. - // std::unordered_map column_compressed_sizes; - // for (const auto & [name, sizes] : column_sizes) - // column_compressed_sizes[name] = sizes.data_compressed; + if (try_move_to_prewhere + && storage && storage->canMoveConditionsToPrewhere() + && query.where() && !query.prewhere() + && !query.hasJoin()) /// Join may produce rows with nulls or default values, it's difficult to analyze if they affected or not. + { + /// PREWHERE optimization: transfer some condition from WHERE to PREWHERE if enabled and viable + if (const auto & column_sizes = storage->getColumnSizes(); !column_sizes.empty()) + { + /// Extract column compressed sizes. + std::unordered_map column_compressed_sizes; + for (const auto & [name, sizes] : column_sizes) + column_compressed_sizes[name] = sizes.data_compressed; - // SelectQueryInfo current_info; - // current_info.query = query_ptr; - // current_info.syntax_analyzer_result = syntax_analyzer_result; + SelectQueryInfo current_info; + current_info.query = query_ptr; + current_info.syntax_analyzer_result = syntax_analyzer_result; - // Names queried_columns = syntax_analyzer_result->requiredSourceColumns(); - // const auto & supported_prewhere_columns = storage->supportedPrewhereColumns(); + Names queried_columns = syntax_analyzer_result->requiredSourceColumns(); + const auto & supported_prewhere_columns = storage->supportedPrewhereColumns(); - // MergeTreeWhereOptimizer where_optimizer{ - // std::move(column_compressed_sizes), - // metadata_snapshot, - // storage->getConditionEstimatorByPredicate(query_info, storage_snapshot, context), - // queried_columns, - // supported_prewhere_columns, - // log}; + MergeTreeWhereOptimizer where_optimizer{ + std::move(column_compressed_sizes), + metadata_snapshot, + storage->getConditionEstimatorByPredicate(query_info, storage_snapshot, context), + queried_columns, + supported_prewhere_columns, + log}; - // where_optimizer.optimize(current_info, context); - // } - // } + where_optimizer.optimize(current_info, context); + } + } if (query.prewhere() && query.where()) { From d3cccf856199d6cbd5c7eb1caff3ee0d9501351c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 9 Feb 2024 15:44:51 +0000 Subject: [PATCH 203/276] Fixing tests. --- .../QueryPlan/Optimizations/optimizeTree.cpp | 2 + ...f_indexes_support_match_function.reference | 12 +++ ...ngrambf_indexes_support_match_function.sql | 89 +++++++++++++++++-- 3 files changed, 97 insertions(+), 6 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index 93430e072bb..55f7e7cb85b 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -132,6 +132,8 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s stack.pop_back(); } + stack.push_back({.node = &root}); + while (!stack.empty()) { { diff --git a/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.reference b/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.reference index 5c6a213a03f..1cf1644fe0a 100644 --- a/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.reference +++ b/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.reference @@ -2,8 +2,12 @@ 2 Hello World 1 Hello ClickHouse 2 Hello World + Granules: 6/6 + Granules: 2/6 Granules: 6/6 Granules: 2/6 + Granules: 6/6 + Granules: 2/6 Granules: 6/6 Granules: 2/6 --- @@ -13,14 +17,22 @@ 1 Hello ClickHouse 2 Hello World 6 World Champion + Granules: 6/6 + Granules: 3/6 Granules: 6/6 Granules: 3/6 + Granules: 6/6 + Granules: 3/6 Granules: 6/6 Granules: 3/6 --- 5 OLAP Database 5 OLAP Database + Granules: 6/6 + Granules: 1/6 Granules: 6/6 Granules: 1/6 + Granules: 6/6 + Granules: 1/6 Granules: 6/6 Granules: 1/6 diff --git a/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.sql b/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.sql index 5db9697d018..49d39c601ef 100644 --- a/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.sql +++ b/tests/queries/0_stateless/02943_tokenbf_and_ngrambf_indexes_support_match_function.sql @@ -38,7 +38,20 @@ FROM SELECT * FROM tokenbf_tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id ) WHERE - explain LIKE '%Granules: %'; + explain LIKE '%Granules: %' +SETTINGS + allow_experimental_analyzer = 0; + +SELECT * +FROM +( + EXPLAIN PLAN indexes=1 + SELECT * FROM tokenbf_tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id +) +WHERE + explain LIKE '%Granules: %' +SETTINGS + allow_experimental_analyzer = 1; SELECT * FROM @@ -47,7 +60,20 @@ FROM SELECT * FROM ngrambf_tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id ) WHERE - explain LIKE '%Granules: %'; + explain LIKE '%Granules: %' +SETTINGS + allow_experimental_analyzer = 0; + +SELECT * +FROM +( + EXPLAIN PLAN indexes=1 + SELECT * FROM ngrambf_tab WHERE match(str, 'Hello (ClickHouse|World)') ORDER BY id +) +WHERE + explain LIKE '%Granules: %' +SETTINGS + allow_experimental_analyzer = 1; SELECT '---'; @@ -66,7 +92,20 @@ FROM SELECT * FROM tokenbf_tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id ) WHERE - explain LIKE '%Granules: %'; + explain LIKE '%Granules: %' +SETTINGS + allow_experimental_analyzer = 0; + +SELECT * +FROM +( + EXPLAIN PLAN indexes = 1 + SELECT * FROM tokenbf_tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id +) +WHERE + explain LIKE '%Granules: %' +SETTINGS + allow_experimental_analyzer = 1; SELECT * FROM @@ -75,7 +114,20 @@ FROM SELECT * FROM ngrambf_tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id ) WHERE - explain LIKE '%Granules: %'; + explain LIKE '%Granules: %' +SETTINGS + allow_experimental_analyzer = 0; + +SELECT * +FROM +( + EXPLAIN PLAN indexes = 1 + SELECT * FROM ngrambf_tab WHERE match(str, '.*(ClickHouse|World)') ORDER BY id +) +WHERE + explain LIKE '%Granules: %' +SETTINGS + allow_experimental_analyzer = 1; SELECT '---'; @@ -93,7 +145,19 @@ FROM SELECT * FROM tokenbf_tab WHERE match(str, 'OLAP (.*?)*') ORDER BY id ) WHERE - explain LIKE '%Granules: %'; + explain LIKE '%Granules: %' +SETTINGS + allow_experimental_analyzer = 0; +SELECT * +FROM +( + EXPLAIN PLAN indexes = 1 + SELECT * FROM tokenbf_tab WHERE match(str, 'OLAP (.*?)*') ORDER BY id +) +WHERE + explain LIKE '%Granules: %' +SETTINGS + allow_experimental_analyzer = 1; SELECT * FROM @@ -102,7 +166,20 @@ FROM SELECT * FROM ngrambf_tab WHERE match(str, 'OLAP (.*?)*') ORDER BY id ) WHERE - explain LIKE '%Granules: %'; + explain LIKE '%Granules: %' +SETTINGS + allow_experimental_analyzer = 0; + +SELECT * +FROM +( + EXPLAIN PLAN indexes = 1 + SELECT * FROM ngrambf_tab WHERE match(str, 'OLAP (.*?)*') ORDER BY id +) +WHERE + explain LIKE '%Granules: %' +SETTINGS + allow_experimental_analyzer = 1; DROP TABLE tokenbf_tab; DROP TABLE ngrambf_tab; From b23ba02d4c75eebdabfd90d03d44218420f93582 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 9 Feb 2024 16:50:37 +0100 Subject: [PATCH 204/276] Enforce tests with enabled analyzer in CI --- tests/ci/ci_config.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 998d0876527..8ab6ceebb42 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -1130,6 +1130,8 @@ REQUIRED_CHECKS = [ JobNames.UNIT_TEST, JobNames.UNIT_TEST_TSAN, JobNames.UNIT_TEST_UBSAN, + JobNames.INTEGRATION_TEST_ASAN_ANALYZER, + JobNames.STATELESS_TEST_ANALYZER_RELEASE, ] From f056e8b2c400fd676a37ab59727ebaf6d929e27c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 9 Feb 2024 16:54:00 +0100 Subject: [PATCH 205/276] Handle different timestamp related aspects of zip-files --- .../build_and_deploy_archive.sh | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/tests/ci/team_keys_lambda/build_and_deploy_archive.sh b/tests/ci/team_keys_lambda/build_and_deploy_archive.sh index 5b377d3c184..3c6c8e0ac1e 100644 --- a/tests/ci/team_keys_lambda/build_and_deploy_archive.sh +++ b/tests/ci/team_keys_lambda/build_and_deploy_archive.sh @@ -22,18 +22,29 @@ mkdir "$PACKAGE" cp app.py "$PACKAGE" if [ -f requirements.txt ]; then VENV=lambda-venv - rm -rf "$VENV" lambda-package.zip + rm -rf "$VENV" docker run --net=host --rm --user="${UID}" -e HOME=/tmp --entrypoint=/bin/bash \ --volume="${WORKDIR}/..:/ci" --workdir="/ci/${DIR_NAME}" "${DOCKER_IMAGE}" \ -exc " '$PY_EXEC' -m venv '$VENV' && source '$VENV/bin/activate' && - pip install -r requirements.txt + pip install -r requirements.txt && + # To have consistent pyc files + find '$VENV/lib' -name '*.pyc' -delete + find '$VENV/lib' ! -type d -exec touch -t 201212121212 {} + + python -m compileall " cp -rT "$VENV/lib/$PY_EXEC/site-packages/" "$PACKAGE" rm -r "$PACKAGE"/{pip,pip-*,setuptools,setuptools-*} + # zip stores metadata about timestamps + find "$PACKAGE" ! -type d -exec touch -t 201212121212 {} + fi -( cd "$PACKAGE" && zip -9 -r ../"$PACKAGE".zip . ) +( + export LC_ALL=c + cd "$PACKAGE" + # zip uses random files order by default, so we sort the files alphabetically + find . ! -type d -print0 | sort -z | tr '\0' '\n' | zip -XD -0 ../"$PACKAGE".zip --names-stdin +) ECHO=() if [ -n "$DRY_RUN" ]; then From ca2f51493155f5ea2663880d209c1b0a13e4cc94 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 9 Feb 2024 17:36:44 +0100 Subject: [PATCH 206/276] Fix msan unpoison --- src/Common/MemorySanitizer.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/MemorySanitizer.h b/src/Common/MemorySanitizer.h index 02747d1e130..bd44ff62acb 100644 --- a/src/Common/MemorySanitizer.h +++ b/src/Common/MemorySanitizer.h @@ -13,7 +13,7 @@ #undef __msan_unpoison_string #define __msan_unpoison(X, Y) /// NOLINT -/// Given a pointer and **its size**, unpoisons up to 15 bytes **at the end** +/// Given a pointer and **its size**, unpoisons 15 bytes **at the end** /// See memcmpSmall.h / memcpySmall.h #define __msan_unpoison_overflow_15(X, Y) /// NOLINT #define __msan_test_shadow(X, Y) (false) /// NOLINT @@ -28,7 +28,7 @@ # undef __msan_unpoison_string # include # undef __msan_unpoison_overflow_15 -# define __msan_unpoison_overflow_15(PTR, PTR_SIZE) __msan_unpoison(&(PTR)[(PTR_SIZE)], ((16 - ((PTR_SIZE) % 16)) % 16)) +# define __msan_unpoison_overflow_15(PTR, PTR_SIZE) __msan_unpoison(&(PTR)[(PTR_SIZE)], 15) # endif #endif From 16f3dbea713840ef9c79faa358330986acdc81a7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 9 Feb 2024 16:43:48 +0000 Subject: [PATCH 207/276] Update optimizePrewjere --- .../QueryPlan/Optimizations/optimizePrewhere.cpp | 4 ++-- .../02149_read_in_order_fixed_prefix.reference | 11 ++++------- 2 files changed, 6 insertions(+), 9 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index 2f790d9892f..ee5ad8d1a8a 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -139,8 +139,8 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) const auto & context = read_from_merge_tree->getContext(); const auto & settings = context->getSettingsRef(); - // if (!settings.allow_experimental_analyzer) - // return; + if (!settings.allow_experimental_analyzer) + return; bool is_final = read_from_merge_tree->isQueryWithFinal(); bool optimize_move_to_prewhere = settings.optimize_move_to_prewhere && (!is_final || settings.optimize_move_to_prewhere_if_final); diff --git a/tests/queries/0_stateless/02149_read_in_order_fixed_prefix.reference b/tests/queries/0_stateless/02149_read_in_order_fixed_prefix.reference index f2a4ef1f634..f3415a34823 100644 --- a/tests/queries/0_stateless/02149_read_in_order_fixed_prefix.reference +++ b/tests/queries/0_stateless/02149_read_in_order_fixed_prefix.reference @@ -76,8 +76,7 @@ ExpressionTransform (Expression) ExpressionTransform (ReadFromMergeTree) - ExpressionTransform - MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1 + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1 2020-10-11 0 0 2020-10-11 0 10 2020-10-11 0 20 @@ -106,8 +105,7 @@ ExpressionTransform (Expression) ExpressionTransform (ReadFromMergeTree) - ExpressionTransform - MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1 + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1 2020-10-12 0 2020-10-12 1 2020-10-12 2 @@ -140,9 +138,8 @@ ExpressionTransform (Expression) ExpressionTransform (ReadFromMergeTree) - ExpressionTransform - ReverseTransform - MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InReverseOrder) 0 → 1 + ReverseTransform + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InReverseOrder) 0 → 1 2020-10-12 99999 2020-10-12 99998 2020-10-12 99997 From 67d5e5885eb330bb41c502695671215fc16b791e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Feb 2024 18:32:42 +0100 Subject: [PATCH 208/276] Update tests/queries/0_stateless/02982_comments_in_system_tables.sh Co-authored-by: Mikhail f. Shiryaev --- tests/queries/0_stateless/02982_comments_in_system_tables.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02982_comments_in_system_tables.sh b/tests/queries/0_stateless/02982_comments_in_system_tables.sh index 0825b1be2d6..2d7fbf4d35a 100755 --- a/tests/queries/0_stateless/02982_comments_in_system_tables.sh +++ b/tests/queries/0_stateless/02982_comments_in_system_tables.sh @@ -4,5 +4,5 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_LOCAL} --query "SELECT 'Table ' || database || '.' || name || ' doesn\'t have a comment' FROM system.tables WHERE name NOT LIKE '%\_log\_%' AND database='system' AND comment==''" -${CLICKHOUSE_CLIENT} --query "SELECT 'Table ' || database || '.' || name || ' doesn\'t have a comment' FROM system.tables WHERE name NOT LIKE '%\_log\_%' AND database='system' AND comment==''" +${CLICKHOUSE_LOCAL} --query "SELECT 'Table ' || database || '.' || name || ' does not have a comment' FROM system.tables WHERE name NOT LIKE '%\_log\_%' AND database='system' AND comment==''" +${CLICKHOUSE_CLIENT} --query "SELECT 'Table ' || database || '.' || name || ' does not have a comment' FROM system.tables WHERE name NOT LIKE '%\_log\_%' AND database='system' AND comment==''" From e54fbd3367b0fef10a82ec3b133129274b87c8d5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 9 Feb 2024 18:32:54 +0100 Subject: [PATCH 209/276] Update tests/ci/ci_config.py Co-authored-by: Mikhail f. Shiryaev --- tests/ci/ci_config.py | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 817018f044c..4a62d686e35 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -202,13 +202,6 @@ class JobConfig: release_only: bool = False -# About the "sparse_checkout" option: -# -# It's needed only to test the option itself. -# No particular sense to use it in the build, -# and it is even slows down the job. - - @dataclass class BuildConfig: name: str @@ -219,6 +212,8 @@ class BuildConfig: coverage: bool = False sanitizer: str = "" tidy: bool = False + # sparse_checkout is needed only to test the option itself. + # No particular sense to use it in every build, since it slows down the job. sparse_checkout: bool = False comment: str = "" static_binary_name: str = "" From a76e07207aa7645316fdba83ae3db5398b650c1a Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 9 Feb 2024 18:58:15 +0100 Subject: [PATCH 210/276] Add new setting azure_max_single_part_copy_size to SettingsChangesHistory.h --- src/Core/SettingsChangesHistory.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 5bd38d600c1..c453dd837eb 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -92,7 +92,8 @@ static std::map sett {"async_insert_busy_timeout_increase_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout increases"}, {"async_insert_busy_timeout_decrease_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout decreases"}, {"split_parts_ranges_into_intersecting_and_non_intersecting_final", true, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}}}, + {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}, + {"azure_max_single_part_copy_size", 256*1024*1024, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage."}}}, {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, {"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"}, From 1d0a86ccdfb94863b09125a554a329b971b16587 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 9 Feb 2024 18:31:51 +0000 Subject: [PATCH 211/276] Try to fix tests. --- .../QueryPlan/Optimizations/optimizeTree.cpp | 9 ++- .../01763_filter_push_down_bugs.reference | 2 +- .../01786_explain_merge_tree.reference | 12 --- .../0_stateless/01786_explain_merge_tree.sh | 4 +- ...771_ignore_data_skipping_indices.reference | 74 +++++++++---------- .../02771_ignore_data_skipping_indices.sql | 2 - ...ndex_in_function_different_types.reference | 44 +++++++++++ ..._key_index_in_function_different_types.sql | 9 +++ 8 files changed, 99 insertions(+), 57 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index 55f7e7cb85b..c64bc308246 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -120,6 +120,12 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s auto & frame = stack.back(); + if (frame.next_child == 0) + { + if (optimization_settings.distinct_in_order) + tryDistinctReadInOrder(frame.node); + } + /// Traverse all children first. if (frame.next_child < frame.node->children.size()) { @@ -154,9 +160,6 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s if (optimization_settings.aggregation_in_order) optimizeAggregationInOrder(*frame.node, nodes); - - if (optimization_settings.distinct_in_order) - tryDistinctReadInOrder(frame.node); } /// Traverse all children first. diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference index c8045dd26f5..80bd7dfd8c0 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference @@ -28,7 +28,7 @@ Expression ((Projection + Before ORDER BY)) Expression ((Project names + Projection)) Filter ((WHERE + DROP unused columns after JOIN)) Join (JOIN FillRightFirst) - Expression (Change column names to column identifiers) + Expression ReadFromMergeTree (default.t1) Indexes: PrimaryKey diff --git a/tests/queries/0_stateless/01786_explain_merge_tree.reference b/tests/queries/0_stateless/01786_explain_merge_tree.reference index 096090f8fa1..fd1bc713b08 100644 --- a/tests/queries/0_stateless/01786_explain_merge_tree.reference +++ b/tests/queries/0_stateless/01786_explain_merge_tree.reference @@ -3,21 +3,18 @@ MinMax Keys: y - Condition: (y in [1, +Inf)) Parts: 4/5 Granules: 11/12 Partition Keys: y bitAnd(z, 3) - Condition: and((y in [1, +Inf)), (bitAnd(z, 3) not in [1, 1])) Parts: 3/4 Granules: 10/11 PrimaryKey Keys: x y - Condition: and((x in [11, +Inf)), (y in [1, +Inf))) Parts: 2/3 Granules: 6/10 Skip @@ -37,7 +34,6 @@ { "Type": "MinMax", "Keys": ["y"], - "Condition": "(y in [1, +Inf))", "Initial Parts": 5, "Selected Parts": 4, "Initial Granules": 12, @@ -46,7 +42,6 @@ { "Type": "Partition", "Keys": ["y", "bitAnd(z, 3)"], - "Condition": "and((y in [1, +Inf)), (bitAnd(z, 3) not in [1, 1]))", "Initial Parts": 4, "Selected Parts": 3, "Initial Granules": 11, @@ -55,7 +50,6 @@ { "Type": "PrimaryKey", "Keys": ["x", "y"], - "Condition": "and((x in [11, +Inf)), (y in [1, +Inf)))", "Initial Parts": 3, "Selected Parts": 2, "Initial Granules": 10, @@ -109,21 +103,18 @@ MinMax Keys: y - Condition: (y in [1, +Inf)) Parts: 4/5 Granules: 11/12 Partition Keys: y bitAnd(z, 3) - Condition: and((y in [1, +Inf)), (bitAnd(z, 3) not in [1, 1])) Parts: 3/4 Granules: 10/11 PrimaryKey Keys: x y - Condition: and((x in [11, +Inf)), (y in [1, +Inf))) Parts: 2/3 Granules: 6/10 Skip @@ -138,7 +129,6 @@ { "Type": "MinMax", "Keys": ["y"], - "Condition": "(y in [1, +Inf))", "Initial Parts": 5, "Selected Parts": 4, "Initial Granules": 12, @@ -147,7 +137,6 @@ { "Type": "Partition", "Keys": ["y", "bitAnd(z, 3)"], - "Condition": "and((y in [1, +Inf)), (bitAnd(z, 3) not in [1, 1]))", "Initial Parts": 4, "Selected Parts": 3, "Initial Granules": 11, @@ -156,7 +145,6 @@ { "Type": "PrimaryKey", "Keys": ["x", "y"], - "Condition": "and((x in [11, +Inf)), (y in [1, +Inf)))", "Initial Parts": 3, "Selected Parts": 2, "Initial Granules": 10, diff --git a/tests/queries/0_stateless/01786_explain_merge_tree.sh b/tests/queries/0_stateless/01786_explain_merge_tree.sh index 23537013204..e3b28acdc41 100755 --- a/tests/queries/0_stateless/01786_explain_merge_tree.sh +++ b/tests/queries/0_stateless/01786_explain_merge_tree.sh @@ -17,13 +17,13 @@ do $CH_CLIENT -q " explain indexes = 1 select *, _part from test_index where t % 19 = 16 and y > 0 and bitAnd(z, 3) != 1 and x > 10 and t % 20 > 14; - " | grep -A 100 "ReadFromMergeTree" # | grep -v "Description" + " | grep -A 100 "ReadFromMergeTree" | grep -v "Condition" echo "-----------------" $CH_CLIENT -q " explain indexes = 1, json = 1 select *, _part from test_index where t % 19 = 16 and y > 0 and bitAnd(z, 3) != 1 and x > 10 and t % 20 > 14 format TSVRaw; - " | grep -A 100 "ReadFromMergeTree" # | grep -v "Description" + " | grep -A 100 "ReadFromMergeTree" | grep -v "Condition" echo "-----------------" diff --git a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference index e23e3094ca3..33df18c8801 100644 --- a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference +++ b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.reference @@ -1,43 +1,43 @@ 1 2 3 1 2 3 1 2 3 - ReadFromMergeTree (default.data_02771) - Indexes: - PrimaryKey - Condition: true - Parts: 1/1 - Granules: 1/1 - Skip - Name: x_idx - Description: minmax GRANULARITY 1 - Parts: 0/1 - Granules: 0/1 - Skip - Name: y_idx - Description: minmax GRANULARITY 1 - Parts: 0/0 - Granules: 0/0 - Skip - Name: xy_idx - Description: minmax GRANULARITY 1 - Parts: 0/0 - Granules: 0/0 - ReadFromMergeTree (default.data_02771) - Indexes: - PrimaryKey - Condition: true - Parts: 1/1 - Granules: 1/1 - Skip - Name: x_idx - Description: minmax GRANULARITY 1 - Parts: 0/1 - Granules: 0/1 - Skip - Name: y_idx - Description: minmax GRANULARITY 1 - Parts: 0/0 - Granules: 0/0 + ReadFromMergeTree (default.data_02771) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 + Skip + Name: x_idx + Description: minmax GRANULARITY 1 + Parts: 0/1 + Granules: 0/1 + Skip + Name: y_idx + Description: minmax GRANULARITY 1 + Parts: 0/0 + Granules: 0/0 + Skip + Name: xy_idx + Description: minmax GRANULARITY 1 + Parts: 0/0 + Granules: 0/0 + ReadFromMergeTree (default.data_02771) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 1/1 + Skip + Name: x_idx + Description: minmax GRANULARITY 1 + Parts: 0/1 + Granules: 0/1 + Skip + Name: y_idx + Description: minmax GRANULARITY 1 + Parts: 0/0 + Granules: 0/0 ReadFromMergeTree (default.data_02771) Indexes: PrimaryKey diff --git a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql index 716421b7342..951d87fd2c0 100644 --- a/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql +++ b/tests/queries/0_stateless/02771_ignore_data_skipping_indices.sql @@ -1,5 +1,3 @@ -SET allow_experimental_analyzer = 0; - DROP TABLE IF EXISTS data_02771; diff --git a/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.reference b/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.reference index 6338d048186..7a5e798359b 100644 --- a/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.reference +++ b/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.reference @@ -1,3 +1,47 @@ +CreatingSets + Expression + ReadFromMergeTree + Indexes: + PrimaryKey + Keys: + id + value + Condition: and((id in (-Inf, 10]), (value in 1-element set)) + Parts: 1/1 + Granules: 1/1 +CreatingSets + Expression + ReadFromMergeTree + Indexes: + PrimaryKey + Keys: + id + value + Condition: and((id in (-Inf, 10]), (value in 1-element set)) + Parts: 1/1 + Granules: 1/1 +CreatingSets + Expression + ReadFromMergeTree + Indexes: + PrimaryKey + Keys: + id + value + Condition: and((id in (-Inf, 10]), (value in 5-element set)) + Parts: 1/1 + Granules: 1/1 +CreatingSets + Expression + ReadFromMergeTree + Indexes: + PrimaryKey + Keys: + id + value + Condition: and((id in (-Inf, 10]), (value in 5-element set)) + Parts: 1/1 + Granules: 1/1 CreatingSets Expression Expression diff --git a/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.sql b/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.sql index 585c2635970..1b1a7607344 100644 --- a/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.sql +++ b/tests/queries/0_stateless/02882_primary_key_index_in_function_different_types.sql @@ -7,6 +7,15 @@ CREATE TABLE test_table INSERT INTO test_table SELECT number, number FROM numbers(10); +set allow_experimental_analyzer = 0; + +EXPLAIN indexes = 1, description=0 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT 5); +EXPLAIN indexes = 1, description=0 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT '5'); +EXPLAIN indexes = 1, description=0 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT toUInt8(number) FROM numbers(5)); +EXPLAIN indexes = 1, description=0 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT toString(number) FROM numbers(5)); + +set allow_experimental_analyzer = 1; + EXPLAIN indexes = 1, description=0 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT 5); EXPLAIN indexes = 1, description=0 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT '5'); EXPLAIN indexes = 1, description=0 SELECT id FROM test_table WHERE id <= 10 AND value IN (SELECT toUInt8(number) FROM numbers(5)); From 69e118e58734aa822f86d33c3596310509af3c42 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 9 Feb 2024 19:34:21 +0100 Subject: [PATCH 212/276] Fix potential endless loop during merge (#59812) --- src/Processors/Merges/Algorithms/MergedData.h | 4 ++-- src/Processors/Transforms/ColumnGathererTransform.h | 12 ++++++++---- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/src/Processors/Merges/Algorithms/MergedData.h b/src/Processors/Merges/Algorithms/MergedData.h index f92d20d22e1..7ffde835ad0 100644 --- a/src/Processors/Merges/Algorithms/MergedData.h +++ b/src/Processors/Merges/Algorithms/MergedData.h @@ -100,7 +100,7 @@ public: merged_rows = 0; sum_blocks_granularity = 0; ++total_chunks; - total_allocated_bytes += chunk.allocatedBytes(); + total_allocated_bytes += chunk.bytes(); need_flush = false; return chunk; @@ -122,7 +122,7 @@ public: { size_t merged_bytes = 0; for (const auto & column : columns) - merged_bytes += column->allocatedBytes(); + merged_bytes += column->byteSize(); if (merged_bytes >= max_block_size_bytes) return true; } diff --git a/src/Processors/Transforms/ColumnGathererTransform.h b/src/Processors/Transforms/ColumnGathererTransform.h index 821d04db0df..4e56cffa46a 100644 --- a/src/Processors/Transforms/ColumnGathererTransform.h +++ b/src/Processors/Transforms/ColumnGathererTransform.h @@ -145,10 +145,14 @@ void ColumnGathererStream::gather(Column & column_res) next_required_source = -1; - while (row_source_pos < row_sources_end - && column_res.size() < block_preferred_size_rows - && column_res.allocatedBytes() < block_preferred_size_bytes) + + /// We use do ... while here to ensure there will be at least one iteration of this loop. + /// Because the column_res.byteSize() could be bigger than block_preferred_size_bytes already at this point. + do { + if (row_source_pos >= row_sources_end) + break; + RowSourcePart row_source = *row_source_pos; size_t source_num = row_source.getSourceNum(); Source & source = sources[source_num]; @@ -191,7 +195,7 @@ void ColumnGathererStream::gather(Column & column_res) } source.pos += len; - } + } while (column_res.size() < block_preferred_size_rows && column_res.byteSize() < block_preferred_size_bytes); } } From dc52def9ebfcd10ac826108ac846d3c4d00792e0 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 5 Feb 2024 12:33:17 +0300 Subject: [PATCH 213/276] Revert "Revert "Poco Logger small refactoring"" --- base/poco/Foundation/CMakeLists.txt | 6 + base/poco/Foundation/include/Poco/Logger.h | 42 ++-- .../include/Poco/RefCountedObject.h | 3 +- base/poco/Foundation/src/Logger.cpp | 226 +++++++++++------- src/Common/Logger.h | 12 + src/Common/tests/gtest_log.cpp | 73 ++++++ .../ObjectStorages/ObjectStorageFactory.cpp | 2 +- .../ObjectStorages/Web/WebObjectStorage.cpp | 2 +- utils/keeper-data-dumper/main.cpp | 2 +- 9 files changed, 255 insertions(+), 113 deletions(-) diff --git a/base/poco/Foundation/CMakeLists.txt b/base/poco/Foundation/CMakeLists.txt index dfb41a33fb1..5fe644d3057 100644 --- a/base/poco/Foundation/CMakeLists.txt +++ b/base/poco/Foundation/CMakeLists.txt @@ -166,6 +166,12 @@ set (SRCS ) add_library (_poco_foundation ${SRCS}) +target_link_libraries (_poco_foundation + PUBLIC + boost::headers_only + boost::system +) + add_library (Poco::Foundation ALIAS _poco_foundation) # TODO: remove these warning exclusions diff --git a/base/poco/Foundation/include/Poco/Logger.h b/base/poco/Foundation/include/Poco/Logger.h index cf202718662..883294a071a 100644 --- a/base/poco/Foundation/include/Poco/Logger.h +++ b/base/poco/Foundation/include/Poco/Logger.h @@ -22,6 +22,9 @@ #include #include #include + +#include + #include "Poco/Channel.h" #include "Poco/Format.h" #include "Poco/Foundation.h" @@ -34,7 +37,7 @@ namespace Poco class Exception; class Logger; -using LoggerPtr = std::shared_ptr; +using LoggerPtr = boost::intrusive_ptr; class Foundation_API Logger : public Channel /// Logger is a special Channel that acts as the main @@ -871,21 +874,11 @@ public: /// If the Logger does not yet exist, it is created, based /// on its parent logger. - static LoggerPtr getShared(const std::string & name); + static LoggerPtr getShared(const std::string & name, bool should_be_owned_by_shared_ptr_if_created = true); /// Returns a shared pointer to the Logger with the given name. /// If the Logger does not yet exist, it is created, based /// on its parent logger. - static Logger & unsafeGet(const std::string & name); - /// Returns a reference to the Logger with the given name. - /// If the Logger does not yet exist, it is created, based - /// on its parent logger. - /// - /// WARNING: This method is not thread safe. You should - /// probably use get() instead. - /// The only time this method should be used is during - /// program initialization, when only one thread is running. - static Logger & create(const std::string & name, Channel * pChannel, int level = Message::PRIO_INFORMATION); /// Creates and returns a reference to a Logger with the /// given name. The Logger's Channel and log level as set as @@ -932,6 +925,16 @@ public: static const std::string ROOT; /// The name of the root logger (""). +public: + struct LoggerEntry + { + Poco::Logger * logger; + bool owned_by_shared_ptr = false; + }; + + using LoggerMap = std::unordered_map; + using LoggerMapIterator = LoggerMap::iterator; + protected: Logger(const std::string & name, Channel * pChannel, int level); ~Logger(); @@ -940,12 +943,19 @@ protected: void log(const std::string & text, Message::Priority prio, const char * file, int line); static std::string format(const std::string & fmt, int argc, std::string argv[]); - static Logger & unsafeCreate(const std::string & name, Channel * pChannel, int level = Message::PRIO_INFORMATION); - static Logger & parent(const std::string & name); - static void add(Logger * pLogger); - static Logger * find(const std::string & name); private: + static std::pair unsafeGet(const std::string & name, bool get_shared); + static Logger * unsafeGetRawPtr(const std::string & name); + static std::pair unsafeCreate(const std::string & name, Channel * pChannel, int level = Message::PRIO_INFORMATION); + static Logger & parent(const std::string & name); + static std::pair add(Logger * pLogger); + static std::optional find(const std::string & name); + static Logger * findRawPtr(const std::string & name); + + friend void intrusive_ptr_add_ref(Logger * ptr); + friend void intrusive_ptr_release(Logger * ptr); + Logger(); Logger(const Logger &); Logger & operator=(const Logger &); diff --git a/base/poco/Foundation/include/Poco/RefCountedObject.h b/base/poco/Foundation/include/Poco/RefCountedObject.h index db966089e00..1f806bdacb1 100644 --- a/base/poco/Foundation/include/Poco/RefCountedObject.h +++ b/base/poco/Foundation/include/Poco/RefCountedObject.h @@ -53,11 +53,10 @@ protected: virtual ~RefCountedObject(); /// Destroys the RefCountedObject. + mutable std::atomic _counter; private: RefCountedObject(const RefCountedObject &); RefCountedObject & operator=(const RefCountedObject &); - - mutable std::atomic _counter; }; diff --git a/base/poco/Foundation/src/Logger.cpp b/base/poco/Foundation/src/Logger.cpp index cfc063c8979..16fc3a0480e 100644 --- a/base/poco/Foundation/src/Logger.cpp +++ b/base/poco/Foundation/src/Logger.cpp @@ -38,14 +38,7 @@ std::mutex & getLoggerMutex() return *logger_mutex; } -struct LoggerEntry -{ - Poco::Logger * logger; - bool owned_by_shared_ptr = false; -}; - -using LoggerMap = std::unordered_map; -LoggerMap * _pLoggerMap = nullptr; +Poco::Logger::LoggerMap * _pLoggerMap = nullptr; } @@ -309,38 +302,9 @@ void Logger::formatDump(std::string& message, const void* buffer, std::size_t le namespace { -struct LoggerDeleter -{ - void operator()(Poco::Logger * logger) - { - std::lock_guard lock(getLoggerMutex()); - - /// If logger infrastructure is destroyed just decrement logger reference count - if (!_pLoggerMap) - { - logger->release(); - return; - } - - auto it = _pLoggerMap->find(logger->name()); - assert(it != _pLoggerMap->end()); - - /** If reference count is 1, this means this shared pointer owns logger - * and need destroy it. - */ - size_t reference_count_before_release = logger->release(); - if (reference_count_before_release == 1) - { - assert(it->second.owned_by_shared_ptr); - _pLoggerMap->erase(it); - } - } -}; - - inline LoggerPtr makeLoggerPtr(Logger & logger) { - return std::shared_ptr(&logger, LoggerDeleter()); + return LoggerPtr(&logger, false /*add_ref*/); } } @@ -350,64 +314,87 @@ Logger& Logger::get(const std::string& name) { std::lock_guard lock(getLoggerMutex()); - Logger & logger = unsafeGet(name); - - /** If there are already shared pointer created for this logger - * we need to increment Logger reference count and now logger - * is owned by logger infrastructure. - */ - auto it = _pLoggerMap->find(name); - if (it->second.owned_by_shared_ptr) - { - it->second.logger->duplicate(); - it->second.owned_by_shared_ptr = false; - } - - return logger; + auto [it, inserted] = unsafeGet(name, false /*get_shared*/); + return *it->second.logger; } -LoggerPtr Logger::getShared(const std::string & name) +LoggerPtr Logger::getShared(const std::string & name, bool should_be_owned_by_shared_ptr_if_created) { std::lock_guard lock(getLoggerMutex()); - bool logger_exists = _pLoggerMap && _pLoggerMap->contains(name); + auto [it, inserted] = unsafeGet(name, true /*get_shared*/); - Logger & logger = unsafeGet(name); - - /** If logger already exists, then this shared pointer does not own it. - * If logger does not exists, logger infrastructure could be already destroyed - * or logger was created. + /** If during `unsafeGet` logger was created, then this shared pointer owns it. + * If logger was already created, then this shared pointer does not own it. */ - if (logger_exists) + if (inserted) { - logger.duplicate(); - } - else if (_pLoggerMap) - { - _pLoggerMap->find(name)->second.owned_by_shared_ptr = true; + if (should_be_owned_by_shared_ptr_if_created) + it->second.owned_by_shared_ptr = true; + else + it->second.logger->duplicate(); } - return makeLoggerPtr(logger); + return makeLoggerPtr(*it->second.logger); } -Logger& Logger::unsafeGet(const std::string& name) +std::pair Logger::unsafeGet(const std::string& name, bool get_shared) { - Logger* pLogger = find(name); - if (!pLogger) + std::optional optional_logger_it = find(name); + + bool should_recreate_logger = false; + + if (optional_logger_it) { + auto & logger_it = *optional_logger_it; + std::optional reference_count_before; + + if (get_shared) + { + reference_count_before = logger_it->second.logger->duplicate(); + } + else if (logger_it->second.owned_by_shared_ptr) + { + reference_count_before = logger_it->second.logger->duplicate(); + logger_it->second.owned_by_shared_ptr = false; + } + + /// Other thread already decided to delete this logger, but did not yet remove it from map + if (reference_count_before && reference_count_before == 0) + should_recreate_logger = true; + } + + if (!optional_logger_it || should_recreate_logger) + { + Logger * logger = nullptr; + if (name == ROOT) { - pLogger = new Logger(name, 0, Message::PRIO_INFORMATION); + logger = new Logger(name, nullptr, Message::PRIO_INFORMATION); } else { Logger& par = parent(name); - pLogger = new Logger(name, par.getChannel(), par.getLevel()); + logger = new Logger(name, par.getChannel(), par.getLevel()); } - add(pLogger); + + if (should_recreate_logger) + { + (*optional_logger_it)->second.logger = logger; + return std::make_pair(*optional_logger_it, true); + } + + return add(logger); } - return *pLogger; + + return std::make_pair(*optional_logger_it, false); +} + + +Logger * Logger::unsafeGetRawPtr(const std::string & name) +{ + return unsafeGet(name, false /*get_shared*/).first->second.logger; } @@ -415,24 +402,24 @@ Logger& Logger::create(const std::string& name, Channel* pChannel, int level) { std::lock_guard lock(getLoggerMutex()); - return unsafeCreate(name, pChannel, level); + return *unsafeCreate(name, pChannel, level).first->second.logger; } LoggerPtr Logger::createShared(const std::string & name, Channel * pChannel, int level) { std::lock_guard lock(getLoggerMutex()); - Logger & logger = unsafeCreate(name, pChannel, level); - _pLoggerMap->find(name)->second.owned_by_shared_ptr = true; + auto [it, inserted] = unsafeCreate(name, pChannel, level); + it->second.owned_by_shared_ptr = true; - return makeLoggerPtr(logger); + return makeLoggerPtr(*it->second.logger); } Logger& Logger::root() { std::lock_guard lock(getLoggerMutex()); - return unsafeGet(ROOT); + return *unsafeGetRawPtr(ROOT); } @@ -440,7 +427,11 @@ Logger* Logger::has(const std::string& name) { std::lock_guard lock(getLoggerMutex()); - return find(name); + auto optional_it = find(name); + if (!optional_it) + return nullptr; + + return (*optional_it)->second.logger; } @@ -459,20 +450,69 @@ void Logger::shutdown() } delete _pLoggerMap; - _pLoggerMap = 0; + _pLoggerMap = nullptr; } } -Logger* Logger::find(const std::string& name) +std::optional Logger::find(const std::string& name) { if (_pLoggerMap) { LoggerMap::iterator it = _pLoggerMap->find(name); if (it != _pLoggerMap->end()) - return it->second.logger; + return it; + + return {}; } - return 0; + + return {}; +} + +Logger * Logger::findRawPtr(const std::string & name) +{ + auto optional_it = find(name); + if (!optional_it) + return nullptr; + + return (*optional_it)->second.logger; +} + + +void intrusive_ptr_add_ref(Logger * ptr) +{ + ptr->duplicate(); +} + + +void intrusive_ptr_release(Logger * ptr) +{ + size_t reference_count_before = ptr->_counter.fetch_sub(1, std::memory_order_acq_rel); + if (reference_count_before != 1) + return; + + { + std::lock_guard lock(getLoggerMutex()); + + if (_pLoggerMap) + { + auto it = _pLoggerMap->find(ptr->name()); + + /** It is possible that during release other thread created logger and + * updated iterator in map. + */ + if (it != _pLoggerMap->end() && ptr == it->second.logger) + { + /** If reference count is 0, this means this intrusive pointer owns logger + * and need destroy it. + */ + assert(it->second.owned_by_shared_ptr); + _pLoggerMap->erase(it); + } + } + } + + delete ptr; } @@ -490,28 +530,28 @@ void Logger::names(std::vector& names) } } -Logger& Logger::unsafeCreate(const std::string & name, Channel * pChannel, int level) + +std::pair Logger::unsafeCreate(const std::string & name, Channel * pChannel, int level) { if (find(name)) throw ExistsException(); Logger* pLogger = new Logger(name, pChannel, level); - add(pLogger); - - return *pLogger; + return add(pLogger); } + Logger& Logger::parent(const std::string& name) { std::string::size_type pos = name.rfind('.'); if (pos != std::string::npos) { std::string pname = name.substr(0, pos); - Logger* pParent = find(pname); + Logger* pParent = findRawPtr(pname); if (pParent) return *pParent; else return parent(pname); } - else return unsafeGet(ROOT); + else return *unsafeGetRawPtr(ROOT); } @@ -579,12 +619,14 @@ namespace } -void Logger::add(Logger* pLogger) +std::pair Logger::add(Logger* pLogger) { if (!_pLoggerMap) - _pLoggerMap = new LoggerMap; + _pLoggerMap = new Logger::LoggerMap; - _pLoggerMap->emplace(pLogger->name(), LoggerEntry{pLogger, false /*owned_by_shared_ptr*/}); + auto result = _pLoggerMap->emplace(pLogger->name(), LoggerEntry{pLogger, false /*owned_by_shared_ptr*/}); + assert(result.second); + return result; } diff --git a/src/Common/Logger.h b/src/Common/Logger.h index 6dcdea9a9d8..0425da8c847 100644 --- a/src/Common/Logger.h +++ b/src/Common/Logger.h @@ -2,6 +2,8 @@ #include +#include + #include #include #include @@ -24,6 +26,16 @@ using LoggerRawPtr = Poco::Logger *; */ LoggerPtr getLogger(const std::string & name); +/** Get Logger with specified name. If the Logger does not exists, it is created. + * This overload was added for specific purpose, when logger is constructed from constexpr string. + * Logger is destroyed only during program shutdown. + */ +template +ALWAYS_INLINE LoggerPtr getLogger(const char (&name)[n]) +{ + return Poco::Logger::getShared(name, false /*should_be_owned_by_shared_ptr_if_created*/); +} + /** Create Logger with specified name, channel and logging level. * If Logger already exists, throws exception. * Logger is destroyed, when last shared ptr that refers to Logger with specified name is destroyed. diff --git a/src/Common/tests/gtest_log.cpp b/src/Common/tests/gtest_log.cpp index 622497fe2f5..6d2bd56ad77 100644 --- a/src/Common/tests/gtest_log.cpp +++ b/src/Common/tests/gtest_log.cpp @@ -9,6 +9,7 @@ #include #include #include +#include TEST(Logger, Log) @@ -100,3 +101,75 @@ TEST(Logger, SideEffects) LOG_TRACE(log, "test no throw {}", getLogMessageParamOrThrow()); } + +TEST(Logger, SharedRawLogger) +{ + { + std::ostringstream stream; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + auto stream_channel = Poco::AutoPtr(new Poco::StreamChannel(stream)); + + auto shared_logger = getLogger("Logger_1"); + shared_logger->setChannel(stream_channel.get()); + shared_logger->setLevel("trace"); + + LOG_TRACE(shared_logger, "SharedLogger1Log1"); + LOG_TRACE(getRawLogger("Logger_1"), "RawLogger1Log"); + LOG_TRACE(shared_logger, "SharedLogger1Log2"); + + auto actual = stream.str(); + EXPECT_EQ(actual, "SharedLogger1Log1\nRawLogger1Log\nSharedLogger1Log2\n"); + } + { + std::ostringstream stream; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + auto stream_channel = Poco::AutoPtr(new Poco::StreamChannel(stream)); + + auto * raw_logger = getRawLogger("Logger_2"); + raw_logger->setChannel(stream_channel.get()); + raw_logger->setLevel("trace"); + + LOG_TRACE(getLogger("Logger_2"), "SharedLogger2Log1"); + LOG_TRACE(raw_logger, "RawLogger2Log"); + LOG_TRACE(getLogger("Logger_2"), "SharedLogger2Log2"); + + auto actual = stream.str(); + EXPECT_EQ(actual, "SharedLogger2Log1\nRawLogger2Log\nSharedLogger2Log2\n"); + } +} + +TEST(Logger, SharedLoggersThreadSafety) +{ + static size_t threads_count = std::thread::hardware_concurrency(); + static constexpr size_t loggers_count = 10; + static constexpr size_t logger_get_count = 1000; + + Poco::Logger::root(); + + std::vector names; + + Poco::Logger::names(names); + size_t loggers_size_before = names.size(); + + std::vector threads; + + for (size_t thread_index = 0; thread_index < threads_count; ++thread_index) + { + threads.emplace_back([]() + { + for (size_t logger_index = 0; logger_index < loggers_count; ++logger_index) + { + for (size_t iteration = 0; iteration < logger_get_count; ++iteration) + { + getLogger("Logger_" + std::to_string(logger_index)); + } + } + }); + } + + for (auto & thread : threads) + thread.join(); + + Poco::Logger::names(names); + size_t loggers_size_after = names.size(); + + EXPECT_EQ(loggers_size_before, loggers_size_after); +} diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index 866373db44a..b3626135177 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -102,7 +102,7 @@ void checkS3Capabilities( if (s3_capabilities.support_batch_delete && !checkBatchRemove(storage, key_with_trailing_slash)) { LOG_WARNING( - &Poco::Logger::get("S3ObjectStorage"), + getLogger("S3ObjectStorage"), "Storage for disk {} does not support batch delete operations, " "so `s3_capabilities.support_batch_delete` was automatically turned off during the access check. " "To remove this message set `s3_capabilities.support_batch_delete` for the disk to `false`.", diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp index 0223c24973e..786b23caf48 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp @@ -82,7 +82,7 @@ WebObjectStorage::loadFiles(const String & path, const std::unique_lock::max()); SnapshotsQueue snapshots_queue{1}; CoordinationSettingsPtr settings = std::make_shared(); From cb702f72ef040e3974d3f25e228aaeb8971cbdb7 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 5 Feb 2024 16:30:04 +0300 Subject: [PATCH 214/276] Updated implementation --- base/poco/Foundation/CMakeLists.txt | 6 - base/poco/Foundation/include/Poco/Logger.h | 7 +- .../include/Poco/RefCountedObject.h | 3 +- base/poco/Foundation/src/Logger.cpp | 112 +++++++----------- 4 files changed, 46 insertions(+), 82 deletions(-) diff --git a/base/poco/Foundation/CMakeLists.txt b/base/poco/Foundation/CMakeLists.txt index 5fe644d3057..dfb41a33fb1 100644 --- a/base/poco/Foundation/CMakeLists.txt +++ b/base/poco/Foundation/CMakeLists.txt @@ -166,12 +166,6 @@ set (SRCS ) add_library (_poco_foundation ${SRCS}) -target_link_libraries (_poco_foundation - PUBLIC - boost::headers_only - boost::system -) - add_library (Poco::Foundation ALIAS _poco_foundation) # TODO: remove these warning exclusions diff --git a/base/poco/Foundation/include/Poco/Logger.h b/base/poco/Foundation/include/Poco/Logger.h index 883294a071a..2a1cb33b407 100644 --- a/base/poco/Foundation/include/Poco/Logger.h +++ b/base/poco/Foundation/include/Poco/Logger.h @@ -23,8 +23,6 @@ #include #include -#include - #include "Poco/Channel.h" #include "Poco/Format.h" #include "Poco/Foundation.h" @@ -37,7 +35,7 @@ namespace Poco class Exception; class Logger; -using LoggerPtr = boost::intrusive_ptr; +using LoggerPtr = std::shared_ptr; class Foundation_API Logger : public Channel /// Logger is a special Channel that acts as the main @@ -953,9 +951,6 @@ private: static std::optional find(const std::string & name); static Logger * findRawPtr(const std::string & name); - friend void intrusive_ptr_add_ref(Logger * ptr); - friend void intrusive_ptr_release(Logger * ptr); - Logger(); Logger(const Logger &); Logger & operator=(const Logger &); diff --git a/base/poco/Foundation/include/Poco/RefCountedObject.h b/base/poco/Foundation/include/Poco/RefCountedObject.h index 1f806bdacb1..db966089e00 100644 --- a/base/poco/Foundation/include/Poco/RefCountedObject.h +++ b/base/poco/Foundation/include/Poco/RefCountedObject.h @@ -53,10 +53,11 @@ protected: virtual ~RefCountedObject(); /// Destroys the RefCountedObject. - mutable std::atomic _counter; private: RefCountedObject(const RefCountedObject &); RefCountedObject & operator=(const RefCountedObject &); + + mutable std::atomic _counter; }; diff --git a/base/poco/Foundation/src/Logger.cpp b/base/poco/Foundation/src/Logger.cpp index 16fc3a0480e..779af384b0b 100644 --- a/base/poco/Foundation/src/Logger.cpp +++ b/base/poco/Foundation/src/Logger.cpp @@ -302,9 +302,40 @@ void Logger::formatDump(std::string& message, const void* buffer, std::size_t le namespace { -inline LoggerPtr makeLoggerPtr(Logger & logger) +struct LoggerDeleter { - return LoggerPtr(&logger, false /*add_ref*/); + void operator()(Poco::Logger * logger) + { + std::lock_guard lock(getLoggerMutex()); + + /// If logger infrastructure is destroyed just decrement logger reference count + if (!_pLoggerMap) + { + logger->release(); + return; + } + + auto it = _pLoggerMap->find(logger->name()); + assert(it != _pLoggerMap->end()); + + /** If reference count is 1, this means this shared pointer owns logger + * and need destroy it. + */ + size_t reference_count_before_release = logger->release(); + if (reference_count_before_release == 1) + { + assert(it->second.owned_by_shared_ptr); + _pLoggerMap->erase(it); + } + } +}; + +inline LoggerPtr makeLoggerPtr(Logger & logger, bool owned_by_shared_ptr) +{ + if (owned_by_shared_ptr) + return LoggerPtr(&logger, LoggerDeleter()); + + return LoggerPtr(std::shared_ptr{}, &logger); } } @@ -327,15 +358,10 @@ LoggerPtr Logger::getShared(const std::string & name, bool should_be_owned_by_sh /** If during `unsafeGet` logger was created, then this shared pointer owns it. * If logger was already created, then this shared pointer does not own it. */ - if (inserted) - { - if (should_be_owned_by_shared_ptr_if_created) - it->second.owned_by_shared_ptr = true; - else - it->second.logger->duplicate(); - } + if (inserted && should_be_owned_by_shared_ptr_if_created) + it->second.owned_by_shared_ptr = true; - return makeLoggerPtr(*it->second.logger); + return makeLoggerPtr(*it->second.logger, it->second.owned_by_shared_ptr); } @@ -343,29 +369,20 @@ std::pair Logger::unsafeGet(const std::string& { std::optional optional_logger_it = find(name); - bool should_recreate_logger = false; - if (optional_logger_it) { auto & logger_it = *optional_logger_it; - std::optional reference_count_before; - if (get_shared) + if (logger_it->second.owned_by_shared_ptr) { - reference_count_before = logger_it->second.logger->duplicate(); - } - else if (logger_it->second.owned_by_shared_ptr) - { - reference_count_before = logger_it->second.logger->duplicate(); - logger_it->second.owned_by_shared_ptr = false; - } + logger_it->second.logger->duplicate(); - /// Other thread already decided to delete this logger, but did not yet remove it from map - if (reference_count_before && reference_count_before == 0) - should_recreate_logger = true; + if (!get_shared) + logger_it->second.owned_by_shared_ptr = false; + } } - if (!optional_logger_it || should_recreate_logger) + if (!optional_logger_it) { Logger * logger = nullptr; @@ -379,12 +396,6 @@ std::pair Logger::unsafeGet(const std::string& logger = new Logger(name, par.getChannel(), par.getLevel()); } - if (should_recreate_logger) - { - (*optional_logger_it)->second.logger = logger; - return std::make_pair(*optional_logger_it, true); - } - return add(logger); } @@ -412,7 +423,7 @@ LoggerPtr Logger::createShared(const std::string & name, Channel * pChannel, int auto [it, inserted] = unsafeCreate(name, pChannel, level); it->second.owned_by_shared_ptr = true; - return makeLoggerPtr(*it->second.logger); + return makeLoggerPtr(*it->second.logger, it->second.owned_by_shared_ptr); } Logger& Logger::root() @@ -479,43 +490,6 @@ Logger * Logger::findRawPtr(const std::string & name) } -void intrusive_ptr_add_ref(Logger * ptr) -{ - ptr->duplicate(); -} - - -void intrusive_ptr_release(Logger * ptr) -{ - size_t reference_count_before = ptr->_counter.fetch_sub(1, std::memory_order_acq_rel); - if (reference_count_before != 1) - return; - - { - std::lock_guard lock(getLoggerMutex()); - - if (_pLoggerMap) - { - auto it = _pLoggerMap->find(ptr->name()); - - /** It is possible that during release other thread created logger and - * updated iterator in map. - */ - if (it != _pLoggerMap->end() && ptr == it->second.logger) - { - /** If reference count is 0, this means this intrusive pointer owns logger - * and need destroy it. - */ - assert(it->second.owned_by_shared_ptr); - _pLoggerMap->erase(it); - } - } - } - - delete ptr; -} - - void Logger::names(std::vector& names) { std::lock_guard lock(getLoggerMutex()); From 2fc8895ae8aae3cb861faf04e424482ffcd081b3 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 7 Feb 2024 20:04:51 +0300 Subject: [PATCH 215/276] Fixed tests --- src/Server/GRPCServer.cpp | 10 +++++----- src/Server/GRPCServer.h | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 7c532312612..15765f99b4b 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -76,7 +76,7 @@ namespace static std::once_flag once_flag; std::call_once(once_flag, [&config] { - static LoggerPtr logger = getLogger("grpc"); + static LoggerRawPtr logger = getRawLogger("grpc"); gpr_set_log_function([](gpr_log_func_args* args) { if (args->severity == GPR_LOG_SEVERITY_DEBUG) @@ -622,7 +622,7 @@ namespace class Call { public: - Call(CallType call_type_, std::unique_ptr responder_, IServer & iserver_, LoggerPtr log_); + Call(CallType call_type_, std::unique_ptr responder_, IServer & iserver_, LoggerRawPtr log_); ~Call(); void start(const std::function & on_finish_call_callback); @@ -664,7 +664,7 @@ namespace const CallType call_type; std::unique_ptr responder; IServer & iserver; - LoggerPtr log = nullptr; + LoggerRawPtr log = nullptr; std::optional session; ContextMutablePtr query_context; @@ -726,7 +726,7 @@ namespace }; // NOLINTEND(clang-analyzer-optin.performance.Padding) - Call::Call(CallType call_type_, std::unique_ptr responder_, IServer & iserver_, LoggerPtr log_) + Call::Call(CallType call_type_, std::unique_ptr responder_, IServer & iserver_, LoggerRawPtr log_) : call_type(call_type_), responder(std::move(responder_)), iserver(iserver_), log(log_) { } @@ -1851,7 +1851,7 @@ private: GRPCServer::GRPCServer(IServer & iserver_, const Poco::Net::SocketAddress & address_to_listen_) : iserver(iserver_) , address_to_listen(address_to_listen_) - , log(getLogger("GRPCServer")) + , log(getRawLogger("GRPCServer")) , runner(std::make_unique(*this)) {} diff --git a/src/Server/GRPCServer.h b/src/Server/GRPCServer.h index a9c8161298f..f86c2fe4ab7 100644 --- a/src/Server/GRPCServer.h +++ b/src/Server/GRPCServer.h @@ -47,7 +47,7 @@ private: IServer & iserver; const Poco::Net::SocketAddress address_to_listen; - LoggerPtr log; + LoggerRawPtr log; GRPCService grpc_service; std::unique_ptr grpc_server; std::unique_ptr queue; From 9c3b363c81a2ea394a4b4d4c55468f6dcf7b8c8b Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 9 Feb 2024 21:38:56 +0300 Subject: [PATCH 216/276] Fixed tests --- src/Server/GRPCServer.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Server/GRPCServer.h b/src/Server/GRPCServer.h index f86c2fe4ab7..70c16d3e9af 100644 --- a/src/Server/GRPCServer.h +++ b/src/Server/GRPCServer.h @@ -3,10 +3,11 @@ #include "config.h" #if USE_GRPC + +#include "clickhouse_grpc.grpc.pb.h" #include #include #include -#include "clickhouse_grpc.grpc.pb.h" namespace Poco { class Logger; } From 769078e2d81cf8090c5b96bac24f80b5d3cae495 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 9 Feb 2024 19:32:53 +0000 Subject: [PATCH 217/276] Another attempt. --- src/Processors/QueryPlan/Optimizations/optimizeTree.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index c64bc308246..daf0a1b959b 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -122,6 +122,10 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s if (frame.next_child == 0) { + + if (optimization_settings.read_in_order) + optimizeReadInOrder(*frame.node, nodes); + if (optimization_settings.distinct_in_order) tryDistinctReadInOrder(frame.node); } @@ -150,9 +154,6 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s { has_reading_from_mt |= typeid_cast(frame.node->step.get()) != nullptr; - if (optimization_settings.read_in_order) - optimizeReadInOrder(*frame.node, nodes); - /// Projection optimization relies on PK optimization if (optimization_settings.optimize_projection) num_applied_projection From 99b66f75bddf42c008fc7bc7efa64e6881dd6dc7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 10 Feb 2024 02:27:55 +0100 Subject: [PATCH 218/276] Sequential run --- docker/test/stateless/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 55349c6c406..aec2add2857 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -188,7 +188,7 @@ function run_tests() # Too many tests fail for DatabaseReplicated in parallel. ADDITIONAL_OPTIONS+=('--jobs') ADDITIONAL_OPTIONS+=('2') - elif [[ 1 == $(clickhouse-client --query "SELECT value LIKE '%sanitize-coverage%' FROM system.build_options WHERE name = 'CXX_FLAGS'") ]]; then + elif [[ 1 == $(clickhouse-client --query "SELECT value LIKE '%SANITIZE_COVERAGE%' FROM system.build_options WHERE name = 'CXX_FLAGS'") ]]; then # Coverage on a per-test basis could only be collected sequentially. # Do not set the --jobs parameter. echo "Running tests with coverage collection." From ed02154a4a5d00cc8bb0523582161f1342279511 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 10 Feb 2024 07:31:54 +0100 Subject: [PATCH 219/276] Fix "Too many parts" --- docker/test/base/setup_export_logs.sh | 2 +- .../0_stateless/01459_manual_write_to_replicas_quorum.sh | 5 +++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 917701b8d35..9e854dce65a 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -143,7 +143,7 @@ function setup_logs_replication time DateTime COMMENT 'The time of test run', test_name String COMMENT 'The name of the test', coverage Array(UInt64) COMMENT 'An array of addresses of the code (a subset of addresses instrumented for coverage) that were encountered during the test run' - ) ENGINE = MergeTree ORDER BY test_name COMMENT 'Contains information about per-test coverage from the CI' + ) ENGINE = Null COMMENT 'Contains information about per-test coverage from the CI, but used only for exporting to the CI cluster' " # For each system log table: diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh index 209e18e3329..379f83c6271 100755 --- a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh @@ -8,6 +8,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +# This test does many invocations of clickhouse-client in a loop, +# leading to "Too many parts" in the system.coverage_log, +# but we are not interested in client-side coverage here. +unset CLICKHOUSE_WRITE_COVERAGE + NUM_REPLICAS=10 for i in $(seq 1 $NUM_REPLICAS); do From 43d8a879fefbca12bd03328d8cb9d483223b6520 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 10 Feb 2024 11:38:21 +0000 Subject: [PATCH 220/276] Fixing tests. --- .../QueryPlan/Optimizations/optimizeTree.cpp | 10 +- .../02521_aggregation_by_partitions.reference | 141 +++++++++--------- 2 files changed, 74 insertions(+), 77 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index daf0a1b959b..816850cc82c 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -126,6 +126,9 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s if (optimization_settings.read_in_order) optimizeReadInOrder(*frame.node, nodes); + if (optimization_settings.aggregation_in_order) + optimizeAggregationInOrder(*frame.node, nodes); + if (optimization_settings.distinct_in_order) tryDistinctReadInOrder(frame.node); } @@ -139,6 +142,8 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s continue; } + enableMemoryBoundMerging(*stack.back().node, nodes); + stack.pop_back(); } @@ -158,9 +163,6 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s if (optimization_settings.optimize_projection) num_applied_projection += optimizeUseAggregateProjections(*frame.node, nodes, optimization_settings.optimize_use_implicit_projections); - - if (optimization_settings.aggregation_in_order) - optimizeAggregationInOrder(*frame.node, nodes); } /// Traverse all children first. @@ -192,8 +194,6 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s } } - enableMemoryBoundMerging(*stack.back().node, nodes); - stack.pop_back(); } diff --git a/tests/queries/0_stateless/02521_aggregation_by_partitions.reference b/tests/queries/0_stateless/02521_aggregation_by_partitions.reference index 67a131ff853..d32e6c7d416 100644 --- a/tests/queries/0_stateless/02521_aggregation_by_partitions.reference +++ b/tests/queries/0_stateless/02521_aggregation_by_partitions.reference @@ -91,19 +91,18 @@ ExpressionTransform × 16 (Expression) ExpressionTransform × 4 (ReadFromMergeTree) - ExpressionTransform × 4 - MergingSortedTransform 2 → 1 - ExpressionTransform × 2 - MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 - MergingSortedTransform 2 → 1 - ExpressionTransform × 2 - MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 - MergingSortedTransform 2 → 1 - ExpressionTransform × 2 - MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 - MergingSortedTransform 2 → 1 - ExpressionTransform × 2 - MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 + MergingSortedTransform 2 → 1 + ExpressionTransform × 2 + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 + MergingSortedTransform 2 → 1 + ExpressionTransform × 2 + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 + MergingSortedTransform 2 → 1 + ExpressionTransform × 2 + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 + MergingSortedTransform 2 → 1 + ExpressionTransform × 2 + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 1000000 (Expression) ExpressionTransform × 16 @@ -114,41 +113,6 @@ ExpressionTransform × 16 (Expression) ExpressionTransform × 8 (ReadFromMergeTree) - ExpressionTransform × 8 - MergingSortedTransform 2 → 1 - ExpressionTransform × 2 - MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 - MergingSortedTransform 2 → 1 - ExpressionTransform × 2 - MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 - MergingSortedTransform 2 → 1 - ExpressionTransform × 2 - MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 - MergingSortedTransform 2 → 1 - ExpressionTransform × 2 - MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 - MergingSortedTransform 2 → 1 - ExpressionTransform × 2 - MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 - MergingSortedTransform 2 → 1 - ExpressionTransform × 2 - MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 - MergingSortedTransform 2 → 1 - ExpressionTransform × 2 - MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 - MergingSortedTransform 2 → 1 - ExpressionTransform × 2 - MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 -1000000 -(Expression) -ExpressionTransform × 16 - (Aggregating) - FinalizeAggregatedTransform × 16 - AggregatingInOrderTransform × 16 - (Expression) - ExpressionTransform × 16 - (ReadFromMergeTree) - ExpressionTransform × 16 MergingSortedTransform 2 → 1 ExpressionTransform × 2 MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 @@ -173,30 +137,63 @@ ExpressionTransform × 16 MergingSortedTransform 2 → 1 ExpressionTransform × 2 MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 - MergingSortedTransform 2 → 1 - ExpressionTransform × 2 - MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 - MergingSortedTransform 2 → 1 - ExpressionTransform × 2 - MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 - MergingSortedTransform 2 → 1 - ExpressionTransform × 2 - MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 - MergingSortedTransform 2 → 1 - ExpressionTransform × 2 - MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 - MergingSortedTransform 2 → 1 - ExpressionTransform × 2 - MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 - MergingSortedTransform 2 → 1 - ExpressionTransform × 2 - MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 - MergingSortedTransform 2 → 1 - ExpressionTransform × 2 - MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 - MergingSortedTransform 2 → 1 - ExpressionTransform × 2 - MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 +1000000 +(Expression) +ExpressionTransform × 16 + (Aggregating) + FinalizeAggregatedTransform × 16 + AggregatingInOrderTransform × 16 + (Expression) + ExpressionTransform × 16 + (ReadFromMergeTree) + MergingSortedTransform 2 → 1 + ExpressionTransform × 2 + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 + MergingSortedTransform 2 → 1 + ExpressionTransform × 2 + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 + MergingSortedTransform 2 → 1 + ExpressionTransform × 2 + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 + MergingSortedTransform 2 → 1 + ExpressionTransform × 2 + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 + MergingSortedTransform 2 → 1 + ExpressionTransform × 2 + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 + MergingSortedTransform 2 → 1 + ExpressionTransform × 2 + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 + MergingSortedTransform 2 → 1 + ExpressionTransform × 2 + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 + MergingSortedTransform 2 → 1 + ExpressionTransform × 2 + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 + MergingSortedTransform 2 → 1 + ExpressionTransform × 2 + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 + MergingSortedTransform 2 → 1 + ExpressionTransform × 2 + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 + MergingSortedTransform 2 → 1 + ExpressionTransform × 2 + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 + MergingSortedTransform 2 → 1 + ExpressionTransform × 2 + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 + MergingSortedTransform 2 → 1 + ExpressionTransform × 2 + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 + MergingSortedTransform 2 → 1 + ExpressionTransform × 2 + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 + MergingSortedTransform 2 → 1 + ExpressionTransform × 2 + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 + MergingSortedTransform 2 → 1 + ExpressionTransform × 2 + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 1000000 Skip merging: 1 Skip merging: 1 From 74caa8e44e66f0ce2803b088d8bd0bb796de5bef Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 10 Feb 2024 11:43:35 +0000 Subject: [PATCH 221/276] Updaye test. --- .../02521_aggregation_by_partitions.reference | 12 ++++++++++++ .../0_stateless/02521_aggregation_by_partitions.sql | 12 ++++++++++++ 2 files changed, 24 insertions(+) diff --git a/tests/queries/0_stateless/02521_aggregation_by_partitions.reference b/tests/queries/0_stateless/02521_aggregation_by_partitions.reference index d32e6c7d416..87b2d5c3430 100644 --- a/tests/queries/0_stateless/02521_aggregation_by_partitions.reference +++ b/tests/queries/0_stateless/02521_aggregation_by_partitions.reference @@ -1,3 +1,5 @@ +-- { echoOn } +explain pipeline select a from t1 group by a; (Expression) ExpressionTransform × 16 (Aggregating) @@ -15,6 +17,8 @@ ExpressionTransform × 16 Resize 3 → 1 MergeTreeSelect(pool: ReadPool, algorithm: Thread) × 3 0 → 1 1000000 +-- { echoOn } +explain pipeline select a from t2 group by a; (Expression) ExpressionTransform × 16 (Aggregating) @@ -40,6 +44,8 @@ ExpressionTransform × 16 Resize 2 → 1 MergeTreeSelect(pool: ReadPool, algorithm: Thread) × 2 0 → 1 1000000 +-- { echoOn } +explain pipeline select a from t3 group by a; (Expression) ExpressionTransform × 16 (Aggregating) @@ -82,6 +88,8 @@ ExpressionTransform × 16 MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 1000000 1000000 +-- { echoOn } +explain pipeline select a from t4 group by a settings read_in_order_two_level_merge_threshold = 1e12; (Expression) ExpressionTransform × 16 (Aggregating) @@ -104,6 +112,8 @@ ExpressionTransform × 16 ExpressionTransform × 2 MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 1000000 +-- { echoOn } +explain pipeline select a from t5 group by a settings read_in_order_two_level_merge_threshold = 1e12; (Expression) ExpressionTransform × 16 (Aggregating) @@ -138,6 +148,8 @@ ExpressionTransform × 16 ExpressionTransform × 2 MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) × 2 0 → 1 1000000 +-- { echoOn } +explain pipeline select a from t6 group by a settings read_in_order_two_level_merge_threshold = 1e12; (Expression) ExpressionTransform × 16 (Aggregating) diff --git a/tests/queries/0_stateless/02521_aggregation_by_partitions.sql b/tests/queries/0_stateless/02521_aggregation_by_partitions.sql index 87317e5fba4..5b013ca5aef 100644 --- a/tests/queries/0_stateless/02521_aggregation_by_partitions.sql +++ b/tests/queries/0_stateless/02521_aggregation_by_partitions.sql @@ -15,7 +15,9 @@ system stop merges t1; insert into t1 select number from numbers_mt(1e6); insert into t1 select number from numbers_mt(1e6); +-- { echoOn } explain pipeline select a from t1 group by a; +-- { echoOff } select count() from (select throwIf(count() != 2) from t1 group by a); @@ -28,7 +30,9 @@ system stop merges t2; insert into t2 select number from numbers_mt(1e6); insert into t2 select number from numbers_mt(1e6); +-- { echoOn } explain pipeline select a from t2 group by a; +-- { echoOff } select count() from (select throwIf(count() != 2) from t2 group by a); @@ -41,7 +45,9 @@ system stop merges t3; insert into t3 select number from numbers_mt(1e6); insert into t3 select number from numbers_mt(1e6); +-- { echoOn } explain pipeline select a from t3 group by a; +-- { echoOff } select count() from (select throwIf(count() != 2) from t3 group by a); @@ -63,7 +69,9 @@ system stop merges t4; insert into t4 select number from numbers_mt(1e6); insert into t4 select number from numbers_mt(1e6); +-- { echoOn } explain pipeline select a from t4 group by a settings read_in_order_two_level_merge_threshold = 1e12; +-- { echoOff } select count() from (select throwIf(count() != 2) from t4 group by a); @@ -76,7 +84,9 @@ system stop merges t5; insert into t5 select number from numbers_mt(1e6); insert into t5 select number from numbers_mt(1e6); +-- { echoOn } explain pipeline select a from t5 group by a settings read_in_order_two_level_merge_threshold = 1e12; +-- { echoOff } select count() from (select throwIf(count() != 2) from t5 group by a); @@ -89,7 +99,9 @@ system stop merges t6; insert into t6 select number from numbers_mt(1e6); insert into t6 select number from numbers_mt(1e6); +-- { echoOn } explain pipeline select a from t6 group by a settings read_in_order_two_level_merge_threshold = 1e12; +-- { echoOff } select count() from (select throwIf(count() != 2) from t6 group by a); From bcae537810a221af9e83551bf3b083190e82f70e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 10 Feb 2024 11:50:06 +0000 Subject: [PATCH 222/276] Remove commented code. --- .../Optimizations/optimizePrewhere.cpp | 320 +----------------- 1 file changed, 1 insertion(+), 319 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index ee5ad8d1a8a..6676f935b67 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -16,49 +16,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -// namespace -// { - -// void matchDAGOutputNodesOrderWithHeader(ActionsDAGPtr & actions_dag, const Block & expected_header) -// { -// std::unordered_map output_name_to_node; -// for (const auto * output_node : actions_dag->getOutputs()) -// output_name_to_node.emplace(output_node->result_name, output_node); - -// std::unordered_set used_output_nodes; - -// ActionsDAG::NodeRawConstPtrs updated_outputs; -// updated_outputs.reserve(expected_header.columns()); - -// for (const auto & column : expected_header) -// { -// auto output_node_it = output_name_to_node.find(column.name); -// if (output_node_it == output_name_to_node.end()) -// throw Exception(ErrorCodes::LOGICAL_ERROR, -// "Invalid move to PREWHERE optimization. Cannot find column {} in output", -// column.name); - -// updated_outputs.push_back(output_node_it->second); -// used_output_nodes.insert(output_node_it->second); -// } - -// ActionsDAG::NodeRawConstPtrs unused_outputs; -// for (const auto * output_node : actions_dag->getOutputs()) -// { -// if (used_output_nodes.contains(output_node)) -// continue; - -// unused_outputs.push_back(output_node); -// } - -// auto & actions_dag_outputs = actions_dag->getOutputs(); -// actions_dag_outputs = std::move(updated_outputs); -// actions_dag_outputs.insert(actions_dag_outputs.end(), unused_outputs.begin(), unused_outputs.end()); -// } - -// } - - namespace QueryPlanOptimizations { @@ -102,40 +59,6 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) if (!filter_step) return; - /** Collect required filter output columns. - * Collect output nodes that are mapped to input nodes. - * Collect input node to output nodes mapping. - */ - ColumnsWithTypeAndName required_columns_after_filter; - // std::unordered_set output_nodes_mapped_to_input; - // std::unordered_map> input_node_to_output_names; - - // for (const auto * output_node : filter_step->getExpression()->getOutputs()) - // { - // const auto * node_without_alias = output_node; - // while (node_without_alias->type == ActionsDAG::ActionType::ALIAS) - // node_without_alias = node_without_alias->children[0]; - - // if (node_without_alias->type == ActionsDAG::ActionType::INPUT) - // { - // output_nodes_mapped_to_input.emplace(output_node->result_name); - - // auto output_names_it = input_node_to_output_names.find(node_without_alias->result_name); - // if (output_names_it == input_node_to_output_names.end()) - // { - // auto [insert_it, _] = input_node_to_output_names.emplace(node_without_alias->result_name, std::vector()); - // output_names_it = insert_it; - // } - - // output_names_it->second.push_back(output_node->result_name); - // } - - // if (output_node->result_name == filter_step->getFilterColumnName() && filter_step->removesFilterColumn()) - // continue; - - // required_columns_after_filter.push_back(ColumnWithTypeAndName(output_node->result_type, output_node->result_name)); - // } - const auto & context = read_from_merge_tree->getContext(); const auto & settings = context->getSettingsRef(); @@ -149,6 +72,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) const auto & storage_snapshot = read_from_merge_tree->getStorageSnapshot(); + ColumnsWithTypeAndName required_columns_after_filter; if (read_from_merge_tree->isQueryWithSampling()) { const auto & sampling_key = storage_snapshot->getMetadataForQuery()->getSamplingKey(); @@ -195,10 +119,6 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) prewhere_info = std::make_shared(); prewhere_info->need_filter = true; - // std::cerr << filter_step->getExpression()->dumpDAG() << std::endl; - - // QueryPlan::Node * replace_old_filter_node = nullptr; - // bool remove_filter_node = false; auto filter_expression = filter_step->getExpression(); const auto & filter_column_name = filter_step->getFilterColumnName(); @@ -213,19 +133,8 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) outputs.resize(size); } - // if (!optimize_result.fully_moved_to_prewhere) - // { auto split_result = filter_step->getExpression()->split(optimize_result.prewhere_nodes, true); - // std::cerr << split_result.first->dumpDAG() << std::endl; - // std::cerr << split_result.second->dumpDAG() << std::endl; - - // for (const auto * input : split_result.first->getInputs()) - // std::cerr << "in 1" << input->result_name << std::endl; - // for (const auto * input : split_result.second->getInputs()) - // std::cerr << "in 2" << input->result_name << std::endl; - - /// This is the leak of abstraction. /// Splited actions may have inputs which are needed only for PREWHERE. /// This is fine for ActionsDAG to have such a split, but it breaks defaults calculation. @@ -239,13 +148,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) /// /// So, here we restore removed inputs for PREWHERE actions { - // const auto & virtuals = read_from_merge_tree->getVirtualColumnNames(); - // NameSet virtual_names(virtuals.begin(), virtuals.end()); - - //std::unordered_set first_inputs(split_result.first->getInputs().begin(), split_result.first->getInputs().end()); std::unordered_set first_outputs(split_result.first->getOutputs().begin(), split_result.first->getOutputs().end()); - ///std::unordered_set second_inputs(split_result.second->getInputs().begin(), split_result.second->getInputs().end()); - for (const auto * input : split_result.first->getInputs()) { if (!first_outputs.contains(input)) @@ -256,22 +159,12 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) split_result.second->addInput(input->result_name, input->result_type); } } - - // NameSet input_columns; - // for (const auto * input : split_result.first->getInputs()) - // input_columns.insert(input->result_name); - - // auto header = read_from_merge_tree->getStorageSnapshot()->getSampleBlockForColumns(read_from_merge_tree->getRealColumnNames()); - // header = MergeTreeSelectProcessor::transformHeader(std::move(header), prewhere_info, {}, {}); } ActionsDAG::NodeRawConstPtrs conditions; conditions.reserve(split_result.split_nodes_mapping.size()); for (const auto * condition : optimize_result.prewhere_nodes) - { - // std::cerr << ".. " << condition->result_name << std::endl; conditions.push_back(split_result.split_nodes_mapping.at(condition)); - } prewhere_info->prewhere_actions = std::move(split_result.first); prewhere_info->remove_prewhere_column = optimize_result.fully_moved_to_prewhere && filter_step->removesFilterColumn(); @@ -291,14 +184,8 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) prewhere_info->prewhere_actions->getOutputs().push_back(node); } - // std::cerr << read_from_merge_tree->getOutputStream().header.dumpStructure() << std::endl; - // std::cerr << read_from_merge_tree->getOutputStream().header.dumpIndex() << std::endl; - read_from_merge_tree->updatePrewhereInfo(prewhere_info); - // std::cerr << read_from_merge_tree->getOutputStream().header.dumpStructure() << std::endl; - // std::cerr << read_from_merge_tree->getOutputStream().header.dumpIndex() << std::endl; - if (!optimize_result.fully_moved_to_prewhere) { filter_node->step = std::make_unique( @@ -309,215 +196,10 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) } else { - // std::cerr << split_result.second->dumpDAG() << std::endl; - // std::cerr << read_from_merge_tree->getOutputStream().header.dumpStructure() << std::endl; - // std::cerr << read_from_merge_tree->getOutputStream().header.dumpIndex() << std::endl; - filter_node->step = std::make_unique( read_from_merge_tree->getOutputStream(), std::move(split_result.second)); } - // return; - // } - - // prewhere_info->prewhere_actions = filter_step->getExpression(); - // prewhere_info->prewhere_actions->projectInput(false); - // std::cerr << prewhere_info->prewhere_actions->dumpDAG() << std::endl; - // prewhere_info->prewhere_column_name = filter_step->getFilterColumnName(); - // prewhere_info->remove_prewhere_column = filter_step->removesFilterColumn(); - - // read_from_merge_tree->updatePrewhereInfo(prewhere_info); - - // replace_old_filter_node = frame.node; - // remove_filter_node = true; - - // auto & prewhere_filter_actions = optimize_result->prewhere_filter_actions; - - // ActionsChain actions_chain; - - // std::string prewere_filter_node_name = prewhere_filter_actions->getOutputs().at(0)->result_name; - // actions_chain.addStep(std::make_unique(prewhere_filter_actions)); - - // auto & filter_actions = optimize_result->filter_actions; - - // /** Merge tree where optimizer splits conjunctions in filter expression into 2 parts: - // * 1. Filter expressions. - // * 2. Prewhere filter expressions. - // * - // * There can be cases when all expressions are moved to PREWHERE, but it is not - // * enough to produce required filter output columns. - // * - // * Example: SELECT (a AND b) AS cond FROM test_table WHERE cond AND c; - // * In this example condition expressions `a`, `b`, `c` can move to PREWHERE, but PREWHERE will not contain expression `and(a, b)`. - // * It will contain only `a`, `b`, `c`, `and(a, b, c)` expressions. - // * - // * In such scenario we need to create additional step to calculate `and(a, b)` expression after PREWHERE. - // */ - // bool need_additional_filter_after_prewhere = false; - - // if (!filter_actions) - // { - // /// Any node from PREWHERE filter actions can be used as possible output node - // std::unordered_set possible_prewhere_output_nodes; - // for (const auto & node : prewhere_filter_actions->getNodes()) - // possible_prewhere_output_nodes.insert(node.result_name); - - // for (auto & required_column : required_columns_after_filter) - // { - // if (!possible_prewhere_output_nodes.contains(required_column.name) && - // !output_nodes_mapped_to_input.contains(required_column.name)) - // { - // need_additional_filter_after_prewhere = true; - // break; - // } - // } - // } - - // /** If there are additional filter actions after PREWHERE filter actions, we create filter actions dag using PREWHERE filter - // * actions output columns as filter actions dag input columns. - // * Then we merge this filter actions dag nodes with old filter step actions dag nodes, to reuse some expressions from - // * PREWHERE filter actions. - // */ - // if (need_additional_filter_after_prewhere || filter_actions) - // { - // auto merged_filter_actions = std::make_shared(actions_chain.getLastStepAvailableOutputColumns()); - // merged_filter_actions->getOutputs().clear(); - // merged_filter_actions->mergeNodes(std::move(*filter_step->getExpression()->clone())); - - // /// Add old filter step filter column to outputs - // for (const auto & node : merged_filter_actions->getNodes()) - // { - // if (node.result_name == filter_step->getFilterColumnName()) - // { - // merged_filter_actions->getOutputs().push_back(&node); - // break; - // } - // } - - // filter_actions = std::move(merged_filter_actions); - - // /// If there is filter after PREWHERE, we can ignore filtering during PREWHERE stage - // prewhere_info->need_filter = false; - - // actions_chain.addStep(std::make_unique(filter_actions)); - // } - - // auto required_output_actions = std::make_shared(required_columns_after_filter); - // actions_chain.addStep(std::make_unique(required_output_actions)); - - // actions_chain.finalize(); - - // prewhere_filter_actions->projectInput(false); - - // auto & prewhere_actions_chain_node = actions_chain[0]; - // prewhere_info->prewhere_actions = std::move(prewhere_filter_actions); - // prewhere_info->prewhere_column_name = prewere_filter_node_name; - // prewhere_info->remove_prewhere_column = !prewhere_actions_chain_node->getChildRequiredOutputColumnsNames().contains(prewere_filter_node_name); - - // read_from_merge_tree->updatePrewhereInfo(prewhere_info); - - // QueryPlan::Node * replace_old_filter_node = nullptr; - // bool remove_filter_node = false; - - // if (filter_actions) - // { - // filter_actions->projectInput(false); - - // /// Match dag output nodes with old filter step header - // matchDAGOutputNodesOrderWithHeader(filter_actions, filter_step->getOutputStream().header); - - // auto & filter_actions_chain_node = actions_chain[1]; - // bool remove_filter_column = !filter_actions_chain_node->getChildRequiredOutputColumnsNames().contains(filter_step->getFilterColumnName()); - // auto after_prewhere_filter_step = std::make_unique(read_from_merge_tree->getOutputStream(), - // filter_actions, - // filter_step->getFilterColumnName(), - // remove_filter_column); - - // auto & node = nodes.emplace_back(); - // node.children.emplace_back(frame.node); - // node.step = std::move(after_prewhere_filter_step); - - // replace_old_filter_node = &node; - // } - // else - // { - // auto rename_actions_dag = std::make_shared(read_from_merge_tree->getOutputStream().header.getColumnsWithTypeAndName()); - // bool apply_rename_step = false; - - // ActionsDAG::NodeRawConstPtrs updated_outputs; - - // /** If in output after read from merge tree there are column names without aliases, - // * apply old filter step aliases to them. - // */ - // for (const auto * output_node : rename_actions_dag->getOutputs()) - // { - // const auto alias_it = input_node_to_output_names.find(output_node->result_name); - // if (alias_it == input_node_to_output_names.end()) - // { - // updated_outputs.push_back(output_node); - // continue; - // } - - // for (auto & output_name : alias_it->second) - // { - // if (output_name == output_node->result_name) - // { - // updated_outputs.push_back(output_node); - // continue; - // } - - // updated_outputs.push_back(&rename_actions_dag->addAlias(*output_node, output_name)); - // apply_rename_step = true; - // } - // } - - // rename_actions_dag->getOutputs() = std::move(updated_outputs); - - // bool apply_match_step = false; - - // /// If column order does not match old filter step column order, match dag output nodes with header - // if (!blocksHaveEqualStructure(read_from_merge_tree->getOutputStream().header, filter_step->getOutputStream().header)) - // { - // apply_match_step = true; - // matchDAGOutputNodesOrderWithHeader(rename_actions_dag, filter_step->getOutputStream().header); - // } - - // if (apply_rename_step || apply_match_step) - // { - // auto rename_step = std::make_unique(read_from_merge_tree->getOutputStream(), rename_actions_dag); - // if (apply_rename_step) - // rename_step->setStepDescription("Change column names to column identifiers"); - - // auto & node = nodes.emplace_back(); - // node.children.emplace_back(frame.node); - // node.step = std::move(rename_step); - - // replace_old_filter_node = &node; - // } - // else - // { - // replace_old_filter_node = frame.node; - // remove_filter_node = true; - // } - // } - - // QueryPlan::Node * filter_parent_node = (stack.rbegin() + 2)->node; - - // for (auto & filter_parent_child : filter_parent_node->children) - // { - // if (filter_parent_child == filter_node) - // { - // filter_parent_child = frame.node; - - // size_t stack_size = stack.size(); - - // /// Step is completely replaced with PREWHERE filter actions, remove it from stack. - // std::swap(stack[stack_size - 1], stack[stack_size - 2]); - // stack.pop_back(); - - // break; - // } - // } } } From 8b3c27d45bf6721c97231de0bba063463ecf2e0f Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 10 Feb 2024 15:00:31 +0300 Subject: [PATCH 223/276] Updated a list of trusted contributors --- tests/ci/lambda_shared_package/lambda_shared/pr.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/lambda_shared_package/lambda_shared/pr.py b/tests/ci/lambda_shared_package/lambda_shared/pr.py index ce38475b3ee..a23d4bbb4c2 100644 --- a/tests/ci/lambda_shared_package/lambda_shared/pr.py +++ b/tests/ci/lambda_shared_package/lambda_shared/pr.py @@ -44,6 +44,7 @@ TRUSTED_CONTRIBUTORS = { "kitaisreal", "k-morozov", # Konstantin Morozov, Yandex Cloud "justindeguzman", # ClickHouse, Inc + "jrdi", # ClickHouse contributor, TinyBird ] } From fb8723a15c00cc6eff7c2ea2765a528788d11b59 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 10 Feb 2024 13:13:26 +0100 Subject: [PATCH 224/276] Update pr.py --- tests/ci/lambda_shared_package/lambda_shared/pr.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/lambda_shared_package/lambda_shared/pr.py b/tests/ci/lambda_shared_package/lambda_shared/pr.py index a23d4bbb4c2..1b4f827cc0a 100644 --- a/tests/ci/lambda_shared_package/lambda_shared/pr.py +++ b/tests/ci/lambda_shared_package/lambda_shared/pr.py @@ -44,7 +44,7 @@ TRUSTED_CONTRIBUTORS = { "kitaisreal", "k-morozov", # Konstantin Morozov, Yandex Cloud "justindeguzman", # ClickHouse, Inc - "jrdi", # ClickHouse contributor, TinyBird + "jrdi", # ClickHouse contributor, TinyBird ] } From e6d96e43252df4df9198a0c08bd053b168aff097 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 10 Feb 2024 14:49:43 +0100 Subject: [PATCH 225/276] Better warning for disabled kernel.task_delayacct Signed-off-by: Azat Khuzhin --- programs/server/Server.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 7d3953ae37e..f00da445c16 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -557,7 +557,7 @@ static void sanityChecks(Server & server) { const char * filename = "/proc/sys/kernel/task_delayacct"; if (readNumber(filename) == 0) - server.context()->addWarningMessage("Delay accounting is not enabled, OSIOWaitMicroseconds will not be gathered. Check " + String(filename)); + server.context()->addWarningMessage("Delay accounting is not enabled, OSIOWaitMicroseconds will not be gathered. You can enable it using `echo 1 > " + String(filename) + "` or by using sysctl."); } catch (...) // NOLINT(bugprone-empty-catch) { From 8b25d868e269de6407377657b406114c62a0f2be Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 10 Feb 2024 17:04:27 +0100 Subject: [PATCH 226/276] Update optimizePrewhere.cpp --- src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index 6676f935b67..ec07f028f20 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -11,11 +11,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - namespace QueryPlanOptimizations { From 306f5047818ea3dc13ec5575c5d6d96f5b25373f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 10 Feb 2024 19:53:03 +0000 Subject: [PATCH 227/276] Fixing test. --- .../QueryPlan/Optimizations/optimizeTree.cpp | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index 816850cc82c..fafd6d1dc00 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -126,9 +126,6 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s if (optimization_settings.read_in_order) optimizeReadInOrder(*frame.node, nodes); - if (optimization_settings.aggregation_in_order) - optimizeAggregationInOrder(*frame.node, nodes); - if (optimization_settings.distinct_in_order) tryDistinctReadInOrder(frame.node); } @@ -142,8 +139,6 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s continue; } - enableMemoryBoundMerging(*stack.back().node, nodes); - stack.pop_back(); } @@ -163,6 +158,10 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s if (optimization_settings.optimize_projection) num_applied_projection += optimizeUseAggregateProjections(*frame.node, nodes, optimization_settings.optimize_use_implicit_projections); + + + if (optimization_settings.aggregation_in_order) + optimizeAggregationInOrder(*frame.node, nodes); } /// Traverse all children first. @@ -194,6 +193,8 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s } } + enableMemoryBoundMerging(*stack.back().node, nodes); + stack.pop_back(); } From 40ea04faf57226e5d67c4f2224dee4e5a9ff488e Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 10 Feb 2024 13:44:55 +0100 Subject: [PATCH 228/276] Add check that recursiveRemoveLowCardinality() doesn't change the size of a column. --- .../DataTypeLowCardinalityHelpers.cpp | 60 +++++++++---------- 1 file changed, 30 insertions(+), 30 deletions(-) diff --git a/src/DataTypes/DataTypeLowCardinalityHelpers.cpp b/src/DataTypes/DataTypeLowCardinalityHelpers.cpp index 98eb76267a4..116e806f89c 100644 --- a/src/DataTypes/DataTypeLowCardinalityHelpers.cpp +++ b/src/DataTypes/DataTypeLowCardinalityHelpers.cpp @@ -20,6 +20,7 @@ namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int TYPE_MISMATCH; + extern const int LOGICAL_ERROR; } DataTypePtr recursiveRemoveLowCardinality(const DataTypePtr & type) @@ -55,62 +56,61 @@ DataTypePtr recursiveRemoveLowCardinality(const DataTypePtr & type) ColumnPtr recursiveRemoveLowCardinality(const ColumnPtr & column) { - if (!column) - return column; + ColumnPtr res = column; if (const auto * column_array = typeid_cast(column.get())) { const auto & data = column_array->getDataPtr(); auto data_no_lc = recursiveRemoveLowCardinality(data); - if (data.get() == data_no_lc.get()) - return column; - - return ColumnArray::create(data_no_lc, column_array->getOffsetsPtr()); + if (data.get() != data_no_lc.get()) + res = ColumnArray::create(data_no_lc, column_array->getOffsetsPtr()); } - - if (const auto * column_const = typeid_cast(column.get())) + else if (const auto * column_const = typeid_cast(column.get())) { const auto & nested = column_const->getDataColumnPtr(); auto nested_no_lc = recursiveRemoveLowCardinality(nested); - if (nested.get() == nested_no_lc.get()) - return column; - - return ColumnConst::create(nested_no_lc, column_const->size()); + if (nested.get() != nested_no_lc.get()) + res = ColumnConst::create(nested_no_lc, column_const->size()); } - - if (const auto * column_tuple = typeid_cast(column.get())) + else if (const auto * column_tuple = typeid_cast(column.get())) { auto columns = column_tuple->getColumns(); for (auto & element : columns) element = recursiveRemoveLowCardinality(element); - return ColumnTuple::create(columns); + res = ColumnTuple::create(columns); } - - if (const auto * column_map = typeid_cast(column.get())) + else if (const auto * column_map = typeid_cast(column.get())) { const auto & nested = column_map->getNestedColumnPtr(); auto nested_no_lc = recursiveRemoveLowCardinality(nested); - if (nested.get() == nested_no_lc.get()) - return column; - - return ColumnMap::create(nested_no_lc); + if (nested.get() != nested_no_lc.get()) + res = ColumnMap::create(nested_no_lc); } - /// Special case when column is a lazy argument of short circuit function. /// We should call recursiveRemoveLowCardinality on the result column /// when function will be executed. - if (const auto * column_function = typeid_cast(column.get())) + else if (const auto * column_function = typeid_cast(column.get())) { - if (!column_function->isShortCircuitArgument()) - return column; - - return column_function->recursivelyConvertResultToFullColumnIfLowCardinality(); + if (column_function->isShortCircuitArgument()) + res = column_function->recursivelyConvertResultToFullColumnIfLowCardinality(); + } + else if (const auto * column_low_cardinality = typeid_cast(column.get())) + { + res = column_low_cardinality->convertToFullColumn(); } - if (const auto * column_low_cardinality = typeid_cast(column.get())) - return column_low_cardinality->convertToFullColumn(); + if (res != column) + { + /// recursiveRemoveLowCardinality() must not change the size of a passed column! + if (res->size() != column->size()) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "recursiveRemoveLowCardinality() somehow changed the size of column {}. Old size={}, new size={}. It's a bug", + column->getName(), column->size(), res->size()); + } + } - return column; + return res; } ColumnPtr recursiveLowCardinalityTypeConversion(const ColumnPtr & column, const DataTypePtr & from_type, const DataTypePtr & to_type) From 5424f235a92331c8ed2dba72b29de24d3ff1e8f7 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Sat, 10 Feb 2024 00:27:15 +0000 Subject: [PATCH 229/276] Fix StorageURL doing some of the query execution in one thread instead of max_threads --- src/Storages/StorageURL.cpp | 6 ++++-- .../0_stateless/02723_parallelize_output_setting.reference | 4 ++++ .../0_stateless/02723_parallelize_output_setting.sql | 3 +++ 3 files changed, 11 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 433f4ed7700..ce9b0cd366b 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -904,6 +904,7 @@ public: , context(std::move(context_)) , max_block_size(max_block_size_) , num_streams(num_streams_) + , max_num_streams(num_streams_) { } @@ -920,6 +921,7 @@ private: size_t max_block_size; size_t num_streams; + const size_t max_num_streams; std::shared_ptr iterator_wrapper; bool is_url_with_globs = false; @@ -1093,8 +1095,8 @@ void ReadFromURL::initializePipeline(QueryPipelineBuilder & pipeline, const Buil auto pipe = Pipe::unitePipes(std::move(pipes)); size_t output_ports = pipe.numOutputPorts(); const bool parallelize_output = context->getSettingsRef().parallelize_output_from_storages; - if (parallelize_output && storage->parallelizeOutputAfterReading(context) && output_ports > 0 && output_ports < num_streams) - pipe.resize(num_streams); + if (parallelize_output && storage->parallelizeOutputAfterReading(context) && output_ports > 0 && output_ports < max_num_streams) + pipe.resize(max_num_streams); if (pipe.empty()) pipe = Pipe(std::make_shared(info.source_header)); diff --git a/tests/queries/0_stateless/02723_parallelize_output_setting.reference b/tests/queries/0_stateless/02723_parallelize_output_setting.reference index 0f2a396f471..36e4e68ecd5 100644 --- a/tests/queries/0_stateless/02723_parallelize_output_setting.reference +++ b/tests/queries/0_stateless/02723_parallelize_output_setting.reference @@ -5,3 +5,7 @@ select startsWith(trimLeft(explain),'Resize') as resize from (explain pipeline s -- no Resize in pipeline set parallelize_output_from_storages=0; select startsWith(trimLeft(explain),'Resize') as resize from (explain pipeline select * from file(data_02723.csv)) where resize; +-- Data from URL source is immediately resized to max_treads streams, before any ExpressionTransform. +set parallelize_output_from_storages=1; +select match(arrayStringConcat(groupArray(explain), ''), '.*Resize 1 → 2 *URL 0 → 1 *$') from (explain pipeline select x, count() from url('https://example.com', Parquet, 'x Int64') group by x order by count() limit 10); +1 diff --git a/tests/queries/0_stateless/02723_parallelize_output_setting.sql b/tests/queries/0_stateless/02723_parallelize_output_setting.sql index 7db28ca4dec..12786b80f69 100644 --- a/tests/queries/0_stateless/02723_parallelize_output_setting.sql +++ b/tests/queries/0_stateless/02723_parallelize_output_setting.sql @@ -10,3 +10,6 @@ select startsWith(trimLeft(explain),'Resize') as resize from (explain pipeline s set parallelize_output_from_storages=0; select startsWith(trimLeft(explain),'Resize') as resize from (explain pipeline select * from file(data_02723.csv)) where resize; +-- Data from URL source is immediately resized to max_treads streams, before any ExpressionTransform. +set parallelize_output_from_storages=1; +select match(arrayStringConcat(groupArray(explain), ''), '.*Resize 1 → 2 *URL 0 → 1 *$') from (explain pipeline select x, count() from url('https://example.com', Parquet, 'x Int64') group by x order by count() limit 10); \ No newline at end of file From b5fd68a2b6b9c50accc62671f1b87ea4cd30785c Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Sat, 10 Feb 2024 03:40:55 +0000 Subject: [PATCH 230/276] no-fasttest --- tests/queries/0_stateless/02723_parallelize_output_setting.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02723_parallelize_output_setting.sql b/tests/queries/0_stateless/02723_parallelize_output_setting.sql index 12786b80f69..86e6d4b4e3d 100644 --- a/tests/queries/0_stateless/02723_parallelize_output_setting.sql +++ b/tests/queries/0_stateless/02723_parallelize_output_setting.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel +-- Tags: no-parallel, no-fasttest insert into function file(data_02723.csv) select number from numbers(5) settings engine_file_truncate_on_insert=1; From 64e53fee7bb935a7be84aed5ecc31300e18e4b3d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 11 Feb 2024 10:07:21 +0100 Subject: [PATCH 231/276] Reintroduce 02590_interserver_mode_client_info_initial_query_start_time Reintroduce the test without assumption that initial_query_start_time is the same as query_start_time. This reverts commit 075da5602fdc03d4d5b15cd8d769704259b168a9. --- ...nt_info_initial_query_start_time.reference | 8 +++ ...de_client_info_initial_query_start_time.sh | 67 +++++++++++++++++++ 2 files changed, 75 insertions(+) create mode 100644 tests/queries/0_stateless/02590_interserver_mode_client_info_initial_query_start_time.reference create mode 100755 tests/queries/0_stateless/02590_interserver_mode_client_info_initial_query_start_time.sh diff --git a/tests/queries/0_stateless/02590_interserver_mode_client_info_initial_query_start_time.reference b/tests/queries/0_stateless/02590_interserver_mode_client_info_initial_query_start_time.reference new file mode 100644 index 00000000000..eafcfc23fb8 --- /dev/null +++ b/tests/queries/0_stateless/02590_interserver_mode_client_info_initial_query_start_time.reference @@ -0,0 +1,8 @@ +SELECT +3 1 +3 1 +INSERT +CHECK +1 +2 +6 2 diff --git a/tests/queries/0_stateless/02590_interserver_mode_client_info_initial_query_start_time.sh b/tests/queries/0_stateless/02590_interserver_mode_client_info_initial_query_start_time.sh new file mode 100755 index 00000000000..3b0d2309784 --- /dev/null +++ b/tests/queries/0_stateless/02590_interserver_mode_client_info_initial_query_start_time.sh @@ -0,0 +1,67 @@ +#!/usr/bin/env bash +# Tags: no-fasttest +# Tag no-fasttest: interserver mode requires SSL +# +# Test that checks that some of ClientInfo correctly passed in inter-server mode. +# NOTE: we need .sh test (.sql is not enough) because queries on remote nodes does not have current_database = currentDatabase() +# +# Check-style suppression: select * from system.query_log where current_database = currentDatabase(); + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +function get_query_id() { random_str 10; } + +$CLICKHOUSE_CLIENT -nm -q " + drop table if exists buf; + drop table if exists dist; + drop table if exists data; + + create table data (key Int) engine=Memory(); + create table dist as data engine=Distributed(test_cluster_interserver_secret, currentDatabase(), data, key); + create table dist_dist as data engine=Distributed(test_cluster_interserver_secret, currentDatabase(), dist, key); + system stop distributed sends dist; +" + +echo "SELECT" +query_id="$(get_query_id)" +# initialize connection, but actually if there are other tables that uses this +# cluster then, it will be created long time ago, but this is OK for this +# test, since we care about the difference between NOW() and there should +# not be any significant difference. +$CLICKHOUSE_CLIENT --prefer_localhost_replica=0 --query_id "$query_id" -q "select * from dist" +$CLICKHOUSE_CLIENT -nm --param_query_id "$query_id" -q " + system flush logs; + select count(), count(distinct initial_query_start_time_microseconds) from system.query_log where type = 'QueryFinish' and initial_query_id = {query_id:String}; +" + +sleep 1 + +query_id="$(get_query_id)" +# this query (and all subsequent) should reuse the previous connection (at least most of the time) +$CLICKHOUSE_CLIENT --prefer_localhost_replica=0 --query_id "$query_id" -q "select * from dist" + +$CLICKHOUSE_CLIENT -nm --param_query_id "$query_id" -q " + system flush logs; + select count(), count(distinct initial_query_start_time_microseconds) from system.query_log where type = 'QueryFinish' and initial_query_id = {query_id:String}; +" + +echo "INSERT" +query_id="$(get_query_id)" +$CLICKHOUSE_CLIENT --prefer_localhost_replica=0 --query_id "$query_id" -nm -q " + insert into dist_dist values (1),(2); + select * from data; +" + +sleep 1 +$CLICKHOUSE_CLIENT -nm --param_query_id "$query_id" -q "system flush distributed dist_dist" +sleep 1 +$CLICKHOUSE_CLIENT -nm --param_query_id "$query_id" -q "system flush distributed dist" + +echo "CHECK" +$CLICKHOUSE_CLIENT -nm --param_query_id "$query_id" -q " + select * from data order by key; + system flush logs; + select count(), count(distinct initial_query_start_time_microseconds) from system.query_log where type = 'QueryFinish' and initial_query_id = {query_id:String}; +" From a3938e7df0c1b4a8c38b19a23cdb6253f86dc330 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 11 Feb 2024 10:42:36 +0100 Subject: [PATCH 232/276] Respect CMAKE_OSX_DEPLOYMENT_TARGET for Rust targets This should fix the following warnings: ld: warning: object file (rust/prql/Debug/lib_ch_rust_prql.a[74](aarch_aapcs64.o)) was built for newer 'macOS' version (14.0) than being linked (11.0) ld: warning: object file (rust/skim/Debug/lib_ch_rust_skim_rust.a[40](c0e230e4543ad627-lib.rs.o)) was built for newer 'macOS' version (14.0) than being linked (11.0) ld: warning: object file (rust/skim/Debug/lib_ch_rust_skim_rust.a[46](6c3a6ae2099d4914-cxx.o)) was built for newer 'macOS' version (14.0) than being linked (11.0) Signed-off-by: Azat Khuzhin --- rust/CMakeLists.txt | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/rust/CMakeLists.txt b/rust/CMakeLists.txt index 66694ee16f8..6715a54221a 100644 --- a/rust/CMakeLists.txt +++ b/rust/CMakeLists.txt @@ -14,6 +14,11 @@ macro(configure_rustc) set(RUST_CFLAGS "${RUST_CFLAGS} --sysroot ${CMAKE_SYSROOT}") endif() + if (CMAKE_OSX_DEPLOYMENT_TARGET) + set(RUST_CXXFLAGS "${RUST_CXXFLAGS} -mmacosx-version-min=${CMAKE_OSX_DEPLOYMENT_TARGET}") + set(RUST_CFLAGS "${RUST_CFLAGS} -mmacosx-version-min=${CMAKE_OSX_DEPLOYMENT_TARGET}") + endif() + if (USE_MUSL) set(RUST_CXXFLAGS "${RUST_CXXFLAGS} -D_LIBCPP_HAS_MUSL_LIBC=1") endif () @@ -25,14 +30,23 @@ macro(configure_rustc) set(RUSTCWRAPPER "") endif() - set(RUSTFLAGS "[]") + set(RUSTFLAGS) + if (CMAKE_OSX_DEPLOYMENT_TARGET) + list(APPEND RUSTFLAGS "-C" "link-arg=-mmacosx-version-min=${CMAKE_OSX_DEPLOYMENT_TARGET}") + endif() + set(RUST_CARGO_BUILD_STD "") # For more info: https://doc.rust-lang.org/beta/unstable-book/compiler-flags/sanitizer.html#memorysanitizer if (SANITIZE STREQUAL "memory") set(RUST_CARGO_BUILD_STD "build-std = [\"std\", \"panic_abort\", \"core\", \"alloc\"]") - set(RUSTFLAGS "[\"-Zsanitizer=memory\", \"-Zsanitizer-memory-track-origins\"]") + list(APPEND RUSTFLAGS "-Zsanitizer=memory" "-Zsanitizer-memory-track-origins") endif() + list(TRANSFORM RUSTFLAGS PREPEND "\"") + list(TRANSFORM RUSTFLAGS APPEND "\"") + list(JOIN RUSTFLAGS "," RUSTFLAGS) + set(RUSTFLAGS "[${RUSTFLAGS}]") + message(STATUS "RUST_CFLAGS: ${RUST_CFLAGS}") message(STATUS "RUST_CXXFLAGS: ${RUST_CXXFLAGS}") message(STATUS "RUSTFLAGS: ${RUSTFLAGS}") From 309db427e76aea295ffb3b5e6e787fa6fc0bdf2c Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Sun, 11 Feb 2024 13:13:49 +0100 Subject: [PATCH 233/276] Do not reinitialize ZooKeeperWithFaultInjection on each chunk --- src/Storages/System/StorageSystemZooKeeper.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index a4920d49b6d..abf93bf1ac0 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -220,6 +220,7 @@ private: const UInt64 max_block_size; Paths paths; ContextPtr context; + ZooKeeperWithFaultInjection::Ptr zookeeper; bool started = false; }; @@ -484,7 +485,6 @@ Chunk SystemZooKeeperSource::generate() settings.insert_keeper_retry_initial_backoff_ms, settings.insert_keeper_retry_max_backoff_ms); - ZooKeeperWithFaultInjection::Ptr zookeeper; /// Handles reconnects when needed auto get_zookeeper = [&] () { From cb812d84cf6fb215b3a40d1278c6c5c09cec587a Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 11 Feb 2024 22:31:33 +0000 Subject: [PATCH 234/276] Fix: check if std::function is set before calling it --- src/Interpreters/executeQuery.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 9cc8d346aa4..8b36790a269 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1374,8 +1374,11 @@ void executeQuery( /// Force an update of the headers before we start writing result_details.content_type = output_format->getContentType(); result_details.format = format_name; - set_result_details(result_details); - set_result_details = nullptr; + if (set_result_details) + { + set_result_details(result_details); + set_result_details = nullptr; + } } } catch (const DB::Exception & e) From c9c0e12b4cdc7272135089d5ceae7d0ffca9ab43 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 12 Feb 2024 09:15:40 +0100 Subject: [PATCH 235/276] Address PR reviews --- src/Coordination/KeeperStorage.h | 40 ++++++++++++++++++-------------- 1 file changed, 23 insertions(+), 17 deletions(-) diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 9743e051422..f20f3cca668 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -40,22 +40,6 @@ public: int64_t mtime{0}; - struct - { - bool is_ephemeral : 1; - int64_t ctime : 63; - } is_ephemeral_and_ctime{false, 0}; - - union - { - int64_t ephemeral_owner; - struct - { - int32_t seq_num; - int32_t num_children; - } children_info; - } ephemeral_or_children_data{0}; - std::unique_ptr data{nullptr}; uint32_t data_size{0}; @@ -174,12 +158,34 @@ public: // (e.g. we don't need to copy list of children) void shallowCopy(const Node & other); private: + /// as ctime can't be negative because it stores the timestamp when the + /// node was created, we can use the MSB for a bool + struct + { + bool is_ephemeral : 1; + int64_t ctime : 63; + } is_ephemeral_and_ctime{false, 0}; + + /// ephemeral notes cannot have children so a node can set either + /// ephemeral_owner OR seq_num + num_children + union + { + int64_t ephemeral_owner; + struct + { + int32_t seq_num; + int32_t num_children; + } children_info; + } ephemeral_or_children_data{0}; + ChildrenSet children{}; }; #if !defined(ADDRESS_SANITIZER) && !defined(MEMORY_SANITIZER) static_assert( - sizeof(ListNode) <= 144, "std::list node containing ListNode is > 160 bytes which will increase memory consumption"); + sizeof(ListNode) <= 144, + "std::list node containing ListNode is > 160 bytes (sizeof(ListNode) + 16 bytes for pointers) which will increase " + "memory consumption"); #endif enum DigestVersion : uint8_t From a0507c204465ac5a0715736cd1f0cba84b2683cd Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 12 Feb 2024 10:36:35 +0100 Subject: [PATCH 236/276] Whitespace --- src/Coordination/KeeperStorage.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index f20f3cca668..6618ec0bd85 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -167,7 +167,7 @@ public: } is_ephemeral_and_ctime{false, 0}; /// ephemeral notes cannot have children so a node can set either - /// ephemeral_owner OR seq_num + num_children + /// ephemeral_owner OR seq_num + num_children union { int64_t ephemeral_owner; From 504e3b0f1bb3075219af8fbcc5fac51fd3bc0b3f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 12 Feb 2024 06:08:57 +0100 Subject: [PATCH 237/276] Install ch/chc/chl via nfpm Signed-off-by: Azat Khuzhin --- packages/clickhouse-client.yaml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/packages/clickhouse-client.yaml b/packages/clickhouse-client.yaml index 4d707b28ad9..34b42d92adf 100644 --- a/packages/clickhouse-client.yaml +++ b/packages/clickhouse-client.yaml @@ -49,6 +49,12 @@ contents: dst: /usr/bin/clickhouse-client - src: root/usr/bin/clickhouse-local dst: /usr/bin/clickhouse-local +- src: root/usr/bin/ch + dst: /usr/bin/ch +- src: root/usr/bin/chc + dst: /usr/bin/chc +- src: root/usr/bin/chl + dst: /usr/bin/chl - src: root/usr/bin/clickhouse-obfuscator dst: /usr/bin/clickhouse-obfuscator # docs From 7fb31fe16081421ae17b2f66fe31e0c3d0293bb4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 11 Feb 2024 20:00:37 +0100 Subject: [PATCH 238/276] Remove ability to disable generic clickhouse components Components like client/server/... are very generic, and there is no point in disabling them, since it does not reduce amount of compiled code a lot anyway (just a few modules for entrypoints, everything else is already included in the clickhouse binary), and eventually they are just symlinks to the clickhouse binary. But there are few, that requires extra libraries, like ODBC bridge or keeper components (and there is also standalone keeper binary compiled with musl), those had been kept. Also add some descriptions for some utils and change exit code to 0 for --help. Signed-off-by: Azat Khuzhin --- programs/CMakeLists.txt | 263 ++++----------------------- programs/benchmark/Benchmark.cpp | 3 +- programs/client/Client.cpp | 1 + programs/compressor/Compressor.cpp | 1 + programs/config_tools.h.in | 15 -- programs/copier/ClusterCopierApp.cpp | 1 + programs/git-import/git-import.cpp | 1 + programs/install/Install.cpp | 3 +- programs/local/LocalServer.cpp | 1 + programs/main.cpp | 97 +++------- programs/su/su.cpp | 1 + 11 files changed, 69 insertions(+), 318 deletions(-) diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index 6e544bac81c..7ebbf52ea7e 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -7,35 +7,16 @@ endif () include(${ClickHouse_SOURCE_DIR}/cmake/split_debug_symbols.cmake) # The `clickhouse` binary is a multi purpose tool that contains multiple execution modes (client, server, etc.), -# each of them may be built and linked as a separate library. -# If you do not know what modes you need, turn this option OFF and enable SERVER and CLIENT only. +# So client/server/... is just a symlink to `clickhouse` binary. +# +# But, there are several components that requires extra libraries, like keeper +# requires NuRaft, that regular binary does not requires, so you can disable +# compilation of this components. +# +# If you do not know what modes you need, turn then all. option (ENABLE_CLICKHOUSE_ALL "Enable all ClickHouse modes by default" ON) -option (ENABLE_CLICKHOUSE_SERVER "Server mode (main mode)" ${ENABLE_CLICKHOUSE_ALL}) -option (ENABLE_CLICKHOUSE_CLIENT "Client mode (interactive tui/shell that connects to the server)" - ${ENABLE_CLICKHOUSE_ALL}) - -# https://clickhouse.com/docs/en/operations/utilities/clickhouse-local/ -option (ENABLE_CLICKHOUSE_LOCAL "Local files fast processing mode" ${ENABLE_CLICKHOUSE_ALL}) - -# https://clickhouse.com/docs/en/operations/utilities/clickhouse-benchmark/ -option (ENABLE_CLICKHOUSE_BENCHMARK "Queries benchmarking mode" ${ENABLE_CLICKHOUSE_ALL}) - -option (ENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG "Configs processor (extract values etc.)" ${ENABLE_CLICKHOUSE_ALL}) - -# https://clickhouse.com/docs/en/operations/utilities/clickhouse-compressor/ -option (ENABLE_CLICKHOUSE_COMPRESSOR "Data compressor and decompressor" ${ENABLE_CLICKHOUSE_ALL}) - -# https://clickhouse.com/docs/en/operations/utilities/clickhouse-copier/ -option (ENABLE_CLICKHOUSE_COPIER "Inter-cluster data copying mode" ${ENABLE_CLICKHOUSE_ALL}) - -option (ENABLE_CLICKHOUSE_FORMAT "Queries pretty-printer and formatter with syntax highlighting" - ${ENABLE_CLICKHOUSE_ALL}) - # https://clickhouse.com/docs/en/operations/utilities/clickhouse-obfuscator/ -option (ENABLE_CLICKHOUSE_OBFUSCATOR "Table data obfuscator (convert real data to benchmark-ready one)" - ${ENABLE_CLICKHOUSE_ALL}) - # https://clickhouse.com/docs/en/operations/utilities/odbc-bridge/ # TODO Also needs NANODBC. if (ENABLE_ODBC AND NOT USE_MUSL) @@ -51,18 +32,12 @@ endif () # https://presentations.clickhouse.com/matemarketing_2020/ option (ENABLE_CLICKHOUSE_GIT_IMPORT "A tool to analyze Git repositories" ${ENABLE_CLICKHOUSE_ALL}) -option (ENABLE_CLICKHOUSE_STATIC_FILES_DISK_UPLOADER "A tool to export table data files to be later put to a static files web server" ${ENABLE_CLICKHOUSE_ALL}) - option (ENABLE_CLICKHOUSE_KEEPER "ClickHouse alternative to ZooKeeper" ${ENABLE_CLICKHOUSE_ALL}) option (ENABLE_CLICKHOUSE_KEEPER_CONVERTER "Util allows to convert ZooKeeper logs and snapshots into clickhouse-keeper snapshot" ${ENABLE_CLICKHOUSE_ALL}) option (ENABLE_CLICKHOUSE_KEEPER_CLIENT "ClickHouse Keeper Client" ${ENABLE_CLICKHOUSE_ALL}) -option (ENABLE_CLICKHOUSE_SU "A tool similar to 'su'" ${ENABLE_CLICKHOUSE_ALL}) - -option (ENABLE_CLICKHOUSE_DISKS "A tool to manage disks" ${ENABLE_CLICKHOUSE_ALL}) - if (NOT ENABLE_NURAFT) # RECONFIGURE_MESSAGE_LEVEL should not be used here, # since ENABLE_NURAFT is set to OFF for FreeBSD and Darwin. @@ -71,27 +46,7 @@ if (NOT ENABLE_NURAFT) set(ENABLE_CLICKHOUSE_KEEPER_CONVERTER OFF) endif() -option(ENABLE_CLICKHOUSE_INSTALL "Install ClickHouse without .deb/.rpm/.tgz packages (having the binary only)" ${ENABLE_CLICKHOUSE_ALL}) - -message(STATUS "ClickHouse modes:") - -if (NOT ENABLE_CLICKHOUSE_SERVER) - message(WARNING "ClickHouse server mode is not going to be built.") -else() - message(STATUS "Server mode: ON") -endif() - -if (NOT ENABLE_CLICKHOUSE_CLIENT) - message(WARNING "ClickHouse client mode is not going to be built. You won't be able to connect to the server and run tests") -else() - message(STATUS "Client mode: ON") -endif() - -if (ENABLE_CLICKHOUSE_LOCAL) - message(STATUS "Local mode: ON") -else() - message(STATUS "Local mode: OFF") -endif() +message(STATUS "ClickHouse extra components:") if (ENABLE_CLICKHOUSE_SELF_EXTRACTING) message(STATUS "Self-extracting executable: ON") @@ -99,42 +54,6 @@ else() message(STATUS "Self-extracting executable: OFF") endif() -if (ENABLE_CLICKHOUSE_BENCHMARK) - message(STATUS "Benchmark mode: ON") -else() - message(STATUS "Benchmark mode: OFF") -endif() - -if (ENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG) - message(STATUS "Extract from config mode: ON") -else() - message(STATUS "Extract from config mode: OFF") -endif() - -if (ENABLE_CLICKHOUSE_COMPRESSOR) - message(STATUS "Compressor mode: ON") -else() - message(STATUS "Compressor mode: OFF") -endif() - -if (ENABLE_CLICKHOUSE_COPIER) - message(STATUS "Copier mode: ON") -else() - message(STATUS "Copier mode: OFF") -endif() - -if (ENABLE_CLICKHOUSE_FORMAT) - message(STATUS "Format mode: ON") -else() - message(STATUS "Format mode: OFF") -endif() - -if (ENABLE_CLICKHOUSE_OBFUSCATOR) - message(STATUS "Obfuscator mode: ON") -else() - message(STATUS "Obfuscator mode: OFF") -endif() - if (ENABLE_CLICKHOUSE_ODBC_BRIDGE) message(STATUS "ODBC bridge mode: ON") else() @@ -147,18 +66,6 @@ else() message(STATUS "Library bridge mode: OFF") endif() -if (ENABLE_CLICKHOUSE_INSTALL) - message(STATUS "ClickHouse install: ON") -else() - message(STATUS "ClickHouse install: OFF") -endif() - -if (ENABLE_CLICKHOUSE_GIT_IMPORT) - message(STATUS "ClickHouse git-import: ON") -else() - message(STATUS "ClickHouse git-import: OFF") -endif() - if (ENABLE_CLICKHOUSE_KEEPER) message(STATUS "ClickHouse keeper mode: ON") else() @@ -177,19 +84,6 @@ else() message(STATUS "ClickHouse keeper-client mode: OFF") endif() - -if (ENABLE_CLICKHOUSE_DISKS) - message(STATUS "Clickhouse disks mode: ON") -else() - message(STATUS "ClickHouse disks mode: OFF") -endif() - -if (ENABLE_CLICKHOUSE_SU) - message(STATUS "ClickHouse su: ON") -else() - message(STATUS "ClickHouse su: OFF") -endif() - configure_file (config_tools.h.in ${CONFIG_INCLUDE_PATH}/config_tools.h) macro(clickhouse_target_link_split_lib target name) @@ -272,42 +166,6 @@ endif () target_link_libraries (clickhouse PRIVATE clickhouse_common_io string_utils ${HARMFUL_LIB}) target_include_directories (clickhouse PRIVATE ${CMAKE_CURRENT_BINARY_DIR}) -if (ENABLE_CLICKHOUSE_SERVER) - clickhouse_target_link_split_lib(clickhouse server) -endif () -if (ENABLE_CLICKHOUSE_CLIENT) - clickhouse_target_link_split_lib(clickhouse client) -endif () -if (ENABLE_CLICKHOUSE_LOCAL) - clickhouse_target_link_split_lib(clickhouse local) -endif () -if (ENABLE_CLICKHOUSE_BENCHMARK) - clickhouse_target_link_split_lib(clickhouse benchmark) -endif () -if (ENABLE_CLICKHOUSE_COPIER) - clickhouse_target_link_split_lib(clickhouse copier) -endif () -if (ENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG) - clickhouse_target_link_split_lib(clickhouse extract-from-config) -endif () -if (ENABLE_CLICKHOUSE_COMPRESSOR) - clickhouse_target_link_split_lib(clickhouse compressor) -endif () -if (ENABLE_CLICKHOUSE_FORMAT) - clickhouse_target_link_split_lib(clickhouse format) -endif () -if (ENABLE_CLICKHOUSE_OBFUSCATOR) - clickhouse_target_link_split_lib(clickhouse obfuscator) -endif () -if (ENABLE_CLICKHOUSE_GIT_IMPORT) - clickhouse_target_link_split_lib(clickhouse git-import) -endif () -if (ENABLE_CLICKHOUSE_STATIC_FILES_DISK_UPLOADER) - clickhouse_target_link_split_lib(clickhouse static-files-disk-uploader) -endif () -if (ENABLE_CLICKHOUSE_SU) - clickhouse_target_link_split_lib(clickhouse su) -endif () if (ENABLE_CLICKHOUSE_KEEPER) clickhouse_target_link_split_lib(clickhouse keeper) endif() @@ -317,83 +175,41 @@ endif() if (ENABLE_CLICKHOUSE_KEEPER_CLIENT) clickhouse_target_link_split_lib(clickhouse keeper-client) endif() -if (ENABLE_CLICKHOUSE_INSTALL) - clickhouse_target_link_split_lib(clickhouse install) -endif () -if (ENABLE_CLICKHOUSE_DISKS) - clickhouse_target_link_split_lib(clickhouse disks) -endif () +clickhouse_target_link_split_lib(clickhouse install) set (CLICKHOUSE_BUNDLE) +macro(clickhouse_program_install name lib_name) + clickhouse_target_link_split_lib(clickhouse ${lib_name}) + add_custom_target (${name} ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse ${name} DEPENDS clickhouse) + install (FILES "${CMAKE_CURRENT_BINARY_DIR}/${name}" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) + list(APPEND CLICKHOUSE_BUNDLE ${name}) +endmacro() + if (ENABLE_CLICKHOUSE_SELF_EXTRACTING) list(APPEND CLICKHOUSE_BUNDLE self-extracting) endif () -if (NOT BUILD_STANDALONE_KEEPER) - add_custom_target (ch ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse ch DEPENDS clickhouse) -endif() -if (ENABLE_CLICKHOUSE_SERVER) - add_custom_target (clickhouse-server ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-server DEPENDS clickhouse) - install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-server" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) - list(APPEND CLICKHOUSE_BUNDLE clickhouse-server) -endif () -if (ENABLE_CLICKHOUSE_CLIENT) - add_custom_target (clickhouse-client ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-client DEPENDS clickhouse) - add_custom_target (chc ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse chc DEPENDS clickhouse) - install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-client" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) - list(APPEND CLICKHOUSE_BUNDLE clickhouse-client) -endif () -if (ENABLE_CLICKHOUSE_LOCAL) - add_custom_target (clickhouse-local ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-local DEPENDS clickhouse) - add_custom_target (chl ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse chl DEPENDS clickhouse) - install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-local" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) - list(APPEND CLICKHOUSE_BUNDLE clickhouse-local) -endif () -if (ENABLE_CLICKHOUSE_BENCHMARK) - add_custom_target (clickhouse-benchmark ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-benchmark DEPENDS clickhouse) - install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-benchmark" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) - list(APPEND CLICKHOUSE_BUNDLE clickhouse-benchmark) -endif () -if (ENABLE_CLICKHOUSE_COPIER) - add_custom_target (clickhouse-copier ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-copier DEPENDS clickhouse) - install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-copier" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) - list(APPEND CLICKHOUSE_BUNDLE clickhouse-copier) -endif () -if (ENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG) - add_custom_target (clickhouse-extract-from-config ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-extract-from-config DEPENDS clickhouse) - install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-extract-from-config" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) - list(APPEND CLICKHOUSE_BUNDLE clickhouse-extract-from-config) -endif () -if (ENABLE_CLICKHOUSE_COMPRESSOR) - add_custom_target (clickhouse-compressor ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-compressor DEPENDS clickhouse) - install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-compressor" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) - list(APPEND CLICKHOUSE_BUNDLE clickhouse-compressor) -endif () -if (ENABLE_CLICKHOUSE_FORMAT) - add_custom_target (clickhouse-format ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-format DEPENDS clickhouse) - install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-format" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) - list(APPEND CLICKHOUSE_BUNDLE clickhouse-format) -endif () -if (ENABLE_CLICKHOUSE_OBFUSCATOR) - add_custom_target (clickhouse-obfuscator ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-obfuscator DEPENDS clickhouse) - install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-obfuscator" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) - list(APPEND CLICKHOUSE_BUNDLE clickhouse-obfuscator) -endif () -if (ENABLE_CLICKHOUSE_GIT_IMPORT) - add_custom_target (clickhouse-git-import ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-git-import DEPENDS clickhouse) - install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-git-import" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) - list(APPEND CLICKHOUSE_BUNDLE clickhouse-git-import) -endif () -if (ENABLE_CLICKHOUSE_STATIC_FILES_DISK_UPLOADER) - add_custom_target (clickhouse-static-files-disk-uploader ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-static-files-disk-uploader DEPENDS clickhouse) - install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-static-files-disk-uploader" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) - list(APPEND CLICKHOUSE_BUNDLE clickhouse-static-files-disk-uploader) -endif () -if (ENABLE_CLICKHOUSE_SU) - add_custom_target (clickhouse-su ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-su DEPENDS clickhouse) - install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-su" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) - list(APPEND CLICKHOUSE_BUNDLE clickhouse-su) -endif () +clickhouse_program_install(clickhouse-server server) + +# client +clickhouse_program_install(clickhouse-client client) +add_custom_target (chc ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse chc DEPENDS clickhouse) + +# local +clickhouse_program_install(clickhouse-local local) +add_custom_target (chl ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse chl DEPENDS clickhouse) +add_custom_target (ch ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse ch DEPENDS clickhouse) + +clickhouse_program_install(clickhouse-benchmark benchmark) +clickhouse_program_install(clickhouse-copier copier) +clickhouse_program_install(clickhouse-extract-from-config extract-from-config) +clickhouse_program_install(clickhouse-compressor compressor) +clickhouse_program_install(clickhouse-format format) +clickhouse_program_install(clickhouse-obfuscator obfuscator) +clickhouse_program_install(clickhouse-git-import git-import) +clickhouse_program_install(clickhouse-static-files-disk-uploader static-files-disk-uploader) +clickhouse_program_install(clickhouse-disks disks) +clickhouse_program_install(clickhouse-su su) if (ENABLE_CLICKHOUSE_KEEPER) if (NOT BUILD_STANDALONE_KEEPER AND CREATE_KEEPER_SYMLINK) @@ -423,11 +239,6 @@ if (ENABLE_CLICKHOUSE_KEEPER_CLIENT) list(APPEND CLICKHOUSE_BUNDLE clickhouse-keeper-client) endif () -if (ENABLE_CLICKHOUSE_DISKS) - add_custom_target (clickhouse-disks ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-disks DEPENDS clickhouse) - install (FILES "${CMAKE_CURRENT_BINARY_DIR}/clickhouse-disks" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) - list(APPEND CLICKHOUSE_BUNDLE clickhouse-disks) -endif () add_custom_target (clickhouse-bundle ALL DEPENDS ${CLICKHOUSE_BUNDLE}) diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index 59fc6c0c17f..961c678b936 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -640,7 +640,8 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) { std::cout << "Usage: " << argv[0] << " [options] < queries.txt\n"; std::cout << desc << "\n"; - return 1; + std::cout << "\nSee also: https://clickhouse.com/docs/en/operations/utilities/clickhouse-benchmark/\n"; + return 0; } print_stacktrace = options.count("stacktrace"); diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index cc142470d7f..fdd262f185d 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1000,6 +1000,7 @@ void Client::printHelpMessage(const OptionsDescription & options_description) std::cout << options_description.external_description.value() << "\n"; std::cout << options_description.hosts_and_ports_description.value() << "\n"; std::cout << "In addition, --param_name=value can be specified for substitution of parameters for parametrized queries.\n"; + std::cout << "\nSee also: https://clickhouse.com/docs/en/integrations/sql-clients/cli\n"; } diff --git a/programs/compressor/Compressor.cpp b/programs/compressor/Compressor.cpp index cc25747702a..7125fdc744f 100644 --- a/programs/compressor/Compressor.cpp +++ b/programs/compressor/Compressor.cpp @@ -100,6 +100,7 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) std::cout << "Usage: " << argv[0] << " [options] < INPUT > OUTPUT" << std::endl; std::cout << "Usage: " << argv[0] << " [options] INPUT OUTPUT" << std::endl; std::cout << desc << std::endl; + std::cout << "\nSee also: https://clickhouse.com/docs/en/operations/utilities/clickhouse-compressor/\n"; return 0; } diff --git a/programs/config_tools.h.in b/programs/config_tools.h.in index 65ef3ca762b..50a1de5628b 100644 --- a/programs/config_tools.h.in +++ b/programs/config_tools.h.in @@ -2,23 +2,8 @@ #pragma once -#cmakedefine01 ENABLE_CLICKHOUSE_SERVER -#cmakedefine01 ENABLE_CLICKHOUSE_CLIENT -#cmakedefine01 ENABLE_CLICKHOUSE_LOCAL -#cmakedefine01 ENABLE_CLICKHOUSE_BENCHMARK -#cmakedefine01 ENABLE_CLICKHOUSE_PERFORMANCE_TEST -#cmakedefine01 ENABLE_CLICKHOUSE_COPIER -#cmakedefine01 ENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG -#cmakedefine01 ENABLE_CLICKHOUSE_COMPRESSOR -#cmakedefine01 ENABLE_CLICKHOUSE_FORMAT -#cmakedefine01 ENABLE_CLICKHOUSE_OBFUSCATOR -#cmakedefine01 ENABLE_CLICKHOUSE_GIT_IMPORT -#cmakedefine01 ENABLE_CLICKHOUSE_INSTALL #cmakedefine01 ENABLE_CLICKHOUSE_ODBC_BRIDGE #cmakedefine01 ENABLE_CLICKHOUSE_LIBRARY_BRIDGE #cmakedefine01 ENABLE_CLICKHOUSE_KEEPER #cmakedefine01 ENABLE_CLICKHOUSE_KEEPER_CLIENT #cmakedefine01 ENABLE_CLICKHOUSE_KEEPER_CONVERTER -#cmakedefine01 ENABLE_CLICKHOUSE_STATIC_FILES_DISK_UPLOADER -#cmakedefine01 ENABLE_CLICKHOUSE_SU -#cmakedefine01 ENABLE_CLICKHOUSE_DISKS diff --git a/programs/copier/ClusterCopierApp.cpp b/programs/copier/ClusterCopierApp.cpp index 53f79888573..fdf07dec61a 100644 --- a/programs/copier/ClusterCopierApp.cpp +++ b/programs/copier/ClusterCopierApp.cpp @@ -78,6 +78,7 @@ void ClusterCopierApp::handleHelp(const std::string &, const std::string &) help_formatter.setHeader("Copies tables from one cluster to another"); help_formatter.setUsage("--config-file --task-path "); help_formatter.format(std::cerr); + help_formatter.setFooter("See also: https://clickhouse.com/docs/en/operations/utilities/clickhouse-copier/"); stopOptionsProcessing(); } diff --git a/programs/git-import/git-import.cpp b/programs/git-import/git-import.cpp index 16244232bee..fdabeacd46e 100644 --- a/programs/git-import/git-import.cpp +++ b/programs/git-import/git-import.cpp @@ -172,6 +172,7 @@ clickhouse-client --query "INSERT INTO git.commits FORMAT TSV" < commits.tsv clickhouse-client --query "INSERT INTO git.file_changes FORMAT TSV" < file_changes.tsv clickhouse-client --query "INSERT INTO git.line_changes FORMAT TSV" < line_changes.tsv +Check out this presentation: https://presentations.clickhouse.com/matemarketing_2020/ )"; namespace po = boost::program_options; diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index a4a4672f5c4..c3d2c61d6d0 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -242,9 +242,10 @@ int mainEntryClickHouseInstall(int argc, char ** argv) if (options.count("help")) { + std::cout << "Install ClickHouse without .deb/.rpm/.tgz packages (having the binary only)\n\n"; std::cout << "Usage: " << formatWithSudo(std::string(argv[0]) + " install [options]", getuid() != 0) << '\n'; std::cout << desc << '\n'; - return 1; + return 0; } /// We need to copy binary to the binary directory. diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 443d4a52fa3..d7acf4112a5 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -828,6 +828,7 @@ void LocalServer::printHelpMessage([[maybe_unused]] const OptionsDescription & o std::cout << options_description.main_description.value() << "\n"; std::cout << getHelpFooter() << "\n"; std::cout << "In addition, --param_name=value can be specified for substitution of parameters for parametrized queries.\n"; + std::cout << "\nSee also: https://clickhouse.com/docs/en/operations/utilities/clickhouse-local/\n"; #endif } diff --git a/programs/main.cpp b/programs/main.cpp index 1ff7e5db560..3896b3819a8 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -24,36 +24,28 @@ /// Universal executable for various clickhouse applications -#if ENABLE_CLICKHOUSE_SERVER int mainEntryClickHouseServer(int argc, char ** argv); -#endif -#if ENABLE_CLICKHOUSE_CLIENT int mainEntryClickHouseClient(int argc, char ** argv); -#endif -#if ENABLE_CLICKHOUSE_LOCAL int mainEntryClickHouseLocal(int argc, char ** argv); -#endif -#if ENABLE_CLICKHOUSE_BENCHMARK int mainEntryClickHouseBenchmark(int argc, char ** argv); -#endif -#if ENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG int mainEntryClickHouseExtractFromConfig(int argc, char ** argv); -#endif -#if ENABLE_CLICKHOUSE_COMPRESSOR int mainEntryClickHouseCompressor(int argc, char ** argv); -#endif -#if ENABLE_CLICKHOUSE_FORMAT int mainEntryClickHouseFormat(int argc, char ** argv); -#endif -#if ENABLE_CLICKHOUSE_COPIER int mainEntryClickHouseClusterCopier(int argc, char ** argv); -#endif -#if ENABLE_CLICKHOUSE_OBFUSCATOR int mainEntryClickHouseObfuscator(int argc, char ** argv); -#endif -#if ENABLE_CLICKHOUSE_GIT_IMPORT int mainEntryClickHouseGitImport(int argc, char ** argv); -#endif +int mainEntryClickHouseStaticFilesDiskUploader(int argc, char ** argv); +int mainEntryClickHouseSU(int argc, char ** argv); +int mainEntryClickHouseDisks(int argc, char ** argv); + +int mainEntryClickHouseHashBinary(int, char **) +{ + /// Intentionally without newline. So you can run: + /// objcopy --add-section .clickhouse.hash=<(./clickhouse hash-binary) clickhouse + std::cout << getHashOfLoadedBinaryHex(); + return 0; +} + #if ENABLE_CLICKHOUSE_KEEPER int mainEntryClickHouseKeeper(int argc, char ** argv); #endif @@ -63,30 +55,13 @@ int mainEntryClickHouseKeeperConverter(int argc, char ** argv); #if ENABLE_CLICKHOUSE_KEEPER_CLIENT int mainEntryClickHouseKeeperClient(int argc, char ** argv); #endif -#if ENABLE_CLICKHOUSE_STATIC_FILES_DISK_UPLOADER -int mainEntryClickHouseStaticFilesDiskUploader(int argc, char ** argv); -#endif -#if ENABLE_CLICKHOUSE_SU -int mainEntryClickHouseSU(int argc, char ** argv); -#endif -#if ENABLE_CLICKHOUSE_INSTALL + +// install int mainEntryClickHouseInstall(int argc, char ** argv); int mainEntryClickHouseStart(int argc, char ** argv); int mainEntryClickHouseStop(int argc, char ** argv); int mainEntryClickHouseStatus(int argc, char ** argv); int mainEntryClickHouseRestart(int argc, char ** argv); -#endif -#if ENABLE_CLICKHOUSE_DISKS -int mainEntryClickHouseDisks(int argc, char ** argv); -#endif - -int mainEntryClickHouseHashBinary(int, char **) -{ - /// Intentionally without newline. So you can run: - /// objcopy --add-section .clickhouse.hash=<(./clickhouse hash-binary) clickhouse - std::cout << getHashOfLoadedBinaryHex(); - return 0; -} namespace { @@ -98,36 +73,22 @@ using MainFunc = int (*)(int, char**); /// Add an item here to register new application std::pair clickhouse_applications[] = { -#if ENABLE_CLICKHOUSE_LOCAL {"local", mainEntryClickHouseLocal}, -#endif -#if ENABLE_CLICKHOUSE_CLIENT {"client", mainEntryClickHouseClient}, -#endif -#if ENABLE_CLICKHOUSE_BENCHMARK {"benchmark", mainEntryClickHouseBenchmark}, -#endif -#if ENABLE_CLICKHOUSE_SERVER {"server", mainEntryClickHouseServer}, -#endif -#if ENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG {"extract-from-config", mainEntryClickHouseExtractFromConfig}, -#endif -#if ENABLE_CLICKHOUSE_COMPRESSOR {"compressor", mainEntryClickHouseCompressor}, -#endif -#if ENABLE_CLICKHOUSE_FORMAT {"format", mainEntryClickHouseFormat}, -#endif -#if ENABLE_CLICKHOUSE_COPIER {"copier", mainEntryClickHouseClusterCopier}, -#endif -#if ENABLE_CLICKHOUSE_OBFUSCATOR {"obfuscator", mainEntryClickHouseObfuscator}, -#endif -#if ENABLE_CLICKHOUSE_GIT_IMPORT {"git-import", mainEntryClickHouseGitImport}, -#endif + {"static-files-disk-uploader", mainEntryClickHouseStaticFilesDiskUploader}, + {"su", mainEntryClickHouseSU}, + {"hash-binary", mainEntryClickHouseHashBinary}, + {"disks", mainEntryClickHouseDisks}, + + // keeper #if ENABLE_CLICKHOUSE_KEEPER {"keeper", mainEntryClickHouseKeeper}, #endif @@ -137,34 +98,20 @@ std::pair clickhouse_applications[] = #if ENABLE_CLICKHOUSE_KEEPER_CLIENT {"keeper-client", mainEntryClickHouseKeeperClient}, #endif -#if ENABLE_CLICKHOUSE_INSTALL + + // install {"install", mainEntryClickHouseInstall}, {"start", mainEntryClickHouseStart}, {"stop", mainEntryClickHouseStop}, {"status", mainEntryClickHouseStatus}, {"restart", mainEntryClickHouseRestart}, -#endif -#if ENABLE_CLICKHOUSE_STATIC_FILES_DISK_UPLOADER - {"static-files-disk-uploader", mainEntryClickHouseStaticFilesDiskUploader}, -#endif -#if ENABLE_CLICKHOUSE_SU - {"su", mainEntryClickHouseSU}, -#endif - {"hash-binary", mainEntryClickHouseHashBinary}, -#if ENABLE_CLICKHOUSE_DISKS - {"disks", mainEntryClickHouseDisks}, -#endif }; /// Add an item here to register a new short name std::pair clickhouse_short_names[] = { -#if ENABLE_CLICKHOUSE_LOCAL {"chl", "local"}, -#endif -#if ENABLE_CLICKHOUSE_CLIENT {"chc", "client"}, -#endif }; int printHelp(int, char **) diff --git a/programs/su/su.cpp b/programs/su/su.cpp index a8f61fb32b6..33d929898f4 100644 --- a/programs/su/su.cpp +++ b/programs/su/su.cpp @@ -107,6 +107,7 @@ try if (argc < 3) { + std::cout << "A tool similar to 'su'" << std::endl; std::cout << "Usage: ./clickhouse su user:group ..." << std::endl; exit(0); // NOLINT(concurrency-mt-unsafe) } From 67b73c3e496b804ee207ead514e25db079613730 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 12 Feb 2024 11:33:40 +0100 Subject: [PATCH 239/276] Fix long shutdown of FileLog storage Previously it was possible to wait up to poll_directory_watch_events_backoff_max (default is 32000) on shutdown, because it was not possible to stop poll of inotify. Before (takes 3 seconds): 2024.02.12 11:27:55.058192 [ 10134 ] {} StorageFileLog (file_log): Waiting for cleanup 2024.02.12 11:27:58.178021 [ 10271 ] {} directory_watch: Execution took 7519 ms. After: 2024.02.12 11:33:29.722403 [ 15866 ] {} StorageFileLog (file_log): Waiting for cleanup 2024.02.12 11:33:29.722473 [ 15956 ] {} directory_watch: Execution took 6399 ms. Signed-off-by: Azat Khuzhin --- src/Storages/FileLog/DirectoryWatcherBase.cpp | 23 +++++++++++-------- src/Storages/FileLog/DirectoryWatcherBase.h | 11 +++++---- 2 files changed, 20 insertions(+), 14 deletions(-) diff --git a/src/Storages/FileLog/DirectoryWatcherBase.cpp b/src/Storages/FileLog/DirectoryWatcherBase.cpp index 8209483fac9..f1cf0866de7 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.cpp +++ b/src/Storages/FileLog/DirectoryWatcherBase.cpp @@ -34,8 +34,8 @@ DirectoryWatcherBase::DirectoryWatcherBase( if (!std::filesystem::is_directory(path)) throw Exception(ErrorCodes::BAD_FILE_TYPE, "Path {} is not a directory", path); - fd = inotify_init(); - if (fd == -1) + inotify_fd = inotify_init(); + if (inotify_fd == -1) throw ErrnoException(ErrorCodes::IO_SETUP_ERROR, "Cannot initialize inotify"); watch_task = getContext()->getSchedulePool().createTask("directory_watch", [this] { watchFunc(); }); @@ -56,7 +56,7 @@ void DirectoryWatcherBase::watchFunc() if (eventMask() & DirectoryWatcherBase::DW_ITEM_MOVED_TO) mask |= IN_MOVED_TO; - int wd = inotify_add_watch(fd, path.c_str(), mask); + int wd = inotify_add_watch(inotify_fd, path.c_str(), mask); if (wd == -1) { owner.onError(Exception(ErrorCodes::IO_SETUP_ERROR, "Watch directory {} failed", path)); @@ -65,16 +65,20 @@ void DirectoryWatcherBase::watchFunc() std::string buffer; buffer.resize(buffer_size); - pollfd pfd; - pfd.fd = fd; - pfd.events = POLLIN; + pollfd pfds[2]; + /// inotify descriptor + pfds[0].fd = inotify_fd; + pfds[0].events = POLLIN; + // notifier + pfds[1].fd = event_pipe.fds_rw[0]; + pfds[1].events = POLLIN; while (!stopped) { const auto & settings = owner.storage.getFileLogSettings(); - if (poll(&pfd, 1, static_cast(milliseconds_to_wait)) > 0 && pfd.revents & POLLIN) + if (poll(pfds, 2, static_cast(milliseconds_to_wait)) > 0 && pfds[0].revents & POLLIN) { milliseconds_to_wait = settings->poll_directory_watch_events_backoff_init.totalMilliseconds(); - ssize_t n = read(fd, buffer.data(), buffer.size()); + ssize_t n = read(inotify_fd, buffer.data(), buffer.size()); int i = 0; if (n > 0) { @@ -130,7 +134,7 @@ void DirectoryWatcherBase::watchFunc() DirectoryWatcherBase::~DirectoryWatcherBase() { stop(); - int err = ::close(fd); + int err = ::close(inotify_fd); chassert(!err || errno == EINTR); } @@ -143,6 +147,7 @@ void DirectoryWatcherBase::start() void DirectoryWatcherBase::stop() { stopped = true; + ::write(event_pipe.fds_rw[1], "\0", 1); if (watch_task) watch_task->deactivate(); } diff --git a/src/Storages/FileLog/DirectoryWatcherBase.h b/src/Storages/FileLog/DirectoryWatcherBase.h index a640f686c8a..0dfb58fbc5c 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.h +++ b/src/Storages/FileLog/DirectoryWatcherBase.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -85,10 +86,6 @@ public: void watchFunc(); -protected: - void start(); - void stop(); - private: FileLogDirectoryWatcher & owner; @@ -102,7 +99,11 @@ private: int event_mask; uint64_t milliseconds_to_wait; - int fd; + int inotify_fd; + PipeFDs event_pipe; + + void start(); + void stop(); }; } From 3145c5d5f5c045a59faace2a3f07d49f8f44f9f3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 12 Feb 2024 12:02:42 +0100 Subject: [PATCH 240/276] Add missing install target for ch/chc/chl Signed-off-by: Azat Khuzhin --- programs/CMakeLists.txt | 3 +++ 1 file changed, 3 insertions(+) diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index 7ebbf52ea7e..dac168bd226 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -194,11 +194,14 @@ clickhouse_program_install(clickhouse-server server) # client clickhouse_program_install(clickhouse-client client) add_custom_target (chc ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse chc DEPENDS clickhouse) +install (FILES "${CMAKE_CURRENT_BINARY_DIR}/chc" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) # local clickhouse_program_install(clickhouse-local local) add_custom_target (chl ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse chl DEPENDS clickhouse) +install (FILES "${CMAKE_CURRENT_BINARY_DIR}/chl" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) add_custom_target (ch ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse ch DEPENDS clickhouse) +install (FILES "${CMAKE_CURRENT_BINARY_DIR}/ch" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) clickhouse_program_install(clickhouse-benchmark benchmark) clickhouse_program_install(clickhouse-copier copier) From 11fddc8d63b274c33d60b8b9955c7ab70f11fb7c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 12 Feb 2024 12:04:17 +0100 Subject: [PATCH 241/276] Unify binary aliases Signed-off-by: Azat Khuzhin --- programs/CMakeLists.txt | 21 ++++++++------------- 1 file changed, 8 insertions(+), 13 deletions(-) diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index dac168bd226..e68b75db944 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -183,6 +183,12 @@ macro(clickhouse_program_install name lib_name) add_custom_target (${name} ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse ${name} DEPENDS clickhouse) install (FILES "${CMAKE_CURRENT_BINARY_DIR}/${name}" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) list(APPEND CLICKHOUSE_BUNDLE ${name}) + + foreach(alias ${ARGN}) + message(STATUS "Adding alias ${alias} for ${name}") + add_custom_target (${alias} ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse ${alias} DEPENDS clickhouse) + install (FILES "${CMAKE_CURRENT_BINARY_DIR}/${alias}" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) + endforeach() endmacro() if (ENABLE_CLICKHOUSE_SELF_EXTRACTING) @@ -190,19 +196,8 @@ if (ENABLE_CLICKHOUSE_SELF_EXTRACTING) endif () clickhouse_program_install(clickhouse-server server) - -# client -clickhouse_program_install(clickhouse-client client) -add_custom_target (chc ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse chc DEPENDS clickhouse) -install (FILES "${CMAKE_CURRENT_BINARY_DIR}/chc" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) - -# local -clickhouse_program_install(clickhouse-local local) -add_custom_target (chl ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse chl DEPENDS clickhouse) -install (FILES "${CMAKE_CURRENT_BINARY_DIR}/chl" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) -add_custom_target (ch ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse ch DEPENDS clickhouse) -install (FILES "${CMAKE_CURRENT_BINARY_DIR}/ch" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) - +clickhouse_program_install(clickhouse-client client chc) +clickhouse_program_install(clickhouse-local local chl ch) clickhouse_program_install(clickhouse-benchmark benchmark) clickhouse_program_install(clickhouse-copier copier) clickhouse_program_install(clickhouse-extract-from-config extract-from-config) From bc2921d1f11941a8f95fb047d0b0ecdcf692cf31 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 12 Feb 2024 11:53:06 +0100 Subject: [PATCH 242/276] tests: fix 02322_sql_insert_format flakiness 02322_sql_insert_format failed from time to time [1] and I found only one reason - structure cache, I guess it may fail when the mtime was the same. [1]: https://s3.amazonaws.com/clickhouse-test-reports/59857/46a9ced0f9031153538446f4b625e0cc34532a90/fast_test.html Signed-off-by: Azat Khuzhin --- tests/queries/0_stateless/02322_sql_insert_format.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02322_sql_insert_format.sql b/tests/queries/0_stateless/02322_sql_insert_format.sql index 34cde1e56b6..ccceaee31d9 100644 --- a/tests/queries/0_stateless/02322_sql_insert_format.sql +++ b/tests/queries/0_stateless/02322_sql_insert_format.sql @@ -1,5 +1,7 @@ -- Tags: no-parallel +set schema_inference_use_cache_for_file=0; + select number as x, number % 3 as y, 'Hello' as z from numbers(5) format SQLInsert; select number as x, number % 3 as y, 'Hello' as z from numbers(5) format SQLInsert settings output_format_sql_insert_max_batch_size=1; select number as x, number % 3 as y, 'Hello' as z from numbers(5) format SQLInsert settings output_format_sql_insert_max_batch_size=2; From 13e38772546237e94864a55d6ea1ad58e1188591 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 12 Feb 2024 14:30:42 +0100 Subject: [PATCH 243/276] Add chc/chl/ch into clickhouse-bundle target Signed-off-by: Azat Khuzhin --- programs/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index e68b75db944..62bcf068879 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -188,6 +188,7 @@ macro(clickhouse_program_install name lib_name) message(STATUS "Adding alias ${alias} for ${name}") add_custom_target (${alias} ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse ${alias} DEPENDS clickhouse) install (FILES "${CMAKE_CURRENT_BINARY_DIR}/${alias}" DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) + list(APPEND CLICKHOUSE_BUNDLE ${alias}) endforeach() endmacro() From fd2ee19f657276f47d280af83e2bfc95132fe919 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 12 Feb 2024 13:37:25 +0000 Subject: [PATCH 244/276] Follow up for #58554. Cleanup. --- src/Interpreters/ActionsDAG.cpp | 15 ++------------- .../QueryPlan/Optimizations/optimizePrewhere.cpp | 5 ++--- .../MergeTree/MergeTreeWhereOptimizer.cpp | 3 --- 3 files changed, 4 insertions(+), 19 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 4fd7c6d9117..7240679abb7 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1777,15 +1777,6 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split child = child_data.to_second; } - - /// Input from second DAG should also be in the first. - // if (copy.type == ActionType::INPUT) - // { - // auto & input_copy = first_nodes.emplace_back(*cur.node); - // assert(cur_data.to_first == nullptr); - // cur_data.to_first = &input_copy; - // new_inputs.push_back(cur.node); - // } } else { @@ -1837,10 +1828,8 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split for (const auto * input : new_inputs) { const auto & cur = data[input]; - if (cur.to_second) - second_inputs.push_back(cur.to_second); - if (cur.to_first) - first_outputs.push_back(cur.to_first); + second_inputs.push_back(cur.to_second); + first_outputs.push_back(cur.to_first); } for (const auto * input_node : inputs) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index ec07f028f20..49e1a49f131 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -4,9 +4,8 @@ #include #include #include -#include -#include "Functions/FunctionsLogical.h" -#include "Functions/IFunctionAdaptors.h" +#include +#include namespace DB { diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index c52a2fee051..d9a89b9d4ef 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -132,9 +132,6 @@ MergeTreeWhereOptimizer::FilterActionsOptimizeResult MergeTreeWhereOptimizer::op if (!optimize_result) return {}; - // if (optimize_result->where_conditions.empty()) - // return {.prewhere_nodes = {}, .fully_moved_to_prewhere = true}; - std::unordered_set prewhere_conditions; for (const auto & condition : optimize_result->prewhere_conditions) prewhere_conditions.insert(condition.node.getDAGNode()); From 87b6d65b9bdd701f88c99670304c805344c7aa7c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 12 Feb 2024 13:46:30 +0000 Subject: [PATCH 245/276] Remove outdated comment. --- src/Planner/Planner.cpp | 9 --------- 1 file changed, 9 deletions(-) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index efccadcbe1a..bcc42dbae7f 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -71,7 +71,6 @@ #include #include #include -#include #include #include #include @@ -98,14 +97,6 @@ namespace ErrorCodes extern const int SUPPORT_IS_DISABLED; } -/** ClickHouse query planner. - * - * TODO: Support projections. - * TODO: Support trivial count using partition predicates. - * TODO: Support trivial count for table functions. - * TODO: Support indexes for IN function. - */ - namespace { From 2d7fdc896a714c48d990264e17443d6c8834620b Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 12 Feb 2024 14:03:45 +0000 Subject: [PATCH 246/276] Add test 02988_join_using_prewhere_pushdown --- ...988_join_using_prewhere_pushdown.reference | 2 ++ .../02988_join_using_prewhere_pushdown.sql | 24 +++++++++++++++++++ 2 files changed, 26 insertions(+) create mode 100644 tests/queries/0_stateless/02988_join_using_prewhere_pushdown.reference create mode 100644 tests/queries/0_stateless/02988_join_using_prewhere_pushdown.sql diff --git a/tests/queries/0_stateless/02988_join_using_prewhere_pushdown.reference b/tests/queries/0_stateless/02988_join_using_prewhere_pushdown.reference new file mode 100644 index 00000000000..c9bf491872a --- /dev/null +++ b/tests/queries/0_stateless/02988_join_using_prewhere_pushdown.reference @@ -0,0 +1,2 @@ +1 a +2 b Int64 diff --git a/tests/queries/0_stateless/02988_join_using_prewhere_pushdown.sql b/tests/queries/0_stateless/02988_join_using_prewhere_pushdown.sql new file mode 100644 index 00000000000..db49f155d3f --- /dev/null +++ b/tests/queries/0_stateless/02988_join_using_prewhere_pushdown.sql @@ -0,0 +1,24 @@ +DROP TABLE IF EXISTS t; + +SET allow_suspicious_low_cardinality_types = 1; + + +CREATE TABLE t (`id` UInt16, `u` LowCardinality(Int32), `s` LowCardinality(String)) +ENGINE = MergeTree ORDER BY id; + +INSERT INTO t VALUES (1,1,'a'),(2,2,'b'); + +SELECT u, s FROM t +INNER JOIN ( SELECT number :: Int32 AS u FROM numbers(10) ) AS t1 +USING (u) +WHERE u != 2 +; + +SELECT u, s, toTypeName(u) FROM t +FULL JOIN ( SELECT number :: UInt32 AS u FROM numbers(10) ) AS t1 +USING (u) +WHERE u == 2 +ORDER BY 1 +; + +DROP TABLE IF EXISTS t; From c8807393114b870bf94c751ed32d054741cf22b0 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Mon, 12 Feb 2024 13:44:14 +0000 Subject: [PATCH 247/276] CI: Fix build job failures due to jepsen artifacts --- tests/jepsen.clickhouse/project.clj | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/jepsen.clickhouse/project.clj b/tests/jepsen.clickhouse/project.clj index 6c714604b56..bb41be1ba10 100644 --- a/tests/jepsen.clickhouse/project.clj +++ b/tests/jepsen.clickhouse/project.clj @@ -13,4 +13,7 @@ [com.hierynomus/sshj "0.34.0"] [com.clickhouse/clickhouse-jdbc "0.3.2-patch11"] [org.apache.zookeeper/zookeeper "3.6.1" :exclusions [org.slf4j/slf4j-log4j12]]] - :repl-options {:init-ns jepsen.clickhouse-keeper.main}) + :repl-options {:init-ns jepsen.clickhouse-keeper.main} + ;; otherwise, target artifacts will be created under the repo root, so that checkout with clear might fail in ci + :target-path "/tmp/jepsen_clickhouse" +) From b094ab9763120ed03015e27eeb2f13247330a017 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 12 Feb 2024 15:27:20 +0000 Subject: [PATCH 248/276] Add comment --- tests/clickhouse-test | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 49c517852a6..9c21f1fd2a2 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -2494,7 +2494,8 @@ def main(args): time DateTime, test_name String, coverage Array(UInt64) - ) ENGINE = MergeTree ORDER BY test_name; + ) ENGINE = MergeTree ORDER BY test_name + COMMENT 'Contains information about per-test coverage from the CI, but used only for exporting to the CI cluster'; """, ) From 0628ae62681d970a37414a251e0894de1e3b4569 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 12 Feb 2024 17:13:30 +0100 Subject: [PATCH 249/276] S3 queue fix uninitialized value --- src/Storages/S3Queue/S3QueueTableMetadata.cpp | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.cpp b/src/Storages/S3Queue/S3QueueTableMetadata.cpp index 3ee2594135d..1830bac4743 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueTableMetadata.cpp @@ -69,16 +69,23 @@ void S3QueueTableMetadata::read(const String & metadata_str) { Poco::JSON::Parser parser; auto json = parser.parse(metadata_str).extract(); + after_processing = json->getValue("after_processing"); mode = json->getValue("mode"); s3queue_tracked_files_limit = json->getValue("s3queue_tracked_files_limit"); s3queue_tracked_file_ttl_sec = json->getValue("s3queue_tracked_file_ttl_sec"); format_name = json->getValue("format_name"); columns = json->getValue("columns"); + if (json->has("s3queue_total_shards_num")) s3queue_total_shards_num = json->getValue("s3queue_total_shards_num"); + else + s3queue_total_shards_num = 1; + if (json->has("s3queue_processing_threads_num")) s3queue_processing_threads_num = json->getValue("s3queue_processing_threads_num"); + else + s3queue_processing_threads_num = 1; } S3QueueTableMetadata S3QueueTableMetadata::parse(const String & metadata_str) From 69273b46451a288f6f686b3e1238d8992e9ce0de Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 12 Feb 2024 17:10:10 +0000 Subject: [PATCH 250/276] Fix parsing of partition expressions surrounded by parens --- src/Parsers/ParserPartition.cpp | 34 +++++++++---------- ...02897_alter_partition_parameters.reference | 5 +++ .../02897_alter_partition_parameters.sql | 29 ++++++++++++++++ 3 files changed, 51 insertions(+), 17 deletions(-) diff --git a/src/Parsers/ParserPartition.cpp b/src/Parsers/ParserPartition.cpp index 80debc13c67..0cbd6898dd9 100644 --- a/src/Parsers/ParserPartition.cpp +++ b/src/Parsers/ParserPartition.cpp @@ -8,6 +8,7 @@ #include #include #include +#include namespace DB { @@ -18,8 +19,6 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_all("ALL"); ParserStringLiteral parser_string_literal; ParserSubstitution parser_substitution; - ParserLiteral literal_parser; - ParserTupleOfLiterals tuple_of_literals; ParserExpression parser_expr; auto partition = std::make_shared(); @@ -45,34 +44,35 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ASTPtr value; std::optional fields_count; - if (literal_parser.parse(pos, value, expected) || tuple_of_literals.parse(pos, value, expected)) - { - auto * literal = value->as(); - if (literal->value.getType() == Field::Types::Tuple) - { - fields_count = literal->value.get().size(); - } - else - { - fields_count = 1; - } - } - else if (parser_substitution.parse(pos, value, expected)) + if (parser_substitution.parse(pos, value, expected)) { /// It can be tuple substitution fields_count = std::nullopt; } else if (parser_expr.parse(pos, value, expected)) { - const auto * tuple_ast = value->as(); - if (tuple_ast && tuple_ast->name == "tuple") + if (const auto * tuple_ast = value->as(); tuple_ast) { + if (tuple_ast->name != "tuple") + return false; + const auto * arguments_ast = tuple_ast->arguments->as(); if (arguments_ast) fields_count = arguments_ast->children.size(); else fields_count = 0; } + else if (const auto* literal_ast = value->as(); literal_ast) + { + if (literal_ast->value.getType() == Field::Types::Tuple) + { + fields_count = literal_ast->value.get().size(); + } + else + { + fields_count = 1; + } + } else return false; } diff --git a/tests/queries/0_stateless/02897_alter_partition_parameters.reference b/tests/queries/0_stateless/02897_alter_partition_parameters.reference index bc6ff2b709c..d4b70c58ae5 100644 --- a/tests/queries/0_stateless/02897_alter_partition_parameters.reference +++ b/tests/queries/0_stateless/02897_alter_partition_parameters.reference @@ -7,3 +7,8 @@ 0 0 0 +0 +0 +0 +0 +0 diff --git a/tests/queries/0_stateless/02897_alter_partition_parameters.sql b/tests/queries/0_stateless/02897_alter_partition_parameters.sql index 62ceb9d9768..0be7308ed1a 100644 --- a/tests/queries/0_stateless/02897_alter_partition_parameters.sql +++ b/tests/queries/0_stateless/02897_alter_partition_parameters.sql @@ -10,6 +10,24 @@ PARTITION BY toMonday(EventDate); INSERT INTO test VALUES(toDate('2023-10-09')); +ALTER TABLE test DROP PARTITION ('2023-10-09'); + +SELECT count() FROM test; + +INSERT INTO test VALUES(toDate('2023-10-09')); + +ALTER TABLE test DROP PARTITION (('2023-10-09')); + +SELECT count() FROM test; + +INSERT INTO test VALUES(toDate('2023-10-09')); + +ALTER TABLE test DROP PARTITION '2023-10-09'; + +SELECT count() FROM test; + +INSERT INTO test VALUES(toDate('2023-10-09')); + SET param_partition='2023-10-09'; ALTER TABLE test DROP PARTITION {partition:String}; @@ -51,6 +69,17 @@ ENGINE = MergeTree ORDER BY tuple() PARTITION BY (a * b, b * b); +INSERT INTO test2 VALUES(1, 2); + +ALTER TABLE test2 DROP PARTITION tuple(2, 4); + +SELECT count() FROM test2; + +INSERT INTO test2 VALUES(1, 2); + +ALTER TABLE test2 DROP PARTITION (2, 4); + +SELECT count() FROM test2; INSERT INTO test2 VALUES(1, 2); From f5a71455b86210c8ec9968071f63b88ae434da5e Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 12 Feb 2024 17:56:05 +0100 Subject: [PATCH 251/276] Do not rebuild a lambda package if it is updated --- .../ci/team_keys_lambda/build_and_deploy_archive.sh | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/tests/ci/team_keys_lambda/build_and_deploy_archive.sh b/tests/ci/team_keys_lambda/build_and_deploy_archive.sh index 3c6c8e0ac1e..6ba0987010a 100644 --- a/tests/ci/team_keys_lambda/build_and_deploy_archive.sh +++ b/tests/ci/team_keys_lambda/build_and_deploy_archive.sh @@ -17,6 +17,18 @@ DOCKER_IMAGE="public.ecr.aws/lambda/python:${PY_VERSION}" LAMBDA_NAME=${DIR_NAME//_/-} # The name of directory with lambda code PACKAGE=lambda-package + +# Do not rebuild and deploy the archive if it's newer than sources +if [ -e "$PACKAGE.zip" ] && [ -z "$FORCE" ]; then + REBUILD="" + for src in app.py build_and_deploy_archive.sh requirements.txt lambda_shared/*; do + if [ "$src" -nt "$PACKAGE.zip" ]; then + REBUILD=1 + fi + done + [ -n "$REBUILD" ] || exit 0 +fi + rm -rf "$PACKAGE" "$PACKAGE".zip mkdir "$PACKAGE" cp app.py "$PACKAGE" From 310b0773b271d165dd152da1474ba4a892b05b54 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 12 Feb 2024 17:25:17 +0000 Subject: [PATCH 252/276] Fix include used for debugging --- src/Parsers/ParserPartition.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Parsers/ParserPartition.cpp b/src/Parsers/ParserPartition.cpp index 0cbd6898dd9..f7d972dd4af 100644 --- a/src/Parsers/ParserPartition.cpp +++ b/src/Parsers/ParserPartition.cpp @@ -8,7 +8,6 @@ #include #include #include -#include namespace DB { From d7850db40c8491d1f023dd5f532eee9ee1e8b80a Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Mon, 12 Feb 2024 09:54:38 -0800 Subject: [PATCH 253/276] [Docs] Remove incorrect statement about Memory table engine --- docs/en/engines/table-engines/special/memory.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/en/engines/table-engines/special/memory.md b/docs/en/engines/table-engines/special/memory.md index 54547b1bc69..0d552a69804 100644 --- a/docs/en/engines/table-engines/special/memory.md +++ b/docs/en/engines/table-engines/special/memory.md @@ -10,7 +10,6 @@ sidebar_label: Memory When using the Memory table engine on ClickHouse Cloud, data is not replicated across all nodes (by design). To guarantee that all queries are routed to the same node and that the Memory table engine works as expected, you can do one of the following: - Execute all operations in the same session - Use a client that uses TCP or the native interface (which enables support for sticky connections) such as [clickhouse-client](/en/interfaces/cli) -- Submit and execute all queries at once using a multi-statement query (required with clients using the HTTP interface such as [clickhouse-connect](/en/integrations/python)) ::: The Memory engine stores data in RAM, in uncompressed form. Data is stored in exactly the same form as it is received when read. In other words, reading from this table is completely free. From 3aefd0f50bc1cbfd6e6c84ab6196417f8e062aa0 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 12 Feb 2024 18:01:43 +0000 Subject: [PATCH 254/276] copy on mac instead renaming --- utils/self-extracting-executable/decompressor.cpp | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 6614403c0ab..071ecb066cb 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -529,14 +529,22 @@ int main(int/* argc*/, char* argv[]) char decompressed_name[decompressed_name_len + 1]; (void)snprintf(decompressed_name, decompressed_name_len + 1, decompressed_name_fmt, self, decompressed_suffix); +#if defined(OS_DARWIN) + // We can't just rename it on Mac due to security issues, so we copy it... std::error_code ec; - + std::filesystem::copy_file(static_cast(decompressed_name), static_cast(self), ec); + if (ec) + { + std::cerr << ec.message() << std::endl; + return 1; + } +#else if (link(decompressed_name, self)) { perror("link"); return 1; } - +#endif if (chmod(self, static_cast(decompressed_umask))) { perror("chmod"); From 6a3c3624435d749c6a83caa0c4f03c67d9f25b50 Mon Sep 17 00:00:00 2001 From: MyroTk <44327070+MyroTk@users.noreply.github.com> Date: Mon, 12 Feb 2024 10:39:26 -0800 Subject: [PATCH 255/276] Update Dockerfile --- docker/test/integration/runner/Dockerfile | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index b876f7b9635..473278104b2 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -62,6 +62,7 @@ RUN curl -fsSL https://download.docker.com/linux/ubuntu/gpg | apt-key add - \ # kazoo 2.10.0 is broken # https://s3.amazonaws.com/clickhouse-test-reports/59337/524625a1d2f4cc608a3f1059e3df2c30f353a649/integration_tests__asan__analyzer__[5_6].html RUN python3 -m pip install --no-cache-dir \ + aerospike==11.1.0 \ PyMySQL==1.1.0 \ asyncio==3.4.3 \ avro==1.10.2 \ From d008ee725f7d1e1bad802c5153111e41622481b1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 12 Feb 2024 20:23:21 +0100 Subject: [PATCH 256/276] Add a test --- src/Storages/S3Queue/S3QueueTableMetadata.h | 8 +-- .../integration/test_storage_s3_queue/test.py | 50 +++++++++++++++++++ 2 files changed, 54 insertions(+), 4 deletions(-) diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.h b/src/Storages/S3Queue/S3QueueTableMetadata.h index 30642869930..84087f72a6a 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.h +++ b/src/Storages/S3Queue/S3QueueTableMetadata.h @@ -21,10 +21,10 @@ struct S3QueueTableMetadata String columns; String after_processing; String mode; - UInt64 s3queue_tracked_files_limit; - UInt64 s3queue_tracked_file_ttl_sec; - UInt64 s3queue_total_shards_num; - UInt64 s3queue_processing_threads_num; + UInt64 s3queue_tracked_files_limit = 0; + UInt64 s3queue_tracked_file_ttl_sec = 0; + UInt64 s3queue_total_shards_num = 1; + UInt64 s3queue_processing_threads_num = 1; S3QueueTableMetadata() = default; S3QueueTableMetadata(const StorageS3::Configuration & configuration, const S3QueueSettings & engine_settings, const StorageInMemoryMetadata & storage_metadata); diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 810c4f29e9d..a7abd840834 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -101,6 +101,15 @@ def started_cluster(): ], stay_alive=True, ) + cluster.add_instance( + "old_instance", + with_zookeeper=True, + image="clickhouse/clickhouse-server", + tag="23.12", + stay_alive=True, + with_installed_binary=True, + allow_analyzer=False, + ) logging.info("Starting cluster...") cluster.start() @@ -1386,3 +1395,44 @@ def test_processed_file_setting_distributed(started_cluster, processing_threads) break time.sleep(1) assert expected_rows == get_count() + + +def test_upgrade(started_cluster): + node = started_cluster.instances["old_instance"] + + table_name = f"test_upgrade" + dst_table_name = f"{table_name}_dst" + keeper_path = f"/clickhouse/test_{table_name}" + files_path = f"{table_name}_data" + files_to_generate = 10 + + create_table( + started_cluster, + node, + table_name, + "ordered", + files_path, + additional_settings={ + "keeper_path": keeper_path, + }, + ) + total_values = generate_random_files( + started_cluster, files_path, files_to_generate, start_ind=0, row_num=1 + ) + + create_mv(node, table_name, dst_table_name) + + def get_count(): + return int(node.query(f"SELECT count() FROM {dst_table_name}")) + + expected_rows = 10 + for _ in range(20): + if expected_rows == get_count(): + break + time.sleep(1) + + assert expected_rows == get_count() + + node.restart_with_latest_version() + + assert expected_rows == get_count() From 50db80a7e3bfd4a0c6135faca865e3bd6d6e6c95 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 12 Feb 2024 19:36:41 +0000 Subject: [PATCH 257/276] Update tests with indexHint for analyzer. --- tests/queries/0_stateless/01739_index_hint.reference | 5 ++++- tests/queries/0_stateless/01739_index_hint.sql | 4 +++- .../0_stateless/02880_indexHint__partition_id.reference | 5 +++-- tests/queries/0_stateless/02880_indexHint__partition_id.sql | 5 +++-- 4 files changed, 13 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01739_index_hint.reference b/tests/queries/0_stateless/01739_index_hint.reference index 21673bf698b..21f4edc0049 100644 --- a/tests/queries/0_stateless/01739_index_hint.reference +++ b/tests/queries/0_stateless/01739_index_hint.reference @@ -35,6 +35,9 @@ SELECT count() FROM XXXX WHERE indexHint(t = toDateTime(0)) SETTINGS optimize_us drop table XXXX; CREATE TABLE XXXX (p Nullable(Int64), k Decimal(76, 39)) ENGINE = MergeTree PARTITION BY toDate(p) ORDER BY k SETTINGS index_granularity = 1, allow_nullable_key = 1; INSERT INTO XXXX FORMAT Values ('2020-09-01 00:01:02', 1), ('2020-09-01 20:01:03', 2), ('2020-09-02 00:01:03', 3); -SELECT count() FROM XXXX WHERE indexHint(p = 1.) SETTINGS optimize_use_implicit_projections = 1; +SELECT count() FROM XXXX WHERE indexHint(p = 1.) SETTINGS optimize_use_implicit_projections = 1, allow_experimental_analyzer=0; 0 +-- TODO: optimize_use_implicit_projections ignores indexHint (with analyzer) because source columns might be aliased. +SELECT count() FROM XXXX WHERE indexHint(p = 1.) SETTINGS optimize_use_implicit_projections = 1, allow_experimental_analyzer=1; +3 drop table XXXX; diff --git a/tests/queries/0_stateless/01739_index_hint.sql b/tests/queries/0_stateless/01739_index_hint.sql index cde46a5a2bf..1eca65f0892 100644 --- a/tests/queries/0_stateless/01739_index_hint.sql +++ b/tests/queries/0_stateless/01739_index_hint.sql @@ -38,6 +38,8 @@ CREATE TABLE XXXX (p Nullable(Int64), k Decimal(76, 39)) ENGINE = MergeTree PART INSERT INTO XXXX FORMAT Values ('2020-09-01 00:01:02', 1), ('2020-09-01 20:01:03', 2), ('2020-09-02 00:01:03', 3); -SELECT count() FROM XXXX WHERE indexHint(p = 1.) SETTINGS optimize_use_implicit_projections = 1; +SELECT count() FROM XXXX WHERE indexHint(p = 1.) SETTINGS optimize_use_implicit_projections = 1, allow_experimental_analyzer=0; +-- TODO: optimize_use_implicit_projections ignores indexHint (with analyzer) because source columns might be aliased. +SELECT count() FROM XXXX WHERE indexHint(p = 1.) SETTINGS optimize_use_implicit_projections = 1, allow_experimental_analyzer=1; drop table XXXX; diff --git a/tests/queries/0_stateless/02880_indexHint__partition_id.reference b/tests/queries/0_stateless/02880_indexHint__partition_id.reference index 365e7b676c7..2cdd2cc1954 100644 --- a/tests/queries/0_stateless/02880_indexHint__partition_id.reference +++ b/tests/queries/0_stateless/02880_indexHint__partition_id.reference @@ -1,9 +1,10 @@ -- { echoOn } select * from data prewhere indexHint(_partition_id = '1'); 1 -select count() from data prewhere indexHint(_partition_id = '1'); +-- TODO: optimize_use_implicit_projections ignores indexHint (with analyzer) because source columns might be aliased. +select count() from data prewhere indexHint(_partition_id = '1') settings optimize_use_implicit_projections = 0; 1 select * from data where indexHint(_partition_id = '1'); 1 -select count() from data where indexHint(_partition_id = '1'); +select count() from data where indexHint(_partition_id = '1') settings optimize_use_implicit_projections = 0; 1 diff --git a/tests/queries/0_stateless/02880_indexHint__partition_id.sql b/tests/queries/0_stateless/02880_indexHint__partition_id.sql index d15b3f4ccea..9d5dc7bcbc2 100644 --- a/tests/queries/0_stateless/02880_indexHint__partition_id.sql +++ b/tests/queries/0_stateless/02880_indexHint__partition_id.sql @@ -4,6 +4,7 @@ insert into data values (1)(2); -- { echoOn } select * from data prewhere indexHint(_partition_id = '1'); -select count() from data prewhere indexHint(_partition_id = '1'); +-- TODO: optimize_use_implicit_projections ignores indexHint (with analyzer) because source columns might be aliased. +select count() from data prewhere indexHint(_partition_id = '1') settings optimize_use_implicit_projections = 0; select * from data where indexHint(_partition_id = '1'); -select count() from data where indexHint(_partition_id = '1'); +select count() from data where indexHint(_partition_id = '1') settings optimize_use_implicit_projections = 0; From 4d220322cb4a48f8817f3e1ea4296223b2c53edd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 12 Feb 2024 19:48:02 +0000 Subject: [PATCH 258/276] Update analyzer_tech_debt.txt --- tests/analyzer_tech_debt.txt | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/analyzer_tech_debt.txt b/tests/analyzer_tech_debt.txt index 53154085b62..29331d674c8 100644 --- a/tests/analyzer_tech_debt.txt +++ b/tests/analyzer_tech_debt.txt @@ -8,8 +8,6 @@ 01584_distributed_buffer_cannot_find_column 01624_soft_constraints 01656_test_query_log_factories_info -01739_index_hint -02880_indexHint__partition_id 01747_join_view_filter_dictionary 01761_cast_to_enum_nullable 01925_join_materialized_columns From 5868cdc708f5815ee22a298473898679ccf31146 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 12 Feb 2024 20:11:44 +0100 Subject: [PATCH 259/276] Remove one unused implementation of Macros::expand() Signed-off-by: Azat Khuzhin --- src/Common/Macros.cpp | 9 --------- src/Common/Macros.h | 2 -- 2 files changed, 11 deletions(-) diff --git a/src/Common/Macros.cpp b/src/Common/Macros.cpp index 9e0977d9bcc..1d95037b38b 100644 --- a/src/Common/Macros.cpp +++ b/src/Common/Macros.cpp @@ -175,15 +175,6 @@ String Macros::expand(const String & s) const return expand(s, info); } -String Macros::expand(const String & s, const StorageID & table_id, bool allow_uuid) const -{ - MacroExpansionInfo info; - info.table_id = table_id; - if (!allow_uuid) - info.table_id.uuid = UUIDHelpers::Nil; - return expand(s, info); -} - Names Macros::expand(const Names & source_names, size_t level) const { Names result_names; diff --git a/src/Common/Macros.h b/src/Common/Macros.h index 8b9eded7dcb..4f72932bdfd 100644 --- a/src/Common/Macros.h +++ b/src/Common/Macros.h @@ -57,8 +57,6 @@ public: String expand(const String & s) const; - String expand(const String & s, const StorageID & table_id, bool allow_uuid) const; - /** Apply expand for the list. */ From c61ac1d3bc9f26ca30dfee7666de230833b855bc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 12 Feb 2024 20:30:12 +0100 Subject: [PATCH 260/276] Convert default_replica_path/default_replica_name into server settings Signed-off-by: Azat Khuzhin --- programs/server/config.xml | 5 +++++ src/Backups/DDLAdjustingForBackupVisitor.cpp | 6 +++--- src/Core/ServerSettings.h | 2 ++ src/Storages/MergeTree/registerStorageMergeTree.cpp | 6 +++--- src/Storages/StorageReplicatedMergeTree.cpp | 12 ------------ src/Storages/StorageReplicatedMergeTree.h | 3 --- 6 files changed, 13 insertions(+), 21 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index 6a40818332b..23f3458110e 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -937,6 +937,11 @@ --> + +