From 9efeddfdd12eb83736626af95de36b6ada913fc4 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 25 Jun 2024 21:11:56 +0000 Subject: [PATCH 01/50] fix ListObject in azure blob storage --- src/Backups/BackupIO_AzureBlobStorage.cpp | 7 +- src/Backups/BackupIO_AzureBlobStorage.h | 4 +- .../IO/ReadBufferFromAzureBlobStorage.cpp | 6 +- src/Disks/IO/ReadBufferFromAzureBlobStorage.h | 10 +-- .../IO/WriteBufferFromAzureBlobStorage.cpp | 2 +- .../IO/WriteBufferFromAzureBlobStorage.h | 2 +- .../AzureBlobStorageCommon.cpp | 44 +++++++++++-- .../AzureBlobStorage/AzureBlobStorageCommon.h | 39 ++++++++--- .../AzureBlobStorage/AzureObjectStorage.cpp | 32 +++++----- .../AzureBlobStorage/AzureObjectStorage.h | 6 +- .../Cached/CachedObjectStorage.h | 2 +- src/Disks/ObjectStorages/IObjectStorage.h | 5 +- .../MetadataStorageFromPlainObjectStorage.cpp | 1 - .../ObjectStorages/ObjectStorageFactory.cpp | 2 +- .../copyAzureBlobStorageFile.cpp | 14 ++-- .../copyAzureBlobStorageFile.h | 6 +- .../03008_azure_plain_rewritable.reference | 11 ++++ .../03008_azure_plain_rewritable.sh | 64 +++++++++++++++++++ 18 files changed, 194 insertions(+), 63 deletions(-) create mode 100644 tests/queries/0_stateless/03008_azure_plain_rewritable.reference create mode 100755 tests/queries/0_stateless/03008_azure_plain_rewritable.sh diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index 0ee0160a969..0459afb993a 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -234,11 +234,8 @@ bool BackupWriterAzureBlobStorage::fileExists(const String & file_name) UInt64 BackupWriterAzureBlobStorage::getFileSize(const String & file_name) { String key = fs::path(blob_path) / 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; + ObjectMetadata object_metadata = object_storage->getObjectMetadata(key); + return object_metadata.size_bytes; } std::unique_ptr BackupWriterAzureBlobStorage::readFile(const String & file_name, size_t /*expected_file_size*/) diff --git a/src/Backups/BackupIO_AzureBlobStorage.h b/src/Backups/BackupIO_AzureBlobStorage.h index c3b88f245ab..1dcc9c52d6b 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.h +++ b/src/Backups/BackupIO_AzureBlobStorage.h @@ -38,7 +38,7 @@ public: private: const DataSourceDescription data_source_description; - std::shared_ptr client; + std::shared_ptr client; AzureBlobStorage::ConnectionParams connection_params; String blob_path; std::unique_ptr object_storage; @@ -87,7 +87,7 @@ private: void removeFilesBatch(const Strings & file_names); const DataSourceDescription data_source_description; - std::shared_ptr client; + std::shared_ptr client; AzureBlobStorage::ConnectionParams connection_params; String blob_path; std::unique_ptr object_storage; diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index da1ea65f2ea..7a547dcb18d 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -36,7 +36,7 @@ namespace ErrorCodes } ReadBufferFromAzureBlobStorage::ReadBufferFromAzureBlobStorage( - std::shared_ptr blob_container_client_, + ContainerClientPtr blob_container_client_, const String & path_, const ReadSettings & read_settings_, size_t max_single_read_retries_, @@ -225,7 +225,7 @@ void ReadBufferFromAzureBlobStorage::initialize() try { ProfileEvents::increment(ProfileEvents::AzureGetObject); - if (blob_container_client->GetClickhouseOptions().IsClientForDisk) + if (blob_container_client->IsClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskAzureGetObject); auto download_response = blob_client->Download(download_options); @@ -279,7 +279,7 @@ size_t ReadBufferFromAzureBlobStorage::readBigAt(char * to, size_t n, size_t ran try { ProfileEvents::increment(ProfileEvents::AzureGetObject); - if (blob_container_client->GetClickhouseOptions().IsClientForDisk) + if (blob_container_client->IsClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskAzureGetObject); Azure::Storage::Blobs::DownloadBlobOptions download_options; diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.h b/src/Disks/IO/ReadBufferFromAzureBlobStorage.h index d328195cc26..d3cf1382e9e 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.h +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.h @@ -8,7 +8,7 @@ #include #include #include -#include +#include namespace DB { @@ -16,9 +16,11 @@ namespace DB class ReadBufferFromAzureBlobStorage : public ReadBufferFromFileBase { public: + using ContainerClientPtr = std::shared_ptr; + using BlobClientPtr = std::unique_ptr; ReadBufferFromAzureBlobStorage( - std::shared_ptr blob_container_client_, + ContainerClientPtr blob_container_client_, const String & path_, const ReadSettings & read_settings_, size_t max_single_read_retries_, @@ -53,8 +55,8 @@ private: void initialize(); std::unique_ptr data_stream; - std::shared_ptr blob_container_client; - std::unique_ptr blob_client; + ContainerClientPtr blob_container_client; + BlobClientPtr blob_client; const String path; size_t max_single_read_retries; diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index c4d9796df3a..4780796e970 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -44,7 +44,7 @@ BufferAllocationPolicyPtr createBufferAllocationPolicy(const AzureBlobStorage::R } WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage( - std::shared_ptr blob_container_client_, + AzureClientPtr blob_container_client_, const String & blob_path_, size_t buf_size_, const WriteSettings & write_settings_, diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h index 3ee497c4e44..51df974e7df 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h @@ -28,7 +28,7 @@ class TaskTracker; class WriteBufferFromAzureBlobStorage : public WriteBufferFromFileBase { public: - using AzureClientPtr = std::shared_ptr; + using AzureClientPtr = std::shared_ptr; WriteBufferFromAzureBlobStorage( AzureClientPtr blob_container_client_, diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp index d9dfedadd48..dd1ab168d84 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp @@ -1,4 +1,6 @@ #include +#include "Common/StackTrace.h" +#include "Common/logger_useful.h" #if USE_AZURE_BLOB_STORAGE @@ -52,6 +54,33 @@ static bool isConnectionString(const std::string & candidate) return !candidate.starts_with("http"); } +ContainerClientWrapper::ContainerClientWrapper(RawContainerClient client_, String blob_prefix_) + : client(std::move(client_)), blob_prefix(std::move(blob_prefix_)) +{ +} + +BlobClient ContainerClientWrapper::GetBlobClient(const String & blob_name) const +{ + return client.GetBlobClient(blob_prefix / blob_name); +} + +BlockBlobClient ContainerClientWrapper::GetBlockBlobClient(const String & blob_name) const +{ + return client.GetBlockBlobClient(blob_prefix / blob_name); +} + +ListBlobsPagedResponse ContainerClientWrapper::ListBlobs(const ListBlobsOptions & options) const +{ + auto new_options = options; + new_options.Prefix = blob_prefix / options.Prefix.ValueOr(""); + return client.ListBlobs(new_options); +} + +bool ContainerClientWrapper::IsClientForDisk() const +{ + return client.GetClickhouseOptions().IsClientForDisk; +} + String ConnectionParams::getConnectionURL() const { if (std::holds_alternative(auth_method)) @@ -70,7 +99,7 @@ std::unique_ptr ConnectionParams::createForService() const if constexpr (std::is_same_v) return std::make_unique(ServiceClient::CreateFromConnectionString(auth.toUnderType(), client_options)); else - return std::make_unique(endpoint.getEndpointWithoutContainer(), auth, client_options); + return std::make_unique(endpoint.getServiceEndpoint(), auth, client_options); }, auth_method); } @@ -79,9 +108,15 @@ std::unique_ptr ConnectionParams::createForContainer() const return std::visit([this](const T & auth) { if constexpr (std::is_same_v) - return std::make_unique(ContainerClient::CreateFromConnectionString(auth.toUnderType(), endpoint.container_name, client_options)); + { + auto raw_client = RawContainerClient::CreateFromConnectionString(auth.toUnderType(), endpoint.container_name, client_options); + return std::make_unique(std::move(raw_client), endpoint.prefix); + } else - return std::make_unique(endpoint.getEndpoint(), auth, client_options); + { + RawContainerClient raw_client{endpoint.getContainerEndpoint(), auth, client_options}; + return std::make_unique(std::move(raw_client), endpoint.prefix); + } }, auth_method); } @@ -221,7 +256,8 @@ std::unique_ptr getContainerClient(const ConnectionParams & par try { auto service_client = params.createForService(); - return std::make_unique(service_client->CreateBlobContainer(params.endpoint.container_name).Value); + auto raw_client = service_client->CreateBlobContainer(params.endpoint.container_name).Value; + return std::make_unique(std::move(raw_client), params.endpoint.prefix); } catch (const Azure::Storage::StorageException & e) { diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h index 19ba48ea225..a77087d8557 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h @@ -16,17 +16,14 @@ #include #include +namespace fs = std::filesystem; + namespace DB { namespace AzureBlobStorage { -using ServiceClient = Azure::Storage::Blobs::BlobServiceClient; -using ContainerClient = Azure::Storage::Blobs::BlobContainerClient; -using BlobClient = Azure::Storage::Blobs::BlobClient; -using BlobClientOptions = Azure::Storage::Blobs::BlobClientOptions; - struct RequestSettings { RequestSettings() = default; @@ -63,7 +60,7 @@ struct Endpoint String sas_auth; std::optional container_already_exists; - String getEndpoint() const + String getContainerEndpoint() const { String url = storage_account_url; if (url.ends_with('/')) @@ -75,16 +72,13 @@ struct Endpoint if (!container_name.empty()) url += "/" + container_name; - if (!prefix.empty()) - url += "/" + prefix; - if (!sas_auth.empty()) url += "?" + sas_auth; return url; } - String getEndpointWithoutContainer() const + String getServiceEndpoint() const { String url = storage_account_url; @@ -98,6 +92,31 @@ struct Endpoint } }; +using BlobClient = Azure::Storage::Blobs::BlobClient; +using BlockBlobClient = Azure::Storage::Blobs::BlockBlobClient; +using RawContainerClient = Azure::Storage::Blobs::BlobContainerClient; + +using Azure::Storage::Blobs::ListBlobsOptions; +using Azure::Storage::Blobs::ListBlobsPagedResponse; + +class ContainerClientWrapper +{ +public: + ContainerClientWrapper(RawContainerClient client_, String blob_prefix_); + + bool IsClientForDisk() const; + BlobClient GetBlobClient(const String & blob_name) const; + BlockBlobClient GetBlockBlobClient(const String & blob_name) const; + ListBlobsPagedResponse ListBlobs(const ListBlobsOptions & options) const; + +private: + RawContainerClient client; + fs::path blob_prefix; +}; + +using ContainerClient = ContainerClientWrapper; +using ServiceClient = Azure::Storage::Blobs::BlobServiceClient; +using BlobClientOptions = Azure::Storage::Blobs::BlobClientOptions; using ConnectionString = StrongTypedef; using AuthMethod = std::variant< diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 648d7c459c7..ed5d5dbe567 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -12,7 +12,6 @@ #include #include #include -#include namespace CurrentMetrics @@ -51,7 +50,7 @@ class AzureIteratorAsync final : public IObjectStorageIteratorAsync public: AzureIteratorAsync( const std::string & path_prefix, - std::shared_ptr client_, + std::shared_ptr client_, size_t max_list_size) : IObjectStorageIteratorAsync( CurrentMetrics::ObjectStorageAzureThreads, @@ -69,7 +68,7 @@ private: bool getBatchAndCheckNext(RelativePathsWithMetadata & batch) override { ProfileEvents::increment(ProfileEvents::AzureListObjects); - if (client->GetClickhouseOptions().IsClientForDisk) + if (client->IsClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskAzureListObjects); batch.clear(); @@ -96,7 +95,7 @@ private: return true; } - std::shared_ptr client; + std::shared_ptr client; Azure::Storage::Blobs::ListBlobsOptions options; }; @@ -128,7 +127,7 @@ bool AzureObjectStorage::exists(const StoredObject & object) const auto client_ptr = client.get(); ProfileEvents::increment(ProfileEvents::AzureGetProperties); - if (client_ptr->GetClickhouseOptions().IsClientForDisk) + if (client_ptr->IsClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskAzureGetProperties); try @@ -170,7 +169,7 @@ void AzureObjectStorage::listObjects(const std::string & path, RelativePathsWith for (auto blob_list_response = client_ptr->ListBlobs(options); blob_list_response.HasPage(); blob_list_response.MoveToNextPage()) { ProfileEvents::increment(ProfileEvents::AzureListObjects); - if (client_ptr->GetClickhouseOptions().IsClientForDisk) + if (client_ptr->IsClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskAzureListObjects); blob_list_response = client_ptr->ListBlobs(options); @@ -292,10 +291,13 @@ std::unique_ptr AzureObjectStorage::writeObject( /// NO std::move(scheduler)); } -void AzureObjectStorage::removeObjectImpl(const StoredObject & object, const SharedAzureClientPtr & client_ptr, bool if_exists) +void AzureObjectStorage::removeObjectImpl( + const StoredObject & object, + const std::shared_ptr & client_ptr, + bool if_exists) { ProfileEvents::increment(ProfileEvents::AzureDeleteObjects); - if (client_ptr->GetClickhouseOptions().IsClientForDisk) + if (client_ptr->IsClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskAzureDeleteObjects); const auto & path = object.remote_path; @@ -303,7 +305,7 @@ void AzureObjectStorage::removeObjectImpl(const StoredObject & object, const Sha try { - auto delete_info = client_ptr->DeleteBlob(path); + auto delete_info = client_ptr->GetBlobClient(path).Delete(); if (!if_exists && !delete_info.Value.Deleted) throw Exception( ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Failed to delete file (path: {}) in AzureBlob Storage, reason: {}", @@ -314,7 +316,7 @@ void AzureObjectStorage::removeObjectImpl(const StoredObject & object, const Sha if (!if_exists) throw; - /// If object doesn't exist... + /// If object doesn't exist. if (e.StatusCode == Azure::Core::Http::HttpStatusCode::NotFound) return; @@ -357,7 +359,7 @@ ObjectMetadata AzureObjectStorage::getObjectMetadata(const std::string & path) c auto properties = blob_client.GetProperties().Value; ProfileEvents::increment(ProfileEvents::AzureGetProperties); - if (client_ptr->GetClickhouseOptions().IsClientForDisk) + if (client_ptr->IsClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskAzureGetProperties); ObjectMetadata result; @@ -391,7 +393,7 @@ void AzureObjectStorage::copyObject( /// NOLINT } ProfileEvents::increment(ProfileEvents::AzureCopyObject); - if (client_ptr->GetClickhouseOptions().IsClientForDisk) + if (client_ptr->IsClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskAzureCopyObject); dest_blob_client.CopyFromUri(source_blob_client.GetUrl(), copy_options); @@ -409,7 +411,7 @@ void AzureObjectStorage::applyNewSettings( if (!options.allow_client_change) return; - bool is_client_for_disk = client.get()->GetClickhouseOptions().IsClientForDisk; + bool is_client_for_disk = client.get()->IsClientForDisk(); AzureBlobStorage::ConnectionParams params { @@ -430,7 +432,7 @@ std::unique_ptr AzureObjectStorage::cloneObjectStorage( ContextPtr context) { auto new_settings = AzureBlobStorage::getRequestSettings(config, config_prefix, context); - bool is_client_for_disk = client.get()->GetClickhouseOptions().IsClientForDisk; + bool is_client_for_disk = client.get()->IsClientForDisk(); AzureBlobStorage::ConnectionParams params { @@ -440,7 +442,7 @@ std::unique_ptr AzureObjectStorage::cloneObjectStorage( }; auto new_client = AzureBlobStorage::getContainerClient(params, /*readonly=*/ true); - return std::make_unique(name, std::move(new_client), std::move(new_settings), new_namespace, params.endpoint.getEndpointWithoutContainer()); + return std::make_unique(name, std::move(new_client), std::move(new_settings), new_namespace, params.endpoint.getServiceEndpoint()); } } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index baad3bdf223..525c388ba1d 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -111,8 +111,10 @@ public: bool supportParallelWrite() const override { return true; } private: - using SharedAzureClientPtr = std::shared_ptr; - void removeObjectImpl(const StoredObject & object, const SharedAzureClientPtr & client_ptr, bool if_exists); + void removeObjectImpl( + const StoredObject & object, + const std::shared_ptr & client_ptr, + bool if_exists); const String name; /// client used to access the files in the Blob Storage cloud diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index 5ecbf457db3..4f7cf59e5e2 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -127,7 +127,7 @@ public: const FileCacheSettings & getCacheSettings() const { return cache_settings; } #if USE_AZURE_BLOB_STORAGE - std::shared_ptr getAzureBlobStorageClient() const override + std::shared_ptr getAzureBlobStorageClient() const override { return object_storage->getAzureBlobStorageClient(); } diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 864b9e3201d..0289460842e 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -28,8 +28,7 @@ #include "config.h" #if USE_AZURE_BLOB_STORAGE -#include -#include +#include #endif #if USE_AWS_S3 @@ -258,7 +257,7 @@ public: virtual void setKeysGenerator(ObjectStorageKeysGeneratorPtr) { } #if USE_AZURE_BLOB_STORAGE - virtual std::shared_ptr getAzureBlobStorageClient() const + virtual std::shared_ptr getAzureBlobStorageClient() const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "This function is only implemented for AzureBlobStorage"); } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 30111d04d20..cee9670008a 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -6,7 +6,6 @@ #include #include -#include namespace DB { diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index 092277aca50..a3f74b4dfb2 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -332,7 +332,7 @@ void registerAzureObjectStorage(ObjectStorageFactory & factory) ObjectStorageType::Azure, config, config_prefix, name, AzureBlobStorage::getContainerClient(params, /*readonly=*/ false), std::move(azure_settings), params.endpoint.prefix.empty() ? params.endpoint.container_name : params.endpoint.container_name + "/" + params.endpoint.prefix, - params.endpoint.getEndpointWithoutContainer()); + params.endpoint.getServiceEndpoint()); }; factory.registerObjectStorageType("azure_blob_storage", creator); diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp index 49ec08a9835..2dc6a743627 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp @@ -40,7 +40,7 @@ namespace public: UploadHelper( const CreateReadBuffer & create_read_buffer_, - std::shared_ptr client_, + std::shared_ptr client_, size_t offset_, size_t total_size_, const String & dest_container_for_logging_, @@ -65,7 +65,7 @@ namespace protected: std::function()> create_read_buffer; - std::shared_ptr client; + std::shared_ptr client; size_t offset; size_t total_size; const String & dest_container_for_logging; @@ -260,7 +260,7 @@ namespace void processUploadPartRequest(UploadPartTask & task) { ProfileEvents::increment(ProfileEvents::AzureUploadPart); - if (client->GetClickhouseOptions().IsClientForDisk) + if (client->IsClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskAzureUploadPart); auto block_blob_client = client->GetBlockBlobClient(dest_blob); @@ -309,7 +309,7 @@ void copyDataToAzureBlobStorageFile( const std::function()> & create_read_buffer, size_t offset, size_t size, - std::shared_ptr dest_client, + std::shared_ptr dest_client, const String & dest_container_for_logging, const String & dest_blob, std::shared_ptr settings, @@ -321,8 +321,8 @@ void copyDataToAzureBlobStorageFile( void copyAzureBlobStorageFile( - std::shared_ptr src_client, - std::shared_ptr 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, @@ -337,7 +337,7 @@ void copyAzureBlobStorageFile( { LOG_TRACE(getLogger("copyAzureBlobStorageFile"), "Copying Blob: {} from Container: {} using native copy", src_container_for_logging, src_blob); ProfileEvents::increment(ProfileEvents::AzureCopyObject); - if (dest_client->GetClickhouseOptions().IsClientForDisk) + if (dest_client->IsClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskAzureCopyObject); auto block_blob_client_src = src_client->GetBlockBlobClient(src_blob); diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h index c8e48fcd372..79539a87086 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h @@ -20,8 +20,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, + std::shared_ptr src_client, + std::shared_ptr dest_client, const String & src_container_for_logging, const String & src_blob, size_t src_offset, @@ -42,7 +42,7 @@ void copyDataToAzureBlobStorageFile( const std::function()> & create_read_buffer, size_t offset, size_t size, - std::shared_ptr client, + std::shared_ptr client, const String & dest_container_for_logging, const String & dest_blob, std::shared_ptr settings, diff --git a/tests/queries/0_stateless/03008_azure_plain_rewritable.reference b/tests/queries/0_stateless/03008_azure_plain_rewritable.reference new file mode 100644 index 00000000000..9aa9873514a --- /dev/null +++ b/tests/queries/0_stateless/03008_azure_plain_rewritable.reference @@ -0,0 +1,11 @@ +10006 +0 0 0 +1 1 1 +1 2 0 +2 2 2 +2 2 2 +3 1 9 +3 3 3 +4 4 4 +4 7 7 +5 5 5 diff --git a/tests/queries/0_stateless/03008_azure_plain_rewritable.sh b/tests/queries/0_stateless/03008_azure_plain_rewritable.sh new file mode 100755 index 00000000000..29bfc92282f --- /dev/null +++ b/tests/queries/0_stateless/03008_azure_plain_rewritable.sh @@ -0,0 +1,64 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-shared-merge-tree +# Tag no-fasttest: requires Azure +# Tag no-shared-merge-tree: does not support replication + +set -e + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +container="cont-$(echo "${CLICKHOUSE_TEST_UNIQUE_NAME}" | tr _ -)" + +${CLICKHOUSE_CLIENT} --query "drop table if exists test_azure_mt" + +${CLICKHOUSE_CLIENT} -nm --query " +create table test_azure_mt (a Int32, b Int64, c Int64) engine = MergeTree() partition by intDiv(a, 1000) order by tuple(a, b) +settings disk = disk( + type = object_storage, + metadata_type = plain_rewritable, + object_storage_type = azure_blob_storage, + name = '${container}', + path='/var/lib/clickhouse/disks/${container}/tables', + container_name = '${container}', + endpoint = 'http://localhost:10000/devstoreaccount1/${container}/plain-tables', + account_name = 'devstoreaccount1', + account_key = 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw=='); +" + +${CLICKHOUSE_CLIENT} -nm --query " +insert into test_azure_mt (*) values (1, 2, 0), (2, 2, 2), (3, 1, 9), (4, 7, 7), (5, 10, 2), (6, 12, 5); +insert into test_azure_mt (*) select number, number, number from numbers_mt(10000); +select count(*) from test_azure_mt; +select (*) from test_azure_mt order by tuple(a, b) limit 10; +" + +${CLICKHOUSE_CLIENT} --query "optimize table test_azure_mt final" + +${CLICKHOUSE_CLIENT} -m --query " +alter table test_azure_mt add projection test_azure_mt_projection (select * order by b)" 2>&1 | grep -Fq "SUPPORT_IS_DISABLED" + +${CLICKHOUSE_CLIENT} -nm --query " +alter table test_azure_mt update c = 0 where a % 2 = 1; +alter table test_azure_mt add column d Int64 after c; +alter table test_azure_mt drop column c; +" 2>&1 | grep -Fq "SUPPORT_IS_DISABLED" + +${CLICKHOUSE_CLIENT} -nm --query " +detach table test_azure_mt; +attach table test_azure_mt; +" + +${CLICKHOUSE_CLIENT} --query "drop table if exists test_azure_mt_dst" + +${CLICKHOUSE_CLIENT} -m --query " +create table test_azure_mt_dst (a Int32, b Int64, c Int64) +engine = MergeTree() partition by intDiv(a, 1000) order by tuple(a, b) +settings disk = '${container}' +" + +${CLICKHOUSE_CLIENT} -m --query " +alter table test_azure_mt move partition 0 to table test_azure_mt_dst" 2>&1 | grep -Fq "SUPPORT_IS_DISABLED" + +${CLICKHOUSE_CLIENT} --query "drop table test_azure_mt sync" From 2025f8487e9adec27d8df8c3d5f05bd3d3f004ad Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 8 Jul 2024 09:19:51 +0000 Subject: [PATCH 02/50] fix tests --- docker/test/stateless/run.sh | 2 ++ src/Backups/BackupIO_AzureBlobStorage.cpp | 1 - 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 5ece9743498..13627afb554 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -42,6 +42,8 @@ source /utils.lib # install test configs /usr/share/clickhouse-test/config/install.sh +azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --silent --inMemoryPersistence & + ./setup_minio.sh stateless ./setup_hdfs_minicluster.sh diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index a7861c04a1c..546ab1afc65 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -24,7 +24,6 @@ namespace DB { namespace ErrorCodes { - extern const int AZURE_BLOB_STORAGE_ERROR; extern const int LOGICAL_ERROR; } From f62635b5fd9b0fb346f5efae82c2c95d206a9344 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 8 Jul 2024 14:26:44 +0000 Subject: [PATCH 03/50] fix build --- src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp | 6 +++--- .../AzureBlobStorage/AzureBlobStorageCommon.cpp | 2 -- .../AzureBlobStorage/AzureBlobStorageCommon.h | 2 ++ .../ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp | 3 --- src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp | 2 +- 5 files changed, 6 insertions(+), 9 deletions(-) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index e03710e6088..cead17eeb4f 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -144,7 +144,7 @@ void WriteBufferFromAzureBlobStorage::preFinalize() if (block_ids.empty() && detached_part_data.size() == 1 && detached_part_data.front().data_size <= max_single_part_upload_size) { ProfileEvents::increment(ProfileEvents::AzureUpload); - if (blob_container_client->GetClickhouseOptions().IsClientForDisk) + if (blob_container_client->IsClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskAzureUpload); auto part_data = std::move(detached_part_data.front()); @@ -178,7 +178,7 @@ void WriteBufferFromAzureBlobStorage::finalizeImpl() { auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); ProfileEvents::increment(ProfileEvents::AzureCommitBlockList); - if (blob_container_client->GetClickhouseOptions().IsClientForDisk) + if (blob_container_client->IsClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskAzureCommitBlockList); execWithRetry([&](){ block_blob_client.CommitBlockList(block_ids); }, max_unexpected_write_error_retries); @@ -287,7 +287,7 @@ void WriteBufferFromAzureBlobStorage::writePart(WriteBufferFromAzureBlobStorage: auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); ProfileEvents::increment(ProfileEvents::AzureStageBlock); - if (blob_container_client->GetClickhouseOptions().IsClientForDisk) + if (blob_container_client->IsClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskAzureStageBlock); Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(std::get<1>(*worker_data).memory.data()), data_size); diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp index dd1ab168d84..c22dfd18ba0 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp @@ -1,6 +1,4 @@ #include -#include "Common/StackTrace.h" -#include "Common/logger_useful.h" #if USE_AZURE_BLOB_STORAGE diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h index a77087d8557..d4a89168d1e 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h @@ -99,6 +99,8 @@ using RawContainerClient = Azure::Storage::Blobs::BlobContainerClient; using Azure::Storage::Blobs::ListBlobsOptions; using Azure::Storage::Blobs::ListBlobsPagedResponse; +/// A wrapper for ContainerClient that correctly handles the prefix of blobs. +/// See AzureBlobStorageEndpoint and processAzureBlobStorageEndpoint for details. class ContainerClientWrapper { public: diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 3eb897e2543..5531c07f2e2 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -155,9 +155,6 @@ void AzureObjectStorage::listObjects(const std::string & path, RelativePathsWith { auto client_ptr = client.get(); - /// NOTE: list doesn't work if endpoint contains non-empty prefix for blobs. - /// See AzureBlobStorageEndpoint and processAzureBlobStorageEndpoint for details. - Azure::Storage::Blobs::ListBlobsOptions options; options.Prefix = path; if (max_keys) diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp index 628d46291ef..507cc3c80eb 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.cpp @@ -159,7 +159,7 @@ namespace { auto block_blob_client = client->GetBlockBlobClient(dest_blob); ProfileEvents::increment(ProfileEvents::AzureCommitBlockList); - if (client->GetClickhouseOptions().IsClientForDisk) + if (client->IsClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskAzureCommitBlockList); block_blob_client.CommitBlockList(block_ids); From ef175d65b189efc2226108dfda513846a472d2f4 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 19 Jul 2024 12:41:04 +0000 Subject: [PATCH 04/50] fix build --- .../ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h index 129d913edcd..a82e0f07a3f 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h @@ -14,6 +14,7 @@ #include #include #include +#include namespace fs = std::filesystem; From b3b9d65c045d716a8a76f8119ea00eaab7cd1a33 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 7 Aug 2024 12:43:36 +0000 Subject: [PATCH 05/50] fix ListObject in azure client --- .../AzureBlobStorageCommon.cpp | 22 +++++++++++++++++-- .../AzureBlobStorage/AzureBlobStorageCommon.h | 2 ++ 2 files changed, 22 insertions(+), 2 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp index 79f371eb8f5..f8b0eb9ac56 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp @@ -25,6 +25,7 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; } namespace AzureBlobStorage @@ -76,11 +77,28 @@ BlockBlobClient ContainerClientWrapper::GetBlockBlobClient(const String & blob_n return client.GetBlockBlobClient(blob_prefix / blob_name); } +BlobContainerPropertiesRespones ContainerClientWrapper::GetProperties() const +{ + return client.GetProperties(); +} + ListBlobsPagedResponse ContainerClientWrapper::ListBlobs(const ListBlobsOptions & options) const { auto new_options = options; new_options.Prefix = blob_prefix / options.Prefix.ValueOr(""); - return client.ListBlobs(new_options); + + auto response = client.ListBlobs(new_options); + auto blob_prefix_str = blob_prefix.string() + "/"; + + for (auto & blob : response.Blobs) + { + if (!blob.Name.starts_with(blob_prefix_str)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected prefix '{}' in blob name '{}'", blob_prefix_str, blob.Name); + + blob.Name = blob.Name.substr(blob_prefix_str.size()); + } + + return response; } bool ContainerClientWrapper::IsClientForDisk() const @@ -258,7 +276,7 @@ void processURL(const String & url, const String & container_name, Endpoint & en static bool containerExists(const ContainerClient & client) { ProfileEvents::increment(ProfileEvents::AzureGetProperties); - if (client.GetClickhouseOptions().IsClientForDisk) + if (client.IsClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskAzureGetProperties); try diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h index a82e0f07a3f..af47fbbf772 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.h @@ -100,6 +100,7 @@ using RawContainerClient = Azure::Storage::Blobs::BlobContainerClient; using Azure::Storage::Blobs::ListBlobsOptions; using Azure::Storage::Blobs::ListBlobsPagedResponse; +using BlobContainerPropertiesRespones = Azure::Response; /// A wrapper for ContainerClient that correctly handles the prefix of blobs. /// See AzureBlobStorageEndpoint and processAzureBlobStorageEndpoint for details. @@ -111,6 +112,7 @@ public: bool IsClientForDisk() const; BlobClient GetBlobClient(const String & blob_name) const; BlockBlobClient GetBlockBlobClient(const String & blob_name) const; + BlobContainerPropertiesRespones GetProperties() const; ListBlobsPagedResponse ListBlobs(const ListBlobsOptions & options) const; private: From f413ccd75912b8d9abeec0f3ebfbca659bdd972a Mon Sep 17 00:00:00 2001 From: zhanglistar Date: Wed, 23 Oct 2024 16:35:57 +0800 Subject: [PATCH 06/50] cast support spark text output --- src/Core/FormatFactorySettingsDeclaration.h | 3 ++ .../Serializations/SerializationArray.cpp | 19 ++++++--- .../Serializations/SerializationMap.cpp | 41 +++++++++++++++---- .../Serializations/SerializationMap.h | 3 +- .../Serializations/SerializationTuple.cpp | 18 ++++++-- src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + .../03259_to_string_spark_format.reference | 16 ++++++++ .../03259_to_string_spark_format.sql | 18 ++++++++ 9 files changed, 103 insertions(+), 17 deletions(-) create mode 100644 tests/queries/0_stateless/03259_to_string_spark_format.reference create mode 100644 tests/queries/0_stateless/03259_to_string_spark_format.sql diff --git a/src/Core/FormatFactorySettingsDeclaration.h b/src/Core/FormatFactorySettingsDeclaration.h index 28fae8e305f..9b4b89c9b78 100644 --- a/src/Core/FormatFactorySettingsDeclaration.h +++ b/src/Core/FormatFactorySettingsDeclaration.h @@ -1230,6 +1230,9 @@ Set the quoting rule for identifiers in SHOW CREATE query )", 0) \ M(IdentifierQuotingStyle, show_create_query_identifier_quoting_style, IdentifierQuotingStyle::Backticks, R"( Set the quoting style for identifiers in SHOW CREATE query +)", 0) \ + M(Bool, spark_text_output_format, false, R"( +If true, use Apache Spark output format )", 0) \ // End of FORMAT_FACTORY_SETTINGS diff --git a/src/DataTypes/Serializations/SerializationArray.cpp b/src/DataTypes/Serializations/SerializationArray.cpp index b7c002c95fc..a1bdfa5e68a 100644 --- a/src/DataTypes/Serializations/SerializationArray.cpp +++ b/src/DataTypes/Serializations/SerializationArray.cpp @@ -401,7 +401,7 @@ void SerializationArray::deserializeBinaryBulkWithMultipleStreams( template -static void serializeTextImpl(const IColumn & column, size_t row_num, WriteBuffer & ostr, Writer && write_nested) +static void serializeTextImpl(const IColumn & column, size_t row_num, const FormatSettings & settings, WriteBuffer & ostr, Writer && write_nested) { const ColumnArray & column_array = assert_cast(column); const ColumnArray::Offsets & offsets = column_array.getOffsets(); @@ -412,10 +412,14 @@ static void serializeTextImpl(const IColumn & column, size_t row_num, WriteBuffe const IColumn & nested_column = column_array.getData(); writeChar('[', ostr); - for (size_t i = offset; i < next_offset; ++i) + + if (next_offset != offset) + write_nested(nested_column, offset); + for (size_t i = offset + 1; i < next_offset; ++i) { - if (i != offset) - writeChar(',', ostr); + writeChar(',', ostr); + if (settings.spark_text_output_format) + writeChar(' ', ostr); write_nested(nested_column, i); } writeChar(']', ostr); @@ -520,10 +524,13 @@ static ReturnType deserializeTextImpl(IColumn & column, ReadBuffer & istr, Reade void SerializationArray::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - serializeTextImpl(column, row_num, ostr, + serializeTextImpl(column, row_num, settings, ostr, [&](const IColumn & nested_column, size_t i) { - nested->serializeTextQuoted(nested_column, i, ostr, settings); + if (settings.spark_text_output_format) + nested->serializeText(nested_column, i, ostr, settings); + else + nested->serializeTextQuoted(nested_column, i, ostr, settings); }); } diff --git a/src/DataTypes/Serializations/SerializationMap.cpp b/src/DataTypes/Serializations/SerializationMap.cpp index 6538589e4f8..b7140de2893 100644 --- a/src/DataTypes/Serializations/SerializationMap.cpp +++ b/src/DataTypes/Serializations/SerializationMap.cpp @@ -90,6 +90,7 @@ template void SerializationMap::serializeTextImpl( const IColumn & column, size_t row_num, + const FormatSettings & settings, WriteBuffer & ostr, KeyWriter && key_writer, ValueWriter && value_writer) const @@ -104,13 +105,36 @@ void SerializationMap::serializeTextImpl( size_t next_offset = offsets[row_num]; writeChar('{', ostr); - for (size_t i = offset; i < next_offset; ++i) + if (offset != next_offset) { - if (i != offset) - writeChar(',', ostr); + key_writer(ostr, key, nested_tuple.getColumn(0), offset); + if (settings.spark_text_output_format) + { + writeChar(' ', ostr); + writeChar('-', ostr); + writeChar('>', ostr); + writeChar(' ', ostr); + } + else + writeChar(':', ostr); + value_writer(ostr, value, nested_tuple.getColumn(1), offset); + } + for (size_t i = offset + 1; i < next_offset; ++i) + { + writeChar(',', ostr); + if (settings.spark_text_output_format) + writeChar(' ', ostr); key_writer(ostr, key, nested_tuple.getColumn(0), i); - writeChar(':', ostr); + if (settings.spark_text_output_format) + { + writeChar(' ', ostr); + writeChar('-', ostr); + writeChar('>', ostr); + writeChar(' ', ostr); + } + else + writeChar(':', ostr); value_writer(ostr, value, nested_tuple.getColumn(1), i); } writeChar('}', ostr); @@ -221,10 +245,13 @@ void SerializationMap::serializeText(const IColumn & column, size_t row_num, Wri { auto writer = [&settings](WriteBuffer & buf, const SerializationPtr & subcolumn_serialization, const IColumn & subcolumn, size_t pos) { - subcolumn_serialization->serializeTextQuoted(subcolumn, pos, buf, settings); + if (settings.spark_text_output_format) + subcolumn_serialization->serializeText(subcolumn, pos, buf, settings); + else + subcolumn_serialization->serializeTextQuoted(subcolumn, pos, buf, settings); }; - serializeTextImpl(column, row_num, ostr, writer, writer); + serializeTextImpl(column, row_num, settings, ostr, writer, writer); } void SerializationMap::deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, bool whole) const @@ -266,7 +293,7 @@ bool SerializationMap::tryDeserializeText(IColumn & column, ReadBuffer & istr, c void SerializationMap::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - serializeTextImpl(column, row_num, ostr, + serializeTextImpl(column, row_num, settings, ostr, [&settings](WriteBuffer & buf, const SerializationPtr & subcolumn_serialization, const IColumn & subcolumn, size_t pos) { /// We need to double-quote all keys (including integers) to produce valid JSON. diff --git a/src/DataTypes/Serializations/SerializationMap.h b/src/DataTypes/Serializations/SerializationMap.h index 007d153ec7e..49e31756286 100644 --- a/src/DataTypes/Serializations/SerializationMap.h +++ b/src/DataTypes/Serializations/SerializationMap.h @@ -1,6 +1,7 @@ #pragma once #include +#include "Formats/FormatSettings.h" namespace DB @@ -70,7 +71,7 @@ public: private: template - void serializeTextImpl(const IColumn & column, size_t row_num, WriteBuffer & ostr, KeyWriter && key_writer, ValueWriter && value_writer) const; + void serializeTextImpl(const IColumn & column, size_t row_num, const FormatSettings & settings, WriteBuffer & ostr, KeyWriter && key_writer, ValueWriter && value_writer) const; template ReturnType deserializeTextImpl(IColumn & column, ReadBuffer & istr, Reader && reader) const; diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index 366949e7ac0..6ccb2cfd604 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -137,11 +137,23 @@ void SerializationTuple::deserializeBinary(IColumn & column, ReadBuffer & istr, void SerializationTuple::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeChar('(', ostr); + if (elems.size()) + { + if (settings.spark_text_output_format) + elems[0]->serializeText(extractElementColumn(column, 0), row_num, ostr, settings); + else + elems[0]->serializeTextQuoted(extractElementColumn(column, 0), row_num, ostr, settings); + } for (size_t i = 0; i < elems.size(); ++i) { - if (i != 0) - writeChar(',', ostr); - elems[i]->serializeTextQuoted(extractElementColumn(column, i), row_num, ostr, settings); + writeChar(',', ostr); + if (settings.spark_text_output_format) + { + writeChar(' ', ostr); + elems[i]->serializeText(extractElementColumn(column, i), row_num, ostr, settings); + } + else + elems[i]->serializeTextQuoted(extractElementColumn(column, i), row_num, ostr, settings); } writeChar(')', ostr); } diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index f2142b857cf..38104eebd5b 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -249,6 +249,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.values.deduce_templates_of_expressions = settings[Setting::input_format_values_deduce_templates_of_expressions]; format_settings.values.interpret_expressions = settings[Setting::input_format_values_interpret_expressions]; format_settings.values.escape_quote_with_quote = settings[Setting::output_format_values_escape_quote_with_quote]; + format_settings.spark_text_output_format = settings[Setting::spark_text_output_format]; format_settings.with_names_use_header = settings[Setting::input_format_with_names_use_header]; format_settings.with_types_use_header = settings[Setting::input_format_with_types_use_header]; format_settings.write_statistics = settings[Setting::output_format_write_statistics]; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 8f551f54e7f..a8c5a11944f 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -38,6 +38,7 @@ struct FormatSettings bool try_infer_variant = false; bool seekable_read = true; + bool spark_text_output_format = false; UInt64 max_rows_to_read_for_schema_inference = 25000; UInt64 max_bytes_to_read_for_schema_inference = 32 * 1024 * 1024; diff --git a/tests/queries/0_stateless/03259_to_string_spark_format.reference b/tests/queries/0_stateless/03259_to_string_spark_format.reference new file mode 100644 index 00000000000..b9c59143d24 --- /dev/null +++ b/tests/queries/0_stateless/03259_to_string_spark_format.reference @@ -0,0 +1,16 @@ +-- array format +[\'1\'] +[1, 2, abc, \'1\'] +[1, 2, abc, \'1\'] +[1, 2, abc, \'1\'] +[1, 2, abc, \'1\'] +-- map format +{1343 -> fe, afe -> fefe} +{1343 -> fe, afe -> fefe} +{1343 -> fe, afe -> fefe} +{1343 -> fe, afe -> fefe} +-- tuple format +(1, 1, 3, abc) +(1, 1, 3, abc) +(1, 1, 3, abc) +(1, 1, 3, abc) diff --git a/tests/queries/0_stateless/03259_to_string_spark_format.sql b/tests/queries/0_stateless/03259_to_string_spark_format.sql new file mode 100644 index 00000000000..a202d417a64 --- /dev/null +++ b/tests/queries/0_stateless/03259_to_string_spark_format.sql @@ -0,0 +1,18 @@ +SELECT '-- array format'; +SELECT CAST(array('\'1\'') , 'String') SETTINGS spark_text_output_format=1; +SELECT CAST([materialize('1'), '2', 'abc', '\'1\''], 'String') SETTINGS spark_text_output_format = 1; +SELECT CAST([materialize('1'), materialize('2'), 'abc', '\'1\''], 'String') SETTINGS spark_text_output_format = 1; +SELECT CAST([materialize('1'), materialize('2'), materialize('abc'), '\'1\''], 'String') SETTINGS spark_text_output_format = 1; +SELECT CAST([materialize('1'), materialize('2'), materialize('abc'), materialize('\'1\'')], 'String') SETTINGS spark_text_output_format = 1; + +SELECT '-- map format'; +SELECT toString(map('1343', 'fe', 'afe', 'fefe')) SETTINGS spark_text_output_format = 1; +SELECT toString(map(materialize('1343'), materialize('fe'), 'afe', 'fefe')) SETTINGS spark_text_output_format = 1; +SELECT toString(map(materialize('1343'), materialize('fe'), materialize('afe'), 'fefe')) SETTINGS spark_text_output_format = 1; +SELECT toString(map(materialize('1343'), materialize('fe'), materialize('afe'), materialize('fefe'))) SETTINGS spark_text_output_format = 1; + +SELECT '-- tuple format'; +SELECT toString(('1', '3', 'abc')) SETTINGS spark_text_output_format = 1; +SELECT toString((materialize('1'), '3', 'abc')) SETTINGS spark_text_output_format = 1; +SELECT toString((materialize('1'), materialize('3'), 'abc')) SETTINGS spark_text_output_format = 1; +SELECT toString((materialize('1'), materialize('3'), materialize('abc'))) SETTINGS spark_text_output_format = 1; From d73c302d44b506320baaeb6b386c56a1a74920a4 Mon Sep 17 00:00:00 2001 From: zhanglistar Date: Wed, 23 Oct 2024 17:16:41 +0800 Subject: [PATCH 07/50] clean code --- src/DataTypes/Serializations/SerializationMap.cpp | 14 ++------------ src/DataTypes/Serializations/SerializationMap.h | 2 +- 2 files changed, 3 insertions(+), 13 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationMap.cpp b/src/DataTypes/Serializations/SerializationMap.cpp index b7140de2893..737b357a565 100644 --- a/src/DataTypes/Serializations/SerializationMap.cpp +++ b/src/DataTypes/Serializations/SerializationMap.cpp @@ -109,12 +109,7 @@ void SerializationMap::serializeTextImpl( { key_writer(ostr, key, nested_tuple.getColumn(0), offset); if (settings.spark_text_output_format) - { - writeChar(' ', ostr); - writeChar('-', ostr); - writeChar('>', ostr); - writeChar(' ', ostr); - } + writeString(std::string_view(" -> "), ostr); else writeChar(':', ostr); value_writer(ostr, value, nested_tuple.getColumn(1), offset); @@ -127,12 +122,7 @@ void SerializationMap::serializeTextImpl( key_writer(ostr, key, nested_tuple.getColumn(0), i); if (settings.spark_text_output_format) - { - writeChar(' ', ostr); - writeChar('-', ostr); - writeChar('>', ostr); - writeChar(' ', ostr); - } + writeString(std::string_view(" -> "), ostr); else writeChar(':', ostr); value_writer(ostr, value, nested_tuple.getColumn(1), i); diff --git a/src/DataTypes/Serializations/SerializationMap.h b/src/DataTypes/Serializations/SerializationMap.h index 49e31756286..ad6f7808cb6 100644 --- a/src/DataTypes/Serializations/SerializationMap.h +++ b/src/DataTypes/Serializations/SerializationMap.h @@ -1,7 +1,7 @@ #pragma once #include -#include "Formats/FormatSettings.h" +//#include namespace DB From 24a3d1574d9ce15337e9234dfab2780ada6eba68 Mon Sep 17 00:00:00 2001 From: zhanglistar Date: Wed, 23 Oct 2024 17:18:18 +0800 Subject: [PATCH 08/50] delete useless header --- src/DataTypes/Serializations/SerializationMap.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationMap.h b/src/DataTypes/Serializations/SerializationMap.h index ad6f7808cb6..74229bcfb15 100644 --- a/src/DataTypes/Serializations/SerializationMap.h +++ b/src/DataTypes/Serializations/SerializationMap.h @@ -1,7 +1,6 @@ #pragma once #include -//#include namespace DB From 67d2210dd8843e748f15d22b02124801c0cce450 Mon Sep 17 00:00:00 2001 From: zhanglistar Date: Thu, 24 Oct 2024 09:30:10 +0800 Subject: [PATCH 09/50] fix tuple index bug and clean code --- .../Serializations/SerializationMap.cpp | 25 +++++++++++-------- .../Serializations/SerializationTuple.cpp | 15 +++++------ .../03259_to_string_spark_format.reference | 8 +++--- 3 files changed, 26 insertions(+), 22 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationMap.cpp b/src/DataTypes/Serializations/SerializationMap.cpp index 737b357a565..aa38ef2aab4 100644 --- a/src/DataTypes/Serializations/SerializationMap.cpp +++ b/src/DataTypes/Serializations/SerializationMap.cpp @@ -114,19 +114,22 @@ void SerializationMap::serializeTextImpl( writeChar(':', ostr); value_writer(ostr, value, nested_tuple.getColumn(1), offset); } - for (size_t i = offset + 1; i < next_offset; ++i) - { - writeChar(',', ostr); - if (settings.spark_text_output_format) - writeChar(' ', ostr); - - key_writer(ostr, key, nested_tuple.getColumn(0), i); - if (settings.spark_text_output_format) + if (settings.spark_text_output_format) + for (size_t i = offset + 1; i < next_offset; ++i) + { + writeString(std::string_view(", "), ostr); + key_writer(ostr, key, nested_tuple.getColumn(0), i); writeString(std::string_view(" -> "), ostr); - else + value_writer(ostr, value, nested_tuple.getColumn(1), i); + } + else + for (size_t i = offset + 1; i < next_offset; ++i) + { + writeChar(',', ostr); + key_writer(ostr, key, nested_tuple.getColumn(0), i); writeChar(':', ostr); - value_writer(ostr, value, nested_tuple.getColumn(1), i); - } + value_writer(ostr, value, nested_tuple.getColumn(1), i); + } writeChar('}', ostr); } diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index 6ccb2cfd604..b2cba0d1b71 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -144,17 +144,18 @@ void SerializationTuple::serializeText(const IColumn & column, size_t row_num, W else elems[0]->serializeTextQuoted(extractElementColumn(column, 0), row_num, ostr, settings); } - for (size_t i = 0; i < elems.size(); ++i) - { - writeChar(',', ostr); - if (settings.spark_text_output_format) + if (settings.spark_text_output_format) + for (size_t i = 1; i < elems.size(); ++i) { - writeChar(' ', ostr); + writeString(std::string_view(", "), ostr); elems[i]->serializeText(extractElementColumn(column, i), row_num, ostr, settings); } - else + else + for (size_t i = 1; i < elems.size(); ++i) + { + writeChar(',', ostr); elems[i]->serializeTextQuoted(extractElementColumn(column, i), row_num, ostr, settings); - } + } writeChar(')', ostr); } diff --git a/tests/queries/0_stateless/03259_to_string_spark_format.reference b/tests/queries/0_stateless/03259_to_string_spark_format.reference index b9c59143d24..14f5b157ca8 100644 --- a/tests/queries/0_stateless/03259_to_string_spark_format.reference +++ b/tests/queries/0_stateless/03259_to_string_spark_format.reference @@ -10,7 +10,7 @@ {1343 -> fe, afe -> fefe} {1343 -> fe, afe -> fefe} -- tuple format -(1, 1, 3, abc) -(1, 1, 3, abc) -(1, 1, 3, abc) -(1, 1, 3, abc) +(1, 3, abc) +(1, 3, abc) +(1, 3, abc) +(1, 3, abc) From 17c88bf5a26f356e883405c58e0d5bcc2091d0b0 Mon Sep 17 00:00:00 2001 From: zhanglistar Date: Thu, 24 Oct 2024 11:24:58 +0800 Subject: [PATCH 10/50] add SettingsChangesHistory --- src/Core/SettingsChangesHistory.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 0cec234b7cd..9406422dcb6 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -98,6 +98,7 @@ static std::initializer_list Date: Thu, 24 Oct 2024 18:18:25 +0800 Subject: [PATCH 11/50] fix debug build fail --- src/DataTypes/Serializations/SerializationTuple.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index b2cba0d1b71..c0eff79037c 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -137,7 +137,7 @@ void SerializationTuple::deserializeBinary(IColumn & column, ReadBuffer & istr, void SerializationTuple::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { writeChar('(', ostr); - if (elems.size()) + if (!elems.empty()) { if (settings.spark_text_output_format) elems[0]->serializeText(extractElementColumn(column, 0), row_num, ostr, settings); From d592bd147dbc0ab3ad24dcc3d71a920e55ade9db Mon Sep 17 00:00:00 2001 From: zhanglistar Date: Fri, 25 Oct 2024 09:22:59 +0800 Subject: [PATCH 12/50] fix build after refactor --- src/Core/FormatFactorySettings.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Core/FormatFactorySettings.h b/src/Core/FormatFactorySettings.h index a095bffc4c9..208ad66611a 100644 --- a/src/Core/FormatFactorySettings.h +++ b/src/Core/FormatFactorySettings.h @@ -1242,6 +1242,9 @@ Set the quoting rule for identifiers in SHOW CREATE query )", 0) \ DECLARE(IdentifierQuotingStyle, show_create_query_identifier_quoting_style, IdentifierQuotingStyle::Backticks, R"( Set the quoting style for identifiers in SHOW CREATE query +)", 0) \ + DECLARE(Bool, spark_text_output_format, false, R"( +Set whether use Apache Spark output text format for arrays, map and tuples. )", 0) \ // End of FORMAT_FACTORY_SETTINGS From 838408ff0a5f16deeeb8ca1f856ea6de30ccb004 Mon Sep 17 00:00:00 2001 From: lzydmxy <13126752315@163.com> Date: Thu, 31 Oct 2024 16:09:25 +0800 Subject: [PATCH 13/50] Force_connected for retry resume connection establisher --- src/Client/ConnectionEstablisher.cpp | 3 ++- src/Client/ConnectionEstablisher.h | 3 +++ src/Client/HedgedConnectionsFactory.cpp | 2 +- 3 files changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp index 178e26b124c..d1960e51e03 100644 --- a/src/Client/ConnectionEstablisher.cpp +++ b/src/Client/ConnectionEstablisher.cpp @@ -151,7 +151,8 @@ void ConnectionEstablisherAsync::Task::run(AsyncCallback async_callback, Suspend { connection_establisher_async.reset(); connection_establisher_async.connection_establisher.setAsyncCallback(async_callback); - connection_establisher_async.connection_establisher.run(connection_establisher_async.result, connection_establisher_async.fail_message); + connection_establisher_async.connection_establisher.run(connection_establisher_async.result, + connection_establisher_async.fail_message, connection_establisher_async.force_connected); connection_establisher_async.is_finished = true; } diff --git a/src/Client/ConnectionEstablisher.h b/src/Client/ConnectionEstablisher.h index ff071e59aea..e7833237600 100644 --- a/src/Client/ConnectionEstablisher.h +++ b/src/Client/ConnectionEstablisher.h @@ -76,6 +76,8 @@ public: const std::string & getFailMessage() const { return fail_message; } + void resumeConnectionWithForceOption(bool force_connected_) {force_connected = force_connected_; resume();} + private: bool checkBeforeTaskResume() override; @@ -125,6 +127,7 @@ private: bool is_finished = false; bool restarted = false; + bool force_connected = false; }; #endif diff --git a/src/Client/HedgedConnectionsFactory.cpp b/src/Client/HedgedConnectionsFactory.cpp index df63a124539..0900f1c0293 100644 --- a/src/Client/HedgedConnectionsFactory.cpp +++ b/src/Client/HedgedConnectionsFactory.cpp @@ -281,7 +281,7 @@ int HedgedConnectionsFactory::getReadyFileDescriptor(bool blocking, AsyncCallbac HedgedConnectionsFactory::State HedgedConnectionsFactory::resumeConnectionEstablisher(int index, Connection *& connection_out) { - replicas[index].connection_establisher->resume(); + replicas[index].connection_establisher->resumeConnectionWithForceOption(/*force_connected_*/ shuffled_pools[index].error_count != 0); if (replicas[index].connection_establisher->isCancelled()) return State::CANNOT_CHOOSE; From 30c33aa1a7c5c05097f7f8b32c66950c84a68669 Mon Sep 17 00:00:00 2001 From: lzydmxy <13126752315@163.com> Date: Tue, 12 Nov 2024 10:18:37 +0800 Subject: [PATCH 14/50] Fix test_distributed_respect_user_timeouts expected --- tests/integration/test_distributed_respect_user_timeouts/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_distributed_respect_user_timeouts/test.py b/tests/integration/test_distributed_respect_user_timeouts/test.py index dd0953c47dd..8ea4efde701 100644 --- a/tests/integration/test_distributed_respect_user_timeouts/test.py +++ b/tests/integration/test_distributed_respect_user_timeouts/test.py @@ -80,6 +80,7 @@ def _check_exception(exception, expected_tries=3): for i, line in enumerate(lines[3 : 3 + expected_tries]): expected_lines = ( "Code: 209. " + EXCEPTION_NETWORK + EXCEPTION_TIMEOUT, + "Code: 209. " + EXCEPTION_NETWORK + "Timeout: connect timed out", EXCEPTION_CONNECT_TIMEOUT, EXCEPTION_TIMEOUT, ) From 7dbead934c602f3e6c0ec9da80a726208ec2d639 Mon Sep 17 00:00:00 2001 From: zhanglistar Date: Wed, 20 Nov 2024 18:25:41 +0800 Subject: [PATCH 15/50] add composed_data_type_output_format_mode --- src/Core/FormatFactorySettings.h | 4 +-- src/Core/SettingsChangesHistory.cpp | 2 +- .../Serializations/SerializationArray.cpp | 4 +-- .../Serializations/SerializationMap.cpp | 6 ++-- .../Serializations/SerializationTuple.cpp | 4 +-- src/Formats/FormatFactory.cpp | 2 +- src/Formats/FormatSettings.h | 2 +- .../03259_to_string_spark_format.sql | 32 +++++++++---------- 8 files changed, 28 insertions(+), 28 deletions(-) diff --git a/src/Core/FormatFactorySettings.h b/src/Core/FormatFactorySettings.h index 208ad66611a..45f8d9bb9b6 100644 --- a/src/Core/FormatFactorySettings.h +++ b/src/Core/FormatFactorySettings.h @@ -1243,8 +1243,8 @@ Set the quoting rule for identifiers in SHOW CREATE query DECLARE(IdentifierQuotingStyle, show_create_query_identifier_quoting_style, IdentifierQuotingStyle::Backticks, R"( Set the quoting style for identifiers in SHOW CREATE query )", 0) \ - DECLARE(Bool, spark_text_output_format, false, R"( -Set whether use Apache Spark output text format for arrays, map and tuples. + DECLARE(String, composed_data_type_output_format_mode, "default", R"( +Set composed data type output format mode, default or spark. )", 0) \ // End of FORMAT_FACTORY_SETTINGS diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 0a2ce0e13b1..8a73de3e422 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -112,7 +112,7 @@ static std::initializer_listserializeText(nested_column, i, ostr, settings); else nested->serializeTextQuoted(nested_column, i, ostr, settings); diff --git a/src/DataTypes/Serializations/SerializationMap.cpp b/src/DataTypes/Serializations/SerializationMap.cpp index aa38ef2aab4..c51255d1dce 100644 --- a/src/DataTypes/Serializations/SerializationMap.cpp +++ b/src/DataTypes/Serializations/SerializationMap.cpp @@ -108,13 +108,13 @@ void SerializationMap::serializeTextImpl( if (offset != next_offset) { key_writer(ostr, key, nested_tuple.getColumn(0), offset); - if (settings.spark_text_output_format) + if (settings.composed_data_type_output_format_mode == "spark") writeString(std::string_view(" -> "), ostr); else writeChar(':', ostr); value_writer(ostr, value, nested_tuple.getColumn(1), offset); } - if (settings.spark_text_output_format) + if (settings.composed_data_type_output_format_mode == "spark") for (size_t i = offset + 1; i < next_offset; ++i) { writeString(std::string_view(", "), ostr); @@ -238,7 +238,7 @@ void SerializationMap::serializeText(const IColumn & column, size_t row_num, Wri { auto writer = [&settings](WriteBuffer & buf, const SerializationPtr & subcolumn_serialization, const IColumn & subcolumn, size_t pos) { - if (settings.spark_text_output_format) + if (settings.composed_data_type_output_format_mode == "spark") subcolumn_serialization->serializeText(subcolumn, pos, buf, settings); else subcolumn_serialization->serializeTextQuoted(subcolumn, pos, buf, settings); diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index c0eff79037c..ec92ceb9110 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -139,12 +139,12 @@ void SerializationTuple::serializeText(const IColumn & column, size_t row_num, W writeChar('(', ostr); if (!elems.empty()) { - if (settings.spark_text_output_format) + if (settings.composed_data_type_output_format_mode == "spark") elems[0]->serializeText(extractElementColumn(column, 0), row_num, ostr, settings); else elems[0]->serializeTextQuoted(extractElementColumn(column, 0), row_num, ostr, settings); } - if (settings.spark_text_output_format) + if (settings.composed_data_type_output_format_mode == "spark") for (size_t i = 1; i < elems.size(); ++i) { writeString(std::string_view(", "), ostr); diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 813a550a124..fa464b9fb7c 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -251,7 +251,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.values.deduce_templates_of_expressions = settings[Setting::input_format_values_deduce_templates_of_expressions]; format_settings.values.interpret_expressions = settings[Setting::input_format_values_interpret_expressions]; format_settings.values.escape_quote_with_quote = settings[Setting::output_format_values_escape_quote_with_quote]; - format_settings.spark_text_output_format = settings[Setting::spark_text_output_format]; + format_settings.composed_data_type_output_format_mode = settings[Setting::composed_data_type_output_format_mode]; format_settings.with_names_use_header = settings[Setting::input_format_with_names_use_header]; format_settings.with_types_use_header = settings[Setting::input_format_with_types_use_header]; format_settings.write_statistics = settings[Setting::output_format_write_statistics]; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index c726b918a19..b81616bf017 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -38,7 +38,7 @@ struct FormatSettings bool try_infer_variant = false; bool seekable_read = true; - bool spark_text_output_format = false; + String composed_data_type_output_format_mode = "default"; UInt64 max_rows_to_read_for_schema_inference = 25000; UInt64 max_bytes_to_read_for_schema_inference = 32 * 1024 * 1024; diff --git a/tests/queries/0_stateless/03259_to_string_spark_format.sql b/tests/queries/0_stateless/03259_to_string_spark_format.sql index a202d417a64..ff9682c7571 100644 --- a/tests/queries/0_stateless/03259_to_string_spark_format.sql +++ b/tests/queries/0_stateless/03259_to_string_spark_format.sql @@ -1,18 +1,18 @@ -SELECT '-- array format'; -SELECT CAST(array('\'1\'') , 'String') SETTINGS spark_text_output_format=1; -SELECT CAST([materialize('1'), '2', 'abc', '\'1\''], 'String') SETTINGS spark_text_output_format = 1; -SELECT CAST([materialize('1'), materialize('2'), 'abc', '\'1\''], 'String') SETTINGS spark_text_output_format = 1; -SELECT CAST([materialize('1'), materialize('2'), materialize('abc'), '\'1\''], 'String') SETTINGS spark_text_output_format = 1; -SELECT CAST([materialize('1'), materialize('2'), materialize('abc'), materialize('\'1\'')], 'String') SETTINGS spark_text_output_format = 1; +SELECT '-- array format --'; +SELECT CAST(array('\'1\'') , 'String') SETTINGS composed_data_type_output_format_mode = "spark"; +SELECT CAST([materialize('1'), '2', 'abc', '\'1\''], 'String') SETTINGS composed_data_type_output_format_mode = "spark"; +SELECT CAST([materialize('1'), materialize('2'), 'abc', '\'1\''], 'String') SETTINGS composed_data_type_output_format_mode = "spark"; +SELECT CAST([materialize('1'), materialize('2'), materialize('abc'), '\'1\''], 'String') SETTINGS composed_data_type_output_format_mode = "spark"; +SELECT CAST([materialize('1'), materialize('2'), materialize('abc'), materialize('\'1\'')], 'String') SETTINGS composed_data_type_output_format_mode = "spark"; -SELECT '-- map format'; -SELECT toString(map('1343', 'fe', 'afe', 'fefe')) SETTINGS spark_text_output_format = 1; -SELECT toString(map(materialize('1343'), materialize('fe'), 'afe', 'fefe')) SETTINGS spark_text_output_format = 1; -SELECT toString(map(materialize('1343'), materialize('fe'), materialize('afe'), 'fefe')) SETTINGS spark_text_output_format = 1; -SELECT toString(map(materialize('1343'), materialize('fe'), materialize('afe'), materialize('fefe'))) SETTINGS spark_text_output_format = 1; +SELECT '-- map format --'; +SELECT toString(map('1343', 'fe', 'afe', 'fefe')) SETTINGS composed_data_type_output_format_mode = "spark"; +SELECT toString(map(materialize('1343'), materialize('fe'), 'afe', 'fefe')) SETTINGS composed_data_type_output_format_mode = "spark"; +SELECT toString(map(materialize('1343'), materialize('fe'), materialize('afe'), 'fefe')) SETTINGS composed_data_type_output_format_mode = "spark"; +SELECT toString(map(materialize('1343'), materialize('fe'), materialize('afe'), materialize('fefe'))) SETTINGS composed_data_type_output_format_mode = "spark"; -SELECT '-- tuple format'; -SELECT toString(('1', '3', 'abc')) SETTINGS spark_text_output_format = 1; -SELECT toString((materialize('1'), '3', 'abc')) SETTINGS spark_text_output_format = 1; -SELECT toString((materialize('1'), materialize('3'), 'abc')) SETTINGS spark_text_output_format = 1; -SELECT toString((materialize('1'), materialize('3'), materialize('abc'))) SETTINGS spark_text_output_format = 1; +SELECT '-- tuple format --'; +SELECT toString(('1', '3', 'abc')) SETTINGS composed_data_type_output_format_mode = "spark"; +SELECT toString((materialize('1'), '3', 'abc')) SETTINGS composed_data_type_output_format_mode = "spark"; +SELECT toString((materialize('1'), materialize('3'), 'abc')) SETTINGS composed_data_type_output_format_mode = "spark"; +SELECT toString((materialize('1'), materialize('3'), materialize('abc'))) SETTINGS composed_data_type_output_format_mode = "spark"; From de626a88c8b1af2e2bbc97124aaacaadeb1b64c4 Mon Sep 17 00:00:00 2001 From: zhanglistar Date: Wed, 20 Nov 2024 18:31:30 +0800 Subject: [PATCH 16/50] fix compile error --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 6b92d1e8954..d320bf88536 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -137,7 +137,7 @@ static std::initializer_list Date: Wed, 20 Nov 2024 13:00:28 +0000 Subject: [PATCH 17/50] fix ListObjects in azure --- .../ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp index 942eb18e8ce..c1db590f1e8 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageCommon.cpp @@ -108,7 +108,7 @@ ListBlobsPagedResponse ContainerClientWrapper::ListBlobs(const ListBlobsOptions new_options.Prefix = blob_prefix / options.Prefix.ValueOr(""); auto response = client.ListBlobs(new_options); - auto blob_prefix_str = blob_prefix.string() + "/"; + auto blob_prefix_str = blob_prefix.empty() ? "" : blob_prefix.string() + "/"; for (auto & blob : response.Blobs) { From 255b168a4ec4d5737153f3825f4d5c1a15dd1de4 Mon Sep 17 00:00:00 2001 From: zhanglistar Date: Thu, 21 Nov 2024 08:50:55 +0800 Subject: [PATCH 18/50] add composed_data_type_output_format_mode --- src/Core/SettingsChangesHistory.cpp | 2 +- .../03259_to_string_spark_format.reference | 6 ++--- .../03259_to_string_spark_format.sql | 26 +++++++++---------- 3 files changed, 17 insertions(+), 17 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index d320bf88536..a892e077ecd 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -138,7 +138,7 @@ static std::initializer_list fe, afe -> fefe} {1343 -> fe, afe -> fefe} {1343 -> fe, afe -> fefe} {1343 -> fe, afe -> fefe} --- tuple format +-- tuple format -- (1, 3, abc) (1, 3, abc) (1, 3, abc) diff --git a/tests/queries/0_stateless/03259_to_string_spark_format.sql b/tests/queries/0_stateless/03259_to_string_spark_format.sql index ff9682c7571..7a657f803bd 100644 --- a/tests/queries/0_stateless/03259_to_string_spark_format.sql +++ b/tests/queries/0_stateless/03259_to_string_spark_format.sql @@ -1,18 +1,18 @@ SELECT '-- array format --'; -SELECT CAST(array('\'1\'') , 'String') SETTINGS composed_data_type_output_format_mode = "spark"; -SELECT CAST([materialize('1'), '2', 'abc', '\'1\''], 'String') SETTINGS composed_data_type_output_format_mode = "spark"; -SELECT CAST([materialize('1'), materialize('2'), 'abc', '\'1\''], 'String') SETTINGS composed_data_type_output_format_mode = "spark"; -SELECT CAST([materialize('1'), materialize('2'), materialize('abc'), '\'1\''], 'String') SETTINGS composed_data_type_output_format_mode = "spark"; -SELECT CAST([materialize('1'), materialize('2'), materialize('abc'), materialize('\'1\'')], 'String') SETTINGS composed_data_type_output_format_mode = "spark"; +SELECT CAST(array('\'1\'') , 'String') SETTINGS composed_data_type_output_format_mode = 'spark'; +SELECT CAST([materialize('1'), '2', 'abc', '\'1\''], 'String') SETTINGS composed_data_type_output_format_mode = 'spark'; +SELECT CAST([materialize('1'), materialize('2'), 'abc', '\'1\''], 'String') SETTINGS composed_data_type_output_format_mode = 'spark'; +SELECT CAST([materialize('1'), materialize('2'), materialize('abc'), '\'1\''], 'String') SETTINGS composed_data_type_output_format_mode = 'spark'; +SELECT CAST([materialize('1'), materialize('2'), materialize('abc'), materialize('\'1\'')], 'String') SETTINGS composed_data_type_output_format_mode = 'spark'; SELECT '-- map format --'; -SELECT toString(map('1343', 'fe', 'afe', 'fefe')) SETTINGS composed_data_type_output_format_mode = "spark"; -SELECT toString(map(materialize('1343'), materialize('fe'), 'afe', 'fefe')) SETTINGS composed_data_type_output_format_mode = "spark"; -SELECT toString(map(materialize('1343'), materialize('fe'), materialize('afe'), 'fefe')) SETTINGS composed_data_type_output_format_mode = "spark"; -SELECT toString(map(materialize('1343'), materialize('fe'), materialize('afe'), materialize('fefe'))) SETTINGS composed_data_type_output_format_mode = "spark"; +SELECT toString(map('1343', 'fe', 'afe', 'fefe')) SETTINGS composed_data_type_output_format_mode = 'spark'; +SELECT toString(map(materialize('1343'), materialize('fe'), 'afe', 'fefe')) SETTINGS composed_data_type_output_format_mode = 'spark'; +SELECT toString(map(materialize('1343'), materialize('fe'), materialize('afe'), 'fefe')) SETTINGS composed_data_type_output_format_mode = 'spark'; +SELECT toString(map(materialize('1343'), materialize('fe'), materialize('afe'), materialize('fefe'))) SETTINGS composed_data_type_output_format_mode = 'spark'; SELECT '-- tuple format --'; -SELECT toString(('1', '3', 'abc')) SETTINGS composed_data_type_output_format_mode = "spark"; -SELECT toString((materialize('1'), '3', 'abc')) SETTINGS composed_data_type_output_format_mode = "spark"; -SELECT toString((materialize('1'), materialize('3'), 'abc')) SETTINGS composed_data_type_output_format_mode = "spark"; -SELECT toString((materialize('1'), materialize('3'), materialize('abc'))) SETTINGS composed_data_type_output_format_mode = "spark"; +SELECT toString(('1', '3', 'abc')) SETTINGS composed_data_type_output_format_mode = 'spark'; +SELECT toString((materialize('1'), '3', 'abc')) SETTINGS composed_data_type_output_format_mode = 'spark'; +SELECT toString((materialize('1'), materialize('3'), 'abc')) SETTINGS composed_data_type_output_format_mode = 'spark'; +SELECT toString((materialize('1'), materialize('3'), materialize('abc'))) SETTINGS composed_data_type_output_format_mode = 'spark'; From faf05a78079e654a0a5383e222cb5cab0bdb58a8 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 21 Nov 2024 14:36:55 +0000 Subject: [PATCH 19/50] bump azurite version --- docker/test/stateless/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index 69f81b35a95..246d537a8bd 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -86,5 +86,5 @@ ENV MINIO_ROOT_USER="clickhouse" ENV MINIO_ROOT_PASSWORD="clickhouse" ENV EXPORT_S3_STORAGE_POLICIES=1 -RUN npm install -g azurite@3.30.0 \ +RUN npm install -g azurite@3.33.0 \ && npm install -g tslib && npm install -g node From fbcb384f1018add51f725cfa83980d8d49cecb73 Mon Sep 17 00:00:00 2001 From: zhanglistar Date: Fri, 22 Nov 2024 09:19:29 +0800 Subject: [PATCH 20/50] Update SettingsChangesHistory.cpp --- src/Core/SettingsChangesHistory.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index a892e077ecd..0f06af93015 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -138,7 +138,6 @@ static std::initializer_list Date: Mon, 25 Nov 2024 17:44:55 +0100 Subject: [PATCH 21/50] Better error message on bad keeper snapshots --- src/Coordination/KeeperStateMachine.cpp | 22 ++++++++++++++-------- src/Coordination/SnapshotableHashTable.h | 14 +++++++++++--- 2 files changed, 25 insertions(+), 11 deletions(-) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 704d3365fa2..9f9de7ae2f1 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -142,13 +142,14 @@ void KeeperStateMachine::init() } catch (...) { - tryLogCurrentException( + LOG_FATAL( log, - fmt::format( - "Aborting because of failure to load from latest snapshot with index {}. Problematic snapshot can be removed but it will " - "lead to data loss", - latest_log_index)); - std::abort(); + "Failure to load from latest snapshot with index {}: {}", + latest_log_index, + getCurrentExceptionMessage(true, true, false)); + LOG_FATAL( + log, "Manual intervention is necessary for recovery. Problematic snapshot can be removed but it will lead to data loss"); + abort(); } } @@ -427,8 +428,13 @@ bool KeeperStateMachine::preprocess(const KeeperStorageBase::RequestFor } catch (...) { - tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("Failed to preprocess stored log at index {}, aborting to avoid inconsistent state", request_for_session.log_idx)); - std::abort(); + LOG_FATAL( + log, + "Failed to preprocess stored log at index {}: {}", + request_for_session.log_idx, + getCurrentExceptionMessage(true, true, false)); + LOG_FATAL(log, "Aborting to avoid inconsistent state"); + abort(); } if (keeper_context->digestEnabled() && request_for_session.digest) diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index de089913ea7..bfe987de1cb 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -8,6 +8,11 @@ namespace DB { +namespace ErrorCodes +{ +extern const int LOGICAL_ERROR; +} + template struct ListNode { @@ -292,7 +297,8 @@ public: { size_t hash_value = map.hash(key); auto it = map.find(key, hash_value); - chassert(it != map.end()); + if (it != map.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not find key: '{}'", key); auto list_itr = it->getMapped(); uint64_t old_value_size = list_itr->value.sizeInBytes(); @@ -348,7 +354,8 @@ public: const V & getValue(StringRef key) const { auto it = map.find(key); - chassert(it); + if (it != map.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not find key: '{}'", key); return it->getMapped()->value; } @@ -356,7 +363,8 @@ public: { for (auto & itr : snapshot_invalid_iters) { - chassert(!itr->isActiveInMap()); + if (itr->isActiveInMap()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "{} is not active in map", itr->key); updateDataSize(ERASE, itr->key.size, 0, itr->value.sizeInBytes(), /*remove_old=*/true); if (itr->getFreeKey()) arena.free(const_cast(itr->key.data), itr->key.size); From bd96303e1776496cdfd63a5d06f8bceedbff3079 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 25 Nov 2024 19:39:50 +0100 Subject: [PATCH 22/50] Fix asserts --- src/Coordination/SnapshotableHashTable.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index bfe987de1cb..462ca18c502 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -297,7 +297,7 @@ public: { size_t hash_value = map.hash(key); auto it = map.find(key, hash_value); - if (it != map.end()) + if (it == map.end()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not find key: '{}'", key); auto list_itr = it->getMapped(); @@ -354,7 +354,7 @@ public: const V & getValue(StringRef key) const { auto it = map.find(key); - if (it != map.end()) + if (it == map.end()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not find key: '{}'", key); return it->getMapped()->value; } From bf37c3a2687e94f4df3c05eb7ba48ee773dfb767 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 25 Nov 2024 22:17:08 +0100 Subject: [PATCH 23/50] Remove obsolete usage of ABORT_ON_LOGICAL_ERROR --- src/Processors/Transforms/DeduplicationTokenTransforms.cpp | 2 +- src/Processors/Transforms/DeduplicationTokenTransforms.h | 2 +- src/Processors/Transforms/buildPushingToViewsChain.cpp | 6 +++--- src/Storages/MaterializedView/RefreshTask.cpp | 2 +- src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp | 2 +- src/Storages/WindowView/StorageWindowView.cpp | 6 +++--- 6 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Processors/Transforms/DeduplicationTokenTransforms.cpp b/src/Processors/Transforms/DeduplicationTokenTransforms.cpp index 841090f029e..79b5bb3be4a 100644 --- a/src/Processors/Transforms/DeduplicationTokenTransforms.cpp +++ b/src/Processors/Transforms/DeduplicationTokenTransforms.cpp @@ -142,7 +142,7 @@ size_t TokenInfo::getTotalSize() const return size + parts.size() - 1; } -#ifdef ABORT_ON_LOGICAL_ERROR +#ifdef DEBUG_OR_SANITIZER_BUILD void CheckTokenTransform::transform(Chunk & chunk) { auto token_info = chunk.getChunkInfos().get(); diff --git a/src/Processors/Transforms/DeduplicationTokenTransforms.h b/src/Processors/Transforms/DeduplicationTokenTransforms.h index 7e5a6101734..f22746f6705 100644 --- a/src/Processors/Transforms/DeduplicationTokenTransforms.h +++ b/src/Processors/Transforms/DeduplicationTokenTransforms.h @@ -93,7 +93,7 @@ namespace DeduplicationToken }; -#ifdef ABORT_ON_LOGICAL_ERROR +#ifdef DEBUG_OR_SANITIZER_BUILD /// use that class only with debug builds in CI for introspection class CheckTokenTransform : public ISimpleTransform { diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 9d914149b4a..46440a2a953 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -378,7 +378,7 @@ std::optional generateViewChain( table_prefers_large_blocks ? settings[Setting::min_insert_block_size_bytes] : 0ULL)); } -#ifdef ABORT_ON_LOGICAL_ERROR +#ifdef DEBUG_OR_SANITIZER_BUILD out.addSource(std::make_shared("Before squashing", out.getInputHeader())); #endif @@ -424,7 +424,7 @@ std::optional generateViewChain( if (type == QueryViewsLogElement::ViewType::MATERIALIZED) { -#ifdef ABORT_ON_LOGICAL_ERROR +#ifdef DEBUG_OR_SANITIZER_BUILD out.addSource(std::make_shared("Right after Inner query", out.getInputHeader())); #endif @@ -434,7 +434,7 @@ std::optional generateViewChain( out.addSource(std::move(executing_inner_query)); -#ifdef ABORT_ON_LOGICAL_ERROR +#ifdef DEBUG_OR_SANITIZER_BUILD out.addSource(std::make_shared("Right before Inner query", out.getInputHeader())); #endif } diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index 7aa05ede737..27a2c28f26c 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -364,7 +364,7 @@ void RefreshTask::refreshTask() if (coordination.root_znode.last_attempt_replica == coordination.replica_name) { LOG_ERROR(log, "Znode {} indicates that this replica is running a refresh, but it isn't. Likely a bug.", coordination.path + "/running"); -#ifdef ABORT_ON_LOGICAL_ERROR +#ifdef DEBUG_OR_SANITIZER_BUILD abortOnFailedAssertion("Unexpected refresh lock in keeper"); #else coordination.running_znode_exists = false; diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp index ef0adc15186..a50865994d0 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp @@ -209,7 +209,7 @@ struct DeltaLakeMetadataImpl if (!object) throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to parse metadata file"); -#ifdef ABORT_ON_LOGICAL_ERROR +#ifdef DEBUG_OR_SANITIZER_BUILD std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM object->stringify(oss); LOG_TEST(log, "Metadata: {}", oss.str()); diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 0739c03b963..b7b58887a4f 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1596,7 +1596,7 @@ void StorageWindowView::writeIntoWindowView( return std::make_shared(stream_header); }); -#ifdef ABORT_ON_LOGICAL_ERROR +#ifdef DEBUG_OR_SANITIZER_BUILD builder.addSimpleTransform([&](const Block & stream_header) { return std::make_shared("StorageWindowView: Afrer tmp table before squashing", stream_header); @@ -1643,7 +1643,7 @@ void StorageWindowView::writeIntoWindowView( lateness_upper_bound); }); -#ifdef ABORT_ON_LOGICAL_ERROR +#ifdef DEBUG_OR_SANITIZER_BUILD builder.addSimpleTransform([&](const Block & stream_header) { return std::make_shared("StorageWindowView: Afrer WatermarkTransform", stream_header); @@ -1668,7 +1668,7 @@ void StorageWindowView::writeIntoWindowView( builder.addSimpleTransform([&](const Block & header_) { return std::make_shared(header_, convert_actions); }); } -#ifdef ABORT_ON_LOGICAL_ERROR +#ifdef DEBUG_OR_SANITIZER_BUILD builder.addSimpleTransform([&](const Block & stream_header) { return std::make_shared("StorageWindowView: Before out", stream_header); From d11c97f95f61013895e11a422299d159ee8f54ad Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 25 Nov 2024 23:18:07 +0100 Subject: [PATCH 24/50] Removed MOVE PARTITION from the test --- .../0_stateless/03008_azure_plain_rewritable.sh | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/tests/queries/0_stateless/03008_azure_plain_rewritable.sh b/tests/queries/0_stateless/03008_azure_plain_rewritable.sh index 29bfc92282f..fe56b5cedd3 100755 --- a/tests/queries/0_stateless/03008_azure_plain_rewritable.sh +++ b/tests/queries/0_stateless/03008_azure_plain_rewritable.sh @@ -50,15 +50,4 @@ detach table test_azure_mt; attach table test_azure_mt; " -${CLICKHOUSE_CLIENT} --query "drop table if exists test_azure_mt_dst" - -${CLICKHOUSE_CLIENT} -m --query " -create table test_azure_mt_dst (a Int32, b Int64, c Int64) -engine = MergeTree() partition by intDiv(a, 1000) order by tuple(a, b) -settings disk = '${container}' -" - -${CLICKHOUSE_CLIENT} -m --query " -alter table test_azure_mt move partition 0 to table test_azure_mt_dst" 2>&1 | grep -Fq "SUPPORT_IS_DISABLED" - ${CLICKHOUSE_CLIENT} --query "drop table test_azure_mt sync" From 575e17758d4268c6dcc668ec6b6e943fced68b45 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 26 Nov 2024 13:34:00 +0100 Subject: [PATCH 25/50] Adjust tests --- tests/integration/test_keeper_snapshots/test.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_keeper_snapshots/test.py b/tests/integration/test_keeper_snapshots/test.py index 1af3318e24b..b48657f4c3d 100644 --- a/tests/integration/test_keeper_snapshots/test.py +++ b/tests/integration/test_keeper_snapshots/test.py @@ -191,8 +191,9 @@ def test_invalid_snapshot(started_cluster): ] ) node.start_clickhouse(start_wait_sec=120, expected_to_fail=True) + assert node.contains_in_log("Failure to load from latest snapshot with index") assert node.contains_in_log( - "Aborting because of failure to load from latest snapshot with index" + "Manual intervention is necessary for recovery. Problematic snapshot can be removed but it will lead to data loss" ) node.stop_clickhouse() From 92a79b750e98d177d9b9327341001a0d72dd7f81 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 27 Nov 2024 23:50:23 +0100 Subject: [PATCH 26/50] Fix --- src/Storages/MergeTree/FutureMergedMutatedPart.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/FutureMergedMutatedPart.cpp b/src/Storages/MergeTree/FutureMergedMutatedPart.cpp index 959a0e50bca..83919e90569 100644 --- a/src/Storages/MergeTree/FutureMergedMutatedPart.cpp +++ b/src/Storages/MergeTree/FutureMergedMutatedPart.cpp @@ -88,7 +88,8 @@ void FutureMergedMutatedPart::assign(MergeTreeData::DataPartsVector parts_, Merg void FutureMergedMutatedPart::updatePath(const MergeTreeData & storage, const IReservation * reservation) { - path = storage.getFullPathOnDisk(reservation->getDisk()) + name + "/"; + path = fs::path(storage.getFullPathOnDisk(reservation->getDisk())) / name; + path += "/"; } } From 644437cfb8442a01acbd208475a2c2488d9e4af0 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 28 Nov 2024 01:23:53 +0100 Subject: [PATCH 27/50] Fix parsing a glob with one element --- src/Common/parseGlobs.cpp | 11 ++++++++++- src/Common/tests/gtest_makeRegexpPatternFromGlobs.cpp | 2 ++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/src/Common/parseGlobs.cpp b/src/Common/parseGlobs.cpp index de6caec3149..696a71ed403 100644 --- a/src/Common/parseGlobs.cpp +++ b/src/Common/parseGlobs.cpp @@ -35,7 +35,7 @@ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_glob } std::string escaped_with_globs = buf_for_escaping.str(); - static const re2::RE2 enum_or_range(R"({([\d]+\.\.[\d]+|[^{}*,]+,[^{}*]*[^{}*,])})"); /// regexp for {expr1,expr2,expr3} or {M..N}, where M and N - non-negative integers, expr's should be without "{", "}", "*" and "," + static const re2::RE2 enum_or_range(R"({([\d]+\.\.[\d]+|[^{}*,]+[^{}*]*[^{}*,])})"); /// regexp for {expr1,expr2,expr3} or {M..N}, where M and N - non-negative integers, expr's should be without "{", "}", "*" and "," std::string_view input(escaped_with_globs); std::string_view matched; std::ostringstream oss_for_replacing; // STYLE_CHECK_ALLOW_STD_STRING_STREAM @@ -48,6 +48,15 @@ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_glob if (!buffer.contains(',')) { + /// No dot or one dot in the filename. This is not a range. + if (buffer.find('.') == std::string::npos || buffer.find('.') == buffer.find_last_of('.')) + { + oss_for_replacing << buffer; + oss_for_replacing << ")"; + current_index = input.data() - escaped_with_globs.data(); + break; + } + size_t range_begin = 0; size_t range_end = 0; char point; diff --git a/src/Common/tests/gtest_makeRegexpPatternFromGlobs.cpp b/src/Common/tests/gtest_makeRegexpPatternFromGlobs.cpp index 6e15c0f712c..654767efc7f 100644 --- a/src/Common/tests/gtest_makeRegexpPatternFromGlobs.cpp +++ b/src/Common/tests/gtest_makeRegexpPatternFromGlobs.cpp @@ -12,6 +12,8 @@ TEST(Common, makeRegexpPatternFromGlobs) EXPECT_EQ(makeRegexpPatternFromGlobs("*"), "[^/]*"); EXPECT_EQ(makeRegexpPatternFromGlobs("/?"), "/[^/]"); EXPECT_EQ(makeRegexpPatternFromGlobs("/*"), "/[^/]*"); + EXPECT_EQ(makeRegexpPatternFromGlobs("{123}"), "(123)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("{test}"), "(test)"); EXPECT_EQ(makeRegexpPatternFromGlobs("*_{{a,b,c,d}}/?.csv"), "[^/]*_\\{(a|b|c|d)\\}/[^/]\\.csv"); /* Regex Parsing for {..} can have three possible cases 1) The left range width == the right range width From 8f723820521ffa8128c10cdd84e115204c222dde Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 28 Nov 2024 02:52:50 +0100 Subject: [PATCH 28/50] Split in two regexes --- src/Common/parseGlobs.cpp | 95 ++++++++++--------- .../gtest_makeRegexpPatternFromGlobs.cpp | 1 + 2 files changed, 49 insertions(+), 47 deletions(-) diff --git a/src/Common/parseGlobs.cpp b/src/Common/parseGlobs.cpp index 696a71ed403..b785a660285 100644 --- a/src/Common/parseGlobs.cpp +++ b/src/Common/parseGlobs.cpp @@ -35,70 +35,71 @@ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_glob } std::string escaped_with_globs = buf_for_escaping.str(); - static const re2::RE2 enum_or_range(R"({([\d]+\.\.[\d]+|[^{}*,]+[^{}*]*[^{}*,])})"); /// regexp for {expr1,expr2,expr3} or {M..N}, where M and N - non-negative integers, expr's should be without "{", "}", "*" and "," - std::string_view input(escaped_with_globs); + static const re2::RE2 range_regex(R"({([\d]+\.\.[\d]+)})"); /// regexp for {M..N}, where M and N - non-negative integers + static const re2::RE2 enum_regex(R"({([^{}*,]+[^{}*]*[^{}*,])})"); /// regexp for {expr1,expr2,expr3}, expr's should be without "{", "}", "*" and "," + std::string_view matched; + std::string_view input(escaped_with_globs); std::ostringstream oss_for_replacing; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss_for_replacing.exceptions(std::ios::failbit); size_t current_index = 0; - while (RE2::FindAndConsume(&input, enum_or_range, &matched)) + + while (RE2::FindAndConsume(&input, range_regex, &matched)) { std::string buffer(matched); oss_for_replacing << escaped_with_globs.substr(current_index, matched.data() - escaped_with_globs.data() - current_index - 1) << '('; - if (!buffer.contains(',')) + size_t range_begin = 0; + size_t range_end = 0; + char point; + ReadBufferFromString buf_range(buffer); + buf_range >> range_begin >> point >> point >> range_end; + + size_t range_begin_width = buffer.find('.'); + size_t range_end_width = buffer.size() - buffer.find_last_of('.') - 1; + bool leading_zeros = buffer[0] == '0'; + size_t output_width = 0; + + if (range_begin > range_end) //Descending Sequence {20..15} {9..01} { - /// No dot or one dot in the filename. This is not a range. - if (buffer.find('.') == std::string::npos || buffer.find('.') == buffer.find_last_of('.')) - { - oss_for_replacing << buffer; - oss_for_replacing << ")"; - current_index = input.data() - escaped_with_globs.data(); - break; - } + std::swap(range_begin,range_end); + leading_zeros = buffer[buffer.find_last_of('.')+1]=='0'; + std::swap(range_begin_width,range_end_width); + } + if (range_begin_width == 1 && leading_zeros) + output_width = 1; ///Special Case: {0..10} {0..999} + else + output_width = std::max(range_begin_width, range_end_width); - size_t range_begin = 0; - size_t range_end = 0; - char point; - ReadBufferFromString buf_range(buffer); - buf_range >> range_begin >> point >> point >> range_end; - - size_t range_begin_width = buffer.find('.'); - size_t range_end_width = buffer.size() - buffer.find_last_of('.') - 1; - bool leading_zeros = buffer[0] == '0'; - size_t output_width = 0; - - if (range_begin > range_end) //Descending Sequence {20..15} {9..01} - { - std::swap(range_begin,range_end); - leading_zeros = buffer[buffer.find_last_of('.')+1]=='0'; - std::swap(range_begin_width,range_end_width); - } - if (range_begin_width == 1 && leading_zeros) - output_width = 1; ///Special Case: {0..10} {0..999} - else - output_width = std::max(range_begin_width, range_end_width); + if (leading_zeros) + oss_for_replacing << std::setfill('0') << std::setw(static_cast(output_width)); + oss_for_replacing << range_begin; + for (size_t i = range_begin + 1; i <= range_end; ++i) + { + oss_for_replacing << '|'; if (leading_zeros) oss_for_replacing << std::setfill('0') << std::setw(static_cast(output_width)); - oss_for_replacing << range_begin; + oss_for_replacing << i; + } - for (size_t i = range_begin + 1; i <= range_end; ++i) - { - oss_for_replacing << '|'; - if (leading_zeros) - oss_for_replacing << std::setfill('0') << std::setw(static_cast(output_width)); - oss_for_replacing << i; - } - } - else - { - std::replace(buffer.begin(), buffer.end(), ',', '|'); - oss_for_replacing << buffer; - } oss_for_replacing << ")"; current_index = input.data() - escaped_with_globs.data(); } + + while (RE2::FindAndConsume(&input, enum_regex, &matched)) + { + std::string buffer(matched); + + oss_for_replacing << escaped_with_globs.substr(current_index, matched.data() - escaped_with_globs.data() - current_index - 1) << '('; + std::replace(buffer.begin(), buffer.end(), ',', '|'); + + oss_for_replacing << buffer; + oss_for_replacing << ")"; + + current_index = input.data() - escaped_with_globs.data(); + } + oss_for_replacing << escaped_with_globs.substr(current_index); std::string almost_res = oss_for_replacing.str(); WriteBufferFromOwnString buf_final_processing; diff --git a/src/Common/tests/gtest_makeRegexpPatternFromGlobs.cpp b/src/Common/tests/gtest_makeRegexpPatternFromGlobs.cpp index 654767efc7f..f1e04fc6dfd 100644 --- a/src/Common/tests/gtest_makeRegexpPatternFromGlobs.cpp +++ b/src/Common/tests/gtest_makeRegexpPatternFromGlobs.cpp @@ -14,6 +14,7 @@ TEST(Common, makeRegexpPatternFromGlobs) EXPECT_EQ(makeRegexpPatternFromGlobs("/*"), "/[^/]*"); EXPECT_EQ(makeRegexpPatternFromGlobs("{123}"), "(123)"); EXPECT_EQ(makeRegexpPatternFromGlobs("{test}"), "(test)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("{test.tar.gz}"), "(test\\.tar\\.gz)"); EXPECT_EQ(makeRegexpPatternFromGlobs("*_{{a,b,c,d}}/?.csv"), "[^/]*_\\{(a|b|c|d)\\}/[^/]\\.csv"); /* Regex Parsing for {..} can have three possible cases 1) The left range width == the right range width From 67504deac440657d58adf069c0f381e6d8fedb8e Mon Sep 17 00:00:00 2001 From: zhanglistar Date: Fri, 29 Nov 2024 16:14:41 +0800 Subject: [PATCH 29/50] move composed_data_type_output_format_mode to 24.12 --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index c60ceb4e867..db1a35a2deb 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -64,6 +64,7 @@ static std::initializer_list Date: Fri, 29 Nov 2024 16:18:59 +0800 Subject: [PATCH 30/50] move composed_data_type_output_format_mode to 24.12 --- src/Core/SettingsChangesHistory.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index db1a35a2deb..83fd91119db 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -64,7 +64,7 @@ static std::initializer_list Date: Fri, 29 Nov 2024 18:45:53 +0100 Subject: [PATCH 31/50] Init --- src/Interpreters/Set.cpp | 2 +- .../0_stateless/03278_dateTime_in_dateTime_bug.reference | 2 ++ .../queries/0_stateless/03278_dateTime_in_dateTime_bug.sql | 7 +++++++ 3 files changed, 10 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03278_dateTime_in_dateTime_bug.reference create mode 100644 tests/queries/0_stateless/03278_dateTime_in_dateTime_bug.sql diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index c6f0455652a..3a394a5dd02 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -440,7 +440,7 @@ ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) co } // If the original column is DateTime64, check for sub-second precision - if (isDateTime64(column_to_cast.column->getDataType())) + if (isDateTime64(column_to_cast.column->getDataType()) && !isDateTime64(result->getDataType())) { processDateTime64Column(column_to_cast, result, null_map_holder, null_map); } diff --git a/tests/queries/0_stateless/03278_dateTime_in_dateTime_bug.reference b/tests/queries/0_stateless/03278_dateTime_in_dateTime_bug.reference new file mode 100644 index 00000000000..b451df8c07a --- /dev/null +++ b/tests/queries/0_stateless/03278_dateTime_in_dateTime_bug.reference @@ -0,0 +1,2 @@ +1 2001-01-11 01:11:21.100 2001-01-11 01:11:21.100 +1 2001-01-11 01:11:21.100 2001-01-11 01:11:21.100 diff --git a/tests/queries/0_stateless/03278_dateTime_in_dateTime_bug.sql b/tests/queries/0_stateless/03278_dateTime_in_dateTime_bug.sql new file mode 100644 index 00000000000..8b8bf3b56da --- /dev/null +++ b/tests/queries/0_stateless/03278_dateTime_in_dateTime_bug.sql @@ -0,0 +1,7 @@ +CREATE TABLE datetime64_issue (id int, dt DateTime64(3), dtn Nullable(DateTime64(3))) ENGINE = MergeTree() ORDER BY id PRIMARY KEY id; + +INSERT INTO datetime64_issue(id, dt, dtn) VALUES (1, toDateTime64('2001-01-11 01:11:21.100', 3), toDateTime64('2001-01-11 01:11:21.100', 3)); + +SELECT * FROM datetime64_issue WHERE dt in (toDateTime64('2001-01-11 01:11:21.100', 3)); + +SELECT * FROM datetime64_issue WHERE dtn in (toDateTime64('2001-01-11 01:11:21.100', 3)); From 98a358db10bc1401d8c4199138c42959061a0618 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 29 Nov 2024 18:49:09 +0100 Subject: [PATCH 32/50] Fix tests --- ...g.reference => 03278_dateTime64_in_dateTime64_bug.reference} | 0 ..._dateTime_bug.sql => 03278_dateTime64_in_dateTime64_bug.sql} | 2 ++ 2 files changed, 2 insertions(+) rename tests/queries/0_stateless/{03278_dateTime_in_dateTime_bug.reference => 03278_dateTime64_in_dateTime64_bug.reference} (100%) rename tests/queries/0_stateless/{03278_dateTime_in_dateTime_bug.sql => 03278_dateTime64_in_dateTime64_bug.sql} (93%) diff --git a/tests/queries/0_stateless/03278_dateTime_in_dateTime_bug.reference b/tests/queries/0_stateless/03278_dateTime64_in_dateTime64_bug.reference similarity index 100% rename from tests/queries/0_stateless/03278_dateTime_in_dateTime_bug.reference rename to tests/queries/0_stateless/03278_dateTime64_in_dateTime64_bug.reference diff --git a/tests/queries/0_stateless/03278_dateTime_in_dateTime_bug.sql b/tests/queries/0_stateless/03278_dateTime64_in_dateTime64_bug.sql similarity index 93% rename from tests/queries/0_stateless/03278_dateTime_in_dateTime_bug.sql rename to tests/queries/0_stateless/03278_dateTime64_in_dateTime64_bug.sql index 8b8bf3b56da..5d6bd825faf 100644 --- a/tests/queries/0_stateless/03278_dateTime_in_dateTime_bug.sql +++ b/tests/queries/0_stateless/03278_dateTime64_in_dateTime64_bug.sql @@ -5,3 +5,5 @@ INSERT INTO datetime64_issue(id, dt, dtn) VALUES (1, toDateTime64('2001-01-11 01 SELECT * FROM datetime64_issue WHERE dt in (toDateTime64('2001-01-11 01:11:21.100', 3)); SELECT * FROM datetime64_issue WHERE dtn in (toDateTime64('2001-01-11 01:11:21.100', 3)); + +DROP TABLE datetime64_issue; From ca5faf14ca3e2e6546bb7c7a7f106ec535003a2e Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 29 Nov 2024 20:18:41 +0100 Subject: [PATCH 33/50] Update Set.cpp --- src/Interpreters/Set.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 3a394a5dd02..0b1b8d3c2b5 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -440,7 +440,7 @@ ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) co } // If the original column is DateTime64, check for sub-second precision - if (isDateTime64(column_to_cast.column->getDataType()) && !isDateTime64(result->getDataType())) + if (isDateTime64(column_to_cast.column->getDataType()) && isDateTime64(result->getDataType())) { processDateTime64Column(column_to_cast, result, null_map_holder, null_map); } From 5e0285d38a6cbb8c31fa9032fbf27787b9ee1970 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 29 Nov 2024 20:49:55 +0100 Subject: [PATCH 34/50] Measure time consumed by logging --- src/Common/ProfileEvents.cpp | 6 ++++++ src/Common/ProfileEvents.h | 3 +++ src/Common/logger_useful.h | 3 +++ 3 files changed, 12 insertions(+) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 2d0ef3e6c6a..6e7e0476800 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -814,6 +814,7 @@ The server successfully detected this situation and will download merged part fr M(LogWarning, "Number of log messages with level Warning", ValueType::Number) \ M(LogError, "Number of log messages with level Error", ValueType::Number) \ M(LogFatal, "Number of log messages with level Fatal", ValueType::Number) \ + M(LoggerElapsedNanoseconds, "Number of log messages with level Fatal", ValueType::Nanoseconds) \ \ M(InterfaceHTTPSendBytes, "Number of bytes sent through HTTP interfaces", ValueType::Bytes) \ M(InterfaceHTTPReceiveBytes, "Number of bytes received through HTTP interfaces", ValueType::Bytes) \ @@ -1087,6 +1088,11 @@ void incrementForLogMessage(Poco::Message::Priority priority) } } +void incrementLoggerElapsedNanoseconds(UInt64 ns) +{ + increment(LoggerElapsedNanoseconds, ns); +} + CountersIncrement::CountersIncrement(Counters::Snapshot const & snapshot) { init(); diff --git a/src/Common/ProfileEvents.h b/src/Common/ProfileEvents.h index 012fe4ffdfa..4235322ec2a 100644 --- a/src/Common/ProfileEvents.h +++ b/src/Common/ProfileEvents.h @@ -173,6 +173,9 @@ namespace ProfileEvents /// Increment a counter for log messages. void incrementForLogMessage(Poco::Message::Priority priority); + /// Increment time consumed by logging. + void incrementLoggerElapsedNanoseconds(UInt64 ns); + /// Get name of event by identifier. Returns statically allocated string. const char * getName(Event event); diff --git a/src/Common/logger_useful.h b/src/Common/logger_useful.h index 3488d6b56d8..8e008cfc5fc 100644 --- a/src/Common/logger_useful.h +++ b/src/Common/logger_useful.h @@ -11,6 +11,7 @@ #include #include #include +#include #define LogToStr(x, y) std::make_unique(x, y) @@ -69,6 +70,7 @@ namespace impl if (!_is_clients_log && !_logger->is((PRIORITY))) \ break; \ \ + Stopwatch _logger_watch; \ try \ { \ ProfileEvents::incrementForLogMessage(PRIORITY); \ @@ -122,6 +124,7 @@ namespace impl { \ ::write(STDERR_FILENO, static_cast(MESSAGE_FOR_EXCEPTION_ON_LOGGING), sizeof(MESSAGE_FOR_EXCEPTION_ON_LOGGING)); \ } \ + ProfileEvents::incrementLoggerElapsedNanoseconds(_logger_watch.elapsedNanoseconds()); \ } while (false) From bc3bc8913e3e2b9d24f817c0d12fe45b78d653f4 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 29 Nov 2024 20:59:07 +0100 Subject: [PATCH 35/50] Add nullable check for data types --- src/Interpreters/Set.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 0b1b8d3c2b5..60328123aa0 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -440,7 +440,7 @@ ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) co } // If the original column is DateTime64, check for sub-second precision - if (isDateTime64(column_to_cast.column->getDataType()) && isDateTime64(result->getDataType())) + if (isDateTime64(column_to_cast.column->getDataType()) && !isDateTime64(removeNullable(result)->getDataType())) { processDateTime64Column(column_to_cast, result, null_map_holder, null_map); } From 333b2918ba6470f88e9927255e65364ba85b937b Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 29 Nov 2024 21:25:28 +0100 Subject: [PATCH 36/50] Update check-style --- utils/check-style/check-style | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index c3b42be1519..d86ab8e029d 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -69,6 +69,7 @@ EXTERN_TYPES_EXCLUDES=( ProfileEvents::end ProfileEvents::increment ProfileEvents::incrementForLogMessage + ProfileEvents::incrementLoggerElapsedNanoseconds ProfileEvents::getName ProfileEvents::Timer ProfileEvents::Type From fc01a17c896bd186c23814a0eb852d3bd2155475 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 29 Nov 2024 22:31:35 +0100 Subject: [PATCH 37/50] Fix description --- src/Common/ProfileEvents.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 6e7e0476800..e1e62e93c81 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -814,7 +814,7 @@ The server successfully detected this situation and will download merged part fr M(LogWarning, "Number of log messages with level Warning", ValueType::Number) \ M(LogError, "Number of log messages with level Error", ValueType::Number) \ M(LogFatal, "Number of log messages with level Fatal", ValueType::Number) \ - M(LoggerElapsedNanoseconds, "Number of log messages with level Fatal", ValueType::Nanoseconds) \ + M(LoggerElapsedNanoseconds, "Cumulative time spend in logging", ValueType::Nanoseconds) \ \ M(InterfaceHTTPSendBytes, "Number of bytes sent through HTTP interfaces", ValueType::Bytes) \ M(InterfaceHTTPReceiveBytes, "Number of bytes received through HTTP interfaces", ValueType::Bytes) \ From d7f2f0e0985cd2f67cfd467e67a49a3c92060c5e Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 30 Nov 2024 01:05:00 +0000 Subject: [PATCH 38/50] better prewarm of mark and primary index caches --- src/Common/ProfileEvents.cpp | 1 + src/Storages/MergeTree/IMergeTreeDataPart.cpp | 56 +++++++++++++++++-- src/Storages/MergeTree/IMergeTreeDataPart.h | 18 ++++++ .../MergeTree/IMergedBlockOutputStream.cpp | 6 -- .../MergeTree/IMergedBlockOutputStream.h | 6 +- .../MergeTree/MergeFromLogEntryTask.cpp | 6 +- .../MergeTree/MergePlainMergeTreeTask.cpp | 6 +- src/Storages/MergeTree/MergeTask.cpp | 2 + src/Storages/MergeTree/MergeTreeData.cpp | 38 +++++++++---- src/Storages/MergeTree/MergeTreeData.h | 8 +-- .../MergeTree/MergeTreeDataPartCompact.cpp | 10 ++++ .../MergeTree/MergeTreeDataPartCompact.h | 1 + .../MergeTree/MergeTreeDataPartWide.cpp | 21 +++++++ .../MergeTree/MergeTreeDataPartWide.h | 1 + .../MergeTree/MergeTreeDataWriter.cpp | 11 ++-- .../MergeTree/MergeTreeMarksLoader.cpp | 4 +- src/Storages/MergeTree/MergeTreeMarksLoader.h | 3 + src/Storages/MergeTree/MergeTreeSettings.cpp | 1 + .../MergeTree/MergedBlockOutputStream.cpp | 5 +- .../MergeTree/MergedBlockOutputStream.h | 1 + .../MergedColumnOnlyOutputStream.cpp | 5 +- .../MergeTree/MergedColumnOnlyOutputStream.h | 1 + src/Storages/MergeTree/MutateTask.cpp | 8 ++- .../ReplicatedMergeTreeCleanupThread.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeSink.h | 1 - src/Storages/StorageMergeTree.cpp | 6 +- src/Storages/StorageReplicatedMergeTree.cpp | 10 ++-- .../03277_prewarm_cache_2.reference | 12 ++++ .../0_stateless/03277_prewarm_cache_2.sql | 54 ++++++++++++++++++ 29 files changed, 252 insertions(+), 52 deletions(-) create mode 100644 tests/queries/0_stateless/03277_prewarm_cache_2.reference create mode 100644 tests/queries/0_stateless/03277_prewarm_cache_2.sql diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 2d0ef3e6c6a..ab0d9f558a8 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -229,6 +229,7 @@ \ M(WaitMarksLoadMicroseconds, "Time spent loading marks", ValueType::Microseconds) \ M(BackgroundLoadingMarksTasks, "Number of background tasks for loading marks", ValueType::Number) \ + M(LoadedMarksFiles, "Number of mark files loaded.", ValueType::Number) \ M(LoadedMarksCount, "Number of marks loaded (total across columns).", ValueType::Number) \ M(LoadedMarksMemoryBytes, "Size of in-memory representations of loaded marks.", ValueType::Bytes) \ M(LoadedPrimaryIndexFiles, "Number of primary index files loaded.", ValueType::Number) \ diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index d098107da01..e880c70ba8d 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -387,7 +387,7 @@ IMergeTreeDataPart::IndexPtr IMergeTreeDataPart::getIndex() const IMergeTreeDataPart::IndexPtr IMergeTreeDataPart::loadIndexToCache(PrimaryIndexCache & index_cache) const { - auto key = PrimaryIndexCache::hash(getDataPartStorage().getFullPath()); + auto key = PrimaryIndexCache::hash(getRelativePathOfActivePart()); auto callback = [this] { return loadIndex(); }; return index_cache.getOrSet(key, callback); } @@ -398,7 +398,7 @@ void IMergeTreeDataPart::moveIndexToCache(PrimaryIndexCache & index_cache) if (!index) return; - auto key = PrimaryIndexCache::hash(getDataPartStorage().getFullPath()); + auto key = PrimaryIndexCache::hash(getRelativePathOfActivePart()); index_cache.set(key, std::const_pointer_cast(index)); index.reset(); @@ -406,6 +406,15 @@ void IMergeTreeDataPart::moveIndexToCache(PrimaryIndexCache & index_cache) projection->moveIndexToCache(index_cache); } +void IMergeTreeDataPart::removeIndexFromCache(PrimaryIndexCache * index_cache) const +{ + if (!index_cache) + return; + + auto key = PrimaryIndexCache::hash(getRelativePathOfActivePart()); + index_cache->remove(key); +} + void IMergeTreeDataPart::setIndex(Columns index_columns) { std::scoped_lock lock(index_mutex); @@ -574,17 +583,49 @@ bool IMergeTreeDataPart::isMovingPart() const return part_directory_path.parent_path().filename() == "moving"; } +void IMergeTreeDataPart::clearCaches() +{ + if (cleared_data_in_caches.exchange(true) || is_duplicate) + return; + + size_t uncompressed_bytes = getBytesUncompressedOnDisk(); + + /// Remove index and marks from cache if it was prewarmed to avoid threshing it with outdated data. + /// Do not remove in other cases to avoid extra contention on caches. + removeMarksFromCache(storage.getMarkCacheToPrewarm(uncompressed_bytes).get()); + removeIndexFromCache(storage.getPrimaryIndexCacheToPrewarm(uncompressed_bytes).get()); +} + +bool IMergeTreeDataPart::mayStoreDataInCaches() const +{ + size_t uncompressed_bytes = getBytesUncompressedOnDisk(); + + auto mark_cache = storage.getMarkCacheToPrewarm(uncompressed_bytes); + auto index_cache = storage.getPrimaryIndexCacheToPrewarm(uncompressed_bytes); + + return (mark_cache || index_cache) && !cleared_data_in_caches; +} + void IMergeTreeDataPart::removeIfNeeded() noexcept { assert(assertHasValidVersionMetadata()); - if (!is_temp && state != MergeTreeDataPartState::DeleteOnDestroy) - return; - std::string path; + try { path = getDataPartStorage().getRelativePath(); + clearCaches(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__, fmt::format("while removing part {} with path {}", name, path)); + } + if (!is_temp && state != MergeTreeDataPartState::DeleteOnDestroy) + return; + + try + { if (!getDataPartStorage().exists()) // path return; @@ -2113,6 +2154,11 @@ std::optional IMergeTreeDataPart::getRelativePathForDetachedPart(const S return {}; } +String IMergeTreeDataPart::getRelativePathOfActivePart() const +{ + return fs::path(getDataPartStorage().getFullRootPath()) / name / ""; +} + void IMergeTreeDataPart::renameToDetached(const String & prefix) { auto path_to_detach = getRelativePathForDetachedPart(prefix, /* broken */ false); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 013d024aaa8..e563a4e1906 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -185,6 +186,15 @@ public: /// Loads marks and saves them into mark cache for specified columns. virtual void loadMarksToCache(const Names & column_names, MarkCache * mark_cache) const = 0; + /// Removes marks from cache for all columns in part. + virtual void removeMarksFromCache(MarkCache * mark_cache) const = 0; + + /// Removes data related to data part from mark and primary index caches. + void clearCaches(); + + /// Returns true if data related to data part may be stored in mark and primary index caches. + bool mayStoreDataInCaches() const; + String getMarksFileExtension() const { return index_granularity_info.mark_type.getFileExtension(); } /// Generate the new name for this part according to `new_part_info` and min/max dates from the old name. @@ -376,6 +386,7 @@ public: IndexPtr getIndex() const; IndexPtr loadIndexToCache(PrimaryIndexCache & index_cache) const; void moveIndexToCache(PrimaryIndexCache & index_cache); + void removeIndexFromCache(PrimaryIndexCache * index_cache) const; void setIndex(Columns index_columns); void unloadIndex(); @@ -436,6 +447,10 @@ public: std::optional getRelativePathForPrefix(const String & prefix, bool detached = false, bool broken = false) const; + /// This method ignores current tmp prefix of part and returns + /// the name of part when it was or will be in Active state. + String getRelativePathOfActivePart() const; + bool isProjectionPart() const { return parent_part != nullptr; } /// Check if the part is in the `/moving` directory @@ -757,6 +772,9 @@ private: /// This ugly flag is needed for debug assertions only mutable bool part_is_probably_removed_from_disk = false; + + /// If it's true then data related to this part is cleared from mark and index caches. + mutable std::atomic_bool cleared_data_in_caches = false; }; using MergeTreeDataPartPtr = std::shared_ptr; diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index dd4b02de0a3..eb904a8e2ef 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -120,10 +120,4 @@ NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart( return remove_files; } -PlainMarksByName IMergedBlockOutputStream::releaseCachedMarks() -{ - if (!writer) - return {}; - return writer->releaseCachedMarks(); -} } diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.h b/src/Storages/MergeTree/IMergedBlockOutputStream.h index 77f22711b8d..efc5784dd43 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.h +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.h @@ -35,7 +35,10 @@ public: return writer->getIndexGranularity(); } - PlainMarksByName releaseCachedMarks(); + PlainMarksByName releaseCachedMarks() + { + return writer ? writer->releaseCachedMarks() : PlainMarksByName{}; + } size_t getNumberOfOpenStreams() const { @@ -43,7 +46,6 @@ public: } protected: - /// Remove all columns marked expired in data_part. Also, clears checksums /// and columns array. Return set of removed files names. NameSet removeEmptyColumnsFromPart( diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index fa03b3f63fb..0b89fd711a8 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -445,12 +445,14 @@ bool MergeFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrite finish_callback = [storage_ptr = &storage]() { storage_ptr->merge_selecting_task->schedule(); }; ProfileEvents::increment(ProfileEvents::ReplicatedPartMerges); - if (auto mark_cache = storage.getMarkCacheToPrewarm()) + size_t bytes_uncompressed = part->getBytesUncompressedOnDisk(); + + if (auto mark_cache = storage.getMarkCacheToPrewarm(bytes_uncompressed)) addMarksToCache(*part, cached_marks, mark_cache.get()); /// Move index to cache and reset it here because we need /// a correct part name after rename for a key of cache entry. - if (auto index_cache = storage.getPrimaryIndexCacheToPrewarm()) + if (auto index_cache = storage.getPrimaryIndexCacheToPrewarm(bytes_uncompressed)) part->moveIndexToCache(*index_cache); write_part_log({}); diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp index 54fc58d6056..750c7b0de2c 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp @@ -152,13 +152,15 @@ void MergePlainMergeTreeTask::finish() ThreadFuzzer::maybeInjectSleep(); ThreadFuzzer::maybeInjectMemoryLimitException(); - if (auto mark_cache = storage.getMarkCacheToPrewarm()) + size_t bytes_uncompressed = new_part->getBytesUncompressedOnDisk(); + + if (auto mark_cache = storage.getMarkCacheToPrewarm(bytes_uncompressed)) { auto marks = merge_task->releaseCachedMarks(); addMarksToCache(*new_part, marks, mark_cache.get()); } - if (auto index_cache = storage.getPrimaryIndexCacheToPrewarm()) + if (auto index_cache = storage.getPrimaryIndexCacheToPrewarm(bytes_uncompressed)) { /// Move index to cache and reset it here because we need /// a correct part name after rename for a key of cache entry. diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index f9eeea73606..b296f654651 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -578,6 +578,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() const ctx->compression_codec, std::move(index_granularity_ptr), global_ctx->txn ? global_ctx->txn->tid : Tx::PrehistoricTID, + global_ctx->merge_list_element_ptr->total_size_bytes_compressed, /*reset_columns=*/ true, ctx->blocks_are_granules_size, global_ctx->context->getWriteSettings()); @@ -1125,6 +1126,7 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const getStatisticsForColumns(columns_list, global_ctx->metadata_snapshot), ctx->compression_codec, global_ctx->to->getIndexGranularity(), + global_ctx->merge_list_element_ptr->total_size_bytes_uncompressed, &global_ctx->written_offset_columns); ctx->column_elems_written = 0; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 8ca976a343d..cf1162985b0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -238,6 +238,7 @@ namespace MergeTreeSetting extern const MergeTreeSettingsBool prewarm_mark_cache; extern const MergeTreeSettingsBool primary_key_lazy_load; extern const MergeTreeSettingsBool enforce_index_structure_match_on_partition_manipulation; + extern const MergeTreeSettingsUInt64 min_bytes_to_prewarm_caches; } namespace ServerSetting @@ -2366,19 +2367,31 @@ PrimaryIndexCachePtr MergeTreeData::getPrimaryIndexCache() const return getContext()->getPrimaryIndexCache(); } -PrimaryIndexCachePtr MergeTreeData::getPrimaryIndexCacheToPrewarm() const +PrimaryIndexCachePtr MergeTreeData::getPrimaryIndexCacheToPrewarm(size_t part_uncompressed_bytes) const { if (!(*getSettings())[MergeTreeSetting::prewarm_primary_key_cache]) return nullptr; + /// Do not load data to caches for small parts because + /// they will be likely replaced by merge immediately. + size_t min_bytes_to_prewarm = (*getSettings())[MergeTreeSetting::min_bytes_to_prewarm_caches]; + if (part_uncompressed_bytes && part_uncompressed_bytes < min_bytes_to_prewarm) + return nullptr; + return getPrimaryIndexCache(); } -MarkCachePtr MergeTreeData::getMarkCacheToPrewarm() const +MarkCachePtr MergeTreeData::getMarkCacheToPrewarm(size_t part_uncompressed_bytes) const { if (!(*getSettings())[MergeTreeSetting::prewarm_mark_cache]) return nullptr; + /// Do not load data to caches for small parts because + /// they will be likely replaced by merge immediately. + size_t min_bytes_to_prewarm = (*getSettings())[MergeTreeSetting::min_bytes_to_prewarm_caches]; + if (part_uncompressed_bytes && part_uncompressed_bytes < min_bytes_to_prewarm) + return nullptr; + return getContext()->getMarkCache(); } @@ -9033,7 +9046,8 @@ std::pair MergeTreeData::createE ColumnsStatistics{}, compression_codec, std::make_shared(), - txn ? txn->tid : Tx::PrehistoricTID); + txn ? txn->tid : Tx::PrehistoricTID, + /*part_uncompressed_bytes=*/ 0); bool sync_on_insert = (*settings)[MergeTreeSetting::fsync_after_insert]; @@ -9106,14 +9120,14 @@ void MergeTreeData::unloadPrimaryKeys() } } -size_t MergeTreeData::unloadPrimaryKeysOfOutdatedParts() +size_t MergeTreeData::unloadPrimaryKeysAndClearCachesOfOutdatedParts() { /// If the method is already called from another thread, then we don't need to do anything. std::unique_lock lock(unload_primary_key_mutex, std::defer_lock); if (!lock.try_lock()) return 0; - DataPartsVector parts_to_unload_index; + DataPartsVector parts_to_clear; { auto parts_lock = lockParts(); @@ -9124,18 +9138,22 @@ size_t MergeTreeData::unloadPrimaryKeysOfOutdatedParts() /// Outdated part may be hold by SELECT query and still needs the index. /// This check requires lock of index_mutex but if outdated part is unique then there is no /// contention on it, so it's relatively cheap and it's ok to check under a global parts lock. - if (isSharedPtrUnique(part) && part->isIndexLoaded()) - parts_to_unload_index.push_back(part); + if (isSharedPtrUnique(part) && (part->isIndexLoaded() || part->mayStoreDataInCaches())) + parts_to_clear.push_back(part); } } - for (const auto & part : parts_to_unload_index) + for (const auto & part : parts_to_clear) { - const_cast(*part).unloadIndex(); + auto & part_mut = const_cast(*part); + + part_mut.unloadIndex(); + part_mut.clearCaches(); + LOG_TEST(log, "Unloaded primary key for outdated part {}", part->name); } - return parts_to_unload_index.size(); + return parts_to_clear.size(); } void MergeTreeData::verifySortingKey(const KeyDescription & sorting_key) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 848e26a0a8b..fcaf956f480 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -511,9 +511,9 @@ public: /// Returns a pointer to primary index cache if it is enabled. PrimaryIndexCachePtr getPrimaryIndexCache() const; /// Returns a pointer to primary index cache if it is enabled and required to be prewarmed. - PrimaryIndexCachePtr getPrimaryIndexCacheToPrewarm() const; + PrimaryIndexCachePtr getPrimaryIndexCacheToPrewarm(size_t part_uncompressed_bytes) const; /// Returns a pointer to primary mark cache if it is required to be prewarmed. - MarkCachePtr getMarkCacheToPrewarm() const; + MarkCachePtr getMarkCacheToPrewarm(size_t part_uncompressed_bytes) const; /// Prewarm mark cache and primary index cache for the most recent data parts. void prewarmCaches(ThreadPool & pool, MarkCachePtr mark_cache, PrimaryIndexCachePtr index_cache); @@ -1166,7 +1166,7 @@ public: /// Unloads primary keys of outdated parts that are not used by any query. /// Returns the number of parts for which index was unloaded. - size_t unloadPrimaryKeysOfOutdatedParts(); + size_t unloadPrimaryKeysAndClearCachesOfOutdatedParts(); protected: friend class IMergeTreeDataPart; @@ -1335,7 +1335,7 @@ protected: std::mutex grab_old_parts_mutex; /// The same for clearOldTemporaryDirectories. std::mutex clear_old_temporary_directories_mutex; - /// The same for unloadPrimaryKeysOfOutdatedParts. + /// The same for unloadPrimaryKeysAndClearCachesOfOutdatedParts. std::mutex unload_primary_key_mutex; void checkProperties( diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 401184eeb36..91e70c68b36 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -175,6 +175,16 @@ void MergeTreeDataPartCompact::loadMarksToCache(const Names & column_names, Mark loader.loadMarks(); } +void MergeTreeDataPartCompact::removeMarksFromCache(MarkCache * mark_cache) const +{ + if (!mark_cache) + return; + + auto mark_path = index_granularity_info.getMarksFilePath(DATA_FILE_NAME); + auto key = MarkCache::hash(fs::path(getRelativePathOfActivePart()) / mark_path); + mark_cache->remove(key); +} + bool MergeTreeDataPartCompact::hasColumnFiles(const NameAndTypePair & column) const { if (!getColumnPosition(column.getNameInStorage())) diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index d9128fc5e0c..f55de304502 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -55,6 +55,7 @@ public: std::optional getFileNameForColumn(const NameAndTypePair & /* column */) const override { return DATA_FILE_NAME; } void loadMarksToCache(const Names & column_names, MarkCache * mark_cache) const override; + void removeMarksFromCache(MarkCache * mark_cache) const override; ~MergeTreeDataPartCompact() override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index f40b89898c8..06cb3c865a0 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -244,6 +244,27 @@ void MergeTreeDataPartWide::loadMarksToCache(const Names & column_names, MarkCac loader->loadMarks(); } +void MergeTreeDataPartWide::removeMarksFromCache(MarkCache * mark_cache) const +{ + if (!mark_cache) + return; + + const auto & serializations = getSerializations(); + for (const auto & [column_name, serialization] : serializations) + { + serialization->enumerateStreams([&](const auto & subpath) + { + auto stream_name = getStreamNameForColumn(column_name, subpath, checksums); + if (!stream_name) + return; + + auto mark_path = index_granularity_info.getMarksFilePath(*stream_name); + auto key = MarkCache::hash(fs::path(getRelativePathOfActivePart()) / mark_path); + mark_cache->remove(key); + }); + } +} + bool MergeTreeDataPartWide::isStoredOnRemoteDisk() const { return getDataPartStorage().isStoredOnRemoteDisk(); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index 2af47b7b943..c845977423e 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -52,6 +52,7 @@ public: std::optional getColumnModificationTime(const String & column_name) const override; void loadMarksToCache(const Names & column_names, MarkCache * mark_cache) const override; + void removeMarksFromCache(MarkCache * mark_cache) const override; protected: static void loadIndexGranularityImpl( diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index e6e1388515e..897471acee9 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -225,12 +225,13 @@ void MergeTreeDataWriter::TemporaryPart::finalize() projection->getDataPartStorage().precommitTransaction(); } +/// This method must be called after rename and commit of part +/// because a correct path is required for the keys of caches. void MergeTreeDataWriter::TemporaryPart::prewarmCaches() { - /// This method must be called after rename and commit of part - /// because a correct path is required for the keys of caches. + size_t bytes_uncompressed = part->getBytesUncompressedOnDisk(); - if (auto mark_cache = part->storage.getMarkCacheToPrewarm()) + if (auto mark_cache = part->storage.getMarkCacheToPrewarm(bytes_uncompressed)) { for (const auto & stream : streams) { @@ -239,7 +240,7 @@ void MergeTreeDataWriter::TemporaryPart::prewarmCaches() } } - if (auto index_cache = part->storage.getPrimaryIndexCacheToPrewarm()) + if (auto index_cache = part->storage.getPrimaryIndexCacheToPrewarm(bytes_uncompressed)) { /// Index was already set during writing. Now move it to cache. part->moveIndexToCache(*index_cache); @@ -726,6 +727,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( compression_codec, std::move(index_granularity_ptr), context->getCurrentTransaction() ? context->getCurrentTransaction()->tid : Tx::PrehistoricTID, + block.bytes(), /*reset_columns=*/ false, /*blocks_are_granules_size=*/ false, context->getWriteSettings()); @@ -880,6 +882,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( compression_codec, std::move(index_granularity_ptr), Tx::PrehistoricTID, + block.bytes(), /*reset_columns=*/ false, /*blocks_are_granules_size=*/ false, data.getContext()->getWriteSettings()); diff --git a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp index 30fe5080ee5..b4523c9116d 100644 --- a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp +++ b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp @@ -16,6 +16,7 @@ namespace ProfileEvents { extern const Event WaitMarksLoadMicroseconds; extern const Event BackgroundLoadingMarksTasks; + extern const Event LoadedMarksFiles; extern const Event LoadedMarksCount; extern const Event LoadedMarksMemoryBytes; } @@ -203,6 +204,7 @@ MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksImpl() auto res = std::make_shared(plain_marks); + ProfileEvents::increment(ProfileEvents::LoadedMarksFiles); ProfileEvents::increment(ProfileEvents::LoadedMarksCount, marks_count * num_columns_in_mark); ProfileEvents::increment(ProfileEvents::LoadedMarksMemoryBytes, res->approximateMemoryUsage()); @@ -264,7 +266,7 @@ void addMarksToCache(const IMergeTreeDataPart & part, const PlainMarksByName & c for (const auto & [stream_name, marks] : cached_marks) { auto mark_path = part.index_granularity_info.getMarksFilePath(stream_name); - auto key = MarkCache::hash(fs::path(part.getDataPartStorage().getFullPath()) / mark_path); + auto key = MarkCache::hash(fs::path(part.getRelativePathOfActivePart()) / mark_path); mark_cache->set(key, std::make_shared(*marks)); } } diff --git a/src/Storages/MergeTree/MergeTreeMarksLoader.h b/src/Storages/MergeTree/MergeTreeMarksLoader.h index e031700d6a7..de202ef76ba 100644 --- a/src/Storages/MergeTree/MergeTreeMarksLoader.h +++ b/src/Storages/MergeTree/MergeTreeMarksLoader.h @@ -83,6 +83,9 @@ struct MergeTreeSettings; /// Adds computed marks for part to the marks cache. void addMarksToCache(const IMergeTreeDataPart & part, const PlainMarksByName & cached_marks, MarkCache * mark_cache); +/// Removes cached marks for all columns from part. +void removeMarksFromCache(const IMergeTreeDataPart & part, MarkCache * mark_cache); + /// Returns the list of columns suitable for prewarming of mark cache according to settings. Names getColumnsToPrewarmMarks(const MergeTreeSettings & settings, const NamesAndTypesList & columns_list); diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index f62c7e53a46..33d06227b82 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -244,6 +244,7 @@ namespace ErrorCodes DECLARE(Bool, prewarm_primary_key_cache, false, "If true primary index cache will be prewarmed by saving marks to mark cache on inserts, merges, fetches and on startup of server", 0) \ DECLARE(Bool, prewarm_mark_cache, false, "If true mark cache will be prewarmed by saving marks to mark cache on inserts, merges, fetches and on startup of server", 0) \ DECLARE(String, columns_to_prewarm_mark_cache, "", "List of columns to prewarm mark cache for (if enabled). Empty means all columns", 0) \ + DECLARE(UInt64, min_bytes_to_prewarm_caches, 0, "Minimal size (uncomressed bytes) to prewarm mark cache and primary index cache for new parts", 0) \ /** Projection settings. */ \ DECLARE(UInt64, max_projections, 25, "The maximum number of merge tree projections.", 0) \ DECLARE(LightweightMutationProjectionMode, lightweight_mutation_projection_mode, LightweightMutationProjectionMode::THROW, "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop projections of this table's relevant parts, or rebuild the projections.", 0) \ diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index a9fd1fad6c5..87b3d238e3b 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -29,6 +29,7 @@ MergedBlockOutputStream::MergedBlockOutputStream( CompressionCodecPtr default_codec_, MergeTreeIndexGranularityPtr index_granularity_ptr, TransactionID tid, + size_t part_uncompressed_bytes, bool reset_columns_, bool blocks_are_granules_size, const WriteSettings & write_settings_) @@ -38,9 +39,9 @@ MergedBlockOutputStream::MergedBlockOutputStream( , write_settings(write_settings_) { /// Save marks in memory if prewarm is enabled to avoid re-reading marks file. - bool save_marks_in_cache = data_part->storage.getMarkCacheToPrewarm() != nullptr; + bool save_marks_in_cache = data_part->storage.getMarkCacheToPrewarm(part_uncompressed_bytes) != nullptr; /// Save primary index in memory if cache is disabled or is enabled with prewarm to avoid re-reading primary index file. - bool save_primary_index_in_memory = !data_part->storage.getPrimaryIndexCache() || data_part->storage.getPrimaryIndexCacheToPrewarm(); + bool save_primary_index_in_memory = !data_part->storage.getPrimaryIndexCache() || data_part->storage.getPrimaryIndexCacheToPrewarm(part_uncompressed_bytes); MergeTreeWriterSettings writer_settings( data_part->storage.getContext()->getSettingsRef(), diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index 5e06b040425..2f6205427d1 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -24,6 +24,7 @@ public: CompressionCodecPtr default_codec_, MergeTreeIndexGranularityPtr index_granularity_ptr, TransactionID tid, + size_t part_uncompressed_bytes, bool reset_columns_ = false, bool blocks_are_granules_size = false, const WriteSettings & write_settings = {}); diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 9f6ab952bc3..1f8668fc796 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -21,13 +21,14 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( const ColumnsStatistics & stats_to_recalc, CompressionCodecPtr default_codec, MergeTreeIndexGranularityPtr index_granularity_ptr, + size_t part_uncompressed_bytes, WrittenOffsetColumns * offset_columns) : IMergedBlockOutputStream(data_part->storage.getSettings(), data_part->getDataPartStoragePtr(), metadata_snapshot_, columns_list_, /*reset_columns=*/ true) { /// Save marks in memory if prewarm is enabled to avoid re-reading marks file. - bool save_marks_in_cache = data_part->storage.getMarkCacheToPrewarm() != nullptr; + bool save_marks_in_cache = data_part->storage.getMarkCacheToPrewarm(part_uncompressed_bytes) != nullptr; /// Save primary index in memory if cache is disabled or is enabled with prewarm to avoid re-reading priamry index file. - bool save_primary_index_in_memory = !data_part->storage.getPrimaryIndexCache() || data_part->storage.getPrimaryIndexCacheToPrewarm(); + bool save_primary_index_in_memory = !data_part->storage.getPrimaryIndexCache() || data_part->storage.getPrimaryIndexCacheToPrewarm(part_uncompressed_bytes); /// Granularity is never recomputed while writing only columns. MergeTreeWriterSettings writer_settings( diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h index 5fc8e3beb42..2fe054c782b 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h @@ -22,6 +22,7 @@ public: const ColumnsStatistics & stats_to_recalc, CompressionCodecPtr default_codec, MergeTreeIndexGranularityPtr index_granularity_ptr, + size_t part_uncompressed_bytes, WrittenOffsetColumns * offset_columns = nullptr); void write(const Block & block) override; diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index e54511166a2..da49740511e 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1636,8 +1636,8 @@ private: else { index_granularity_ptr = createMergeTreeIndexGranularity( - ctx->new_data_part->rows_count, - ctx->new_data_part->getBytesUncompressedOnDisk(), + ctx->source_part->rows_count, + ctx->source_part->getBytesUncompressedOnDisk(), *ctx->data->getSettings(), ctx->new_data_part->index_granularity_info, /*blocks_are_granules=*/ false); @@ -1652,6 +1652,7 @@ private: ctx->compression_codec, std::move(index_granularity_ptr), ctx->txn ? ctx->txn->tid : Tx::PrehistoricTID, + ctx->source_part->getBytesUncompressedOnDisk(), /*reset_columns=*/ true, /*blocks_are_granules_size=*/ false, ctx->context->getWriteSettings()); @@ -1887,7 +1888,8 @@ private: std::vector(ctx->indices_to_recalc.begin(), ctx->indices_to_recalc.end()), ColumnsStatistics(ctx->stats_to_recalc.begin(), ctx->stats_to_recalc.end()), ctx->compression_codec, - ctx->source_part->index_granularity); + ctx->source_part->index_granularity, + ctx->source_part->getBytesUncompressedOnDisk()); ctx->mutating_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); ctx->mutating_pipeline.setProgressCallback(ctx->progress_callback); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index ad03995d273..40d4058453e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -193,7 +193,7 @@ Float32 ReplicatedMergeTreeCleanupThread::iterate() cleaned_part_like += storage.clearEmptyParts(); } - cleaned_part_like += storage.unloadPrimaryKeysOfOutdatedParts(); + cleaned_part_like += storage.unloadPrimaryKeysAndClearCachesOfOutdatedParts(); /// We need to measure the number of removed objects somehow (for better scheduling), /// but just summing the number of removed async blocks, logs, and empty parts does not make any sense. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index b467cc167f8..c505a841ea2 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -130,7 +130,6 @@ private: std::unique_ptr delayed_chunk; void finishDelayedChunk(const ZooKeeperWithFaultInjectionPtr & zookeeper); - void prewarmCaches(const MergeTreeDataWriter::TemporaryPart & temp_part) const; }; using ReplicatedMergeTreeSinkWithAsyncDeduplicate = ReplicatedMergeTreeSinkImpl; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 166305e8910..a99be13ae24 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -158,8 +158,8 @@ StorageMergeTree::StorageMergeTree( prewarmCaches( getActivePartsLoadingThreadPool().get(), - getMarkCacheToPrewarm(), - getPrimaryIndexCacheToPrewarm()); + getMarkCacheToPrewarm(0), + getPrimaryIndexCacheToPrewarm(0)); } @@ -1522,7 +1522,7 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign cleared_count += clearOldPartsFromFilesystem(); cleared_count += clearOldMutations(); cleared_count += clearEmptyParts(); - cleared_count += unloadPrimaryKeysOfOutdatedParts(); + cleared_count += unloadPrimaryKeysAndClearCachesOfOutdatedParts(); return cleared_count; /// TODO maybe take into account number of cleared objects when calculating backoff }, common_assignee_trigger, getStorageID()), /* need_trigger */ false); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index bd476625081..ac1fbed2ec7 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -515,8 +515,8 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( prewarmCaches( getActivePartsLoadingThreadPool().get(), - getMarkCacheToPrewarm(), - getPrimaryIndexCacheToPrewarm()); + getMarkCacheToPrewarm(0), + getPrimaryIndexCacheToPrewarm(0)); if (LoadingStrictnessLevel::ATTACH <= mode) { @@ -5089,13 +5089,15 @@ bool StorageReplicatedMergeTree::fetchPart( ProfileEvents::increment(ProfileEvents::ObsoleteReplicatedParts); } - if (auto mark_cache = getMarkCacheToPrewarm()) + size_t bytes_uncompressed = part->getBytesUncompressedOnDisk(); + + if (auto mark_cache = getMarkCacheToPrewarm(bytes_uncompressed)) { auto column_names = getColumnsToPrewarmMarks(*getSettings(), part->getColumns()); part->loadMarksToCache(column_names, mark_cache.get()); } - if (auto index_cache = getPrimaryIndexCacheToPrewarm()) + if (auto index_cache = getPrimaryIndexCacheToPrewarm(bytes_uncompressed)) { part->loadIndexToCache(*index_cache); } diff --git a/tests/queries/0_stateless/03277_prewarm_cache_2.reference b/tests/queries/0_stateless/03277_prewarm_cache_2.reference new file mode 100644 index 00000000000..1bb2a47bd1e --- /dev/null +++ b/tests/queries/0_stateless/03277_prewarm_cache_2.reference @@ -0,0 +1,12 @@ +MarkCacheFiles 3 +PrimaryIndexCacheFiles 1 +550 +MarkCacheFiles 5 +PrimaryIndexCacheFiles 2 +550 +MarkCacheFiles 3 +PrimaryIndexCacheFiles 1 +MarkCacheFiles 0 +PrimaryIndexCacheFiles 0 +2 1 +0 0 diff --git a/tests/queries/0_stateless/03277_prewarm_cache_2.sql b/tests/queries/0_stateless/03277_prewarm_cache_2.sql new file mode 100644 index 00000000000..d8974b3cee0 --- /dev/null +++ b/tests/queries/0_stateless/03277_prewarm_cache_2.sql @@ -0,0 +1,54 @@ +-- Tags: no-parallel, no-random-merge-tree-settings + +DROP TABLE IF EXISTS t_prewarm_cache_rmt_1; + +CREATE TABLE t_prewarm_cache_rmt_1 (a UInt64, b UInt64, c UInt64) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03277_prewarms_caches/t_prewarm_cache', '1') +ORDER BY a +SETTINGS + index_granularity = 100, + min_bytes_for_wide_part = 0, + use_primary_key_cache = 1, + prewarm_primary_key_cache = 1, + prewarm_mark_cache = 1, + min_bytes_to_prewarm_caches = 30000; + +SYSTEM DROP MARK CACHE; +SYSTEM DROP PRIMARY INDEX CACHE; + +INSERT INTO t_prewarm_cache_rmt_1 SELECT number, rand(), rand() FROM numbers(100, 100); +INSERT INTO t_prewarm_cache_rmt_1 SELECT number, rand(), rand() FROM numbers(1000, 2000); + +SYSTEM RELOAD ASYNCHRONOUS METRICS; +SELECT metric, value FROM system.asynchronous_metrics WHERE metric IN ('PrimaryIndexCacheFiles', 'MarkCacheFiles') ORDER BY metric; + +SELECT count() FROM t_prewarm_cache_rmt_1 WHERE a % 2 = 0 AND a >= 100 AND a < 2000 AND NOT ignore(a, b); + +SYSTEM RELOAD ASYNCHRONOUS METRICS; +SELECT metric, value FROM system.asynchronous_metrics WHERE metric IN ('PrimaryIndexCacheFiles', 'MarkCacheFiles') ORDER BY metric; + +SYSTEM DROP MARK CACHE; +SYSTEM DROP PRIMARY INDEX CACHE; + +OPTIMIZE TABLE t_prewarm_cache_rmt_1 FINAL; + +SELECT count() FROM t_prewarm_cache_rmt_1 WHERE a % 2 = 0 AND a >= 100 AND a < 2000 AND NOT ignore(a, b); + +SYSTEM RELOAD ASYNCHRONOUS METRICS; +SELECT metric, value FROM system.asynchronous_metrics WHERE metric IN ('PrimaryIndexCacheFiles', 'MarkCacheFiles') ORDER BY metric; + +TRUNCATE TABLE t_prewarm_cache_rmt_1; + +SYSTEM RELOAD ASYNCHRONOUS METRICS; +SELECT metric, value FROM system.asynchronous_metrics WHERE metric IN ('PrimaryIndexCacheFiles', 'MarkCacheFiles') ORDER BY metric; + +SYSTEM FLUSH LOGS; + +SELECT + ProfileEvents['LoadedMarksFiles'], + ProfileEvents['LoadedPrimaryIndexFiles'] +FROM system.query_log +WHERE current_database = currentDatabase() AND type = 'QueryFinish' AND query LIKE 'SELECT count() FROM t_prewarm_cache_rmt_1%' +ORDER BY event_time_microseconds; + +DROP TABLE IF EXISTS t_prewarm_cache_rmt_1; From a81bacea75766fbf370ccdb9fcca52a497dacd79 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Pawe=C5=82=20Kudzia?= <30438339+pakud@users.noreply.github.com> Date: Sat, 30 Nov 2024 11:18:06 +0100 Subject: [PATCH 39/50] Adding example for JSONExtract which refers to a nested entry in JSON Hopefully this one prevents others from asking question like in https://github.com/ClickHouse/ClickHouse/issues/72530 --- docs/en/sql-reference/functions/json-functions.md | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/docs/en/sql-reference/functions/json-functions.md b/docs/en/sql-reference/functions/json-functions.md index 286dde68f45..287af5d5104 100644 --- a/docs/en/sql-reference/functions/json-functions.md +++ b/docs/en/sql-reference/functions/json-functions.md @@ -733,6 +733,18 @@ SELECT JSONExtract('{"day": "Thursday"}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday SELECT JSONExtract('{"day": 5}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday\' = 1, \'Tuesday\' = 2, \'Wednesday\' = 3, \'Thursday\' = 4, \'Friday\' = 5, \'Saturday\' = 6)') = 'Friday' ``` +Referring to a nested values by passing multiple indices_or_keys parameters: +``` +SELECT JSONExtract('{"a":{"b":"hello","c":{"d":[1,2,3],"e":[1,3,7]}}}','a','c','Map(String, Array(UInt8))') AS val, toTypeName(val), val['d']; +``` +Result: +``` +┌─val───────────────────────┬─toTypeName(val)───────────┬─arrayElement(val, 'd')─┐ +│ {'d':[1,2,3],'e':[1,3,7]} │ Map(String, Array(UInt8)) │ [1,2,3] │ +└───────────────────────────┴───────────────────────────┴────────────────────────┘ +``` + + ### JSONExtractKeysAndValues Parses key-value pairs from JSON where the values are of the given ClickHouse data type. From 6d0876b828daa6499a036ad9d5171e193c2c5011 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 30 Nov 2024 13:47:41 +0000 Subject: [PATCH 40/50] try to fix test --- .../0_stateless/03277_prewarm_cache_2.sh | 71 +++++++++++++++++++ .../0_stateless/03277_prewarm_cache_2.sql | 54 -------------- 2 files changed, 71 insertions(+), 54 deletions(-) create mode 100755 tests/queries/0_stateless/03277_prewarm_cache_2.sh delete mode 100644 tests/queries/0_stateless/03277_prewarm_cache_2.sql diff --git a/tests/queries/0_stateless/03277_prewarm_cache_2.sh b/tests/queries/0_stateless/03277_prewarm_cache_2.sh new file mode 100755 index 00000000000..d25cebbb6f3 --- /dev/null +++ b/tests/queries/0_stateless/03277_prewarm_cache_2.sh @@ -0,0 +1,71 @@ +#!/usr/bin/env bash +# Tags: no-parallel, no-random-merge-tree-settings + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh +$CLICKHOUSE_CLIENT --query " + + DROP TABLE IF EXISTS t_prewarm_cache_rmt_1; + + CREATE TABLE t_prewarm_cache_rmt_1 (a UInt64, b UInt64, c UInt64) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03277_prewarms_caches/t_prewarm_cache', '1') + ORDER BY a + SETTINGS + index_granularity = 100, + min_bytes_for_wide_part = 0, + use_primary_key_cache = 1, + prewarm_primary_key_cache = 1, + prewarm_mark_cache = 1, + min_bytes_to_prewarm_caches = 30000; + + SYSTEM DROP MARK CACHE; + SYSTEM DROP PRIMARY INDEX CACHE; + + INSERT INTO t_prewarm_cache_rmt_1 SELECT number, rand(), rand() FROM numbers(100, 100); + INSERT INTO t_prewarm_cache_rmt_1 SELECT number, rand(), rand() FROM numbers(1000, 2000); + + SYSTEM RELOAD ASYNCHRONOUS METRICS; + SELECT metric, value FROM system.asynchronous_metrics WHERE metric IN ('PrimaryIndexCacheFiles', 'MarkCacheFiles') ORDER BY metric; + + SELECT count() FROM t_prewarm_cache_rmt_1 WHERE a % 2 = 0 AND a >= 100 AND a < 2000 AND NOT ignore(a, b); + + SYSTEM RELOAD ASYNCHRONOUS METRICS; + SELECT metric, value FROM system.asynchronous_metrics WHERE metric IN ('PrimaryIndexCacheFiles', 'MarkCacheFiles') ORDER BY metric; + + SYSTEM DROP MARK CACHE; + SYSTEM DROP PRIMARY INDEX CACHE; + + OPTIMIZE TABLE t_prewarm_cache_rmt_1 FINAL; + + SELECT count() FROM t_prewarm_cache_rmt_1 WHERE a % 2 = 0 AND a >= 100 AND a < 2000 AND NOT ignore(a, b); + + SYSTEM RELOAD ASYNCHRONOUS METRICS; + SELECT metric, value FROM system.asynchronous_metrics WHERE metric IN ('PrimaryIndexCacheFiles', 'MarkCacheFiles') ORDER BY metric; + + TRUNCATE TABLE t_prewarm_cache_rmt_1; +" + +for _ in {1..100}; do + res=$($CLICKHOUSE_CLIENT -q "SELECT metric FROM system.asynchronous_metrics WHERE metric = 'PrimaryIndexCacheFiles'") + if [[ $res -eq 0 ]]; then + break + fi + sleep 0.3 +done + +$CLICKHOUSE_CLIENT --query " + SYSTEM RELOAD ASYNCHRONOUS METRICS; + SELECT metric, value FROM system.asynchronous_metrics WHERE metric IN ('PrimaryIndexCacheFiles', 'MarkCacheFiles') ORDER BY metric; + + SYSTEM FLUSH LOGS; + + SELECT + ProfileEvents['LoadedMarksFiles'], + ProfileEvents['LoadedPrimaryIndexFiles'] + FROM system.query_log + WHERE current_database = currentDatabase() AND type = 'QueryFinish' AND query LIKE 'SELECT count() FROM t_prewarm_cache_rmt_1%' + ORDER BY event_time_microseconds; + + DROP TABLE IF EXISTS t_prewarm_cache_rmt_1; +" diff --git a/tests/queries/0_stateless/03277_prewarm_cache_2.sql b/tests/queries/0_stateless/03277_prewarm_cache_2.sql deleted file mode 100644 index d8974b3cee0..00000000000 --- a/tests/queries/0_stateless/03277_prewarm_cache_2.sql +++ /dev/null @@ -1,54 +0,0 @@ --- Tags: no-parallel, no-random-merge-tree-settings - -DROP TABLE IF EXISTS t_prewarm_cache_rmt_1; - -CREATE TABLE t_prewarm_cache_rmt_1 (a UInt64, b UInt64, c UInt64) -ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03277_prewarms_caches/t_prewarm_cache', '1') -ORDER BY a -SETTINGS - index_granularity = 100, - min_bytes_for_wide_part = 0, - use_primary_key_cache = 1, - prewarm_primary_key_cache = 1, - prewarm_mark_cache = 1, - min_bytes_to_prewarm_caches = 30000; - -SYSTEM DROP MARK CACHE; -SYSTEM DROP PRIMARY INDEX CACHE; - -INSERT INTO t_prewarm_cache_rmt_1 SELECT number, rand(), rand() FROM numbers(100, 100); -INSERT INTO t_prewarm_cache_rmt_1 SELECT number, rand(), rand() FROM numbers(1000, 2000); - -SYSTEM RELOAD ASYNCHRONOUS METRICS; -SELECT metric, value FROM system.asynchronous_metrics WHERE metric IN ('PrimaryIndexCacheFiles', 'MarkCacheFiles') ORDER BY metric; - -SELECT count() FROM t_prewarm_cache_rmt_1 WHERE a % 2 = 0 AND a >= 100 AND a < 2000 AND NOT ignore(a, b); - -SYSTEM RELOAD ASYNCHRONOUS METRICS; -SELECT metric, value FROM system.asynchronous_metrics WHERE metric IN ('PrimaryIndexCacheFiles', 'MarkCacheFiles') ORDER BY metric; - -SYSTEM DROP MARK CACHE; -SYSTEM DROP PRIMARY INDEX CACHE; - -OPTIMIZE TABLE t_prewarm_cache_rmt_1 FINAL; - -SELECT count() FROM t_prewarm_cache_rmt_1 WHERE a % 2 = 0 AND a >= 100 AND a < 2000 AND NOT ignore(a, b); - -SYSTEM RELOAD ASYNCHRONOUS METRICS; -SELECT metric, value FROM system.asynchronous_metrics WHERE metric IN ('PrimaryIndexCacheFiles', 'MarkCacheFiles') ORDER BY metric; - -TRUNCATE TABLE t_prewarm_cache_rmt_1; - -SYSTEM RELOAD ASYNCHRONOUS METRICS; -SELECT metric, value FROM system.asynchronous_metrics WHERE metric IN ('PrimaryIndexCacheFiles', 'MarkCacheFiles') ORDER BY metric; - -SYSTEM FLUSH LOGS; - -SELECT - ProfileEvents['LoadedMarksFiles'], - ProfileEvents['LoadedPrimaryIndexFiles'] -FROM system.query_log -WHERE current_database = currentDatabase() AND type = 'QueryFinish' AND query LIKE 'SELECT count() FROM t_prewarm_cache_rmt_1%' -ORDER BY event_time_microseconds; - -DROP TABLE IF EXISTS t_prewarm_cache_rmt_1; From 0041fdf497b2c7780fe5bcb3eabe54b71ded40fa Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sun, 1 Dec 2024 18:50:55 +0100 Subject: [PATCH 41/50] impl --- src/Interpreters/executeQuery.cpp | 2 +- ...l_format_matching_case_insensitive.reference | 1 + ...276_null_format_matching_case_insensitive.sh | 17 +++++++++++++++++ 3 files changed, 19 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03276_null_format_matching_case_insensitive.reference create mode 100755 tests/queries/0_stateless/03276_null_format_matching_case_insensitive.sh diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index a94eca3d7b1..29280c91f1a 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1538,7 +1538,7 @@ std::pair executeQuery( ? getIdentifierName(ast_query_with_output->format) : context->getDefaultFormat(); - if (format_name == "Null") + if (boost::iequals(format_name, "Null")) res.null_format = true; } diff --git a/tests/queries/0_stateless/03276_null_format_matching_case_insensitive.reference b/tests/queries/0_stateless/03276_null_format_matching_case_insensitive.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/03276_null_format_matching_case_insensitive.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/03276_null_format_matching_case_insensitive.sh b/tests/queries/0_stateless/03276_null_format_matching_case_insensitive.sh new file mode 100755 index 00000000000..d1736d58832 --- /dev/null +++ b/tests/queries/0_stateless/03276_null_format_matching_case_insensitive.sh @@ -0,0 +1,17 @@ +#!/usr/bin/env bash + +set -e + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +query_id="03276_null_format_matching_case_insensitive_$RANDOM$RANDOM" +$CLICKHOUSE_CLIENT --query_id "$query_id" -q "select * from numbers_mt(1e8) format null" + +$CLICKHOUSE_CLIENT -q " + SYSTEM FLUSH LOGS; + + -- SendBytes should be close to 0, previously for this query it was around 800MB + select ProfileEvents['NetworkSendBytes'] < 1e6 from system.query_log where current_database = currentDatabase() and event_date >= yesterday() and query_id = '$query_id' and type = 'QueryFinish'; +" From 76dee30deebaad1e8e7bdb463d10aaa31ee59444 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 1 Dec 2024 17:54:05 +0000 Subject: [PATCH 42/50] make Field rvalue reference safeGet to return a copy of its content instead of reference --- src/Core/Field.h | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/src/Core/Field.h b/src/Core/Field.h index 5a6ee9cdf29..560002de4c6 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -479,12 +479,20 @@ public: return true; } - template auto & safeGet() const + template const auto & safeGet() const & { return const_cast(this)->safeGet(); } + template auto safeGet() const && + { + return safeGet(); + } - template auto & safeGet(); + template auto & safeGet() &; + template auto safeGet() && + { + return safeGet(); + } bool operator< (const Field & rhs) const { @@ -880,7 +888,7 @@ constexpr bool isInt64OrUInt64orBoolFieldType(Field::Types::Which t) } template -auto & Field::safeGet() +auto & Field::safeGet() & { const Types::Which target = TypeToEnum>>::value; From 88a3dbe716796e5575be5b7a3c579c8e8d32f479 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 28 Nov 2024 18:15:14 +0100 Subject: [PATCH 43/50] Fix inaccurate MemoryTracking metric in case of allocation failures MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit MemoryTracking metric takes into account memory allocation even in case of this allocation will fail with MEMORY_LIMIT_EXCEEDED, which is not good, which eventually will lead to `amount` and `MemoryTracking` mismatch, I found one server with **43x difference**.
```sql SELECT event_time_microseconds, message FROM system.text_log WHERE (event_date = (today() - 1)) AND (logger_name = 'MemoryTracker') AND (message LIKE '%total%') ORDER BY 1 DESC LIMIT 1 Query id: 64d60852-fa14-4ed1-adb1-d4bbd6159475 ┌────event_time_microseconds─┬─message───────────────────────────────────┐ 1. │ 2024-11-27 05:09:48.157608 │ Current memory usage (total): 471.00 GiB. │ └────────────────────────────┴───────────────────────────────────────────┘ ``` ```sql SELECT metric, formatReadableSize(value) FROM system.metrics WHERE (metric ILIKE '%mem%') OR (metric ILIKE '%jemalloc%') ORDER BY value ASC Query id: af7908a8-956a-4684-b7c5-b2e0c6fa06f4 ┌─metric────────────────────────┬─formatReadableSize(value)─┐ 1. │ MergesMutationsMemoryTracking │ 0.00 B │ 2. │ MemoryTracking │ 20.37 TiB │ └───────────────────────────────┴───────────────────────────┘ ```
Signed-off-by: Azat Khuzhin --- src/Common/MemoryTracker.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index f4af019605e..829d5dfcd79 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -256,10 +256,6 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed Int64 will_be = size ? size + amount.fetch_add(size, std::memory_order_relaxed) : amount.load(std::memory_order_relaxed); Int64 will_be_rss = size ? size + rss.fetch_add(size, std::memory_order_relaxed) : rss.load(std::memory_order_relaxed); - auto metric_loaded = metric.load(std::memory_order_relaxed); - if (metric_loaded != CurrentMetrics::end() && size) - CurrentMetrics::add(metric_loaded, size); - Int64 current_hard_limit = hard_limit.load(std::memory_order_relaxed); Int64 current_profiler_limit = profiler_limit.load(std::memory_order_relaxed); @@ -371,6 +367,10 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed } } + auto metric_loaded = metric.load(std::memory_order_relaxed); + if (metric_loaded != CurrentMetrics::end() && size) + CurrentMetrics::add(metric_loaded, size); + if (peak_updated && allocation_traced) { MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); From a2b18f2e2b8d3f214ab725c623822bd461c3fbea Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sun, 1 Dec 2024 23:31:51 +0100 Subject: [PATCH 44/50] fix test --- .../0_stateless/03276_null_format_matching_case_insensitive.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03276_null_format_matching_case_insensitive.sh b/tests/queries/0_stateless/03276_null_format_matching_case_insensitive.sh index d1736d58832..6f8b0946cff 100755 --- a/tests/queries/0_stateless/03276_null_format_matching_case_insensitive.sh +++ b/tests/queries/0_stateless/03276_null_format_matching_case_insensitive.sh @@ -7,7 +7,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh query_id="03276_null_format_matching_case_insensitive_$RANDOM$RANDOM" -$CLICKHOUSE_CLIENT --query_id "$query_id" -q "select * from numbers_mt(1e8) format null" +$CLICKHOUSE_CLIENT --query_id "$query_id" -q "select * from numbers_mt(1e8) format null settings max_rows_to_read=0" $CLICKHOUSE_CLIENT -q " SYSTEM FLUSH LOGS; From 54108d5c1616a5f7d55b6ae0165eb9f072cc1557 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 2 Dec 2024 00:08:00 +0000 Subject: [PATCH 45/50] fix tidy --- src/Interpreters/InterpreterSelectQuery.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 7e65e20724e..901abf8641f 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -406,8 +406,8 @@ ASTPtr parseAdditionalFilterConditionForTable( for (const auto & additional_filter : additional_table_filters) { const auto & tuple = additional_filter.safeGet(); - auto & table = tuple.at(0).safeGet(); - auto & filter = tuple.at(1).safeGet(); + const auto & table = tuple.at(0).safeGet(); + const auto & filter = tuple.at(1).safeGet(); if (table == target.alias || (table == target.table && context.getCurrentDatabase() == target.database) || From 499a39c614a4d749a3892f4cece1135ca6c14440 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 2 Dec 2024 10:09:44 +0100 Subject: [PATCH 46/50] Smoke test --- .../0_stateless/03277_logging_elapsed_ns.reference | 2 ++ tests/queries/0_stateless/03277_logging_elapsed_ns.sql | 9 +++++++++ 2 files changed, 11 insertions(+) create mode 100644 tests/queries/0_stateless/03277_logging_elapsed_ns.reference create mode 100644 tests/queries/0_stateless/03277_logging_elapsed_ns.sql diff --git a/tests/queries/0_stateless/03277_logging_elapsed_ns.reference b/tests/queries/0_stateless/03277_logging_elapsed_ns.reference new file mode 100644 index 00000000000..57569e954f0 --- /dev/null +++ b/tests/queries/0_stateless/03277_logging_elapsed_ns.reference @@ -0,0 +1,2 @@ +42 +1 1 diff --git a/tests/queries/0_stateless/03277_logging_elapsed_ns.sql b/tests/queries/0_stateless/03277_logging_elapsed_ns.sql new file mode 100644 index 00000000000..cff3455029c --- /dev/null +++ b/tests/queries/0_stateless/03277_logging_elapsed_ns.sql @@ -0,0 +1,9 @@ +SELECT 42 SETTINGS log_comment='03277_logging_elapsed_ns'; + +SYSTEM FLUSH LOGS; + +SELECT + ProfileEvents['LogDebug'] + ProfileEvents['LogTrace'] > 0, + ProfileEvents['LoggerElapsedNanoseconds'] > 0 +FROM system.query_log +WHERE current_database = currentDatabase() AND log_comment = '03277_logging_elapsed_ns' AND type = 'QueryFinish'; From 4aecc57555b145d4a4ce27e7d818d7122b87f105 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 2 Dec 2024 12:58:34 +0100 Subject: [PATCH 47/50] Increase cpu time limit in sanitizer builds --- CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/CMakeLists.txt b/CMakeLists.txt index a165be799c0..03c20325f1e 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -66,6 +66,7 @@ if (ENABLE_CHECK_HEAVY_BUILDS) # Twice as large set (RLIMIT_DATA 10000000000) set (RLIMIT_AS 20000000000) + set (RLIMIT_CPU 2000) endif() # For some files currently building RISCV64/LOONGARCH64 might be too slow. From d4f930536235d713ca5a5060337085b52cb08a78 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 2 Dec 2024 15:05:47 +0000 Subject: [PATCH 48/50] fix test --- tests/queries/0_stateless/03277_prewarm_cache_2.sh | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03277_prewarm_cache_2.sh b/tests/queries/0_stateless/03277_prewarm_cache_2.sh index d25cebbb6f3..d0af75f272a 100755 --- a/tests/queries/0_stateless/03277_prewarm_cache_2.sh +++ b/tests/queries/0_stateless/03277_prewarm_cache_2.sh @@ -17,6 +17,8 @@ $CLICKHOUSE_CLIENT --query " use_primary_key_cache = 1, prewarm_primary_key_cache = 1, prewarm_mark_cache = 1, + max_cleanup_delay_period = 1, + cleanup_delay_period = 1, min_bytes_to_prewarm_caches = 30000; SYSTEM DROP MARK CACHE; @@ -47,7 +49,10 @@ $CLICKHOUSE_CLIENT --query " " for _ in {1..100}; do - res=$($CLICKHOUSE_CLIENT -q "SELECT metric FROM system.asynchronous_metrics WHERE metric = 'PrimaryIndexCacheFiles'") + res=$($CLICKHOUSE_CLIENT -q " + SYSTEM RELOAD ASYNCHRONOUS METRICS; + SELECT value FROM system.asynchronous_metrics WHERE metric = 'PrimaryIndexCacheFiles'; + ") if [[ $res -eq 0 ]]; then break fi From 6291daec7202a41d17ef725cf3213182de9ebb6c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 2 Dec 2024 18:23:11 +0100 Subject: [PATCH 49/50] Remove heavy header from WriteHelpers.h --- .../FunctionsBinaryRepresentation.cpp | 1 + src/IO/WriteHelpers.cpp | 49 +++++++++++++++- src/IO/WriteHelpers.h | 58 ++----------------- 3 files changed, 51 insertions(+), 57 deletions(-) diff --git a/src/Functions/FunctionsBinaryRepresentation.cpp b/src/Functions/FunctionsBinaryRepresentation.cpp index 50a3c0862f4..db8e705d53b 100644 --- a/src/Functions/FunctionsBinaryRepresentation.cpp +++ b/src/Functions/FunctionsBinaryRepresentation.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include diff --git a/src/IO/WriteHelpers.cpp b/src/IO/WriteHelpers.cpp index 622f93719ac..0ed8e6959e8 100644 --- a/src/IO/WriteHelpers.cpp +++ b/src/IO/WriteHelpers.cpp @@ -1,13 +1,22 @@ #include -#include -#include -#include +#include #include +#include +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wunused-parameter" +#pragma clang diagnostic ignored "-Wsign-compare" +#include +#pragma clang diagnostic pop namespace DB { +namespace ErrorCodes +{ +extern const int CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER; +} + template void formatHex(IteratorSrc src, IteratorDst dst, size_t num_bytes) { @@ -127,4 +136,38 @@ String fourSpaceIndent(size_t indent) { return std::string(indent * 4, ' '); } + +template +requires is_floating_point +size_t writeFloatTextFastPath(T x, char * buffer) +{ + Int64 result = 0; + + if constexpr (std::is_same_v) + { + /// The library dragonbox has low performance on integers. + /// This workaround improves performance 6..10 times. + + if (DecomposedFloat64(x).isIntegerInRepresentableRange()) + result = itoa(Int64(x), buffer) - buffer; + else + result = jkj::dragonbox::to_chars_n(x, buffer) - buffer; + } + else if constexpr (std::is_same_v || std::is_same_v) + { + Float32 f32 = Float32(x); + if (DecomposedFloat32(f32).isIntegerInRepresentableRange()) + result = itoa(Int32(f32), buffer) - buffer; + else + result = jkj::dragonbox::to_chars_n(f32, buffer) - buffer; + } + + if (result <= 0) + throw Exception(ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, "Cannot print floating point number"); + return result; +} + +template size_t writeFloatTextFastPath(Float64 x, char * buffer); +template size_t writeFloatTextFastPath(Float32 x, char * buffer); +template size_t writeFloatTextFastPath(BFloat16 x, char * buffer); } diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index b186ae20ebf..a63f3631e38 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -4,8 +4,6 @@ #include #include #include -#include -#include #include #include @@ -18,19 +16,14 @@ #include #include #include -#include #include #include -#include #include #include -#include #include -#include -#include #include #include #include @@ -38,24 +31,11 @@ #include #include -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wunused-parameter" -#pragma clang diagnostic ignored "-Wsign-compare" -#include -#pragma clang diagnostic pop - #include - namespace DB { -namespace ErrorCodes -{ - extern const int CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER; -} - - /// Helper functions for formatted and binary output. inline void writeChar(char x, WriteBuffer & buf) @@ -151,41 +131,11 @@ inline void writeBoolText(bool x, WriteBuffer & buf) template requires is_floating_point -inline size_t writeFloatTextFastPath(T x, char * buffer) -{ - Int64 result = 0; +size_t writeFloatTextFastPath(T x, char * buffer); - if constexpr (std::is_same_v) - { - /// The library Ryu has low performance on integers. - /// This workaround improves performance 6..10 times. - - if (DecomposedFloat64(x).isIntegerInRepresentableRange()) - result = itoa(Int64(x), buffer) - buffer; - else - result = jkj::dragonbox::to_chars_n(x, buffer) - buffer; - } - else if constexpr (std::is_same_v) - { - if (DecomposedFloat32(x).isIntegerInRepresentableRange()) - result = itoa(Int32(x), buffer) - buffer; - else - result = jkj::dragonbox::to_chars_n(x, buffer) - buffer; - } - else if constexpr (std::is_same_v) - { - Float32 f32 = Float32(x); - - if (DecomposedFloat32(f32).isIntegerInRepresentableRange()) - result = itoa(Int32(f32), buffer) - buffer; - else - result = jkj::dragonbox::to_chars_n(f32, buffer) - buffer; - } - - if (result <= 0) - throw Exception(ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER, "Cannot print floating point number"); - return result; -} +extern template size_t writeFloatTextFastPath(Float64 x, char * buffer); +extern template size_t writeFloatTextFastPath(Float32 x, char * buffer); +extern template size_t writeFloatTextFastPath(BFloat16 x, char * buffer); template requires is_floating_point From 57e77cc8158383e5496cf0886b4cd59e90551b4c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 2 Dec 2024 21:46:57 +0000 Subject: [PATCH 50/50] add move semantics --- src/Core/Field.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/Field.h b/src/Core/Field.h index 560002de4c6..ebccd366089 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -485,13 +485,13 @@ public: } template auto safeGet() const && { - return safeGet(); + return std::move(const_cast(this)->safeGet()); } template auto & safeGet() &; template auto safeGet() && { - return safeGet(); + return std::move(safeGet()); } bool operator< (const Field & rhs) const