From 9efeddfdd12eb83736626af95de36b6ada913fc4 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 25 Jun 2024 21:11:56 +0000 Subject: [PATCH 01/59] 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/59] 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/59] 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/59] 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/59] 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 272eccc93c39558e0ec27266fb30f9bb465a1e06 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 24 Sep 2024 15:01:24 +0000 Subject: [PATCH 06/59] wip better handling of join on expression --- src/Analyzer/Resolve/QueryAnalyzer.cpp | 2 +- src/Analyzer/Utils.cpp | 28 ++++-- src/Analyzer/Utils.h | 3 +- src/Planner/PlannerJoinTree.cpp | 119 ++++++++++++++++++++----- src/Planner/PlannerJoins.cpp | 52 ++++++----- src/Planner/PlannerJoins.h | 10 +-- 6 files changed, 150 insertions(+), 64 deletions(-) diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 8d71abc0f02..a9a7fa9978a 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -1910,7 +1910,7 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher( * expressions that have same names as columns in USING clause must be skipped. */ - auto table_expressions_stack = buildTableExpressionsStack(nearest_query_scope_query_node->getJoinTree()); + auto table_expressions_stack = buildTableExpressionsStack(nearest_query_scope->scope_node); std::vector table_expressions_column_nodes_with_names_stack; std::unordered_set table_expression_column_names_to_skip; diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index c737822282c..be3f126b5fd 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -552,35 +552,40 @@ QueryTreeNodePtr extractLeftTableExpression(const QueryTreeNodePtr & join_tree_n namespace { -void buildTableExpressionsStackImpl(const QueryTreeNodePtr & join_tree_node, QueryTreeNodes & result) +void buildTableExpressionsStackImpl(const QueryTreeNodePtr & join_tree_node, QueryTreeNodePtr nearest_query_node, QueryTreeNodes & result, QueryTreeNodes & nearest_query) { auto node_type = join_tree_node->getNodeType(); - switch (node_type) { - case QueryTreeNodeType::TABLE: - [[fallthrough]]; case QueryTreeNodeType::QUERY: + { + nearest_query_node = join_tree_node; + [[fallthrough]]; + } + case QueryTreeNodeType::TABLE: [[fallthrough]]; case QueryTreeNodeType::UNION: [[fallthrough]]; case QueryTreeNodeType::TABLE_FUNCTION: { + nearest_query.push_back(nearest_query_node); result.push_back(join_tree_node); break; } case QueryTreeNodeType::ARRAY_JOIN: { auto & array_join_node = join_tree_node->as(); - buildTableExpressionsStackImpl(array_join_node.getTableExpression(), result); + buildTableExpressionsStackImpl(array_join_node.getTableExpression(), nearest_query_node, result, nearest_query); + nearest_query.push_back(nearest_query_node); result.push_back(join_tree_node); break; } case QueryTreeNodeType::JOIN: { auto & join_node = join_tree_node->as(); - buildTableExpressionsStackImpl(join_node.getLeftTableExpression(), result); - buildTableExpressionsStackImpl(join_node.getRightTableExpression(), result); + buildTableExpressionsStackImpl(join_node.getLeftTableExpression(), nearest_query_node, result, nearest_query); + buildTableExpressionsStackImpl(join_node.getRightTableExpression(), nearest_query_node, result, nearest_query); + nearest_query.push_back(nearest_query_node); result.push_back(join_tree_node); break; } @@ -595,10 +600,15 @@ void buildTableExpressionsStackImpl(const QueryTreeNodePtr & join_tree_node, Que } -QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & join_tree_node) +QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & query_node, QueryTreeNodes * out_nearest_query) { QueryTreeNodes result; - buildTableExpressionsStackImpl(join_tree_node, result); + + QueryTreeNodes nearest_query_tmp; + if (out_nearest_query == nullptr) + out_nearest_query = &nearest_query_tmp; + + buildTableExpressionsStackImpl(query_node->as()->getJoinTree(), query_node, result, *out_nearest_query); return result; } diff --git a/src/Analyzer/Utils.h b/src/Analyzer/Utils.h index f2e2c500384..340ae9967ce 100644 --- a/src/Analyzer/Utils.h +++ b/src/Analyzer/Utils.h @@ -76,7 +76,8 @@ QueryTreeNodePtr extractLeftTableExpression(const QueryTreeNodePtr & join_tree_n * 4. t2 * 5. t1 */ -QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & join_tree_node); +QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & query_node, QueryTreeNodes * out_nearest_query = nullptr); + /** Assert that there are no function nodes with specified function name in node children. * Do not visit subqueries. diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 8691f454ef8..0f8f8d4613d 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -70,6 +70,9 @@ #include #include +#include +#include + namespace DB { namespace Setting @@ -1241,6 +1244,7 @@ void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextP } JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_expression, + const QueryTreeNodePtr & nearest_query_node, JoinTreeQueryPlan left_join_tree_query_plan, JoinTreeQueryPlan right_join_tree_query_plan, const ColumnIdentifierSet & outer_scope_columns, @@ -1273,23 +1277,52 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ if (join_strictness == JoinStrictness::All || join_strictness == JoinStrictness::Semi || join_strictness == JoinStrictness::Anti) join_constant = tryExtractConstantFromJoinNode(join_table_expression); + if (!join_constant && join_node.isOnJoinExpression()) { + UNUSED(nearest_query_node); join_clauses_and_actions = buildJoinClausesAndActions(left_plan_output_columns, right_plan_output_columns, join_table_expression, + // nearest_query_node, planner_context); + + bool can_move_out_residuals = join_clauses_and_actions.join_clauses.size() == 1 + && (join_strictness == JoinStrictness::Any || join_strictness == JoinStrictness::All) + && (join_kind == JoinKind::Inner || join_kind == JoinKind::Cross || join_kind == JoinKind::Comma) + && join_node.isOnJoinExpression(); + + const auto & left_pre_filters = join_clauses_and_actions.join_clauses[0].getLeftFilterConditionNodes(); + if (!left_pre_filters.empty() && left_pre_filters.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected only one left pre-filter condition node. Actual [{}]", fmt::join(left_pre_filters | std::views::transform([](const auto & node) { return node->result_name; }), ", ")); + join_clauses_and_actions.left_join_expressions_actions.appendInputsForUnusedColumns(left_plan.getCurrentDataStream().header); - auto left_join_expressions_actions_step = std::make_unique(left_plan.getCurrentDataStream(), std::move(join_clauses_and_actions.left_join_expressions_actions)); + appendSetsFromActionsDAG(join_clauses_and_actions.left_join_expressions_actions, left_join_tree_query_plan.useful_sets); + + QueryPlanStepPtr left_join_expressions_actions_step; + if (can_move_out_residuals && !left_pre_filters.empty()) + left_join_expressions_actions_step = std::make_unique(left_plan.getCurrentDataStream(), std::move(join_clauses_and_actions.left_join_expressions_actions), left_pre_filters[0]->result_name, false); + else + left_join_expressions_actions_step = std::make_unique(left_plan.getCurrentDataStream(), std::move(join_clauses_and_actions.left_join_expressions_actions)); + left_join_expressions_actions_step->setStepDescription("JOIN actions"); - appendSetsFromActionsDAG(left_join_expressions_actions_step->getExpression(), left_join_tree_query_plan.useful_sets); left_plan.addStep(std::move(left_join_expressions_actions_step)); + + const auto & right_pre_filters = join_clauses_and_actions.join_clauses[0].getRightFilterConditionNodes(); + if (!right_pre_filters.empty() && right_pre_filters.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected only one right pre-filter condition node. Actual [{}]", fmt::join(right_pre_filters | std::views::transform([](const auto & node) { return node->result_name; }), ", ")); + join_clauses_and_actions.right_join_expressions_actions.appendInputsForUnusedColumns(right_plan.getCurrentDataStream().header); - auto right_join_expressions_actions_step = std::make_unique(right_plan.getCurrentDataStream(), std::move(join_clauses_and_actions.right_join_expressions_actions)); + appendSetsFromActionsDAG(join_clauses_and_actions.right_join_expressions_actions, right_join_tree_query_plan.useful_sets); + + QueryPlanStepPtr right_join_expressions_actions_step; + if (can_move_out_residuals && !right_pre_filters.empty()) + right_join_expressions_actions_step = std::make_unique(right_plan.getCurrentDataStream(), std::move(join_clauses_and_actions.right_join_expressions_actions), right_pre_filters[0]->result_name, false); + else + right_join_expressions_actions_step = std::make_unique(right_plan.getCurrentDataStream(), std::move(join_clauses_and_actions.right_join_expressions_actions)); right_join_expressions_actions_step->setStepDescription("JOIN actions"); - appendSetsFromActionsDAG(right_join_expressions_actions_step->getExpression(), right_join_tree_query_plan.useful_sets); right_plan.addStep(std::move(right_join_expressions_actions_step)); } @@ -1402,20 +1435,23 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ const auto & join_clauses = join_clauses_and_actions.join_clauses; bool is_asof = table_join->strictness() == JoinStrictness::Asof; - if (join_clauses.size() > 1) + if (join_clauses.size() != 1 && is_asof) { - if (is_asof) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "ASOF join {} doesn't support multiple ORs for keys in JOIN ON section", - join_node.formatASTForErrorMessage()); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "ASOF join doesn't support JOIN ON expression {}", + join_node.formatASTForErrorMessage()); } + bool can_move_out_residuals = join_clauses.size() == 1 + && (join_strictness == JoinStrictness::Any || join_strictness == JoinStrictness::All) + && (join_kind == JoinKind::Inner || join_kind == JoinKind::Cross || join_kind == JoinKind::Comma) + && join_node.isOnJoinExpression(); + auto & table_join_clauses = table_join->getClauses(); for (const auto & join_clause : join_clauses) { table_join_clauses.emplace_back(); - auto & table_join_clause = table_join_clauses.back(); const auto & join_clause_left_key_nodes = join_clause.getLeftKeyNodes(); const auto & join_clause_right_key_nodes = join_clause.getRightKeyNodes(); @@ -1423,6 +1459,19 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ size_t join_clause_key_nodes_size = join_clause_left_key_nodes.size(); chassert(join_clause_key_nodes_size == join_clause_right_key_nodes.size()); + if (join_clause_key_nodes_size == 0 && !can_move_out_residuals) + throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, "Cannot determine join keys in {}", join_node.formatASTForErrorMessage()); + + if (join_clause_key_nodes_size == 0 && can_move_out_residuals) + { + join_kind = JoinKind::Cross; + table_join->getTableJoin().kind = join_kind; + table_join->setIsJoinWithConstant(true); + table_join_clauses.pop_back(); + continue; + } + + auto & table_join_clause = table_join_clauses.back(); for (size_t i = 0; i < join_clause_key_nodes_size; ++i) { table_join_clause.addKey(join_clause_left_key_nodes[i]->result_name, @@ -1439,8 +1488,11 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ join_node.formatASTForErrorMessage(), join_clause_get_left_filter_condition_nodes.size()); - const auto & join_clause_left_filter_condition_name = join_clause_get_left_filter_condition_nodes[0]->result_name; - table_join_clause.analyzer_left_filter_condition_column_name = join_clause_left_filter_condition_name; + if (!can_move_out_residuals) + { + const auto & join_clause_left_filter_condition_name = join_clause_get_left_filter_condition_nodes[0]->result_name; + table_join_clause.analyzer_left_filter_condition_column_name = join_clause_left_filter_condition_name; + } } const auto & join_clause_get_right_filter_condition_nodes = join_clause.getRightFilterConditionNodes(); @@ -1452,8 +1504,11 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ join_node.formatASTForErrorMessage(), join_clause_get_right_filter_condition_nodes.size()); - const auto & join_clause_right_filter_condition_name = join_clause_get_right_filter_condition_nodes[0]->result_name; - table_join_clause.analyzer_right_filter_condition_column_name = join_clause_right_filter_condition_name; + if (!can_move_out_residuals) + { + const auto & join_clause_right_filter_condition_name = join_clause_get_right_filter_condition_nodes[0]->result_name; + table_join_clause.analyzer_right_filter_condition_column_name = join_clause_right_filter_condition_name; + } } if (is_asof) @@ -1478,7 +1533,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ } } - if (join_clauses_and_actions.mixed_join_expressions_actions) + if (!can_move_out_residuals && join_clauses_and_actions.mixed_join_expressions_actions) { ExpressionActionsPtr & mixed_join_expression = table_join->getMixedJoinExpression(); mixed_join_expression = std::make_shared( @@ -1486,6 +1541,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ ExpressionActionsSettings::fromContext(planner_context->getQueryContext())); appendSetsFromActionsDAG(mixed_join_expression->getActionsDAG(), left_join_tree_query_plan.useful_sets); + join_clauses_and_actions.mixed_join_expressions_actions.reset(); } } else if (join_node.isUsingJoinExpression()) @@ -1634,6 +1690,26 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ result_plan.unitePlans(std::move(join_step), {std::move(plans)}); } + if (join_clauses_and_actions.mixed_join_expressions_actions) + { + auto outputs = join_clauses_and_actions.mixed_join_expressions_actions->getOutputs(); + if (outputs.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 1 output column in JOIN actions, got {}", + join_clauses_and_actions.mixed_join_expressions_actions->dumpDAG()); + + join_clauses_and_actions.mixed_join_expressions_actions->appendInputsForUnusedColumns(result_plan.getCurrentDataStream().header); + for (const auto * input_node : join_clauses_and_actions.mixed_join_expressions_actions->getInputs()) + join_clauses_and_actions.mixed_join_expressions_actions->addOrReplaceInOutputs(*input_node); + + auto filter_step = std::make_unique(result_plan.getCurrentDataStream(), + std::move(*join_clauses_and_actions.mixed_join_expressions_actions), + outputs[0]->result_name, + /* remove_column = */ false); /// Unused columns will be removed by next step + filter_step->setStepDescription("Residual JOIN filter"); + result_plan.addStep(std::move(filter_step)); + join_clauses_and_actions.mixed_join_expressions_actions.reset(); + } + ActionsDAG drop_unused_columns_after_join_actions_dag(result_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); ActionsDAG::NodeRawConstPtrs drop_unused_columns_after_join_actions_dag_updated_outputs; std::unordered_set drop_unused_columns_after_join_actions_dag_updated_outputs_names; @@ -1796,7 +1872,8 @@ JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node, const ColumnIdentifierSet & outer_scope_columns, PlannerContextPtr & planner_context) { - auto table_expressions_stack = buildTableExpressionsStack(query_node->as().getJoinTree()); + QueryTreeNodes table_expressions_stack_queries; + auto table_expressions_stack = buildTableExpressionsStack(query_node, &table_expressions_stack_queries); size_t table_expressions_stack_size = table_expressions_stack.size(); bool is_single_table_expression = table_expressions_stack_size == 1; @@ -1857,6 +1934,7 @@ JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node, for (size_t i = 0; i < table_expressions_stack_size; ++i) { const auto & table_expression = table_expressions_stack[i]; + const auto & nearest_query = table_expressions_stack_queries[i]; auto table_expression_node_type = table_expression->getNodeType(); if (table_expression_node_type == QueryTreeNodeType::ARRAY_JOIN) @@ -1885,11 +1963,10 @@ JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node, auto left_query_plan = std::move(query_plans_stack.back()); query_plans_stack.pop_back(); - query_plans_stack.push_back(buildQueryPlanForJoinNode(table_expression, - std::move(left_query_plan), - std::move(right_query_plan), - table_expressions_outer_scope_columns[i], - planner_context)); + query_plans_stack.push_back(buildQueryPlanForJoinNode( + table_expression, nearest_query, + std::move(left_query_plan), std::move(right_query_plan), + table_expressions_outer_scope_columns[i], planner_context)); } else { diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 8857fda5b4c..274da438bd4 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -204,7 +204,7 @@ const ActionsDAG::Node * appendExpression( void buildJoinClause( ActionsDAG & left_dag, ActionsDAG & right_dag, - ActionsDAG & mixed_dag, + ActionsDAG & joined_dag, const PlannerContextPtr & planner_context, const QueryTreeNodePtr & join_expression, const TableExpressionSet & left_table_expressions, @@ -225,7 +225,7 @@ void buildJoinClause( buildJoinClause( left_dag, right_dag, - mixed_dag, + joined_dag, planner_context, child, left_table_expressions, @@ -330,8 +330,8 @@ void buildJoinClause( { /// expression involves both tables. /// `expr1(left.col1, right.col2) == expr2(left.col3, right.col4)` - const auto * node = appendExpression(mixed_dag, join_expression, planner_context, join_node); - join_clause.addMixedCondition(node); + const auto * node = appendExpression(joined_dag, join_expression, planner_context, join_node); + join_clause.addResidualCondition(node); } else { @@ -359,21 +359,23 @@ void buildJoinClause( } else { - auto support_mixed_join_condition = planner_context->getQueryContext()->getSettingsRef()[Setting::allow_experimental_join_condition]; auto join_use_nulls = planner_context->getQueryContext()->getSettingsRef()[Setting::join_use_nulls]; - /// If join_use_nulls = true, the columns' nullability will be changed later which make this expression not right. - if (support_mixed_join_condition && !join_use_nulls) + /// If join_use_nulls = true, the columns' nullability will be changed later which make this expression not applicable. + auto strictness = join_node.getStrictness(); + auto kind = join_node.getKind(); + bool can_be_moved_out = (strictness == JoinStrictness::Any || strictness == JoinStrictness::All) + && (kind == JoinKind::Inner || kind == JoinKind::Cross || kind == JoinKind::Comma); + if (can_be_moved_out || !join_use_nulls) { - /// expression involves both tables. - const auto * node = appendExpression(mixed_dag, join_expression, planner_context, join_node); - join_clause.addMixedCondition(node); + const auto * node = appendExpression(joined_dag, join_expression, planner_context, join_node); + join_clause.addResidualCondition(node); } else { throw Exception( ErrorCodes::INVALID_JOIN_ON_EXPRESSION, - "JOIN {} join expression contains column from left and right table, you may try experimental support of this feature by `SET allow_experimental_join_condition = 1`", - join_node.formatASTForErrorMessage()); + "OUTER JOIN ON expression {} contains column from left and right table, which is not supported with join_use_nulls", + join_expression->formatASTForErrorMessage()); } } } @@ -387,16 +389,16 @@ JoinClausesAndActions buildJoinClausesAndActions( { ActionsDAG left_join_actions(left_table_expression_columns); ActionsDAG right_join_actions(right_table_expression_columns); - ColumnsWithTypeAndName mixed_table_expression_columns; + ColumnsWithTypeAndName result_relation_columns; for (const auto & left_column : left_table_expression_columns) { - mixed_table_expression_columns.push_back(left_column); + result_relation_columns.push_back(left_column); } for (const auto & right_column : right_table_expression_columns) { - mixed_table_expression_columns.push_back(right_column); + result_relation_columns.push_back(right_column); } - ActionsDAG mixed_join_actions(mixed_table_expression_columns); + ActionsDAG post_join_actions(result_relation_columns); /** It is possible to have constant value in JOIN ON section, that we need to ignore during DAG construction. * If we do not ignore it, this function will be replaced by underlying constant. @@ -450,7 +452,7 @@ JoinClausesAndActions buildJoinClausesAndActions( JoinClausesAndActions result; - bool is_inequal_join = false; + bool has_residual_filters = false; const auto & function_name = function_node->getFunction()->getName(); if (function_name == "or") { @@ -461,14 +463,14 @@ JoinClausesAndActions buildJoinClausesAndActions( buildJoinClause( left_join_actions, right_join_actions, - mixed_join_actions, + post_join_actions, planner_context, child, join_left_table_expressions, join_right_table_expressions, join_node, result.join_clauses.back()); - is_inequal_join |= !result.join_clauses.back().getMixedFilterConditionNodes().empty(); + has_residual_filters |= !result.join_clauses.back().getMixedFilterConditionNodes().empty(); } } else @@ -478,14 +480,14 @@ JoinClausesAndActions buildJoinClausesAndActions( buildJoinClause( left_join_actions, right_join_actions, - mixed_join_actions, + post_join_actions, planner_context, join_expression, join_left_table_expressions, join_right_table_expressions, join_node, result.join_clauses.back()); - is_inequal_join |= !result.join_clauses.back().getMixedFilterConditionNodes().empty(); + has_residual_filters |= !result.join_clauses.back().getMixedFilterConditionNodes().empty(); } auto and_function = FunctionFactory::instance().get("and", planner_context->getQueryContext()); @@ -542,10 +544,6 @@ JoinClausesAndActions buildJoinClausesAndActions( assert(join_clause.getLeftKeyNodes().size() == join_clause.getRightKeyNodes().size()); size_t join_clause_key_nodes_size = join_clause.getLeftKeyNodes().size(); - if (join_clause_key_nodes_size == 0) - throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, "Cannot determine join keys in {}", - join_node.formatASTForErrorMessage()); - for (size_t i = 0; i < join_clause_key_nodes_size; ++i) { auto & left_key_node = join_clause.getLeftKeyNodes()[i]; @@ -610,13 +608,13 @@ JoinClausesAndActions buildJoinClausesAndActions( result.right_join_tmp_expression_actions = std::move(right_join_actions); result.right_join_expressions_actions.removeUnusedActions(join_right_actions_names); - if (is_inequal_join) + if (has_residual_filters) { /// In case of multiple disjuncts and any inequal join condition, we need to build full join on expression actions. /// So, for each column, we recalculate the value of the whole expression from JOIN ON to check if rows should be joined. if (result.join_clauses.size() > 1) { - ActionsDAG mixed_join_expressions_actions(mixed_table_expression_columns); + ActionsDAG mixed_join_expressions_actions(result_relation_columns); PlannerActionsVisitor join_expression_visitor(planner_context); auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(mixed_join_expressions_actions, join_expression); if (join_expression_dag_node_raw_pointers.size() != 1) diff --git a/src/Planner/PlannerJoins.h b/src/Planner/PlannerJoins.h index d8665ab7739..a95233674e7 100644 --- a/src/Planner/PlannerJoins.h +++ b/src/Planner/PlannerJoins.h @@ -142,17 +142,17 @@ public: ActionsDAG::NodeRawConstPtrs & getMixedFilterConditionNodes() { - return mixed_filter_condition_nodes; + return residual_filter_condition_nodes; } - void addMixedCondition(const ActionsDAG::Node * condition_node) + void addResidualCondition(const ActionsDAG::Node * condition_node) { - mixed_filter_condition_nodes.push_back(condition_node); + residual_filter_condition_nodes.push_back(condition_node); } const ActionsDAG::NodeRawConstPtrs & getMixedFilterConditionNodes() const { - return mixed_filter_condition_nodes; + return residual_filter_condition_nodes; } /// Dump clause into buffer @@ -170,7 +170,7 @@ private: ActionsDAG::NodeRawConstPtrs left_filter_condition_nodes; ActionsDAG::NodeRawConstPtrs right_filter_condition_nodes; /// conditions which involve both left and right tables - ActionsDAG::NodeRawConstPtrs mixed_filter_condition_nodes; + ActionsDAG::NodeRawConstPtrs residual_filter_condition_nodes; std::unordered_set nullsafe_compare_key_indexes; }; From e7a247e91e00066853fc5262126725631872b663 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 26 Sep 2024 15:00:30 +0000 Subject: [PATCH 07/59] wip --- src/Planner/PlannerJoinTree.cpp | 48 ++++++++++++------- src/Processors/QueryPlan/FilterStep.cpp | 7 +++ src/Processors/QueryPlan/FilterStep.h | 2 + src/Processors/Transforms/FilterTransform.cpp | 7 ++- src/Processors/Transforms/FilterTransform.h | 2 + 5 files changed, 49 insertions(+), 17 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 0f8f8d4613d..f97325b303a 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1278,6 +1278,8 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ join_constant = tryExtractConstantFromJoinNode(join_table_expression); + bool can_move_out_residuals = false; + if (!join_constant && join_node.isOnJoinExpression()) { UNUSED(nearest_query_node); @@ -1287,16 +1289,40 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ // nearest_query_node, planner_context); - - bool can_move_out_residuals = join_clauses_and_actions.join_clauses.size() == 1 - && (join_strictness == JoinStrictness::Any || join_strictness == JoinStrictness::All) - && (join_kind == JoinKind::Inner || join_kind == JoinKind::Cross || join_kind == JoinKind::Comma) - && join_node.isOnJoinExpression(); - const auto & left_pre_filters = join_clauses_and_actions.join_clauses[0].getLeftFilterConditionNodes(); if (!left_pre_filters.empty() && left_pre_filters.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected only one left pre-filter condition node. Actual [{}]", fmt::join(left_pre_filters | std::views::transform([](const auto & node) { return node->result_name; }), ", ")); + const auto & right_pre_filters = join_clauses_and_actions.join_clauses[0].getRightFilterConditionNodes(); + if (!right_pre_filters.empty() && right_pre_filters.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected only one right pre-filter condition node. Actual [{}]", fmt::join(right_pre_filters | std::views::transform([](const auto & node) { return node->result_name; }), ", ")); + + can_move_out_residuals = join_clauses_and_actions.join_clauses.size() == 1 + && (join_strictness == JoinStrictness::Any || join_strictness == JoinStrictness::All) + && (join_kind == JoinKind::Inner || join_kind == JoinKind::Cross || join_kind == JoinKind::Comma) + && (right_pre_filters.empty() || FilterStep::canUseType(right_pre_filters[0]->result_type)) + && (left_pre_filters.empty() || FilterStep::canUseType(left_pre_filters[0]->result_type)); + if (can_move_out_residuals && join_kind != JoinKind::Cross && join_kind != JoinKind::Comma) + { + /* In case we have only conditions like x.column1 = x.column2, do not move them to WHERE. + * ClickHouse has pretty complicated logic to resolve qualified column names, so a user may write something like: + * SELECT dummy FROM one l JOIN one r ON l.dummy = dummy; + * (tests/queries/0_stateless/01018_ambiguous_column.sql) + * That case is definitely not a cross join with condition `dummy = dummy`, + * and we should keep the behavior of throwing an exception. + */ + const auto & join_clause = join_clauses_and_actions.join_clauses[0]; + bool has_only_single_table_conditions = join_clause.getMixedFilterConditionNodes().size() == 0 + && join_clause.getLeftKeyNodes().size() == 0 + && join_clause.getRightKeyNodes().size() == 0; + + bool has_only_single_table_equals = has_only_single_table_conditions + && (right_pre_filters.empty() || (right_pre_filters[0]->function_base && right_pre_filters[0]->function_base->getName() != "equals")) + && (left_pre_filters.empty() || (left_pre_filters[0]->function_base && left_pre_filters[0]->function_base->getName() != "equals")); + + can_move_out_residuals = has_only_single_table_equals; + } + join_clauses_and_actions.left_join_expressions_actions.appendInputsForUnusedColumns(left_plan.getCurrentDataStream().header); appendSetsFromActionsDAG(join_clauses_and_actions.left_join_expressions_actions, left_join_tree_query_plan.useful_sets); @@ -1309,11 +1335,6 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ left_join_expressions_actions_step->setStepDescription("JOIN actions"); left_plan.addStep(std::move(left_join_expressions_actions_step)); - - const auto & right_pre_filters = join_clauses_and_actions.join_clauses[0].getRightFilterConditionNodes(); - if (!right_pre_filters.empty() && right_pre_filters.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected only one right pre-filter condition node. Actual [{}]", fmt::join(right_pre_filters | std::views::transform([](const auto & node) { return node->result_name; }), ", ")); - join_clauses_and_actions.right_join_expressions_actions.appendInputsForUnusedColumns(right_plan.getCurrentDataStream().header); appendSetsFromActionsDAG(join_clauses_and_actions.right_join_expressions_actions, right_join_tree_query_plan.useful_sets); @@ -1442,11 +1463,6 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ join_node.formatASTForErrorMessage()); } - bool can_move_out_residuals = join_clauses.size() == 1 - && (join_strictness == JoinStrictness::Any || join_strictness == JoinStrictness::All) - && (join_kind == JoinKind::Inner || join_kind == JoinKind::Cross || join_kind == JoinKind::Comma) - && join_node.isOnJoinExpression(); - auto & table_join_clauses = table_join->getClauses(); for (const auto & join_clause : join_clauses) diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index 04c218cb096..30dde1b9da0 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -106,4 +106,11 @@ void FilterStep::updateOutputStream() return; } + +bool FilterStep::canUseType(const DataTypePtr & filter_type) +{ + return FilterTransform::canUseType(filter_type); +} + + } diff --git a/src/Processors/QueryPlan/FilterStep.h b/src/Processors/QueryPlan/FilterStep.h index b5a31bef5fc..5607cc461d1 100644 --- a/src/Processors/QueryPlan/FilterStep.h +++ b/src/Processors/QueryPlan/FilterStep.h @@ -26,6 +26,8 @@ public: const String & getFilterColumnName() const { return filter_column_name; } bool removesFilterColumn() const { return remove_filter_column; } + static bool canUseType(const DataTypePtr & type); + private: void updateOutputStream() override; diff --git a/src/Processors/Transforms/FilterTransform.cpp b/src/Processors/Transforms/FilterTransform.cpp index cd87019a8e0..dd9daec7e95 100644 --- a/src/Processors/Transforms/FilterTransform.cpp +++ b/src/Processors/Transforms/FilterTransform.cpp @@ -36,13 +36,18 @@ static void replaceFilterToConstant(Block & block, const String & filter_column_ } } +bool FilterTransform::canUseType(const DataTypePtr & filter_type) +{ + return filter_type->onlyNull() || isUInt8(removeNullable(removeLowCardinality(filter_type))); +} + Block FilterTransform::transformHeader( const Block & header, const ActionsDAG * expression, const String & filter_column_name, bool remove_filter_column) { Block result = expression ? expression->updateHeader(header) : header; auto filter_type = result.getByName(filter_column_name).type; - if (!filter_type->onlyNull() && !isUInt8(removeNullable(removeLowCardinality(filter_type)))) + if (!canUseType(filter_type)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, "Illegal type {} of column {} for filter. Must be UInt8 or Nullable(UInt8).", filter_type->getName(), filter_column_name); diff --git a/src/Processors/Transforms/FilterTransform.h b/src/Processors/Transforms/FilterTransform.h index 23c694eed0b..9fc0dcaf73d 100644 --- a/src/Processors/Transforms/FilterTransform.h +++ b/src/Processors/Transforms/FilterTransform.h @@ -31,6 +31,8 @@ public: void transform(Chunk & chunk) override; + static bool canUseType(const DataTypePtr & type); + private: ExpressionActionsPtr expression; String filter_column_name; From 6434da72499f612a48aa2b7c80dea1d5fafa3ca2 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 27 Sep 2024 16:49:53 +0000 Subject: [PATCH 08/59] up tests --- .../0_stateless/01429_join_on_error_messages.sql | 10 ---------- .../01881_join_on_conditions_hash.sql.j2 | 16 ++++++++-------- .../02834_array_exists_segfault.reference | 2 ++ .../0_stateless/02834_array_exists_segfault.sql | 4 +++- 4 files changed, 13 insertions(+), 19 deletions(-) diff --git a/tests/queries/0_stateless/01429_join_on_error_messages.sql b/tests/queries/0_stateless/01429_join_on_error_messages.sql index c123bdd6b38..9e8dec63c58 100644 --- a/tests/queries/0_stateless/01429_join_on_error_messages.sql +++ b/tests/queries/0_stateless/01429_join_on_error_messages.sql @@ -4,11 +4,6 @@ SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON (A.a = arrayJoin([1])); -- { SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON equals(a); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH, 62 } SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON less(a); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH, 62 } -SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b AND a > b; -- { serverError INVALID_JOIN_ON_EXPRESSION } -SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b AND a < b; -- { serverError INVALID_JOIN_ON_EXPRESSION } -SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b AND a >= b; -- { serverError INVALID_JOIN_ON_EXPRESSION } -SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b AND a <= b; -- { serverError INVALID_JOIN_ON_EXPRESSION } - SET join_algorithm = 'partial_merge'; SELECT 1 FROM (select 1 a) A JOIN (select 1 b, 1 c) B ON a = b OR a = c; -- { serverError NOT_IMPLEMENTED } -- works for a = b OR a = b because of equivalent disjunct optimization @@ -16,8 +11,3 @@ SELECT 1 FROM (select 1 a) A JOIN (select 1 b, 1 c) B ON a = b OR a = c; -- { se SET join_algorithm = 'grace_hash'; SELECT 1 FROM (select 1 a) A JOIN (select 1 b, 1 c) B ON a = b OR a = c; -- { serverError NOT_IMPLEMENTED } -- works for a = b OR a = b because of equivalent disjunct optimization - -SET join_algorithm = 'hash'; - --- conditions for different table joined via OR -SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); -- { serverError INVALID_JOIN_ON_EXPRESSION } diff --git a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 index c2d85cefb18..ecca94eacea 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 @@ -74,21 +74,21 @@ SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.key; -- { serverErro SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverError 43, 403 } SELECT * FROM t1 JOIN t2_nullable as t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 } -SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError 403 } +SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id FORMAT Null; SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id; -- non-equi condition containing columns from different tables doesn't supported yet -SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id; -- { serverError 403 } -SELECT * FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t2.key); -- { serverError 403 } +SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id FORMAT Null; +SELECT * FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t2.key) FORMAT Null; SELECT '--'; -- length(t1.key2) == length(t2.key2) is expression for columns from both tables, it works because it part of joining key SELECT t1.*, t2.* FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND length(t1.key2) == length(t2.key2) AND t1.key != '333'; -SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t22.key == t22.key2 OR t1.id == t22.id); -- { serverError 403 } -SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t1.id == t22.id OR t22.key == t22.key2); -- { serverError 403 } -SELECT 't22', * FROM t1 JOIN t22 ON (t22.key == t22.key2 OR t1.id == t22.id) and t1.id == t22.idd; -- { serverError 403 } -SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and t1.id == t22.idd; -- { serverError 403 } -SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and (t1.id == t22.idd AND (t1.key2 = 'a1' OR t1.key2 = 'a2' OR t1.key2 = 'a3' OR t1.key2 = 'a4' OR t1.key2 = 'a5' OR t1.key2 = 'a6' OR t1.key2 = 'a7' OR t1.key2 = 'a8' OR t1.key2 = 'a9' OR t1.key2 = 'a10' OR t1.key2 = 'a11' OR t1.key2 = 'a12' OR t1.key2 = 'a13' OR t1.key2 = 'a14' OR t1.key2 = 'a15' OR t1.key2 = 'a16' OR t1.key2 = 'a17' OR t1.key2 = 'a18' OR t1.key2 = 'a19' OR t1.key2 = '111')); -- { serverError 403 } +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t22.key == t22.key2 OR t1.id == t22.id) FORMAT Null; +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t1.id == t22.id OR t22.key == t22.key2) FORMAT Null; +SELECT 't22', * FROM t1 JOIN t22 ON (t22.key == t22.key2 OR t1.id == t22.id) and t1.id == t22.idd FORMAT Null; +SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and t1.id == t22.idd FORMAT Null; +SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and (t1.id == t22.idd AND (t1.key2 = 'a1' OR t1.key2 = 'a2' OR t1.key2 = 'a3' OR t1.key2 = 'a4' OR t1.key2 = 'a5' OR t1.key2 = 'a6' OR t1.key2 = 'a7' OR t1.key2 = 'a8' OR t1.key2 = 'a9' OR t1.key2 = 'a10' OR t1.key2 = 'a11' OR t1.key2 = 'a12' OR t1.key2 = 'a13' OR t1.key2 = 'a14' OR t1.key2 = 'a15' OR t1.key2 = 'a16' OR t1.key2 = 'a17' OR t1.key2 = 'a18' OR t1.key2 = 'a19' OR t1.key2 = '111')) FORMAT Null; SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t22.key == t22.key2 OR t1.id == t22.idd and t1.id == t22.id; SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t1.id == t22.id OR t1.id == t22.idd and t22.key == t22.key2; SELECT 't22', * FROM t1 JOIN t22 ON t22.key == t22.key2 and t1.id == t22.idd OR t1.id == t22.id and t1.id == t22.idd; diff --git a/tests/queries/0_stateless/02834_array_exists_segfault.reference b/tests/queries/0_stateless/02834_array_exists_segfault.reference index e69de29bb2d..e3681f7ad68 100644 --- a/tests/queries/0_stateless/02834_array_exists_segfault.reference +++ b/tests/queries/0_stateless/02834_array_exists_segfault.reference @@ -0,0 +1,2 @@ +1 [1] +3 [1,3] diff --git a/tests/queries/0_stateless/02834_array_exists_segfault.sql b/tests/queries/0_stateless/02834_array_exists_segfault.sql index 3cf457610fa..efb1e8fc35c 100644 --- a/tests/queries/0_stateless/02834_array_exists_segfault.sql +++ b/tests/queries/0_stateless/02834_array_exists_segfault.sql @@ -1,4 +1,6 @@ DROP TABLE IF EXISTS 02834_t; CREATE TABLE 02834_t (id UInt64, arr Array(UInt64)) ENGINE = MergeTree ORDER BY id; -WITH subquery AS (SELECT []) SELECT t.* FROM 02834_t AS t JOIN subquery ON arrayExists(x -> x = 1, t.arr); -- { serverError INVALID_JOIN_ON_EXPRESSION } +WITH subquery AS (SELECT []) SELECT t.* FROM 02834_t AS t JOIN subquery ON arrayExists(x -> x = 1, t.arr); +INSERT INTO 02834_t VALUES (1, [1]), (2, [2]), (3, [1, 3]); +WITH subquery AS (SELECT []) SELECT t.* FROM 02834_t AS t JOIN subquery ON arrayExists(x -> x = 1, t.arr) ORDER BY t.id; DROP TABLE 02834_t; From ccc23e7b2e9162c064617f359e303f0f7c94ee5f Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 3 Oct 2024 12:26:39 +0000 Subject: [PATCH 09/59] upd tests --- src/Planner/PlannerJoinTree.cpp | 20 ------------------- .../01881_join_on_conditions_hash.sql.j2 | 10 ---------- .../01881_join_on_conditions_merge.sql.j2 | 6 +----- .../02834_array_exists_segfault.sql | 3 +++ 4 files changed, 4 insertions(+), 35 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index f97325b303a..135a90fb0eb 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1302,26 +1302,6 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ && (join_kind == JoinKind::Inner || join_kind == JoinKind::Cross || join_kind == JoinKind::Comma) && (right_pre_filters.empty() || FilterStep::canUseType(right_pre_filters[0]->result_type)) && (left_pre_filters.empty() || FilterStep::canUseType(left_pre_filters[0]->result_type)); - if (can_move_out_residuals && join_kind != JoinKind::Cross && join_kind != JoinKind::Comma) - { - /* In case we have only conditions like x.column1 = x.column2, do not move them to WHERE. - * ClickHouse has pretty complicated logic to resolve qualified column names, so a user may write something like: - * SELECT dummy FROM one l JOIN one r ON l.dummy = dummy; - * (tests/queries/0_stateless/01018_ambiguous_column.sql) - * That case is definitely not a cross join with condition `dummy = dummy`, - * and we should keep the behavior of throwing an exception. - */ - const auto & join_clause = join_clauses_and_actions.join_clauses[0]; - bool has_only_single_table_conditions = join_clause.getMixedFilterConditionNodes().size() == 0 - && join_clause.getLeftKeyNodes().size() == 0 - && join_clause.getRightKeyNodes().size() == 0; - - bool has_only_single_table_equals = has_only_single_table_conditions - && (right_pre_filters.empty() || (right_pre_filters[0]->function_base && right_pre_filters[0]->function_base->getName() != "equals")) - && (left_pre_filters.empty() || (left_pre_filters[0]->function_base && left_pre_filters[0]->function_base->getName() != "equals")); - - can_move_out_residuals = has_only_single_table_equals; - } join_clauses_and_actions.left_join_expressions_actions.appendInputsForUnusedColumns(left_plan.getCurrentDataStream().header); appendSetsFromActionsDAG(join_clauses_and_actions.left_join_expressions_actions, left_join_tree_query_plan.useful_sets); diff --git a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 index ecca94eacea..2f965407e87 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 @@ -72,23 +72,13 @@ SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id + 2; -- { serverE SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id + 2; -- { serverError 403 } SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.key; -- { serverError 43, 403 } SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverError 43, 403 } -SELECT * FROM t1 JOIN t2_nullable as t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 } -SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id FORMAT Null; SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id; --- non-equi condition containing columns from different tables doesn't supported yet -SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id FORMAT Null; -SELECT * FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t2.key) FORMAT Null; SELECT '--'; -- length(t1.key2) == length(t2.key2) is expression for columns from both tables, it works because it part of joining key SELECT t1.*, t2.* FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND length(t1.key2) == length(t2.key2) AND t1.key != '333'; -SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t22.key == t22.key2 OR t1.id == t22.id) FORMAT Null; -SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t1.id == t22.id OR t22.key == t22.key2) FORMAT Null; -SELECT 't22', * FROM t1 JOIN t22 ON (t22.key == t22.key2 OR t1.id == t22.id) and t1.id == t22.idd FORMAT Null; -SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and t1.id == t22.idd FORMAT Null; -SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and (t1.id == t22.idd AND (t1.key2 = 'a1' OR t1.key2 = 'a2' OR t1.key2 = 'a3' OR t1.key2 = 'a4' OR t1.key2 = 'a5' OR t1.key2 = 'a6' OR t1.key2 = 'a7' OR t1.key2 = 'a8' OR t1.key2 = 'a9' OR t1.key2 = 'a10' OR t1.key2 = 'a11' OR t1.key2 = 'a12' OR t1.key2 = 'a13' OR t1.key2 = 'a14' OR t1.key2 = 'a15' OR t1.key2 = 'a16' OR t1.key2 = 'a17' OR t1.key2 = 'a18' OR t1.key2 = 'a19' OR t1.key2 = '111')) FORMAT Null; SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t22.key == t22.key2 OR t1.id == t22.idd and t1.id == t22.id; SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t1.id == t22.id OR t1.id == t22.idd and t22.key == t22.key2; SELECT 't22', * FROM t1 JOIN t22 ON t22.key == t22.key2 and t1.id == t22.idd OR t1.id == t22.id and t1.id == t22.idd; diff --git a/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 index c2dcf88885c..6960e1ddd94 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 @@ -70,13 +70,9 @@ SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id + 2; -- { serverE SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id + 2; -- { serverError INVALID_JOIN_ON_EXPRESSION } SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.key; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT, INVALID_JOIN_ON_EXPRESSION } SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT, INVALID_JOIN_ON_EXPRESSION } -SELECT * FROM t1 JOIN t2_nullable as t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError INVALID_JOIN_ON_EXPRESSION } +SELECT * FROM t1 JOIN t2_nullable as t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError NOT_IMPLEMENTED,INVALID_JOIN_ON_EXPRESSION } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError INVALID_JOIN_ON_EXPRESSION } -SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError INVALID_JOIN_ON_EXPRESSION } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id; -- { serverError NOT_IMPLEMENTED } --- non-equi condition containing columns from different tables doesn't supported yet -SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id; -- { serverError INVALID_JOIN_ON_EXPRESSION } -SELECT * FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t2.key); -- { serverError INVALID_JOIN_ON_EXPRESSION } SELECT '--'; -- length(t1.key2) == length(t2.key2) is expression for columns from both tables, it works because it part of joining key diff --git a/tests/queries/0_stateless/02834_array_exists_segfault.sql b/tests/queries/0_stateless/02834_array_exists_segfault.sql index efb1e8fc35c..faf90589174 100644 --- a/tests/queries/0_stateless/02834_array_exists_segfault.sql +++ b/tests/queries/0_stateless/02834_array_exists_segfault.sql @@ -1,5 +1,8 @@ DROP TABLE IF EXISTS 02834_t; CREATE TABLE 02834_t (id UInt64, arr Array(UInt64)) ENGINE = MergeTree ORDER BY id; +SET enable_analyzer = 0; +WITH subquery AS (SELECT []) SELECT t.* FROM 02834_t AS t JOIN subquery ON arrayExists(x -> x = 1, t.arr); -- { serverError INVALID_JOIN_ON_EXPRESSION } +SET enable_analyzer = 1; WITH subquery AS (SELECT []) SELECT t.* FROM 02834_t AS t JOIN subquery ON arrayExists(x -> x = 1, t.arr); INSERT INTO 02834_t VALUES (1, [1]), (2, [2]), (3, [1, 3]); WITH subquery AS (SELECT []) SELECT t.* FROM 02834_t AS t JOIN subquery ON arrayExists(x -> x = 1, t.arr) ORDER BY t.id; From c4b4ce7a5212c257111e31170c3829664ab46522 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 3 Oct 2024 15:35:44 +0000 Subject: [PATCH 10/59] fixes --- src/Planner/PlannerJoinTree.cpp | 4 +++- src/Planner/PlannerJoins.cpp | 2 +- .../01018_Distributed__shard_num.reference | 12 ++++++++++-- .../0_stateless/01018_Distributed__shard_num.sql | 9 +++++++-- .../queries/0_stateless/01018_ambiguous_column.sql | 2 -- .../01081_PartialSortingTransform_full_column.sql | 14 ++++++++++++-- .../0_stateless/01429_join_on_error_messages.sql | 3 --- tests/queries/0_stateless/01661_join_complex.sql | 5 ----- .../01925_join_materialized_columns.sql | 1 - tests/queries/0_stateless/02000_join_on_const.sql | 1 - .../02276_full_sort_join_unsupported.sql | 4 ---- .../03006_join_on_inequal_expression_fast.sql.j2 | 4 ++-- .../queries/0_stateless/03246_join_on_asterisk.sql | 2 +- 13 files changed, 36 insertions(+), 27 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 135a90fb0eb..ec2a37ded85 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1298,7 +1298,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected only one right pre-filter condition node. Actual [{}]", fmt::join(right_pre_filters | std::views::transform([](const auto & node) { return node->result_name; }), ", ")); can_move_out_residuals = join_clauses_and_actions.join_clauses.size() == 1 - && (join_strictness == JoinStrictness::Any || join_strictness == JoinStrictness::All) + && join_strictness == JoinStrictness::All && (join_kind == JoinKind::Inner || join_kind == JoinKind::Cross || join_kind == JoinKind::Comma) && (right_pre_filters.empty() || FilterStep::canUseType(right_pre_filters[0]->result_type)) && (left_pre_filters.empty() || FilterStep::canUseType(left_pre_filters[0]->result_type)); @@ -1697,12 +1697,14 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ for (const auto * input_node : join_clauses_and_actions.mixed_join_expressions_actions->getInputs()) join_clauses_and_actions.mixed_join_expressions_actions->addOrReplaceInOutputs(*input_node); + appendSetsFromActionsDAG(*join_clauses_and_actions.mixed_join_expressions_actions, left_join_tree_query_plan.useful_sets); auto filter_step = std::make_unique(result_plan.getCurrentDataStream(), std::move(*join_clauses_and_actions.mixed_join_expressions_actions), outputs[0]->result_name, /* remove_column = */ false); /// Unused columns will be removed by next step filter_step->setStepDescription("Residual JOIN filter"); result_plan.addStep(std::move(filter_step)); + join_clauses_and_actions.mixed_join_expressions_actions.reset(); } diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 274da438bd4..0d0ccb47206 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -363,7 +363,7 @@ void buildJoinClause( /// If join_use_nulls = true, the columns' nullability will be changed later which make this expression not applicable. auto strictness = join_node.getStrictness(); auto kind = join_node.getKind(); - bool can_be_moved_out = (strictness == JoinStrictness::Any || strictness == JoinStrictness::All) + bool can_be_moved_out = strictness == JoinStrictness::All && (kind == JoinKind::Inner || kind == JoinKind::Cross || kind == JoinKind::Comma); if (can_be_moved_out || !join_use_nulls) { diff --git a/tests/queries/0_stateless/01018_Distributed__shard_num.reference b/tests/queries/0_stateless/01018_Distributed__shard_num.reference index de223d4f464..340e9d58677 100644 --- a/tests/queries/0_stateless/01018_Distributed__shard_num.reference +++ b/tests/queries/0_stateless/01018_Distributed__shard_num.reference @@ -74,7 +74,11 @@ SELECT _shard_num, key, b.host_name, b.host_address IN ('::1', '127.0.0.1'), b.p FROM dist_1 a JOIN system.clusters b ON _shard_num = b.shard_num -WHERE b.cluster = 'test_cluster_two_shards_localhost'; -- { serverError INVALID_JOIN_ON_EXPRESSION } +WHERE b.cluster = 'test_cluster_two_shards_localhost' +ORDER BY key +SETTINGS enable_analyzer = 1; +1 10 localhost 1 9000 +1 20 localhost 1 9000 SELECT 'Rewrite with alias'; Rewrite with alias SELECT a._shard_num, key FROM dist_1 a; @@ -85,7 +89,11 @@ SELECT a._shard_num, a.key, b.host_name, b.host_address IN ('::1', '127.0.0.1'), FROM dist_1 a JOIN system.clusters b ON a._shard_num = b.shard_num -WHERE b.cluster = 'test_cluster_two_shards_localhost'; -- { serverError UNKNOWN_IDENTIFIER, 403 } +WHERE b.cluster = 'test_cluster_two_shards_localhost' +ORDER BY key +SETTINGS enable_analyzer = 1; +1 10 localhost 1 9000 +1 20 localhost 1 9000 SELECT 'dist_3'; dist_3 SELECT * FROM dist_3; diff --git a/tests/queries/0_stateless/01018_Distributed__shard_num.sql b/tests/queries/0_stateless/01018_Distributed__shard_num.sql index 3b793da6dfb..662336febcb 100644 --- a/tests/queries/0_stateless/01018_Distributed__shard_num.sql +++ b/tests/queries/0_stateless/01018_Distributed__shard_num.sql @@ -73,16 +73,21 @@ SELECT _shard_num, key, b.host_name, b.host_address IN ('::1', '127.0.0.1'), b.p FROM dist_1 a JOIN system.clusters b ON _shard_num = b.shard_num -WHERE b.cluster = 'test_cluster_two_shards_localhost'; -- { serverError INVALID_JOIN_ON_EXPRESSION } +WHERE b.cluster = 'test_cluster_two_shards_localhost' +ORDER BY key +SETTINGS enable_analyzer = 1; SELECT 'Rewrite with alias'; SELECT a._shard_num, key FROM dist_1 a; + -- the same with JOIN, just in case SELECT a._shard_num, a.key, b.host_name, b.host_address IN ('::1', '127.0.0.1'), b.port FROM dist_1 a JOIN system.clusters b ON a._shard_num = b.shard_num -WHERE b.cluster = 'test_cluster_two_shards_localhost'; -- { serverError UNKNOWN_IDENTIFIER, 403 } +WHERE b.cluster = 'test_cluster_two_shards_localhost' +ORDER BY key +SETTINGS enable_analyzer = 1; SELECT 'dist_3'; SELECT * FROM dist_3; diff --git a/tests/queries/0_stateless/01018_ambiguous_column.sql b/tests/queries/0_stateless/01018_ambiguous_column.sql index b2e0e8fc522..aa1519482b3 100644 --- a/tests/queries/0_stateless/01018_ambiguous_column.sql +++ b/tests/queries/0_stateless/01018_ambiguous_column.sql @@ -12,9 +12,7 @@ USE system; SELECT dummy FROM one AS A JOIN one ON A.dummy = one.dummy; SELECT dummy FROM one JOIN one AS A ON A.dummy = one.dummy; SELECT dummy FROM one l JOIN one r ON dummy = r.dummy; -SELECT dummy FROM one l JOIN one r ON l.dummy = dummy; -- { serverError INVALID_JOIN_ON_EXPRESSION } SELECT dummy FROM one l JOIN one r ON one.dummy = r.dummy; -SELECT dummy FROM one l JOIN one r ON l.dummy = one.dummy; -- { serverError INVALID_JOIN_ON_EXPRESSION } SELECT * from one JOIN one A ON one.dummy = A.dummy diff --git a/tests/queries/0_stateless/01081_PartialSortingTransform_full_column.sql b/tests/queries/0_stateless/01081_PartialSortingTransform_full_column.sql index 6e502611479..d891d85175d 100644 --- a/tests/queries/0_stateless/01081_PartialSortingTransform_full_column.sql +++ b/tests/queries/0_stateless/01081_PartialSortingTransform_full_column.sql @@ -2,7 +2,13 @@ drop table if exists test_01081; create table test_01081 (key Int) engine=MergeTree() order by key; insert into test_01081 select * from system.numbers limit 10; -select 1 from remote('127.{1,2}', currentDatabase(), test_01081) lhs join system.one as rhs on rhs.dummy = 1 order by 1; -- { serverError INVALID_JOIN_ON_EXPRESSION } + +select 1 from remote('127.{1,2}', currentDatabase(), test_01081) lhs join system.one as rhs on rhs.dummy = 1 order by 1 +SETTINGS enable_analyzer = 0; -- { serverError INVALID_JOIN_ON_EXPRESSION } + + +select 1 from remote('127.{1,2}', currentDatabase(), test_01081) lhs join system.one as rhs on rhs.dummy = 1 order by 1 +SETTINGS enable_analyzer = 1; -- With multiple blocks triggers: -- @@ -11,6 +17,10 @@ select 1 from remote('127.{1,2}', currentDatabase(), test_01081) lhs join system -- _dummy Int Int32(size = 0), 1 UInt8 Const(size = 0, UInt8(size = 1)). insert into test_01081 select * from system.numbers limit 10; -select 1 from remote('127.{1,2}', currentDatabase(), test_01081) lhs join system.one as rhs on rhs.dummy = 1 order by 1; -- { serverError INVALID_JOIN_ON_EXPRESSION } +select 1 from remote('127.{1,2}', currentDatabase(), test_01081) lhs join system.one as rhs on rhs.dummy = 1 order by 1 +SETTINGS enable_analyzer = 0; -- { serverError INVALID_JOIN_ON_EXPRESSION } + +select 1 from remote('127.{1,2}', currentDatabase(), test_01081) lhs join system.one as rhs on rhs.dummy = 1 order by 1 +SETTINGS enable_analyzer = 1; drop table if exists test_01081; diff --git a/tests/queries/0_stateless/01429_join_on_error_messages.sql b/tests/queries/0_stateless/01429_join_on_error_messages.sql index 9e8dec63c58..66bcc711176 100644 --- a/tests/queries/0_stateless/01429_join_on_error_messages.sql +++ b/tests/queries/0_stateless/01429_join_on_error_messages.sql @@ -1,6 +1,3 @@ -SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON (arrayJoin([1]) = B.b); -- { serverError INVALID_JOIN_ON_EXPRESSION } -SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON (A.a = arrayJoin([1])); -- { serverError INVALID_JOIN_ON_EXPRESSION } - SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON equals(a); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH, 62 } SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON less(a); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH, 62 } diff --git a/tests/queries/0_stateless/01661_join_complex.sql b/tests/queries/0_stateless/01661_join_complex.sql index 7ce426dee22..1ff5da3f4a8 100644 --- a/tests/queries/0_stateless/01661_join_complex.sql +++ b/tests/queries/0_stateless/01661_join_complex.sql @@ -8,11 +8,6 @@ select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (or(t1.d = t2.b and t1.e = t2.e, t1.d = t2.b and t1.f=t2.f, t1.c = t2.b and t1.e = t2.e, t1.c = t2.b and t1.f=t2.f)); select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b and t1.e = t2.e or t1.d = t2.b and t1.f=t2.f or t1.c = t2.b and t1.e = t2.e or t1.c = t2.b and t1.f=t2.f); -select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 left join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b or t1.d = t2.b and t1.d = t2.b) or (t1.e = t2.e and t1.a=t2.a and t2.f=t1.f); -- { serverError INVALID_JOIN_ON_EXPRESSION } -select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 right join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b) or t1.e = t2.e; -- { serverError INVALID_JOIN_ON_EXPRESSION } -select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b) or (t1.e = t2.e and t1.a=t2.a and t2.f=t1.f); -- { serverError INVALID_JOIN_ON_EXPRESSION } -select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b) and (t1.e = t2.e or t1.f=t2.f); -- { serverError INVALID_JOIN_ON_EXPRESSION } - SET joined_subquery_requires_alias = 0; SET max_threads = 1; diff --git a/tests/queries/0_stateless/01925_join_materialized_columns.sql b/tests/queries/0_stateless/01925_join_materialized_columns.sql index ce71cfed8a3..b6fe8efcca1 100644 --- a/tests/queries/0_stateless/01925_join_materialized_columns.sql +++ b/tests/queries/0_stateless/01925_join_materialized_columns.sql @@ -52,6 +52,5 @@ SELECT t1.time as talias FROM t1 JOIN t2 ON talias = t2.time; SELECT t1.time as talias FROM t1 JOIN t2 ON talias = t2.time_alias; SELECT t2.time as talias FROM t1 JOIN t2 ON t1.time = talias; SELECT t2.time as talias FROM t1 JOIN t2 ON t1.time_alias = talias; -SELECT time as talias FROM t1 JOIN t2 ON t1.time = talias; -- { serverError AMBIGUOUS_COLUMN_NAME, INVALID_JOIN_ON_EXPRESSION } SELECT time as talias FROM t1 JOIN t2 ON talias = t2.time settings enable_analyzer=0; -- { serverError AMBIGUOUS_COLUMN_NAME } SELECT time as talias FROM t1 JOIN t2 ON talias = t2.time settings enable_analyzer=1; diff --git a/tests/queries/0_stateless/02000_join_on_const.sql b/tests/queries/0_stateless/02000_join_on_const.sql index da70973ed87..8f939f7b7da 100644 --- a/tests/queries/0_stateless/02000_join_on_const.sql +++ b/tests/queries/0_stateless/02000_join_on_const.sql @@ -63,7 +63,6 @@ SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 == 2 SETTINGS enable_analyzer = SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 != 1 SETTINGS enable_analyzer = 0; -- { serverError INVALID_JOIN_ON_EXPRESSION } SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 1 != 1 SETTINGS enable_analyzer = 1; -SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND NULL; -- { serverError INVALID_JOIN_ON_EXPRESSION } SELECT * FROM t1 JOIN t2 ON t1.id = t2.id AND 'aaa'; -- { serverError INVALID_JOIN_ON_EXPRESSION,ILLEGAL_TYPE_OF_ARGUMENT } SELECT * FROM t1 JOIN t2 ON 'aaa'; -- { serverError INVALID_JOIN_ON_EXPRESSION } diff --git a/tests/queries/0_stateless/02276_full_sort_join_unsupported.sql b/tests/queries/0_stateless/02276_full_sort_join_unsupported.sql index 0b10101d8f2..4ef5ea7ced4 100644 --- a/tests/queries/0_stateless/02276_full_sort_join_unsupported.sql +++ b/tests/queries/0_stateless/02276_full_sort_join_unsupported.sql @@ -9,10 +9,6 @@ INSERT INTO t2 VALUES (1, 2); SET join_algorithm = 'full_sorting_merge'; -SELECT * FROM t1 JOIN t2 ON t1.key = t2.key AND t2.key > 0; -- { serverError NOT_IMPLEMENTED } - -SELECT * FROM t1 JOIN t2 ON t1.key = t2.key AND t1.key > 0; -- { serverError NOT_IMPLEMENTED } - SELECT * FROM t1 JOIN t2 ON t1.key = t2.key OR t1.val = t2.key; -- { serverError NOT_IMPLEMENTED } SELECT * FROM t1 ANTI JOIN t2 ON t1.key = t2.key; -- { serverError NOT_IMPLEMENTED } diff --git a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 index b300881c562..029cf10d3c3 100644 --- a/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 +++ b/tests/queries/0_stateless/03006_join_on_inequal_expression_fast.sql.j2 @@ -76,7 +76,7 @@ SELECT t1.* FROM t1 {{ join_type }} {{ join_strictness }} JOIN t2 ON t1.key = t2 -- test error messages {% for algorithm in ['partial_merge', 'full_sorting_merge', 'parallel_hash', 'auto', 'direct'] -%} SET join_algorithm='{{ algorithm }}'; -{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for join_type in ['LEFT', 'RIGHT', 'FULL'] -%} SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError NOT_IMPLEMENTED } SELECT t1.*, t2.* from t1 {{ join_type }} JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError NOT_IMPLEMENTED } SELECT t1.*, t2.* from t1 {{ join_type }} JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError NOT_IMPLEMENTED } @@ -85,7 +85,7 @@ SELECT t1.*, t2.* from t1 {{ join_type }} JOIN t2 ON t1.key = t2.key and (t1.a < {% for algorithm in ['grace_hash', 'partial_merge', 'full_sorting_merge', 'parallel_hash', 'auto', 'direct'] -%} SET join_algorithm='{{ algorithm }}'; -{% for join_type in ['LEFT', 'INNER', 'RIGHT', 'FULL'] -%} +{% for join_type in ['LEFT', 'RIGHT', 'FULL'] -%} SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY (t1.key, t1.attr, t2.key, t2.attr); -- { serverError NOT_IMPLEMENTED } {% endfor -%} {% endfor -%} diff --git a/tests/queries/0_stateless/03246_join_on_asterisk.sql b/tests/queries/0_stateless/03246_join_on_asterisk.sql index f9b611820a7..4cf19fbdf82 100644 --- a/tests/queries/0_stateless/03246_join_on_asterisk.sql +++ b/tests/queries/0_stateless/03246_join_on_asterisk.sql @@ -3,5 +3,5 @@ DROP TABLE IF EXISTS t0; CREATE TABLE t0 (c0 Int) ENGINE = Memory(); SELECT 1 FROM t0 JOIN t0 ON *; -- { serverError BAD_ARGUMENTS,INVALID_JOIN_ON_EXPRESSION } -SELECT 1 FROM t0 JOIN t0 ON (*,); -- { serverError AMBIGUOUS_COLUMN_NAME,INVALID_JOIN_ON_EXPRESSION } +SELECT 1 FROM t0 JOIN t0 ON (*,); -- { serverError AMBIGUOUS_COLUMN_NAME,INVALID_JOIN_ON_EXPRESSION,ILLEGAL_TYPE_OF_ARGUMENT } SELECT 1 FROM t0 JOIN t0 USING *; -- { serverError BAD_ARGUMENTS,UNSUPPORTED_JOIN_KEYS } From a500b73026b258713eb455976dc3135c8f55b359 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 7 Oct 2024 15:09:35 +0000 Subject: [PATCH 11/59] fix tests --- .../0_stateless/01018_ambiguous_column.reference | 2 ++ .../0_stateless/01018_ambiguous_column.sql | 3 ++- tests/queries/0_stateless/01661_join_complex.sql | 5 +++++ .../01881_join_on_conditions_hash.reference.j2 | 15 +++++++++++++++ .../01881_join_on_conditions_hash.sql.j2 | 9 ++++++++- .../01925_join_materialized_columns.reference | 9 +++++++++ .../01925_join_materialized_columns.sql | 3 +++ .../02276_full_sort_join_unsupported.sql | 3 +++ 8 files changed, 47 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01018_ambiguous_column.reference b/tests/queries/0_stateless/01018_ambiguous_column.reference index ddc2ae1c572..17070feb2e5 100644 --- a/tests/queries/0_stateless/01018_ambiguous_column.reference +++ b/tests/queries/0_stateless/01018_ambiguous_column.reference @@ -6,6 +6,8 @@ 0 0 0 +0 +0 0 ┌─system.one.dummy─┬─A.dummy─┬─B.dummy─┐ 1. │ 0 │ 0 │ 0 │ diff --git a/tests/queries/0_stateless/01018_ambiguous_column.sql b/tests/queries/0_stateless/01018_ambiguous_column.sql index aa1519482b3..57ed86a2e2f 100644 --- a/tests/queries/0_stateless/01018_ambiguous_column.sql +++ b/tests/queries/0_stateless/01018_ambiguous_column.sql @@ -13,6 +13,8 @@ SELECT dummy FROM one AS A JOIN one ON A.dummy = one.dummy; SELECT dummy FROM one JOIN one AS A ON A.dummy = one.dummy; SELECT dummy FROM one l JOIN one r ON dummy = r.dummy; SELECT dummy FROM one l JOIN one r ON one.dummy = r.dummy; +SELECT dummy FROM one l JOIN one r ON l.dummy = dummy; +SELECT dummy FROM one l JOIN one r ON l.dummy = one.dummy; SELECT * from one JOIN one A ON one.dummy = A.dummy @@ -24,5 +26,4 @@ JOIN system.one one ON A.dummy = one.dummy JOIN system.one two ON A.dummy = two.dummy FORMAT PrettyCompact; --- SELECT one.dummy FROM one AS A FULL JOIN (SELECT 0 AS dymmy) AS one USING dummy; SELECT one.dummy FROM one AS A JOIN (SELECT 0 AS dummy) B USING dummy; diff --git a/tests/queries/0_stateless/01661_join_complex.sql b/tests/queries/0_stateless/01661_join_complex.sql index 1ff5da3f4a8..6e8cdabd0b1 100644 --- a/tests/queries/0_stateless/01661_join_complex.sql +++ b/tests/queries/0_stateless/01661_join_complex.sql @@ -8,6 +8,11 @@ select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (or(t1.d = t2.b and t1.e = t2.e, t1.d = t2.b and t1.f=t2.f, t1.c = t2.b and t1.e = t2.e, t1.c = t2.b and t1.f=t2.f)); select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b and t1.e = t2.e or t1.d = t2.b and t1.f=t2.f or t1.c = t2.b and t1.e = t2.e or t1.c = t2.b and t1.f=t2.f); +select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 left join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b or t1.d = t2.b and t1.d = t2.b) or (t1.e = t2.e and t1.a=t2.a and t2.f=t1.f); -- { serverError INVALID_JOIN_ON_EXPRESSION } +select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 right join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b) or t1.e = t2.e; -- { serverError INVALID_JOIN_ON_EXPRESSION } +select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b) or (t1.e = t2.e and t1.a=t2.a and t2.f=t1.f); -- { serverError INVALID_JOIN_ON_EXPRESSION } +select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b) and (t1.e = t2.e or t1.f=t2.f) SETTINGS enable_analyzer = 1; + SET joined_subquery_requires_alias = 0; SET max_threads = 1; diff --git a/tests/queries/0_stateless/01881_join_on_conditions_hash.reference.j2 b/tests/queries/0_stateless/01881_join_on_conditions_hash.reference.j2 index 94b3e19483f..57eef76e6e3 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_hash.reference.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_hash.reference.j2 @@ -63,6 +63,16 @@ 2 222 2 2 AAA AAA 2 222 222 2 AAA AAA 3 333 333 3 BBB BBB +2 222 2 2 AAA AAA +2 222 222 2 AAA AAA +2 222 2 2 AAA AAA +2 222 222 2 AAA AAA +3 333 333 3 BBB BBB +2 222 2 2 AAA AAA +2 222 2 2 AAA a +2 222 222 2 AAA AAA +2 222 222 2 AAA a +3 333 333 3 BBB BBB -- 2 222 2 2 AAA a 2 222 222 2 AAA AAA @@ -70,4 +80,9 @@ t22 1 111 111 2 1 AAA AAA t22 1 111 111 2 1 AAA AAA t22 1 111 111 2 1 AAA AAA t22 1 111 111 2 1 AAA AAA +t22 1 111 111 2 1 AAA AAA +t22 1 111 111 2 1 AAA AAA +t22 1 111 111 2 1 AAA AAA +t22 1 111 111 2 1 AAA AAA +t22 1 111 111 2 1 AAA AAA {% endfor -%} diff --git a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 index 2f965407e87..654fa0961cc 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 @@ -74,15 +74,22 @@ SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.key; -- { serverErro SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverError 43, 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id; +SELECT * FROM t1 JOIN t2_nullable as t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)) ORDER BY ALL SETTINGS enable_analyzer = 1; +SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id ORDER BY ALL SETTINGS enable_analyzer = 1; +SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id ORDER BY ALL SETTINGS enable_analyzer = 1; SELECT '--'; --- length(t1.key2) == length(t2.key2) is expression for columns from both tables, it works because it part of joining key SELECT t1.*, t2.* FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND length(t1.key2) == length(t2.key2) AND t1.key != '333'; SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t22.key == t22.key2 OR t1.id == t22.idd and t1.id == t22.id; SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t1.id == t22.id OR t1.id == t22.idd and t22.key == t22.key2; SELECT 't22', * FROM t1 JOIN t22 ON t22.key == t22.key2 and t1.id == t22.idd OR t1.id == t22.id and t1.id == t22.idd; SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.id and t1.id == t22.idd OR t22.key == t22.key2 and t1.id == t22.idd; +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t22.key == t22.key2 OR t1.id == t22.id) ORDER BY ALL SETTINGS enable_analyzer = 1; +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t1.id == t22.id OR t22.key == t22.key2) ORDER BY ALL SETTINGS enable_analyzer = 1; +SELECT 't22', * FROM t1 JOIN t22 ON (t22.key == t22.key2 OR t1.id == t22.id) and t1.id == t22.idd ORDER BY ALL SETTINGS enable_analyzer = 1; +SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and t1.id == t22.idd ORDER BY ALL SETTINGS enable_analyzer = 1; +SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and (t1.id == t22.idd AND (t1.key2 = 'a1' OR t1.key2 = 'a2' OR t1.key2 = 'a3' OR t1.key2 = 'a4' OR t1.key2 = 'a5' OR t1.key2 = 'a6' OR t1.key2 = 'a7' OR t1.key2 = 'a8' OR t1.key2 = 'a9' OR t1.key2 = 'a10' OR t1.key2 = 'a11' OR t1.key2 = 'a12' OR t1.key2 = 'a13' OR t1.key2 = 'a14' OR t1.key2 = 'a15' OR t1.key2 = 'a16' OR t1.key2 = 'a17' OR t1.key2 = 'a18' OR t1.key2 = 'a19' OR t1.key2 = '111')) ORDER BY ALL SETTINGS enable_analyzer = 1; {% endfor -%} diff --git a/tests/queries/0_stateless/01925_join_materialized_columns.reference b/tests/queries/0_stateless/01925_join_materialized_columns.reference index 1d2ab582038..062d00a55eb 100644 --- a/tests/queries/0_stateless/01925_join_materialized_columns.reference +++ b/tests/queries/0_stateless/01925_join_materialized_columns.reference @@ -35,3 +35,12 @@ fact2t1_val2 fact2t1_val2 2020-01-01 12:00:00 2020-01-01 12:00:00 2020-01-01 12:00:00 +2020-01-01 12:00:00 +2020-01-01 12:00:00 +2020-01-01 12:00:00 +2020-01-01 13:00:00 +2020-01-01 13:00:00 +2020-01-01 13:00:00 +2020-02-02 13:00:00 +2020-02-02 13:00:00 +2020-02-02 13:00:00 diff --git a/tests/queries/0_stateless/01925_join_materialized_columns.sql b/tests/queries/0_stateless/01925_join_materialized_columns.sql index b6fe8efcca1..8721dca87de 100644 --- a/tests/queries/0_stateless/01925_join_materialized_columns.sql +++ b/tests/queries/0_stateless/01925_join_materialized_columns.sql @@ -54,3 +54,6 @@ SELECT t2.time as talias FROM t1 JOIN t2 ON t1.time = talias; SELECT t2.time as talias FROM t1 JOIN t2 ON t1.time_alias = talias; SELECT time as talias FROM t1 JOIN t2 ON talias = t2.time settings enable_analyzer=0; -- { serverError AMBIGUOUS_COLUMN_NAME } SELECT time as talias FROM t1 JOIN t2 ON talias = t2.time settings enable_analyzer=1; + +SELECT time as talias FROM t1 JOIN t2 ON t1.time = talias ORDER BY ALL settings enable_analyzer=1; +SELECT * FROM t1 JOIN t2 ON t1.time = t2.time AND NULL ORDER BY ALL settings enable_analyzer=1; diff --git a/tests/queries/0_stateless/02276_full_sort_join_unsupported.sql b/tests/queries/0_stateless/02276_full_sort_join_unsupported.sql index 4ef5ea7ced4..2c70799e01e 100644 --- a/tests/queries/0_stateless/02276_full_sort_join_unsupported.sql +++ b/tests/queries/0_stateless/02276_full_sort_join_unsupported.sql @@ -26,5 +26,8 @@ SELECT * FROM t1 JOIN ( SELECT key, sum(val) AS val FROM t2 GROUP BY key WITH TO SELECT * FROM ( SELECT key, sum(val) AS val FROM t1 GROUP BY key WITH TOTALS ) as t1 JOIN t2 ON t1.key = t2.key; -- { serverError NOT_IMPLEMENTED } +SELECT * FROM t1 FULL JOIN t2 ON t1.key = t2.key AND t2.key > 0; -- { serverError NOT_IMPLEMENTED } +SELECT * FROM t1 FULL JOIN t2 ON t1.key = t2.key AND t1.key > 0; -- { serverError NOT_IMPLEMENTED } + DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; 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 12/59] 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 08ec69736f827770fcb45a8fb7e8b90e4ec8d5de Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 4 Nov 2024 10:36:30 +0000 Subject: [PATCH 13/59] review fix --- .../sql-reference/statements/select/join.md | 13 +--- src/Analyzer/Utils.cpp | 8 +- src/Analyzer/Utils.h | 7 +- src/Interpreters/JoinUtils.h | 15 ++++ src/Planner/PlannerJoinTree.cpp | 75 +++++++++---------- src/Planner/PlannerJoins.cpp | 32 ++++---- src/Planner/PlannerJoins.h | 6 +- src/Processors/Transforms/FilterTransform.cpp | 2 +- 8 files changed, 83 insertions(+), 75 deletions(-) diff --git a/docs/en/sql-reference/statements/select/join.md b/docs/en/sql-reference/statements/select/join.md index b228f7025c4..679db789f74 100644 --- a/docs/en/sql-reference/statements/select/join.md +++ b/docs/en/sql-reference/statements/select/join.md @@ -173,18 +173,7 @@ Result: └───┴────┴─────┘ ``` -## [experimental] Join with inequality conditions for columns from different tables - -:::note -This feature is experimental. To use it, set `allow_experimental_join_condition` to 1 in your configuration files or by using the `SET` command: - -```sql -SET allow_experimental_join_condition=1 -``` - -Otherwise, you'll get `INVALID_JOIN_ON_EXPRESSION`. - -::: +## Join with inequality conditions for columns from different tables Clickhouse currently supports `ALL/ANY/SEMI/ANTI INNER/LEFT/RIGHT/FULL JOIN` with inequality conditions in addition to equality conditions. The inequality conditions are supported only for `hash` and `grace_hash` join algorithms. The inequality conditions are not supported with `join_use_nulls`. diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index 9e86a309e00..7b2b92bebd7 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -600,15 +600,15 @@ void buildTableExpressionsStackImpl(const QueryTreeNodePtr & join_tree_node, Que } -QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & query_node, QueryTreeNodes * out_nearest_query) +QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & query_node, QueryTreeNodes * nearest_query) { QueryTreeNodes result; QueryTreeNodes nearest_query_tmp; - if (out_nearest_query == nullptr) - out_nearest_query = &nearest_query_tmp; + if (nearest_query == nullptr) + nearest_query = &nearest_query_tmp; - buildTableExpressionsStackImpl(query_node->as()->getJoinTree(), query_node, result, *out_nearest_query); + buildTableExpressionsStackImpl(query_node->as()->getJoinTree(), query_node, result, *nearest_query); return result; } diff --git a/src/Analyzer/Utils.h b/src/Analyzer/Utils.h index 340ae9967ce..a4372556e07 100644 --- a/src/Analyzer/Utils.h +++ b/src/Analyzer/Utils.h @@ -67,6 +67,9 @@ QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node, QueryTreeNodePtr extractLeftTableExpression(const QueryTreeNodePtr & join_tree_node); /** Build table expressions stack that consists from table, table function, query, union, join, array join from join tree. + * Parameters: + * query_node - query node that contains join tree. + * nearest_query - if not nullptr, then nearest query node will be stored in this variable. * * Example: SELECT * FROM t1 INNER JOIN t2 INNER JOIN t3. * Result table expressions stack: @@ -75,8 +78,10 @@ QueryTreeNodePtr extractLeftTableExpression(const QueryTreeNodePtr & join_tree_n * 3. t1 INNER JOIN t2 * 4. t2 * 5. t1 + * + * Nearest query is the query itself in this case. */ -QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & query_node, QueryTreeNodes * out_nearest_query = nullptr); +QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & query_node, QueryTreeNodes * nearest_query = nullptr); /** Assert that there are no function nodes with specified function name in node children. diff --git a/src/Interpreters/JoinUtils.h b/src/Interpreters/JoinUtils.h index f15ee2c2fb2..bfe8b628b24 100644 --- a/src/Interpreters/JoinUtils.h +++ b/src/Interpreters/JoinUtils.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -168,4 +169,18 @@ private: void setRightIndex(size_t right_pos, size_t result_position); }; +/// Call the same func twice - for left arguments and then right arguments +template +void forJoinSides(Func && func, std::tuple && left, std::tuple && right) { + std::apply([&](auto &&... args) + { + func(JoinTableSide::Left, std::forward(args)...); + }, std::forward>(left)); + + std::apply([&](auto &&... args) + { + func(JoinTableSide::Right, std::forward(args)...); + }, std::forward>(right)); +} + } diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 023250b7797..bcd7c2eae6b 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1291,12 +1291,13 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ planner_context); const auto & left_pre_filters = join_clauses_and_actions.join_clauses[0].getLeftFilterConditionNodes(); - if (!left_pre_filters.empty() && left_pre_filters.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected only one left pre-filter condition node. Actual [{}]", fmt::join(left_pre_filters | std::views::transform([](const auto & node) { return node->result_name; }), ", ")); - const auto & right_pre_filters = join_clauses_and_actions.join_clauses[0].getRightFilterConditionNodes(); - if (!right_pre_filters.empty() && right_pre_filters.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected only one right pre-filter condition node. Actual [{}]", fmt::join(right_pre_filters | std::views::transform([](const auto & node) { return node->result_name; }), ", ")); + forJoinSides([](JoinTableSide side, const auto & pre_filters) + { + if (!pre_filters.empty() && pre_filters.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected only one {} pre-filter condition node. Actual [{}]", + side, fmt::join(pre_filters | std::views::transform([](const auto & node) { return node->result_name; }), ", ")); + }, std::make_tuple(std::ref(left_pre_filters)), std::make_tuple(std::ref(right_pre_filters))); can_move_out_residuals = join_clauses_and_actions.join_clauses.size() == 1 && join_strictness == JoinStrictness::All @@ -1304,29 +1305,21 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ && (right_pre_filters.empty() || FilterStep::canUseType(right_pre_filters[0]->result_type)) && (left_pre_filters.empty() || FilterStep::canUseType(left_pre_filters[0]->result_type)); - join_clauses_and_actions.left_join_expressions_actions.appendInputsForUnusedColumns(left_plan.getCurrentHeader()); - appendSetsFromActionsDAG(join_clauses_and_actions.left_join_expressions_actions, left_join_tree_query_plan.useful_sets); + forJoinSides([&](JoinTableSide, ActionsDAG & join_expressions_actions, QueryPlan & plan, const auto & pre_filters) + { + join_expressions_actions.appendInputsForUnusedColumns(left_plan.getCurrentHeader()); + appendSetsFromActionsDAG(join_expressions_actions, left_join_tree_query_plan.useful_sets); - QueryPlanStepPtr left_join_expressions_actions_step; - if (can_move_out_residuals && !left_pre_filters.empty()) - left_join_expressions_actions_step = std::make_unique(left_plan.getCurrentHeader(), std::move(join_clauses_and_actions.left_join_expressions_actions), left_pre_filters[0]->result_name, false); - else - left_join_expressions_actions_step = std::make_unique(left_plan.getCurrentHeader(), std::move(join_clauses_and_actions.left_join_expressions_actions)); + QueryPlanStepPtr join_expressions_actions_step; + if (can_move_out_residuals && !left_pre_filters.empty()) + join_expressions_actions_step = std::make_unique(plan.getCurrentHeader(), std::move(join_expressions_actions), pre_filters[0]->result_name, false); + else + join_expressions_actions_step = std::make_unique(plan.getCurrentHeader(), std::move(join_expressions_actions)); - left_join_expressions_actions_step->setStepDescription("JOIN actions"); - left_plan.addStep(std::move(left_join_expressions_actions_step)); - - join_clauses_and_actions.right_join_expressions_actions.appendInputsForUnusedColumns(right_plan.getCurrentHeader()); - appendSetsFromActionsDAG(join_clauses_and_actions.right_join_expressions_actions, right_join_tree_query_plan.useful_sets); - - QueryPlanStepPtr right_join_expressions_actions_step; - if (can_move_out_residuals && !right_pre_filters.empty()) - right_join_expressions_actions_step = std::make_unique(right_plan.getCurrentHeader(), std::move(join_clauses_and_actions.right_join_expressions_actions), right_pre_filters[0]->result_name, false); - else - right_join_expressions_actions_step = std::make_unique(right_plan.getCurrentHeader(), std::move(join_clauses_and_actions.right_join_expressions_actions)); - - right_join_expressions_actions_step->setStepDescription("JOIN actions"); - right_plan.addStep(std::move(right_join_expressions_actions_step)); + join_expressions_actions_step->setStepDescription("JOIN actions"); + plan.addStep(std::move(join_expressions_actions_step)); + }, std::make_tuple(std::ref(join_clauses_and_actions.left_join_expressions_actions), std::ref(left_plan), std::ref(left_pre_filters)), + std::make_tuple(std::ref(join_clauses_and_actions.right_join_expressions_actions), std::ref(right_plan), std::ref(right_pre_filters))); } std::unordered_map left_plan_column_name_to_cast_type; @@ -1460,6 +1453,9 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ if (join_clause_key_nodes_size == 0 && !can_move_out_residuals) throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, "Cannot determine join keys in {}", join_node.formatASTForErrorMessage()); + /// If there are no keys, but only conditions that cannot be used as keys, then it is a cross join. + /// Example: SELECT * FROM t1 JOIN t2 ON t1.x > t2.y + /// Same as: SELECT * FROM t1 CROSS JOIN t2 WHERE t1.x > t2.y if (join_clause_key_nodes_size == 0 && can_move_out_residuals) { join_kind = JoinKind::Cross; @@ -1531,15 +1527,16 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ } } - if (!can_move_out_residuals && join_clauses_and_actions.mixed_join_expressions_actions) + if (!can_move_out_residuals && join_clauses_and_actions.residual_join_expressions_actions) { + /// Let join algorithm handle residual conditions ExpressionActionsPtr & mixed_join_expression = table_join->getMixedJoinExpression(); mixed_join_expression = std::make_shared( - std::move(*join_clauses_and_actions.mixed_join_expressions_actions), + std::move(*join_clauses_and_actions.residual_join_expressions_actions), ExpressionActionsSettings::fromContext(planner_context->getQueryContext())); appendSetsFromActionsDAG(mixed_join_expression->getActionsDAG(), left_join_tree_query_plan.useful_sets); - join_clauses_and_actions.mixed_join_expressions_actions.reset(); + join_clauses_and_actions.residual_join_expressions_actions.reset(); } } else if (join_node.isUsingJoinExpression()) @@ -1688,26 +1685,28 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ result_plan.unitePlans(std::move(join_step), {std::move(plans)}); } - if (join_clauses_and_actions.mixed_join_expressions_actions) + /// If residuals were not moved to JOIN algorithm, + /// we need to process add then as WHERE condition after JOIN + if (join_clauses_and_actions.residual_join_expressions_actions) { - auto outputs = join_clauses_and_actions.mixed_join_expressions_actions->getOutputs(); + auto outputs = join_clauses_and_actions.residual_join_expressions_actions->getOutputs(); if (outputs.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 1 output column in JOIN actions, got {}", - join_clauses_and_actions.mixed_join_expressions_actions->dumpDAG()); + join_clauses_and_actions.residual_join_expressions_actions->dumpDAG()); - join_clauses_and_actions.mixed_join_expressions_actions->appendInputsForUnusedColumns(result_plan.getCurrentHeader()); - for (const auto * input_node : join_clauses_and_actions.mixed_join_expressions_actions->getInputs()) - join_clauses_and_actions.mixed_join_expressions_actions->addOrReplaceInOutputs(*input_node); + join_clauses_and_actions.residual_join_expressions_actions->appendInputsForUnusedColumns(result_plan.getCurrentHeader()); + for (const auto * input_node : join_clauses_and_actions.residual_join_expressions_actions->getInputs()) + join_clauses_and_actions.residual_join_expressions_actions->addOrReplaceInOutputs(*input_node); - appendSetsFromActionsDAG(*join_clauses_and_actions.mixed_join_expressions_actions, left_join_tree_query_plan.useful_sets); + appendSetsFromActionsDAG(*join_clauses_and_actions.residual_join_expressions_actions, left_join_tree_query_plan.useful_sets); auto filter_step = std::make_unique(result_plan.getCurrentHeader(), - std::move(*join_clauses_and_actions.mixed_join_expressions_actions), + std::move(*join_clauses_and_actions.residual_join_expressions_actions), outputs[0]->result_name, /* remove_column = */ false); /// Unused columns will be removed by next step filter_step->setStepDescription("Residual JOIN filter"); result_plan.addStep(std::move(filter_step)); - join_clauses_and_actions.mixed_join_expressions_actions.reset(); + join_clauses_and_actions.residual_join_expressions_actions.reset(); } ActionsDAG drop_unused_columns_after_join_actions_dag(result_plan.getCurrentHeader().getColumnsWithTypeAndName()); diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index 58d3dd9a2f0..bc2d230504b 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -378,8 +378,8 @@ void buildJoinClause( { throw Exception( ErrorCodes::INVALID_JOIN_ON_EXPRESSION, - "OUTER JOIN ON expression {} contains column from left and right table, which is not supported with join_use_nulls", - join_expression->formatASTForErrorMessage()); + "{} JOIN ON expression {} contains column from left and right table, which is not supported with `join_use_nulls`", + toString(join_node.getKind()), join_expression->formatASTForErrorMessage()); } } } @@ -474,7 +474,7 @@ JoinClausesAndActions buildJoinClausesAndActions( join_right_table_expressions, join_node, result.join_clauses.back()); - has_residual_filters |= !result.join_clauses.back().getMixedFilterConditionNodes().empty(); + has_residual_filters |= !result.join_clauses.back().getResidualFilterConditionNodes().empty(); } } else @@ -491,7 +491,7 @@ JoinClausesAndActions buildJoinClausesAndActions( join_right_table_expressions, join_node, result.join_clauses.back()); - has_residual_filters |= !result.join_clauses.back().getMixedFilterConditionNodes().empty(); + has_residual_filters |= !result.join_clauses.back().getResidualFilterConditionNodes().empty(); } auto and_function = FunctionFactory::instance().get("and", planner_context->getQueryContext()); @@ -618,29 +618,29 @@ JoinClausesAndActions buildJoinClausesAndActions( /// So, for each column, we recalculate the value of the whole expression from JOIN ON to check if rows should be joined. if (result.join_clauses.size() > 1) { - ActionsDAG mixed_join_expressions_actions(result_relation_columns); + ActionsDAG residual_join_expressions_actions(result_relation_columns); PlannerActionsVisitor join_expression_visitor(planner_context); - auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(mixed_join_expressions_actions, join_expression); + auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(residual_join_expressions_actions, join_expression); if (join_expression_dag_node_raw_pointers.size() != 1) throw Exception( ErrorCodes::LOGICAL_ERROR, "JOIN {} ON clause contains multiple expressions", join_node.formatASTForErrorMessage()); - mixed_join_expressions_actions.addOrReplaceInOutputs(*join_expression_dag_node_raw_pointers[0]); + residual_join_expressions_actions.addOrReplaceInOutputs(*join_expression_dag_node_raw_pointers[0]); Names required_names{join_expression_dag_node_raw_pointers[0]->result_name}; - mixed_join_expressions_actions.removeUnusedActions(required_names); - result.mixed_join_expressions_actions = std::move(mixed_join_expressions_actions); + residual_join_expressions_actions.removeUnusedActions(required_names); + result.residual_join_expressions_actions = std::move(residual_join_expressions_actions); } else { const auto & join_clause = result.join_clauses.front(); - const auto & mixed_filter_condition_nodes = join_clause.getMixedFilterConditionNodes(); - auto mixed_join_expressions_actions = ActionsDAG::buildFilterActionsDAG(mixed_filter_condition_nodes, {}, true); - result.mixed_join_expressions_actions = std::move(mixed_join_expressions_actions); + const auto & residual_filter_condition_nodes = join_clause.getResidualFilterConditionNodes(); + auto residual_join_expressions_actions = ActionsDAG::buildFilterActionsDAG(residual_filter_condition_nodes, {}, true); + result.residual_join_expressions_actions = std::move(residual_join_expressions_actions); } - auto outputs = result.mixed_join_expressions_actions->getOutputs(); + auto outputs = result.residual_join_expressions_actions->getOutputs(); if (outputs.size() != 1) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Only one output is expected, got: {}", result.mixed_join_expressions_actions->dumpDAG()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Only one output is expected, got: {}", result.residual_join_expressions_actions->dumpDAG()); } auto output_type = removeNullable(outputs[0]->result_type); WhichDataType which_type(output_type); @@ -648,8 +648,8 @@ JoinClausesAndActions buildJoinClausesAndActions( { DataTypePtr uint8_ty = std::make_shared(); auto true_col = ColumnWithTypeAndName(uint8_ty->createColumnConst(1, 1), uint8_ty, "true"); - const auto * true_node = &result.mixed_join_expressions_actions->addColumn(true_col); - result.mixed_join_expressions_actions = ActionsDAG::buildFilterActionsDAG({outputs[0], true_node}); + const auto * true_node = &result.residual_join_expressions_actions->addColumn(true_col); + result.residual_join_expressions_actions = ActionsDAG::buildFilterActionsDAG({outputs[0], true_node}); } } diff --git a/src/Planner/PlannerJoins.h b/src/Planner/PlannerJoins.h index a95233674e7..dea3280b80c 100644 --- a/src/Planner/PlannerJoins.h +++ b/src/Planner/PlannerJoins.h @@ -140,7 +140,7 @@ public: return right_filter_condition_nodes; } - ActionsDAG::NodeRawConstPtrs & getMixedFilterConditionNodes() + ActionsDAG::NodeRawConstPtrs & getResidualFilterConditionNodes() { return residual_filter_condition_nodes; } @@ -150,7 +150,7 @@ public: residual_filter_condition_nodes.push_back(condition_node); } - const ActionsDAG::NodeRawConstPtrs & getMixedFilterConditionNodes() const + const ActionsDAG::NodeRawConstPtrs & getResidualFilterConditionNodes() const { return residual_filter_condition_nodes; } @@ -190,7 +190,7 @@ struct JoinClausesAndActions ActionsDAG right_join_expressions_actions; /// Originally used for inequal join. it's the total join expression. /// If there is no inequal join conditions, it's null. - std::optional mixed_join_expressions_actions; + std::optional residual_join_expressions_actions; }; /** Calculate join clauses and actions for JOIN ON section. diff --git a/src/Processors/Transforms/FilterTransform.cpp b/src/Processors/Transforms/FilterTransform.cpp index e709a1be127..756a42d4088 100644 --- a/src/Processors/Transforms/FilterTransform.cpp +++ b/src/Processors/Transforms/FilterTransform.cpp @@ -17,7 +17,7 @@ namespace ErrorCodes bool FilterTransform::canUseType(const DataTypePtr & filter_type) { - return filter_type->onlyNull() || isUInt8(removeNullable(removeLowCardinality(filter_type))); + return filter_type->onlyNull() || isUInt8(removeLowCardinalityAndNullable(filter_type)); } Block FilterTransform::transformHeader( From f7122a875349f563973ee32a818a8deb5f1f132b Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 4 Nov 2024 10:43:19 +0000 Subject: [PATCH 14/59] Do not collect nearest_query in buildTableExpressionsStack --- src/Analyzer/Resolve/QueryAnalyzer.cpp | 2 +- src/Analyzer/Utils.cpp | 28 +++++++++----------------- src/Analyzer/Utils.h | 8 +------- src/Planner/PlannerJoinTree.cpp | 9 ++------- 4 files changed, 13 insertions(+), 34 deletions(-) diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 23e22ad55e7..cb3087af707 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -1918,7 +1918,7 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher( * expressions that have same names as columns in USING clause must be skipped. */ - auto table_expressions_stack = buildTableExpressionsStack(nearest_query_scope->scope_node); + auto table_expressions_stack = buildTableExpressionsStack(nearest_query_scope_query_node->getJoinTree()); std::vector table_expressions_column_nodes_with_names_stack; std::unordered_set table_expression_column_names_to_skip; diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index 7b2b92bebd7..c73400532ba 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -552,40 +552,35 @@ QueryTreeNodePtr extractLeftTableExpression(const QueryTreeNodePtr & join_tree_n namespace { -void buildTableExpressionsStackImpl(const QueryTreeNodePtr & join_tree_node, QueryTreeNodePtr nearest_query_node, QueryTreeNodes & result, QueryTreeNodes & nearest_query) +void buildTableExpressionsStackImpl(const QueryTreeNodePtr & join_tree_node, QueryTreeNodes & result) { auto node_type = join_tree_node->getNodeType(); + switch (node_type) { - case QueryTreeNodeType::QUERY: - { - nearest_query_node = join_tree_node; - [[fallthrough]]; - } case QueryTreeNodeType::TABLE: [[fallthrough]]; + case QueryTreeNodeType::QUERY: + [[fallthrough]]; case QueryTreeNodeType::UNION: [[fallthrough]]; case QueryTreeNodeType::TABLE_FUNCTION: { - nearest_query.push_back(nearest_query_node); result.push_back(join_tree_node); break; } case QueryTreeNodeType::ARRAY_JOIN: { auto & array_join_node = join_tree_node->as(); - buildTableExpressionsStackImpl(array_join_node.getTableExpression(), nearest_query_node, result, nearest_query); - nearest_query.push_back(nearest_query_node); + buildTableExpressionsStackImpl(array_join_node.getTableExpression(), result); result.push_back(join_tree_node); break; } case QueryTreeNodeType::JOIN: { auto & join_node = join_tree_node->as(); - buildTableExpressionsStackImpl(join_node.getLeftTableExpression(), nearest_query_node, result, nearest_query); - buildTableExpressionsStackImpl(join_node.getRightTableExpression(), nearest_query_node, result, nearest_query); - nearest_query.push_back(nearest_query_node); + buildTableExpressionsStackImpl(join_node.getLeftTableExpression(), result); + buildTableExpressionsStackImpl(join_node.getRightTableExpression(), result); result.push_back(join_tree_node); break; } @@ -600,15 +595,10 @@ void buildTableExpressionsStackImpl(const QueryTreeNodePtr & join_tree_node, Que } -QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & query_node, QueryTreeNodes * nearest_query) +QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & join_tree_node) { QueryTreeNodes result; - - QueryTreeNodes nearest_query_tmp; - if (nearest_query == nullptr) - nearest_query = &nearest_query_tmp; - - buildTableExpressionsStackImpl(query_node->as()->getJoinTree(), query_node, result, *nearest_query); + buildTableExpressionsStackImpl(join_tree_node, result); return result; } diff --git a/src/Analyzer/Utils.h b/src/Analyzer/Utils.h index a4372556e07..f2e2c500384 100644 --- a/src/Analyzer/Utils.h +++ b/src/Analyzer/Utils.h @@ -67,9 +67,6 @@ QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node, QueryTreeNodePtr extractLeftTableExpression(const QueryTreeNodePtr & join_tree_node); /** Build table expressions stack that consists from table, table function, query, union, join, array join from join tree. - * Parameters: - * query_node - query node that contains join tree. - * nearest_query - if not nullptr, then nearest query node will be stored in this variable. * * Example: SELECT * FROM t1 INNER JOIN t2 INNER JOIN t3. * Result table expressions stack: @@ -78,11 +75,8 @@ QueryTreeNodePtr extractLeftTableExpression(const QueryTreeNodePtr & join_tree_n * 3. t1 INNER JOIN t2 * 4. t2 * 5. t1 - * - * Nearest query is the query itself in this case. */ -QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & query_node, QueryTreeNodes * nearest_query = nullptr); - +QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & join_tree_node); /** Assert that there are no function nodes with specified function name in node children. * Do not visit subqueries. diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index bcd7c2eae6b..e2168cacc07 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1245,7 +1245,6 @@ void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextP } JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_expression, - const QueryTreeNodePtr & nearest_query_node, JoinTreeQueryPlan left_join_tree_query_plan, JoinTreeQueryPlan right_join_tree_query_plan, const ColumnIdentifierSet & outer_scope_columns, @@ -1283,11 +1282,9 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ if (!join_constant && join_node.isOnJoinExpression()) { - UNUSED(nearest_query_node); join_clauses_and_actions = buildJoinClausesAndActions(left_plan_output_columns, right_plan_output_columns, join_table_expression, - // nearest_query_node, planner_context); const auto & left_pre_filters = join_clauses_and_actions.join_clauses[0].getLeftFilterConditionNodes(); @@ -1871,8 +1868,7 @@ JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node, const ColumnIdentifierSet & outer_scope_columns, PlannerContextPtr & planner_context) { - QueryTreeNodes table_expressions_stack_queries; - auto table_expressions_stack = buildTableExpressionsStack(query_node, &table_expressions_stack_queries); + auto table_expressions_stack = buildTableExpressionsStack(query_node->as().getJoinTree()); size_t table_expressions_stack_size = table_expressions_stack.size(); bool is_single_table_expression = table_expressions_stack_size == 1; @@ -1933,7 +1929,6 @@ JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node, for (size_t i = 0; i < table_expressions_stack_size; ++i) { const auto & table_expression = table_expressions_stack[i]; - const auto & nearest_query = table_expressions_stack_queries[i]; auto table_expression_node_type = table_expression->getNodeType(); if (table_expression_node_type == QueryTreeNodeType::ARRAY_JOIN) @@ -1963,7 +1958,7 @@ JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node, query_plans_stack.pop_back(); query_plans_stack.push_back(buildQueryPlanForJoinNode( - table_expression, nearest_query, + table_expression, std::move(left_query_plan), std::move(right_query_plan), table_expressions_outer_scope_columns[i], planner_context)); } From d17d4c6d5c3edd6c2359df5f9807881ae85ae401 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 4 Nov 2024 10:55:14 +0000 Subject: [PATCH 15/59] fix stylecheck --- src/Interpreters/JoinUtils.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/JoinUtils.h b/src/Interpreters/JoinUtils.h index bfe8b628b24..77ecf38fa6b 100644 --- a/src/Interpreters/JoinUtils.h +++ b/src/Interpreters/JoinUtils.h @@ -171,7 +171,8 @@ private: /// Call the same func twice - for left arguments and then right arguments template -void forJoinSides(Func && func, std::tuple && left, std::tuple && right) { +void forJoinSides(Func && func, std::tuple && left, std::tuple && right) +{ std::apply([&](auto &&... args) { func(JoinTableSide::Left, std::forward(args)...); From 43486bd634b636555feff0ca17d7dbd04bcf3856 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 5 Nov 2024 16:35:40 +0000 Subject: [PATCH 16/59] simple --- src/Interpreters/JoinUtils.h | 15 --------------- src/Planner/PlannerJoinTree.cpp | 13 ++++++++----- 2 files changed, 8 insertions(+), 20 deletions(-) diff --git a/src/Interpreters/JoinUtils.h b/src/Interpreters/JoinUtils.h index 77ecf38fa6b..8c1c60584d8 100644 --- a/src/Interpreters/JoinUtils.h +++ b/src/Interpreters/JoinUtils.h @@ -169,19 +169,4 @@ private: void setRightIndex(size_t right_pos, size_t result_position); }; -/// Call the same func twice - for left arguments and then right arguments -template -void forJoinSides(Func && func, std::tuple && left, std::tuple && right) -{ - std::apply([&](auto &&... args) - { - func(JoinTableSide::Left, std::forward(args)...); - }, std::forward>(left)); - - std::apply([&](auto &&... args) - { - func(JoinTableSide::Right, std::forward(args)...); - }, std::forward>(right)); -} - } diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 5b66d831fc7..9a0408b52f6 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1339,12 +1339,14 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ const auto & left_pre_filters = join_clauses_and_actions.join_clauses[0].getLeftFilterConditionNodes(); const auto & right_pre_filters = join_clauses_and_actions.join_clauses[0].getRightFilterConditionNodes(); - forJoinSides([](JoinTableSide side, const auto & pre_filters) + auto check_pre_filter = [](JoinTableSide side, const auto & pre_filters) { if (!pre_filters.empty() && pre_filters.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected only one {} pre-filter condition node. Actual [{}]", side, fmt::join(pre_filters | std::views::transform([](const auto & node) { return node->result_name; }), ", ")); - }, std::make_tuple(std::ref(left_pre_filters)), std::make_tuple(std::ref(right_pre_filters))); + }; + check_pre_filter(left_pre_filters); + check_pre_filter(right_pre_filters); can_move_out_residuals = join_clauses_and_actions.join_clauses.size() == 1 && join_strictness == JoinStrictness::All @@ -1352,7 +1354,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ && (right_pre_filters.empty() || FilterStep::canUseType(right_pre_filters[0]->result_type)) && (left_pre_filters.empty() || FilterStep::canUseType(left_pre_filters[0]->result_type)); - forJoinSides([&](JoinTableSide, ActionsDAG & join_expressions_actions, QueryPlan & plan, const auto & pre_filters) + auto add_pre_filter = [&](JoinTableSide, ActionsDAG & join_expressions_actions, QueryPlan & plan, const auto & pre_filters) { join_expressions_actions.appendInputsForUnusedColumns(left_plan.getCurrentHeader()); appendSetsFromActionsDAG(join_expressions_actions, left_join_tree_query_plan.useful_sets); @@ -1365,8 +1367,9 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ join_expressions_actions_step->setStepDescription("JOIN actions"); plan.addStep(std::move(join_expressions_actions_step)); - }, std::make_tuple(std::ref(join_clauses_and_actions.left_join_expressions_actions), std::ref(left_plan), std::ref(left_pre_filters)), - std::make_tuple(std::ref(join_clauses_and_actions.right_join_expressions_actions), std::ref(right_plan), std::ref(right_pre_filters))); + }; + add_pre_filter(join_clauses_and_actions.left_join_expressions_actions, left_plan, left_pre_filters); + add_pre_filter(join_clauses_and_actions.right_join_expressions_actions, right_plan, right_pre_filters); } std::unordered_map left_plan_column_name_to_cast_type; From de3f05291212bd40f3e952738731fac66e52b76a Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 5 Nov 2024 16:53:31 +0000 Subject: [PATCH 17/59] fix build --- src/Planner/PlannerJoinTree.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 9a0408b52f6..60d5ca8a8ff 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1345,8 +1345,8 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected only one {} pre-filter condition node. Actual [{}]", side, fmt::join(pre_filters | std::views::transform([](const auto & node) { return node->result_name; }), ", ")); }; - check_pre_filter(left_pre_filters); - check_pre_filter(right_pre_filters); + check_pre_filter(JoinTableSide::Left, left_pre_filters); + check_pre_filter(JoinTableSide::Right, right_pre_filters); can_move_out_residuals = join_clauses_and_actions.join_clauses.size() == 1 && join_strictness == JoinStrictness::All @@ -1354,7 +1354,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ && (right_pre_filters.empty() || FilterStep::canUseType(right_pre_filters[0]->result_type)) && (left_pre_filters.empty() || FilterStep::canUseType(left_pre_filters[0]->result_type)); - auto add_pre_filter = [&](JoinTableSide, ActionsDAG & join_expressions_actions, QueryPlan & plan, const auto & pre_filters) + auto add_pre_filter = [&](ActionsDAG & join_expressions_actions, QueryPlan & plan, const auto & pre_filters) { join_expressions_actions.appendInputsForUnusedColumns(left_plan.getCurrentHeader()); appendSetsFromActionsDAG(join_expressions_actions, left_join_tree_query_plan.useful_sets); From 19a46944b7d44e9efff0c5524a4e1546590db500 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 6 Nov 2024 10:46:35 +0000 Subject: [PATCH 18/59] fix --- src/Planner/PlannerJoinTree.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 60d5ca8a8ff..c6478e14be8 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1354,13 +1354,13 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ && (right_pre_filters.empty() || FilterStep::canUseType(right_pre_filters[0]->result_type)) && (left_pre_filters.empty() || FilterStep::canUseType(left_pre_filters[0]->result_type)); - auto add_pre_filter = [&](ActionsDAG & join_expressions_actions, QueryPlan & plan, const auto & pre_filters) + auto add_pre_filter = [can_move_out_residuals](ActionsDAG & join_expressions_actions, QueryPlan & plan, UsefulSets & useful_sets, const auto & pre_filters) { - join_expressions_actions.appendInputsForUnusedColumns(left_plan.getCurrentHeader()); - appendSetsFromActionsDAG(join_expressions_actions, left_join_tree_query_plan.useful_sets); + join_expressions_actions.appendInputsForUnusedColumns(plan.getCurrentHeader()); + appendSetsFromActionsDAG(join_expressions_actions, useful_sets); QueryPlanStepPtr join_expressions_actions_step; - if (can_move_out_residuals && !left_pre_filters.empty()) + if (can_move_out_residuals && !pre_filters.empty()) join_expressions_actions_step = std::make_unique(plan.getCurrentHeader(), std::move(join_expressions_actions), pre_filters[0]->result_name, false); else join_expressions_actions_step = std::make_unique(plan.getCurrentHeader(), std::move(join_expressions_actions)); @@ -1368,8 +1368,8 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ join_expressions_actions_step->setStepDescription("JOIN actions"); plan.addStep(std::move(join_expressions_actions_step)); }; - add_pre_filter(join_clauses_and_actions.left_join_expressions_actions, left_plan, left_pre_filters); - add_pre_filter(join_clauses_and_actions.right_join_expressions_actions, right_plan, right_pre_filters); + add_pre_filter(join_clauses_and_actions.left_join_expressions_actions, left_plan, left_join_tree_query_plan.useful_sets, left_pre_filters); + add_pre_filter(join_clauses_and_actions.right_join_expressions_actions, right_plan, right_join_tree_query_plan.useful_sets, right_pre_filters); } std::unordered_map left_plan_column_name_to_cast_type; From 807f3e7d6195cc70d692c3d8eef48370fffe7cb2 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 7 Nov 2024 11:29:21 +0000 Subject: [PATCH 19/59] fix test --- .../01881_join_on_conditions_hash.sql.j2 | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 index 1b4018a1ecd..cdab509db90 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 @@ -73,26 +73,23 @@ SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id + 2; -- { serverE SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.key; -- { serverError 43, 403 } SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverError 43, 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 } - +SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id; SELECT * FROM t1 JOIN t2_nullable as t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)) ORDER BY ALL SETTINGS enable_analyzer = 1; SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id ORDER BY ALL SETTINGS enable_analyzer = 1; SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id ORDER BY ALL SETTINGS enable_analyzer = 1; -SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id ORDER BY ALL; --- non-equi condition containing columns from different tables doesn't supported yet -SELECT * FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t2.key); -- { serverError 403 } SELECT '--'; SELECT t1.*, t2.* FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND length(t1.key2) == length(t2.key2) AND t1.key != '333'; +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t22.key == t22.key2 OR t1.id == t22.idd and t1.id == t22.id ORDER BY ALL; +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t1.id == t22.id OR t1.id == t22.idd and t22.key == t22.key2 ORDER BY ALL; +SELECT 't22', * FROM t1 JOIN t22 ON t22.key == t22.key2 and t1.id == t22.idd OR t1.id == t22.id and t1.id == t22.idd ORDER BY ALL; +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.id and t1.id == t22.idd OR t22.key == t22.key2 and t1.id == t22.idd ORDER BY ALL; SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t22.key == t22.key2 OR t1.id == t22.id) ORDER BY ALL SETTINGS enable_analyzer = 1; SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t1.id == t22.id OR t22.key == t22.key2) ORDER BY ALL SETTINGS enable_analyzer = 1; SELECT 't22', * FROM t1 JOIN t22 ON (t22.key == t22.key2 OR t1.id == t22.id) and t1.id == t22.idd ORDER BY ALL SETTINGS enable_analyzer = 1; SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and t1.id == t22.idd ORDER BY ALL SETTINGS enable_analyzer = 1; SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and (t1.id == t22.idd AND (t1.key2 = 'a1' OR t1.key2 = 'a2' OR t1.key2 = 'a3' OR t1.key2 = 'a4' OR t1.key2 = 'a5' OR t1.key2 = 'a6' OR t1.key2 = 'a7' OR t1.key2 = 'a8' OR t1.key2 = 'a9' OR t1.key2 = 'a10' OR t1.key2 = 'a11' OR t1.key2 = 'a12' OR t1.key2 = 'a13' OR t1.key2 = 'a14' OR t1.key2 = 'a15' OR t1.key2 = 'a16' OR t1.key2 = 'a17' OR t1.key2 = 'a18' OR t1.key2 = 'a19' OR t1.key2 = '111')) ORDER BY ALL SETTINGS enable_analyzer = 1; -SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t22.key == t22.key2 OR t1.id == t22.idd and t1.id == t22.id ORDER BY ALL; -SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t1.id == t22.id OR t1.id == t22.idd and t22.key == t22.key2 ORDER BY ALL; -SELECT 't22', * FROM t1 JOIN t22 ON t22.key == t22.key2 and t1.id == t22.idd OR t1.id == t22.id and t1.id == t22.idd ORDER BY ALL; -SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.id and t1.id == t22.idd OR t22.key == t22.key2 and t1.id == t22.idd ORDER BY ALL; {% endfor -%} 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 20/59] 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 c224366513f68ea99dd7707e232dcc7ae2d711c8 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 18 Nov 2024 16:57:09 +0000 Subject: [PATCH 21/59] fix --- src/Planner/PlannerJoinTree.cpp | 49 +++++++++++++++++++-------------- 1 file changed, 28 insertions(+), 21 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index c6478e14be8..268cbda8924 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1620,22 +1620,6 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ table_join->setInputColumns(columns_from_left_table, columns_from_right_table); - for (auto & column_from_joined_table : columns_from_left_table) - { - /// Add columns to output only if they are presented in outer scope, otherwise they can be dropped - if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(column_from_joined_table.name) && - outer_scope_columns.contains(column_from_joined_table.name)) - table_join->setUsedColumn(column_from_joined_table, JoinTableSide::Left); - } - - for (auto & column_from_joined_table : columns_from_right_table) - { - /// Add columns to output only if they are presented in outer scope, otherwise they can be dropped - if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(column_from_joined_table.name) && - outer_scope_columns.contains(column_from_joined_table.name)) - table_join->setUsedColumn(column_from_joined_table, JoinTableSide::Right); - } - if (table_join->getOutputColumns(JoinTableSide::Left).empty() && table_join->getOutputColumns(JoinTableSide::Right).empty()) { if (!columns_from_left_table.empty()) @@ -1735,13 +1719,36 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ auto join_pipeline_type = join_algorithm->pipelineType(); - ColumnIdentifierSet outer_scope_columns_nonempty; - if (outer_scope_columns.empty()) + ColumnIdentifierSet required_columns_after_join = outer_scope_columns; + + if (join_clauses_and_actions.residual_join_expressions_actions) + { + for (const auto * input : join_clauses_and_actions.residual_join_expressions_actions->getInputs()) + required_columns_after_join.insert(input->result_name); + } + + if (required_columns_after_join.empty()) { if (left_header.columns() > 1) - outer_scope_columns_nonempty.insert(left_header.getByPosition(0).name); + required_columns_after_join.insert(left_header.getByPosition(0).name); else if (right_header.columns() > 1) - outer_scope_columns_nonempty.insert(right_header.getByPosition(0).name); + required_columns_after_join.insert(right_header.getByPosition(0).name); + } + + for (auto & column_from_joined_table : columns_from_left_table) + { + /// Add columns to output only if they are presented in outer scope, otherwise they can be dropped + if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(column_from_joined_table.name) && + required_columns_after_join.contains(column_from_joined_table.name)) + table_join->setUsedColumn(column_from_joined_table, JoinTableSide::Left); + } + + for (auto & column_from_joined_table : columns_from_right_table) + { + /// Add columns to output only if they are presented in outer scope, otherwise they can be dropped + if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(column_from_joined_table.name) && + required_columns_after_join.contains(column_from_joined_table.name)) + table_join->setUsedColumn(column_from_joined_table, JoinTableSide::Right); } auto join_step = std::make_unique( @@ -1750,7 +1757,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ std::move(join_algorithm), settings[Setting::max_block_size], settings[Setting::max_threads], - outer_scope_columns.empty() ? outer_scope_columns_nonempty : outer_scope_columns, + required_columns_after_join, false /*optimize_read_in_order*/, true /*optimize_skip_unused_shards*/); join_step->inner_table_selection_mode = settings[Setting::query_plan_join_inner_table_selection]; From e9160ea86011f4f5599a878b8cfb1b5018979b3a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 20 Nov 2024 13:00:28 +0000 Subject: [PATCH 22/59] 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 faf05a78079e654a0a5383e222cb5cab0bdb58a8 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 21 Nov 2024 14:36:55 +0000 Subject: [PATCH 23/59] 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 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 24/59] 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 25/59] 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 92a79b750e98d177d9b9327341001a0d72dd7f81 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 27 Nov 2024 23:50:23 +0100 Subject: [PATCH 26/59] 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/59] 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/59] 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 83f5e82a11713da1cc4d287d55965e28f1622667 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 28 Nov 2024 19:06:01 +0100 Subject: [PATCH 29/59] Analyzer: Proper fix for distributed ARRAY JOINs --- src/Analyzer/ArrayJoinNode.cpp | 5 +- src/Analyzer/Resolve/IdentifierResolver.cpp | 13 +-- src/Analyzer/Resolve/QueryAnalysisPass.cpp | 4 +- src/Analyzer/Resolve/QueryAnalyzer.cpp | 22 ++-- src/Analyzer/Resolve/ScopeAliases.h | 4 - ...cpp => createUniqueAliasesIfNecessary.cpp} | 108 ++++++++++++++++-- src/Analyzer/createUniqueAliasesIfNecessary.h | 14 +++ src/Analyzer/createUniqueTableAliases.h | 6 +- src/Storages/buildQueryTreeForShard.cpp | 4 +- .../03277_analyzer_array_join_fix.reference | 0 .../03277_analyzer_array_join_fix.sql | 21 ++++ 11 files changed, 155 insertions(+), 46 deletions(-) rename src/Analyzer/{createUniqueTableAliases.cpp => createUniqueAliasesIfNecessary.cpp} (57%) create mode 100644 src/Analyzer/createUniqueAliasesIfNecessary.h create mode 100644 tests/queries/0_stateless/03277_analyzer_array_join_fix.reference create mode 100644 tests/queries/0_stateless/03277_analyzer_array_join_fix.sql diff --git a/src/Analyzer/ArrayJoinNode.cpp b/src/Analyzer/ArrayJoinNode.cpp index 0cfb5d80b2a..b37f35fd84c 100644 --- a/src/Analyzer/ArrayJoinNode.cpp +++ b/src/Analyzer/ArrayJoinNode.cpp @@ -66,10 +66,7 @@ ASTPtr ArrayJoinNode::toASTImpl(const ConvertToASTOptions & options) const auto * column_node = array_join_expression->as(); if (column_node && column_node->getExpression()) { - if (const auto * function_node = column_node->getExpression()->as(); function_node && function_node->getFunctionName() == "nested") - array_join_expression_ast = array_join_expression->toAST(options); - else - array_join_expression_ast = column_node->getExpression()->toAST(options); + array_join_expression_ast = column_node->getExpression()->toAST(options); } else array_join_expression_ast = array_join_expression->toAST(options); diff --git a/src/Analyzer/Resolve/IdentifierResolver.cpp b/src/Analyzer/Resolve/IdentifierResolver.cpp index 317a02a60f2..b81db64dbb4 100644 --- a/src/Analyzer/Resolve/IdentifierResolver.cpp +++ b/src/Analyzer/Resolve/IdentifierResolver.cpp @@ -565,7 +565,7 @@ QueryTreeNodePtr IdentifierResolver::tryResolveIdentifierFromExpressionArguments bool IdentifierResolver::tryBindIdentifierToAliases(const IdentifierLookup & identifier_lookup, const IdentifierResolveScope & scope) { - return scope.aliases.find(identifier_lookup, ScopeAliases::FindOption::FIRST_NAME) != nullptr || scope.aliases.array_join_aliases.contains(identifier_lookup.identifier.front()); + return scope.aliases.find(identifier_lookup, ScopeAliases::FindOption::FIRST_NAME) != nullptr; } /** Resolve identifier from table columns. @@ -1415,9 +1415,6 @@ QueryTreeNodePtr IdentifierResolver::tryResolveIdentifierFromArrayJoin(const Ide IdentifierView identifier_view(identifier_lookup.identifier); - if (identifier_view.isCompound() && from_array_join_node.hasAlias() && identifier_view.front() == from_array_join_node.getAlias()) - identifier_view.popFirst(); - const auto & alias_or_name = array_join_column_expression_typed.hasAlias() ? array_join_column_expression_typed.getAlias() : array_join_column_expression_typed.getColumnName(); @@ -1429,18 +1426,16 @@ QueryTreeNodePtr IdentifierResolver::tryResolveIdentifierFromArrayJoin(const Ide else continue; + auto array_join_column = std::make_shared(array_join_column_expression_typed.getColumn(), + array_join_column_expression_typed.getColumnSource()); if (identifier_view.empty()) - { - auto array_join_column = std::make_shared(array_join_column_expression_typed.getColumn(), - array_join_column_expression_typed.getColumnSource()); return array_join_column; - } /// Resolve subcolumns. Example : SELECT x.y.z FROM tab ARRAY JOIN arr AS x auto compound_expr = tryResolveIdentifierFromCompoundExpression( identifier_lookup.identifier, identifier_lookup.identifier.getPartsSize() - identifier_view.getPartsSize() /*identifier_bind_size*/, - array_join_column_expression, + array_join_column, {} /* compound_expression_source */, scope, true /* can_be_not_found */); diff --git a/src/Analyzer/Resolve/QueryAnalysisPass.cpp b/src/Analyzer/Resolve/QueryAnalysisPass.cpp index 36c747555fc..896154c63b0 100644 --- a/src/Analyzer/Resolve/QueryAnalysisPass.cpp +++ b/src/Analyzer/Resolve/QueryAnalysisPass.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include namespace DB { @@ -16,7 +16,7 @@ void QueryAnalysisPass::run(QueryTreeNodePtr & query_tree_node, ContextPtr conte { QueryAnalyzer analyzer(only_analyze); analyzer.resolve(query_tree_node, table_expression, context); - createUniqueTableAliases(query_tree_node, table_expression, context); + createUniqueAliasesIfNecessary(query_tree_node, context); } } diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index d118cb281ae..b6a90ca1fe5 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -4977,6 +4977,16 @@ void QueryAnalyzer::resolveArrayJoin(QueryTreeNodePtr & array_join_node, Identif throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "ARRAY JOIN requires at least single expression"); + /// Register expression aliases in the scope + for (const auto & elem : array_join_nodes) + { + for (auto & child : elem->getChildren()) + { + if (child) + expressions_visitor.visit(child); + } + } + std::vector array_join_column_expressions; array_join_column_expressions.reserve(array_join_nodes_size); @@ -4984,18 +4994,6 @@ void QueryAnalyzer::resolveArrayJoin(QueryTreeNodePtr & array_join_node, Identif { auto array_join_expression_alias = array_join_expression->getAlias(); - for (const auto & elem : array_join_nodes) - { - if (elem->hasAlias()) - scope.aliases.array_join_aliases.insert(elem->getAlias()); - - for (auto & child : elem->getChildren()) - { - if (child) - expressions_visitor.visit(child); - } - } - std::string identifier_full_name; if (auto * identifier_node = array_join_expression->as()) diff --git a/src/Analyzer/Resolve/ScopeAliases.h b/src/Analyzer/Resolve/ScopeAliases.h index 830ae72144b..c67a94ef234 100644 --- a/src/Analyzer/Resolve/ScopeAliases.h +++ b/src/Analyzer/Resolve/ScopeAliases.h @@ -27,10 +27,6 @@ struct ScopeAliases std::unordered_set nodes_with_duplicated_aliases; std::vector cloned_nodes_with_duplicated_aliases; - /// Names which are aliases from ARRAY JOIN. - /// This is needed to properly qualify columns from matchers and avoid name collision. - std::unordered_set array_join_aliases; - std::unordered_map & getAliasMap(IdentifierLookupContext lookup_context) { switch (lookup_context) diff --git a/src/Analyzer/createUniqueTableAliases.cpp b/src/Analyzer/createUniqueAliasesIfNecessary.cpp similarity index 57% rename from src/Analyzer/createUniqueTableAliases.cpp rename to src/Analyzer/createUniqueAliasesIfNecessary.cpp index 8f850fe8dec..bec4946d24e 100644 --- a/src/Analyzer/createUniqueTableAliases.cpp +++ b/src/Analyzer/createUniqueAliasesIfNecessary.cpp @@ -1,6 +1,7 @@ -#include -#include #include + +#include +#include #include #include #include @@ -48,8 +49,6 @@ public: case QueryTreeNodeType::TABLE: [[fallthrough]]; case QueryTreeNodeType::TABLE_FUNCTION: - [[fallthrough]]; - case QueryTreeNodeType::ARRAY_JOIN: { auto & alias = table_expression_to_alias[node]; if (alias.empty()) @@ -130,12 +129,105 @@ private: std::unordered_map table_expression_to_alias; }; -} - - -void createUniqueTableAliases(QueryTreeNodePtr & node, const QueryTreeNodePtr & /*table_expression*/, const ContextPtr & context) +class CreateUniqueArrayJoinAliasesVisitor : public InDepthQueryTreeVisitorWithContext { +public: + using Base = InDepthQueryTreeVisitorWithContext; + using Base::Base; + + bool needChildVisit(VisitQueryTreeNodeType & parent [[maybe_unused]], VisitQueryTreeNodeType & child [[maybe_unused]]) + { + auto * array_join = parent->as(); + if (!array_join) + return true; + return child != array_join->getJoinExpressionsNode(); + } + + void enterImpl(QueryTreeNodePtr & node) + { + if (auto * array_join_typed = node->as()) + { + populateRenamingMap(array_join_typed, renaming[array_join_typed]); + return; + } + + auto * column_node = node->as(); + if (!column_node || replaced_nodes_set.contains(node)) + return; + + auto column_source = column_node->getColumnSource(); + auto * array_join = column_source->as(); + if (!array_join) + return; + + auto & renaming_map = getRenamingMap(array_join); + + auto new_column = column_node->getColumn(); + new_column.name = renaming_map[column_node->getColumnName()]; + auto new_column_node = std::make_shared(new_column, column_source); + + node = std::move(new_column_node); + replaced_nodes_set.insert(node); + } + +private: + + using RenamingMap = std::unordered_map; + + void populateRenamingMap(ArrayJoinNode * array_join, RenamingMap & result) + { + if (result.empty()) + { + for (auto & array_join_expression : array_join->getJoinExpressions()) + { + auto * array_join_column = array_join_expression->as(); + chassert(array_join_column != nullptr); + + String unique_name = fmt::format("__array_join_exp_{}", ++next_id); + result.emplace(array_join_column->getColumnName(), unique_name); + + auto replacement_column = array_join_column->getColumn(); + replacement_column.name = unique_name; + auto replacement_column_node = std::make_shared(replacement_column, array_join_column->getExpression(), array_join_column->getColumnSource()); + replacement_column_node->setAlias(unique_name); + + array_join_expression = std::move(replacement_column_node); + replaced_nodes_set.insert(array_join_expression); + } + } + } + + RenamingMap & getRenamingMap(ArrayJoinNode * array_join) + { + auto & result = renaming[array_join]; + + populateRenamingMap(array_join, result); + + return result; + } + + size_t next_id = 0; + + std::unordered_map renaming; + + // TODO: Remove this field when identifier resolution cache removed from analyzer. + std::unordered_set replaced_nodes_set; +}; + +} + +void createUniqueAliasesIfNecessary(QueryTreeNodePtr & node, const ContextPtr & context) +{ + /* + * For each table expression in the Query Tree generate and add a unique alias. + * If table expression had an alias in initial query tree, override it. + */ CreateUniqueTableAliasesVisitor(context).visit(node); + + /* Generate unique aliases for array join expressions. + * It's required to create a valid AST for distributed query. + */ + CreateUniqueArrayJoinAliasesVisitor(context).visit(node); } } diff --git a/src/Analyzer/createUniqueAliasesIfNecessary.h b/src/Analyzer/createUniqueAliasesIfNecessary.h new file mode 100644 index 00000000000..078bcada387 --- /dev/null +++ b/src/Analyzer/createUniqueAliasesIfNecessary.h @@ -0,0 +1,14 @@ +#pragma once + +#include +#include + +class IQueryTreeNode; +using QueryTreeNodePtr = std::shared_ptr; + +namespace DB +{ + +void createUniqueAliasesIfNecessary(QueryTreeNodePtr & node, const ContextPtr & context); + +} diff --git a/src/Analyzer/createUniqueTableAliases.h b/src/Analyzer/createUniqueTableAliases.h index d57a198498c..078bcada387 100644 --- a/src/Analyzer/createUniqueTableAliases.h +++ b/src/Analyzer/createUniqueTableAliases.h @@ -9,10 +9,6 @@ using QueryTreeNodePtr = std::shared_ptr; namespace DB { -/* - * For each table expression in the Query Tree generate and add a unique alias. - * If table expression had an alias in initial query tree, override it. - */ -void createUniqueTableAliases(QueryTreeNodePtr & node, const QueryTreeNodePtr & table_expression, const ContextPtr & context); +void createUniqueAliasesIfNecessary(QueryTreeNodePtr & node, const ContextPtr & context); } diff --git a/src/Storages/buildQueryTreeForShard.cpp b/src/Storages/buildQueryTreeForShard.cpp index bce30260954..20ceae6a06a 100644 --- a/src/Storages/buildQueryTreeForShard.cpp +++ b/src/Storages/buildQueryTreeForShard.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include @@ -427,7 +427,7 @@ QueryTreeNodePtr buildQueryTreeForShard(const PlannerContextPtr & planner_contex removeGroupingFunctionSpecializations(query_tree_to_modify); - createUniqueTableAliases(query_tree_to_modify, nullptr, planner_context->getQueryContext()); + createUniqueAliasesIfNecessary(query_tree_to_modify, planner_context->getQueryContext()); // Get rid of the settings clause so we don't send them to remote. Thus newly non-important // settings won't break any remote parser. It's also more reasonable since the query settings diff --git a/tests/queries/0_stateless/03277_analyzer_array_join_fix.reference b/tests/queries/0_stateless/03277_analyzer_array_join_fix.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03277_analyzer_array_join_fix.sql b/tests/queries/0_stateless/03277_analyzer_array_join_fix.sql new file mode 100644 index 00000000000..1ad128df5b9 --- /dev/null +++ b/tests/queries/0_stateless/03277_analyzer_array_join_fix.sql @@ -0,0 +1,21 @@ +CREATE TABLE IF NOT EXISTS repro +( + `a` LowCardinality(String), + `foos` Nested(x LowCardinality(String)) +) +ENGINE = MergeTree +ORDER BY a; + +CREATE TABLE IF NOT EXISTS repro_dist +( + "a" LowCardinality(String), + "foos" Nested( + "x" LowCardinality(String), + ) +) ENGINE = Distributed('test_cluster_two_shards', 'default', 'repro'); + +SELECT + a, + foo.x +FROM repro_dist +ARRAY JOIN foos AS foo; From ebf77fb0dce041202864071b9d75e0bbe57835f0 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 28 Nov 2024 19:18:51 +0100 Subject: [PATCH 30/59] Fix files --- src/Analyzer/createUniqueAliasesIfNecessary.cpp | 2 +- src/Analyzer/createUniqueTableAliases.h | 14 -------------- 2 files changed, 1 insertion(+), 15 deletions(-) delete mode 100644 src/Analyzer/createUniqueTableAliases.h diff --git a/src/Analyzer/createUniqueAliasesIfNecessary.cpp b/src/Analyzer/createUniqueAliasesIfNecessary.cpp index bec4946d24e..e4d880ac6f0 100644 --- a/src/Analyzer/createUniqueAliasesIfNecessary.cpp +++ b/src/Analyzer/createUniqueAliasesIfNecessary.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/Analyzer/createUniqueTableAliases.h b/src/Analyzer/createUniqueTableAliases.h deleted file mode 100644 index 078bcada387..00000000000 --- a/src/Analyzer/createUniqueTableAliases.h +++ /dev/null @@ -1,14 +0,0 @@ -#pragma once - -#include -#include - -class IQueryTreeNode; -using QueryTreeNodePtr = std::shared_ptr; - -namespace DB -{ - -void createUniqueAliasesIfNecessary(QueryTreeNodePtr & node, const ContextPtr & context); - -} From 7c15d83a210c86b740077f8664f39cd29611160a Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 29 Nov 2024 09:52:05 +0100 Subject: [PATCH 31/59] Fix test --- tests/queries/0_stateless/03277_analyzer_array_join_fix.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03277_analyzer_array_join_fix.sql b/tests/queries/0_stateless/03277_analyzer_array_join_fix.sql index 1ad128df5b9..9d6f05dd826 100644 --- a/tests/queries/0_stateless/03277_analyzer_array_join_fix.sql +++ b/tests/queries/0_stateless/03277_analyzer_array_join_fix.sql @@ -12,7 +12,7 @@ CREATE TABLE IF NOT EXISTS repro_dist "foos" Nested( "x" LowCardinality(String), ) -) ENGINE = Distributed('test_cluster_two_shards', 'default', 'repro'); +) ENGINE = Distributed('test_cluster_two_shards', currentDatabase(), 'repro'); SELECT a, From a528b51a97f906d054df2f966e20855963fe1da1 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 29 Nov 2024 17:00:14 +0100 Subject: [PATCH 32/59] Fix identifier qualification in the presence of ARRAY JOIN --- src/Analyzer/Resolve/IdentifierResolveScope.h | 3 +++ src/Analyzer/Resolve/IdentifierResolver.cpp | 21 +++++++++++++++++++ src/Analyzer/Resolve/IdentifierResolver.h | 3 +++ src/Analyzer/Resolve/QueryAnalyzer.cpp | 3 ++- .../createUniqueAliasesIfNecessary.cpp | 8 ------- 5 files changed, 29 insertions(+), 9 deletions(-) diff --git a/src/Analyzer/Resolve/IdentifierResolveScope.h b/src/Analyzer/Resolve/IdentifierResolveScope.h index 9a09c645189..ec46ce4370e 100644 --- a/src/Analyzer/Resolve/IdentifierResolveScope.h +++ b/src/Analyzer/Resolve/IdentifierResolveScope.h @@ -165,6 +165,9 @@ struct IdentifierResolveScope /// Table expression node to data std::unordered_map table_expression_node_to_data; + /// Table expression nodes that appear in the join tree of the corresponding query + std::unordered_set registered_table_expression_nodes; + QueryTreeNodePtrWithHashIgnoreTypesSet nullable_group_by_keys; /// Here we count the number of nullable GROUP BY keys we met resolving expression. /// E.g. for a query `SELECT tuple(tuple(number)) FROM numbers(10) GROUP BY (number, tuple(number)) with cube` diff --git a/src/Analyzer/Resolve/IdentifierResolver.cpp b/src/Analyzer/Resolve/IdentifierResolver.cpp index b81db64dbb4..60cff0d62c7 100644 --- a/src/Analyzer/Resolve/IdentifierResolver.cpp +++ b/src/Analyzer/Resolve/IdentifierResolver.cpp @@ -680,6 +680,27 @@ bool IdentifierResolver::tryBindIdentifierToTableExpressions(const IdentifierLoo return can_bind_identifier_to_table_expression; } +bool IdentifierResolver::tryBindIdentifierToArrayJoinExpressions(const IdentifierLookup & identifier_lookup, const IdentifierResolveScope & scope) +{ + bool result = false; + + for (const auto & table_expression : scope.registered_table_expression_nodes) + { + auto * array_join_node = table_expression->as(); + if (!array_join_node) + continue; + + for (const auto & array_join_expression : array_join_node->getJoinExpressions()) + { + auto array_join_expression_alias = array_join_expression->getAlias(); + if (identifier_lookup.identifier.front() == array_join_expression_alias) + return true; + } + } + + return result; +} + QueryTreeNodePtr IdentifierResolver::tryResolveIdentifierFromStorage( const Identifier & identifier, const QueryTreeNodePtr & table_expression_node, diff --git a/src/Analyzer/Resolve/IdentifierResolver.h b/src/Analyzer/Resolve/IdentifierResolver.h index cdbd7610b5e..ea7b940427a 100644 --- a/src/Analyzer/Resolve/IdentifierResolver.h +++ b/src/Analyzer/Resolve/IdentifierResolver.h @@ -109,6 +109,9 @@ public: const QueryTreeNodePtr & table_expression_node, const IdentifierResolveScope & scope); + static bool tryBindIdentifierToArrayJoinExpressions(const IdentifierLookup & identifier_lookup, + const IdentifierResolveScope & scope); + QueryTreeNodePtr tryResolveIdentifierFromTableExpression(const IdentifierLookup & identifier_lookup, const QueryTreeNodePtr & table_expression_node, IdentifierResolveScope & scope); diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index b6a90ca1fe5..6c4e2551c6c 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -1593,7 +1593,7 @@ void QueryAnalyzer::qualifyColumnNodesWithProjectionNames(const QueryTreeNodes & if (need_to_qualify) need_to_qualify = IdentifierResolver::tryBindIdentifierToTableExpressions(identifier_lookup, table_expression_node, scope); - if (IdentifierResolver::tryBindIdentifierToAliases(identifier_lookup, scope)) + if (IdentifierResolver::tryBindIdentifierToAliases(identifier_lookup, scope) || IdentifierResolver::tryBindIdentifierToArrayJoinExpressions(identifier_lookup, scope)) need_to_qualify = true; if (need_to_qualify) @@ -5366,6 +5366,7 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, }; add_table_expression_alias_into_scope(join_tree_node); + scope.registered_table_expression_nodes.insert(join_tree_node); scope.table_expressions_in_resolve_process.erase(join_tree_node.get()); } diff --git a/src/Analyzer/createUniqueAliasesIfNecessary.cpp b/src/Analyzer/createUniqueAliasesIfNecessary.cpp index e4d880ac6f0..309bff13274 100644 --- a/src/Analyzer/createUniqueAliasesIfNecessary.cpp +++ b/src/Analyzer/createUniqueAliasesIfNecessary.cpp @@ -135,14 +135,6 @@ public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - bool needChildVisit(VisitQueryTreeNodeType & parent [[maybe_unused]], VisitQueryTreeNodeType & child [[maybe_unused]]) - { - auto * array_join = parent->as(); - if (!array_join) - return true; - return child != array_join->getJoinExpressionsNode(); - } - void enterImpl(QueryTreeNodePtr & node) { if (auto * array_join_typed = node->as()) From 27ffde3f3ed757cdcb8020eec53c0b6f1a0afeba Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 29 Nov 2024 18:45:53 +0100 Subject: [PATCH 33/59] 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 34/59] 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 35/59] 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 36/59] 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 37/59] 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 38/59] 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 39/59] 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 40/59] 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 41/59] 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 42/59] 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 43/59] 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 44/59] 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 45/59] 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 90e1ca8cb09a11af3b6c4191c6cb98bdfcf1cee4 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 1 Dec 2024 19:07:43 +0000 Subject: [PATCH 46/59] add reference to relevant settings for mysql table function --- docs/en/sql-reference/table-functions/mysql.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/en/sql-reference/table-functions/mysql.md b/docs/en/sql-reference/table-functions/mysql.md index 5fd9708317c..b3de6038d6d 100644 --- a/docs/en/sql-reference/table-functions/mysql.md +++ b/docs/en/sql-reference/table-functions/mysql.md @@ -50,6 +50,10 @@ SELECT name FROM mysql(`mysql1:3306|mysql2:3306|mysql3:3306`, 'mysql_database', A table object with the same columns as the original MySQL table. +:::note +Some data types of MySQL can be mapped to different ClickHouse types - this is addressed by query-level setting [mysql_datatypes_support_level](/docs/en/operations/settings/settings#mysql_datatypes_support_level). +::: + :::note In the `INSERT` query to distinguish table function `mysql(...)` from table name with column names list, you must use keywords `FUNCTION` or `TABLE FUNCTION`. See examples below. ::: @@ -141,3 +145,4 @@ WHERE id > (SELECT max(id) from mysql_copy); - [The ‘MySQL’ table engine](../../engines/table-engines/integrations/mysql.md) - [Using MySQL as a dictionary source](../../sql-reference/dictionaries/index.md#dictionary-sources#dicts-external_dicts_dict_sources-mysql) +- Relevant settings: [mysql_datatypes_support_level](/docs/en/operations/settings/settings#mysql_datatypes_support_level), [mysql_map_fixed_string_to_text_in_show_columns](/docs/en/operations/settings/settings#mysql_map_fixed_string_to_text_in_show_columns), [mysql_map_string_to_text_in_show_columns](/docs/en/operations/settings/settings#mysql_map_string_to_text_in_show_columns), [mysql_max_rows_to_insert](/docs/en/operations/settings/settings#mysql_max_rows_to_insert) From a929dc7df5cab093c1dfda48fa3d05a50a6ba10f Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 1 Dec 2024 19:16:41 +0000 Subject: [PATCH 47/59] fix --- docs/en/sql-reference/table-functions/mysql.md | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/table-functions/mysql.md b/docs/en/sql-reference/table-functions/mysql.md index b3de6038d6d..f5ec02fa149 100644 --- a/docs/en/sql-reference/table-functions/mysql.md +++ b/docs/en/sql-reference/table-functions/mysql.md @@ -51,7 +51,7 @@ SELECT name FROM mysql(`mysql1:3306|mysql2:3306|mysql3:3306`, 'mysql_database', A table object with the same columns as the original MySQL table. :::note -Some data types of MySQL can be mapped to different ClickHouse types - this is addressed by query-level setting [mysql_datatypes_support_level](/docs/en/operations/settings/settings#mysql_datatypes_support_level). +Some data types of MySQL can be mapped to different ClickHouse types - this is addressed by query-level setting [mysql_datatypes_support_level](/docs/en/operations/settings/settings.md#mysql_datatypes_support_level) ::: :::note @@ -145,4 +145,7 @@ WHERE id > (SELECT max(id) from mysql_copy); - [The ‘MySQL’ table engine](../../engines/table-engines/integrations/mysql.md) - [Using MySQL as a dictionary source](../../sql-reference/dictionaries/index.md#dictionary-sources#dicts-external_dicts_dict_sources-mysql) -- Relevant settings: [mysql_datatypes_support_level](/docs/en/operations/settings/settings#mysql_datatypes_support_level), [mysql_map_fixed_string_to_text_in_show_columns](/docs/en/operations/settings/settings#mysql_map_fixed_string_to_text_in_show_columns), [mysql_map_string_to_text_in_show_columns](/docs/en/operations/settings/settings#mysql_map_string_to_text_in_show_columns), [mysql_max_rows_to_insert](/docs/en/operations/settings/settings#mysql_max_rows_to_insert) +- [mysql_datatypes_support_level](/docs/en/operations/settings/settings.md#mysql_datatypes_support_level) +- [mysql_map_fixed_string_to_text_in_show_columns](/docs/en/operations/settings/settings.md#mysql_map_fixed_string_to_text_in_show_columns) +- [mysql_map_string_to_text_in_show_columns](/docs/en/operations/settings/settings.md#mysql_map_string_to_text_in_show_columns) +- [mysql_max_rows_to_insert](/docs/en/operations/settings/settings.md#mysql_max_rows_to_insert) From a2b18f2e2b8d3f214ab725c623822bd461c3fbea Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sun, 1 Dec 2024 23:31:51 +0100 Subject: [PATCH 48/59] 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 49/59] 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 50/59] 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 ad5a571d7a2d7f0dfd44f28924e02ef31ee8a598 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 2 Dec 2024 10:58:26 +0100 Subject: [PATCH 51/59] Preserve table expression counting --- src/Analyzer/createUniqueAliasesIfNecessary.cpp | 16 ++++++++++++---- .../01655_plan_optimizations.reference | 4 ++-- .../0_stateless/01655_plan_optimizations.sh | 2 +- 3 files changed, 15 insertions(+), 7 deletions(-) diff --git a/src/Analyzer/createUniqueAliasesIfNecessary.cpp b/src/Analyzer/createUniqueAliasesIfNecessary.cpp index 309bff13274..7a56cf07048 100644 --- a/src/Analyzer/createUniqueAliasesIfNecessary.cpp +++ b/src/Analyzer/createUniqueAliasesIfNecessary.cpp @@ -59,6 +59,12 @@ public: } break; } + case QueryTreeNodeType::ARRAY_JOIN: + { + /// Simulate previous behaviour and preserve table naming with previous versions + ++next_id; + break; + } default: break; } @@ -170,18 +176,20 @@ private: { if (result.empty()) { + String unique_array_join_name = fmt::format("__array_join_{}", ++next_id); + for (auto & array_join_expression : array_join->getJoinExpressions()) { auto * array_join_column = array_join_expression->as(); chassert(array_join_column != nullptr); - String unique_name = fmt::format("__array_join_exp_{}", ++next_id); - result.emplace(array_join_column->getColumnName(), unique_name); + String unique_expression_name = fmt::format("{}.{}", unique_array_join_name, array_join_column->getColumnName()); + result.emplace(array_join_column->getColumnName(), unique_expression_name); auto replacement_column = array_join_column->getColumn(); - replacement_column.name = unique_name; + replacement_column.name = unique_expression_name; auto replacement_column_node = std::make_shared(replacement_column, array_join_column->getExpression(), array_join_column->getColumnSource()); - replacement_column_node->setAlias(unique_name); + replacement_column_node->setAlias(unique_expression_name); array_join_expression = std::move(replacement_column_node); replaced_nodes_set.insert(array_join_expression); diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index fb27e0c3d74..d79c408f1a4 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -121,8 +121,8 @@ Filter column: and(notEquals(y, 2), notEquals(x, 0)) ARRAY JOIN x Filter column: notEquals(y, 2) > (analyzer) filter is split, one part is filtered before ARRAY JOIN -Filter column: and(notEquals(__table2.y, 2_UInt8), notEquals(__table1.x, 0_UInt8)) -ARRAY JOIN __table1.x +Filter column: and(notEquals(__table2.y, 2_UInt8), notEquals(__array_join_1.x, 0_UInt8)) +ARRAY JOIN __array_join_1.x Filter column: notEquals(__table2.y, 2_UInt8) 1 3 > filter is pushed down before Distinct diff --git a/tests/queries/0_stateless/01655_plan_optimizations.sh b/tests/queries/0_stateless/01655_plan_optimizations.sh index 52bfdb91467..622f71f00df 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations.sh @@ -134,7 +134,7 @@ $CLICKHOUSE_CLIENT --enable_analyzer=1 -q " explain actions = 1 select x, y from ( select range(number) as x, number + 1 as y from numbers(3) ) array join x where y != 2 and x != 0" | - grep -o "Filter column: and(notEquals(__table2.y, 2_UInt8), notEquals(__table1.x, 0_UInt8))\|ARRAY JOIN __table1.x\|Filter column: notEquals(__table2.y, 2_UInt8)" + grep -o "Filter column: and(notEquals(__table2.y, 2_UInt8), notEquals(__array_join_1.x, 0_UInt8))\|ARRAY JOIN __array_join_1.x\|Filter column: notEquals(__table2.y, 2_UInt8)" $CLICKHOUSE_CLIENT -q " select x, y from ( select range(number) as x, number + 1 as y from numbers(3) From 7e00e604fa2b5244aa268043371dbffca8d7b77e Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 2 Dec 2024 12:01:24 +0100 Subject: [PATCH 52/59] Fix test --- src/Analyzer/createUniqueAliasesIfNecessary.cpp | 4 +--- tests/queries/0_stateless/01655_plan_optimizations.reference | 4 ++-- tests/queries/0_stateless/01655_plan_optimizations.sh | 2 +- 3 files changed, 4 insertions(+), 6 deletions(-) diff --git a/src/Analyzer/createUniqueAliasesIfNecessary.cpp b/src/Analyzer/createUniqueAliasesIfNecessary.cpp index 7a56cf07048..fa2ccb09e43 100644 --- a/src/Analyzer/createUniqueAliasesIfNecessary.cpp +++ b/src/Analyzer/createUniqueAliasesIfNecessary.cpp @@ -176,14 +176,12 @@ private: { if (result.empty()) { - String unique_array_join_name = fmt::format("__array_join_{}", ++next_id); - for (auto & array_join_expression : array_join->getJoinExpressions()) { auto * array_join_column = array_join_expression->as(); chassert(array_join_column != nullptr); - String unique_expression_name = fmt::format("{}.{}", unique_array_join_name, array_join_column->getColumnName()); + String unique_expression_name = fmt::format("__array_join_exp_{}", ++next_id); result.emplace(array_join_column->getColumnName(), unique_expression_name); auto replacement_column = array_join_column->getColumn(); diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index d79c408f1a4..eb13997f0d1 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -121,8 +121,8 @@ Filter column: and(notEquals(y, 2), notEquals(x, 0)) ARRAY JOIN x Filter column: notEquals(y, 2) > (analyzer) filter is split, one part is filtered before ARRAY JOIN -Filter column: and(notEquals(__table2.y, 2_UInt8), notEquals(__array_join_1.x, 0_UInt8)) -ARRAY JOIN __array_join_1.x +Filter column: and(notEquals(__table2.y, 2_UInt8), notEquals(__array_join_exp_1, 0_UInt8)) +ARRAY JOIN __array_join_exp_1 Filter column: notEquals(__table2.y, 2_UInt8) 1 3 > filter is pushed down before Distinct diff --git a/tests/queries/0_stateless/01655_plan_optimizations.sh b/tests/queries/0_stateless/01655_plan_optimizations.sh index 622f71f00df..1a283a3027b 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations.sh @@ -134,7 +134,7 @@ $CLICKHOUSE_CLIENT --enable_analyzer=1 -q " explain actions = 1 select x, y from ( select range(number) as x, number + 1 as y from numbers(3) ) array join x where y != 2 and x != 0" | - grep -o "Filter column: and(notEquals(__table2.y, 2_UInt8), notEquals(__array_join_1.x, 0_UInt8))\|ARRAY JOIN __array_join_1.x\|Filter column: notEquals(__table2.y, 2_UInt8)" + grep -o "Filter column: and(notEquals(__table2.y, 2_UInt8), notEquals(__array_join_exp_1, 0_UInt8))\|ARRAY JOIN __array_join_exp_1\|Filter column: notEquals(__table2.y, 2_UInt8)" $CLICKHOUSE_CLIENT -q " select x, y from ( select range(number) as x, number + 1 as y from numbers(3) From ceea266253132ff3f567946b95dbfef9e98b8a5d Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 2 Dec 2024 11:09:03 +0000 Subject: [PATCH 53/59] fix --- src/Planner/PlannerJoinTree.cpp | 16 ---------------- 1 file changed, 16 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index a459658195e..48a68d97440 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1787,22 +1787,6 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode( required_columns_after_join.insert(right_header.getByPosition(0).name); } - for (auto & column_from_joined_table : columns_from_left_table) - { - /// Add columns to output only if they are presented in outer scope, otherwise they can be dropped - if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(column_from_joined_table.name) && - required_columns_after_join.contains(column_from_joined_table.name)) - table_join->setUsedColumn(column_from_joined_table, JoinTableSide::Left); - } - - for (auto & column_from_joined_table : columns_from_right_table) - { - /// Add columns to output only if they are presented in outer scope, otherwise they can be dropped - if (planner_context->getGlobalPlannerContext()->hasColumnIdentifier(column_from_joined_table.name) && - required_columns_after_join.contains(column_from_joined_table.name)) - table_join->setUsedColumn(column_from_joined_table, JoinTableSide::Right); - } - auto join_step = std::make_unique( left_plan.getCurrentHeader(), right_plan.getCurrentHeader(), From 4aecc57555b145d4a4ce27e7d818d7122b87f105 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 2 Dec 2024 12:58:34 +0100 Subject: [PATCH 54/59] 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 ebe76f939b7ca14170b91b6aab093ce45229f85b Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 2 Dec 2024 13:12:20 +0000 Subject: [PATCH 55/59] fix test --- tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 index cdab509db90..1c9176a9504 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 @@ -73,7 +73,7 @@ SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id + 2; -- { serverE SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.key; -- { serverError 43, 403 } SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverError 43, 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 } -SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id; +SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id ORDER BY ALL; SELECT * FROM t1 JOIN t2_nullable as t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)) ORDER BY ALL SETTINGS enable_analyzer = 1; SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id ORDER BY ALL SETTINGS enable_analyzer = 1; SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id ORDER BY ALL SETTINGS enable_analyzer = 1; From 9412e8a25eb196b11b92ebade47b7fa2d6922065 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 2 Dec 2024 14:49:20 +0100 Subject: [PATCH 56/59] Update reference file --- tests/queries/0_stateless/01823_explain_json.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01823_explain_json.reference b/tests/queries/0_stateless/01823_explain_json.reference index 6612fd232bf..0a6100c85eb 100644 --- a/tests/queries/0_stateless/01823_explain_json.reference +++ b/tests/queries/0_stateless/01823_explain_json.reference @@ -81,7 +81,7 @@ "Node Type": "ArrayJoin", "Node Id": "ArrayJoin_4", "Left": false, - "Columns": ["__table1.x", "__table1.y"], + "Columns": ["__array_join_exp_1", "__array_join_exp_2"], -------- "Node Type": "Distinct", "Node Id": "Distinct_4", From d4f930536235d713ca5a5060337085b52cb08a78 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 2 Dec 2024 15:05:47 +0000 Subject: [PATCH 57/59] 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 58/59] 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 59/59] 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