From d12ecdc5f06689d6259e2ef082a916f8b2f1836f Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 13 Feb 2024 12:35:17 +0100 Subject: [PATCH 001/102] Asynchronous WriteBuffer for AzureBlobStorage --- src/Backups/BackupIO_AzureBlobStorage.cpp | 3 +- src/Core/Settings.h | 3 +- .../IO/WriteBufferFromAzureBlobStorage.cpp | 116 +++++++++++------- .../IO/WriteBufferFromAzureBlobStorage.h | 22 +++- .../AzureBlobStorage/AzureBlobStorageAuth.cpp | 3 +- .../AzureBlobStorage/AzureObjectStorage.cpp | 3 +- .../AzureBlobStorage/AzureObjectStorage.h | 5 +- src/IO/WriteBufferFromS3.cpp | 2 +- src/IO/WriteBufferFromS3.h | 5 +- src/IO/WriteBufferFromS3TaskTracker.cpp | 21 ++-- src/IO/WriteBufferFromS3TaskTracker.h | 8 +- 11 files changed, 124 insertions(+), 67 deletions(-) diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index 52ce20d5108..44a72f80456 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -278,7 +278,8 @@ std::unique_ptr BackupWriterAzureBlobStorage::writeFile(const Strin settings->max_single_part_upload_size, settings->max_unexpected_write_error_retries, DBMS_DEFAULT_BUFFER_SIZE, - write_settings); + write_settings, + settings->max_inflight_parts_for_one_file); } void BackupWriterAzureBlobStorage::removeFile(const String & file_name) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 44badfefabb..53de245bdfc 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -80,7 +80,8 @@ class IColumn; M(UInt64, s3_max_upload_part_size, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to S3.", 0) \ M(UInt64, s3_upload_part_size_multiply_factor, 2, "Multiply s3_min_upload_part_size by this factor each time s3_multiply_parts_count_threshold parts were uploaded from a single write to S3.", 0) \ M(UInt64, s3_upload_part_size_multiply_parts_count_threshold, 500, "Each time this number of parts was uploaded to S3, s3_min_upload_part_size is multiplied by s3_upload_part_size_multiply_factor.", 0) \ - M(UInt64, s3_max_inflight_parts_for_one_file, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited. You ", 0) \ + M(UInt64, s3_max_inflight_parts_for_one_file, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited.", 0) \ + M(UInt64, azure_max_inflight_parts_for_one_file, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited.", 0) \ M(UInt64, s3_max_single_part_upload_size, 32*1024*1024, "The maximum size of object to upload using singlepart upload to S3.", 0) \ M(UInt64, azure_max_single_part_upload_size, 100*1024*1024, "The maximum size of object to upload using singlepart upload to Azure blob storage.", 0) \ M(UInt64, azure_max_single_part_copy_size, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage.", 0) \ diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 905114f50e9..cbe2367823d 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -18,13 +18,21 @@ namespace ProfileEvents namespace DB { +struct WriteBufferFromAzureBlobStorage::PartData +{ + Memory<> memory; + size_t data_size = 0; +}; + WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage( std::shared_ptr blob_container_client_, const String & blob_path_, size_t max_single_part_upload_size_, size_t max_unexpected_write_error_retries_, size_t buf_size_, - const WriteSettings & write_settings_) + const WriteSettings & write_settings_, + size_t max_inflight_parts_for_one_file_, + ThreadPoolCallbackRunner schedule_) : WriteBufferFromFileBase(buf_size_, nullptr, 0) , log(getLogger("WriteBufferFromAzureBlobStorage")) , max_single_part_upload_size(max_single_part_upload_size_) @@ -32,7 +40,13 @@ WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage( , blob_path(blob_path_) , write_settings(write_settings_) , blob_container_client(blob_container_client_) + , task_tracker( + std::make_unique( + std::move(schedule_), + max_inflight_parts_for_one_file_, + limitedLog)) { + allocateBuffer(); } @@ -79,60 +93,80 @@ void WriteBufferFromAzureBlobStorage::finalizeImpl() { execWithRetry([this](){ next(); }, max_unexpected_write_error_retries); - if (tmp_buffer_write_offset > 0) - uploadBlock(tmp_buffer->data(), tmp_buffer_write_offset); + task_tracker->waitAll(); auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); execWithRetry([&](){ block_blob_client.CommitBlockList(block_ids); }, max_unexpected_write_error_retries); - LOG_TRACE(log, "Committed {} blocks for blob `{}`", block_ids.size(), blob_path); -} - -void WriteBufferFromAzureBlobStorage::uploadBlock(const char * data, size_t size) -{ - auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); - const std::string & block_id = block_ids.emplace_back(getRandomASCIIString(64)); - - Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(data), size); - execWithRetry([&](){ block_blob_client.StageBlock(block_id, memory_stream); }, max_unexpected_write_error_retries, size); - tmp_buffer_write_offset = 0; - - LOG_TRACE(log, "Staged block (id: {}) of size {} (blob path: {}).", block_id, size, blob_path); -} - -WriteBufferFromAzureBlobStorage::MemoryBufferPtr WriteBufferFromAzureBlobStorage::allocateBuffer() const -{ - return std::make_unique>(max_single_part_upload_size); + LOG_DEBUG(log, "Committed {} blocks for blob `{}`", block_ids.size(), blob_path); } void WriteBufferFromAzureBlobStorage::nextImpl() { - size_t size_to_upload = offset(); + task_tracker->waitIfAny(); - if (size_to_upload == 0) - return; + reallocateBuffer(); + detachBuffer(); - if (!tmp_buffer) - tmp_buffer = allocateBuffer(); - - size_t uploaded_size = 0; - while (uploaded_size != size_to_upload) + while (!detached_part_data.empty()) { - size_t memory_buffer_remaining_size = max_single_part_upload_size - tmp_buffer_write_offset; - if (memory_buffer_remaining_size == 0) - uploadBlock(tmp_buffer->data(), tmp_buffer->size()); - - size_t size = std::min(memory_buffer_remaining_size, size_to_upload - uploaded_size); - memcpy(tmp_buffer->data() + tmp_buffer_write_offset, working_buffer.begin() + uploaded_size, size); - uploaded_size += size; - tmp_buffer_write_offset += size; + writePart(std::move(detached_part_data.front())); + detached_part_data.pop_front(); } - if (tmp_buffer_write_offset == max_single_part_upload_size) - uploadBlock(tmp_buffer->data(), tmp_buffer->size()); + allocateBuffer(); +} - if (write_settings.remote_throttler) - write_settings.remote_throttler->add(size_to_upload, ProfileEvents::RemoteWriteThrottlerBytes, ProfileEvents::RemoteWriteThrottlerSleepMicroseconds); +void WriteBufferFromAzureBlobStorage::allocateBuffer() +{ + memory = Memory(max_single_part_upload_size); + WriteBuffer::set(memory.data(), memory.size()); +} + + +void WriteBufferFromAzureBlobStorage::reallocateBuffer() +{ + chassert(offset() == 0); + + if (available() > 0) + return; + + if (memory.size() == max_single_part_upload_size) + return; + + memory.resize(max_single_part_upload_size); + + WriteBuffer::set(memory.data(), memory.size()); + + chassert(offset() == 0); +} + +void WriteBufferFromAzureBlobStorage::detachBuffer() +{ + size_t data_size = size_t(position() - memory.data()); + auto buf = std::move(memory); + WriteBuffer::set(nullptr, 0); + detached_part_data.push_back({std::move(buf), data_size}); +} + +void WriteBufferFromAzureBlobStorage::writePart(WriteBufferFromAzureBlobStorage::PartData && data) +{ + if (data.data_size == 0) + return; + + auto upload_worker = [&] () + { + auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); + const std::string & block_id = block_ids.emplace_back(getRandomASCIIString(64)); + + Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(data.memory.data()), data.data_size); + execWithRetry([&](){ block_blob_client.StageBlock(block_id, memory_stream); }, max_unexpected_write_error_retries, data.data_size); + + if (write_settings.remote_throttler) + write_settings.remote_throttler->add(data.data_size, ProfileEvents::RemoteWriteThrottlerBytes, ProfileEvents::RemoteWriteThrottlerSleepMicroseconds); + }; + + task_tracker->add(std::move(upload_worker)); } } diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h index f105b35c121..2d11014fa2a 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h @@ -11,6 +11,7 @@ #include #include #include +#include namespace Poco @@ -21,6 +22,8 @@ class Logger; namespace DB { +class TaskTracker; + class WriteBufferFromAzureBlobStorage : public WriteBufferFromFileBase { public: @@ -32,7 +35,9 @@ public: size_t max_single_part_upload_size_, size_t max_unexpected_write_error_retries_, size_t buf_size_, - const WriteSettings & write_settings_); + const WriteSettings & write_settings_, + size_t max_inflight_parts_for_one_file_, + ThreadPoolCallbackRunner schedule_ = {}); ~WriteBufferFromAzureBlobStorage() override; @@ -42,11 +47,21 @@ public: void sync() override { next(); } private: + struct PartData; + + void writePart(WriteBufferFromAzureBlobStorage::PartData && data); + void detachBuffer(); + void allocateBuffer(); + void allocateFirstBuffer(); + void reallocateFirstBuffer(); + void reallocateBuffer(); + void finalizeImpl() override; void execWithRetry(std::function func, size_t num_tries, size_t cost = 0); void uploadBlock(const char * data, size_t size); LoggerPtr log; + LogSeriesLimiterPtr limitedLog = std::make_shared(log, 1, 5); const size_t max_single_part_upload_size; const size_t max_unexpected_write_error_retries; @@ -61,6 +76,11 @@ private: size_t tmp_buffer_write_offset = 0; MemoryBufferPtr allocateBuffer() const; + + bool first_buffer=true; + + std::unique_ptr task_tracker; + std::deque detached_part_data; }; } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp index 72c4abee5c9..f99586b2d1a 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -169,7 +169,8 @@ std::unique_ptr getAzureBlobStorageSettings(const Po config.getUInt64(config_prefix + ".max_upload_part_size", 5ULL * 1024 * 1024 * 1024), config.getUInt64(config_prefix + ".max_single_part_copy_size", context->getSettings().azure_max_single_part_copy_size), config.getBool(config_prefix + ".use_native_copy", false), - config.getUInt64(config_prefix + ".max_unexpected_write_error_retries", context->getSettings().azure_max_unexpected_write_error_retries) + config.getUInt64(config_prefix + ".max_unexpected_write_error_retries", context->getSettings().azure_max_unexpected_write_error_retries), + config.getUInt64(config_prefix + ".max_inflight_parts_for_one_file", context->getSettings().azure_max_inflight_parts_for_one_file) ); } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 74389aedb64..844789ea5b5 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -268,7 +268,8 @@ std::unique_ptr AzureObjectStorage::writeObject( /// NO settings.get()->max_single_part_upload_size, settings.get()->max_unexpected_write_error_retries, buf_size, - patchSettings(write_settings)); + patchSettings(write_settings), + settings.get()->max_inflight_parts_for_one_file); } /// Remove file. Throws exception if file doesn't exists or it's a directory. diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index f16c35fb52c..1b473a01304 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -27,7 +27,8 @@ struct AzureObjectStorageSettings size_t max_upload_part_size_, size_t max_single_part_copy_size_, bool use_native_copy_, - size_t max_unexpected_write_error_retries_) + size_t max_unexpected_write_error_retries_, + size_t max_inflight_parts_for_one_file_) : max_single_part_upload_size(max_single_part_upload_size_) , min_bytes_for_seek(min_bytes_for_seek_) , max_single_read_retries(max_single_read_retries_) @@ -37,6 +38,7 @@ struct AzureObjectStorageSettings , max_single_part_copy_size(max_single_part_copy_size_) , use_native_copy(use_native_copy_) , max_unexpected_write_error_retries (max_unexpected_write_error_retries_) + , max_inflight_parts_for_one_file (max_inflight_parts_for_one_file_) { } @@ -52,6 +54,7 @@ struct AzureObjectStorageSettings size_t max_single_part_copy_size = 256 * 1024 * 1024; bool use_native_copy = false; size_t max_unexpected_write_error_retries = 4; + size_t max_inflight_parts_for_one_file = 20; }; using AzureClient = Azure::Storage::Blobs::BlobContainerClient; diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 5bb01050591..6fc0a35672f 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -95,7 +95,7 @@ WriteBufferFromS3::WriteBufferFromS3( , object_metadata(std::move(object_metadata_)) , buffer_allocation_policy(ChooseBufferPolicy(upload_settings)) , task_tracker( - std::make_unique( + std::make_unique( std::move(schedule_), upload_settings.max_inflight_parts_for_one_file, limitedLog)) diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 230f39b074e..f3637122ee4 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -12,6 +12,7 @@ #include #include #include +#include #include #include @@ -26,6 +27,8 @@ namespace DB * Data is divided on chunks with size greater than 'minimum_upload_part_size'. Last chunk can be less than this threshold. * Each chunk is written as a part to S3. */ +class TaskTracker; + class WriteBufferFromS3 final : public WriteBufferFromFileBase { public: @@ -118,7 +121,7 @@ private: size_t total_size = 0; size_t hidden_size = 0; - class TaskTracker; +// class TaskTracker; std::unique_ptr task_tracker; BlobStorageLogWriterPtr blob_log; diff --git a/src/IO/WriteBufferFromS3TaskTracker.cpp b/src/IO/WriteBufferFromS3TaskTracker.cpp index bce122dd6c8..e62de261fc2 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.cpp +++ b/src/IO/WriteBufferFromS3TaskTracker.cpp @@ -1,7 +1,5 @@ #include "config.h" -#if USE_AWS_S3 - #include namespace ProfileEvents @@ -12,19 +10,19 @@ namespace ProfileEvents namespace DB { -WriteBufferFromS3::TaskTracker::TaskTracker(ThreadPoolCallbackRunner scheduler_, size_t max_tasks_inflight_, LogSeriesLimiterPtr limitedLog_) +TaskTracker::TaskTracker(ThreadPoolCallbackRunner scheduler_, size_t max_tasks_inflight_, LogSeriesLimiterPtr limitedLog_) : is_async(bool(scheduler_)) , scheduler(scheduler_ ? std::move(scheduler_) : syncRunner()) , max_tasks_inflight(max_tasks_inflight_) , limitedLog(limitedLog_) {} -WriteBufferFromS3::TaskTracker::~TaskTracker() +TaskTracker::~TaskTracker() { safeWaitAll(); } -ThreadPoolCallbackRunner WriteBufferFromS3::TaskTracker::syncRunner() +ThreadPoolCallbackRunner TaskTracker::syncRunner() { return [](Callback && callback, int64_t) mutable -> std::future { @@ -35,7 +33,7 @@ ThreadPoolCallbackRunner WriteBufferFromS3::TaskTracker::syncRunner() }; } -void WriteBufferFromS3::TaskTracker::waitAll() +void TaskTracker::waitAll() { /// Exceptions are propagated for (auto & future : futures) @@ -48,7 +46,7 @@ void WriteBufferFromS3::TaskTracker::waitAll() finished_futures.clear(); } -void WriteBufferFromS3::TaskTracker::safeWaitAll() +void TaskTracker::safeWaitAll() { for (auto & future : futures) { @@ -71,7 +69,7 @@ void WriteBufferFromS3::TaskTracker::safeWaitAll() finished_futures.clear(); } -void WriteBufferFromS3::TaskTracker::waitIfAny() +void TaskTracker::waitIfAny() { if (futures.empty()) return; @@ -99,7 +97,7 @@ void WriteBufferFromS3::TaskTracker::waitIfAny() ProfileEvents::increment(ProfileEvents::WriteBufferFromS3WaitInflightLimitMicroseconds, watch.elapsedMicroseconds()); } -void WriteBufferFromS3::TaskTracker::add(Callback && func) +void TaskTracker::add(Callback && func) { /// All this fuzz is about 2 things. This is the most critical place of TaskTracker. /// The first is not to fail insertion in the list `futures`. @@ -134,7 +132,7 @@ void WriteBufferFromS3::TaskTracker::add(Callback && func) waitTilInflightShrink(); } -void WriteBufferFromS3::TaskTracker::waitTilInflightShrink() +void TaskTracker::waitTilInflightShrink() { if (!max_tasks_inflight) return; @@ -166,11 +164,10 @@ void WriteBufferFromS3::TaskTracker::waitTilInflightShrink() ProfileEvents::increment(ProfileEvents::WriteBufferFromS3WaitInflightLimitMicroseconds, watch.elapsedMicroseconds()); } -bool WriteBufferFromS3::TaskTracker::isAsync() const +bool TaskTracker::isAsync() const { return is_async; } } -#endif diff --git a/src/IO/WriteBufferFromS3TaskTracker.h b/src/IO/WriteBufferFromS3TaskTracker.h index 815e041ae52..134abbbc4c1 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.h +++ b/src/IO/WriteBufferFromS3TaskTracker.h @@ -1,9 +1,7 @@ #pragma once #include "config.h" - -#if USE_AWS_S3 - +#include #include "WriteBufferFromS3.h" #include @@ -22,7 +20,7 @@ namespace DB /// Basic exception safety is provided. If exception occurred the object has to be destroyed. /// No thread safety is provided. Use this object with no concurrency. -class WriteBufferFromS3::TaskTracker +class TaskTracker { public: using Callback = std::function; @@ -68,5 +66,3 @@ private: }; } - -#endif From 26fd3d0d852986b6bbaf595087cb0d06bdff9f93 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 14 Feb 2024 16:13:53 +0100 Subject: [PATCH 002/102] Removed offset check --- src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index cbe2367823d..d700090303a 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -126,8 +126,6 @@ void WriteBufferFromAzureBlobStorage::allocateBuffer() void WriteBufferFromAzureBlobStorage::reallocateBuffer() { - chassert(offset() == 0); - if (available() > 0) return; From 7bf42fd86e9599357282f947312c98d2bec1047f Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 16 Feb 2024 11:16:14 +0100 Subject: [PATCH 003/102] Fix upgrade check --- src/Core/SettingsChangesHistory.h | 3 ++- src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index c453dd837eb..b6d07d7057a 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -93,7 +93,8 @@ static std::map sett {"async_insert_busy_timeout_decrease_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout decreases"}, {"split_parts_ranges_into_intersecting_and_non_intersecting_final", true, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}, - {"azure_max_single_part_copy_size", 256*1024*1024, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage."}}}, + {"azure_max_single_part_copy_size", 256*1024*1024, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage."}, + {"azure_max_inflight_parts_for_one_file", 20, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited."}}}, {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, {"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"}, diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index d700090303a..74a8949b235 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -98,7 +98,7 @@ void WriteBufferFromAzureBlobStorage::finalizeImpl() auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); execWithRetry([&](){ block_blob_client.CommitBlockList(block_ids); }, max_unexpected_write_error_retries); - LOG_DEBUG(log, "Committed {} blocks for blob `{}`", block_ids.size(), blob_path); + LOG_TRACE(log, "Committed {} blocks for blob `{}`", block_ids.size(), blob_path); } void WriteBufferFromAzureBlobStorage::nextImpl() From 5641fd8ba9c4f27794367e22632365df5cdf0303 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 29 Feb 2024 16:13:05 +0100 Subject: [PATCH 004/102] Fix build after merge --- src/IO/WriteBufferFromS3TaskTracker.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromS3TaskTracker.h b/src/IO/WriteBufferFromS3TaskTracker.h index 134abbbc4c1..4061f084a76 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.h +++ b/src/IO/WriteBufferFromS3TaskTracker.h @@ -1,7 +1,7 @@ #pragma once #include "config.h" -#include +#include #include "WriteBufferFromS3.h" #include From 0f2d47e5a444bf78ffef6b2506e50079e6bb55c9 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 1 Mar 2024 10:52:44 +0100 Subject: [PATCH 005/102] Renamed WriteBufferFromS3TaskTracker to ThreadPoolTaskTracker --- .../ThreadPoolTaskTracker.cpp} | 2 +- .../ThreadPoolTaskTracker.h} | 6 +++--- src/Disks/IO/WriteBufferFromAzureBlobStorage.h | 2 +- src/IO/WriteBufferFromS3.cpp | 2 +- src/IO/WriteBufferFromS3.h | 2 +- 5 files changed, 7 insertions(+), 7 deletions(-) rename src/{IO/WriteBufferFromS3TaskTracker.cpp => Common/ThreadPoolTaskTracker.cpp} (99%) rename src/{IO/WriteBufferFromS3TaskTracker.h => Common/ThreadPoolTaskTracker.h} (94%) diff --git a/src/IO/WriteBufferFromS3TaskTracker.cpp b/src/Common/ThreadPoolTaskTracker.cpp similarity index 99% rename from src/IO/WriteBufferFromS3TaskTracker.cpp rename to src/Common/ThreadPoolTaskTracker.cpp index e62de261fc2..10207eb6296 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.cpp +++ b/src/Common/ThreadPoolTaskTracker.cpp @@ -1,6 +1,6 @@ #include "config.h" -#include +#include "ThreadPoolTaskTracker.h" namespace ProfileEvents { diff --git a/src/IO/WriteBufferFromS3TaskTracker.h b/src/Common/ThreadPoolTaskTracker.h similarity index 94% rename from src/IO/WriteBufferFromS3TaskTracker.h rename to src/Common/ThreadPoolTaskTracker.h index 4061f084a76..d37b759a913 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.h +++ b/src/Common/ThreadPoolTaskTracker.h @@ -1,10 +1,10 @@ #pragma once #include "config.h" -#include -#include "WriteBufferFromS3.h" +#include "threadPoolCallbackRunner.h" +#include "IO/WriteBufferFromS3.h" -#include +#include "logger_useful.h" #include diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h index 2d11014fa2a..4897ca9a846 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h @@ -11,7 +11,7 @@ #include #include #include -#include +#include namespace Poco diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 6fc0a35672f..510d9bef4d3 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -4,8 +4,8 @@ #include "StdIStreamFromMemory.h" #include "WriteBufferFromS3.h" -#include "WriteBufferFromS3TaskTracker.h" +#include #include #include #include diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 28754d180bf..afd8b9909c1 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include #include From d529389522311e7bca11a3beebc07e0439efcfb4 Mon Sep 17 00:00:00 2001 From: Zach Naimon Date: Tue, 5 Mar 2024 14:20:20 -0500 Subject: [PATCH 006/102] Add support for 'START TRANSACTION' syntax --- src/Parsers/ParserTransactionControl.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Parsers/ParserTransactionControl.cpp b/src/Parsers/ParserTransactionControl.cpp index da593170002..fc3077bb0b6 100644 --- a/src/Parsers/ParserTransactionControl.cpp +++ b/src/Parsers/ParserTransactionControl.cpp @@ -14,6 +14,8 @@ bool ParserTransactionControl::parseImpl(Pos & pos, ASTPtr & node, Expected & ex if (ParserKeyword("BEGIN TRANSACTION").ignore(pos, expected)) action = ASTTransactionControl::BEGIN; + else if (ParserKeyword("START TRANSACTION").ignore(pos, expected)) + action = ASTTransactionControl::BEGIN; else if (ParserKeyword("COMMIT").ignore(pos, expected)) action = ASTTransactionControl::COMMIT; else if (ParserKeyword("ROLLBACK").ignore(pos, expected)) From d596de73847c712d766e0f8598ce43ed078ec968 Mon Sep 17 00:00:00 2001 From: Zach Naimon Date: Tue, 5 Mar 2024 14:25:48 -0500 Subject: [PATCH 007/102] update documentation --- docs/en/sql-reference/transactions.md | 42 +++++++++++++++++++++------ 1 file changed, 33 insertions(+), 9 deletions(-) diff --git a/docs/en/sql-reference/transactions.md b/docs/en/sql-reference/transactions.md index cb89a091d68..b9c9afc20f9 100644 --- a/docs/en/sql-reference/transactions.md +++ b/docs/en/sql-reference/transactions.md @@ -1,26 +1,29 @@ --- slug: /en/guides/developer/transactional --- + # Transactional (ACID) support -## Case 1: INSERT into one partition, of one table, of the MergeTree* family +## Case 1: INSERT into one partition, of one table, of the MergeTree\* family This is transactional (ACID) if the inserted rows are packed and inserted as a single block (see Notes): + - Atomic: an INSERT succeeds or is rejected as a whole: if a confirmation is sent to the client, then all rows were inserted; if an error is sent to the client, then no rows were inserted. - Consistent: if there are no table constraints violated, then all rows in an INSERT are inserted and the INSERT succeeds; if constraints are violated, then no rows are inserted. - Isolated: concurrent clients observe a consistent snapshot of the table–the state of the table either as it was before the INSERT attempt, or after the successful INSERT; no partial state is seen - Durable: a successful INSERT is written to the filesystem before answering to the client, on a single replica or multiple replicas (controlled by the `insert_quorum` setting), and ClickHouse can ask the OS to sync the filesystem data on the storage media (controlled by the `fsync_after_insert` setting). - INSERT into multiple tables with one statement is possible if materialized views are involved (the INSERT from the client is to a table which has associate materialized views). -## Case 2: INSERT into multiple partitions, of one table, of the MergeTree* family +## Case 2: INSERT into multiple partitions, of one table, of the MergeTree\* family Same as Case 1 above, with this detail: + - If table has many partitions and INSERT covers many partitions–then insertion into every partition is transactional on its own - -## Case 3: INSERT into one distributed table of the MergeTree* family +## Case 3: INSERT into one distributed table of the MergeTree\* family Same as Case 1 above, with this detail: + - INSERT into Distributed table is not transactional as a whole, while insertion into every shard is transactional ## Case 4: Using a Buffer table @@ -30,9 +33,11 @@ Same as Case 1 above, with this detail: ## Case 5: Using async_insert Same as Case 1 above, with this detail: + - atomicity is ensured even if `async_insert` is enabled and `wait_for_async_insert` is set to 1 (the default), but if `wait_for_async_insert` is set to 0, then atomicity is not ensured. ## Notes + - rows inserted from the client in some data format are packed into a single block when: - the insert format is row-based (like CSV, TSV, Values, JSONEachRow, etc) and the data contains less then `max_insert_block_size` rows (~1 000 000 by default) or less then `min_chunk_bytes_for_parallel_parsing` bytes (10 MB by default) in case of parallel parsing is used (enabled by default) - the insert format is column-based (like Native, Parquet, ORC, etc) and the data contains only one block of data @@ -61,8 +66,9 @@ In addition to the functionality described at the top of this document, ClickHou ``` ### Notes + - This is an experimental feature, and changes should be expected. -- If an exception occurs during a transaction, you cannot commit the transaction. This includes all exceptions, including `UNKNOWN_FUNCTION` exceptions caused by typos. +- If an exception occurs during a transaction, you cannot commit the transaction. This includes all exceptions, including `UNKNOWN_FUNCTION` exceptions caused by typos. - Nested transactions are not supported; finish the current transaction and start a new one instead ### Configuration @@ -80,7 +86,7 @@ These examples are with a single node ClickHouse server with ClickHouse Keeper e #### Basic configuration for a single ClickHouse server node with ClickHouse Keeper enabled :::note -See the [deployment](docs/en/deployment-guides/terminology.md) documentation for details on deploying ClickHouse server and a proper quorum of ClickHouse Keeper nodes. The configuration shown here is for experimental purposes. +See the [deployment](docs/en/deployment-guides/terminology.md) documentation for details on deploying ClickHouse server and a proper quorum of ClickHouse Keeper nodes. The configuration shown here is for experimental purposes. ::: ```xml title=/etc/clickhouse-server/config.d/config.xml @@ -127,17 +133,19 @@ See the [deployment](docs/en/deployment-guides/terminology.md) documentation for #### Verify that experimental transactions are enabled -Issue a `BEGIN TRANSACTION` followed by a `ROLLBACK` to verify that experimental transactions are enabled, and that ClickHouse Keeper is enabled as it is used to track transactions. +Issue a `BEGIN TRANSACTION` or `START TRANSACTION` followed by a `ROLLBACK` to verify that experimental transactions are enabled, and that ClickHouse Keeper is enabled as it is used to track transactions. ```sql BEGIN TRANSACTION ``` + ```response Ok. ``` :::tip If you see the following error, then check your configuration file to make sure that `allow_experimental_transactions` is set to `1` (or any value other than `0` or `false`). + ``` Code: 48. DB::Exception: Received from localhost:9000. DB::Exception: Transactions are not supported. @@ -145,15 +153,18 @@ DB::Exception: Transactions are not supported. ``` You can also check ClickHouse Keeper by issuing + ``` echo ruok | nc localhost 9181 ``` + ClickHouse Keeper should respond with `imok`. ::: ```sql ROLLBACK ``` + ```response Ok. ``` @@ -161,7 +172,7 @@ Ok. #### Create a table for testing :::tip -Creation of tables is not transactional. Run this DDL query outside of a transaction. +Creation of tables is not transactional. Run this DDL query outside of a transaction. ::: ```sql @@ -172,6 +183,7 @@ CREATE TABLE mergetree_table ENGINE = MergeTree ORDER BY n ``` + ```response Ok. ``` @@ -181,6 +193,7 @@ Ok. ```sql BEGIN TRANSACTION ``` + ```response Ok. ``` @@ -188,6 +201,7 @@ Ok. ```sql INSERT INTO mergetree_table FORMAT Values (10) ``` + ```response Ok. ``` @@ -196,11 +210,13 @@ Ok. SELECT * FROM mergetree_table ``` + ```response ┌──n─┐ │ 10 │ └────┘ ``` + :::note You can query the table from within a transaction and see that the row was inserted even though it has not yet been committed. ::: @@ -208,16 +224,20 @@ You can query the table from within a transaction and see that the row was inser #### Rollback the transaction, and query the table again Verify that the transaction is rolled back: + ```sql ROLLBACK ``` + ```response Ok. ``` + ```sql SELECT * FROM mergetree_table ``` + ```response Ok. @@ -229,6 +249,7 @@ Ok. ```sql BEGIN TRANSACTION ``` + ```response Ok. ``` @@ -236,6 +257,7 @@ Ok. ```sql INSERT INTO mergetree_table FORMAT Values (42) ``` + ```response Ok. ``` @@ -243,6 +265,7 @@ Ok. ```sql COMMIT ``` + ```response Ok. Elapsed: 0.002 sec. ``` @@ -251,6 +274,7 @@ Ok. Elapsed: 0.002 sec. SELECT * FROM mergetree_table ``` + ```response ┌──n─┐ │ 42 │ @@ -267,6 +291,7 @@ SELECT * FROM system.transactions FORMAT Vertical ``` + ```response Row 1: ────── @@ -280,4 +305,3 @@ state: RUNNING ## More Details See this [meta issue](https://github.com/ClickHouse/ClickHouse/issues/48794) to find much more extensive tests and to keep up to date with the progress. - From 34d327a08cc2de20510eb429c2da30b4d135a3f0 Mon Sep 17 00:00:00 2001 From: Zach Naimon Date: Tue, 5 Mar 2024 14:26:57 -0500 Subject: [PATCH 008/102] Revert "update documentation" This reverts commit d596de73847c712d766e0f8598ce43ed078ec968. --- docs/en/sql-reference/transactions.md | 42 ++++++--------------------- 1 file changed, 9 insertions(+), 33 deletions(-) diff --git a/docs/en/sql-reference/transactions.md b/docs/en/sql-reference/transactions.md index b9c9afc20f9..cb89a091d68 100644 --- a/docs/en/sql-reference/transactions.md +++ b/docs/en/sql-reference/transactions.md @@ -1,29 +1,26 @@ --- slug: /en/guides/developer/transactional --- - # Transactional (ACID) support -## Case 1: INSERT into one partition, of one table, of the MergeTree\* family +## Case 1: INSERT into one partition, of one table, of the MergeTree* family This is transactional (ACID) if the inserted rows are packed and inserted as a single block (see Notes): - - Atomic: an INSERT succeeds or is rejected as a whole: if a confirmation is sent to the client, then all rows were inserted; if an error is sent to the client, then no rows were inserted. - Consistent: if there are no table constraints violated, then all rows in an INSERT are inserted and the INSERT succeeds; if constraints are violated, then no rows are inserted. - Isolated: concurrent clients observe a consistent snapshot of the table–the state of the table either as it was before the INSERT attempt, or after the successful INSERT; no partial state is seen - Durable: a successful INSERT is written to the filesystem before answering to the client, on a single replica or multiple replicas (controlled by the `insert_quorum` setting), and ClickHouse can ask the OS to sync the filesystem data on the storage media (controlled by the `fsync_after_insert` setting). - INSERT into multiple tables with one statement is possible if materialized views are involved (the INSERT from the client is to a table which has associate materialized views). -## Case 2: INSERT into multiple partitions, of one table, of the MergeTree\* family +## Case 2: INSERT into multiple partitions, of one table, of the MergeTree* family Same as Case 1 above, with this detail: - - If table has many partitions and INSERT covers many partitions–then insertion into every partition is transactional on its own -## Case 3: INSERT into one distributed table of the MergeTree\* family + +## Case 3: INSERT into one distributed table of the MergeTree* family Same as Case 1 above, with this detail: - - INSERT into Distributed table is not transactional as a whole, while insertion into every shard is transactional ## Case 4: Using a Buffer table @@ -33,11 +30,9 @@ Same as Case 1 above, with this detail: ## Case 5: Using async_insert Same as Case 1 above, with this detail: - - atomicity is ensured even if `async_insert` is enabled and `wait_for_async_insert` is set to 1 (the default), but if `wait_for_async_insert` is set to 0, then atomicity is not ensured. ## Notes - - rows inserted from the client in some data format are packed into a single block when: - the insert format is row-based (like CSV, TSV, Values, JSONEachRow, etc) and the data contains less then `max_insert_block_size` rows (~1 000 000 by default) or less then `min_chunk_bytes_for_parallel_parsing` bytes (10 MB by default) in case of parallel parsing is used (enabled by default) - the insert format is column-based (like Native, Parquet, ORC, etc) and the data contains only one block of data @@ -66,9 +61,8 @@ In addition to the functionality described at the top of this document, ClickHou ``` ### Notes - - This is an experimental feature, and changes should be expected. -- If an exception occurs during a transaction, you cannot commit the transaction. This includes all exceptions, including `UNKNOWN_FUNCTION` exceptions caused by typos. +- If an exception occurs during a transaction, you cannot commit the transaction. This includes all exceptions, including `UNKNOWN_FUNCTION` exceptions caused by typos. - Nested transactions are not supported; finish the current transaction and start a new one instead ### Configuration @@ -86,7 +80,7 @@ These examples are with a single node ClickHouse server with ClickHouse Keeper e #### Basic configuration for a single ClickHouse server node with ClickHouse Keeper enabled :::note -See the [deployment](docs/en/deployment-guides/terminology.md) documentation for details on deploying ClickHouse server and a proper quorum of ClickHouse Keeper nodes. The configuration shown here is for experimental purposes. +See the [deployment](docs/en/deployment-guides/terminology.md) documentation for details on deploying ClickHouse server and a proper quorum of ClickHouse Keeper nodes. The configuration shown here is for experimental purposes. ::: ```xml title=/etc/clickhouse-server/config.d/config.xml @@ -133,19 +127,17 @@ See the [deployment](docs/en/deployment-guides/terminology.md) documentation for #### Verify that experimental transactions are enabled -Issue a `BEGIN TRANSACTION` or `START TRANSACTION` followed by a `ROLLBACK` to verify that experimental transactions are enabled, and that ClickHouse Keeper is enabled as it is used to track transactions. +Issue a `BEGIN TRANSACTION` followed by a `ROLLBACK` to verify that experimental transactions are enabled, and that ClickHouse Keeper is enabled as it is used to track transactions. ```sql BEGIN TRANSACTION ``` - ```response Ok. ``` :::tip If you see the following error, then check your configuration file to make sure that `allow_experimental_transactions` is set to `1` (or any value other than `0` or `false`). - ``` Code: 48. DB::Exception: Received from localhost:9000. DB::Exception: Transactions are not supported. @@ -153,18 +145,15 @@ DB::Exception: Transactions are not supported. ``` You can also check ClickHouse Keeper by issuing - ``` echo ruok | nc localhost 9181 ``` - ClickHouse Keeper should respond with `imok`. ::: ```sql ROLLBACK ``` - ```response Ok. ``` @@ -172,7 +161,7 @@ Ok. #### Create a table for testing :::tip -Creation of tables is not transactional. Run this DDL query outside of a transaction. +Creation of tables is not transactional. Run this DDL query outside of a transaction. ::: ```sql @@ -183,7 +172,6 @@ CREATE TABLE mergetree_table ENGINE = MergeTree ORDER BY n ``` - ```response Ok. ``` @@ -193,7 +181,6 @@ Ok. ```sql BEGIN TRANSACTION ``` - ```response Ok. ``` @@ -201,7 +188,6 @@ Ok. ```sql INSERT INTO mergetree_table FORMAT Values (10) ``` - ```response Ok. ``` @@ -210,13 +196,11 @@ Ok. SELECT * FROM mergetree_table ``` - ```response ┌──n─┐ │ 10 │ └────┘ ``` - :::note You can query the table from within a transaction and see that the row was inserted even though it has not yet been committed. ::: @@ -224,20 +208,16 @@ You can query the table from within a transaction and see that the row was inser #### Rollback the transaction, and query the table again Verify that the transaction is rolled back: - ```sql ROLLBACK ``` - ```response Ok. ``` - ```sql SELECT * FROM mergetree_table ``` - ```response Ok. @@ -249,7 +229,6 @@ Ok. ```sql BEGIN TRANSACTION ``` - ```response Ok. ``` @@ -257,7 +236,6 @@ Ok. ```sql INSERT INTO mergetree_table FORMAT Values (42) ``` - ```response Ok. ``` @@ -265,7 +243,6 @@ Ok. ```sql COMMIT ``` - ```response Ok. Elapsed: 0.002 sec. ``` @@ -274,7 +251,6 @@ Ok. Elapsed: 0.002 sec. SELECT * FROM mergetree_table ``` - ```response ┌──n─┐ │ 42 │ @@ -291,7 +267,6 @@ SELECT * FROM system.transactions FORMAT Vertical ``` - ```response Row 1: ────── @@ -305,3 +280,4 @@ state: RUNNING ## More Details See this [meta issue](https://github.com/ClickHouse/ClickHouse/issues/48794) to find much more extensive tests and to keep up to date with the progress. + From f5cf61c32e9f511933b56048f22aaf43fad67a6c Mon Sep 17 00:00:00 2001 From: Zach Naimon Date: Tue, 5 Mar 2024 14:28:48 -0500 Subject: [PATCH 009/102] update documentation without autoformat --- docs/en/sql-reference/transactions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/transactions.md b/docs/en/sql-reference/transactions.md index cb89a091d68..09cdc192b03 100644 --- a/docs/en/sql-reference/transactions.md +++ b/docs/en/sql-reference/transactions.md @@ -127,7 +127,7 @@ See the [deployment](docs/en/deployment-guides/terminology.md) documentation for #### Verify that experimental transactions are enabled -Issue a `BEGIN TRANSACTION` followed by a `ROLLBACK` to verify that experimental transactions are enabled, and that ClickHouse Keeper is enabled as it is used to track transactions. +Issue a `BEGIN TRANSACTION` or `START TRANSACTION` followed by a `ROLLBACK` to verify that experimental transactions are enabled, and that ClickHouse Keeper is enabled as it is used to track transactions. ```sql BEGIN TRANSACTION From b074477ffb841587c19313063c249bc4c35ef301 Mon Sep 17 00:00:00 2001 From: Zach Naimon Date: Tue, 5 Mar 2024 14:45:09 -0500 Subject: [PATCH 010/102] use new syntax in a test --- tests/integration/test_transactions/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_transactions/test.py b/tests/integration/test_transactions/test.py index 46660581223..584e59ba71b 100644 --- a/tests/integration/test_transactions/test.py +++ b/tests/integration/test_transactions/test.py @@ -67,8 +67,8 @@ def test_rollback_unfinished_on_restart1(start_cluster): tx(1, "insert into mt values (5, 50)") tx(1, "alter table mt update m = m+n in partition id '1' where 1") - # check that uncommitted insert will be rolled back on restart - tx(3, "begin transaction") + # check that uncommitted insert will be rolled back on restart (using `START TRANSACTION syntax`) + tx(3, "start transaction") tid5 = tx(3, "select transactionID()").strip() tx(3, "insert into mt values (6, 70)") From 7ed1be6f3c51db3bd4abb57c429c77e09b0cd8a8 Mon Sep 17 00:00:00 2001 From: Zach Naimon Date: Tue, 5 Mar 2024 14:46:39 -0500 Subject: [PATCH 011/102] fix test comment --- tests/integration/test_transactions/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_transactions/test.py b/tests/integration/test_transactions/test.py index 584e59ba71b..d63b7b6f545 100644 --- a/tests/integration/test_transactions/test.py +++ b/tests/integration/test_transactions/test.py @@ -67,7 +67,7 @@ def test_rollback_unfinished_on_restart1(start_cluster): tx(1, "insert into mt values (5, 50)") tx(1, "alter table mt update m = m+n in partition id '1' where 1") - # check that uncommitted insert will be rolled back on restart (using `START TRANSACTION syntax`) + # check that uncommitted insert will be rolled back on restart (using `START TRANSACTION` syntax) tx(3, "start transaction") tid5 = tx(3, "select transactionID()").strip() tx(3, "insert into mt values (6, 70)") From 10b5ce8ab3d1b412f6500d03bc96e205965178d7 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 7 Mar 2024 10:26:50 +0100 Subject: [PATCH 012/102] Updated BufferAllocationPolicy --- src/Backups/BackupIO_AzureBlobStorage.cpp | 4 +-- .../BufferAllocationPolicy.cpp} | 25 +++++-------- src/Common/BufferAllocationPolicy.h | 35 +++++++++++++++++++ src/Common/ThreadPoolTaskTracker.h | 2 -- src/Core/Settings.h | 5 +++ .../IO/WriteBufferFromAzureBlobStorage.cpp | 23 +++++++----- .../IO/WriteBufferFromAzureBlobStorage.h | 9 ++--- .../AzureBlobStorage/AzureBlobStorageAuth.cpp | 29 ++++++++------- .../AzureBlobStorage/AzureObjectStorage.cpp | 4 +-- .../AzureBlobStorage/AzureObjectStorage.h | 17 +++++++-- src/IO/WriteBufferFromS3.cpp | 7 +++- src/IO/WriteBufferFromS3.h | 13 +------ 12 files changed, 107 insertions(+), 66 deletions(-) rename src/{IO/WriteBufferFromS3BufferAllocationPolicy.cpp => Common/BufferAllocationPolicy.cpp} (74%) create mode 100644 src/Common/BufferAllocationPolicy.h diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index fb36248433d..8d2b217ad21 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -275,11 +275,9 @@ std::unique_ptr BackupWriterAzureBlobStorage::writeFile(const Strin return std::make_unique( client, key, - settings->max_single_part_upload_size, - settings->max_unexpected_write_error_retries, DBMS_DEFAULT_BUFFER_SIZE, write_settings, - settings->max_inflight_parts_for_one_file); + settings); } void BackupWriterAzureBlobStorage::removeFile(const String & file_name) diff --git a/src/IO/WriteBufferFromS3BufferAllocationPolicy.cpp b/src/Common/BufferAllocationPolicy.cpp similarity index 74% rename from src/IO/WriteBufferFromS3BufferAllocationPolicy.cpp rename to src/Common/BufferAllocationPolicy.cpp index 6347c1acfd7..1456233eb03 100644 --- a/src/IO/WriteBufferFromS3BufferAllocationPolicy.cpp +++ b/src/Common/BufferAllocationPolicy.cpp @@ -1,21 +1,17 @@ -#include "config.h" - -#if USE_AWS_S3 - -#include +#include "BufferAllocationPolicy.h" #include -namespace +namespace DB { -class FixedSizeBufferAllocationPolicy : public DB::WriteBufferFromS3::IBufferAllocationPolicy +class FixedSizeBufferAllocationPolicy : public IBufferAllocationPolicy { const size_t buffer_size = 0; size_t buffer_number = 0; public: - explicit FixedSizeBufferAllocationPolicy(const DB::S3Settings::RequestSettings::PartUploadSettings & settings_) + explicit FixedSizeBufferAllocationPolicy(const BufferAllocationSettings & settings_) : buffer_size(settings_.strict_upload_part_size) { chassert(buffer_size > 0); @@ -36,7 +32,7 @@ public: }; -class ExpBufferAllocationPolicy : public DB::WriteBufferFromS3::IBufferAllocationPolicy +class ExpBufferAllocationPolicy : public DB::IBufferAllocationPolicy { const size_t first_size = 0; const size_t second_size = 0; @@ -49,7 +45,7 @@ class ExpBufferAllocationPolicy : public DB::WriteBufferFromS3::IBufferAllocatio size_t buffer_number = 0; public: - explicit ExpBufferAllocationPolicy(const DB::S3Settings::RequestSettings::PartUploadSettings & settings_) + explicit ExpBufferAllocationPolicy(const BufferAllocationSettings & settings_) : first_size(std::max(settings_.max_single_part_upload_size, settings_.min_upload_part_size)) , second_size(settings_.min_upload_part_size) , multiply_factor(settings_.upload_part_size_multiply_factor) @@ -92,14 +88,10 @@ public: } }; -} -namespace DB -{ +IBufferAllocationPolicy::~IBufferAllocationPolicy() = default; -WriteBufferFromS3::IBufferAllocationPolicy::~IBufferAllocationPolicy() = default; - -WriteBufferFromS3::IBufferAllocationPolicyPtr WriteBufferFromS3::ChooseBufferPolicy(const S3Settings::RequestSettings::PartUploadSettings & settings_) +IBufferAllocationPolicyPtr ChooseBufferPolicy(BufferAllocationSettings settings_) { if (settings_.strict_upload_part_size > 0) return std::make_unique(settings_); @@ -109,4 +101,3 @@ WriteBufferFromS3::IBufferAllocationPolicyPtr WriteBufferFromS3::ChooseBufferPol } -#endif diff --git a/src/Common/BufferAllocationPolicy.h b/src/Common/BufferAllocationPolicy.h new file mode 100644 index 00000000000..b759d22ede6 --- /dev/null +++ b/src/Common/BufferAllocationPolicy.h @@ -0,0 +1,35 @@ +#pragma once + +#include "config.h" + +#include "logger_useful.h" + +#include + +namespace DB +{ + +struct BufferAllocationSettings +{ + size_t strict_upload_part_size = 0; + size_t min_upload_part_size = 16 * 1024 * 1024; + size_t max_upload_part_size = 5ULL * 1024 * 1024 * 1024; + size_t upload_part_size_multiply_factor = 2; + size_t upload_part_size_multiply_parts_count_threshold = 500; + size_t max_single_part_upload_size = 32 * 1024 * 1024; +}; + +class IBufferAllocationPolicy +{ + public: + virtual size_t getBufferNumber() const = 0; + virtual size_t getBufferSize() const = 0; + virtual void nextBuffer() = 0; + virtual ~IBufferAllocationPolicy() = 0; +}; + +using IBufferAllocationPolicyPtr = std::unique_ptr; + +IBufferAllocationPolicyPtr ChooseBufferPolicy(BufferAllocationSettings settings_); + +} diff --git a/src/Common/ThreadPoolTaskTracker.h b/src/Common/ThreadPoolTaskTracker.h index d37b759a913..72591648d30 100644 --- a/src/Common/ThreadPoolTaskTracker.h +++ b/src/Common/ThreadPoolTaskTracker.h @@ -11,8 +11,6 @@ namespace DB { -/// That class is used only in WriteBufferFromS3 for now. -/// Therefore it declared as a part of WriteBufferFromS3. /// TaskTracker takes a Callback which is run by scheduler in some external shared ThreadPool. /// TaskTracker brings the methods waitIfAny, waitAll/safeWaitAll /// to help with coordination of the running tasks. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b186ca6fe01..8ad08b7e348 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -78,10 +78,15 @@ class IColumn; M(UInt64, distributed_connections_pool_size, 1024, "Maximum number of connections with one remote server in the pool.", 0) \ M(UInt64, connections_with_failover_max_tries, 3, "The maximum number of attempts to connect to replicas.", 0) \ M(UInt64, s3_strict_upload_part_size, 0, "The exact size of part to upload during multipart upload to S3 (some implementations does not supports variable size parts).", 0) \ + M(UInt64, azure_strict_upload_part_size, 0, "The exact size of part to upload during multipart upload to Azure blob storage.", 0) \ M(UInt64, s3_min_upload_part_size, 16*1024*1024, "The minimum size of part to upload during multipart upload to S3.", 0) \ M(UInt64, s3_max_upload_part_size, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to S3.", 0) \ + M(UInt64, azure_min_upload_part_size, 16*1024*1024, "The minimum size of part to upload during multipart upload to Azure blob storage.", 0) \ + M(UInt64, azure_max_upload_part_size, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to Azure blob storage.", 0) \ M(UInt64, s3_upload_part_size_multiply_factor, 2, "Multiply s3_min_upload_part_size by this factor each time s3_multiply_parts_count_threshold parts were uploaded from a single write to S3.", 0) \ M(UInt64, s3_upload_part_size_multiply_parts_count_threshold, 500, "Each time this number of parts was uploaded to S3, s3_min_upload_part_size is multiplied by s3_upload_part_size_multiply_factor.", 0) \ + M(UInt64, azure_upload_part_size_multiply_factor, 2, "Multiply azure_min_upload_part_size by this factor each time azure_multiply_parts_count_threshold parts were uploaded from a single write to Azure blob storage.", 0) \ + M(UInt64, azure_upload_part_size_multiply_parts_count_threshold, 500, "Each time this number of parts was uploaded to Azure blob storage, azure_min_upload_part_size is multiplied by azure_upload_part_size_multiply_factor.", 0) \ M(UInt64, s3_max_inflight_parts_for_one_file, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited.", 0) \ M(UInt64, azure_max_inflight_parts_for_one_file, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited.", 0) \ M(UInt64, s3_max_single_part_upload_size, 32*1024*1024, "The maximum size of object to upload using singlepart upload to S3.", 0) \ diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 74a8949b235..bc11d445a51 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -27,23 +27,27 @@ struct WriteBufferFromAzureBlobStorage::PartData WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage( std::shared_ptr blob_container_client_, const String & blob_path_, - size_t max_single_part_upload_size_, - size_t max_unexpected_write_error_retries_, size_t buf_size_, const WriteSettings & write_settings_, - size_t max_inflight_parts_for_one_file_, + std::shared_ptr settings_, ThreadPoolCallbackRunner schedule_) : WriteBufferFromFileBase(buf_size_, nullptr, 0) , log(getLogger("WriteBufferFromAzureBlobStorage")) - , max_single_part_upload_size(max_single_part_upload_size_) - , max_unexpected_write_error_retries(max_unexpected_write_error_retries_) + , buffer_allocation_policy(ChooseBufferPolicy({settings_->strict_upload_part_size, + settings_->min_upload_part_size, + settings_->max_upload_part_size, + settings_->upload_part_size_multiply_factor, + settings_->upload_part_size_multiply_parts_count_threshold, + settings_->max_single_part_upload_size})) + , max_single_part_upload_size(settings_->max_single_part_upload_size) + , max_unexpected_write_error_retries(settings_->max_unexpected_write_error_retries) , blob_path(blob_path_) , write_settings(write_settings_) , blob_container_client(blob_container_client_) , task_tracker( std::make_unique( std::move(schedule_), - max_inflight_parts_for_one_file_, + settings_->max_inflight_parts_for_one_file, limitedLog)) { allocateBuffer(); @@ -119,7 +123,8 @@ void WriteBufferFromAzureBlobStorage::nextImpl() void WriteBufferFromAzureBlobStorage::allocateBuffer() { - memory = Memory(max_single_part_upload_size); + buffer_allocation_policy->nextBuffer(); + memory = Memory(buffer_allocation_policy->getBufferSize()); WriteBuffer::set(memory.data(), memory.size()); } @@ -129,10 +134,10 @@ void WriteBufferFromAzureBlobStorage::reallocateBuffer() if (available() > 0) return; - if (memory.size() == max_single_part_upload_size) + if (memory.size() == buffer_allocation_policy->getBufferSize()) return; - memory.resize(max_single_part_upload_size); + memory.resize(buffer_allocation_policy->getBufferSize()); WriteBuffer::set(memory.data(), memory.size()); diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h index 4897ca9a846..7223f66693e 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h @@ -12,7 +12,8 @@ #include #include #include - +#include +#include namespace Poco { @@ -32,11 +33,9 @@ public: WriteBufferFromAzureBlobStorage( AzureClientPtr blob_container_client_, const String & blob_path_, - size_t max_single_part_upload_size_, - size_t max_unexpected_write_error_retries_, size_t buf_size_, const WriteSettings & write_settings_, - size_t max_inflight_parts_for_one_file_, + std::shared_ptr settings_, ThreadPoolCallbackRunner schedule_ = {}); ~WriteBufferFromAzureBlobStorage() override; @@ -63,6 +62,8 @@ private: LoggerPtr log; LogSeriesLimiterPtr limitedLog = std::make_shared(log, 1, 5); + IBufferAllocationPolicyPtr buffer_allocation_policy; + const size_t max_single_part_upload_size; const size_t max_unexpected_write_error_retries; const std::string blob_path; diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp index f99586b2d1a..1d01e2f45e3 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -160,18 +160,23 @@ std::unique_ptr getAzureBlobContainerClient( std::unique_ptr getAzureBlobStorageSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context) { - return std::make_unique( - config.getUInt64(config_prefix + ".max_single_part_upload_size", 100 * 1024 * 1024), - config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024), - config.getInt(config_prefix + ".max_single_read_retries", 3), - config.getInt(config_prefix + ".max_single_download_retries", 3), - config.getInt(config_prefix + ".list_object_keys_size", 1000), - config.getUInt64(config_prefix + ".max_upload_part_size", 5ULL * 1024 * 1024 * 1024), - config.getUInt64(config_prefix + ".max_single_part_copy_size", context->getSettings().azure_max_single_part_copy_size), - config.getBool(config_prefix + ".use_native_copy", false), - config.getUInt64(config_prefix + ".max_unexpected_write_error_retries", context->getSettings().azure_max_unexpected_write_error_retries), - config.getUInt64(config_prefix + ".max_inflight_parts_for_one_file", context->getSettings().azure_max_inflight_parts_for_one_file) - ); + std::unique_ptr settings = std::make_unique(); + settings->max_single_part_upload_size = config.getUInt64(config_prefix + ".max_single_part_upload_size", 100 * 1024 * 1024); + settings->min_bytes_for_seek = config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024); + settings->max_single_read_retries = config.getInt(config_prefix + ".max_single_read_retries", 3); + settings->max_single_download_retries = config.getInt(config_prefix + ".max_single_download_retries", 3); + settings->list_object_keys_size = config.getInt(config_prefix + ".list_object_keys_size", 1000); + settings->min_upload_part_size = config.getUInt64(config_prefix + ".min_upload_part_size", context->getSettings().azure_min_upload_part_size); + settings->max_upload_part_size = config.getUInt64(config_prefix + ".max_upload_part_size", context->getSettings().azure_max_upload_part_size); + settings->max_single_part_copy_size = config.getUInt64(config_prefix + ".max_single_part_copy_size", context->getSettings().azure_max_single_part_copy_size); + settings->use_native_copy = config.getBool(config_prefix + ".use_native_copy", false); + settings->max_unexpected_write_error_retries = config.getUInt64(config_prefix + ".max_unexpected_write_error_retries", context->getSettings().azure_max_unexpected_write_error_retries); + settings->max_inflight_parts_for_one_file = config.getUInt64(config_prefix + ".max_inflight_parts_for_one_file", context->getSettings().azure_max_inflight_parts_for_one_file); + settings->strict_upload_part_size = config.getUInt64(config_prefix + ".strict_upload_part_size", context->getSettings().azure_strict_upload_part_size); + settings->upload_part_size_multiply_factor = config.getUInt64(config_prefix + ".upload_part_size_multiply_factor", context->getSettings().azure_upload_part_size_multiply_factor); + settings->upload_part_size_multiply_parts_count_threshold = config.getUInt64(config_prefix + ".upload_part_size_multiply_parts_count_threshold", context->getSettings().azure_upload_part_size_multiply_parts_count_threshold); + + return settings; } } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 844789ea5b5..15ab55d5611 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -265,11 +265,9 @@ std::unique_ptr AzureObjectStorage::writeObject( /// NO return std::make_unique( client.get(), object.remote_path, - settings.get()->max_single_part_upload_size, - settings.get()->max_unexpected_write_error_retries, buf_size, patchSettings(write_settings), - settings.get()->max_inflight_parts_for_one_file); + settings.get()); } /// Remove file. Throws exception if file doesn't exists or it's a directory. diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 1b473a01304..b97d706a4d9 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -24,21 +24,29 @@ struct AzureObjectStorageSettings int max_single_read_retries_, int max_single_download_retries_, int list_object_keys_size_, + size_t min_upload_part_size_, size_t max_upload_part_size_, size_t max_single_part_copy_size_, bool use_native_copy_, size_t max_unexpected_write_error_retries_, - size_t max_inflight_parts_for_one_file_) + size_t max_inflight_parts_for_one_file_, + size_t strict_upload_part_size_, + size_t upload_part_size_multiply_factor_, + size_t upload_part_size_multiply_parts_count_threshold_) : max_single_part_upload_size(max_single_part_upload_size_) , min_bytes_for_seek(min_bytes_for_seek_) , max_single_read_retries(max_single_read_retries_) , max_single_download_retries(max_single_download_retries_) , list_object_keys_size(list_object_keys_size_) + , min_upload_part_size(min_upload_part_size_) , max_upload_part_size(max_upload_part_size_) , max_single_part_copy_size(max_single_part_copy_size_) , use_native_copy(use_native_copy_) - , max_unexpected_write_error_retries (max_unexpected_write_error_retries_) - , max_inflight_parts_for_one_file (max_inflight_parts_for_one_file_) + , max_unexpected_write_error_retries(max_unexpected_write_error_retries_) + , max_inflight_parts_for_one_file(max_inflight_parts_for_one_file_) + , strict_upload_part_size(strict_upload_part_size_) + , upload_part_size_multiply_factor(upload_part_size_multiply_factor_) + , upload_part_size_multiply_parts_count_threshold(upload_part_size_multiply_parts_count_threshold_) { } @@ -55,6 +63,9 @@ struct AzureObjectStorageSettings bool use_native_copy = false; size_t max_unexpected_write_error_retries = 4; size_t max_inflight_parts_for_one_file = 20; + size_t strict_upload_part_size = 0; + size_t upload_part_size_multiply_factor = 2; + size_t upload_part_size_multiply_parts_count_threshold = 500; }; using AzureClient = Azure::Storage::Blobs::BlobContainerClient; diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 510d9bef4d3..60fa828d6c4 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -93,7 +93,12 @@ WriteBufferFromS3::WriteBufferFromS3( , write_settings(write_settings_) , client_ptr(std::move(client_ptr_)) , object_metadata(std::move(object_metadata_)) - , buffer_allocation_policy(ChooseBufferPolicy(upload_settings)) + , buffer_allocation_policy(ChooseBufferPolicy({upload_settings.strict_upload_part_size, + upload_settings.min_upload_part_size, + upload_settings.max_upload_part_size, + upload_settings.upload_part_size_multiply_factor, + upload_settings.upload_part_size_multiply_parts_count_threshold, + upload_settings.max_single_part_upload_size})) , task_tracker( std::make_unique( std::move(schedule_), diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index afd8b9909c1..840274c8ace 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -13,6 +13,7 @@ #include #include #include +#include #include #include @@ -49,18 +50,6 @@ public: std::string getFileName() const override { return key; } void sync() override { next(); } - class IBufferAllocationPolicy - { - public: - virtual size_t getBufferNumber() const = 0; - virtual size_t getBufferSize() const = 0; - virtual void nextBuffer() = 0; - virtual ~IBufferAllocationPolicy() = 0; - }; - using IBufferAllocationPolicyPtr = std::unique_ptr; - - static IBufferAllocationPolicyPtr ChooseBufferPolicy(const S3Settings::RequestSettings::PartUploadSettings & settings_); - private: /// Receives response from the server after sending all data. void finalizeImpl() override; From 8a11afeba1e80f3f0210af5d19338550f631ceab Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 11 Mar 2024 11:44:34 +0100 Subject: [PATCH 013/102] Updated settings changed history --- src/Core/SettingsChangesHistory.h | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index b68789d5f43..d458f935edf 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -93,6 +93,12 @@ static std::map sett {"input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects", false, false, "Allow to use String type for ambiguous paths during named tuple inference from JSON objects"}, {"throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert", false, true, "Deduplication is dependent materialized view cannot work together with async inserts."}, {"parallel_replicas_allow_in_with_subquery", false, true, "If true, subquery for IN will be executed on every follower replica"}, + {"azure_max_inflight_parts_for_one_file", 20, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited."}, + {"azure_strict_upload_part_size", 0, 0, "The exact size of part to upload during multipart upload to Azure blob storage."}, + {"azure_min_upload_part_size", 16*1024*1024, 16*1024*1024, "The minimum size of part to upload during multipart upload to Azure blob storage."}, + {"azure_max_upload_part_size", 5ull*1024*1024*1024, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to Azure blob storage."}, + {"azure_upload_part_size_multiply_factor", 2, 2, "Multiply azure_min_upload_part_size by this factor each time azure_multiply_parts_count_threshold parts were uploaded from a single write to Azure blob storage."}, + {"azure_upload_part_size_multiply_parts_count_threshold", 500, 500, "Each time this number of parts was uploaded to Azure blob storage, azure_min_upload_part_size is multiplied by azure_upload_part_size_multiply_factor."}, }}, {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, @@ -122,7 +128,6 @@ static std::map sett {"default_normal_view_sql_security", "INVOKER", "INVOKER", "Allows to set default `SQL SECURITY` option while creating a normal view"}, {"mysql_map_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, {"mysql_map_fixed_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - {"azure_max_inflight_parts_for_one_file", 20, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited."}, }}, {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, From 12289a9ebe07ce15937ceb4cffe8fe4cf033dcdd Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sun, 17 Mar 2024 22:34:41 +0100 Subject: [PATCH 014/102] Add support of ParallelReadBuffer to ReadBufferFromAzureBlobStorage --- .../IO/ReadBufferFromAzureBlobStorage.cpp | 51 +++++++++++++++++++ src/Disks/IO/ReadBufferFromAzureBlobStorage.h | 4 ++ 2 files changed, 55 insertions(+) diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index 8de977ef876..7cda8898a65 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -8,7 +8,9 @@ #include #include #include +#include +#include namespace ProfileEvents { @@ -262,6 +264,55 @@ size_t ReadBufferFromAzureBlobStorage::getFileSize() return *file_size; } +size_t ReadBufferFromAzureBlobStorage::readBigAt(char * to, size_t n, size_t range_begin, const std::function & progress_callback) const +{ + size_t initial_n = n; + + size_t sleep_time_with_backoff_milliseconds = 100; + auto handle_exception = [&, this](const auto & e, size_t i) + { + LOG_INFO(log, "Exception caught during Azure Download for file {} at offset {} at attempt {}/{}: {}", path, offset, i + 1, max_single_download_retries, e.Message); + if (i + 1 == max_single_download_retries) + throw; + + sleepForMilliseconds(sleep_time_with_backoff_milliseconds); + sleep_time_with_backoff_milliseconds *= 2; + }; + + Azure::Storage::Blobs::DownloadBlobOptions download_options; + download_options.Range = {static_cast(range_begin), range_begin+n}; + + for (size_t i = 0; i < max_single_download_retries; ++i) + { + size_t bytes_copied = 0; + try + { + auto download_response = blob_client->Download(download_options); + std::unique_ptr body_stream = std::move(download_response.Value.BodyStream); + auto length = body_stream->Length(); + char buffer[length]; + body_stream->Read(reinterpret_cast(buffer), length); + std::istringstream string_stream(String(static_cast(buffer),length)); + copyFromIStreamWithProgressCallback(string_stream, to, n, progress_callback, &bytes_copied); + + if (read_settings.remote_throttler) + read_settings.remote_throttler->add(bytes_copied, ProfileEvents::RemoteReadThrottlerBytes, ProfileEvents::RemoteReadThrottlerSleepMicroseconds); + + break; + } + catch (const Azure::Core::RequestFailedException & e) + { + handle_exception(e,i); + } + + range_begin += bytes_copied; + to += bytes_copied; + n -= bytes_copied; + } + + return initial_n; +} + } #endif diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.h b/src/Disks/IO/ReadBufferFromAzureBlobStorage.h index 68058b84a2f..d328195cc26 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.h +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.h @@ -44,6 +44,10 @@ public: size_t getFileSize() override; + size_t readBigAt(char * to, size_t n, size_t range_begin, const std::function & progress_callback) const override; + + bool supportsReadAt() override { return true; } + private: void initialize(); From bf85f7f368c09a11946ccba02d11b55ef71ffdf4 Mon Sep 17 00:00:00 2001 From: Zhuo Qiu Date: Mon, 18 Mar 2024 11:02:02 +0800 Subject: [PATCH 015/102] add docs & minor fix for lwd-aware merge settings --- .../settings/merge-tree-settings.md | 28 +++++++++++++++++++ src/Storages/MergeTree/IMergeTreeDataPart.cpp | 5 ++-- src/Storages/MergeTree/IMergeTreeDataPart.h | 3 +- 3 files changed, 33 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index c7e461d15ae..9ce705ff805 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -867,3 +867,31 @@ Default value: `Never` Persists virtual column `_block_number` on merges. Default value: false. + +## exclude_deleted_rows_for_part_size_in_merge {#exclude_deleted_rows_for_part_size_in_merge} + +If enabled, estimated actual size of data parts (i.e., excluding those rows that have been deleted through `DELETE FROM`) will be used when selecting parts to merge. Note that this behavior is only triggered for data parts affected by `DELETE FROM` executed after this setting is enabled. + +Possible values: + +- true, false + +Default value: false + +**See Also** + +- [load_existing_rows_count_for_old_parts](#load_existing_rows_count_for_old_parts) setting + +## load_existing_rows_count_for_old_parts {#load_existing_rows_count_for_old_parts} + +If enabled along with [exclude_deleted_rows_for_part_size_in_merge](#exclude_deleted_rows_for_part_size_in_merge), deleted rows count for existing data parts will be calculated during table starting up. + +Possible values: + +- true, false + +Default value: false + +**See Also** + +- [exclude_deleted_rows_for_part_size_in_merge](#exclude_deleted_rows_for_part_size_in_merge) setting diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 023202019e4..3ae3b678f37 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1349,8 +1349,9 @@ void IMergeTreeDataPart::loadExistingRowsCount() if (existing_rows_count.has_value()) return; - if (!rows_count || !storage.getSettings()->load_existing_rows_count_for_old_parts || !supportLightweightDeleteMutate() - || !hasLightweightDelete()) + if (!rows_count || !supportLightweightDeleteMutate() || !hasLightweightDelete() + || !storage.getSettings()->exclude_deleted_rows_for_part_size_in_merge + || !storage.getSettings()->load_existing_rows_count_for_old_parts) existing_rows_count = rows_count; else existing_rows_count = readExistingRowsCount(); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 8bd32e777bc..075a5c45710 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -674,7 +674,8 @@ private: /// For the older format version calculates rows count from the size of a column with a fixed size. void loadRowsCount(); - /// Load existing rows count from _row_exists column if load_existing_rows_count_for_old_parts is true. + /// Load existing rows count from _row_exists column + /// if load_existing_rows_count_for_old_parts and exclude_deleted_rows_for_part_size_in_merge are both enabled. void loadExistingRowsCount(); static void appendFilesOfRowsCount(Strings & files); From 248a8a60f1722414523a4ce19977cf7a68c636a6 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 18 Mar 2024 10:02:52 +0100 Subject: [PATCH 016/102] Fix style check --- src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index 7cda8898a65..13f62c7ac25 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -292,7 +292,7 @@ size_t ReadBufferFromAzureBlobStorage::readBigAt(char * to, size_t n, size_t ran auto length = body_stream->Length(); char buffer[length]; body_stream->Read(reinterpret_cast(buffer), length); - std::istringstream string_stream(String(static_cast(buffer),length)); + std::istringstream string_stream(String(static_cast(buffer),length)); // STYLE_CHECK_ALLOW_STD_STRING_STREAM copyFromIStreamWithProgressCallback(string_stream, to, n, progress_callback, &bytes_copied); if (read_settings.remote_throttler) From 62c91fdddc8c0da90b8ef14ecd5a0f8c442073d3 Mon Sep 17 00:00:00 2001 From: Zhuo Qiu Date: Mon, 18 Mar 2024 22:33:07 +0800 Subject: [PATCH 017/102] revise docs Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> --- docs/en/operations/settings/merge-tree-settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 9ce705ff805..3e411a51ff4 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -884,7 +884,7 @@ Default value: false ## load_existing_rows_count_for_old_parts {#load_existing_rows_count_for_old_parts} -If enabled along with [exclude_deleted_rows_for_part_size_in_merge](#exclude_deleted_rows_for_part_size_in_merge), deleted rows count for existing data parts will be calculated during table starting up. +If enabled along with [exclude_deleted_rows_for_part_size_in_merge](#exclude_deleted_rows_for_part_size_in_merge), deleted rows count for existing data parts will be calculated during table starting up. Note that it may slow down start up table loading. Possible values: From 0530055842a14468218d774d020baf0153abfea7 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 18 Mar 2024 15:59:05 +0100 Subject: [PATCH 018/102] Updated names in BufferAllocationPolicy --- src/Common/BufferAllocationPolicy.cpp | 6 +-- src/Common/BufferAllocationPolicy.h | 41 ++++++++++--------- .../IO/WriteBufferFromAzureBlobStorage.cpp | 12 +++--- .../IO/WriteBufferFromAzureBlobStorage.h | 2 +- src/IO/WriteBufferFromS3.cpp | 12 +++--- src/IO/WriteBufferFromS3.h | 2 +- 6 files changed, 39 insertions(+), 36 deletions(-) diff --git a/src/Common/BufferAllocationPolicy.cpp b/src/Common/BufferAllocationPolicy.cpp index 1456233eb03..2b00d4f84d0 100644 --- a/src/Common/BufferAllocationPolicy.cpp +++ b/src/Common/BufferAllocationPolicy.cpp @@ -11,7 +11,7 @@ class FixedSizeBufferAllocationPolicy : public IBufferAllocationPolicy size_t buffer_number = 0; public: - explicit FixedSizeBufferAllocationPolicy(const BufferAllocationSettings & settings_) + explicit FixedSizeBufferAllocationPolicy(const IBufferAllocationPolicy::Settings & settings_) : buffer_size(settings_.strict_upload_part_size) { chassert(buffer_size > 0); @@ -45,7 +45,7 @@ class ExpBufferAllocationPolicy : public DB::IBufferAllocationPolicy size_t buffer_number = 0; public: - explicit ExpBufferAllocationPolicy(const BufferAllocationSettings & settings_) + explicit ExpBufferAllocationPolicy(const IBufferAllocationPolicy::Settings & settings_) : first_size(std::max(settings_.max_single_part_upload_size, settings_.min_upload_part_size)) , second_size(settings_.min_upload_part_size) , multiply_factor(settings_.upload_part_size_multiply_factor) @@ -91,7 +91,7 @@ public: IBufferAllocationPolicy::~IBufferAllocationPolicy() = default; -IBufferAllocationPolicyPtr ChooseBufferPolicy(BufferAllocationSettings settings_) +IBufferAllocationPolicy::IBufferAllocationPolicyPtr IBufferAllocationPolicy::create(IBufferAllocationPolicy::Settings settings_) { if (settings_.strict_upload_part_size > 0) return std::make_unique(settings_); diff --git a/src/Common/BufferAllocationPolicy.h b/src/Common/BufferAllocationPolicy.h index b759d22ede6..3ada9e52fdf 100644 --- a/src/Common/BufferAllocationPolicy.h +++ b/src/Common/BufferAllocationPolicy.h @@ -9,27 +9,30 @@ namespace DB { -struct BufferAllocationSettings -{ - size_t strict_upload_part_size = 0; - size_t min_upload_part_size = 16 * 1024 * 1024; - size_t max_upload_part_size = 5ULL * 1024 * 1024 * 1024; - size_t upload_part_size_multiply_factor = 2; - size_t upload_part_size_multiply_parts_count_threshold = 500; - size_t max_single_part_upload_size = 32 * 1024 * 1024; -}; - class IBufferAllocationPolicy { - public: - virtual size_t getBufferNumber() const = 0; - virtual size_t getBufferSize() const = 0; - virtual void nextBuffer() = 0; - virtual ~IBufferAllocationPolicy() = 0; +public: + + struct Settings + { + size_t strict_upload_part_size = 0; + size_t min_upload_part_size = 16 * 1024 * 1024; + size_t max_upload_part_size = 5ULL * 1024 * 1024 * 1024; + size_t upload_part_size_multiply_factor = 2; + size_t upload_part_size_multiply_parts_count_threshold = 500; + size_t max_single_part_upload_size = 32 * 1024 * 1024; + }; + + + virtual size_t getBufferNumber() const = 0; + virtual size_t getBufferSize() const = 0; + virtual void nextBuffer() = 0; + virtual ~IBufferAllocationPolicy() = 0; + + using IBufferAllocationPolicyPtr = std::unique_ptr; + + static IBufferAllocationPolicyPtr create(Settings settings_); + }; -using IBufferAllocationPolicyPtr = std::unique_ptr; - -IBufferAllocationPolicyPtr ChooseBufferPolicy(BufferAllocationSettings settings_); - } diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index bc11d445a51..41d7187afda 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -33,12 +33,12 @@ WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage( ThreadPoolCallbackRunner schedule_) : WriteBufferFromFileBase(buf_size_, nullptr, 0) , log(getLogger("WriteBufferFromAzureBlobStorage")) - , buffer_allocation_policy(ChooseBufferPolicy({settings_->strict_upload_part_size, - settings_->min_upload_part_size, - settings_->max_upload_part_size, - settings_->upload_part_size_multiply_factor, - settings_->upload_part_size_multiply_parts_count_threshold, - settings_->max_single_part_upload_size})) + , buffer_allocation_policy(IBufferAllocationPolicy::create({settings_->strict_upload_part_size, + settings_->min_upload_part_size, + settings_->max_upload_part_size, + settings_->upload_part_size_multiply_factor, + settings_->upload_part_size_multiply_parts_count_threshold, + settings_->max_single_part_upload_size})) , max_single_part_upload_size(settings_->max_single_part_upload_size) , max_unexpected_write_error_retries(settings_->max_unexpected_write_error_retries) , blob_path(blob_path_) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h index 7223f66693e..f047dd53724 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h @@ -62,7 +62,7 @@ private: LoggerPtr log; LogSeriesLimiterPtr limitedLog = std::make_shared(log, 1, 5); - IBufferAllocationPolicyPtr buffer_allocation_policy; + IBufferAllocationPolicy::IBufferAllocationPolicyPtr buffer_allocation_policy; const size_t max_single_part_upload_size; const size_t max_unexpected_write_error_retries; diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 60fa828d6c4..c907c4cb6b6 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -93,12 +93,12 @@ WriteBufferFromS3::WriteBufferFromS3( , write_settings(write_settings_) , client_ptr(std::move(client_ptr_)) , object_metadata(std::move(object_metadata_)) - , buffer_allocation_policy(ChooseBufferPolicy({upload_settings.strict_upload_part_size, - upload_settings.min_upload_part_size, - upload_settings.max_upload_part_size, - upload_settings.upload_part_size_multiply_factor, - upload_settings.upload_part_size_multiply_parts_count_threshold, - upload_settings.max_single_part_upload_size})) + , buffer_allocation_policy(IBufferAllocationPolicy::create({upload_settings.strict_upload_part_size, + upload_settings.min_upload_part_size, + upload_settings.max_upload_part_size, + upload_settings.upload_part_size_multiply_factor, + upload_settings.upload_part_size_multiply_parts_count_threshold, + upload_settings.max_single_part_upload_size})) , task_tracker( std::make_unique( std::move(schedule_), diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 840274c8ace..201743e7f9c 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -86,7 +86,7 @@ private: LoggerPtr log = getLogger("WriteBufferFromS3"); LogSeriesLimiterPtr limitedLog = std::make_shared(log, 1, 5); - IBufferAllocationPolicyPtr buffer_allocation_policy; + IBufferAllocationPolicy::IBufferAllocationPolicyPtr buffer_allocation_policy; /// Upload in S3 is made in parts. /// We initiate upload, then upload each part and get ETag as a response, and then finalizeImpl() upload with listing all our parts. From 69ef7c6f19d4d383fe8fdb7f510147e376bff016 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 18 Mar 2024 18:15:54 +0000 Subject: [PATCH 019/102] add logging --- src/Common/AsyncLoader.cpp | 35 ++++++++++++++++++++++--- src/Common/AsyncLoader.h | 4 ++- src/Common/tests/gtest_async_loader.cpp | 2 +- src/Interpreters/Context.cpp | 3 ++- 4 files changed, 37 insertions(+), 7 deletions(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 4a39454ccbb..e4896ce32ab 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -196,9 +197,10 @@ void LoadTask::remove() } } -AsyncLoader::AsyncLoader(std::vector pool_initializers, bool log_failures_, bool log_progress_) +AsyncLoader::AsyncLoader(std::vector pool_initializers, bool log_failures_, bool log_progress_, bool log_events_) : log_failures(log_failures_) , log_progress(log_progress_) + , log_events(log_events_) , log(getLogger("AsyncLoader")) { pools.reserve(pool_initializers.size()); @@ -327,6 +329,8 @@ void AsyncLoader::schedule(const LoadJobSet & jobs_to_schedule) ALLOW_ALLOCATIONS_IN_SCOPE; scheduled_jobs.try_emplace(job); job->scheduled(++last_job_id); + if (log_events) + LOG_DEBUG(log, "Schedule load job '{}' into {}", job->name, getPoolName(job->pool())); }); } @@ -582,6 +586,9 @@ void AsyncLoader::finish(const LoadJobPtr & job, LoadStatus status, std::excepti else if (status == LoadStatus::CANCELED) job->canceled(reason); + if (log_events) + LOG_DEBUG(log, "Finish load job '{}' with status {}", job->name, magic_enum::enum_name(status)); + Info & info = scheduled_jobs[job]; if (info.isReady()) { @@ -661,6 +668,9 @@ void AsyncLoader::prioritize(const LoadJobPtr & job, size_t new_pool_id, std::un job->pool_id.store(new_pool_id); + if (log_events) + LOG_DEBUG(log, "Prioritize load job '{}': {} -> {}", job->name, old_pool.name, new_pool.name); + // Recurse into dependencies for (const auto & dep : job->dependencies) prioritize(dep, new_pool_id, lock); @@ -765,6 +775,9 @@ void AsyncLoader::wait(std::unique_lock & job_lock, const LoadJobPtr if (job->load_status != LoadStatus::PENDING) // Shortcut just to avoid incrementing ProfileEvents return; + if (log_events) + LOG_DEBUG(log, "Wait load job '{}' in {}", job->name, getPoolName(job->executionPool())); + Stopwatch watch; job->waiters++; job->finished.wait(job_lock, [&] { return job->load_status != LoadStatus::PENDING; }); @@ -789,6 +802,13 @@ bool AsyncLoader::canWorkerLive(Pool & pool, std::unique_lock &) && (!current_priority || *current_priority >= pool.priority); } +void AsyncLoader::setCurrentPriority(std::unique_lock &, std::optional priority) +{ + if (log_event && current_priority != priority) + LOG_DEBUG(log, "Change current priority: {} -> {}", current_priority, priority); + current_priority = priority; +} + void AsyncLoader::updateCurrentPriorityAndSpawn(std::unique_lock & lock) { // Find current priority. @@ -799,7 +819,7 @@ void AsyncLoader::updateCurrentPriorityAndSpawn(std::unique_lock & l if (pool.isActive() && (!priority || *priority > pool.priority)) priority = pool.priority; } - current_priority = priority; + setCurrentPriority(lock, priority); // Spawn workers in all pools with current priority for (Pool & pool : pools) @@ -809,10 +829,12 @@ void AsyncLoader::updateCurrentPriorityAndSpawn(std::unique_lock & l } } -void AsyncLoader::spawn(Pool & pool, std::unique_lock &) +void AsyncLoader::spawn(Pool & pool, std::unique_lock & lock) { + setCurrentPriority(lock, pool.priority); // canSpawnWorker() ensures this would not decrease current_priority pool.workers++; - current_priority = pool.priority; // canSpawnWorker() ensures this would not decrease current_priority + if (log_events) + LOG_DEBUG(log, "Spawn loader worker #{} in {}", pool.workers, pool.name); NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; pool.thread_pool->scheduleOrThrowOnError([this, &pool] { worker(pool); }); @@ -842,6 +864,8 @@ void AsyncLoader::worker(Pool & pool) if (!canWorkerLive(pool, lock)) { + if (log_events) + LOG_DEBUG(log, "Stop worked in {}", pool.name); if (--pool.workers == 0) updateCurrentPriorityAndSpawn(lock); // It will spawn lower priority workers if needed return; @@ -852,6 +876,9 @@ void AsyncLoader::worker(Pool & pool) job = it->second; pool.ready_queue.erase(it); scheduled_jobs.find(job)->second.ready_seqno = 0; // This job is no longer in the ready queue + + if (log_events) + LOG_DEBUG(log, "Execute load job '{}' in {}", job->name, pool.name); } ALLOW_ALLOCATIONS_IN_SCOPE; diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index b1b336d24dc..a5c7049ee0e 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -327,7 +327,7 @@ private: }; public: - AsyncLoader(std::vector pool_initializers, bool log_failures_, bool log_progress_); + AsyncLoader(std::vector pool_initializers, bool log_failures_, bool log_progress_, bool log_events_); // WARNING: all tasks instances should be destructed before associated AsyncLoader. ~AsyncLoader(); @@ -407,6 +407,7 @@ private: void wait(std::unique_lock & job_lock, const LoadJobPtr & job); bool canSpawnWorker(Pool & pool, std::unique_lock & lock); bool canWorkerLive(Pool & pool, std::unique_lock & lock); + void setCurrentPriority(std::unique_lock & lock, std::optional priority); void updateCurrentPriorityAndSpawn(std::unique_lock & lock); void spawn(Pool & pool, std::unique_lock & lock); void worker(Pool & pool); @@ -415,6 +416,7 @@ private: // Logging const bool log_failures; // Worker should log all exceptions caught from job functions. const bool log_progress; // Periodically log total progress + const bool log_events; // Log all important events: job start/end, waits, prioritizations LoggerPtr log; mutable std::mutex mutex; // Guards all the fields below. diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index fc2537abcfc..b7b14bbedd4 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -50,7 +50,7 @@ struct AsyncLoaderTest pcg64 rng{randomSeed()}; explicit AsyncLoaderTest(std::vector initializers) - : loader(getPoolInitializers(initializers), /* log_failures = */ false, /* log_progress = */ false) + : loader(getPoolInitializers(initializers), /* log_failures = */ false, /* log_progress = */ false, /* log_events = */ false) { loader.stop(); // All tests call `start()` manually to better control ordering } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 6a0657a842c..48c0618f5a9 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2461,7 +2461,8 @@ AsyncLoader & Context::getAsyncLoader() const } }, /* log_failures = */ true, - /* log_progress = */ true); + /* log_progress = */ true, + /* log_events = */ true); }); return *shared->async_loader; From e2c692e194e8f02a623a21568d1b50980754b019 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 18 Mar 2024 18:24:36 +0000 Subject: [PATCH 020/102] fix build --- src/Common/AsyncLoader.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index e4896ce32ab..02dada7b8f9 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -804,8 +804,10 @@ bool AsyncLoader::canWorkerLive(Pool & pool, std::unique_lock &) void AsyncLoader::setCurrentPriority(std::unique_lock &, std::optional priority) { - if (log_event && current_priority != priority) - LOG_DEBUG(log, "Change current priority: {} -> {}", current_priority, priority); + if (log_events && current_priority != priority) + LOG_DEBUG(log, "Change current priority: {} -> {}", + current_priority ? std::to_string(*current_priority) : "none", + priority ? std::to_string(*priority) : "none"); current_priority = priority; } From 72932424eb00da56c0e6d2655d1ceb19fddcf86e Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 18 Mar 2024 19:36:45 +0000 Subject: [PATCH 021/102] fix issue with allocations --- src/Common/AsyncLoader.cpp | 43 ++++++++++++++++++++++++++++++-------- 1 file changed, 34 insertions(+), 9 deletions(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 02dada7b8f9..7d420b3beef 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -587,7 +587,12 @@ void AsyncLoader::finish(const LoadJobPtr & job, LoadStatus status, std::excepti job->canceled(reason); if (log_events) - LOG_DEBUG(log, "Finish load job '{}' with status {}", job->name, magic_enum::enum_name(status)); + { + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + LOG_DEBUG(log, "Finish load job '{}' with status {}", job->name, magic_enum::enum_name(status)); + }); + } Info & info = scheduled_jobs[job]; if (info.isReady()) @@ -669,7 +674,12 @@ void AsyncLoader::prioritize(const LoadJobPtr & job, size_t new_pool_id, std::un job->pool_id.store(new_pool_id); if (log_events) - LOG_DEBUG(log, "Prioritize load job '{}': {} -> {}", job->name, old_pool.name, new_pool.name); + { + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + LOG_DEBUG(log, "Prioritize load job '{}': {} -> {}", job->name, old_pool.name, new_pool.name); + }); + } // Recurse into dependencies for (const auto & dep : job->dependencies) @@ -805,9 +815,14 @@ bool AsyncLoader::canWorkerLive(Pool & pool, std::unique_lock &) void AsyncLoader::setCurrentPriority(std::unique_lock &, std::optional priority) { if (log_events && current_priority != priority) - LOG_DEBUG(log, "Change current priority: {} -> {}", - current_priority ? std::to_string(*current_priority) : "none", - priority ? std::to_string(*priority) : "none"); + { + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + LOG_DEBUG(log, "Change current priority: {} -> {}", + current_priority ? std::to_string(*current_priority) : "none", + priority ? std::to_string(*priority) : "none"); + }); + } current_priority = priority; } @@ -835,10 +850,10 @@ void AsyncLoader::spawn(Pool & pool, std::unique_lock & lock) { setCurrentPriority(lock, pool.priority); // canSpawnWorker() ensures this would not decrease current_priority pool.workers++; - if (log_events) - LOG_DEBUG(log, "Spawn loader worker #{} in {}", pool.workers, pool.name); NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; + if (log_events) + LOG_DEBUG(log, "Spawn loader worker #{} in {}", pool.workers, pool.name); pool.thread_pool->scheduleOrThrowOnError([this, &pool] { worker(pool); }); }); } @@ -867,7 +882,12 @@ void AsyncLoader::worker(Pool & pool) if (!canWorkerLive(pool, lock)) { if (log_events) - LOG_DEBUG(log, "Stop worked in {}", pool.name); + { + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + LOG_DEBUG(log, "Stop worked in {}", pool.name); + }); + } if (--pool.workers == 0) updateCurrentPriorityAndSpawn(lock); // It will spawn lower priority workers if needed return; @@ -880,7 +900,12 @@ void AsyncLoader::worker(Pool & pool) scheduled_jobs.find(job)->second.ready_seqno = 0; // This job is no longer in the ready queue if (log_events) - LOG_DEBUG(log, "Execute load job '{}' in {}", job->name, pool.name); + { + NOEXCEPT_SCOPE({ + ALLOW_ALLOCATIONS_IN_SCOPE; + LOG_DEBUG(log, "Execute load job '{}' in {}", job->name, pool.name); + }); + } } ALLOW_ALLOCATIONS_IN_SCOPE; From 083d0776edb3ba6f6253c452a6b06ff0908ea0f9 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 18 Mar 2024 20:47:31 +0100 Subject: [PATCH 022/102] Wait for done mutation properly --- src/Storages/StorageReplicatedMergeTree.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c41403e312b..c89be57dc07 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -615,8 +615,8 @@ void StorageReplicatedMergeTree::waitMutationToFinishOnReplicas( else if (mutation_pointer_value >= mutation_id) /// Maybe we already processed more fresh mutation break; /// (numbers like 0000000000 and 0000000001) - /// Replica can become inactive, so wait with timeout and recheck it - if (wait_event->tryWait(1000)) + /// Replica can become inactive, so wait with timeout, if nothing happened -> recheck it + if (!wait_event->tryWait(1000)) continue; /// Here we check mutation for errors on local replica. If they happen on this replica From 6aa88ed5e958899def7dd8d35ae686a4030588bf Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 18 Mar 2024 19:49:05 +0000 Subject: [PATCH 023/102] typo --- src/Common/AsyncLoader.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 7d420b3beef..52cfbba8e0c 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -786,7 +786,7 @@ void AsyncLoader::wait(std::unique_lock & job_lock, const LoadJobPtr return; if (log_events) - LOG_DEBUG(log, "Wait load job '{}' in {}", job->name, getPoolName(job->executionPool())); + LOG_DEBUG(log, "Wait load job '{}' in {}", job->name, getPoolName(job->pool_id)); Stopwatch watch; job->waiters++; @@ -885,7 +885,7 @@ void AsyncLoader::worker(Pool & pool) { NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; - LOG_DEBUG(log, "Stop worked in {}", pool.name); + LOG_DEBUG(log, "Stop worker in {}", pool.name); }); } if (--pool.workers == 0) From 1264b5337e47619a692e1a6fae0d97cf4370f122 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 18 Mar 2024 20:57:14 +0100 Subject: [PATCH 024/102] More diagnostics --- src/Storages/StorageReplicatedMergeTree.cpp | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c89be57dc07..7a63c2ee8c0 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -613,18 +613,26 @@ void StorageReplicatedMergeTree::waitMutationToFinishOnReplicas( break; } else if (mutation_pointer_value >= mutation_id) /// Maybe we already processed more fresh mutation + { + LOG_TRACE(log, "Mutation {} is done because mutation pointer is {}", mutation_id, mutation_pointer_value); break; /// (numbers like 0000000000 and 0000000001) - - /// Replica can become inactive, so wait with timeout, if nothing happened -> recheck it - if (!wait_event->tryWait(1000)) - continue; + } /// Here we check mutation for errors on local replica. If they happen on this replica /// they will happen on each replica, so we can check only in-memory info. auto mutation_status = queue.getIncompleteMutationsStatus(mutation_id); /// If mutation status is empty, than local replica may just not loaded it into memory. - if (mutation_status && !mutation_status->latest_fail_reason.empty()) + if (mutation_status && (mutation_status->is_done || !mutation_status->latest_fail_reason.empty())) + { + LOG_DEBUG(log, "Mutation {} is done {} or falied {} (status: '{}')", mutation_id, mutation_status->is_done, !mutation_status->latest_fail_reason.empty(), mutation_status->latest_fail_reason); break; + } + + /// Replica can become inactive, so wait with timeout, if nothing happened -> recheck it + if (!wait_event->tryWait(1000)) + { + LOG_TRACE(log, "Failed to wait for mutation '{}', will recheck", mutation_id) + } } /// This replica inactive, don't check anything From 82da2d56e9725145e2d84f36653360927357dba8 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 18 Mar 2024 20:58:15 +0100 Subject: [PATCH 025/102] Add chassert --- src/Storages/StorageReplicatedMergeTree.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 7a63c2ee8c0..a79fe768940 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -663,6 +663,7 @@ void StorageReplicatedMergeTree::waitMutationToFinishOnReplicas( /// they will happen on each replica, so we can check only in-memory info. auto mutation_status = queue.getIncompleteMutationsStatus(mutation_id, &mutation_ids); checkMutationStatus(mutation_status, mutation_ids); + chassert(mutation_status.is_done); } if (!inactive_replicas.empty()) From a3a549bc242253866272db5945c107baea0b3803 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 18 Mar 2024 21:03:00 +0100 Subject: [PATCH 026/102] Wait for update of value in memory and in zookeeper --- src/Storages/StorageReplicatedMergeTree.cpp | 22 +++++++++++++++------ 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index a79fe768940..39fa65967d3 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -604,7 +604,12 @@ void StorageReplicatedMergeTree::waitMutationToFinishOnReplicas( break; } + /// Here we check mutation for errors on local replica. If they happen on this replica + /// they will happen on each replica, so we can check only in-memory info. + auto mutation_status = queue.getIncompleteMutationsStatus(mutation_id); + String mutation_pointer = fs::path(zookeeper_path) / "replicas" / replica / "mutation_pointer"; + std::string mutation_pointer_value; /// Replica could be removed if (!zookeeper->tryGet(mutation_pointer, mutation_pointer_value, nullptr, wait_event)) @@ -614,15 +619,20 @@ void StorageReplicatedMergeTree::waitMutationToFinishOnReplicas( } else if (mutation_pointer_value >= mutation_id) /// Maybe we already processed more fresh mutation { - LOG_TRACE(log, "Mutation {} is done because mutation pointer is {}", mutation_id, mutation_pointer_value); - break; /// (numbers like 0000000000 and 0000000001) + bool mutation_killed_or_done_locally = !mutation_status || mutation_status->is_done; + if (mutation_killed_or_done_locally) + { + LOG_TRACE(log, "Mutation {} is done because mutation pointer is {}", mutation_id, mutation_pointer_value); + break; /// (numbers like 0000000000 and 0000000001) + } + else + { + LOG_TRACE(log, "Mutation {} is done because mutation pointer is {}, but state is not updated in memory, will wait", mutation_id, mutation_pointer_value); + } } - /// Here we check mutation for errors on local replica. If they happen on this replica - /// they will happen on each replica, so we can check only in-memory info. - auto mutation_status = queue.getIncompleteMutationsStatus(mutation_id); /// If mutation status is empty, than local replica may just not loaded it into memory. - if (mutation_status && (mutation_status->is_done || !mutation_status->latest_fail_reason.empty())) + if (mutation_status && !mutation_status->latest_fail_reason.empty()) { LOG_DEBUG(log, "Mutation {} is done {} or falied {} (status: '{}')", mutation_id, mutation_status->is_done, !mutation_status->latest_fail_reason.empty(), mutation_status->latest_fail_reason); break; From a742b235a4c7cfca6f5e6ce65fd6ebd897a2364b Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Tue, 19 Mar 2024 17:12:13 +0800 Subject: [PATCH 027/102] add pass covert in single value to equal --- src/Analyzer/Passes/ConvertInToEqualsPass.cpp | 55 +++++++++++++++++++ src/Analyzer/Passes/ConvertInToEqualsPass.h | 24 ++++++++ src/Analyzer/QueryTreePassManager.cpp | 2 + src/Core/Settings.h | 1 + 4 files changed, 82 insertions(+) create mode 100644 src/Analyzer/Passes/ConvertInToEqualsPass.cpp create mode 100644 src/Analyzer/Passes/ConvertInToEqualsPass.h diff --git a/src/Analyzer/Passes/ConvertInToEqualsPass.cpp b/src/Analyzer/Passes/ConvertInToEqualsPass.cpp new file mode 100644 index 00000000000..cf9f1dd6ad1 --- /dev/null +++ b/src/Analyzer/Passes/ConvertInToEqualsPass.cpp @@ -0,0 +1,55 @@ +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +using FunctionEquals = FunctionComparison; + +class ConvertInToEqualsPassVisitor : public InDepthQueryTreeVisitorWithContext +{ +public: + using Base = InDepthQueryTreeVisitorWithContext; + using Base::Base; + + FunctionOverloadResolverPtr createInternalFunctionEqualOverloadResolver() + { + return std::make_unique(std::make_shared(getContext()->getSettings().decimal_check_overflow)); + } + + void enterImpl(QueryTreeNodePtr & node) + { + if (!getSettings().optimize_in_to_equal) + return; + auto * func_node = node->as(); + if (!func_node || func_node->getFunctionName() != "in" || func_node->getArguments().getNodes().size() != 2) + return ; + auto args = func_node->getArguments().getNodes(); + auto * column_node = args[0]->as(); + auto * constant_node = args[1]->as(); + if (!column_node || !constant_node) + return ; + if (constant_node->getValue().getType() == Field::Types::Which::Tuple) + return; + auto const_value = std::make_shared(constant_node->getValue()); + + auto equal_resolver = createInternalFunctionEqualOverloadResolver(); + auto equal = std::make_shared("equals"); + QueryTreeNodes arguments{column_node->clone(), const_value}; + equal->getArguments().getNodes() = std::move(arguments); + equal->resolveAsFunction(equal_resolver); + node = equal; + } +}; + +void ConvertInToEqualsPass::run(QueryTreeNodePtr & query_tree_node, ContextPtr context) +{ + ConvertInToEqualsPassVisitor visitor(std::move(context)); + visitor.visit(query_tree_node); +} +} diff --git a/src/Analyzer/Passes/ConvertInToEqualsPass.h b/src/Analyzer/Passes/ConvertInToEqualsPass.h new file mode 100644 index 00000000000..1f40645498a --- /dev/null +++ b/src/Analyzer/Passes/ConvertInToEqualsPass.h @@ -0,0 +1,24 @@ +#pragma once + +#include + +namespace DB { + +/** Optimize `in` to `equals` if possible. + * + * Example: SELECT * from test where x IN (1); + * Result: SELECT * from test where x = 1; + * + */ +class ConvertInToEqualsPass final : public IQueryTreePass { +public: + String getName() override { return "ConvertInToEqualsPass"; } + + String getDescription() override { return "Convert in to equal"; } + + void run(QueryTreeNodePtr & query_tree_node, ContextPtr context) override; +}; +} + + + diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index 9c07884a464..284e6bc2809 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -28,6 +28,7 @@ #include #include #include +#include #include #include #include @@ -263,6 +264,7 @@ void addQueryTreePasses(QueryTreePassManager & manager, bool only_analyze) manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); + manager.addPass(std::make_unique()); /// should before AggregateFunctionsArithmericOperationsPass manager.addPass(std::make_unique()); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e6adb00137f..19655d6493d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -593,6 +593,7 @@ class IColumn; M(Bool, optimize_functions_to_subcolumns, false, "Transform functions to subcolumns, if possible, to reduce amount of read data. E.g. 'length(arr)' -> 'arr.size0', 'col IS NULL' -> 'col.null' ", 0) \ M(Bool, optimize_using_constraints, false, "Use constraints for query optimization", 0) \ M(Bool, optimize_substitute_columns, false, "Use constraints for column substitution", 0) \ + M(Bool, optimize_in_to_equal, true, "Convert the constant in to equal", 0) \ M(Bool, optimize_append_index, false, "Use constraints in order to append index condition (indexHint)", 0) \ M(Bool, optimize_time_filter_with_preimage, true, "Optimize Date and DateTime predicates by converting functions into equivalent comparisons without conversions (e.g. toYear(col) = 2023 -> col >= '2023-01-01' AND col <= '2023-12-31')", 0) \ M(Bool, normalize_function_names, true, "Normalize function names to their canonical names", 0) \ From 75e94438e03fba799146f4ae44edbf376c48b9ce Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 19 Mar 2024 10:21:19 +0100 Subject: [PATCH 028/102] Update src/Storages/StorageReplicatedMergeTree.cpp --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 39fa65967d3..559a7074e73 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -634,7 +634,7 @@ void StorageReplicatedMergeTree::waitMutationToFinishOnReplicas( /// If mutation status is empty, than local replica may just not loaded it into memory. if (mutation_status && !mutation_status->latest_fail_reason.empty()) { - LOG_DEBUG(log, "Mutation {} is done {} or falied {} (status: '{}')", mutation_id, mutation_status->is_done, !mutation_status->latest_fail_reason.empty(), mutation_status->latest_fail_reason); + LOG_DEBUG(log, "Mutation {} is done {} or failed {} (status: '{}')", mutation_id, mutation_status->is_done, !mutation_status->latest_fail_reason.empty(), mutation_status->latest_fail_reason); break; } From 532d72a7d8b74eeb1608f103e8540a7b7deb1294 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Tue, 19 Mar 2024 17:30:16 +0800 Subject: [PATCH 029/102] add test case --- ...qualsPass.cpp => ConvertInToEqualPass.cpp} | 10 ++--- ...nToEqualsPass.h => ConvertInToEqualPass.h} | 4 +- src/Analyzer/QueryTreePassManager.cpp | 4 +- tests/performance/function_in.xml | 30 +++++++++++++++ .../03013_optimize_in_to_equal.reference | 37 +++++++++++++++++++ .../03013_optimize_in_to_equal.sql | 10 +++++ 6 files changed, 86 insertions(+), 9 deletions(-) rename src/Analyzer/Passes/{ConvertInToEqualsPass.cpp => ConvertInToEqualPass.cpp} (84%) rename src/Analyzer/Passes/{ConvertInToEqualsPass.h => ConvertInToEqualPass.h} (75%) create mode 100644 tests/performance/function_in.xml create mode 100644 tests/queries/0_stateless/03013_optimize_in_to_equal.reference create mode 100644 tests/queries/0_stateless/03013_optimize_in_to_equal.sql diff --git a/src/Analyzer/Passes/ConvertInToEqualsPass.cpp b/src/Analyzer/Passes/ConvertInToEqualPass.cpp similarity index 84% rename from src/Analyzer/Passes/ConvertInToEqualsPass.cpp rename to src/Analyzer/Passes/ConvertInToEqualPass.cpp index cf9f1dd6ad1..26db3a4f4aa 100644 --- a/src/Analyzer/Passes/ConvertInToEqualsPass.cpp +++ b/src/Analyzer/Passes/ConvertInToEqualPass.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include @@ -11,10 +11,10 @@ namespace DB using FunctionEquals = FunctionComparison; -class ConvertInToEqualsPassVisitor : public InDepthQueryTreeVisitorWithContext +class ConvertInToEqualPassVisitor : public InDepthQueryTreeVisitorWithContext { public: - using Base = InDepthQueryTreeVisitorWithContext; + using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; FunctionOverloadResolverPtr createInternalFunctionEqualOverloadResolver() @@ -47,9 +47,9 @@ public: } }; -void ConvertInToEqualsPass::run(QueryTreeNodePtr & query_tree_node, ContextPtr context) +void ConvertInToEqualPass::run(QueryTreeNodePtr & query_tree_node, ContextPtr context) { - ConvertInToEqualsPassVisitor visitor(std::move(context)); + ConvertInToEqualPassVisitor visitor(std::move(context)); visitor.visit(query_tree_node); } } diff --git a/src/Analyzer/Passes/ConvertInToEqualsPass.h b/src/Analyzer/Passes/ConvertInToEqualPass.h similarity index 75% rename from src/Analyzer/Passes/ConvertInToEqualsPass.h rename to src/Analyzer/Passes/ConvertInToEqualPass.h index 1f40645498a..19b83c0f65f 100644 --- a/src/Analyzer/Passes/ConvertInToEqualsPass.h +++ b/src/Analyzer/Passes/ConvertInToEqualPass.h @@ -10,9 +10,9 @@ namespace DB { * Result: SELECT * from test where x = 1; * */ -class ConvertInToEqualsPass final : public IQueryTreePass { +class ConvertInToEqualPass final : public IQueryTreePass { public: - String getName() override { return "ConvertInToEqualsPass"; } + String getName() override { return "ConvertInToEqualPass"; } String getDescription() override { return "Convert in to equal"; } diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index 284e6bc2809..14eb179680c 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -28,7 +28,7 @@ #include #include #include -#include +#include #include #include #include @@ -264,7 +264,7 @@ void addQueryTreePasses(QueryTreePassManager & manager, bool only_analyze) manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); manager.addPass(std::make_unique()); - manager.addPass(std::make_unique()); + manager.addPass(std::make_unique()); /// should before AggregateFunctionsArithmericOperationsPass manager.addPass(std::make_unique()); diff --git a/tests/performance/function_in.xml b/tests/performance/function_in.xml new file mode 100644 index 00000000000..16e6bb91469 --- /dev/null +++ b/tests/performance/function_in.xml @@ -0,0 +1,30 @@ + + + 8 + 1 + + + + CREATE TABLE t_nullable + ( + key_string1 Nullable(String), + key_string2 Nullable(String), + key_string3 Nullable(String), + key_int64_1 Nullable(Int64), + key_int64_2 Nullable(Int64), + key_int64_3 Nullable(Int64), + key_int64_4 Nullable(Int64), + key_int64_5 Nullable(Int64), + m1 Int64, + m2 Int64 + ) + ENGINE = Memory + + insert into t_nullable select ['aaaaaa','bbaaaa','ccaaaa','ddaaaa'][number % 101 + 1], ['aa','bb','cc','dd'][number % 100 + 1], ['aa','bb','cc','dd'][number % 102 + 1], number%10+1, number%10+2, number%10+3, number%10+4,number%10+5, number%6000+1, number%5000+2 from numbers_mt(30000000) + select * from t_nullable where key_string1 in ('aaaaaa') format Null + select * from t_nullable where key_string1 in ('aaaaaa') format Null SETTINGS allow_experimental_analyzer=1 + select * from t_nullable where key_string2 in ('3') format Null + select * from t_nullable where key_string2 in ('3') format Null SETTINGS allow_experimental_analyzer=1 + drop table if exists t_nullable + + \ No newline at end of file diff --git a/tests/queries/0_stateless/03013_optimize_in_to_equal.reference b/tests/queries/0_stateless/03013_optimize_in_to_equal.reference new file mode 100644 index 00000000000..f55a7b76eaf --- /dev/null +++ b/tests/queries/0_stateless/03013_optimize_in_to_equal.reference @@ -0,0 +1,37 @@ +a 1 +------------------- +QUERY id: 0 + PROJECTION COLUMNS + x String + y Int32 + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: x, result_type: String, source_id: 3 + COLUMN id: 4, column_name: y, result_type: Int32, source_id: 3 + JOIN TREE + TABLE id: 3, alias: __table1, table_name: default.test + WHERE + FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 6, nodes: 2 + COLUMN id: 7, column_name: x, result_type: String, source_id: 3 + CONSTANT id: 8, constant_value: \'a\', constant_value_type: String + SETTINGS allow_experimental_analyzer=1 +------------------- +QUERY id: 0 + PROJECTION COLUMNS + x String + y Int32 + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: x, result_type: String, source_id: 3 + COLUMN id: 4, column_name: y, result_type: Int32, source_id: 3 + JOIN TREE + TABLE id: 3, alias: __table1, table_name: default.test + WHERE + FUNCTION id: 5, function_name: in, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 6, nodes: 2 + COLUMN id: 7, column_name: x, result_type: String, source_id: 3 + CONSTANT id: 8, constant_value: Tuple_(\'a\', \'b\'), constant_value_type: Tuple(String, String) + SETTINGS allow_experimental_analyzer=1 diff --git a/tests/queries/0_stateless/03013_optimize_in_to_equal.sql b/tests/queries/0_stateless/03013_optimize_in_to_equal.sql new file mode 100644 index 00000000000..9d80825ceb4 --- /dev/null +++ b/tests/queries/0_stateless/03013_optimize_in_to_equal.sql @@ -0,0 +1,10 @@ +DROP TABLE IF EXISTS test; +CREATE TABLE test (x String, y Int32) ENGINE = MergeTree() ORDER BY x; + +INSERT INTO test VALUES ('a', 1), ('b', 2), ('c', 3), ('d', 4), ('e', 5); + +select * from test where x in ('a') SETTINGS allow_experimental_analyzer = 1; +select '-------------------'; +explain query tree select * from test where x in ('a') SETTINGS allow_experimental_analyzer = 1; +select '-------------------'; +explain query tree select * from test where x in ('a','b') SETTINGS allow_experimental_analyzer = 1; \ No newline at end of file From 761c612295c8f09ff12fc697e482069ac7bd61a2 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Tue, 19 Mar 2024 18:29:07 +0800 Subject: [PATCH 030/102] fix style --- src/Analyzer/Passes/ConvertInToEqualPass.h | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/src/Analyzer/Passes/ConvertInToEqualPass.h b/src/Analyzer/Passes/ConvertInToEqualPass.h index 19b83c0f65f..7de75e94f04 100644 --- a/src/Analyzer/Passes/ConvertInToEqualPass.h +++ b/src/Analyzer/Passes/ConvertInToEqualPass.h @@ -2,15 +2,15 @@ #include -namespace DB { - +namespace DB +{ /** Optimize `in` to `equals` if possible. * * Example: SELECT * from test where x IN (1); * Result: SELECT * from test where x = 1; - * */ -class ConvertInToEqualPass final : public IQueryTreePass { +class ConvertInToEqualPass final : public IQueryTreePass +{ public: String getName() override { return "ConvertInToEqualPass"; } @@ -19,6 +19,3 @@ public: void run(QueryTreeNodePtr & query_tree_node, ContextPtr context) override; }; } - - - From 667aade4e31aedebf6429a6119dd9a5991fbe258 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 19 Mar 2024 12:12:35 +0100 Subject: [PATCH 031/102] Fix build --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 559a7074e73..ddd9bf9fbae 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -641,7 +641,7 @@ void StorageReplicatedMergeTree::waitMutationToFinishOnReplicas( /// Replica can become inactive, so wait with timeout, if nothing happened -> recheck it if (!wait_event->tryWait(1000)) { - LOG_TRACE(log, "Failed to wait for mutation '{}', will recheck", mutation_id) + LOG_TRACE(log, "Failed to wait for mutation '{}', will recheck", mutation_id); } } From 9b14d17d7b207ccaa83a7e51b90ab9be79782d69 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 19 Mar 2024 12:58:36 +0100 Subject: [PATCH 032/102] Fxi --- src/Storages/StorageReplicatedMergeTree.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ddd9bf9fbae..30646e0138c 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -673,7 +673,8 @@ void StorageReplicatedMergeTree::waitMutationToFinishOnReplicas( /// they will happen on each replica, so we can check only in-memory info. auto mutation_status = queue.getIncompleteMutationsStatus(mutation_id, &mutation_ids); checkMutationStatus(mutation_status, mutation_ids); - chassert(mutation_status.is_done); + /// Mutation should be done or we should throw exception + chassert(mutation_status->is_done); } if (!inactive_replicas.empty()) From 8494e7389d21307781b91a030e22f05dac6deeb7 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 19 Mar 2024 13:47:20 +0100 Subject: [PATCH 033/102] Update setting names --- src/Common/BufferAllocationPolicy.cpp | 14 +++++++------- src/Common/BufferAllocationPolicy.h | 12 ++++++------ 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/src/Common/BufferAllocationPolicy.cpp b/src/Common/BufferAllocationPolicy.cpp index 2b00d4f84d0..14d009008b9 100644 --- a/src/Common/BufferAllocationPolicy.cpp +++ b/src/Common/BufferAllocationPolicy.cpp @@ -12,7 +12,7 @@ class FixedSizeBufferAllocationPolicy : public IBufferAllocationPolicy public: explicit FixedSizeBufferAllocationPolicy(const IBufferAllocationPolicy::Settings & settings_) - : buffer_size(settings_.strict_upload_part_size) + : buffer_size(settings_.strict_size) { chassert(buffer_size > 0); } @@ -46,11 +46,11 @@ class ExpBufferAllocationPolicy : public DB::IBufferAllocationPolicy public: explicit ExpBufferAllocationPolicy(const IBufferAllocationPolicy::Settings & settings_) - : first_size(std::max(settings_.max_single_part_upload_size, settings_.min_upload_part_size)) - , second_size(settings_.min_upload_part_size) - , multiply_factor(settings_.upload_part_size_multiply_factor) - , multiply_threshold(settings_.upload_part_size_multiply_parts_count_threshold) - , max_size(settings_.max_upload_part_size) + : first_size(std::max(settings_.max_single_size, settings_.min_size)) + , second_size(settings_.min_size) + , multiply_factor(settings_.multiply_factor) + , multiply_threshold(settings_.multiply_parts_count_threshold) + , max_size(settings_.max_size) { chassert(first_size > 0); chassert(second_size > 0); @@ -93,7 +93,7 @@ IBufferAllocationPolicy::~IBufferAllocationPolicy() = default; IBufferAllocationPolicy::IBufferAllocationPolicyPtr IBufferAllocationPolicy::create(IBufferAllocationPolicy::Settings settings_) { - if (settings_.strict_upload_part_size > 0) + if (settings_.strict_size > 0) return std::make_unique(settings_); else return std::make_unique(settings_); diff --git a/src/Common/BufferAllocationPolicy.h b/src/Common/BufferAllocationPolicy.h index 3ada9e52fdf..dd04139e694 100644 --- a/src/Common/BufferAllocationPolicy.h +++ b/src/Common/BufferAllocationPolicy.h @@ -15,12 +15,12 @@ public: struct Settings { - size_t strict_upload_part_size = 0; - size_t min_upload_part_size = 16 * 1024 * 1024; - size_t max_upload_part_size = 5ULL * 1024 * 1024 * 1024; - size_t upload_part_size_multiply_factor = 2; - size_t upload_part_size_multiply_parts_count_threshold = 500; - size_t max_single_part_upload_size = 32 * 1024 * 1024; + size_t strict_size = 0; + size_t min_size = 16 * 1024 * 1024; + size_t max_size = 5ULL * 1024 * 1024 * 1024; + size_t multiply_factor = 2; + size_t multiply_parts_count_threshold = 500; + size_t max_single_size = 32 * 1024 * 1024; }; From c16922493a614779d989701b9bed4195a503d6e9 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 19 Mar 2024 14:54:06 +0100 Subject: [PATCH 034/102] Removed detachPart & reallocateBuffer functions --- .../IO/WriteBufferFromAzureBlobStorage.cpp | 58 +++++-------------- .../IO/WriteBufferFromAzureBlobStorage.h | 7 +-- 2 files changed, 16 insertions(+), 49 deletions(-) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 41d7187afda..50763f7f189 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -18,12 +18,6 @@ namespace ProfileEvents namespace DB { -struct WriteBufferFromAzureBlobStorage::PartData -{ - Memory<> memory; - size_t data_size = 0; -}; - WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage( std::shared_ptr blob_container_client_, const String & blob_path_, @@ -108,53 +102,31 @@ void WriteBufferFromAzureBlobStorage::finalizeImpl() void WriteBufferFromAzureBlobStorage::nextImpl() { task_tracker->waitIfAny(); - - reallocateBuffer(); - detachBuffer(); - - while (!detached_part_data.empty()) - { - writePart(std::move(detached_part_data.front())); - detached_part_data.pop_front(); - } - + writePart(); allocateBuffer(); } void WriteBufferFromAzureBlobStorage::allocateBuffer() { buffer_allocation_policy->nextBuffer(); - memory = Memory(buffer_allocation_policy->getBufferSize()); + auto size = buffer_allocation_policy->getBufferSize(); + + if (buffer_allocation_policy->getBufferNumber() == 1) + { + size = std::min(size_t(DBMS_DEFAULT_BUFFER_SIZE), size); + } + + memory = Memory(size); WriteBuffer::set(memory.data(), memory.size()); } - -void WriteBufferFromAzureBlobStorage::reallocateBuffer() -{ - if (available() > 0) - return; - - if (memory.size() == buffer_allocation_policy->getBufferSize()) - return; - - memory.resize(buffer_allocation_policy->getBufferSize()); - - WriteBuffer::set(memory.data(), memory.size()); - - chassert(offset() == 0); -} - -void WriteBufferFromAzureBlobStorage::detachBuffer() +void WriteBufferFromAzureBlobStorage::writePart() { size_t data_size = size_t(position() - memory.data()); - auto buf = std::move(memory); + auto data = std::move(memory); WriteBuffer::set(nullptr, 0); - detached_part_data.push_back({std::move(buf), data_size}); -} -void WriteBufferFromAzureBlobStorage::writePart(WriteBufferFromAzureBlobStorage::PartData && data) -{ - if (data.data_size == 0) + if (data_size == 0) return; auto upload_worker = [&] () @@ -162,11 +134,11 @@ void WriteBufferFromAzureBlobStorage::writePart(WriteBufferFromAzureBlobStorage: auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); const std::string & block_id = block_ids.emplace_back(getRandomASCIIString(64)); - Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(data.memory.data()), data.data_size); - execWithRetry([&](){ block_blob_client.StageBlock(block_id, memory_stream); }, max_unexpected_write_error_retries, data.data_size); + Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(data.data()), data_size); + execWithRetry([&](){ block_blob_client.StageBlock(block_id, memory_stream); }, max_unexpected_write_error_retries, data_size); if (write_settings.remote_throttler) - write_settings.remote_throttler->add(data.data_size, ProfileEvents::RemoteWriteThrottlerBytes, ProfileEvents::RemoteWriteThrottlerSleepMicroseconds); + write_settings.remote_throttler->add(data_size, ProfileEvents::RemoteWriteThrottlerBytes, ProfileEvents::RemoteWriteThrottlerSleepMicroseconds); }; task_tracker->add(std::move(upload_worker)); diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h index f047dd53724..a210a75030a 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h @@ -48,12 +48,8 @@ public: private: struct PartData; - void writePart(WriteBufferFromAzureBlobStorage::PartData && data); - void detachBuffer(); + void writePart(); void allocateBuffer(); - void allocateFirstBuffer(); - void reallocateFirstBuffer(); - void reallocateBuffer(); void finalizeImpl() override; void execWithRetry(std::function func, size_t num_tries, size_t cost = 0); @@ -81,7 +77,6 @@ private: bool first_buffer=true; std::unique_ptr task_tracker; - std::deque detached_part_data; }; } From 824092b6192b043e28cd775284b27ca720d3e7fe Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 19 Mar 2024 18:38:02 +0100 Subject: [PATCH 035/102] Updated to use single part upload for single block of small size and fixed test fail --- .../IO/WriteBufferFromAzureBlobStorage.cpp | 20 ++++++++++++++++++- .../AzureBlobStorage/AzureBlobStorageAuth.cpp | 2 +- .../test.py | 1 + 3 files changed, 21 insertions(+), 2 deletions(-) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 50763f7f189..f966e431506 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -89,11 +89,25 @@ void WriteBufferFromAzureBlobStorage::execWithRetry(std::function func, void WriteBufferFromAzureBlobStorage::finalizeImpl() { + auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); + + if (buffer_allocation_policy->getBufferNumber() == 1) + { + size_t data_size = size_t(position() - memory.data()); + if (data_size <= max_single_part_upload_size) + { + Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(memory.data()), data_size); + execWithRetry([&](){ block_blob_client.Upload(memory_stream); }, max_unexpected_write_error_retries, data_size); + LOG_TRACE(log, "Committed single block for blob `{}`", blob_path); + return; + } + } + + execWithRetry([this](){ next(); }, max_unexpected_write_error_retries); task_tracker->waitAll(); - auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); execWithRetry([&](){ block_blob_client.CommitBlockList(block_ids); }, max_unexpected_write_error_retries); LOG_TRACE(log, "Committed {} blocks for blob `{}`", block_ids.size(), blob_path); @@ -116,6 +130,8 @@ void WriteBufferFromAzureBlobStorage::allocateBuffer() size = std::min(size_t(DBMS_DEFAULT_BUFFER_SIZE), size); } + LOG_INFO(log, "allocateBuffer size = {}", size); + memory = Memory(size); WriteBuffer::set(memory.data(), memory.size()); } @@ -129,6 +145,8 @@ void WriteBufferFromAzureBlobStorage::writePart() if (data_size == 0) return; + LOG_TRACE(log, "writePart data size `{}`", data_size); + auto upload_worker = [&] () { auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp index 3c06801a070..e7ee768876f 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -213,7 +213,7 @@ std::unique_ptr getAzureBlobContainerClient( std::unique_ptr getAzureBlobStorageSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context) { std::unique_ptr settings = std::make_unique(); - settings->max_single_part_upload_size = config.getUInt64(config_prefix + ".max_single_part_upload_size", 100 * 1024 * 1024); + settings->max_single_part_upload_size = config.getUInt64(config_prefix + ".max_single_part_upload_size", context->getSettings().azure_max_single_part_upload_size); settings->min_bytes_for_seek = config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024); settings->max_single_read_retries = config.getInt(config_prefix + ".max_single_read_retries", 3); settings->max_single_download_retries = config.getInt(config_prefix + ".max_single_download_retries", 3); diff --git a/tests/integration/test_merge_tree_azure_blob_storage/test.py b/tests/integration/test_merge_tree_azure_blob_storage/test.py index 55deb87a97e..cffab672bd1 100644 --- a/tests/integration/test_merge_tree_azure_blob_storage/test.py +++ b/tests/integration/test_merge_tree_azure_blob_storage/test.py @@ -37,6 +37,7 @@ def generate_cluster_def(port): devstoreaccount1 Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw== 100000 + 100000 10 10 From a29d4f5db2aa6554ece3d1dbcef98928d899f6f4 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Wed, 20 Mar 2024 10:24:34 +0800 Subject: [PATCH 036/102] fix bug --- src/Analyzer/Passes/ConvertInToEqualPass.cpp | 3 +-- src/Core/SettingsChangesHistory.h | 1 + tests/performance/function_in.xml | 4 +--- 3 files changed, 3 insertions(+), 5 deletions(-) diff --git a/src/Analyzer/Passes/ConvertInToEqualPass.cpp b/src/Analyzer/Passes/ConvertInToEqualPass.cpp index 26db3a4f4aa..e89300cd967 100644 --- a/src/Analyzer/Passes/ConvertInToEqualPass.cpp +++ b/src/Analyzer/Passes/ConvertInToEqualPass.cpp @@ -36,11 +36,10 @@ public: return ; if (constant_node->getValue().getType() == Field::Types::Which::Tuple) return; - auto const_value = std::make_shared(constant_node->getValue()); auto equal_resolver = createInternalFunctionEqualOverloadResolver(); auto equal = std::make_shared("equals"); - QueryTreeNodes arguments{column_node->clone(), const_value}; + QueryTreeNodes arguments{column_node->clone(), constant_node->clone()}; equal->getArguments().getNodes() = std::move(arguments); equal->resolveAsFunction(equal_resolver); node = equal; diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 8e2b2915c2a..886dce72176 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -96,6 +96,7 @@ static std::map sett {"parallel_replicas_allow_in_with_subquery", false, true, "If true, subquery for IN will be executed on every follower replica"}, {"filesystem_cache_reserve_space_wait_lock_timeout_milliseconds", 1000, 1000, "Wait time to lock cache for sapce reservation in filesystem cache"}, {"max_parser_backtracks", 0, 1000000, "Limiting the complexity of parsing"}, + {"optimize_in_to_equal", false, true, "Convert in single value to equal"}, }}, {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, diff --git a/tests/performance/function_in.xml b/tests/performance/function_in.xml index 16e6bb91469..af4f8737ba7 100644 --- a/tests/performance/function_in.xml +++ b/tests/performance/function_in.xml @@ -21,10 +21,8 @@ ENGINE = Memory insert into t_nullable select ['aaaaaa','bbaaaa','ccaaaa','ddaaaa'][number % 101 + 1], ['aa','bb','cc','dd'][number % 100 + 1], ['aa','bb','cc','dd'][number % 102 + 1], number%10+1, number%10+2, number%10+3, number%10+4,number%10+5, number%6000+1, number%5000+2 from numbers_mt(30000000) - select * from t_nullable where key_string1 in ('aaaaaa') format Null select * from t_nullable where key_string1 in ('aaaaaa') format Null SETTINGS allow_experimental_analyzer=1 - select * from t_nullable where key_string2 in ('3') format Null select * from t_nullable where key_string2 in ('3') format Null SETTINGS allow_experimental_analyzer=1 drop table if exists t_nullable - \ No newline at end of file + From 862abf2f6efecf8a5acf6bf49f681b0d696c3db9 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Wed, 20 Mar 2024 10:40:22 +0800 Subject: [PATCH 037/102] remove config --- src/Analyzer/Passes/ConvertInToEqualPass.cpp | 2 -- src/Core/Settings.h | 1 - src/Core/SettingsChangesHistory.h | 1 - 3 files changed, 4 deletions(-) diff --git a/src/Analyzer/Passes/ConvertInToEqualPass.cpp b/src/Analyzer/Passes/ConvertInToEqualPass.cpp index e89300cd967..ce2cec9f9ed 100644 --- a/src/Analyzer/Passes/ConvertInToEqualPass.cpp +++ b/src/Analyzer/Passes/ConvertInToEqualPass.cpp @@ -24,8 +24,6 @@ public: void enterImpl(QueryTreeNodePtr & node) { - if (!getSettings().optimize_in_to_equal) - return; auto * func_node = node->as(); if (!func_node || func_node->getFunctionName() != "in" || func_node->getArguments().getNodes().size() != 2) return ; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 19655d6493d..e6adb00137f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -593,7 +593,6 @@ class IColumn; M(Bool, optimize_functions_to_subcolumns, false, "Transform functions to subcolumns, if possible, to reduce amount of read data. E.g. 'length(arr)' -> 'arr.size0', 'col IS NULL' -> 'col.null' ", 0) \ M(Bool, optimize_using_constraints, false, "Use constraints for query optimization", 0) \ M(Bool, optimize_substitute_columns, false, "Use constraints for column substitution", 0) \ - M(Bool, optimize_in_to_equal, true, "Convert the constant in to equal", 0) \ M(Bool, optimize_append_index, false, "Use constraints in order to append index condition (indexHint)", 0) \ M(Bool, optimize_time_filter_with_preimage, true, "Optimize Date and DateTime predicates by converting functions into equivalent comparisons without conversions (e.g. toYear(col) = 2023 -> col >= '2023-01-01' AND col <= '2023-12-31')", 0) \ M(Bool, normalize_function_names, true, "Normalize function names to their canonical names", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 886dce72176..8e2b2915c2a 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -96,7 +96,6 @@ static std::map sett {"parallel_replicas_allow_in_with_subquery", false, true, "If true, subquery for IN will be executed on every follower replica"}, {"filesystem_cache_reserve_space_wait_lock_timeout_milliseconds", 1000, 1000, "Wait time to lock cache for sapce reservation in filesystem cache"}, {"max_parser_backtracks", 0, 1000000, "Limiting the complexity of parsing"}, - {"optimize_in_to_equal", false, true, "Convert in single value to equal"}, }}, {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, From 45bbef689a053489a141bd8e2e20dc422c1d11d2 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Wed, 20 Mar 2024 12:29:48 +0800 Subject: [PATCH 038/102] fix error for in Null --- src/Analyzer/Passes/ConvertInToEqualPass.cpp | 7 +++++-- .../0_stateless/03013_optimize_in_to_equal.reference | 6 ++++++ tests/queries/0_stateless/03013_optimize_in_to_equal.sql | 4 +++- 3 files changed, 14 insertions(+), 3 deletions(-) diff --git a/src/Analyzer/Passes/ConvertInToEqualPass.cpp b/src/Analyzer/Passes/ConvertInToEqualPass.cpp index ce2cec9f9ed..45b8a91f7ad 100644 --- a/src/Analyzer/Passes/ConvertInToEqualPass.cpp +++ b/src/Analyzer/Passes/ConvertInToEqualPass.cpp @@ -32,9 +32,12 @@ public: auto * constant_node = args[1]->as(); if (!column_node || !constant_node) return ; + // IN multiple values is not supported if (constant_node->getValue().getType() == Field::Types::Which::Tuple) - return; - + return ; + // x IN null not equivalent to x = null + if (constant_node->hasSourceExpression() || constant_node->getValue().isNull()) + return ; auto equal_resolver = createInternalFunctionEqualOverloadResolver(); auto equal = std::make_shared("equals"); QueryTreeNodes arguments{column_node->clone(), constant_node->clone()}; diff --git a/tests/queries/0_stateless/03013_optimize_in_to_equal.reference b/tests/queries/0_stateless/03013_optimize_in_to_equal.reference index f55a7b76eaf..9ef4de0f607 100644 --- a/tests/queries/0_stateless/03013_optimize_in_to_equal.reference +++ b/tests/queries/0_stateless/03013_optimize_in_to_equal.reference @@ -1,5 +1,11 @@ a 1 ------------------- +0 +0 +0 +0 +0 +------------------- QUERY id: 0 PROJECTION COLUMNS x String diff --git a/tests/queries/0_stateless/03013_optimize_in_to_equal.sql b/tests/queries/0_stateless/03013_optimize_in_to_equal.sql index 9d80825ceb4..e4dba3a0aa2 100644 --- a/tests/queries/0_stateless/03013_optimize_in_to_equal.sql +++ b/tests/queries/0_stateless/03013_optimize_in_to_equal.sql @@ -5,6 +5,8 @@ INSERT INTO test VALUES ('a', 1), ('b', 2), ('c', 3), ('d', 4), ('e', 5); select * from test where x in ('a') SETTINGS allow_experimental_analyzer = 1; select '-------------------'; +select x in Null from test; +select '-------------------'; explain query tree select * from test where x in ('a') SETTINGS allow_experimental_analyzer = 1; select '-------------------'; -explain query tree select * from test where x in ('a','b') SETTINGS allow_experimental_analyzer = 1; \ No newline at end of file +explain query tree select * from test where x in ('a','b') SETTINGS allow_experimental_analyzer = 1; From d2d5f3b323c3e8b7a4a9823ea14b48b516030907 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Wed, 20 Mar 2024 17:29:35 +0800 Subject: [PATCH 039/102] add not in --- src/Analyzer/Passes/ConvertInToEqualPass.cpp | 38 ++++++++++++------- src/Analyzer/Passes/ConvertInToEqualPass.h | 8 +++- src/Functions/CMakeLists.txt | 2 + src/Functions/equals.cpp | 5 +++ src/Functions/equals.h | 11 ++++++ src/Functions/notEquals.cpp | 5 +++ src/Functions/notEquals.h | 11 ++++++ .../03013_optimize_in_to_equal.reference | 27 ++++++++++++- .../03013_optimize_in_to_equal.sql | 10 +++-- 9 files changed, 98 insertions(+), 19 deletions(-) create mode 100644 src/Functions/equals.h create mode 100644 src/Functions/notEquals.h diff --git a/src/Analyzer/Passes/ConvertInToEqualPass.cpp b/src/Analyzer/Passes/ConvertInToEqualPass.cpp index 45b8a91f7ad..fbac09de7aa 100644 --- a/src/Analyzer/Passes/ConvertInToEqualPass.cpp +++ b/src/Analyzer/Passes/ConvertInToEqualPass.cpp @@ -3,29 +3,28 @@ #include #include #include -#include -#include +#include +#include namespace DB { -using FunctionEquals = FunctionComparison; - class ConvertInToEqualPassVisitor : public InDepthQueryTreeVisitorWithContext { public: using Base = InDepthQueryTreeVisitorWithContext; using Base::Base; - FunctionOverloadResolverPtr createInternalFunctionEqualOverloadResolver() - { - return std::make_unique(std::make_shared(getContext()->getSettings().decimal_check_overflow)); - } - void enterImpl(QueryTreeNodePtr & node) { + static const std::unordered_map MAPPING = { + {"in", "equals"}, + {"notIn", "notEquals"} + }; auto * func_node = node->as(); - if (!func_node || func_node->getFunctionName() != "in" || func_node->getArguments().getNodes().size() != 2) + if (!func_node + || !MAPPING.contains(func_node->getFunctionName()) + || func_node->getArguments().getNodes().size() != 2) return ; auto args = func_node->getArguments().getNodes(); auto * column_node = args[0]->as(); @@ -38,13 +37,26 @@ public: // x IN null not equivalent to x = null if (constant_node->hasSourceExpression() || constant_node->getValue().isNull()) return ; - auto equal_resolver = createInternalFunctionEqualOverloadResolver(); - auto equal = std::make_shared("equals"); + auto result_func_name = MAPPING.at(func_node->getFunctionName()); + auto equal = std::make_shared(result_func_name); QueryTreeNodes arguments{column_node->clone(), constant_node->clone()}; equal->getArguments().getNodes() = std::move(arguments); - equal->resolveAsFunction(equal_resolver); + FunctionOverloadResolverPtr resolver; + bool decimal_check_overflow = getContext()->getSettingsRef().decimal_check_overflow; + if (result_func_name == "equals") + { + resolver = createInternalFunctionEqualOverloadResolver(decimal_check_overflow); + } + else + { + resolver = createInternalFunctionNotEqualOverloadResolver(decimal_check_overflow); + } + equal->resolveAsFunction(resolver); node = equal; } +private: + FunctionOverloadResolverPtr equal_resolver; + FunctionOverloadResolverPtr not_equal_resolver; }; void ConvertInToEqualPass::run(QueryTreeNodePtr & query_tree_node, ContextPtr context) diff --git a/src/Analyzer/Passes/ConvertInToEqualPass.h b/src/Analyzer/Passes/ConvertInToEqualPass.h index 7de75e94f04..bd4f8607c88 100644 --- a/src/Analyzer/Passes/ConvertInToEqualPass.h +++ b/src/Analyzer/Passes/ConvertInToEqualPass.h @@ -5,9 +5,15 @@ namespace DB { /** Optimize `in` to `equals` if possible. - * + * 1. convert in single value to equal * Example: SELECT * from test where x IN (1); * Result: SELECT * from test where x = 1; + * + * 2. convert not in single value to notEqual + * Example: SELECT * from test where x NOT IN (1); + * Result: SELECT * from test where x != 1; + * + * If value is null or tuple, do not convert. */ class ConvertInToEqualPass final : public IQueryTreePass { diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index ac3e3671ae0..399394011b5 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -14,6 +14,8 @@ extract_into_parent_list(clickhouse_functions_sources dbms_sources multiMatchAny.cpp checkHyperscanRegexp.cpp array/has.cpp + equals.cpp + notEquals.cpp CastOverloadResolver.cpp ) extract_into_parent_list(clickhouse_functions_headers dbms_headers diff --git a/src/Functions/equals.cpp b/src/Functions/equals.cpp index 5c59daf0537..512abaa6fc7 100644 --- a/src/Functions/equals.cpp +++ b/src/Functions/equals.cpp @@ -13,6 +13,11 @@ REGISTER_FUNCTION(Equals) factory.registerFunction(); } +FunctionOverloadResolverPtr createInternalFunctionEqualOverloadResolver(bool decimal_check_overflow) +{ + return std::make_unique(std::make_shared(decimal_check_overflow)); +} + template <> ColumnPtr FunctionComparison::executeTupleImpl( const ColumnsWithTypeAndName & x, const ColumnsWithTypeAndName & y, size_t tuple_size, size_t input_rows_count) const diff --git a/src/Functions/equals.h b/src/Functions/equals.h new file mode 100644 index 00000000000..855cba4db3e --- /dev/null +++ b/src/Functions/equals.h @@ -0,0 +1,11 @@ +#pragma once +#include + +namespace DB +{ + +class IFunctionOverloadResolver; +using FunctionOverloadResolverPtr = std::shared_ptr; + +FunctionOverloadResolverPtr createInternalFunctionEqualOverloadResolver(bool decimal_check_overflow); +} diff --git a/src/Functions/notEquals.cpp b/src/Functions/notEquals.cpp index 3a63db46711..744a0997d95 100644 --- a/src/Functions/notEquals.cpp +++ b/src/Functions/notEquals.cpp @@ -12,6 +12,11 @@ REGISTER_FUNCTION(NotEquals) factory.registerFunction(); } +FunctionOverloadResolverPtr createInternalFunctionNotEqualOverloadResolver(bool decimal_check_overflow) +{ + return std::make_unique(std::make_shared(decimal_check_overflow)); +} + template <> ColumnPtr FunctionComparison::executeTupleImpl( const ColumnsWithTypeAndName & x, const ColumnsWithTypeAndName & y, size_t tuple_size, size_t input_rows_count) const diff --git a/src/Functions/notEquals.h b/src/Functions/notEquals.h new file mode 100644 index 00000000000..961889d68d7 --- /dev/null +++ b/src/Functions/notEquals.h @@ -0,0 +1,11 @@ +#pragma once +#include + +namespace DB +{ + +class IFunctionOverloadResolver; +using FunctionOverloadResolverPtr = std::shared_ptr; + +FunctionOverloadResolverPtr createInternalFunctionNotEqualOverloadResolver(bool decimal_check_overflow); +} diff --git a/tests/queries/0_stateless/03013_optimize_in_to_equal.reference b/tests/queries/0_stateless/03013_optimize_in_to_equal.reference index 9ef4de0f607..dcaa14abfb1 100644 --- a/tests/queries/0_stateless/03013_optimize_in_to_equal.reference +++ b/tests/queries/0_stateless/03013_optimize_in_to_equal.reference @@ -3,8 +3,6 @@ a 1 0 0 0 -0 -0 ------------------- QUERY id: 0 PROJECTION COLUMNS @@ -41,3 +39,28 @@ QUERY id: 0 COLUMN id: 7, column_name: x, result_type: String, source_id: 3 CONSTANT id: 8, constant_value: Tuple_(\'a\', \'b\'), constant_value_type: Tuple(String, String) SETTINGS allow_experimental_analyzer=1 +------------------- +b 2 +c 3 +------------------- +QUERY id: 0 + PROJECTION COLUMNS + x String + y Int32 + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: x, result_type: String, source_id: 3 + COLUMN id: 4, column_name: y, result_type: Int32, source_id: 3 + JOIN TREE + TABLE id: 3, alias: __table1, table_name: default.test + WHERE + FUNCTION id: 5, function_name: notEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 6, nodes: 2 + COLUMN id: 7, column_name: x, result_type: String, source_id: 3 + CONSTANT id: 8, constant_value: \'a\', constant_value_type: String + SETTINGS allow_experimental_analyzer=1 +------------------- +a 1 +b 2 +c 3 diff --git a/tests/queries/0_stateless/03013_optimize_in_to_equal.sql b/tests/queries/0_stateless/03013_optimize_in_to_equal.sql index e4dba3a0aa2..be4dc2275a9 100644 --- a/tests/queries/0_stateless/03013_optimize_in_to_equal.sql +++ b/tests/queries/0_stateless/03013_optimize_in_to_equal.sql @@ -1,8 +1,6 @@ DROP TABLE IF EXISTS test; CREATE TABLE test (x String, y Int32) ENGINE = MergeTree() ORDER BY x; - -INSERT INTO test VALUES ('a', 1), ('b', 2), ('c', 3), ('d', 4), ('e', 5); - +INSERT INTO test VALUES ('a', 1), ('b', 2), ('c', 3); select * from test where x in ('a') SETTINGS allow_experimental_analyzer = 1; select '-------------------'; select x in Null from test; @@ -10,3 +8,9 @@ select '-------------------'; explain query tree select * from test where x in ('a') SETTINGS allow_experimental_analyzer = 1; select '-------------------'; explain query tree select * from test where x in ('a','b') SETTINGS allow_experimental_analyzer = 1; +select '-------------------'; +select * from test where x not in ('a') SETTINGS allow_experimental_analyzer = 1; +select '-------------------'; +explain query tree select * from test where x not in ('a') SETTINGS allow_experimental_analyzer = 1; +select '-------------------'; +select * from test where x not in (NULL) SETTINGS allow_experimental_analyzer = 1; From 2c0e266cfe13b48f9cb149683a5405a6b2fa41e1 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 20 Mar 2024 10:58:46 +0100 Subject: [PATCH 040/102] Addressed review comments --- src/Common/BufferAllocationPolicy.cpp | 2 +- src/Common/BufferAllocationPolicy.h | 3 +- .../IO/WriteBufferFromAzureBlobStorage.cpp | 53 +++++++++++-------- src/IO/WriteBufferFromS3.cpp | 32 +++++------ src/IO/WriteBufferFromS3.h | 1 - 5 files changed, 47 insertions(+), 44 deletions(-) diff --git a/src/Common/BufferAllocationPolicy.cpp b/src/Common/BufferAllocationPolicy.cpp index 14d009008b9..359da0f8313 100644 --- a/src/Common/BufferAllocationPolicy.cpp +++ b/src/Common/BufferAllocationPolicy.cpp @@ -73,7 +73,7 @@ public: if (1 == buffer_number) { - current_size = first_size; + current_size = std::min(size_t(DBMS_DEFAULT_BUFFER_SIZE), first_size); return; } diff --git a/src/Common/BufferAllocationPolicy.h b/src/Common/BufferAllocationPolicy.h index dd04139e694..4ac20f1605b 100644 --- a/src/Common/BufferAllocationPolicy.h +++ b/src/Common/BufferAllocationPolicy.h @@ -9,6 +9,7 @@ namespace DB { +/// Buffer number starts with 0 class IBufferAllocationPolicy { public: @@ -20,7 +21,7 @@ public: size_t max_size = 5ULL * 1024 * 1024 * 1024; size_t multiply_factor = 2; size_t multiply_parts_count_threshold = 500; - size_t max_single_size = 32 * 1024 * 1024; + size_t max_single_size = 32 * 1024 * 1024; /// Max size for a single buffer/block }; diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index f966e431506..1ef54272295 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -18,6 +18,25 @@ namespace ProfileEvents namespace DB { +struct WriteBufferFromAzureBlobStorage::PartData +{ + Memory<> memory; + size_t data_size = 0; +}; + +IBufferAllocationPolicy::IBufferAllocationPolicyPtr createBufferAllocationPolicy(const AzureObjectStorageSettings & settings) +{ + IBufferAllocationPolicy::Settings allocation_settings; + allocation_settings.strict_size = settings.strict_upload_part_size; + allocation_settings.min_size = settings.min_upload_part_size; + allocation_settings.max_size = settings.max_upload_part_size; + allocation_settings.multiply_factor = settings.upload_part_size_multiply_factor; + allocation_settings.multiply_parts_count_threshold = settings.upload_part_size_multiply_parts_count_threshold; + allocation_settings.max_single_size = settings.max_single_part_upload_size; + + return IBufferAllocationPolicy::create(allocation_settings); +} + WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage( std::shared_ptr blob_container_client_, const String & blob_path_, @@ -27,12 +46,7 @@ WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage( ThreadPoolCallbackRunner schedule_) : WriteBufferFromFileBase(buf_size_, nullptr, 0) , log(getLogger("WriteBufferFromAzureBlobStorage")) - , buffer_allocation_policy(IBufferAllocationPolicy::create({settings_->strict_upload_part_size, - settings_->min_upload_part_size, - settings_->max_upload_part_size, - settings_->upload_part_size_multiply_factor, - settings_->upload_part_size_multiply_parts_count_threshold, - settings_->max_single_part_upload_size})) + , buffer_allocation_policy(createBufferAllocationPolicy(*settings_.get())) , max_single_part_upload_size(settings_->max_single_part_upload_size) , max_unexpected_write_error_retries(settings_->max_unexpected_write_error_retries) , blob_path(blob_path_) @@ -91,6 +105,8 @@ void WriteBufferFromAzureBlobStorage::finalizeImpl() { auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); + /// If there is only one block and size is less than or equal to max_single_part_upload_size + /// then we use single part upload instead of multi part upload if (buffer_allocation_policy->getBufferNumber() == 1) { size_t data_size = size_t(position() - memory.data()); @@ -124,39 +140,30 @@ void WriteBufferFromAzureBlobStorage::allocateBuffer() { buffer_allocation_policy->nextBuffer(); auto size = buffer_allocation_policy->getBufferSize(); - - if (buffer_allocation_policy->getBufferNumber() == 1) - { - size = std::min(size_t(DBMS_DEFAULT_BUFFER_SIZE), size); - } - - LOG_INFO(log, "allocateBuffer size = {}", size); - memory = Memory(size); WriteBuffer::set(memory.data(), memory.size()); } void WriteBufferFromAzureBlobStorage::writePart() { - size_t data_size = size_t(position() - memory.data()); - auto data = std::move(memory); + std::shared_ptr part_data; + auto data_size = size_t(position() - memory.data()); + part_data = std::make_shared(std::move(memory), data_size); WriteBuffer::set(nullptr, 0); - if (data_size == 0) + if (part_data->data_size == 0) return; - LOG_TRACE(log, "writePart data size `{}`", data_size); - - auto upload_worker = [&] () + auto upload_worker = [&, part_data] () { auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); const std::string & block_id = block_ids.emplace_back(getRandomASCIIString(64)); - Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(data.data()), data_size); - execWithRetry([&](){ block_blob_client.StageBlock(block_id, memory_stream); }, max_unexpected_write_error_retries, data_size); + Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(part_data->memory.data()), part_data->data_size); + execWithRetry([&](){ block_blob_client.StageBlock(block_id, memory_stream); }, max_unexpected_write_error_retries, part_data->data_size); if (write_settings.remote_throttler) - write_settings.remote_throttler->add(data_size, ProfileEvents::RemoteWriteThrottlerBytes, ProfileEvents::RemoteWriteThrottlerSleepMicroseconds); + write_settings.remote_throttler->add(part_data->data_size, ProfileEvents::RemoteWriteThrottlerBytes, ProfileEvents::RemoteWriteThrottlerSleepMicroseconds); }; task_tracker->add(std::move(upload_worker)); diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index c907c4cb6b6..3ee59c42079 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -74,6 +74,19 @@ struct WriteBufferFromS3::PartData } }; +IBufferAllocationPolicy::IBufferAllocationPolicyPtr createBufferAllocationPolicy(const S3Settings::RequestSettings::PartUploadSettings & settings) +{ + IBufferAllocationPolicy::Settings allocation_settings; + allocation_settings.strict_size = settings.strict_upload_part_size; + allocation_settings.min_size = settings.min_upload_part_size; + allocation_settings.max_size = settings.max_upload_part_size; + allocation_settings.multiply_factor = settings.upload_part_size_multiply_factor; + allocation_settings.multiply_parts_count_threshold = settings.upload_part_size_multiply_parts_count_threshold; + allocation_settings.max_single_size = settings.max_single_part_upload_size; + + return IBufferAllocationPolicy::create(allocation_settings); +} + WriteBufferFromS3::WriteBufferFromS3( std::shared_ptr client_ptr_, @@ -93,12 +106,7 @@ WriteBufferFromS3::WriteBufferFromS3( , write_settings(write_settings_) , client_ptr(std::move(client_ptr_)) , object_metadata(std::move(object_metadata_)) - , buffer_allocation_policy(IBufferAllocationPolicy::create({upload_settings.strict_upload_part_size, - upload_settings.min_upload_part_size, - upload_settings.max_upload_part_size, - upload_settings.upload_part_size_multiply_factor, - upload_settings.upload_part_size_multiply_parts_count_threshold, - upload_settings.max_single_part_upload_size})) + , buffer_allocation_policy(createBufferAllocationPolicy(upload_settings)) , task_tracker( std::make_unique( std::move(schedule_), @@ -327,22 +335,10 @@ void WriteBufferFromS3::detachBuffer() detached_part_data.push_back({std::move(buf), data_size}); } -void WriteBufferFromS3::allocateFirstBuffer() -{ - const auto max_first_buffer = buffer_allocation_policy->getBufferSize(); - const auto size = std::min(size_t(DBMS_DEFAULT_BUFFER_SIZE), max_first_buffer); - memory = Memory(size); - WriteBuffer::set(memory.data(), memory.size()); -} - void WriteBufferFromS3::allocateBuffer() { buffer_allocation_policy->nextBuffer(); chassert(0 == hidden_size); - - if (buffer_allocation_policy->getBufferNumber() == 1) - return allocateFirstBuffer(); - memory = Memory(buffer_allocation_policy->getBufferSize()); WriteBuffer::set(memory.data(), memory.size()); } diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 201743e7f9c..3d2aed74e88 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -59,7 +59,6 @@ private: struct PartData; void hidePartialData(); - void allocateFirstBuffer(); void reallocateFirstBuffer(); void detachBuffer(); void allocateBuffer(); From 040b191899c8dab75ebc0c8149a170b177264a46 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 20 Mar 2024 20:25:38 +0100 Subject: [PATCH 041/102] Fix readBigAt --- src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index 13f62c7ac25..5941dca0b48 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -279,26 +279,24 @@ size_t ReadBufferFromAzureBlobStorage::readBigAt(char * to, size_t n, size_t ran sleep_time_with_backoff_milliseconds *= 2; }; - Azure::Storage::Blobs::DownloadBlobOptions download_options; - download_options.Range = {static_cast(range_begin), range_begin+n}; - for (size_t i = 0; i < max_single_download_retries; ++i) + + for (size_t i = 0; i < max_single_download_retries && n > 0; ++i) { size_t bytes_copied = 0; try { + Azure::Storage::Blobs::DownloadBlobOptions download_options; + download_options.Range = {static_cast(range_begin), n}; auto download_response = blob_client->Download(download_options); + std::unique_ptr body_stream = std::move(download_response.Value.BodyStream); - auto length = body_stream->Length(); - char buffer[length]; - body_stream->Read(reinterpret_cast(buffer), length); - std::istringstream string_stream(String(static_cast(buffer),length)); // STYLE_CHECK_ALLOW_STD_STRING_STREAM + auto bytes = body_stream->ReadToCount(reinterpret_cast(data_ptr), body_stream->Length()); + std::istringstream string_stream(String(static_cast(data_ptr),bytes)); // STYLE_CHECK_ALLOW_STD_STRING_STREAM copyFromIStreamWithProgressCallback(string_stream, to, n, progress_callback, &bytes_copied); if (read_settings.remote_throttler) read_settings.remote_throttler->add(bytes_copied, ProfileEvents::RemoteReadThrottlerBytes, ProfileEvents::RemoteReadThrottlerSleepMicroseconds); - - break; } catch (const Azure::Core::RequestFailedException & e) { From dcaa71245164289ff9d1b74fc2516950558170a8 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Tue, 19 Mar 2024 23:43:01 +0000 Subject: [PATCH 042/102] disallow LowCardinality input type for JSONExtract Workaraund for a memory corruption issue https://github.com/ClickHouse/ClickHouse/issues/61562 It seems that the root cause lies not within the parser itself, but rather either with the Columns/ColumnLowCardinality or Functions/IFunction code paths. --- src/Functions/FunctionsJSON.h | 4 +--- tests/queries/0_stateless/00918_json_functions.reference | 1 + tests/queries/0_stateless/00918_json_functions.sql | 2 ++ 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Functions/FunctionsJSON.h b/src/Functions/FunctionsJSON.h index 2539fa1aeb4..53515985e39 100644 --- a/src/Functions/FunctionsJSON.h +++ b/src/Functions/FunctionsJSON.h @@ -348,6 +348,7 @@ public: String getName() const override { return Name::name; } bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { @@ -469,9 +470,6 @@ public: else return_type = json_return_type; - /// Top-level LowCardinality columns are processed outside JSON parser. - json_return_type = removeLowCardinality(json_return_type); - DataTypes argument_types; argument_types.reserve(arguments.size()); for (const auto & argument : arguments) diff --git a/tests/queries/0_stateless/00918_json_functions.reference b/tests/queries/0_stateless/00918_json_functions.reference index 43b15ded93d..7b725111755 100644 --- a/tests/queries/0_stateless/00918_json_functions.reference +++ b/tests/queries/0_stateless/00918_json_functions.reference @@ -69,6 +69,7 @@ hello (3333.6,'test') (3333.6333333333,'test') (3333.6333333333,'test') +\N 123456.1234 Decimal(20, 4) 123456.1234 Decimal(20, 4) 123456789012345.12 Decimal(30, 4) diff --git a/tests/queries/0_stateless/00918_json_functions.sql b/tests/queries/0_stateless/00918_json_functions.sql index e19dd17670e..61fcb21fcbd 100644 --- a/tests/queries/0_stateless/00918_json_functions.sql +++ b/tests/queries/0_stateless/00918_json_functions.sql @@ -81,6 +81,7 @@ SELECT JSONExtract('{"a":3333.6333333333333333333333, "b":"test"}', 'Tuple(a Dec SELECT JSONExtract('{"a":"3333.6333333333333333333333", "b":"test"}', 'Tuple(a Decimal(10,1), b LowCardinality(String))'); SELECT JSONExtract('{"a":3333.6333333333333333333333, "b":"test"}', 'Tuple(a Decimal(20,10), b LowCardinality(String))'); SELECT JSONExtract('{"a":"3333.6333333333333333333333", "b":"test"}', 'Tuple(a Decimal(20,10), b LowCardinality(String))'); +SELECT JSONExtract(materialize('{"string_value":null}'), materialize('string_value'), 'LowCardinality(Nullable(String))'); SELECT JSONExtract('{"a":123456.123456}', 'a', 'Decimal(20, 4)') as a, toTypeName(a); SELECT JSONExtract('{"a":"123456.123456"}', 'a', 'Decimal(20, 4)') as a, toTypeName(a); SELECT JSONExtract('{"a":"123456789012345.12"}', 'a', 'Decimal(30, 4)') as a, toTypeName(a); @@ -326,3 +327,4 @@ SELECT JSONExtract('[]', JSONExtract('0', 'UInt256'), 'UInt256'); -- { serverErr SELECT '--show error: key of map type should be String'; SELECT JSONExtract('{"a": [100.0, 200], "b": [-100, 200.0, 300]}', 'Map(Int64, Array(Float64))'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT JSONExtract(materialize(toLowCardinality('{"string_value":null}')), materialize('string_value'), 'LowCardinality(Nullable(String))'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT } From ada4384deb5fe0c4dd95459da9ac8a225cf92a51 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Thu, 21 Mar 2024 10:38:51 +0800 Subject: [PATCH 043/102] fix ut failed --- src/Analyzer/Passes/ConvertInToEqualPass.cpp | 16 +++++++++++----- ...est_transform_query_for_external_database.cpp | 6 ++++-- 2 files changed, 15 insertions(+), 7 deletions(-) diff --git a/src/Analyzer/Passes/ConvertInToEqualPass.cpp b/src/Analyzer/Passes/ConvertInToEqualPass.cpp index fbac09de7aa..ddd8a8f266f 100644 --- a/src/Analyzer/Passes/ConvertInToEqualPass.cpp +++ b/src/Analyzer/Passes/ConvertInToEqualPass.cpp @@ -32,7 +32,8 @@ public: if (!column_node || !constant_node) return ; // IN multiple values is not supported - if (constant_node->getValue().getType() == Field::Types::Which::Tuple) + if (constant_node->getValue().getType() == Field::Types::Which::Tuple + || constant_node->getValue().getType() == Field::Types::Which::Array) return ; // x IN null not equivalent to x = null if (constant_node->hasSourceExpression() || constant_node->getValue().isNull()) @@ -51,12 +52,17 @@ public: { resolver = createInternalFunctionNotEqualOverloadResolver(decimal_check_overflow); } - equal->resolveAsFunction(resolver); + try + { + equal->resolveAsFunction(resolver); + } + catch (...) + { + // When function resolver fails, we should not replace the function node + return; + } node = equal; } -private: - FunctionOverloadResolverPtr equal_resolver; - FunctionOverloadResolverPtr not_equal_resolver; }; void ConvertInToEqualPass::run(QueryTreeNodePtr & query_tree_node, ContextPtr context) diff --git a/src/Storages/tests/gtest_transform_query_for_external_database.cpp b/src/Storages/tests/gtest_transform_query_for_external_database.cpp index 7e2d393c3d1..6490498d717 100644 --- a/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -306,7 +306,8 @@ TEST(TransformQueryForExternalDatabase, Aliases) check(state, 1, {"field"}, "SELECT field AS value, field AS display FROM table WHERE field NOT IN ('') AND display LIKE '%test%'", - R"(SELECT "field" FROM "test"."table" WHERE ("field" NOT IN ('')) AND ("field" LIKE '%test%'))"); + R"(SELECT "field" FROM "test"."table" WHERE ("field" NOT IN ('')) AND ("field" LIKE '%test%'))", + R"(SELECT "field" FROM "test"."table" WHERE ("field" != '') AND ("field" LIKE '%test%'))"); } TEST(TransformQueryForExternalDatabase, ForeignColumnInWhere) @@ -408,5 +409,6 @@ TEST(TransformQueryForExternalDatabase, Analyzer) check(state, 1, {"column", "apply_id", "apply_type", "apply_status", "create_time", "field", "value", "a", "b", "foo"}, "SELECT * FROM table WHERE (column) IN (1)", - R"(SELECT "column", "apply_id", "apply_type", "apply_status", "create_time", "field", "value", "a", "b", "foo" FROM "test"."table" WHERE "column" IN (1))"); + R"(SELECT "column", "apply_id", "apply_type", "apply_status", "create_time", "field", "value", "a", "b", "foo" FROM "test"."table" WHERE "column" IN (1))", + R"(SELECT "column", "apply_id", "apply_type", "apply_status", "create_time", "field", "value", "a", "b", "foo" FROM "test"."table" WHERE "column" = 1)"); } From 8d9a58c93614cfc01bbd93765e3e01dc26da4e76 Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Thu, 21 Mar 2024 02:17:58 +0000 Subject: [PATCH 044/102] LowCardinalityFixedStringColumn: allow generic types Fix tests and add new ones. --- src/Functions/FunctionsJSON.h | 9 ++------- .../02474_extract_fixedstring_from_json.reference | 7 +++++++ .../0_stateless/02474_extract_fixedstring_from_json.sql | 7 +++++++ 3 files changed, 16 insertions(+), 7 deletions(-) diff --git a/src/Functions/FunctionsJSON.h b/src/Functions/FunctionsJSON.h index 53515985e39..8a193785f87 100644 --- a/src/Functions/FunctionsJSON.h +++ b/src/Functions/FunctionsJSON.h @@ -865,11 +865,9 @@ struct JSONExtractTree explicit LowCardinalityFixedStringNode(const size_t fixed_length_) : fixed_length(fixed_length_) { } bool insertResultToColumn(IColumn & dest, const Element & element) override { - // If element is an object we delegate the insertion to JSONExtractRawImpl - if (element.isObject()) + // For types other than string, delegate the insertion to JSONExtractRawImpl. + if (!element.isString()) return JSONExtractRawImpl::insertResultToLowCardinalityFixedStringColumn(dest, element, fixed_length); - else if (!element.isString()) - return false; auto str = element.getString(); if (str.size() > fixed_length) @@ -1484,9 +1482,6 @@ public: // We use insertResultToLowCardinalityFixedStringColumn in case we are inserting raw data in a Low Cardinality FixedString column static bool insertResultToLowCardinalityFixedStringColumn(IColumn & dest, const Element & element, size_t fixed_length) { - if (element.getObject().size() > fixed_length) - return false; - ColumnFixedString::Chars chars; WriteBufferFromVector buf(chars, AppendModeTag()); traverse(element, buf); diff --git a/tests/queries/0_stateless/02474_extract_fixedstring_from_json.reference b/tests/queries/0_stateless/02474_extract_fixedstring_from_json.reference index 783d12fcf1a..21ddf5d3512 100644 --- a/tests/queries/0_stateless/02474_extract_fixedstring_from_json.reference +++ b/tests/queries/0_stateless/02474_extract_fixedstring_from_json.reference @@ -8,3 +8,10 @@ \0\0\0\0\0 131231 131231 +1234 +1234 +{"b":131231} +\0\0\0\0 +1234567890 +18446744073709551615 +-9223372036854775807 diff --git a/tests/queries/0_stateless/02474_extract_fixedstring_from_json.sql b/tests/queries/0_stateless/02474_extract_fixedstring_from_json.sql index cfc47e00cba..bbb9f55062b 100644 --- a/tests/queries/0_stateless/02474_extract_fixedstring_from_json.sql +++ b/tests/queries/0_stateless/02474_extract_fixedstring_from_json.sql @@ -6,3 +6,10 @@ SELECT JSONExtract('{"a": 123456}', 'a', 'FixedString(5)'); SELECT JSONExtract('{"a": 123456}', 'a', 'FixedString(6)'); SELECT JSONExtract(materialize('{"a": 131231}'), 'a', 'LowCardinality(FixedString(5))') FROM numbers(2); SELECT JSONExtract(materialize('{"a": 131231}'), 'a', 'LowCardinality(FixedString(6))') FROM numbers(2); +SELECT JSONExtract(materialize('{"a": 131231, "b": 1234}'), 'b', 'LowCardinality(FixedString(4))'); +SELECT JSONExtract(materialize('{"a": 131231, "b": "1234"}'), 'b', 'LowCardinality(FixedString(4))'); +SELECT JSONExtract(materialize('{"a": {"b": 131231} }'), 'a', 'LowCardinality(FixedString(12))'); +SELECT JSONExtract(materialize('{"a": 131231, "b": 1234567890}'), 'b', 'LowCardinality(FixedString(4))'); +SELECT JSONExtract(materialize('{"a": 131231, "b": 1234567890}'), 'b', 'LowCardinality(FixedString(10))'); +SELECT JSONExtract(materialize('{"a": 18446744073709551615}'), 'a', 'LowCardinality(FixedString(20))'); +SELECT JSONExtract(materialize('{"a": -9223372036854775807}'), 'a', 'LowCardinality(FixedString(20))'); From 7a3ab461bfe88b6d073ac8284ac5f12841b120cf Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 21 Mar 2024 09:24:35 +0100 Subject: [PATCH 045/102] Fix style check --- src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp | 7 ------- 1 file changed, 7 deletions(-) diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index 5941dca0b48..3555c71abee 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -29,7 +29,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } - ReadBufferFromAzureBlobStorage::ReadBufferFromAzureBlobStorage( std::shared_ptr blob_container_client_, const String & path_, @@ -58,7 +57,6 @@ ReadBufferFromAzureBlobStorage::ReadBufferFromAzureBlobStorage( } } - void ReadBufferFromAzureBlobStorage::setReadUntilEnd() { if (read_until_position) @@ -141,7 +139,6 @@ bool ReadBufferFromAzureBlobStorage::nextImpl() return true; } - off_t ReadBufferFromAzureBlobStorage::seek(off_t offset_, int whence) { if (offset_ == getPosition() && whence == SEEK_SET) @@ -195,13 +192,11 @@ off_t ReadBufferFromAzureBlobStorage::seek(off_t offset_, int whence) return offset; } - off_t ReadBufferFromAzureBlobStorage::getPosition() { return offset - available(); } - void ReadBufferFromAzureBlobStorage::initialize() { if (initialized) @@ -279,8 +274,6 @@ size_t ReadBufferFromAzureBlobStorage::readBigAt(char * to, size_t n, size_t ran sleep_time_with_backoff_milliseconds *= 2; }; - - for (size_t i = 0; i < max_single_download_retries && n > 0; ++i) { size_t bytes_copied = 0; From e0d14a1eaf2028ca806ebef0a64b555798a57988 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 21 Mar 2024 11:10:54 +0100 Subject: [PATCH 046/102] Updated lambda and name of BufferAllocationPolicy --- src/Common/BufferAllocationPolicy.cpp | 12 +++++----- src/Common/BufferAllocationPolicy.h | 12 +++++----- .../IO/WriteBufferFromAzureBlobStorage.cpp | 22 +++++++++---------- .../IO/WriteBufferFromAzureBlobStorage.h | 2 +- src/IO/WriteBufferFromS3.cpp | 6 ++--- src/IO/WriteBufferFromS3.h | 3 +-- 6 files changed, 28 insertions(+), 29 deletions(-) diff --git a/src/Common/BufferAllocationPolicy.cpp b/src/Common/BufferAllocationPolicy.cpp index 359da0f8313..980cbcca729 100644 --- a/src/Common/BufferAllocationPolicy.cpp +++ b/src/Common/BufferAllocationPolicy.cpp @@ -5,13 +5,13 @@ namespace DB { -class FixedSizeBufferAllocationPolicy : public IBufferAllocationPolicy +class FixedSizeBufferAllocationPolicy : public BufferAllocationPolicy { const size_t buffer_size = 0; size_t buffer_number = 0; public: - explicit FixedSizeBufferAllocationPolicy(const IBufferAllocationPolicy::Settings & settings_) + explicit FixedSizeBufferAllocationPolicy(const BufferAllocationPolicy::Settings & settings_) : buffer_size(settings_.strict_size) { chassert(buffer_size > 0); @@ -32,7 +32,7 @@ public: }; -class ExpBufferAllocationPolicy : public DB::IBufferAllocationPolicy +class ExpBufferAllocationPolicy : public DB::BufferAllocationPolicy { const size_t first_size = 0; const size_t second_size = 0; @@ -45,7 +45,7 @@ class ExpBufferAllocationPolicy : public DB::IBufferAllocationPolicy size_t buffer_number = 0; public: - explicit ExpBufferAllocationPolicy(const IBufferAllocationPolicy::Settings & settings_) + explicit ExpBufferAllocationPolicy(const BufferAllocationPolicy::Settings & settings_) : first_size(std::max(settings_.max_single_size, settings_.min_size)) , second_size(settings_.min_size) , multiply_factor(settings_.multiply_factor) @@ -89,9 +89,9 @@ public: }; -IBufferAllocationPolicy::~IBufferAllocationPolicy() = default; +BufferAllocationPolicy::~BufferAllocationPolicy() = default; -IBufferAllocationPolicy::IBufferAllocationPolicyPtr IBufferAllocationPolicy::create(IBufferAllocationPolicy::Settings settings_) +BufferAllocationPolicyPtr BufferAllocationPolicy::create(BufferAllocationPolicy::Settings settings_) { if (settings_.strict_size > 0) return std::make_unique(settings_); diff --git a/src/Common/BufferAllocationPolicy.h b/src/Common/BufferAllocationPolicy.h index 4ac20f1605b..7017891f9e2 100644 --- a/src/Common/BufferAllocationPolicy.h +++ b/src/Common/BufferAllocationPolicy.h @@ -9,8 +9,11 @@ namespace DB { +class BufferAllocationPolicy; +using BufferAllocationPolicyPtr = std::unique_ptr; + /// Buffer number starts with 0 -class IBufferAllocationPolicy +class BufferAllocationPolicy { public: @@ -24,15 +27,12 @@ public: size_t max_single_size = 32 * 1024 * 1024; /// Max size for a single buffer/block }; - virtual size_t getBufferNumber() const = 0; virtual size_t getBufferSize() const = 0; virtual void nextBuffer() = 0; - virtual ~IBufferAllocationPolicy() = 0; + virtual ~BufferAllocationPolicy() = 0; - using IBufferAllocationPolicyPtr = std::unique_ptr; - - static IBufferAllocationPolicyPtr create(Settings settings_); + static BufferAllocationPolicyPtr create(Settings settings_); }; diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 1ef54272295..ed3ce76bd94 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -22,11 +22,12 @@ struct WriteBufferFromAzureBlobStorage::PartData { Memory<> memory; size_t data_size = 0; + std::string block_id; }; -IBufferAllocationPolicy::IBufferAllocationPolicyPtr createBufferAllocationPolicy(const AzureObjectStorageSettings & settings) +BufferAllocationPolicyPtr createBufferAllocationPolicy(const AzureObjectStorageSettings & settings) { - IBufferAllocationPolicy::Settings allocation_settings; + BufferAllocationPolicy::Settings allocation_settings; allocation_settings.strict_size = settings.strict_upload_part_size; allocation_settings.min_size = settings.min_upload_part_size; allocation_settings.max_size = settings.max_upload_part_size; @@ -34,7 +35,7 @@ IBufferAllocationPolicy::IBufferAllocationPolicyPtr createBufferAllocationPolicy allocation_settings.multiply_parts_count_threshold = settings.upload_part_size_multiply_parts_count_threshold; allocation_settings.max_single_size = settings.max_single_part_upload_size; - return IBufferAllocationPolicy::create(allocation_settings); + return BufferAllocationPolicy::create(allocation_settings); } WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage( @@ -146,21 +147,20 @@ void WriteBufferFromAzureBlobStorage::allocateBuffer() void WriteBufferFromAzureBlobStorage::writePart() { - std::shared_ptr part_data; auto data_size = size_t(position() - memory.data()); - part_data = std::make_shared(std::move(memory), data_size); - WriteBuffer::set(nullptr, 0); - - if (part_data->data_size == 0) + if (data_size == 0) return; - auto upload_worker = [&, part_data] () + const std::string & block_id = block_ids.emplace_back(getRandomASCIIString(64)); + std::shared_ptr part_data = std::make_shared(std::move(memory), data_size, block_id); + WriteBuffer::set(nullptr, 0); + + auto upload_worker = [this, part_data] () { auto block_blob_client = blob_container_client->GetBlockBlobClient(blob_path); - const std::string & block_id = block_ids.emplace_back(getRandomASCIIString(64)); Azure::Core::IO::MemoryBodyStream memory_stream(reinterpret_cast(part_data->memory.data()), part_data->data_size); - execWithRetry([&](){ block_blob_client.StageBlock(block_id, memory_stream); }, max_unexpected_write_error_retries, part_data->data_size); + execWithRetry([&](){ block_blob_client.StageBlock(part_data->block_id, memory_stream); }, max_unexpected_write_error_retries, part_data->data_size); if (write_settings.remote_throttler) write_settings.remote_throttler->add(part_data->data_size, ProfileEvents::RemoteWriteThrottlerBytes, ProfileEvents::RemoteWriteThrottlerSleepMicroseconds); diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h index a210a75030a..6e10c07b255 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h @@ -58,7 +58,7 @@ private: LoggerPtr log; LogSeriesLimiterPtr limitedLog = std::make_shared(log, 1, 5); - IBufferAllocationPolicy::IBufferAllocationPolicyPtr buffer_allocation_policy; + BufferAllocationPolicyPtr buffer_allocation_policy; const size_t max_single_part_upload_size; const size_t max_unexpected_write_error_retries; diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 3ee59c42079..865bac86ff5 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -74,9 +74,9 @@ struct WriteBufferFromS3::PartData } }; -IBufferAllocationPolicy::IBufferAllocationPolicyPtr createBufferAllocationPolicy(const S3Settings::RequestSettings::PartUploadSettings & settings) +BufferAllocationPolicyPtr createBufferAllocationPolicy(const S3Settings::RequestSettings::PartUploadSettings & settings) { - IBufferAllocationPolicy::Settings allocation_settings; + BufferAllocationPolicy::Settings allocation_settings; allocation_settings.strict_size = settings.strict_upload_part_size; allocation_settings.min_size = settings.min_upload_part_size; allocation_settings.max_size = settings.max_upload_part_size; @@ -84,7 +84,7 @@ IBufferAllocationPolicy::IBufferAllocationPolicyPtr createBufferAllocationPolicy allocation_settings.multiply_parts_count_threshold = settings.upload_part_size_multiply_parts_count_threshold; allocation_settings.max_single_size = settings.max_single_part_upload_size; - return IBufferAllocationPolicy::create(allocation_settings); + return BufferAllocationPolicy::create(allocation_settings); } diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 3d2aed74e88..0eab20891bf 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -85,7 +85,7 @@ private: LoggerPtr log = getLogger("WriteBufferFromS3"); LogSeriesLimiterPtr limitedLog = std::make_shared(log, 1, 5); - IBufferAllocationPolicy::IBufferAllocationPolicyPtr buffer_allocation_policy; + BufferAllocationPolicyPtr buffer_allocation_policy; /// Upload in S3 is made in parts. /// We initiate upload, then upload each part and get ETag as a response, and then finalizeImpl() upload with listing all our parts. @@ -109,7 +109,6 @@ private: size_t total_size = 0; size_t hidden_size = 0; -// class TaskTracker; std::unique_ptr task_tracker; BlobStorageLogWriterPtr blob_log; From 81fd904b54327578c5f66e4452c77456178621a9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 21 Mar 2024 00:05:21 +0100 Subject: [PATCH 047/102] I hate REPLACE_RANGE and actual_part_name --- .../MergeTree/ReplicatedMergeTreeLogEntry.h | 1 + .../MergeTree/ReplicatedMergeTreeQueue.cpp | 65 ++++++++++++------- ..._move_partition_inactive_replica.reference | 8 +++ .../02916_move_partition_inactive_replica.sql | 46 +++++++++++++ 4 files changed, 98 insertions(+), 22 deletions(-) create mode 100644 tests/queries/0_stateless/02916_move_partition_inactive_replica.reference create mode 100644 tests/queries/0_stateless/02916_move_partition_inactive_replica.sql diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h index 054c576cfc5..7693f34cc1e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h @@ -93,6 +93,7 @@ struct ReplicatedMergeTreeLogEntryData MergeTreeDataPartFormat new_part_format; String block_id; /// For parts of level zero, the block identifier for deduplication (node name in /blocks/). mutable String actual_new_part_name; /// GET_PART could actually fetch a part covering 'new_part_name'. + mutable std::unordered_set replace_range_actual_new_part_names; /// Same as above, but for REPLACE_RANGE UUID new_part_uuid = UUIDHelpers::Nil; Strings source_parts; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 42f564f40da..ee4ed87d456 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -342,6 +342,11 @@ void ReplicatedMergeTreeQueue::updateStateOnQueueEntryRemoval( /// NOTE actual_new_part_name is very confusing and error-prone. This approach must be fixed. removeCoveredPartsFromMutations(entry->actual_new_part_name, /*remove_part = */ false, /*remove_covered_parts = */ true); } + for (const auto & actual_part : entry->replace_range_actual_new_part_names) + { + LOG_TEST(log, "Entry {} has actual new part name {}, removing it from mutations", entry->znode_name, actual_part); + removeCoveredPartsFromMutations(actual_part, /*remove_part = */ false, /*remove_covered_parts = */ true); + } LOG_TEST(log, "Adding parts [{}] to current parts", fmt::join(entry_virtual_parts, ", ")); @@ -1180,9 +1185,9 @@ bool ReplicatedMergeTreeQueue::isCoveredByFuturePartsImpl(const LogEntry & entry if (entry_for_same_part_it != future_parts.end()) { const LogEntry & another_entry = *entry_for_same_part_it->second; - constexpr auto fmt_string = "Not executing log entry {} of type {} for part {} " + constexpr auto fmt_string = "Not executing log entry {} of type {} for part {} (actual part {})" "because another log entry {} of type {} for the same part ({}) is being processed."; - LOG_INFO(LogToStr(out_reason, log), fmt_string, entry.znode_name, entry.type, entry.new_part_name, + LOG_INFO(LogToStr(out_reason, log), fmt_string, entry.znode_name, entry.type, entry.new_part_name, new_part_name, another_entry.znode_name, another_entry.type, another_entry.new_part_name); return true; @@ -1198,6 +1203,7 @@ bool ReplicatedMergeTreeQueue::isCoveredByFuturePartsImpl(const LogEntry & entry auto result_part = MergeTreePartInfo::fromPartName(new_part_name, format_version); /// It can slow down when the size of `future_parts` is large. But it can not be large, since background pool is limited. + /// (well, it can actually, thanks to REPLACE_RANGE, but it's a rare case) for (const auto & future_part_elem : future_parts) { auto future_part = MergeTreePartInfo::fromPartName(future_part_elem.first, format_version); @@ -1608,26 +1614,39 @@ void ReplicatedMergeTreeQueue::CurrentlyExecuting::setActualPartName( std::unique_lock & state_lock, std::vector & covered_entries_to_wait) { - if (!entry.actual_new_part_name.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Entry actual part isn't empty yet. This is a bug."); + if (actual_part_name.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Actual part name is empty"); - entry.actual_new_part_name = actual_part_name; + if (!entry.actual_new_part_name.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Entry {} actual part isn't empty yet: '{}'. This is a bug.", + entry.znode_name, entry.actual_new_part_name); + + auto actual_part_info = MergeTreePartInfo::fromPartName(actual_part_name, queue.format_version); + for (const auto & other_part_name : entry.replace_range_actual_new_part_names) + if (!MergeTreePartInfo::fromPartName(other_part_name, queue.format_version).isDisjoint(actual_part_info)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Entry {} already has actual part {} non-disjoint with {}. This is a bug.", + entry.actual_new_part_name, other_part_name, actual_part_name); /// Check if it is the same (and already added) part. - if (entry.actual_new_part_name == entry.new_part_name) + if (actual_part_name == entry.new_part_name) return; - if (!queue.future_parts.emplace(entry.actual_new_part_name, entry.shared_from_this()).second) + if (!queue.future_parts.emplace(actual_part_name, entry.shared_from_this()).second) throw Exception(ErrorCodes::LOGICAL_ERROR, "Attaching already existing future part {}. This is a bug. " "It happened on attempt to execute {}: {}", - entry.actual_new_part_name, entry.znode_name, entry.toString()); + actual_part_name, entry.znode_name, entry.toString()); + + if (entry.type == LogEntry::REPLACE_RANGE) + entry.replace_range_actual_new_part_names.insert(actual_part_name); + else + entry.actual_new_part_name = actual_part_name; for (LogEntryPtr & covered_entry : covered_entries_to_wait) { if (&entry == covered_entry.get()) continue; - LOG_TRACE(queue.log, "Waiting for {} producing {} to finish before executing {} producing not disjoint part {}", - covered_entry->znode_name, covered_entry->new_part_name, entry.znode_name, entry.new_part_name); + LOG_TRACE(queue.log, "Waiting for {} producing {} to finish before executing {} producing not disjoint part {} (actual part {})", + covered_entry->znode_name, covered_entry->new_part_name, entry.znode_name, entry.new_part_name, actual_part_name); covered_entry->execution_complete.wait(state_lock, [&covered_entry] { return !covered_entry->currently_executing; }); } } @@ -1646,25 +1665,27 @@ ReplicatedMergeTreeQueue::CurrentlyExecuting::~CurrentlyExecuting() entry->currently_executing = false; entry->execution_complete.notify_all(); - for (const String & new_part_name : entry->getVirtualPartNames(queue.format_version)) + auto erase_and_check = [this](const String & part_name) { - if (!queue.future_parts.erase(new_part_name)) + if (!queue.future_parts.erase(part_name)) { - LOG_ERROR(queue.log, "Untagging already untagged future part {}. This is a bug.", new_part_name); + LOG_ERROR(queue.log, "Untagging already untagged future part {}. This is a bug.", part_name); assert(false); } - } + }; + + for (const String & new_part_name : entry->getVirtualPartNames(queue.format_version)) + erase_and_check(new_part_name); if (!entry->actual_new_part_name.empty()) - { - if (entry->actual_new_part_name != entry->new_part_name && !queue.future_parts.erase(entry->actual_new_part_name)) - { - LOG_ERROR(queue.log, "Untagging already untagged future part {}. This is a bug.", entry->actual_new_part_name); - assert(false); - } + erase_and_check(entry->actual_new_part_name); - entry->actual_new_part_name.clear(); - } + entry->actual_new_part_name.clear(); + + for (const auto & actual_part : entry->replace_range_actual_new_part_names) + erase_and_check(actual_part); + + entry->replace_range_actual_new_part_names.clear(); } diff --git a/tests/queries/0_stateless/02916_move_partition_inactive_replica.reference b/tests/queries/0_stateless/02916_move_partition_inactive_replica.reference new file mode 100644 index 00000000000..b64cc6289d5 --- /dev/null +++ b/tests/queries/0_stateless/02916_move_partition_inactive_replica.reference @@ -0,0 +1,8 @@ +all_0_0_0 0 +all_0_1_1 1 +all_1_1_0 0 +all_2_2_0 0 +all_2_3_1 1 +all_3_3_0 0 +0 +40 1580 diff --git a/tests/queries/0_stateless/02916_move_partition_inactive_replica.sql b/tests/queries/0_stateless/02916_move_partition_inactive_replica.sql new file mode 100644 index 00000000000..d597c037612 --- /dev/null +++ b/tests/queries/0_stateless/02916_move_partition_inactive_replica.sql @@ -0,0 +1,46 @@ +-- Tags: no-parallel + +create database if not exists shard_0; +create database if not exists shard_1; + +drop table if exists shard_0.from_1; +drop table if exists shard_1.from_1; +drop table if exists shard_0.to; +drop table if exists shard_1.to; + +create table shard_0.from_1 (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/from_1_' || currentDatabase(), '0') order by x settings old_parts_lifetime=1, max_cleanup_delay_period=1, cleanup_delay_period=1; +create table shard_1.from_1 (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/from_1_' || currentDatabase(), '1') order by x settings old_parts_lifetime=1, max_cleanup_delay_period=1, cleanup_delay_period=1; + +system stop merges shard_0.from_1; +insert into shard_0.from_1 select number + 20 from numbers(10); +insert into shard_0.from_1 select number + 30 from numbers(10); + +insert into shard_0.from_1 select number + 40 from numbers(10); +insert into shard_0.from_1 select number + 50 from numbers(10); + +system sync replica shard_1.from_1; + +create table shard_0.to (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/to_' || currentDatabase(), '0') order by x settings old_parts_lifetime=1, max_cleanup_delay_period=1, cleanup_delay_period=1, max_parts_to_merge_at_once=2; + +create table shard_1.to (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/to_' || currentDatabase(), '1') order by x settings old_parts_lifetime=1, max_cleanup_delay_period=1, cleanup_delay_period=1, max_parts_to_merge_at_once=2; + +detach table shard_1.to; + +alter table shard_0.from_1 on cluster test_cluster_two_shards_different_databases move partition tuple() to table shard_0.to format Null settings distributed_ddl_output_mode='never_throw', distributed_ddl_task_timeout = 1; + +drop table if exists shard_0.from_1; +drop table if exists shard_1.from_1; +OPTIMIZE TABLE shard_0.to; +OPTIMIZE TABLE shard_0.to; +select name, active from system.parts where database='shard_0' and table='to' order by name; + +system restart replica shard_0.to; + +select sleep(3); + +attach table shard_1.to; +system sync replica shard_1.to; +select count(), sum(x) from shard_1.to; + +drop table if exists shard_0.to; +drop table if exists shard_1.to; From 929173cce2e934a5510a0b9753d92734f39bf8d1 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 21 Mar 2024 19:18:16 +0100 Subject: [PATCH 048/102] Fix S3 buffer allocation --- src/Common/BufferAllocationPolicy.cpp | 2 +- src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp | 4 ++++ src/IO/WriteBufferFromS3.cpp | 12 ++++++++++++ src/IO/WriteBufferFromS3.h | 1 + 4 files changed, 18 insertions(+), 1 deletion(-) diff --git a/src/Common/BufferAllocationPolicy.cpp b/src/Common/BufferAllocationPolicy.cpp index 980cbcca729..9f9665be9b9 100644 --- a/src/Common/BufferAllocationPolicy.cpp +++ b/src/Common/BufferAllocationPolicy.cpp @@ -73,7 +73,7 @@ public: if (1 == buffer_number) { - current_size = std::min(size_t(DBMS_DEFAULT_BUFFER_SIZE), first_size); + current_size = first_size; return; } diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index ed3ce76bd94..1e589cf8bf3 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -141,6 +141,10 @@ void WriteBufferFromAzureBlobStorage::allocateBuffer() { buffer_allocation_policy->nextBuffer(); auto size = buffer_allocation_policy->getBufferSize(); + + if (buffer_allocation_policy->getBufferNumber() == 1) + size = std::min(size_t(DBMS_DEFAULT_BUFFER_SIZE), size); + memory = Memory(size); WriteBuffer::set(memory.data(), memory.size()); } diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 865bac86ff5..4ddff4ce898 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -339,10 +339,22 @@ void WriteBufferFromS3::allocateBuffer() { buffer_allocation_policy->nextBuffer(); chassert(0 == hidden_size); + + if (buffer_allocation_policy->getBufferNumber() == 1) + return allocateFirstBuffer(); + memory = Memory(buffer_allocation_policy->getBufferSize()); WriteBuffer::set(memory.data(), memory.size()); } +void WriteBufferFromS3::allocateFirstBuffer() +{ + const auto max_first_buffer = buffer_allocation_policy->getBufferSize(); + const auto size = std::min(size_t(DBMS_DEFAULT_BUFFER_SIZE), max_first_buffer); + memory = Memory(size); + WriteBuffer::set(memory.data(), memory.size()); +} + void WriteBufferFromS3::setFakeBufferWhenPreFinalized() { WriteBuffer::set(fake_buffer_when_prefinalized, sizeof(fake_buffer_when_prefinalized)); diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 0eab20891bf..4139cbdde6e 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -62,6 +62,7 @@ private: void reallocateFirstBuffer(); void detachBuffer(); void allocateBuffer(); + void allocateFirstBuffer(); void setFakeBufferWhenPreFinalized(); S3::UploadPartRequest getUploadRequest(size_t part_number, PartData & data); From 2658cca513b43a910f570f8adf957f2c63f143a8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 21 Mar 2024 19:42:46 +0100 Subject: [PATCH 049/102] Update 02916_move_partition_inactive_replica.sql --- .../0_stateless/02916_move_partition_inactive_replica.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02916_move_partition_inactive_replica.sql b/tests/queries/0_stateless/02916_move_partition_inactive_replica.sql index d597c037612..25dd0365d97 100644 --- a/tests/queries/0_stateless/02916_move_partition_inactive_replica.sql +++ b/tests/queries/0_stateless/02916_move_partition_inactive_replica.sql @@ -32,7 +32,7 @@ drop table if exists shard_0.from_1; drop table if exists shard_1.from_1; OPTIMIZE TABLE shard_0.to; OPTIMIZE TABLE shard_0.to; -select name, active from system.parts where database='shard_0' and table='to' order by name; +select name, active from system.parts where database='shard_0' and table='to' and active order by name; system restart replica shard_0.to; From 3745df45900dd89348baa46b10ec7c55967d548c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 21 Mar 2024 19:43:12 +0100 Subject: [PATCH 050/102] Update 02916_move_partition_inactive_replica.reference --- .../02916_move_partition_inactive_replica.reference | 4 ---- 1 file changed, 4 deletions(-) diff --git a/tests/queries/0_stateless/02916_move_partition_inactive_replica.reference b/tests/queries/0_stateless/02916_move_partition_inactive_replica.reference index b64cc6289d5..361a0d1e9bb 100644 --- a/tests/queries/0_stateless/02916_move_partition_inactive_replica.reference +++ b/tests/queries/0_stateless/02916_move_partition_inactive_replica.reference @@ -1,8 +1,4 @@ -all_0_0_0 0 all_0_1_1 1 -all_1_1_0 0 -all_2_2_0 0 all_2_3_1 1 -all_3_3_0 0 0 40 1580 From 3f653bbc3258793be95c94ce749142b8b6e0f4db Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 21 Mar 2024 20:28:52 +0100 Subject: [PATCH 051/102] Added test with log check --- .../IO/ReadBufferFromAzureBlobStorage.cpp | 18 ++++--- .../test_storage_azure_blob_storage/test.py | 23 +++++++++ ...xx_test_dictionary_create_access.reference | 0 .../xxx_test_dictionary_create_access.sh | 47 +++++++++++++++++++ 4 files changed, 78 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/xxx_test_dictionary_create_access.reference create mode 100755 tests/queries/0_stateless/xxx_test_dictionary_create_access.sh diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index 3555c71abee..0c4acebf347 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -264,15 +264,6 @@ size_t ReadBufferFromAzureBlobStorage::readBigAt(char * to, size_t n, size_t ran size_t initial_n = n; size_t sleep_time_with_backoff_milliseconds = 100; - auto handle_exception = [&, this](const auto & e, size_t i) - { - LOG_INFO(log, "Exception caught during Azure Download for file {} at offset {} at attempt {}/{}: {}", path, offset, i + 1, max_single_download_retries, e.Message); - if (i + 1 == max_single_download_retries) - throw; - - sleepForMilliseconds(sleep_time_with_backoff_milliseconds); - sleep_time_with_backoff_milliseconds *= 2; - }; for (size_t i = 0; i < max_single_download_retries && n > 0; ++i) { @@ -288,12 +279,19 @@ size_t ReadBufferFromAzureBlobStorage::readBigAt(char * to, size_t n, size_t ran std::istringstream string_stream(String(static_cast(data_ptr),bytes)); // STYLE_CHECK_ALLOW_STD_STRING_STREAM copyFromIStreamWithProgressCallback(string_stream, to, n, progress_callback, &bytes_copied); + LOG_INFO(log, "AzureBlobStorage readBigAt read bytes {}", bytes_copied); + if (read_settings.remote_throttler) read_settings.remote_throttler->add(bytes_copied, ProfileEvents::RemoteReadThrottlerBytes, ProfileEvents::RemoteReadThrottlerSleepMicroseconds); } catch (const Azure::Core::RequestFailedException & e) { - handle_exception(e,i); + LOG_INFO(log, "Exception caught during Azure Download for file {} at offset {} at attempt {}/{}: {}", path, offset, i + 1, max_single_download_retries, e.Message); + if (i + 1 == max_single_download_retries) + throw; + + sleepForMilliseconds(sleep_time_with_backoff_milliseconds); + sleep_time_with_backoff_milliseconds *= 2; } range_begin += bytes_copied; diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index e1d636f3831..6918be9f78a 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -16,6 +16,7 @@ from helpers.cluster import ClickHouseCluster, ClickHouseInstance from helpers.network import PartitionManager from helpers.mock_servers import start_mock_servers from helpers.test_tools import exec_query_with_retry +from helpers.test_tools import assert_logs_contain_with_retry @pytest.fixture(scope="module") @@ -1320,3 +1321,25 @@ def test_format_detection(cluster): ) assert result == expected_result + + +def test_parallel_read(cluster): + node = cluster.instances["node"] + connection_string = cluster.env_variables["AZURITE_CONNECTION_STRING"] + storage_account_url = cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"] + account_name = "devstoreaccount1" + account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==" + + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_parallel_read.parquet', '{account_name}', '{account_key}') " + f"select * from numbers(10000) settings azure_truncate_on_insert=1", + ) + time.sleep(1) + + res = azure_query( + node, + f"select count() from azureBlobStorage('{connection_string}', 'cont', 'test_parallel_read.parquet')", + ) + assert int(res) == 10000 + assert_logs_contain_with_retry(node, "AzureBlobStorage readBigAt read bytes") \ No newline at end of file diff --git a/tests/queries/0_stateless/xxx_test_dictionary_create_access.reference b/tests/queries/0_stateless/xxx_test_dictionary_create_access.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/xxx_test_dictionary_create_access.sh b/tests/queries/0_stateless/xxx_test_dictionary_create_access.sh new file mode 100755 index 00000000000..4478eb59c95 --- /dev/null +++ b/tests/queries/0_stateless/xxx_test_dictionary_create_access.sh @@ -0,0 +1,47 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +readonly PID=$$ +readonly TEST_USER=$"02834_USER_${PID}" + +#${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${TEST_USER}&password=pass" \ +# -d "SELECT * FROM remote('127.0.0.1:${CLICKHOUSE_PORT_TCP}', 'system', 'one', '${TEST_USER}', 'pass')" + +${CLICKHOUSE_CLIENT} -q "CREATE DATABASE db;" +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS db.table;" + +${CLICKHOUSE_CLIENT} -q "DROP DICTIONARY IF EXISTS db.dict_name;" + +${CLICKHOUSE_CLIENT} -q "CREATE TABLE db.table (label_data_combination_id UInt64) engine = MergeTree Order BY label_data_combination_id;" +${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER};" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON db.* TO ${TEST_USER};" +#${CLICKHOUSE_CLIENT} -q "GRANT INSERT ON db.* TO ${TEST_USER};" +${CLICKHOUSE_CLIENT} -q "GRANT CREATE ON db.* TO ${TEST_USER};" +#${CLICKHOUSE_CLIENT} -q "GRANT DROP ON db.* TO ${TEST_USER};" + +${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${TEST_USER}&password=pass" \ + -d "CREATE OR REPLACE DICTIONARY db.dict_name + ( + label_data_combination_id UInt64 + ) + PRIMARY KEY label_data_combination_id + SOURCE(CLICKHOUSE( + QUERY + 'SELECT + label_data_combination_id + FROM db.table' + ) + ) + LAYOUT(HASHED(SHARDS 16 SHARD_LOAD_QUEUE_BACKLOG 10000)) + LIFETIME(0) + SETTINGS(dictionary_use_async_executor = 1, max_threads = 16) + COMMENT 'Dictionary mapping of label_data_combination_id to the underlying data.';" + + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS db.table;" +${CLICKHOUSE_CLIENT} -q "DROP DICTIONARY IF EXISTS db.dict_name;" +${CLICKHOUSE_CLIENT} -q "DROP DATABASE db;" From a2836b0887e03bc8c16098ca084a64b145aa634d Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 21 Mar 2024 22:17:18 +0100 Subject: [PATCH 052/102] Removed unwanted tests --- ...xx_test_dictionary_create_access.reference | 0 .../xxx_test_dictionary_create_access.sh | 47 ------------------- 2 files changed, 47 deletions(-) delete mode 100644 tests/queries/0_stateless/xxx_test_dictionary_create_access.reference delete mode 100755 tests/queries/0_stateless/xxx_test_dictionary_create_access.sh diff --git a/tests/queries/0_stateless/xxx_test_dictionary_create_access.reference b/tests/queries/0_stateless/xxx_test_dictionary_create_access.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/xxx_test_dictionary_create_access.sh b/tests/queries/0_stateless/xxx_test_dictionary_create_access.sh deleted file mode 100755 index 4478eb59c95..00000000000 --- a/tests/queries/0_stateless/xxx_test_dictionary_create_access.sh +++ /dev/null @@ -1,47 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-fasttest - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -readonly PID=$$ -readonly TEST_USER=$"02834_USER_${PID}" - -#${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${TEST_USER}&password=pass" \ -# -d "SELECT * FROM remote('127.0.0.1:${CLICKHOUSE_PORT_TCP}', 'system', 'one', '${TEST_USER}', 'pass')" - -${CLICKHOUSE_CLIENT} -q "CREATE DATABASE db;" -${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS db.table;" - -${CLICKHOUSE_CLIENT} -q "DROP DICTIONARY IF EXISTS db.dict_name;" - -${CLICKHOUSE_CLIENT} -q "CREATE TABLE db.table (label_data_combination_id UInt64) engine = MergeTree Order BY label_data_combination_id;" -${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER};" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON db.* TO ${TEST_USER};" -#${CLICKHOUSE_CLIENT} -q "GRANT INSERT ON db.* TO ${TEST_USER};" -${CLICKHOUSE_CLIENT} -q "GRANT CREATE ON db.* TO ${TEST_USER};" -#${CLICKHOUSE_CLIENT} -q "GRANT DROP ON db.* TO ${TEST_USER};" - -${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${TEST_USER}&password=pass" \ - -d "CREATE OR REPLACE DICTIONARY db.dict_name - ( - label_data_combination_id UInt64 - ) - PRIMARY KEY label_data_combination_id - SOURCE(CLICKHOUSE( - QUERY - 'SELECT - label_data_combination_id - FROM db.table' - ) - ) - LAYOUT(HASHED(SHARDS 16 SHARD_LOAD_QUEUE_BACKLOG 10000)) - LIFETIME(0) - SETTINGS(dictionary_use_async_executor = 1, max_threads = 16) - COMMENT 'Dictionary mapping of label_data_combination_id to the underlying data.';" - - -${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS db.table;" -${CLICKHOUSE_CLIENT} -q "DROP DICTIONARY IF EXISTS db.dict_name;" -${CLICKHOUSE_CLIENT} -q "DROP DATABASE db;" From 50bc8acc3724b8fa81d2aa1e7b40515a16268d08 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 21 Mar 2024 21:26:53 +0000 Subject: [PATCH 053/102] Automatic style fix --- tests/integration/test_storage_azure_blob_storage/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 6918be9f78a..7d30265e4f8 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -1342,4 +1342,4 @@ def test_parallel_read(cluster): f"select count() from azureBlobStorage('{connection_string}', 'cont', 'test_parallel_read.parquet')", ) assert int(res) == 10000 - assert_logs_contain_with_retry(node, "AzureBlobStorage readBigAt read bytes") \ No newline at end of file + assert_logs_contain_with_retry(node, "AzureBlobStorage readBigAt read bytes") From 257c263596f5832faafff8abfe344c53bf549ea7 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 21 Mar 2024 22:33:34 +0100 Subject: [PATCH 054/102] Avoid extra copy of data --- src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index 0c4acebf347..2b8986a265e 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -259,7 +259,7 @@ size_t ReadBufferFromAzureBlobStorage::getFileSize() return *file_size; } -size_t ReadBufferFromAzureBlobStorage::readBigAt(char * to, size_t n, size_t range_begin, const std::function & progress_callback) const +size_t ReadBufferFromAzureBlobStorage::readBigAt(char * to, size_t n, size_t range_begin, const std::function & /*progress_callback*/) const { size_t initial_n = n; @@ -275,9 +275,7 @@ size_t ReadBufferFromAzureBlobStorage::readBigAt(char * to, size_t n, size_t ran auto download_response = blob_client->Download(download_options); std::unique_ptr body_stream = std::move(download_response.Value.BodyStream); - auto bytes = body_stream->ReadToCount(reinterpret_cast(data_ptr), body_stream->Length()); - std::istringstream string_stream(String(static_cast(data_ptr),bytes)); // STYLE_CHECK_ALLOW_STD_STRING_STREAM - copyFromIStreamWithProgressCallback(string_stream, to, n, progress_callback, &bytes_copied); + bytes_copied = body_stream->ReadToCount(reinterpret_cast(to), body_stream->Length()); LOG_INFO(log, "AzureBlobStorage readBigAt read bytes {}", bytes_copied); From 6a550bc7547c1fd78419550532f5a2fc65178d68 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Fri, 22 Mar 2024 11:23:59 +0800 Subject: [PATCH 055/102] add test case and support constant expression --- src/Analyzer/Passes/ConvertInToEqualPass.cpp | 2 +- .../03013_optimize_in_to_equal.reference | 138 +++++++++++++++++- .../03013_optimize_in_to_equal.sql | 14 +- 3 files changed, 149 insertions(+), 5 deletions(-) diff --git a/src/Analyzer/Passes/ConvertInToEqualPass.cpp b/src/Analyzer/Passes/ConvertInToEqualPass.cpp index ddd8a8f266f..66a37fea5bd 100644 --- a/src/Analyzer/Passes/ConvertInToEqualPass.cpp +++ b/src/Analyzer/Passes/ConvertInToEqualPass.cpp @@ -36,7 +36,7 @@ public: || constant_node->getValue().getType() == Field::Types::Which::Array) return ; // x IN null not equivalent to x = null - if (constant_node->hasSourceExpression() || constant_node->getValue().isNull()) + if (constant_node->getValue().isNull()) return ; auto result_func_name = MAPPING.at(func_node->getFunctionName()); auto equal = std::make_shared(result_func_name); diff --git a/tests/queries/0_stateless/03013_optimize_in_to_equal.reference b/tests/queries/0_stateless/03013_optimize_in_to_equal.reference index dcaa14abfb1..c24f40d545d 100644 --- a/tests/queries/0_stateless/03013_optimize_in_to_equal.reference +++ b/tests/queries/0_stateless/03013_optimize_in_to_equal.reference @@ -22,6 +22,29 @@ QUERY id: 0 CONSTANT id: 8, constant_value: \'a\', constant_value_type: String SETTINGS allow_experimental_analyzer=1 ------------------- +QUERY id: 0 + PROJECTION COLUMNS + x String + y Int32 + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: x, result_type: String, source_id: 3 + COLUMN id: 4, column_name: y, result_type: Int32, source_id: 3 + JOIN TREE + TABLE id: 3, alias: __table1, table_name: default.test + WHERE + FUNCTION id: 5, function_name: equals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 6, nodes: 2 + COLUMN id: 7, column_name: x, result_type: String, source_id: 3 + CONSTANT id: 8, constant_value: \'A\', constant_value_type: String + EXPRESSION + FUNCTION id: 9, function_name: upper, function_type: ordinary, result_type: String + ARGUMENTS + LIST id: 10, nodes: 1 + CONSTANT id: 11, constant_value: \'a\', constant_value_type: String + SETTINGS allow_experimental_analyzer=1 +------------------- QUERY id: 0 PROJECTION COLUMNS x String @@ -40,6 +63,24 @@ QUERY id: 0 CONSTANT id: 8, constant_value: Tuple_(\'a\', \'b\'), constant_value_type: Tuple(String, String) SETTINGS allow_experimental_analyzer=1 ------------------- +QUERY id: 0 + PROJECTION COLUMNS + x String + y Int32 + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: x, result_type: String, source_id: 3 + COLUMN id: 4, column_name: y, result_type: Int32, source_id: 3 + JOIN TREE + TABLE id: 3, alias: __table1, table_name: default.test + WHERE + FUNCTION id: 5, function_name: in, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 6, nodes: 2 + COLUMN id: 7, column_name: x, result_type: String, source_id: 3 + CONSTANT id: 8, constant_value: Array_[\'a\', \'b\'], constant_value_type: Array(String) + SETTINGS allow_experimental_analyzer=1 +------------------- b 2 c 3 ------------------- @@ -61,6 +102,97 @@ QUERY id: 0 CONSTANT id: 8, constant_value: \'a\', constant_value_type: String SETTINGS allow_experimental_analyzer=1 ------------------- -a 1 -b 2 -c 3 +QUERY id: 0 + PROJECTION COLUMNS + x String + y Int32 + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: x, result_type: String, source_id: 3 + COLUMN id: 4, column_name: y, result_type: Int32, source_id: 3 + JOIN TREE + TABLE id: 3, alias: __table1, table_name: default.test + WHERE + FUNCTION id: 5, function_name: notEquals, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 6, nodes: 2 + COLUMN id: 7, column_name: x, result_type: String, source_id: 3 + CONSTANT id: 8, constant_value: \'A\', constant_value_type: String + EXPRESSION + FUNCTION id: 9, function_name: upper, function_type: ordinary, result_type: String + ARGUMENTS + LIST id: 10, nodes: 1 + CONSTANT id: 11, constant_value: \'a\', constant_value_type: String + SETTINGS allow_experimental_analyzer=1 +------------------- +QUERY id: 0 + PROJECTION COLUMNS + x String + y Int32 + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: x, result_type: String, source_id: 3 + COLUMN id: 4, column_name: y, result_type: Int32, source_id: 3 + JOIN TREE + TABLE id: 3, alias: __table1, table_name: default.test + WHERE + FUNCTION id: 5, function_name: notIn, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 6, nodes: 2 + COLUMN id: 7, column_name: x, result_type: String, source_id: 3 + CONSTANT id: 8, constant_value: Tuple_(\'a\', \'b\'), constant_value_type: Tuple(String, String) + SETTINGS allow_experimental_analyzer=1 +------------------- +QUERY id: 0 + PROJECTION COLUMNS + x String + y Int32 + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: x, result_type: String, source_id: 3 + COLUMN id: 4, column_name: y, result_type: Int32, source_id: 3 + JOIN TREE + TABLE id: 3, alias: __table1, table_name: default.test + WHERE + FUNCTION id: 5, function_name: notIn, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 6, nodes: 2 + COLUMN id: 7, column_name: x, result_type: String, source_id: 3 + CONSTANT id: 8, constant_value: Array_[\'a\', \'b\'], constant_value_type: Array(String) + SETTINGS allow_experimental_analyzer=1 +------------------- +QUERY id: 0 + PROJECTION COLUMNS + x String + y Int32 + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: x, result_type: String, source_id: 3 + COLUMN id: 4, column_name: y, result_type: Int32, source_id: 3 + JOIN TREE + TABLE id: 3, alias: __table1, table_name: default.test + WHERE + FUNCTION id: 5, function_name: notIn, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 6, nodes: 2 + COLUMN id: 7, column_name: x, result_type: String, source_id: 3 + CONSTANT id: 8, constant_value: NULL, constant_value_type: Nullable(Nothing) + SETTINGS allow_experimental_analyzer=1 +------------------- +QUERY id: 0 + PROJECTION COLUMNS + x String + y Int32 + PROJECTION + LIST id: 1, nodes: 2 + COLUMN id: 2, column_name: x, result_type: String, source_id: 3 + COLUMN id: 4, column_name: y, result_type: Int32, source_id: 3 + JOIN TREE + TABLE id: 3, alias: __table1, table_name: default.test + WHERE + FUNCTION id: 5, function_name: in, function_type: ordinary, result_type: UInt8 + ARGUMENTS + LIST id: 6, nodes: 2 + COLUMN id: 7, column_name: x, result_type: String, source_id: 3 + CONSTANT id: 8, constant_value: NULL, constant_value_type: Nullable(Nothing) + SETTINGS allow_experimental_analyzer=1 diff --git a/tests/queries/0_stateless/03013_optimize_in_to_equal.sql b/tests/queries/0_stateless/03013_optimize_in_to_equal.sql index be4dc2275a9..c38a741114f 100644 --- a/tests/queries/0_stateless/03013_optimize_in_to_equal.sql +++ b/tests/queries/0_stateless/03013_optimize_in_to_equal.sql @@ -7,10 +7,22 @@ select x in Null from test; select '-------------------'; explain query tree select * from test where x in ('a') SETTINGS allow_experimental_analyzer = 1; select '-------------------'; +explain query tree select * from test where x in (upper('a')) SETTINGS allow_experimental_analyzer = 1; +select '-------------------'; explain query tree select * from test where x in ('a','b') SETTINGS allow_experimental_analyzer = 1; select '-------------------'; +explain query tree select * from test where x in ['a','b'] SETTINGS allow_experimental_analyzer = 1; +select '-------------------'; select * from test where x not in ('a') SETTINGS allow_experimental_analyzer = 1; select '-------------------'; explain query tree select * from test where x not in ('a') SETTINGS allow_experimental_analyzer = 1; select '-------------------'; -select * from test where x not in (NULL) SETTINGS allow_experimental_analyzer = 1; +explain query tree select * from test where x not in (upper('a')) SETTINGS allow_experimental_analyzer = 1; +select '-------------------'; +explain query tree select * from test where x not in ('a','b') SETTINGS allow_experimental_analyzer = 1; +select '-------------------'; +explain query tree select * from test where x not in ['a','b'] SETTINGS allow_experimental_analyzer = 1; +select '-------------------'; +explain query tree select * from test where x not in (NULL) SETTINGS allow_experimental_analyzer = 1; +select '-------------------'; +explain query tree select * from test where x in (NULL) SETTINGS allow_experimental_analyzer = 1; From d969c0cd08f3c158a57b6b4337f9280e5773728d Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 22 Mar 2024 10:30:55 +0100 Subject: [PATCH 056/102] Removed unused includes --- src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index 2b8986a265e..4dc80dd5418 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -10,8 +10,6 @@ #include #include -#include - namespace ProfileEvents { extern const Event RemoteReadThrottlerBytes; From a08c16e8eb9325bf80655a7d70406e68db68d6f3 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 22 Mar 2024 10:45:48 +0100 Subject: [PATCH 057/102] Fixed clang tidy build --- src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 1e589cf8bf3..05b93dd1fa3 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -47,7 +47,7 @@ WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage( ThreadPoolCallbackRunner schedule_) : WriteBufferFromFileBase(buf_size_, nullptr, 0) , log(getLogger("WriteBufferFromAzureBlobStorage")) - , buffer_allocation_policy(createBufferAllocationPolicy(*settings_.get())) + , buffer_allocation_policy(createBufferAllocationPolicy(*settings_)) , max_single_part_upload_size(settings_->max_single_part_upload_size) , max_unexpected_write_error_retries(settings_->max_unexpected_write_error_retries) , blob_path(blob_path_) From 7a154ad373e1b11e73d0311421ff45fed3ed045e Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 22 Mar 2024 11:51:35 +0100 Subject: [PATCH 058/102] Updated log levels --- src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index 4dc80dd5418..5947b742339 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -103,7 +103,7 @@ bool ReadBufferFromAzureBlobStorage::nextImpl() auto handle_exception = [&, this](const auto & e, size_t i) { - LOG_INFO(log, "Exception caught during Azure Read for file {} at attempt {}/{}: {}", path, i + 1, max_single_read_retries, e.Message); + LOG_DEBUG(log, "Exception caught during Azure Read for file {} at attempt {}/{}: {}", path, i + 1, max_single_read_retries, e.Message); if (i + 1 == max_single_read_retries) throw; @@ -215,7 +215,7 @@ void ReadBufferFromAzureBlobStorage::initialize() auto handle_exception = [&, this](const auto & e, size_t i) { - LOG_INFO(log, "Exception caught during Azure Download for file {} at offset {} at attempt {}/{}: {}", path, offset, i + 1, max_single_download_retries, e.Message); + LOG_DEBUG(log, "Exception caught during Azure Download for file {} at offset {} at attempt {}/{}: {}", path, offset, i + 1, max_single_download_retries, e.Message); if (i + 1 == max_single_download_retries) throw; @@ -275,14 +275,14 @@ size_t ReadBufferFromAzureBlobStorage::readBigAt(char * to, size_t n, size_t ran std::unique_ptr body_stream = std::move(download_response.Value.BodyStream); bytes_copied = body_stream->ReadToCount(reinterpret_cast(to), body_stream->Length()); - LOG_INFO(log, "AzureBlobStorage readBigAt read bytes {}", bytes_copied); + LOG_TEST(log, "AzureBlobStorage readBigAt read bytes {}", bytes_copied); if (read_settings.remote_throttler) read_settings.remote_throttler->add(bytes_copied, ProfileEvents::RemoteReadThrottlerBytes, ProfileEvents::RemoteReadThrottlerSleepMicroseconds); } catch (const Azure::Core::RequestFailedException & e) { - LOG_INFO(log, "Exception caught during Azure Download for file {} at offset {} at attempt {}/{}: {}", path, offset, i + 1, max_single_download_retries, e.Message); + LOG_DEBUG(log, "Exception caught during Azure Download for file {} at offset {} at attempt {}/{}: {}", path, offset, i + 1, max_single_download_retries, e.Message); if (i + 1 == max_single_download_retries) throw; From ad218411b373e8df599e739c81d2e28982f25678 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 22 Mar 2024 12:27:37 +0100 Subject: [PATCH 059/102] remove duplicated SETTINGS statement --- .../03013_optimize_in_to_equal.sql | 25 ++++++++++--------- 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/03013_optimize_in_to_equal.sql b/tests/queries/0_stateless/03013_optimize_in_to_equal.sql index c38a741114f..ba6eb5d4f5f 100644 --- a/tests/queries/0_stateless/03013_optimize_in_to_equal.sql +++ b/tests/queries/0_stateless/03013_optimize_in_to_equal.sql @@ -1,28 +1,29 @@ DROP TABLE IF EXISTS test; CREATE TABLE test (x String, y Int32) ENGINE = MergeTree() ORDER BY x; +SET allow_experimental_analyzer = 1; INSERT INTO test VALUES ('a', 1), ('b', 2), ('c', 3); -select * from test where x in ('a') SETTINGS allow_experimental_analyzer = 1; +select * from test where x in ('a'); select '-------------------'; select x in Null from test; select '-------------------'; -explain query tree select * from test where x in ('a') SETTINGS allow_experimental_analyzer = 1; +explain query tree select * from test where x in ('a'); select '-------------------'; -explain query tree select * from test where x in (upper('a')) SETTINGS allow_experimental_analyzer = 1; +explain query tree select * from test where x in (upper('a')); select '-------------------'; -explain query tree select * from test where x in ('a','b') SETTINGS allow_experimental_analyzer = 1; +explain query tree select * from test where x in ('a','b'); select '-------------------'; -explain query tree select * from test where x in ['a','b'] SETTINGS allow_experimental_analyzer = 1; +explain query tree select * from test where x in ['a','b']; select '-------------------'; -select * from test where x not in ('a') SETTINGS allow_experimental_analyzer = 1; +select * from test where x not in ('a'); select '-------------------'; -explain query tree select * from test where x not in ('a') SETTINGS allow_experimental_analyzer = 1; +explain query tree select * from test where x not in ('a'); select '-------------------'; -explain query tree select * from test where x not in (upper('a')) SETTINGS allow_experimental_analyzer = 1; +explain query tree select * from test where x not in (upper('a')); select '-------------------'; -explain query tree select * from test where x not in ('a','b') SETTINGS allow_experimental_analyzer = 1; +explain query tree select * from test where x not in ('a','b'); select '-------------------'; -explain query tree select * from test where x not in ['a','b'] SETTINGS allow_experimental_analyzer = 1; +explain query tree select * from test where x not in ['a','b']; select '-------------------'; -explain query tree select * from test where x not in (NULL) SETTINGS allow_experimental_analyzer = 1; +explain query tree select * from test where x not in (NULL); select '-------------------'; -explain query tree select * from test where x in (NULL) SETTINGS allow_experimental_analyzer = 1; +explain query tree select * from test where x in (NULL); From 5d5f073bab07801e0955d085d88f2b41e18c301e Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 22 Mar 2024 12:48:14 +0100 Subject: [PATCH 060/102] revert previous commit --- .../03013_optimize_in_to_equal.sql | 25 +++++++++---------- 1 file changed, 12 insertions(+), 13 deletions(-) diff --git a/tests/queries/0_stateless/03013_optimize_in_to_equal.sql b/tests/queries/0_stateless/03013_optimize_in_to_equal.sql index ba6eb5d4f5f..c38a741114f 100644 --- a/tests/queries/0_stateless/03013_optimize_in_to_equal.sql +++ b/tests/queries/0_stateless/03013_optimize_in_to_equal.sql @@ -1,29 +1,28 @@ DROP TABLE IF EXISTS test; CREATE TABLE test (x String, y Int32) ENGINE = MergeTree() ORDER BY x; -SET allow_experimental_analyzer = 1; INSERT INTO test VALUES ('a', 1), ('b', 2), ('c', 3); -select * from test where x in ('a'); +select * from test where x in ('a') SETTINGS allow_experimental_analyzer = 1; select '-------------------'; select x in Null from test; select '-------------------'; -explain query tree select * from test where x in ('a'); +explain query tree select * from test where x in ('a') SETTINGS allow_experimental_analyzer = 1; select '-------------------'; -explain query tree select * from test where x in (upper('a')); +explain query tree select * from test where x in (upper('a')) SETTINGS allow_experimental_analyzer = 1; select '-------------------'; -explain query tree select * from test where x in ('a','b'); +explain query tree select * from test where x in ('a','b') SETTINGS allow_experimental_analyzer = 1; select '-------------------'; -explain query tree select * from test where x in ['a','b']; +explain query tree select * from test where x in ['a','b'] SETTINGS allow_experimental_analyzer = 1; select '-------------------'; -select * from test where x not in ('a'); +select * from test where x not in ('a') SETTINGS allow_experimental_analyzer = 1; select '-------------------'; -explain query tree select * from test where x not in ('a'); +explain query tree select * from test where x not in ('a') SETTINGS allow_experimental_analyzer = 1; select '-------------------'; -explain query tree select * from test where x not in (upper('a')); +explain query tree select * from test where x not in (upper('a')) SETTINGS allow_experimental_analyzer = 1; select '-------------------'; -explain query tree select * from test where x not in ('a','b'); +explain query tree select * from test where x not in ('a','b') SETTINGS allow_experimental_analyzer = 1; select '-------------------'; -explain query tree select * from test where x not in ['a','b']; +explain query tree select * from test where x not in ['a','b'] SETTINGS allow_experimental_analyzer = 1; select '-------------------'; -explain query tree select * from test where x not in (NULL); +explain query tree select * from test where x not in (NULL) SETTINGS allow_experimental_analyzer = 1; select '-------------------'; -explain query tree select * from test where x in (NULL); +explain query tree select * from test where x in (NULL) SETTINGS allow_experimental_analyzer = 1; From abbbd21b7eb66a4fa6d41e7038cdb1dc7717a640 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 22 Mar 2024 14:28:49 +0000 Subject: [PATCH 061/102] CI: disable grpc tests on ARM --- tests/integration/test_grpc_protocol/test.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index 0332e5bb90f..145cf03c58e 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -5,7 +5,7 @@ import time import pytz import uuid import grpc -from helpers.cluster import ClickHouseCluster, run_and_check +from helpers.cluster import ClickHouseCluster, is_arm, run_and_check from threading import Thread import gzip import lz4.frame @@ -20,6 +20,10 @@ import clickhouse_grpc_pb2, clickhouse_grpc_pb2_grpc # Execute pb2/generate.py GRPC_PORT = 9100 DEFAULT_ENCODING = "utf-8" +# GRPC is disabled on ARM build - skip tests +if is_arm(): + pytestmark = pytest.mark.skip + # Utilities From 97bd6ecfc19c0ab9103d7c9153c66f8b9c000f32 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 22 Mar 2024 16:00:53 +0100 Subject: [PATCH 062/102] remove duplicated SETTINGS statement --- .../03013_optimize_in_to_equal.reference | 10 -------- .../03013_optimize_in_to_equal.sql | 25 ++++++++++--------- 2 files changed, 13 insertions(+), 22 deletions(-) diff --git a/tests/queries/0_stateless/03013_optimize_in_to_equal.reference b/tests/queries/0_stateless/03013_optimize_in_to_equal.reference index c24f40d545d..93ac91bd957 100644 --- a/tests/queries/0_stateless/03013_optimize_in_to_equal.reference +++ b/tests/queries/0_stateless/03013_optimize_in_to_equal.reference @@ -20,7 +20,6 @@ QUERY id: 0 LIST id: 6, nodes: 2 COLUMN id: 7, column_name: x, result_type: String, source_id: 3 CONSTANT id: 8, constant_value: \'a\', constant_value_type: String - SETTINGS allow_experimental_analyzer=1 ------------------- QUERY id: 0 PROJECTION COLUMNS @@ -43,7 +42,6 @@ QUERY id: 0 ARGUMENTS LIST id: 10, nodes: 1 CONSTANT id: 11, constant_value: \'a\', constant_value_type: String - SETTINGS allow_experimental_analyzer=1 ------------------- QUERY id: 0 PROJECTION COLUMNS @@ -61,7 +59,6 @@ QUERY id: 0 LIST id: 6, nodes: 2 COLUMN id: 7, column_name: x, result_type: String, source_id: 3 CONSTANT id: 8, constant_value: Tuple_(\'a\', \'b\'), constant_value_type: Tuple(String, String) - SETTINGS allow_experimental_analyzer=1 ------------------- QUERY id: 0 PROJECTION COLUMNS @@ -79,7 +76,6 @@ QUERY id: 0 LIST id: 6, nodes: 2 COLUMN id: 7, column_name: x, result_type: String, source_id: 3 CONSTANT id: 8, constant_value: Array_[\'a\', \'b\'], constant_value_type: Array(String) - SETTINGS allow_experimental_analyzer=1 ------------------- b 2 c 3 @@ -100,7 +96,6 @@ QUERY id: 0 LIST id: 6, nodes: 2 COLUMN id: 7, column_name: x, result_type: String, source_id: 3 CONSTANT id: 8, constant_value: \'a\', constant_value_type: String - SETTINGS allow_experimental_analyzer=1 ------------------- QUERY id: 0 PROJECTION COLUMNS @@ -123,7 +118,6 @@ QUERY id: 0 ARGUMENTS LIST id: 10, nodes: 1 CONSTANT id: 11, constant_value: \'a\', constant_value_type: String - SETTINGS allow_experimental_analyzer=1 ------------------- QUERY id: 0 PROJECTION COLUMNS @@ -141,7 +135,6 @@ QUERY id: 0 LIST id: 6, nodes: 2 COLUMN id: 7, column_name: x, result_type: String, source_id: 3 CONSTANT id: 8, constant_value: Tuple_(\'a\', \'b\'), constant_value_type: Tuple(String, String) - SETTINGS allow_experimental_analyzer=1 ------------------- QUERY id: 0 PROJECTION COLUMNS @@ -159,7 +152,6 @@ QUERY id: 0 LIST id: 6, nodes: 2 COLUMN id: 7, column_name: x, result_type: String, source_id: 3 CONSTANT id: 8, constant_value: Array_[\'a\', \'b\'], constant_value_type: Array(String) - SETTINGS allow_experimental_analyzer=1 ------------------- QUERY id: 0 PROJECTION COLUMNS @@ -177,7 +169,6 @@ QUERY id: 0 LIST id: 6, nodes: 2 COLUMN id: 7, column_name: x, result_type: String, source_id: 3 CONSTANT id: 8, constant_value: NULL, constant_value_type: Nullable(Nothing) - SETTINGS allow_experimental_analyzer=1 ------------------- QUERY id: 0 PROJECTION COLUMNS @@ -195,4 +186,3 @@ QUERY id: 0 LIST id: 6, nodes: 2 COLUMN id: 7, column_name: x, result_type: String, source_id: 3 CONSTANT id: 8, constant_value: NULL, constant_value_type: Nullable(Nothing) - SETTINGS allow_experimental_analyzer=1 diff --git a/tests/queries/0_stateless/03013_optimize_in_to_equal.sql b/tests/queries/0_stateless/03013_optimize_in_to_equal.sql index c38a741114f..ba6eb5d4f5f 100644 --- a/tests/queries/0_stateless/03013_optimize_in_to_equal.sql +++ b/tests/queries/0_stateless/03013_optimize_in_to_equal.sql @@ -1,28 +1,29 @@ DROP TABLE IF EXISTS test; CREATE TABLE test (x String, y Int32) ENGINE = MergeTree() ORDER BY x; +SET allow_experimental_analyzer = 1; INSERT INTO test VALUES ('a', 1), ('b', 2), ('c', 3); -select * from test where x in ('a') SETTINGS allow_experimental_analyzer = 1; +select * from test where x in ('a'); select '-------------------'; select x in Null from test; select '-------------------'; -explain query tree select * from test where x in ('a') SETTINGS allow_experimental_analyzer = 1; +explain query tree select * from test where x in ('a'); select '-------------------'; -explain query tree select * from test where x in (upper('a')) SETTINGS allow_experimental_analyzer = 1; +explain query tree select * from test where x in (upper('a')); select '-------------------'; -explain query tree select * from test where x in ('a','b') SETTINGS allow_experimental_analyzer = 1; +explain query tree select * from test where x in ('a','b'); select '-------------------'; -explain query tree select * from test where x in ['a','b'] SETTINGS allow_experimental_analyzer = 1; +explain query tree select * from test where x in ['a','b']; select '-------------------'; -select * from test where x not in ('a') SETTINGS allow_experimental_analyzer = 1; +select * from test where x not in ('a'); select '-------------------'; -explain query tree select * from test where x not in ('a') SETTINGS allow_experimental_analyzer = 1; +explain query tree select * from test where x not in ('a'); select '-------------------'; -explain query tree select * from test where x not in (upper('a')) SETTINGS allow_experimental_analyzer = 1; +explain query tree select * from test where x not in (upper('a')); select '-------------------'; -explain query tree select * from test where x not in ('a','b') SETTINGS allow_experimental_analyzer = 1; +explain query tree select * from test where x not in ('a','b'); select '-------------------'; -explain query tree select * from test where x not in ['a','b'] SETTINGS allow_experimental_analyzer = 1; +explain query tree select * from test where x not in ['a','b']; select '-------------------'; -explain query tree select * from test where x not in (NULL) SETTINGS allow_experimental_analyzer = 1; +explain query tree select * from test where x not in (NULL); select '-------------------'; -explain query tree select * from test where x in (NULL) SETTINGS allow_experimental_analyzer = 1; +explain query tree select * from test where x in (NULL); From e1f9e496408d57acc5dcb2808f77bf38f1e8a524 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 22 Mar 2024 17:21:48 +0100 Subject: [PATCH 063/102] fix test for smt --- .../0_stateless/02916_move_partition_inactive_replica.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02916_move_partition_inactive_replica.sql b/tests/queries/0_stateless/02916_move_partition_inactive_replica.sql index 25dd0365d97..ca153eea221 100644 --- a/tests/queries/0_stateless/02916_move_partition_inactive_replica.sql +++ b/tests/queries/0_stateless/02916_move_partition_inactive_replica.sql @@ -12,6 +12,7 @@ create table shard_0.from_1 (x UInt32) engine = ReplicatedMergeTree('/clickhouse create table shard_1.from_1 (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/from_1_' || currentDatabase(), '1') order by x settings old_parts_lifetime=1, max_cleanup_delay_period=1, cleanup_delay_period=1; system stop merges shard_0.from_1; +system stop merges shard_1.from_1; insert into shard_0.from_1 select number + 20 from numbers(10); insert into shard_0.from_1 select number + 30 from numbers(10); @@ -20,7 +21,7 @@ insert into shard_0.from_1 select number + 50 from numbers(10); system sync replica shard_1.from_1; -create table shard_0.to (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/to_' || currentDatabase(), '0') order by x settings old_parts_lifetime=1, max_cleanup_delay_period=1, cleanup_delay_period=1, max_parts_to_merge_at_once=2; +create table shard_0.to (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/to_' || currentDatabase(), '0') order by x settings old_parts_lifetime=1, max_cleanup_delay_period=1, cleanup_delay_period=1, max_parts_to_merge_at_once=2, shared_merge_tree_disable_merges_and_mutations_assignment=1; create table shard_1.to (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/to_' || currentDatabase(), '1') order by x settings old_parts_lifetime=1, max_cleanup_delay_period=1, cleanup_delay_period=1, max_parts_to_merge_at_once=2; From 91e52ebee593c42a6d27055a70809e79e3128c0a Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 22 Mar 2024 16:40:17 +0000 Subject: [PATCH 064/102] disable mysql57 tests --- tests/integration/test_mysql57_database_engine/test.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_mysql57_database_engine/test.py b/tests/integration/test_mysql57_database_engine/test.py index 26db6637bc6..d10dd50f0a5 100644 --- a/tests/integration/test_mysql57_database_engine/test.py +++ b/tests/integration/test_mysql57_database_engine/test.py @@ -5,9 +5,13 @@ from string import Template import pymysql.cursors import pytest from helpers.client import QueryRuntimeException -from helpers.cluster import ClickHouseCluster +from helpers.cluster import ClickHouseCluster, is_arm from helpers.network import PartitionManager + +if is_arm(): + pytestmark = pytest.mark.skip + cluster = ClickHouseCluster(__file__) clickhouse_node = cluster.add_instance( "node1", From ada32a433a8609debd8eb795f1c99103aebce879 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 22 Mar 2024 19:51:06 +0100 Subject: [PATCH 065/102] Enable `output_format_pretty_row_numbers` by default --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 958fe64902d..9414915484e 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1138,7 +1138,7 @@ class IColumn; \ M(Bool, output_format_enable_streaming, false, "Enable streaming in output formats that support it.", 0) \ M(Bool, output_format_write_statistics, true, "Write statistics about read rows, bytes, time elapsed in suitable output formats.", 0) \ - M(Bool, output_format_pretty_row_numbers, false, "Add row numbers before each row for pretty output format", 0) \ + M(Bool, output_format_pretty_row_numbers, true, "Add row numbers before each row for pretty output format", 0) \ M(UInt64, output_format_pretty_single_large_number_tip_threshold, 1'000'000, "Print a readable number tip on the right side of the table if the block consists of a single number which exceeds this value (except 0)", 0) \ M(Bool, insert_distributed_one_random_shard, false, "If setting is enabled, inserting into distributed table will choose a random shard to write when there is no sharding key", 0) \ \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 8798712ceaa..262d44886c6 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -104,6 +104,7 @@ static std::map sett {"keeper_retry_max_backoff_ms", 5000, 5000, "Max backoff timeout for general keeper operations"}, {"s3queue_allow_experimental_sharded_mode", false, false, "Enable experimental sharded mode of S3Queue table engine. It is experimental because it will be rewritten"}, {"merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability", 0.0, 0.0, "For testing of `PartsSplitter` - split read ranges into intersecting and non intersecting every time you read from MergeTree with the specified probability."}, + {"output_format_pretty_row_numbers", false, true, "It is better for usability."}, }}, {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, From 465b0c16cdb6f17c6e26f612c10432b69930aca0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 23 Mar 2024 03:10:14 +0100 Subject: [PATCH 066/102] Better row numbers --- src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp | 9 ++++++--- .../Formats/Impl/PrettyCompactBlockOutputFormat.cpp | 8 +++++--- .../Formats/Impl/PrettySpaceBlockOutputFormat.cpp | 7 ++++++- tests/queries/0_stateless/01670_neighbor_lc_bug.sql | 2 ++ .../0_stateless/02947_merge_tree_index_table_2.sql | 1 + 5 files changed, 20 insertions(+), 7 deletions(-) diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index c5db8f2e30a..d2bd71188b8 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -287,11 +287,14 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind { // Write row number; auto row_num_string = std::to_string(i + 1 + total_rows) + ". "; + for (size_t j = 0; j < row_number_width - row_num_string.size(); ++j) - { - writeCString(" ", out); - } + writeChar(' ', out); + if (color) + writeCString("\033[90m", out); writeString(row_num_string, out); + if (color) + writeCString("\033[0m", out); } writeCString(grid_symbols.bar, out); diff --git a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp index bda51770838..3f0349d7710 100644 --- a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp @@ -147,10 +147,12 @@ void PrettyCompactBlockOutputFormat::writeRow( // Write row number; auto row_num_string = std::to_string(row_num + 1 + total_rows) + ". "; for (size_t i = 0; i < row_number_width - row_num_string.size(); ++i) - { - writeCString(" ", out); - } + writeChar(' ', out); + if (color) + writeCString("\033[90m", out); writeString(row_num_string, out); + if (color) + writeCString("\033[0m", out); } const GridSymbols & grid_symbols = format_settings.pretty.charset == FormatSettings::Pretty::Charset::UTF8 ? diff --git a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp index e92863c93fb..cc90f654c01 100644 --- a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp @@ -73,8 +73,13 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port // Write row number; auto row_num_string = std::to_string(row + 1 + total_rows) + ". "; for (size_t i = 0; i < row_number_width - row_num_string.size(); ++i) - writeCString(" ", out); + writeChar(' ', out); + if (color) + writeCString("\033[90m", out); writeString(row_num_string, out); + if (color) + writeCString("\033[0m", out); + } for (size_t column = 0; column < num_columns; ++column) { diff --git a/tests/queries/0_stateless/01670_neighbor_lc_bug.sql b/tests/queries/0_stateless/01670_neighbor_lc_bug.sql index 3cb194ccc64..2c95e16e916 100644 --- a/tests/queries/0_stateless/01670_neighbor_lc_bug.sql +++ b/tests/queries/0_stateless/01670_neighbor_lc_bug.sql @@ -1,3 +1,5 @@ +SET output_format_pretty_row_numbers = 0; + SELECT neighbor(n, -2) AS int, neighbor(s, -2) AS str, diff --git a/tests/queries/0_stateless/02947_merge_tree_index_table_2.sql b/tests/queries/0_stateless/02947_merge_tree_index_table_2.sql index 5520962fb7a..f4fb2cec2dc 100644 --- a/tests/queries/0_stateless/02947_merge_tree_index_table_2.sql +++ b/tests/queries/0_stateless/02947_merge_tree_index_table_2.sql @@ -1,5 +1,6 @@ DROP TABLE IF EXISTS t_merge_tree_index; +SET output_format_pretty_row_numbers = 0; SET print_pretty_type_names = 0; CREATE TABLE t_merge_tree_index From be784bb649a012b90c6fa233d370bc293d55fd75 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 23 Mar 2024 03:24:50 +0100 Subject: [PATCH 067/102] Update tests --- ..._visible_width_of_tuple_of_dates.reference | 6 +- .../0_stateless/00098_k_union_all.reference | 24 +- .../00298_enum_width_and_cast.reference | 34 +- .../00305_http_and_readonly.reference | 24 +- ...isible_width_of_array_tuple_enum.reference | 6 +- .../00405_PrettyCompactMonoBlock.reference | 48 +- ...00405_output_format_pretty_color.reference | 696 +++++++++--------- .../00405_pretty_formats.reference | 552 +++++++------- .../00406_tuples_with_nulls.reference | 30 +- .../00476_pretty_formats_and_widths.reference | 126 ++-- ...0569_parse_date_time_best_effort.reference | 210 +++--- ...0695_pretty_max_column_pad_width.reference | 204 ++--- .../0_stateless/00722_inner_join.reference | 72 +- .../00730_unicode_terminal_format.reference | 170 ++--- ...parse_date_time_best_effort_more.reference | 42 +- .../00818_inner_join_bug_3567.reference | 38 +- .../00820_multiple_joins.reference | 30 +- ...le_joins_subquery_requires_alias.reference | 30 +- .../00847_multiple_join_same_column.reference | 56 +- .../01018_ambiguous_column.reference | 12 +- .../01074_partial_revokes.reference | 36 +- .../01293_pretty_max_value_width.reference | 212 +++--- ...1_parse_date_time_best_effort_us.reference | 72 +- .../01472_many_rows_in_totals.reference | 32 +- ...output_format_pretty_row_numbers.reference | 176 ++--- ...01509_output_format_pretty_row_numbers.sql | 1 + .../01553_settings_early_apply.reference | 4 +- .../01645_system_table_engines.reference | 8 +- .../01656_ipv4_bad_formatting.reference | 8 +- .../01671_merge_join_and_constants.reference | 10 +- ...2026_describe_include_subcolumns.reference | 46 +- .../0_stateless/02152_bool_type.reference | 10 +- .../02375_pretty_formats.reference | 200 ++--- ...2381_parseDateTime64BestEffortUS.reference | 10 +- ...rtingAggregatedToChunksTransform.reference | 12 +- .../02790_sql_standard_fetch.reference | 72 +- ...793_implicit_pretty_format_settings.expect | 2 +- ...explain_settings_not_applied_bug.reference | 20 +- .../02890_describe_table_options.reference | 344 ++++----- ...02896_union_distinct_http_format.reference | 14 +- .../02947_merge_tree_index_table_1.reference | 88 +-- .../02947_merge_tree_index_table_2.reference | 30 +- ..._readable_number_on_single_value.reference | 688 ++++++++--------- 43 files changed, 2253 insertions(+), 2252 deletions(-) diff --git a/tests/queries/0_stateless/00085_visible_width_of_tuple_of_dates.reference b/tests/queries/0_stateless/00085_visible_width_of_tuple_of_dates.reference index 39074512621..e0714f93e6b 100644 --- a/tests/queries/0_stateless/00085_visible_width_of_tuple_of_dates.reference +++ b/tests/queries/0_stateless/00085_visible_width_of_tuple_of_dates.reference @@ -1,3 +1,3 @@ -┌─x───────────────────────────┐ -│ ('2000-01-01','2000-01-01') │ -└─────────────────────────────┘ + ┌─x───────────────────────────┐ +1. │ ('2000-01-01','2000-01-01') │ + └─────────────────────────────┘ diff --git a/tests/queries/0_stateless/00098_k_union_all.reference b/tests/queries/0_stateless/00098_k_union_all.reference index a6921e228af..6a12c30ac8e 100644 --- a/tests/queries/0_stateless/00098_k_union_all.reference +++ b/tests/queries/0_stateless/00098_k_union_all.reference @@ -1,18 +1,18 @@ - 1 + 1 - 1 - 1 +1.  1 + 1 - 1 - 1 +1.  1 + 1 - 1 - 1 +2.  1 + 1 - 1 - 1 +1.  1 + 1 - 1 - 1 +2.  1 + 1 - 1 +3.  1 diff --git a/tests/queries/0_stateless/00298_enum_width_and_cast.reference b/tests/queries/0_stateless/00298_enum_width_and_cast.reference index 617494b3101..706e3f2ae98 100644 --- a/tests/queries/0_stateless/00298_enum_width_and_cast.reference +++ b/tests/queries/0_stateless/00298_enum_width_and_cast.reference @@ -1,17 +1,17 @@ -┌─x─────┬─y─┐ -│ Hello │ 0 │ -└───────┴───┘ -┌─x─────┬─y─┐ -│ Hello │ 0 │ -│ \ │ 0 │ -└───────┴───┘ -┌─x────────┬─y─┐ -│ Hello │ 0 │ -│ \ │ 0 │ -│ \t │ 0 │ -└──────────┴───┘ -┌─x────────┬─y─┬─toInt8(x)─┬─s─────┬─casted─┐ -│ Hello │ 0 │ -100 │ Hello │ Hello │ -│ \ │ 0 │ 0 │ \ │ \ │ -│ \t │ 0 │ 111 │ \t │ \t │ -└──────────┴───┴───────────┴───────┴────────┘ + ┌─x─────┬─y─┐ +1. │ Hello │ 0 │ + └───────┴───┘ + ┌─x─────┬─y─┐ +1. │ Hello │ 0 │ +2. │ \ │ 0 │ + └───────┴───┘ + ┌─x────────┬─y─┐ +1. │ Hello │ 0 │ +2. │ \ │ 0 │ +3. │ \t │ 0 │ + └──────────┴───┘ + ┌─x────────┬─y─┬─toInt8(x)─┬─s─────┬─casted─┐ +1. │ Hello │ 0 │ -100 │ Hello │ Hello │ +2. │ \ │ 0 │ 0 │ \ │ \ │ +3. │ \t │ 0 │ 111 │ \t │ \t │ + └──────────┴───┴───────────┴───────┴────────┘ diff --git a/tests/queries/0_stateless/00305_http_and_readonly.reference b/tests/queries/0_stateless/00305_http_and_readonly.reference index 4d0cb6bd6bc..baddaa07bca 100644 --- a/tests/queries/0_stateless/00305_http_and_readonly.reference +++ b/tests/queries/0_stateless/00305_http_and_readonly.reference @@ -1,19 +1,19 @@ - name value changed + name value changed - max_rows_to_read 10000 1 - readonly 0 0 - name value changed +1. max_rows_to_read 10000 1 +2. readonly 0 0 + name value changed - max_rows_to_read 10000 1 - readonly 2 1 - name value changed +1. max_rows_to_read 10000 1 +2. readonly 2 1 + name value changed - max_rows_to_read 10000 1 - readonly 1 1 - name value changed +1. max_rows_to_read 10000 1 +2. readonly 1 1 + name value changed - max_rows_to_read 10000 1 - readonly 2 1 +1. max_rows_to_read 10000 1 +2. readonly 2 1 Ok Ok 0 diff --git a/tests/queries/0_stateless/00367_visible_width_of_array_tuple_enum.reference b/tests/queries/0_stateless/00367_visible_width_of_array_tuple_enum.reference index f31beaf2dc4..069db227ff4 100644 --- a/tests/queries/0_stateless/00367_visible_width_of_array_tuple_enum.reference +++ b/tests/queries/0_stateless/00367_visible_width_of_array_tuple_enum.reference @@ -1,3 +1,3 @@ -┌─x─────────┬─y───────────┐ -│ ['hello'] │ (1,'hello') │ -└───────────┴─────────────┘ + ┌─x─────────┬─y───────────┐ +1. │ ['hello'] │ (1,'hello') │ + └───────────┴─────────────┘ diff --git a/tests/queries/0_stateless/00405_PrettyCompactMonoBlock.reference b/tests/queries/0_stateless/00405_PrettyCompactMonoBlock.reference index 94d3efa6e0a..54c87785fab 100644 --- a/tests/queries/0_stateless/00405_PrettyCompactMonoBlock.reference +++ b/tests/queries/0_stateless/00405_PrettyCompactMonoBlock.reference @@ -1,32 +1,32 @@ one block -┌─number─┐ -│ 0 │ -│ 1 │ -└────────┘ + ┌─number─┐ +1. │ 0 │ +2. │ 1 │ + └────────┘ two blocks -┌─number─┐ -│ 0 │ -│ 0 │ -└────────┘ + ┌─number─┐ +1. │ 0 │ +2. │ 0 │ + └────────┘ extremes -┌─number─┐ -│ 0 │ -│ 1 │ -│ 2 │ -└────────┘ + ┌─number─┐ +1. │ 0 │ +2. │ 1 │ +3. │ 2 │ + └────────┘ Extremes: -┌─number─┐ -│ 0 │ -│ 2 │ -└────────┘ + ┌─number─┐ +1. │ 0 │ +2. │ 2 │ + └────────┘ totals -┌─sum(number)─┐ -│ 2 │ -│ 1 │ -└─────────────┘ + ┌─sum(number)─┐ +1. │ 2 │ +2. │ 1 │ + └─────────────┘ Totals: -┌─sum(number)─┐ -│ 3 │ -└─────────────┘ + ┌─sum(number)─┐ +1. │ 3 │ + └─────────────┘ diff --git a/tests/queries/0_stateless/00405_output_format_pretty_color.reference b/tests/queries/0_stateless/00405_output_format_pretty_color.reference index aebdb5f1343..c48b81411be 100644 --- a/tests/queries/0_stateless/00405_output_format_pretty_color.reference +++ b/tests/queries/0_stateless/00405_output_format_pretty_color.reference @@ -1,363 +1,363 @@ 0 -┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ -┃ hello ┃ world ┃ tuple ┃ sometimes_nulls ┃ -┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ -│ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ -├───────┼───────┼─────────┼─────────────────┤ -│ 1 │ 1 │ (1,'1') │ 1 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 2 │ 2 │ (2,'2') │ 2 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ -├───────┼───────┼─────────┼─────────────────┤ -│ 4 │ 4 │ (4,'4') │ 1 │ -└───────┴───────┴─────────┴─────────────────┘ -┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ -┃ hello ┃ world ┃ tuple ┃ sometimes_nulls ┃ -┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ -│ 5 │ 5 │ (5,'5') │ 2 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ -├───────┼───────┼─────────┼─────────────────┤ -│ 7 │ 7 │ (7,'7') │ 1 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 8 │ 8 │ (8,'8') │ 2 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ -└───────┴───────┴─────────┴─────────────────┘ -┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ -│ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ -│ 1 │ 1 │ (1,'1') │ 1 │ -│ 2 │ 2 │ (2,'2') │ 2 │ -│ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ -│ 4 │ 4 │ (4,'4') │ 1 │ -└───────┴───────┴─────────┴─────────────────┘ -┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ -│ 5 │ 5 │ (5,'5') │ 2 │ -│ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ -│ 7 │ 7 │ (7,'7') │ 1 │ -│ 8 │ 8 │ (8,'8') │ 2 │ -│ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ -└───────┴───────┴─────────┴─────────────────┘ - hello world tuple sometimes_nulls + ┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ + ┃ hello ┃ world ┃ tuple ┃ sometimes_nulls ┃ + ┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ +1. │ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ + ├───────┼───────┼─────────┼─────────────────┤ +2. │ 1 │ 1 │ (1,'1') │ 1 │ + ├───────┼───────┼─────────┼─────────────────┤ +3. │ 2 │ 2 │ (2,'2') │ 2 │ + ├───────┼───────┼─────────┼─────────────────┤ +4. │ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ + ├───────┼───────┼─────────┼─────────────────┤ +5. │ 4 │ 4 │ (4,'4') │ 1 │ + └───────┴───────┴─────────┴─────────────────┘ + ┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ + ┃ hello ┃ world ┃ tuple ┃ sometimes_nulls ┃ + ┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ + 6. │ 5 │ 5 │ (5,'5') │ 2 │ + ├───────┼───────┼─────────┼─────────────────┤ + 7. │ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ + ├───────┼───────┼─────────┼─────────────────┤ + 8. │ 7 │ 7 │ (7,'7') │ 1 │ + ├───────┼───────┼─────────┼─────────────────┤ + 9. │ 8 │ 8 │ (8,'8') │ 2 │ + ├───────┼───────┼─────────┼─────────────────┤ +10. │ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ + └───────┴───────┴─────────┴─────────────────┘ + ┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ +1. │ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ +2. │ 1 │ 1 │ (1,'1') │ 1 │ +3. │ 2 │ 2 │ (2,'2') │ 2 │ +4. │ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ +5. │ 4 │ 4 │ (4,'4') │ 1 │ + └───────┴───────┴─────────┴─────────────────┘ + ┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ + 6. │ 5 │ 5 │ (5,'5') │ 2 │ + 7. │ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ + 8. │ 7 │ 7 │ (7,'7') │ 1 │ + 9. │ 8 │ 8 │ (8,'8') │ 2 │ +10. │ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ + └───────┴───────┴─────────┴─────────────────┘ + hello world tuple sometimes_nulls - 0 0 (0,'0') ᴺᵁᴸᴸ - 1 1 (1,'1') 1 - 2 2 (2,'2') 2 - 3 3 (3,'3') ᴺᵁᴸᴸ - 4 4 (4,'4') 1 - hello world tuple sometimes_nulls +1. 0 0 (0,'0') ᴺᵁᴸᴸ +2. 1 1 (1,'1') 1 +3. 2 2 (2,'2') 2 +4. 3 3 (3,'3') ᴺᵁᴸᴸ +5. 4 4 (4,'4') 1 + hello world tuple sometimes_nulls - 5 5 (5,'5') 2 - 6 6 (6,'6') ᴺᵁᴸᴸ - 7 7 (7,'7') 1 - 8 8 (8,'8') 2 - 9 9 (9,'9') ᴺᵁᴸᴸ -┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ -│ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ -│ 1 │ 1 │ (1,'1') │ 1 │ -│ 2 │ 2 │ (2,'2') │ 2 │ -│ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ -│ 4 │ 4 │ (4,'4') │ 1 │ -│ 5 │ 5 │ (5,'5') │ 2 │ -│ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ -│ 7 │ 7 │ (7,'7') │ 1 │ -│ 8 │ 8 │ (8,'8') │ 2 │ -│ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ -└───────┴───────┴─────────┴─────────────────┘ -┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ -┃ hello ┃ world ┃ tuple ┃ sometimes_nulls ┃ -┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ -│ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ -├───────┼───────┼─────────┼─────────────────┤ -│ 1 │ 1 │ (1,'1') │ 1 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 2 │ 2 │ (2,'2') │ 2 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ -├───────┼───────┼─────────┼─────────────────┤ -│ 4 │ 4 │ (4,'4') │ 1 │ -└───────┴───────┴─────────┴─────────────────┘ -┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ -┃ hello ┃ world ┃ tuple ┃ sometimes_nulls ┃ -┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ -│ 5 │ 5 │ (5,'5') │ 2 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ -├───────┼───────┼─────────┼─────────────────┤ -│ 7 │ 7 │ (7,'7') │ 1 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 8 │ 8 │ (8,'8') │ 2 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ -└───────┴───────┴─────────┴─────────────────┘ -┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ -│ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ -│ 1 │ 1 │ (1,'1') │ 1 │ -│ 2 │ 2 │ (2,'2') │ 2 │ -│ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ -│ 4 │ 4 │ (4,'4') │ 1 │ -└───────┴───────┴─────────┴─────────────────┘ -┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ -│ 5 │ 5 │ (5,'5') │ 2 │ -│ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ -│ 7 │ 7 │ (7,'7') │ 1 │ -│ 8 │ 8 │ (8,'8') │ 2 │ -│ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ -└───────┴───────┴─────────┴─────────────────┘ - hello world tuple sometimes_nulls + 6. 5 5 (5,'5') 2 + 7. 6 6 (6,'6') ᴺᵁᴸᴸ + 8. 7 7 (7,'7') 1 + 9. 8 8 (8,'8') 2 +10. 9 9 (9,'9') ᴺᵁᴸᴸ + ┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ + 1. │ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ + 2. │ 1 │ 1 │ (1,'1') │ 1 │ + 3. │ 2 │ 2 │ (2,'2') │ 2 │ + 4. │ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ + 5. │ 4 │ 4 │ (4,'4') │ 1 │ + 6. │ 5 │ 5 │ (5,'5') │ 2 │ + 7. │ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ + 8. │ 7 │ 7 │ (7,'7') │ 1 │ + 9. │ 8 │ 8 │ (8,'8') │ 2 │ +10. │ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ + └───────┴───────┴─────────┴─────────────────┘ + ┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ + ┃ hello ┃ world ┃ tuple ┃ sometimes_nulls ┃ + ┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ +1. │ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ + ├───────┼───────┼─────────┼─────────────────┤ +2. │ 1 │ 1 │ (1,'1') │ 1 │ + ├───────┼───────┼─────────┼─────────────────┤ +3. │ 2 │ 2 │ (2,'2') │ 2 │ + ├───────┼───────┼─────────┼─────────────────┤ +4. │ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ + ├───────┼───────┼─────────┼─────────────────┤ +5. │ 4 │ 4 │ (4,'4') │ 1 │ + └───────┴───────┴─────────┴─────────────────┘ + ┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ + ┃ hello ┃ world ┃ tuple ┃ sometimes_nulls ┃ + ┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ + 6. │ 5 │ 5 │ (5,'5') │ 2 │ + ├───────┼───────┼─────────┼─────────────────┤ + 7. │ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ + ├───────┼───────┼─────────┼─────────────────┤ + 8. │ 7 │ 7 │ (7,'7') │ 1 │ + ├───────┼───────┼─────────┼─────────────────┤ + 9. │ 8 │ 8 │ (8,'8') │ 2 │ + ├───────┼───────┼─────────┼─────────────────┤ +10. │ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ + └───────┴───────┴─────────┴─────────────────┘ + ┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ +1. │ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ +2. │ 1 │ 1 │ (1,'1') │ 1 │ +3. │ 2 │ 2 │ (2,'2') │ 2 │ +4. │ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ +5. │ 4 │ 4 │ (4,'4') │ 1 │ + └───────┴───────┴─────────┴─────────────────┘ + ┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ + 6. │ 5 │ 5 │ (5,'5') │ 2 │ + 7. │ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ + 8. │ 7 │ 7 │ (7,'7') │ 1 │ + 9. │ 8 │ 8 │ (8,'8') │ 2 │ +10. │ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ + └───────┴───────┴─────────┴─────────────────┘ + hello world tuple sometimes_nulls - 0 0 (0,'0') ᴺᵁᴸᴸ - 1 1 (1,'1') 1 - 2 2 (2,'2') 2 - 3 3 (3,'3') ᴺᵁᴸᴸ - 4 4 (4,'4') 1 - hello world tuple sometimes_nulls +1. 0 0 (0,'0') ᴺᵁᴸᴸ +2. 1 1 (1,'1') 1 +3. 2 2 (2,'2') 2 +4. 3 3 (3,'3') ᴺᵁᴸᴸ +5. 4 4 (4,'4') 1 + hello world tuple sometimes_nulls - 5 5 (5,'5') 2 - 6 6 (6,'6') ᴺᵁᴸᴸ - 7 7 (7,'7') 1 - 8 8 (8,'8') 2 - 9 9 (9,'9') ᴺᵁᴸᴸ + 6. 5 5 (5,'5') 2 + 7. 6 6 (6,'6') ᴺᵁᴸᴸ + 8. 7 7 (7,'7') 1 + 9. 8 8 (8,'8') 2 +10. 9 9 (9,'9') ᴺᵁᴸᴸ 1 -┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ -┃ hello ┃ world ┃ tuple  ┃ sometimes_nulls ┃ -┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ -│ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ -├───────┼───────┼─────────┼─────────────────┤ -│ 1 │ 1 │ (1,'1') │ 1 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 2 │ 2 │ (2,'2') │ 2 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ -├───────┼───────┼─────────┼─────────────────┤ -│ 4 │ 4 │ (4,'4') │ 1 │ -└───────┴───────┴─────────┴─────────────────┘ -┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ -┃ hello ┃ world ┃ tuple  ┃ sometimes_nulls ┃ -┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ -│ 5 │ 5 │ (5,'5') │ 2 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ -├───────┼───────┼─────────┼─────────────────┤ -│ 7 │ 7 │ (7,'7') │ 1 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 8 │ 8 │ (8,'8') │ 2 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ -└───────┴───────┴─────────┴─────────────────┘ -┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ -│ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ -│ 1 │ 1 │ (1,'1') │ 1 │ -│ 2 │ 2 │ (2,'2') │ 2 │ -│ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ -│ 4 │ 4 │ (4,'4') │ 1 │ -└───────┴───────┴─────────┴─────────────────┘ -┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ -│ 5 │ 5 │ (5,'5') │ 2 │ -│ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ -│ 7 │ 7 │ (7,'7') │ 1 │ -│ 8 │ 8 │ (8,'8') │ 2 │ -│ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ -└───────┴───────┴─────────┴─────────────────┘ - hello world tuple sometimes_nulls + ┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ + ┃ hello ┃ world ┃ tuple  ┃ sometimes_nulls ┃ + ┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ +1. │ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ + ├───────┼───────┼─────────┼─────────────────┤ +2. │ 1 │ 1 │ (1,'1') │ 1 │ + ├───────┼───────┼─────────┼─────────────────┤ +3. │ 2 │ 2 │ (2,'2') │ 2 │ + ├───────┼───────┼─────────┼─────────────────┤ +4. │ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ + ├───────┼───────┼─────────┼─────────────────┤ +5. │ 4 │ 4 │ (4,'4') │ 1 │ + └───────┴───────┴─────────┴─────────────────┘ + ┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ + ┃ hello ┃ world ┃ tuple  ┃ sometimes_nulls ┃ + ┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ + 6. │ 5 │ 5 │ (5,'5') │ 2 │ + ├───────┼───────┼─────────┼─────────────────┤ + 7. │ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ + ├───────┼───────┼─────────┼─────────────────┤ + 8. │ 7 │ 7 │ (7,'7') │ 1 │ + ├───────┼───────┼─────────┼─────────────────┤ + 9. │ 8 │ 8 │ (8,'8') │ 2 │ + ├───────┼───────┼─────────┼─────────────────┤ +10. │ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ + └───────┴───────┴─────────┴─────────────────┘ + ┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ +1. │ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ +2. │ 1 │ 1 │ (1,'1') │ 1 │ +3. │ 2 │ 2 │ (2,'2') │ 2 │ +4. │ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ +5. │ 4 │ 4 │ (4,'4') │ 1 │ + └───────┴───────┴─────────┴─────────────────┘ + ┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ + 6. │ 5 │ 5 │ (5,'5') │ 2 │ + 7. │ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ + 8. │ 7 │ 7 │ (7,'7') │ 1 │ + 9. │ 8 │ 8 │ (8,'8') │ 2 │ +10. │ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ + └───────┴───────┴─────────┴─────────────────┘ + hello world tuple sometimes_nulls - 0 0 (0,'0') ᴺᵁᴸᴸ - 1 1 (1,'1') 1 - 2 2 (2,'2') 2 - 3 3 (3,'3') ᴺᵁᴸᴸ - 4 4 (4,'4') 1 - hello world tuple sometimes_nulls +1.  0 0 (0,'0') ᴺᵁᴸᴸ +2.  1 1 (1,'1') 1 +3.  2 2 (2,'2') 2 +4.  3 3 (3,'3') ᴺᵁᴸᴸ +5.  4 4 (4,'4') 1 + hello world tuple sometimes_nulls - 5 5 (5,'5') 2 - 6 6 (6,'6') ᴺᵁᴸᴸ - 7 7 (7,'7') 1 - 8 8 (8,'8') 2 - 9 9 (9,'9') ᴺᵁᴸᴸ -┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ -│ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ -│ 1 │ 1 │ (1,'1') │ 1 │ -│ 2 │ 2 │ (2,'2') │ 2 │ -│ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ -│ 4 │ 4 │ (4,'4') │ 1 │ -│ 5 │ 5 │ (5,'5') │ 2 │ -│ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ -│ 7 │ 7 │ (7,'7') │ 1 │ -│ 8 │ 8 │ (8,'8') │ 2 │ -│ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ -└───────┴───────┴─────────┴─────────────────┘ -┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ -┃ hello ┃ world ┃ tuple ┃ sometimes_nulls ┃ -┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ -│ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ -├───────┼───────┼─────────┼─────────────────┤ -│ 1 │ 1 │ (1,'1') │ 1 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 2 │ 2 │ (2,'2') │ 2 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ -├───────┼───────┼─────────┼─────────────────┤ -│ 4 │ 4 │ (4,'4') │ 1 │ -└───────┴───────┴─────────┴─────────────────┘ -┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ -┃ hello ┃ world ┃ tuple ┃ sometimes_nulls ┃ -┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ -│ 5 │ 5 │ (5,'5') │ 2 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ -├───────┼───────┼─────────┼─────────────────┤ -│ 7 │ 7 │ (7,'7') │ 1 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 8 │ 8 │ (8,'8') │ 2 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ -└───────┴───────┴─────────┴─────────────────┘ -┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ -│ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ -│ 1 │ 1 │ (1,'1') │ 1 │ -│ 2 │ 2 │ (2,'2') │ 2 │ -│ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ -│ 4 │ 4 │ (4,'4') │ 1 │ -└───────┴───────┴─────────┴─────────────────┘ -┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ -│ 5 │ 5 │ (5,'5') │ 2 │ -│ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ -│ 7 │ 7 │ (7,'7') │ 1 │ -│ 8 │ 8 │ (8,'8') │ 2 │ -│ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ -└───────┴───────┴─────────┴─────────────────┘ - hello world tuple sometimes_nulls + 6.  5 5 (5,'5') 2 + 7.  6 6 (6,'6') ᴺᵁᴸᴸ + 8.  7 7 (7,'7') 1 + 9.  8 8 (8,'8') 2 +10.  9 9 (9,'9') ᴺᵁᴸᴸ + ┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ + 1. │ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ + 2. │ 1 │ 1 │ (1,'1') │ 1 │ + 3. │ 2 │ 2 │ (2,'2') │ 2 │ + 4. │ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ + 5. │ 4 │ 4 │ (4,'4') │ 1 │ + 6. │ 5 │ 5 │ (5,'5') │ 2 │ + 7. │ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ + 8. │ 7 │ 7 │ (7,'7') │ 1 │ + 9. │ 8 │ 8 │ (8,'8') │ 2 │ +10. │ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ + └───────┴───────┴─────────┴─────────────────┘ + ┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ + ┃ hello ┃ world ┃ tuple ┃ sometimes_nulls ┃ + ┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ +1. │ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ + ├───────┼───────┼─────────┼─────────────────┤ +2. │ 1 │ 1 │ (1,'1') │ 1 │ + ├───────┼───────┼─────────┼─────────────────┤ +3. │ 2 │ 2 │ (2,'2') │ 2 │ + ├───────┼───────┼─────────┼─────────────────┤ +4. │ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ + ├───────┼───────┼─────────┼─────────────────┤ +5. │ 4 │ 4 │ (4,'4') │ 1 │ + └───────┴───────┴─────────┴─────────────────┘ + ┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ + ┃ hello ┃ world ┃ tuple ┃ sometimes_nulls ┃ + ┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ + 6. │ 5 │ 5 │ (5,'5') │ 2 │ + ├───────┼───────┼─────────┼─────────────────┤ + 7. │ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ + ├───────┼───────┼─────────┼─────────────────┤ + 8. │ 7 │ 7 │ (7,'7') │ 1 │ + ├───────┼───────┼─────────┼─────────────────┤ + 9. │ 8 │ 8 │ (8,'8') │ 2 │ + ├───────┼───────┼─────────┼─────────────────┤ +10. │ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ + └───────┴───────┴─────────┴─────────────────┘ + ┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ +1. │ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ +2. │ 1 │ 1 │ (1,'1') │ 1 │ +3. │ 2 │ 2 │ (2,'2') │ 2 │ +4. │ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ +5. │ 4 │ 4 │ (4,'4') │ 1 │ + └───────┴───────┴─────────┴─────────────────┘ + ┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ + 6. │ 5 │ 5 │ (5,'5') │ 2 │ + 7. │ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ + 8. │ 7 │ 7 │ (7,'7') │ 1 │ + 9. │ 8 │ 8 │ (8,'8') │ 2 │ +10. │ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ + └───────┴───────┴─────────┴─────────────────┘ + hello world tuple sometimes_nulls - 0 0 (0,'0') ᴺᵁᴸᴸ - 1 1 (1,'1') 1 - 2 2 (2,'2') 2 - 3 3 (3,'3') ᴺᵁᴸᴸ - 4 4 (4,'4') 1 - hello world tuple sometimes_nulls +1. 0 0 (0,'0') ᴺᵁᴸᴸ +2. 1 1 (1,'1') 1 +3. 2 2 (2,'2') 2 +4. 3 3 (3,'3') ᴺᵁᴸᴸ +5. 4 4 (4,'4') 1 + hello world tuple sometimes_nulls - 5 5 (5,'5') 2 - 6 6 (6,'6') ᴺᵁᴸᴸ - 7 7 (7,'7') 1 - 8 8 (8,'8') 2 - 9 9 (9,'9') ᴺᵁᴸᴸ + 6. 5 5 (5,'5') 2 + 7. 6 6 (6,'6') ᴺᵁᴸᴸ + 8. 7 7 (7,'7') 1 + 9. 8 8 (8,'8') 2 +10. 9 9 (9,'9') ᴺᵁᴸᴸ auto -┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ -┃ hello ┃ world ┃ tuple ┃ sometimes_nulls ┃ -┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ -│ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ -├───────┼───────┼─────────┼─────────────────┤ -│ 1 │ 1 │ (1,'1') │ 1 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 2 │ 2 │ (2,'2') │ 2 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ -├───────┼───────┼─────────┼─────────────────┤ -│ 4 │ 4 │ (4,'4') │ 1 │ -└───────┴───────┴─────────┴─────────────────┘ -┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ -┃ hello ┃ world ┃ tuple ┃ sometimes_nulls ┃ -┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ -│ 5 │ 5 │ (5,'5') │ 2 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ -├───────┼───────┼─────────┼─────────────────┤ -│ 7 │ 7 │ (7,'7') │ 1 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 8 │ 8 │ (8,'8') │ 2 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ -└───────┴───────┴─────────┴─────────────────┘ -┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ -│ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ -│ 1 │ 1 │ (1,'1') │ 1 │ -│ 2 │ 2 │ (2,'2') │ 2 │ -│ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ -│ 4 │ 4 │ (4,'4') │ 1 │ -└───────┴───────┴─────────┴─────────────────┘ -┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ -│ 5 │ 5 │ (5,'5') │ 2 │ -│ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ -│ 7 │ 7 │ (7,'7') │ 1 │ -│ 8 │ 8 │ (8,'8') │ 2 │ -│ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ -└───────┴───────┴─────────┴─────────────────┘ - hello world tuple sometimes_nulls + ┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ + ┃ hello ┃ world ┃ tuple ┃ sometimes_nulls ┃ + ┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ +1. │ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ + ├───────┼───────┼─────────┼─────────────────┤ +2. │ 1 │ 1 │ (1,'1') │ 1 │ + ├───────┼───────┼─────────┼─────────────────┤ +3. │ 2 │ 2 │ (2,'2') │ 2 │ + ├───────┼───────┼─────────┼─────────────────┤ +4. │ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ + ├───────┼───────┼─────────┼─────────────────┤ +5. │ 4 │ 4 │ (4,'4') │ 1 │ + └───────┴───────┴─────────┴─────────────────┘ + ┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ + ┃ hello ┃ world ┃ tuple ┃ sometimes_nulls ┃ + ┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ + 6. │ 5 │ 5 │ (5,'5') │ 2 │ + ├───────┼───────┼─────────┼─────────────────┤ + 7. │ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ + ├───────┼───────┼─────────┼─────────────────┤ + 8. │ 7 │ 7 │ (7,'7') │ 1 │ + ├───────┼───────┼─────────┼─────────────────┤ + 9. │ 8 │ 8 │ (8,'8') │ 2 │ + ├───────┼───────┼─────────┼─────────────────┤ +10. │ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ + └───────┴───────┴─────────┴─────────────────┘ + ┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ +1. │ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ +2. │ 1 │ 1 │ (1,'1') │ 1 │ +3. │ 2 │ 2 │ (2,'2') │ 2 │ +4. │ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ +5. │ 4 │ 4 │ (4,'4') │ 1 │ + └───────┴───────┴─────────┴─────────────────┘ + ┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ + 6. │ 5 │ 5 │ (5,'5') │ 2 │ + 7. │ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ + 8. │ 7 │ 7 │ (7,'7') │ 1 │ + 9. │ 8 │ 8 │ (8,'8') │ 2 │ +10. │ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ + └───────┴───────┴─────────┴─────────────────┘ + hello world tuple sometimes_nulls - 0 0 (0,'0') ᴺᵁᴸᴸ - 1 1 (1,'1') 1 - 2 2 (2,'2') 2 - 3 3 (3,'3') ᴺᵁᴸᴸ - 4 4 (4,'4') 1 - hello world tuple sometimes_nulls +1. 0 0 (0,'0') ᴺᵁᴸᴸ +2. 1 1 (1,'1') 1 +3. 2 2 (2,'2') 2 +4. 3 3 (3,'3') ᴺᵁᴸᴸ +5. 4 4 (4,'4') 1 + hello world tuple sometimes_nulls - 5 5 (5,'5') 2 - 6 6 (6,'6') ᴺᵁᴸᴸ - 7 7 (7,'7') 1 - 8 8 (8,'8') 2 - 9 9 (9,'9') ᴺᵁᴸᴸ -┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ -│ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ -│ 1 │ 1 │ (1,'1') │ 1 │ -│ 2 │ 2 │ (2,'2') │ 2 │ -│ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ -│ 4 │ 4 │ (4,'4') │ 1 │ -│ 5 │ 5 │ (5,'5') │ 2 │ -│ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ -│ 7 │ 7 │ (7,'7') │ 1 │ -│ 8 │ 8 │ (8,'8') │ 2 │ -│ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ -└───────┴───────┴─────────┴─────────────────┘ -┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ -┃ hello ┃ world ┃ tuple ┃ sometimes_nulls ┃ -┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ -│ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ -├───────┼───────┼─────────┼─────────────────┤ -│ 1 │ 1 │ (1,'1') │ 1 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 2 │ 2 │ (2,'2') │ 2 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ -├───────┼───────┼─────────┼─────────────────┤ -│ 4 │ 4 │ (4,'4') │ 1 │ -└───────┴───────┴─────────┴─────────────────┘ -┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ -┃ hello ┃ world ┃ tuple ┃ sometimes_nulls ┃ -┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ -│ 5 │ 5 │ (5,'5') │ 2 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ -├───────┼───────┼─────────┼─────────────────┤ -│ 7 │ 7 │ (7,'7') │ 1 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 8 │ 8 │ (8,'8') │ 2 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ -└───────┴───────┴─────────┴─────────────────┘ -┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ -│ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ -│ 1 │ 1 │ (1,'1') │ 1 │ -│ 2 │ 2 │ (2,'2') │ 2 │ -│ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ -│ 4 │ 4 │ (4,'4') │ 1 │ -└───────┴───────┴─────────┴─────────────────┘ -┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ -│ 5 │ 5 │ (5,'5') │ 2 │ -│ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ -│ 7 │ 7 │ (7,'7') │ 1 │ -│ 8 │ 8 │ (8,'8') │ 2 │ -│ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ -└───────┴───────┴─────────┴─────────────────┘ - hello world tuple sometimes_nulls + 6. 5 5 (5,'5') 2 + 7. 6 6 (6,'6') ᴺᵁᴸᴸ + 8. 7 7 (7,'7') 1 + 9. 8 8 (8,'8') 2 +10. 9 9 (9,'9') ᴺᵁᴸᴸ + ┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ + 1. │ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ + 2. │ 1 │ 1 │ (1,'1') │ 1 │ + 3. │ 2 │ 2 │ (2,'2') │ 2 │ + 4. │ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ + 5. │ 4 │ 4 │ (4,'4') │ 1 │ + 6. │ 5 │ 5 │ (5,'5') │ 2 │ + 7. │ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ + 8. │ 7 │ 7 │ (7,'7') │ 1 │ + 9. │ 8 │ 8 │ (8,'8') │ 2 │ +10. │ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ + └───────┴───────┴─────────┴─────────────────┘ + ┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ + ┃ hello ┃ world ┃ tuple ┃ sometimes_nulls ┃ + ┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ +1. │ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ + ├───────┼───────┼─────────┼─────────────────┤ +2. │ 1 │ 1 │ (1,'1') │ 1 │ + ├───────┼───────┼─────────┼─────────────────┤ +3. │ 2 │ 2 │ (2,'2') │ 2 │ + ├───────┼───────┼─────────┼─────────────────┤ +4. │ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ + ├───────┼───────┼─────────┼─────────────────┤ +5. │ 4 │ 4 │ (4,'4') │ 1 │ + └───────┴───────┴─────────┴─────────────────┘ + ┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ + ┃ hello ┃ world ┃ tuple ┃ sometimes_nulls ┃ + ┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ + 6. │ 5 │ 5 │ (5,'5') │ 2 │ + ├───────┼───────┼─────────┼─────────────────┤ + 7. │ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ + ├───────┼───────┼─────────┼─────────────────┤ + 8. │ 7 │ 7 │ (7,'7') │ 1 │ + ├───────┼───────┼─────────┼─────────────────┤ + 9. │ 8 │ 8 │ (8,'8') │ 2 │ + ├───────┼───────┼─────────┼─────────────────┤ +10. │ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ + └───────┴───────┴─────────┴─────────────────┘ + ┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ +1. │ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ +2. │ 1 │ 1 │ (1,'1') │ 1 │ +3. │ 2 │ 2 │ (2,'2') │ 2 │ +4. │ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ +5. │ 4 │ 4 │ (4,'4') │ 1 │ + └───────┴───────┴─────────┴─────────────────┘ + ┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ + 6. │ 5 │ 5 │ (5,'5') │ 2 │ + 7. │ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ + 8. │ 7 │ 7 │ (7,'7') │ 1 │ + 9. │ 8 │ 8 │ (8,'8') │ 2 │ +10. │ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ + └───────┴───────┴─────────┴─────────────────┘ + hello world tuple sometimes_nulls - 0 0 (0,'0') ᴺᵁᴸᴸ - 1 1 (1,'1') 1 - 2 2 (2,'2') 2 - 3 3 (3,'3') ᴺᵁᴸᴸ - 4 4 (4,'4') 1 - hello world tuple sometimes_nulls +1. 0 0 (0,'0') ᴺᵁᴸᴸ +2. 1 1 (1,'1') 1 +3. 2 2 (2,'2') 2 +4. 3 3 (3,'3') ᴺᵁᴸᴸ +5. 4 4 (4,'4') 1 + hello world tuple sometimes_nulls - 5 5 (5,'5') 2 - 6 6 (6,'6') ᴺᵁᴸᴸ - 7 7 (7,'7') 1 - 8 8 (8,'8') 2 - 9 9 (9,'9') ᴺᵁᴸᴸ + 6. 5 5 (5,'5') 2 + 7. 6 6 (6,'6') ᴺᵁᴸᴸ + 8. 7 7 (7,'7') 1 + 9. 8 8 (8,'8') 2 +10. 9 9 (9,'9') ᴺᵁᴸᴸ diff --git a/tests/queries/0_stateless/00405_pretty_formats.reference b/tests/queries/0_stateless/00405_pretty_formats.reference index bdfff4a8738..bad93c19bb6 100644 --- a/tests/queries/0_stateless/00405_pretty_formats.reference +++ b/tests/queries/0_stateless/00405_pretty_formats.reference @@ -1,306 +1,306 @@ -┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ -┃ hello ┃ world ┃ tuple  ┃ sometimes_nulls ┃ -┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ -│ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ -├───────┼───────┼─────────┼─────────────────┤ -│ 1 │ 1 │ (1,'1') │ 1 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 2 │ 2 │ (2,'2') │ 2 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ -├───────┼───────┼─────────┼─────────────────┤ -│ 4 │ 4 │ (4,'4') │ 1 │ -└───────┴───────┴─────────┴─────────────────┘ -┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ -┃ hello ┃ world ┃ tuple  ┃ sometimes_nulls ┃ -┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ -│ 5 │ 5 │ (5,'5') │ 2 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ -├───────┼───────┼─────────┼─────────────────┤ -│ 7 │ 7 │ (7,'7') │ 1 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 8 │ 8 │ (8,'8') │ 2 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ -└───────┴───────┴─────────┴─────────────────┘ -┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ -│ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ -│ 1 │ 1 │ (1,'1') │ 1 │ -│ 2 │ 2 │ (2,'2') │ 2 │ -│ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ -│ 4 │ 4 │ (4,'4') │ 1 │ -└───────┴───────┴─────────┴─────────────────┘ -┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ -│ 5 │ 5 │ (5,'5') │ 2 │ -│ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ -│ 7 │ 7 │ (7,'7') │ 1 │ -│ 8 │ 8 │ (8,'8') │ 2 │ -│ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ -└───────┴───────┴─────────┴─────────────────┘ - hello world tuple sometimes_nulls + ┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ + ┃ hello ┃ world ┃ tuple  ┃ sometimes_nulls ┃ + ┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ +1. │ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ + ├───────┼───────┼─────────┼─────────────────┤ +2. │ 1 │ 1 │ (1,'1') │ 1 │ + ├───────┼───────┼─────────┼─────────────────┤ +3. │ 2 │ 2 │ (2,'2') │ 2 │ + ├───────┼───────┼─────────┼─────────────────┤ +4. │ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ + ├───────┼───────┼─────────┼─────────────────┤ +5. │ 4 │ 4 │ (4,'4') │ 1 │ + └───────┴───────┴─────────┴─────────────────┘ + ┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ + ┃ hello ┃ world ┃ tuple  ┃ sometimes_nulls ┃ + ┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ + 6. │ 5 │ 5 │ (5,'5') │ 2 │ + ├───────┼───────┼─────────┼─────────────────┤ + 7. │ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ + ├───────┼───────┼─────────┼─────────────────┤ + 8. │ 7 │ 7 │ (7,'7') │ 1 │ + ├───────┼───────┼─────────┼─────────────────┤ + 9. │ 8 │ 8 │ (8,'8') │ 2 │ + ├───────┼───────┼─────────┼─────────────────┤ +10. │ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ + └───────┴───────┴─────────┴─────────────────┘ + ┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ +1. │ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ +2. │ 1 │ 1 │ (1,'1') │ 1 │ +3. │ 2 │ 2 │ (2,'2') │ 2 │ +4. │ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ +5. │ 4 │ 4 │ (4,'4') │ 1 │ + └───────┴───────┴─────────┴─────────────────┘ + ┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ + 6. │ 5 │ 5 │ (5,'5') │ 2 │ + 7. │ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ + 8. │ 7 │ 7 │ (7,'7') │ 1 │ + 9. │ 8 │ 8 │ (8,'8') │ 2 │ +10. │ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ + └───────┴───────┴─────────┴─────────────────┘ + hello world tuple sometimes_nulls - 0 0 (0,'0') ᴺᵁᴸᴸ - 1 1 (1,'1') 1 - 2 2 (2,'2') 2 - 3 3 (3,'3') ᴺᵁᴸᴸ - 4 4 (4,'4') 1 - hello world tuple sometimes_nulls +1.  0 0 (0,'0') ᴺᵁᴸᴸ +2.  1 1 (1,'1') 1 +3.  2 2 (2,'2') 2 +4.  3 3 (3,'3') ᴺᵁᴸᴸ +5.  4 4 (4,'4') 1 + hello world tuple sometimes_nulls - 5 5 (5,'5') 2 - 6 6 (6,'6') ᴺᵁᴸᴸ - 7 7 (7,'7') 1 - 8 8 (8,'8') 2 - 9 9 (9,'9') ᴺᵁᴸᴸ -┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ -│ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ -│ 1 │ 1 │ (1,'1') │ 1 │ -│ 2 │ 2 │ (2,'2') │ 2 │ -│ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ -│ 4 │ 4 │ (4,'4') │ 1 │ -│ 5 │ 5 │ (5,'5') │ 2 │ -│ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ -│ 7 │ 7 │ (7,'7') │ 1 │ -│ 8 │ 8 │ (8,'8') │ 2 │ -│ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ -└───────┴───────┴─────────┴─────────────────┘ -┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ -┃ hello ┃ world ┃ tuple ┃ sometimes_nulls ┃ -┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ -│ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ -├───────┼───────┼─────────┼─────────────────┤ -│ 1 │ 1 │ (1,'1') │ 1 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 2 │ 2 │ (2,'2') │ 2 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ -├───────┼───────┼─────────┼─────────────────┤ -│ 4 │ 4 │ (4,'4') │ 1 │ -└───────┴───────┴─────────┴─────────────────┘ -┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ -┃ hello ┃ world ┃ tuple ┃ sometimes_nulls ┃ -┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ -│ 5 │ 5 │ (5,'5') │ 2 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ -├───────┼───────┼─────────┼─────────────────┤ -│ 7 │ 7 │ (7,'7') │ 1 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 8 │ 8 │ (8,'8') │ 2 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ -└───────┴───────┴─────────┴─────────────────┘ -┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ -│ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ -│ 1 │ 1 │ (1,'1') │ 1 │ -│ 2 │ 2 │ (2,'2') │ 2 │ -│ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ -│ 4 │ 4 │ (4,'4') │ 1 │ -└───────┴───────┴─────────┴─────────────────┘ -┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ -│ 5 │ 5 │ (5,'5') │ 2 │ -│ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ -│ 7 │ 7 │ (7,'7') │ 1 │ -│ 8 │ 8 │ (8,'8') │ 2 │ -│ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ -└───────┴───────┴─────────┴─────────────────┘ - hello world tuple sometimes_nulls + 6.  5 5 (5,'5') 2 + 7.  6 6 (6,'6') ᴺᵁᴸᴸ + 8.  7 7 (7,'7') 1 + 9.  8 8 (8,'8') 2 +10.  9 9 (9,'9') ᴺᵁᴸᴸ + ┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ + 1. │ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ + 2. │ 1 │ 1 │ (1,'1') │ 1 │ + 3. │ 2 │ 2 │ (2,'2') │ 2 │ + 4. │ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ + 5. │ 4 │ 4 │ (4,'4') │ 1 │ + 6. │ 5 │ 5 │ (5,'5') │ 2 │ + 7. │ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ + 8. │ 7 │ 7 │ (7,'7') │ 1 │ + 9. │ 8 │ 8 │ (8,'8') │ 2 │ +10. │ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ + └───────┴───────┴─────────┴─────────────────┘ + ┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ + ┃ hello ┃ world ┃ tuple ┃ sometimes_nulls ┃ + ┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ +1. │ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ + ├───────┼───────┼─────────┼─────────────────┤ +2. │ 1 │ 1 │ (1,'1') │ 1 │ + ├───────┼───────┼─────────┼─────────────────┤ +3. │ 2 │ 2 │ (2,'2') │ 2 │ + ├───────┼───────┼─────────┼─────────────────┤ +4. │ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ + ├───────┼───────┼─────────┼─────────────────┤ +5. │ 4 │ 4 │ (4,'4') │ 1 │ + └───────┴───────┴─────────┴─────────────────┘ + ┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ + ┃ hello ┃ world ┃ tuple ┃ sometimes_nulls ┃ + ┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ + 6. │ 5 │ 5 │ (5,'5') │ 2 │ + ├───────┼───────┼─────────┼─────────────────┤ + 7. │ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ + ├───────┼───────┼─────────┼─────────────────┤ + 8. │ 7 │ 7 │ (7,'7') │ 1 │ + ├───────┼───────┼─────────┼─────────────────┤ + 9. │ 8 │ 8 │ (8,'8') │ 2 │ + ├───────┼───────┼─────────┼─────────────────┤ +10. │ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ + └───────┴───────┴─────────┴─────────────────┘ + ┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ +1. │ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ +2. │ 1 │ 1 │ (1,'1') │ 1 │ +3. │ 2 │ 2 │ (2,'2') │ 2 │ +4. │ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ +5. │ 4 │ 4 │ (4,'4') │ 1 │ + └───────┴───────┴─────────┴─────────────────┘ + ┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ + 6. │ 5 │ 5 │ (5,'5') │ 2 │ + 7. │ 6 │ 6 │ (6,'6') │ ᴺᵁᴸᴸ │ + 8. │ 7 │ 7 │ (7,'7') │ 1 │ + 9. │ 8 │ 8 │ (8,'8') │ 2 │ +10. │ 9 │ 9 │ (9,'9') │ ᴺᵁᴸᴸ │ + └───────┴───────┴─────────┴─────────────────┘ + hello world tuple sometimes_nulls - 0 0 (0,'0') ᴺᵁᴸᴸ - 1 1 (1,'1') 1 - 2 2 (2,'2') 2 - 3 3 (3,'3') ᴺᵁᴸᴸ - 4 4 (4,'4') 1 - hello world tuple sometimes_nulls +1. 0 0 (0,'0') ᴺᵁᴸᴸ +2. 1 1 (1,'1') 1 +3. 2 2 (2,'2') 2 +4. 3 3 (3,'3') ᴺᵁᴸᴸ +5. 4 4 (4,'4') 1 + hello world tuple sometimes_nulls - 5 5 (5,'5') 2 - 6 6 (6,'6') ᴺᵁᴸᴸ - 7 7 (7,'7') 1 - 8 8 (8,'8') 2 - 9 9 (9,'9') ᴺᵁᴸᴸ -┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ -┃ hello ┃ world ┃ tuple  ┃ sometimes_nulls ┃ -┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ -│ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ -├───────┼───────┼─────────┼─────────────────┤ -│ 1 │ 1 │ (1,'1') │ 1 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 2 │ 2 │ (2,'2') │ 2 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ -├───────┼───────┼─────────┼─────────────────┤ -│ 4 │ 4 │ (4,'4') │ 1 │ -└───────┴───────┴─────────┴─────────────────┘ -┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ -┃ hello ┃ world ┃ tuple  ┃ sometimes_nulls ┃ -┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ -│ 5 │ 5 │ (5,'5') │ 2 │ -└───────┴───────┴─────────┴─────────────────┘ + 6. 5 5 (5,'5') 2 + 7. 6 6 (6,'6') ᴺᵁᴸᴸ + 8. 7 7 (7,'7') 1 + 9. 8 8 (8,'8') 2 +10. 9 9 (9,'9') ᴺᵁᴸᴸ + ┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ + ┃ hello ┃ world ┃ tuple  ┃ sometimes_nulls ┃ + ┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ +1. │ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ + ├───────┼───────┼─────────┼─────────────────┤ +2. │ 1 │ 1 │ (1,'1') │ 1 │ + ├───────┼───────┼─────────┼─────────────────┤ +3. │ 2 │ 2 │ (2,'2') │ 2 │ + ├───────┼───────┼─────────┼─────────────────┤ +4. │ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ + ├───────┼───────┼─────────┼─────────────────┤ +5. │ 4 │ 4 │ (4,'4') │ 1 │ + └───────┴───────┴─────────┴─────────────────┘ + ┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ + ┃ hello ┃ world ┃ tuple  ┃ sometimes_nulls ┃ + ┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ + 6. │ 5 │ 5 │ (5,'5') │ 2 │ + └───────┴───────┴─────────┴─────────────────┘ Showed first 6. -┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ -│ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ -│ 1 │ 1 │ (1,'1') │ 1 │ -│ 2 │ 2 │ (2,'2') │ 2 │ -│ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ -│ 4 │ 4 │ (4,'4') │ 1 │ -└───────┴───────┴─────────┴─────────────────┘ -┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ -│ 5 │ 5 │ (5,'5') │ 2 │ -└───────┴───────┴─────────┴─────────────────┘ + ┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ +1. │ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ +2. │ 1 │ 1 │ (1,'1') │ 1 │ +3. │ 2 │ 2 │ (2,'2') │ 2 │ +4. │ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ +5. │ 4 │ 4 │ (4,'4') │ 1 │ + └───────┴───────┴─────────┴─────────────────┘ + ┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ + 6. │ 5 │ 5 │ (5,'5') │ 2 │ + └───────┴───────┴─────────┴─────────────────┘ Showed first 6. - hello world tuple sometimes_nulls + hello world tuple sometimes_nulls - 0 0 (0,'0') ᴺᵁᴸᴸ - 1 1 (1,'1') 1 - 2 2 (2,'2') 2 - 3 3 (3,'3') ᴺᵁᴸᴸ - 4 4 (4,'4') 1 - hello world tuple sometimes_nulls +1.  0 0 (0,'0') ᴺᵁᴸᴸ +2.  1 1 (1,'1') 1 +3.  2 2 (2,'2') 2 +4.  3 3 (3,'3') ᴺᵁᴸᴸ +5.  4 4 (4,'4') 1 + hello world tuple sometimes_nulls - 5 5 (5,'5') 2 + 6.  5 5 (5,'5') 2 Showed first 6. -┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ -│ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ -│ 1 │ 1 │ (1,'1') │ 1 │ -│ 2 │ 2 │ (2,'2') │ 2 │ -│ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ -│ 4 │ 4 │ (4,'4') │ 1 │ -│ 5 │ 5 │ (5,'5') │ 2 │ -└───────┴───────┴─────────┴─────────────────┘ + ┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ +1. │ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ +2. │ 1 │ 1 │ (1,'1') │ 1 │ +3. │ 2 │ 2 │ (2,'2') │ 2 │ +4. │ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ +5. │ 4 │ 4 │ (4,'4') │ 1 │ +6. │ 5 │ 5 │ (5,'5') │ 2 │ + └───────┴───────┴─────────┴─────────────────┘ Showed first 6. -┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ -┃ hello ┃ world ┃ tuple ┃ sometimes_nulls ┃ -┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ -│ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ -├───────┼───────┼─────────┼─────────────────┤ -│ 1 │ 1 │ (1,'1') │ 1 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 2 │ 2 │ (2,'2') │ 2 │ -├───────┼───────┼─────────┼─────────────────┤ -│ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ -├───────┼───────┼─────────┼─────────────────┤ -│ 4 │ 4 │ (4,'4') │ 1 │ -└───────┴───────┴─────────┴─────────────────┘ -┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ -┃ hello ┃ world ┃ tuple ┃ sometimes_nulls ┃ -┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ -│ 5 │ 5 │ (5,'5') │ 2 │ -└───────┴───────┴─────────┴─────────────────┘ + ┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ + ┃ hello ┃ world ┃ tuple ┃ sometimes_nulls ┃ + ┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ +1. │ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ + ├───────┼───────┼─────────┼─────────────────┤ +2. │ 1 │ 1 │ (1,'1') │ 1 │ + ├───────┼───────┼─────────┼─────────────────┤ +3. │ 2 │ 2 │ (2,'2') │ 2 │ + ├───────┼───────┼─────────┼─────────────────┤ +4. │ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ + ├───────┼───────┼─────────┼─────────────────┤ +5. │ 4 │ 4 │ (4,'4') │ 1 │ + └───────┴───────┴─────────┴─────────────────┘ + ┏━━━━━━━┳━━━━━━━┳━━━━━━━━━┳━━━━━━━━━━━━━━━━━┓ + ┃ hello ┃ world ┃ tuple ┃ sometimes_nulls ┃ + ┡━━━━━━━╇━━━━━━━╇━━━━━━━━━╇━━━━━━━━━━━━━━━━━┩ + 6. │ 5 │ 5 │ (5,'5') │ 2 │ + └───────┴───────┴─────────┴─────────────────┘ Showed first 6. -┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ -│ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ -│ 1 │ 1 │ (1,'1') │ 1 │ -│ 2 │ 2 │ (2,'2') │ 2 │ -│ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ -│ 4 │ 4 │ (4,'4') │ 1 │ -└───────┴───────┴─────────┴─────────────────┘ -┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ -│ 5 │ 5 │ (5,'5') │ 2 │ -└───────┴───────┴─────────┴─────────────────┘ + ┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ +1. │ 0 │ 0 │ (0,'0') │ ᴺᵁᴸᴸ │ +2. │ 1 │ 1 │ (1,'1') │ 1 │ +3. │ 2 │ 2 │ (2,'2') │ 2 │ +4. │ 3 │ 3 │ (3,'3') │ ᴺᵁᴸᴸ │ +5. │ 4 │ 4 │ (4,'4') │ 1 │ + └───────┴───────┴─────────┴─────────────────┘ + ┌─hello─┬─world─┬─tuple───┬─sometimes_nulls─┐ + 6. │ 5 │ 5 │ (5,'5') │ 2 │ + └───────┴───────┴─────────┴─────────────────┘ Showed first 6. - hello world tuple sometimes_nulls + hello world tuple sometimes_nulls - 0 0 (0,'0') ᴺᵁᴸᴸ - 1 1 (1,'1') 1 - 2 2 (2,'2') 2 - 3 3 (3,'3') ᴺᵁᴸᴸ - 4 4 (4,'4') 1 - hello world tuple sometimes_nulls +1. 0 0 (0,'0') ᴺᵁᴸᴸ +2. 1 1 (1,'1') 1 +3. 2 2 (2,'2') 2 +4. 3 3 (3,'3') ᴺᵁᴸᴸ +5. 4 4 (4,'4') 1 + hello world tuple sometimes_nulls - 5 5 (5,'5') 2 + 6. 5 5 (5,'5') 2 Showed first 6. -+-------+-------+---------+-----------------+ -| hello | world | tuple  | sometimes_nulls | -+-------+-------+---------+-----------------+ -| 0 | 0 | (0,'0') | NULL | -+-------+-------+---------+-----------------+ -| 1 | 1 | (1,'1') | 1 | -+-------+-------+---------+-----------------+ -| 2 | 2 | (2,'2') | 2 | -+-------+-------+---------+-----------------+ -| 3 | 3 | (3,'3') | NULL | -+-------+-------+---------+-----------------+ -| 4 | 4 | (4,'4') | 1 | -+-------+-------+---------+-----------------+ -+-------+-------+---------+-----------------+ -| hello | world | tuple  | sometimes_nulls | -+-------+-------+---------+-----------------+ -| 5 | 5 | (5,'5') | 2 | -+-------+-------+---------+-----------------+ + +-------+-------+---------+-----------------+ + | hello | world | tuple  | sometimes_nulls | + +-------+-------+---------+-----------------+ +1. | 0 | 0 | (0,'0') | NULL | + +-------+-------+---------+-----------------+ +2. | 1 | 1 | (1,'1') | 1 | + +-------+-------+---------+-----------------+ +3. | 2 | 2 | (2,'2') | 2 | + +-------+-------+---------+-----------------+ +4. | 3 | 3 | (3,'3') | NULL | + +-------+-------+---------+-----------------+ +5. | 4 | 4 | (4,'4') | 1 | + +-------+-------+---------+-----------------+ + +-------+-------+---------+-----------------+ + | hello | world | tuple  | sometimes_nulls | + +-------+-------+---------+-----------------+ + 6. | 5 | 5 | (5,'5') | 2 | + +-------+-------+---------+-----------------+ Showed first 6. -+-hello-+-world-+-tuple---+-sometimes_nulls-+ -| 0 | 0 | (0,'0') | NULL | -| 1 | 1 | (1,'1') | 1 | -| 2 | 2 | (2,'2') | 2 | -| 3 | 3 | (3,'3') | NULL | -| 4 | 4 | (4,'4') | 1 | -+-------+-------+---------+-----------------+ -+-hello-+-world-+-tuple---+-sometimes_nulls-+ -| 5 | 5 | (5,'5') | 2 | -+-------+-------+---------+-----------------+ + +-hello-+-world-+-tuple---+-sometimes_nulls-+ +1. | 0 | 0 | (0,'0') | NULL | +2. | 1 | 1 | (1,'1') | 1 | +3. | 2 | 2 | (2,'2') | 2 | +4. | 3 | 3 | (3,'3') | NULL | +5. | 4 | 4 | (4,'4') | 1 | + +-------+-------+---------+-----------------+ + +-hello-+-world-+-tuple---+-sometimes_nulls-+ + 6. | 5 | 5 | (5,'5') | 2 | + +-------+-------+---------+-----------------+ Showed first 6. - hello world tuple sometimes_nulls + hello world tuple sometimes_nulls - 0 0 (0,'0') NULL - 1 1 (1,'1') 1 - 2 2 (2,'2') 2 - 3 3 (3,'3') NULL - 4 4 (4,'4') 1 - hello world tuple sometimes_nulls +1.  0 0 (0,'0') NULL +2.  1 1 (1,'1') 1 +3.  2 2 (2,'2') 2 +4.  3 3 (3,'3') NULL +5.  4 4 (4,'4') 1 + hello world tuple sometimes_nulls - 5 5 (5,'5') 2 + 6.  5 5 (5,'5') 2 Showed first 6. -+-hello-+-world-+-tuple---+-sometimes_nulls-+ -| 0 | 0 | (0,'0') | NULL | -| 1 | 1 | (1,'1') | 1 | -| 2 | 2 | (2,'2') | 2 | -| 3 | 3 | (3,'3') | NULL | -| 4 | 4 | (4,'4') | 1 | -| 5 | 5 | (5,'5') | 2 | -+-------+-------+---------+-----------------+ + +-hello-+-world-+-tuple---+-sometimes_nulls-+ +1. | 0 | 0 | (0,'0') | NULL | +2. | 1 | 1 | (1,'1') | 1 | +3. | 2 | 2 | (2,'2') | 2 | +4. | 3 | 3 | (3,'3') | NULL | +5. | 4 | 4 | (4,'4') | 1 | +6. | 5 | 5 | (5,'5') | 2 | + +-------+-------+---------+-----------------+ Showed first 6. -+-------+-------+---------+-----------------+ -| hello | world | tuple | sometimes_nulls | -+-------+-------+---------+-----------------+ -| 0 | 0 | (0,'0') | NULL | -+-------+-------+---------+-----------------+ -| 1 | 1 | (1,'1') | 1 | -+-------+-------+---------+-----------------+ -| 2 | 2 | (2,'2') | 2 | -+-------+-------+---------+-----------------+ -| 3 | 3 | (3,'3') | NULL | -+-------+-------+---------+-----------------+ -| 4 | 4 | (4,'4') | 1 | -+-------+-------+---------+-----------------+ -+-------+-------+---------+-----------------+ -| hello | world | tuple | sometimes_nulls | -+-------+-------+---------+-----------------+ -| 5 | 5 | (5,'5') | 2 | -+-------+-------+---------+-----------------+ + +-------+-------+---------+-----------------+ + | hello | world | tuple | sometimes_nulls | + +-------+-------+---------+-----------------+ +1. | 0 | 0 | (0,'0') | NULL | + +-------+-------+---------+-----------------+ +2. | 1 | 1 | (1,'1') | 1 | + +-------+-------+---------+-----------------+ +3. | 2 | 2 | (2,'2') | 2 | + +-------+-------+---------+-----------------+ +4. | 3 | 3 | (3,'3') | NULL | + +-------+-------+---------+-----------------+ +5. | 4 | 4 | (4,'4') | 1 | + +-------+-------+---------+-----------------+ + +-------+-------+---------+-----------------+ + | hello | world | tuple | sometimes_nulls | + +-------+-------+---------+-----------------+ + 6. | 5 | 5 | (5,'5') | 2 | + +-------+-------+---------+-----------------+ Showed first 6. -+-hello-+-world-+-tuple---+-sometimes_nulls-+ -| 0 | 0 | (0,'0') | NULL | -| 1 | 1 | (1,'1') | 1 | -| 2 | 2 | (2,'2') | 2 | -| 3 | 3 | (3,'3') | NULL | -| 4 | 4 | (4,'4') | 1 | -+-------+-------+---------+-----------------+ -+-hello-+-world-+-tuple---+-sometimes_nulls-+ -| 5 | 5 | (5,'5') | 2 | -+-------+-------+---------+-----------------+ + +-hello-+-world-+-tuple---+-sometimes_nulls-+ +1. | 0 | 0 | (0,'0') | NULL | +2. | 1 | 1 | (1,'1') | 1 | +3. | 2 | 2 | (2,'2') | 2 | +4. | 3 | 3 | (3,'3') | NULL | +5. | 4 | 4 | (4,'4') | 1 | + +-------+-------+---------+-----------------+ + +-hello-+-world-+-tuple---+-sometimes_nulls-+ + 6. | 5 | 5 | (5,'5') | 2 | + +-------+-------+---------+-----------------+ Showed first 6. - hello world tuple sometimes_nulls + hello world tuple sometimes_nulls - 0 0 (0,'0') NULL - 1 1 (1,'1') 1 - 2 2 (2,'2') 2 - 3 3 (3,'3') NULL - 4 4 (4,'4') 1 - hello world tuple sometimes_nulls +1. 0 0 (0,'0') NULL +2. 1 1 (1,'1') 1 +3. 2 2 (2,'2') 2 +4. 3 3 (3,'3') NULL +5. 4 4 (4,'4') 1 + hello world tuple sometimes_nulls - 5 5 (5,'5') 2 + 6. 5 5 (5,'5') 2 Showed first 6. diff --git a/tests/queries/0_stateless/00406_tuples_with_nulls.reference b/tests/queries/0_stateless/00406_tuples_with_nulls.reference index c383c208b64..614cb1ace7f 100644 --- a/tests/queries/0_stateless/00406_tuples_with_nulls.reference +++ b/tests/queries/0_stateless/00406_tuples_with_nulls.reference @@ -1,15 +1,15 @@ -┌─tuple─────────┐ -│ (0,NULL,NULL) │ -│ (1,1,'1') │ -│ (2,2,NULL) │ -│ (3,NULL,'1') │ -│ (4,1,NULL) │ -│ (5,2,'1') │ -│ (6,NULL,NULL) │ -│ (7,1,'1') │ -│ (8,2,NULL) │ -│ (9,NULL,'1') │ -└───────────────┘ -┌─x────┬─y──────┐ -│ ᴺᵁᴸᴸ │ (NULL) │ -└──────┴────────┘ + ┌─tuple─────────┐ + 1. │ (0,NULL,NULL) │ + 2. │ (1,1,'1') │ + 3. │ (2,2,NULL) │ + 4. │ (3,NULL,'1') │ + 5. │ (4,1,NULL) │ + 6. │ (5,2,'1') │ + 7. │ (6,NULL,NULL) │ + 8. │ (7,1,'1') │ + 9. │ (8,2,NULL) │ +10. │ (9,NULL,'1') │ + └───────────────┘ + ┌─x────┬─y──────┐ +1. │ ᴺᵁᴸᴸ │ (NULL) │ + └──────┴────────┘ diff --git a/tests/queries/0_stateless/00476_pretty_formats_and_widths.reference b/tests/queries/0_stateless/00476_pretty_formats_and_widths.reference index eedaa4b5d8b..599d8b6d5e3 100644 --- a/tests/queries/0_stateless/00476_pretty_formats_and_widths.reference +++ b/tests/queries/0_stateless/00476_pretty_formats_and_widths.reference @@ -1,67 +1,67 @@ -┏━━━━━━━━━━━━┳━━━━━━━━━━━━┓ -┃  x ┃ s  ┃ -┡━━━━━━━━━━━━╇━━━━━━━━━━━━┩ -│ 1 │ 1 │ -├────────────┼────────────┤ -│ 10 │ 10 │ -├────────────┼────────────┤ -│ 100 │ 100 │ -├────────────┼────────────┤ -│ 1000 │ 1000 │ -├────────────┼────────────┤ -│ 10000 │ 10000 │ -├────────────┼────────────┤ -│ 100000 │ 100000 │ -├────────────┼────────────┤ -│ 1000000 │ 1000000 │ -├────────────┼────────────┤ -│ 10000000 │ 10000000 │ -├────────────┼────────────┤ -│ 100000000 │ 100000000 │ -├────────────┼────────────┤ -│ 1000000000 │ 1000000000 │ -└────────────┴────────────┘ -┌──────────x─┬─s──────────┐ -│ 1 │ 1 │ -│ 10 │ 10 │ -│ 100 │ 100 │ -│ 1000 │ 1000 │ -│ 10000 │ 10000 │ -│ 100000 │ 100000 │ -│ 1000000 │ 1000000 │ -│ 10000000 │ 10000000 │ -│ 100000000 │ 100000000 │ -│ 1000000000 │ 1000000000 │ -└────────────┴────────────┘ - x s + ┏━━━━━━━━━━━━┳━━━━━━━━━━━━┓ + ┃  x ┃ s  ┃ + ┡━━━━━━━━━━━━╇━━━━━━━━━━━━┩ + 1. │ 1 │ 1 │ + ├────────────┼────────────┤ + 2. │ 10 │ 10 │ + ├────────────┼────────────┤ + 3. │ 100 │ 100 │ + ├────────────┼────────────┤ + 4. │ 1000 │ 1000 │ + ├────────────┼────────────┤ + 5. │ 10000 │ 10000 │ + ├────────────┼────────────┤ + 6. │ 100000 │ 100000 │ + ├────────────┼────────────┤ + 7. │ 1000000 │ 1000000 │ + ├────────────┼────────────┤ + 8. │ 10000000 │ 10000000 │ + ├────────────┼────────────┤ + 9. │ 100000000 │ 100000000 │ + ├────────────┼────────────┤ +10. │ 1000000000 │ 1000000000 │ + └────────────┴────────────┘ + ┌──────────x─┬─s──────────┐ + 1. │ 1 │ 1 │ + 2. │ 10 │ 10 │ + 3. │ 100 │ 100 │ + 4. │ 1000 │ 1000 │ + 5. │ 10000 │ 10000 │ + 6. │ 100000 │ 100000 │ + 7. │ 1000000 │ 1000000 │ + 8. │ 10000000 │ 10000000 │ + 9. │ 100000000 │ 100000000 │ +10. │ 1000000000 │ 1000000000 │ + └────────────┴────────────┘ + x s - 1 1 - 10 10 - 100 100 - 1000 1000 - 10000 10000 - 100000 100000 - 1000000 1000000 - 10000000 10000000 - 100000000 100000000 - 1000000000 1000000000 -┌──────────x─┬─s──────────┐ -│ 1 │ 1 │ -│ 10 │ 10 │ -│ 100 │ 100 │ -│ 1000 │ 1000 │ -│ 10000 │ 10000 │ -│ 100000 │ 100000 │ -│ 1000000 │ 1000000 │ -│ 10000000 │ 10000000 │ -│ 100000000 │ 100000000 │ -│ 1000000000 │ 1000000000 │ -└────────────┴────────────┘ -┏━━━━━━━━━━┓ -┃ '\\\'\'' ┃ -┡━━━━━━━━━━┩ -│ \'' │ -└──────────┘ + 1.  1 1 + 2.  10 10 + 3.  100 100 + 4.  1000 1000 + 5.  10000 10000 + 6.  100000 100000 + 7.  1000000 1000000 + 8.  10000000 10000000 + 9.  100000000 100000000 +10.  1000000000 1000000000 + ┌──────────x─┬─s──────────┐ + 1. │ 1 │ 1 │ + 2. │ 10 │ 10 │ + 3. │ 100 │ 100 │ + 4. │ 1000 │ 1000 │ + 5. │ 10000 │ 10000 │ + 6. │ 100000 │ 100000 │ + 7. │ 1000000 │ 1000000 │ + 8. │ 10000000 │ 10000000 │ + 9. │ 100000000 │ 100000000 │ +10. │ 1000000000 │ 1000000000 │ + └────────────┴────────────┘ + ┏━━━━━━━━━━┓ + ┃ '\\\'\'' ┃ + ┡━━━━━━━━━━┩ +1. │ \'' │ + └──────────┘ Row 1: ────── '\\\'\'': \'' diff --git a/tests/queries/0_stateless/00569_parse_date_time_best_effort.reference b/tests/queries/0_stateless/00569_parse_date_time_best_effort.reference index 0729a7628f2..89f7d1cbf7c 100644 --- a/tests/queries/0_stateless/00569_parse_date_time_best_effort.reference +++ b/tests/queries/0_stateless/00569_parse_date_time_best_effort.reference @@ -1,106 +1,106 @@ - s a b + s a b - 0 ᴺᵁᴸᴸ 1970-01-01 00:00:00 - 0000 ᴺᵁᴸᴸ 1970-01-01 00:00:00 - 2000-01-01 00:00:00 2000-01-01 00:00:00 2000-01-01 00:00:00 - 2000-01-01 01:00:00 2000-01-01 01:00:00 2000-01-01 01:00:00 - 02/01/17 010203 MSK 2017-01-01 22:02:03 2017-01-01 22:02:03 - 02/01/17 010203 MSK+0100 2017-01-01 21:02:03 2017-01-01 21:02:03 - 02/01/17 010203 UTC+0300 2017-01-01 22:02:03 2017-01-01 22:02:03 - 02/01/17 010203Z 2017-01-02 01:02:03 2017-01-02 01:02:03 - 02/01/1970 010203Z 1970-01-02 01:02:03 1970-01-02 01:02:03 - 02/01/70 010203Z 1970-01-02 01:02:03 1970-01-02 01:02:03 - 11 Feb 2018 06:40:50 +0300 2018-02-11 03:40:50 2018-02-11 03:40:50 - 17 Apr 2000 2 1:2:3 2000-04-17 01:02:03 2000-04-17 01:02:03 - 19700102 01:00:00 1970-01-02 01:00:00 1970-01-02 01:00:00 - 1970010201:00:00 ᴺᵁᴸᴸ 1970-01-01 00:00:00 - 19700102010203 1970-01-02 01:02:03 1970-01-02 01:02:03 - 19700102010203Z 1970-01-02 01:02:03 1970-01-02 01:02:03 - 1970/01/02 010203Z 1970-01-02 01:02:03 1970-01-02 01:02:03 - 20 2000 2000-01-20 00:00:00 2000-01-20 00:00:00 - 201 ᴺᵁᴸᴸ 1970-01-01 00:00:00 - 20160101 2016-01-01 00:00:00 2016-01-01 00:00:00 - 2016-01-01 2016-01-01 00:00:00 2016-01-01 00:00:00 - 201601-01 2016-01-01 01:00:00 2016-01-01 01:00:00 - 2016-01-01MSD 2015-12-31 20:00:00 2015-12-31 20:00:00 - 2016-01-01 MSD 2015-12-31 20:00:00 2015-12-31 20:00:00 - 201601-01 MSD 2016-01-01 04:00:00 2016-01-01 04:00:00 - 2016-01-01UTC 2016-01-01 00:00:00 2016-01-01 00:00:00 - 2016-01-01Z 2016-01-01 00:00:00 2016-01-01 00:00:00 - 2017 2017-01-01 00:00:00 2017-01-01 00:00:00 - 2017/01/00 2017-01-01 00:00:00 2017-01-01 00:00:00 - 2017/01/00 MSD 2016-12-31 20:00:00 2016-12-31 20:00:00 - 2017/01/00 MSD Jun 2017-05-31 20:00:00 2017-05-31 20:00:00 - 2017/01/01 2017-01-01 00:00:00 2017-01-01 00:00:00 - 201701 02 010203 UTC+0300 2017-01-01 22:02:03 2017-01-01 22:02:03 - 2017-01-02 03:04:05 2017-01-02 03:04:05 2017-01-02 03:04:05 - 2017-01-0203:04:05 ᴺᵁᴸᴸ 1970-01-01 00:00:00 - 2017-01-02 03:04:05+0 2017-01-02 03:04:05 2017-01-02 03:04:05 - 2017-01-02 03:04:05+00 2017-01-02 03:04:05 2017-01-02 03:04:05 - 2017-01-02 03:04:05+0000 2017-01-02 03:04:05 2017-01-02 03:04:05 - 2017-01-02 03:04:05 -0100 2017-01-02 04:04:05 2017-01-02 04:04:05 - 2017-01-02 03:04:05+030 2017-01-02 02:34:05 2017-01-02 02:34:05 - 2017-01-02 03:04:05+0300 2017-01-02 00:04:05 2017-01-02 00:04:05 - 2017-01-02 03:04:05+1 2017-01-02 02:04:05 2017-01-02 02:04:05 - 2017-01-02 03:04:05+300 2017-01-02 00:04:05 2017-01-02 00:04:05 - 2017-01-02 03:04:05+900 2017-01-01 18:04:05 2017-01-01 18:04:05 - 2017-01-02 03:04:05GMT 2017-01-02 03:04:05 2017-01-02 03:04:05 - 2017-01-02 03:04:05 MSD 2017-01-01 23:04:05 2017-01-01 23:04:05 - 2017-01-02 03:04:05 MSD Feb 2017-02-01 23:04:05 2017-02-01 23:04:05 - 2017-01-02 03:04:05 MSD Jun 2017-06-01 23:04:05 2017-06-01 23:04:05 - 2017-01-02 03:04:05 MSK 2017-01-02 00:04:05 2017-01-02 00:04:05 - 2017-01-02T03:04:05 2017-01-02 03:04:05 2017-01-02 03:04:05 - 2017-01-02T03:04:05+00 2017-01-02 03:04:05 2017-01-02 03:04:05 - 2017-01-02T03:04:05 -0100 2017-01-02 04:04:05 2017-01-02 04:04:05 - 2017-01-02T03:04:05-0100 2017-01-02 04:04:05 2017-01-02 04:04:05 - 2017-01-02T03:04:05+0100 2017-01-02 02:04:05 2017-01-02 02:04:05 - 2017-01-02T03:04:05Z 2017-01-02 03:04:05 2017-01-02 03:04:05 - 2017-01 03:04:05 MSD Jun 2017-05-31 23:04:05 2017-05-31 23:04:05 - 2017-01 03:04 MSD Jun 2017-05-31 23:04:00 2017-05-31 23:04:00 - 2017/01/31 2017-01-31 00:00:00 2017-01-31 00:00:00 - 2017/01/32 ᴺᵁᴸᴸ 1970-01-01 00:00:00 - 2017-01 MSD Jun 2017-05-31 20:00:00 2017-05-31 20:00:00 - 201701 MSD Jun 2017-05-31 20:00:00 2017-05-31 20:00:00 - 2017 25 1:2:3 ᴺᵁᴸᴸ 1970-01-01 00:00:00 - 2017 25 Apr 1:2:3 2017-04-01 01:02:03 2017-04-01 01:02:03 - 2017 Apr 01 11:22:33 2017-04-01 11:22:33 2017-04-01 11:22:33 - 2017 Apr 02 01/02/03 UTC+0300 ᴺᵁᴸᴸ 1970-01-01 00:00:00 - 2017 Apr 02 010203 UTC+0300 2017-04-01 22:02:03 2017-04-01 22:02:03 - 2017 Apr 02 01:2:3 UTC+0300 2017-04-01 22:02:03 2017-04-01 22:02:03 - 2017 Apr 02 1:02:3 2017-04-02 01:02:03 2017-04-02 01:02:03 - 2017 Apr 02 11:22:33 2017-04-02 11:22:33 2017-04-02 11:22:33 - 2017 Apr 02 1:2:03 2017-04-02 01:02:03 2017-04-02 01:02:03 - 2017 Apr 02 1:22:33 2017-04-02 01:22:33 2017-04-02 01:22:33 - 2017 Apr 02 1:2:3 2017-04-02 01:02:03 2017-04-02 01:02:03 - 2017 Apr 02 1:2:33 2017-04-02 01:02:33 2017-04-02 01:02:33 - 2017 Apr 02 1:2:3 MSK 2017-04-01 22:02:03 2017-04-01 22:02:03 - 2017 Apr 02 1:2:3 MSK 2017 ᴺᵁᴸᴸ 1970-01-01 00:00:00 - 2017 Apr 02 1:2:3 MSK 2018 ᴺᵁᴸᴸ 1970-01-01 00:00:00 - 2017 Apr 02 1:2:3 UTC+0000 2017-04-02 01:02:03 2017-04-02 01:02:03 - 2017 Apr 02 1:2:3 UTC+0300 2017-04-01 22:02:03 2017-04-01 22:02:03 - 2017 Apr 02 1:2:3 UTC+0400 2017-04-01 21:02:03 2017-04-01 21:02:03 - 2017 Apr 2 1:2:3 2017-04-02 01:02:03 2017-04-02 01:02:03 - 2017 Jan 02 010203 UTC+0300 2017-01-01 22:02:03 2017-01-01 22:02:03 - 25 Apr 2017 01:02:03 2017-04-25 01:02:03 2017-04-25 01:02:03 - 25 Apr 2017 1:2:3 2017-04-25 01:02:03 2017-04-25 01:02:03 - 25 Jan 2017 1:2:3 2017-01-25 01:02:03 2017-01-25 01:02:03 - 25 Jan 2017 1:2:3 MSK 2017-01-24 22:02:03 2017-01-24 22:02:03 - 25 Jan 2017 1:2:3 PM 2017-01-25 13:02:03 2017-01-25 13:02:03 - 25 Jan 2017 1:2:3Z 2017-01-25 01:02:03 2017-01-25 01:02:03 - 25 Jan 2017 1:2:3 Z 2017-01-25 01:02:03 2017-01-25 01:02:03 - 25 Jan 2017 1:2:3 Z +0300 2017-01-24 22:02:03 2017-01-24 22:02:03 - 25 Jan 2017 1:2:3 Z+03:00 2017-01-24 22:02:03 2017-01-24 22:02:03 - 25 Jan 2017 1:2:3 Z +0300 OM ᴺᵁᴸᴸ 1970-01-01 00:00:00 - 25 Jan 2017 1:2:3 Z +03:00 PM 2017-01-25 10:02:03 2017-01-25 10:02:03 - 25 Jan 2017 1:2:3 Z +0300 PM 2017-01-25 10:02:03 2017-01-25 10:02:03 - 25 Jan 2017 1:2:3 Z+03:00 PM 2017-01-25 10:02:03 2017-01-25 10:02:03 - 25 Jan 2017 1:2:3 Z +03:30 PM 2017-01-25 09:32:03 2017-01-25 09:32:03 - 25 Jan 2017 1:2:3Z Mo ᴺᵁᴸᴸ 1970-01-01 00:00:00 - 25 Jan 2017 1:2:3Z Mon 2017-01-25 01:02:03 2017-01-25 01:02:03 - 25 Jan 2017 1:2:3Z Moo ᴺᵁᴸᴸ 1970-01-01 00:00:00 - 25 Jan 2017 1:2:3 Z PM 2017-01-25 13:02:03 2017-01-25 13:02:03 - 25 Jan 2017 1:2:3Z PM 2017-01-25 13:02:03 2017-01-25 13:02:03 - 25 Jan 2017 1:2:3 Z PM +03:00 2017-01-25 10:02:03 2017-01-25 10:02:03 - Jun, 11 Feb 2018 06:40:50 +0300 ᴺᵁᴸᴸ 1970-01-01 00:00:00 - Sun 11 Feb 2018 06:40:50 +0300 2018-02-11 03:40:50 2018-02-11 03:40:50 - Sun, 11 Feb 2018 06:40:50 +0300 2018-02-11 03:40:50 2018-02-11 03:40:50 + 1. 0 ᴺᵁᴸᴸ 1970-01-01 00:00:00 + 2. 0000 ᴺᵁᴸᴸ 1970-01-01 00:00:00 + 3. 2000-01-01 00:00:00 2000-01-01 00:00:00 2000-01-01 00:00:00 + 4. 2000-01-01 01:00:00 2000-01-01 01:00:00 2000-01-01 01:00:00 + 5. 02/01/17 010203 MSK 2017-01-01 22:02:03 2017-01-01 22:02:03 + 6. 02/01/17 010203 MSK+0100 2017-01-01 21:02:03 2017-01-01 21:02:03 + 7. 02/01/17 010203 UTC+0300 2017-01-01 22:02:03 2017-01-01 22:02:03 + 8. 02/01/17 010203Z 2017-01-02 01:02:03 2017-01-02 01:02:03 + 9. 02/01/1970 010203Z 1970-01-02 01:02:03 1970-01-02 01:02:03 + 10. 02/01/70 010203Z 1970-01-02 01:02:03 1970-01-02 01:02:03 + 11. 11 Feb 2018 06:40:50 +0300 2018-02-11 03:40:50 2018-02-11 03:40:50 + 12. 17 Apr 2000 2 1:2:3 2000-04-17 01:02:03 2000-04-17 01:02:03 + 13. 19700102 01:00:00 1970-01-02 01:00:00 1970-01-02 01:00:00 + 14. 1970010201:00:00 ᴺᵁᴸᴸ 1970-01-01 00:00:00 + 15. 19700102010203 1970-01-02 01:02:03 1970-01-02 01:02:03 + 16. 19700102010203Z 1970-01-02 01:02:03 1970-01-02 01:02:03 + 17. 1970/01/02 010203Z 1970-01-02 01:02:03 1970-01-02 01:02:03 + 18. 20 2000 2000-01-20 00:00:00 2000-01-20 00:00:00 + 19. 201 ᴺᵁᴸᴸ 1970-01-01 00:00:00 + 20. 20160101 2016-01-01 00:00:00 2016-01-01 00:00:00 + 21. 2016-01-01 2016-01-01 00:00:00 2016-01-01 00:00:00 + 22. 201601-01 2016-01-01 01:00:00 2016-01-01 01:00:00 + 23. 2016-01-01MSD 2015-12-31 20:00:00 2015-12-31 20:00:00 + 24. 2016-01-01 MSD 2015-12-31 20:00:00 2015-12-31 20:00:00 + 25. 201601-01 MSD 2016-01-01 04:00:00 2016-01-01 04:00:00 + 26. 2016-01-01UTC 2016-01-01 00:00:00 2016-01-01 00:00:00 + 27. 2016-01-01Z 2016-01-01 00:00:00 2016-01-01 00:00:00 + 28. 2017 2017-01-01 00:00:00 2017-01-01 00:00:00 + 29. 2017/01/00 2017-01-01 00:00:00 2017-01-01 00:00:00 + 30. 2017/01/00 MSD 2016-12-31 20:00:00 2016-12-31 20:00:00 + 31. 2017/01/00 MSD Jun 2017-05-31 20:00:00 2017-05-31 20:00:00 + 32. 2017/01/01 2017-01-01 00:00:00 2017-01-01 00:00:00 + 33. 201701 02 010203 UTC+0300 2017-01-01 22:02:03 2017-01-01 22:02:03 + 34. 2017-01-02 03:04:05 2017-01-02 03:04:05 2017-01-02 03:04:05 + 35. 2017-01-0203:04:05 ᴺᵁᴸᴸ 1970-01-01 00:00:00 + 36. 2017-01-02 03:04:05+0 2017-01-02 03:04:05 2017-01-02 03:04:05 + 37. 2017-01-02 03:04:05+00 2017-01-02 03:04:05 2017-01-02 03:04:05 + 38. 2017-01-02 03:04:05+0000 2017-01-02 03:04:05 2017-01-02 03:04:05 + 39. 2017-01-02 03:04:05 -0100 2017-01-02 04:04:05 2017-01-02 04:04:05 + 40. 2017-01-02 03:04:05+030 2017-01-02 02:34:05 2017-01-02 02:34:05 + 41. 2017-01-02 03:04:05+0300 2017-01-02 00:04:05 2017-01-02 00:04:05 + 42. 2017-01-02 03:04:05+1 2017-01-02 02:04:05 2017-01-02 02:04:05 + 43. 2017-01-02 03:04:05+300 2017-01-02 00:04:05 2017-01-02 00:04:05 + 44. 2017-01-02 03:04:05+900 2017-01-01 18:04:05 2017-01-01 18:04:05 + 45. 2017-01-02 03:04:05GMT 2017-01-02 03:04:05 2017-01-02 03:04:05 + 46. 2017-01-02 03:04:05 MSD 2017-01-01 23:04:05 2017-01-01 23:04:05 + 47. 2017-01-02 03:04:05 MSD Feb 2017-02-01 23:04:05 2017-02-01 23:04:05 + 48. 2017-01-02 03:04:05 MSD Jun 2017-06-01 23:04:05 2017-06-01 23:04:05 + 49. 2017-01-02 03:04:05 MSK 2017-01-02 00:04:05 2017-01-02 00:04:05 + 50. 2017-01-02T03:04:05 2017-01-02 03:04:05 2017-01-02 03:04:05 + 51. 2017-01-02T03:04:05+00 2017-01-02 03:04:05 2017-01-02 03:04:05 + 52. 2017-01-02T03:04:05 -0100 2017-01-02 04:04:05 2017-01-02 04:04:05 + 53. 2017-01-02T03:04:05-0100 2017-01-02 04:04:05 2017-01-02 04:04:05 + 54. 2017-01-02T03:04:05+0100 2017-01-02 02:04:05 2017-01-02 02:04:05 + 55. 2017-01-02T03:04:05Z 2017-01-02 03:04:05 2017-01-02 03:04:05 + 56. 2017-01 03:04:05 MSD Jun 2017-05-31 23:04:05 2017-05-31 23:04:05 + 57. 2017-01 03:04 MSD Jun 2017-05-31 23:04:00 2017-05-31 23:04:00 + 58. 2017/01/31 2017-01-31 00:00:00 2017-01-31 00:00:00 + 59. 2017/01/32 ᴺᵁᴸᴸ 1970-01-01 00:00:00 + 60. 2017-01 MSD Jun 2017-05-31 20:00:00 2017-05-31 20:00:00 + 61. 201701 MSD Jun 2017-05-31 20:00:00 2017-05-31 20:00:00 + 62. 2017 25 1:2:3 ᴺᵁᴸᴸ 1970-01-01 00:00:00 + 63. 2017 25 Apr 1:2:3 2017-04-01 01:02:03 2017-04-01 01:02:03 + 64. 2017 Apr 01 11:22:33 2017-04-01 11:22:33 2017-04-01 11:22:33 + 65. 2017 Apr 02 01/02/03 UTC+0300 ᴺᵁᴸᴸ 1970-01-01 00:00:00 + 66. 2017 Apr 02 010203 UTC+0300 2017-04-01 22:02:03 2017-04-01 22:02:03 + 67. 2017 Apr 02 01:2:3 UTC+0300 2017-04-01 22:02:03 2017-04-01 22:02:03 + 68. 2017 Apr 02 1:02:3 2017-04-02 01:02:03 2017-04-02 01:02:03 + 69. 2017 Apr 02 11:22:33 2017-04-02 11:22:33 2017-04-02 11:22:33 + 70. 2017 Apr 02 1:2:03 2017-04-02 01:02:03 2017-04-02 01:02:03 + 71. 2017 Apr 02 1:22:33 2017-04-02 01:22:33 2017-04-02 01:22:33 + 72. 2017 Apr 02 1:2:3 2017-04-02 01:02:03 2017-04-02 01:02:03 + 73. 2017 Apr 02 1:2:33 2017-04-02 01:02:33 2017-04-02 01:02:33 + 74. 2017 Apr 02 1:2:3 MSK 2017-04-01 22:02:03 2017-04-01 22:02:03 + 75. 2017 Apr 02 1:2:3 MSK 2017 ᴺᵁᴸᴸ 1970-01-01 00:00:00 + 76. 2017 Apr 02 1:2:3 MSK 2018 ᴺᵁᴸᴸ 1970-01-01 00:00:00 + 77. 2017 Apr 02 1:2:3 UTC+0000 2017-04-02 01:02:03 2017-04-02 01:02:03 + 78. 2017 Apr 02 1:2:3 UTC+0300 2017-04-01 22:02:03 2017-04-01 22:02:03 + 79. 2017 Apr 02 1:2:3 UTC+0400 2017-04-01 21:02:03 2017-04-01 21:02:03 + 80. 2017 Apr 2 1:2:3 2017-04-02 01:02:03 2017-04-02 01:02:03 + 81. 2017 Jan 02 010203 UTC+0300 2017-01-01 22:02:03 2017-01-01 22:02:03 + 82. 25 Apr 2017 01:02:03 2017-04-25 01:02:03 2017-04-25 01:02:03 + 83. 25 Apr 2017 1:2:3 2017-04-25 01:02:03 2017-04-25 01:02:03 + 84. 25 Jan 2017 1:2:3 2017-01-25 01:02:03 2017-01-25 01:02:03 + 85. 25 Jan 2017 1:2:3 MSK 2017-01-24 22:02:03 2017-01-24 22:02:03 + 86. 25 Jan 2017 1:2:3 PM 2017-01-25 13:02:03 2017-01-25 13:02:03 + 87. 25 Jan 2017 1:2:3Z 2017-01-25 01:02:03 2017-01-25 01:02:03 + 88. 25 Jan 2017 1:2:3 Z 2017-01-25 01:02:03 2017-01-25 01:02:03 + 89. 25 Jan 2017 1:2:3 Z +0300 2017-01-24 22:02:03 2017-01-24 22:02:03 + 90. 25 Jan 2017 1:2:3 Z+03:00 2017-01-24 22:02:03 2017-01-24 22:02:03 + 91. 25 Jan 2017 1:2:3 Z +0300 OM ᴺᵁᴸᴸ 1970-01-01 00:00:00 + 92. 25 Jan 2017 1:2:3 Z +03:00 PM 2017-01-25 10:02:03 2017-01-25 10:02:03 + 93. 25 Jan 2017 1:2:3 Z +0300 PM 2017-01-25 10:02:03 2017-01-25 10:02:03 + 94. 25 Jan 2017 1:2:3 Z+03:00 PM 2017-01-25 10:02:03 2017-01-25 10:02:03 + 95. 25 Jan 2017 1:2:3 Z +03:30 PM 2017-01-25 09:32:03 2017-01-25 09:32:03 + 96. 25 Jan 2017 1:2:3Z Mo ᴺᵁᴸᴸ 1970-01-01 00:00:00 + 97. 25 Jan 2017 1:2:3Z Mon 2017-01-25 01:02:03 2017-01-25 01:02:03 + 98. 25 Jan 2017 1:2:3Z Moo ᴺᵁᴸᴸ 1970-01-01 00:00:00 + 99. 25 Jan 2017 1:2:3 Z PM 2017-01-25 13:02:03 2017-01-25 13:02:03 +100. 25 Jan 2017 1:2:3Z PM 2017-01-25 13:02:03 2017-01-25 13:02:03 +101. 25 Jan 2017 1:2:3 Z PM +03:00 2017-01-25 10:02:03 2017-01-25 10:02:03 +102. Jun, 11 Feb 2018 06:40:50 +0300 ᴺᵁᴸᴸ 1970-01-01 00:00:00 +103. Sun 11 Feb 2018 06:40:50 +0300 2018-02-11 03:40:50 2018-02-11 03:40:50 +104. Sun, 11 Feb 2018 06:40:50 +0300 2018-02-11 03:40:50 2018-02-11 03:40:50 diff --git a/tests/queries/0_stateless/00695_pretty_max_column_pad_width.reference b/tests/queries/0_stateless/00695_pretty_max_column_pad_width.reference index f464bff67b8..cd11af5d8f7 100644 --- a/tests/queries/0_stateless/00695_pretty_max_column_pad_width.reference +++ b/tests/queries/0_stateless/00695_pretty_max_column_pad_width.reference @@ -1,102 +1,102 @@ -┌─range(number)──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ -│ [] │ -│ [0] │ -│ [0,1] │ -│ [0,1,2] │ -│ [0,1,2,3] │ -│ [0,1,2,3,4] │ -│ [0,1,2,3,4,5] │ -│ [0,1,2,3,4,5,6] │ -│ [0,1,2,3,4,5,6,7] │ -│ [0,1,2,3,4,5,6,7,8] │ -│ [0,1,2,3,4,5,6,7,8,9] │ -│ [0,1,2,3,4,5,6,7,8,9,10] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97] │ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98] │ -└────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ + ┌─range(number)──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ + 1. │ [] │ + 2. │ [0] │ + 3. │ [0,1] │ + 4. │ [0,1,2] │ + 5. │ [0,1,2,3] │ + 6. │ [0,1,2,3,4] │ + 7. │ [0,1,2,3,4,5] │ + 8. │ [0,1,2,3,4,5,6] │ + 9. │ [0,1,2,3,4,5,6,7] │ + 10. │ [0,1,2,3,4,5,6,7,8] │ + 11. │ [0,1,2,3,4,5,6,7,8,9] │ + 12. │ [0,1,2,3,4,5,6,7,8,9,10] │ + 13. │ [0,1,2,3,4,5,6,7,8,9,10,11] │ + 14. │ [0,1,2,3,4,5,6,7,8,9,10,11,12] │ + 15. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13] │ + 16. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14] │ + 17. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15] │ + 18. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16] │ + 19. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] │ + 20. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18] │ + 21. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19] │ + 22. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20] │ + 23. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21] │ + 24. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22] │ + 25. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23] │ + 26. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24] │ + 27. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25] │ + 28. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26] │ + 29. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27] │ + 30. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28] │ + 31. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29] │ + 32. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30] │ + 33. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31] │ + 34. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32] │ + 35. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33] │ + 36. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34] │ + 37. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35] │ + 38. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36] │ + 39. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37] │ + 40. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38] │ + 41. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39] │ + 42. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40] │ + 43. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41] │ + 44. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42] │ + 45. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43] │ + 46. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44] │ + 47. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45] │ + 48. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46] │ + 49. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47] │ + 50. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48] │ + 51. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49] │ + 52. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50] │ + 53. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51] │ + 54. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52] │ + 55. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53] │ + 56. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54] │ + 57. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55] │ + 58. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56] │ + 59. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57] │ + 60. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58] │ + 61. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59] │ + 62. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60] │ + 63. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61] │ + 64. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62] │ + 65. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63] │ + 66. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64] │ + 67. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65] │ + 68. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66] │ + 69. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67] │ + 70. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68] │ + 71. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69] │ + 72. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70] │ + 73. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71] │ + 74. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72] │ + 75. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73] │ + 76. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74] │ + 77. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75] │ + 78. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76] │ + 79. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77] │ + 80. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78] │ + 81. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79] │ + 82. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80] │ + 83. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81] │ + 84. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82] │ + 85. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83] │ + 86. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84] │ + 87. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85] │ + 88. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86] │ + 89. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87] │ + 90. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88] │ + 91. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89] │ + 92. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90] │ + 93. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91] │ + 94. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92] │ + 95. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93] │ + 96. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94] │ + 97. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95] │ + 98. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96] │ + 99. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97] │ +100. │ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98] │ + └────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ diff --git a/tests/queries/0_stateless/00722_inner_join.reference b/tests/queries/0_stateless/00722_inner_join.reference index b5e8a77a20d..22e099829ae 100644 --- a/tests/queries/0_stateless/00722_inner_join.reference +++ b/tests/queries/0_stateless/00722_inner_join.reference @@ -1,39 +1,39 @@ -┌─database─┬─name─┐ -│ system │ one │ -└──────────┴──────┘ -┌─database─┬─name─┐ -│ system │ one │ -└──────────┴──────┘ -┌─database─┬─name─┐ -│ system │ one │ -└──────────┴──────┘ -┌─x──────┬─name─┐ -│ system │ one │ -└────────┴──────┘ -┌─database─┬─name─┐ -│ system │ one │ -└──────────┴──────┘ -┌─x──────┬─name─┐ -│ system │ one │ -└────────┴──────┘ -┌─database─┬─name─┐ -│ system │ one │ -└──────────┴──────┘ -┌─db.x───┬─name─┐ -│ system │ one │ -└────────┴──────┘ -┌─db.name─┬─name─┐ -│ system │ one │ -└─────────┴──────┘ -┌─db.name─┬─name─┐ -│ system │ one │ -└─────────┴──────┘ -┌─database─┬─name─┐ -│ system │ one │ -└──────────┴──────┘ -┌─database─┬─name─┐ -│ system │ one │ -└──────────┴──────┘ + ┌─database─┬─name─┐ +1. │ system │ one │ + └──────────┴──────┘ + ┌─database─┬─name─┐ +1. │ system │ one │ + └──────────┴──────┘ + ┌─database─┬─name─┐ +1. │ system │ one │ + └──────────┴──────┘ + ┌─x──────┬─name─┐ +1. │ system │ one │ + └────────┴──────┘ + ┌─database─┬─name─┐ +1. │ system │ one │ + └──────────┴──────┘ + ┌─x──────┬─name─┐ +1. │ system │ one │ + └────────┴──────┘ + ┌─database─┬─name─┐ +1. │ system │ one │ + └──────────┴──────┘ + ┌─db.x───┬─name─┐ +1. │ system │ one │ + └────────┴──────┘ + ┌─db.name─┬─name─┐ +1. │ system │ one │ + └─────────┴──────┘ + ┌─db.name─┬─name─┐ +1. │ system │ one │ + └─────────┴──────┘ + ┌─database─┬─name─┐ +1. │ system │ one │ + └──────────┴──────┘ + ┌─database─┬─name─┐ +1. │ system │ one │ + └──────────┴──────┘ 2 2 2 diff --git a/tests/queries/0_stateless/00730_unicode_terminal_format.reference b/tests/queries/0_stateless/00730_unicode_terminal_format.reference index fdfcfd5ed1d..0bf728b0743 100644 --- a/tests/queries/0_stateless/00730_unicode_terminal_format.reference +++ b/tests/queries/0_stateless/00730_unicode_terminal_format.reference @@ -1,88 +1,88 @@ -┏━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ c1 ┃ c2 ┃ -┡━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ Здравствуйте │ Этот код можно отредактировать и запустить! │ -└──────────────┴─────────────────────────────────────────────┘ -┏━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ c1 ┃ c2 ┃ -┡━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ 你好 │ 这段代码是可以编辑并且能够运行的! │ -└──────┴────────────────────────────────────┘ -┏━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ c1 ┃ c2 ┃ -┡━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ Hola │ ¡Este código es editable y ejecutable! │ -└──────┴────────────────────────────────────────┘ -┏━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ c1 ┃ c2 ┃ -┡━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ Bonjour │ Ce code est modifiable et exécutable ! │ -└─────────┴────────────────────────────────────────┘ -┏━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ c1 ┃ c2 ┃ -┡━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ Ciao │ Questo codice è modificabile ed eseguibile! │ -└──────┴─────────────────────────────────────────────┘ -┏━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ c1 ┃ c2 ┃ -┡━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ こんにちは │ このコードは編集して実行出来ます! │ -└────────────┴────────────────────────────────────┘ -┏━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ c1 ┃ c2 ┃ -┡━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ 안녕하세요 │ 여기에서 코드를 수정하고 실행할 수 있습니다! │ -└────────────┴──────────────────────────────────────────────┘ -┏━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ c1 ┃ c2 ┃ -┡━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ Cześć │ Ten kod można edytować oraz uruchomić! │ -└───────┴────────────────────────────────────────┘ -┏━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ c1 ┃ c2 ┃ -┡━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ Olá │ Este código é editável e executável! │ -└─────┴──────────────────────────────────────┘ -┏━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ c1 ┃ c2 ┃ -┡━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ Chào bạn │ Bạn có thể edit và run code trực tiếp! │ -└──────────┴────────────────────────────────────────┘ -┏━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ c1 ┃ c2 ┃ -┡━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ Hallo │ Dieser Code kann bearbeitet und ausgeführt werden! │ -└───────┴────────────────────────────────────────────────────┘ -┏━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ c1 ┃ c2 ┃ -┡━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ Hej │ Den här koden kan redigeras och köras! │ -└─────┴────────────────────────────────────────┘ -┏━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ c1 ┃ c2 ┃ -┡━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ Ahoj │ Tento kód můžete upravit a spustit │ -└──────┴────────────────────────────────────┘ -┏━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ c1 ┃ c2 ┃ -┡━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━┩ -│ Tabs Tabs │ Non-first Tabs │ -└─────────────┴───────────────────────┘ -┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ c1 ┃ c2 ┃ -┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ Control characters  with zero width │ Invalid UTF-8 which eats pending characters , or invalid by itself with zero width │ -└─────────────────────────────────────┴─────────────────────────────────────────────────────────────────────────────────────┘ -┏━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ c1 ┃ c2 ┃ -┡━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ Russian ё and ё │ Zero bytes in middle │ -└──────────────────┴────────────────────────┘ -┏━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━┓ -┃ 'Tabs \t Tabs' ┃ 'Long\tTitle' ┃ -┡━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━┩ -│ Tabs Tabs │ Long Title │ -└────────────────┴───────────────┘ + ┏━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ c1 ┃ c2 ┃ + ┡━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +1. │ Здравствуйте │ Этот код можно отредактировать и запустить! │ + └──────────────┴─────────────────────────────────────────────┘ + ┏━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ c1 ┃ c2 ┃ + ┡━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +2. │ 你好 │ 这段代码是可以编辑并且能够运行的! │ + └──────┴────────────────────────────────────┘ + ┏━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ c1 ┃ c2 ┃ + ┡━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +3. │ Hola │ ¡Este código es editable y ejecutable! │ + └──────┴────────────────────────────────────────┘ + ┏━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ c1 ┃ c2 ┃ + ┡━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +4. │ Bonjour │ Ce code est modifiable et exécutable ! │ + └─────────┴────────────────────────────────────────┘ + ┏━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ c1 ┃ c2 ┃ + ┡━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +5. │ Ciao │ Questo codice è modificabile ed eseguibile! │ + └──────┴─────────────────────────────────────────────┘ + ┏━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ c1 ┃ c2 ┃ + ┡━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +6. │ こんにちは │ このコードは編集して実行出来ます! │ + └────────────┴────────────────────────────────────┘ + ┏━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ c1 ┃ c2 ┃ + ┡━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +7. │ 안녕하세요 │ 여기에서 코드를 수정하고 실행할 수 있습니다! │ + └────────────┴──────────────────────────────────────────────┘ + ┏━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ c1 ┃ c2 ┃ + ┡━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +8. │ Cześć │ Ten kod można edytować oraz uruchomić! │ + └───────┴────────────────────────────────────────┘ + ┏━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ c1 ┃ c2 ┃ + ┡━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +9. │ Olá │ Este código é editável e executável! │ + └─────┴──────────────────────────────────────┘ + ┏━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ c1 ┃ c2 ┃ + ┡━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +10. │ Chào bạn │ Bạn có thể edit và run code trực tiếp! │ + └──────────┴────────────────────────────────────────┘ + ┏━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ c1 ┃ c2 ┃ + ┡━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +11. │ Hallo │ Dieser Code kann bearbeitet und ausgeführt werden! │ + └───────┴────────────────────────────────────────────────────┘ + ┏━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ c1 ┃ c2 ┃ + ┡━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +12. │ Hej │ Den här koden kan redigeras och köras! │ + └─────┴────────────────────────────────────────┘ + ┏━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ c1 ┃ c2 ┃ + ┡━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +13. │ Ahoj │ Tento kód můžete upravit a spustit │ + └──────┴────────────────────────────────────┘ + ┏━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ c1 ┃ c2 ┃ + ┡━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━┩ +14. │ Tabs Tabs │ Non-first Tabs │ + └─────────────┴───────────────────────┘ + ┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ c1 ┃ c2 ┃ + ┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +15. │ Control characters  with zero width │ Invalid UTF-8 which eats pending characters , or invalid by itself with zero width │ + └─────────────────────────────────────┴─────────────────────────────────────────────────────────────────────────────────────┘ + ┏━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ c1 ┃ c2 ┃ + ┡━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━━━━━━━━━━┩ +16. │ Russian ё and ё │ Zero bytes in middle │ + └──────────────────┴────────────────────────┘ + ┏━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━━┓ + ┃ 'Tabs \t Tabs' ┃ 'Long\tTitle' ┃ + ┡━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━━┩ +1. │ Tabs Tabs │ Long Title │ + └────────────────┴───────────────┘ Row 1: ────── '你好': 你好 diff --git a/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.reference b/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.reference index 3acae55e846..e9bdbc4f0bf 100644 --- a/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.reference +++ b/tests/queries/0_stateless/00813_parse_date_time_best_effort_more.reference @@ -1,22 +1,22 @@ - s a b + s a b - 24.12.2018 2018-12-24 00:00:00 2018-12-24 00:00:00 - 24-12-2018 2018-12-24 00:00:00 2018-12-24 00:00:00 - 24.12.18 2018-12-24 00:00:00 2018-12-24 00:00:00 - 24-12-18 2018-12-24 00:00:00 2018-12-24 00:00:00 - 24-Dec-18 2018-12-24 00:00:00 2018-12-24 00:00:00 - 24/DEC/18 2018-12-24 00:00:00 2018-12-24 00:00:00 - 24/DEC/2018 2018-12-24 00:00:00 2018-12-24 00:00:00 - 01-OCT-2015 2015-10-01 00:00:00 2015-10-01 00:00:00 - 24.12.2018 2018-12-24 00:00:00 2018-12-24 00:00:00 - 24-12-2018 2018-12-24 00:00:00 2018-12-24 00:00:00 - 24.12.18 2018-12-24 00:00:00 2018-12-24 00:00:00 - 24-12-18 2018-12-24 00:00:00 2018-12-24 00:00:00 - 24-Dec-18 2018-12-24 00:00:00 2018-12-24 00:00:00 - 24/DEC/18 2018-12-24 00:00:00 2018-12-24 00:00:00 - 24/DEC/2018 2018-12-24 00:00:00 2018-12-24 00:00:00 - 01-OCT-2015 2015-10-01 00:00:00 2015-10-01 00:00:00 - 24.12.18 010203 2018-12-24 01:02:03 2018-12-24 01:02:03 - 24.12.18 01:02:03 2018-12-24 01:02:03 2018-12-24 01:02:03 - 24.DEC.18T01:02:03.000+0300 2018-12-23 22:02:03 2018-12-23 22:02:03 - 01-September-2018 11:22 2018-09-01 11:22:00 2018-09-01 11:22:00 + 1. 24.12.2018 2018-12-24 00:00:00 2018-12-24 00:00:00 + 2. 24-12-2018 2018-12-24 00:00:00 2018-12-24 00:00:00 + 3. 24.12.18 2018-12-24 00:00:00 2018-12-24 00:00:00 + 4. 24-12-18 2018-12-24 00:00:00 2018-12-24 00:00:00 + 5. 24-Dec-18 2018-12-24 00:00:00 2018-12-24 00:00:00 + 6. 24/DEC/18 2018-12-24 00:00:00 2018-12-24 00:00:00 + 7. 24/DEC/2018 2018-12-24 00:00:00 2018-12-24 00:00:00 + 8. 01-OCT-2015 2015-10-01 00:00:00 2015-10-01 00:00:00 + 9. 24.12.2018 2018-12-24 00:00:00 2018-12-24 00:00:00 +10. 24-12-2018 2018-12-24 00:00:00 2018-12-24 00:00:00 +11. 24.12.18 2018-12-24 00:00:00 2018-12-24 00:00:00 +12. 24-12-18 2018-12-24 00:00:00 2018-12-24 00:00:00 +13. 24-Dec-18 2018-12-24 00:00:00 2018-12-24 00:00:00 +14. 24/DEC/18 2018-12-24 00:00:00 2018-12-24 00:00:00 +15. 24/DEC/2018 2018-12-24 00:00:00 2018-12-24 00:00:00 +16. 01-OCT-2015 2015-10-01 00:00:00 2015-10-01 00:00:00 +17. 24.12.18 010203 2018-12-24 01:02:03 2018-12-24 01:02:03 +18. 24.12.18 01:02:03 2018-12-24 01:02:03 2018-12-24 01:02:03 +19. 24.DEC.18T01:02:03.000+0300 2018-12-23 22:02:03 2018-12-23 22:02:03 +20. 01-September-2018 11:22 2018-09-01 11:22:00 2018-09-01 11:22:00 diff --git a/tests/queries/0_stateless/00818_inner_join_bug_3567.reference b/tests/queries/0_stateless/00818_inner_join_bug_3567.reference index 41e0d8ea43a..0194a3c7f09 100644 --- a/tests/queries/0_stateless/00818_inner_join_bug_3567.reference +++ b/tests/queries/0_stateless/00818_inner_join_bug_3567.reference @@ -1,19 +1,19 @@ -┌─a─┬──────────b─┐ -│ a │ 2018-01-01 │ -│ b │ 2018-01-01 │ -│ c │ 2018-01-01 │ -└───┴────────────┘ -┌─c─┬─table2.a─┬──────────d─┬─a─┬──────────b─┐ -│ B │ b │ 2018-01-01 │ B │ 2018-01-01 │ -│ C │ c │ 2018-01-01 │ C │ 2018-01-01 │ -│ D │ d │ 2018-01-01 │ D │ 2018-01-01 │ -└───┴──────────┴────────────┴───┴────────────┘ -┌─a─┬──────────b─┬─c─┬──────────d─┬─c─┐ -│ a │ 2018-01-01 │ │ 1970-01-01 │ │ -│ b │ 2018-01-01 │ B │ 2018-01-01 │ B │ -│ c │ 2018-01-01 │ C │ 2018-01-01 │ C │ -└───┴────────────┴───┴────────────┴───┘ -┌─a─┬──────────b─┬─c─┬──────────d─┬─c─┐ -│ b │ 2018-01-01 │ B │ 2018-01-01 │ B │ -│ c │ 2018-01-01 │ C │ 2018-01-01 │ C │ -└───┴────────────┴───┴────────────┴───┘ + ┌─a─┬──────────b─┐ +1. │ a │ 2018-01-01 │ +2. │ b │ 2018-01-01 │ +3. │ c │ 2018-01-01 │ + └───┴────────────┘ + ┌─c─┬─table2.a─┬──────────d─┬─a─┬──────────b─┐ +1. │ B │ b │ 2018-01-01 │ B │ 2018-01-01 │ +2. │ C │ c │ 2018-01-01 │ C │ 2018-01-01 │ +3. │ D │ d │ 2018-01-01 │ D │ 2018-01-01 │ + └───┴──────────┴────────────┴───┴────────────┘ + ┌─a─┬──────────b─┬─c─┬──────────d─┬─c─┐ +1. │ a │ 2018-01-01 │ │ 1970-01-01 │ │ +2. │ b │ 2018-01-01 │ B │ 2018-01-01 │ B │ +3. │ c │ 2018-01-01 │ C │ 2018-01-01 │ C │ + └───┴────────────┴───┴────────────┴───┘ + ┌─a─┬──────────b─┬─c─┬──────────d─┬─c─┐ +1. │ b │ 2018-01-01 │ B │ 2018-01-01 │ B │ +2. │ c │ 2018-01-01 │ C │ 2018-01-01 │ C │ + └───┴────────────┴───┴────────────┴───┘ diff --git a/tests/queries/0_stateless/00820_multiple_joins.reference b/tests/queries/0_stateless/00820_multiple_joins.reference index 3d7054bacfc..64f5e75a667 100644 --- a/tests/queries/0_stateless/00820_multiple_joins.reference +++ b/tests/queries/0_stateless/00820_multiple_joins.reference @@ -5,9 +5,9 @@ 0 0 0 10 100 1000 20 200 2000 -┌─t1.a─┬─t2.a─┬─t2.b─┬─t3.b─┬─t3.c─┬─t5.a─┬─t5.b─┬─t5.c─┐ -│ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ -└──────┴──────┴──────┴──────┴──────┴──────┴──────┴──────┘ + ┌─t1.a─┬─t2.a─┬─t2.b─┬─t3.b─┬─t3.c─┬─t5.a─┬─t5.b─┬─t5.c─┐ +1. │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ + └──────┴──────┴──────┴──────┴──────┴──────┴──────┴──────┘ 0 0 0 0 6 6 60 60 12 12 120 120 @@ -32,18 +32,18 @@ 6 6 60 60 12 12 120 120 18 18 180 180 -┌─t1.a─┬─t2.a─┬─t2.b─┬─t3.b─┬────c─┐ -│ 0 │ 0 │ 0 │ 0 │ 0 │ -│ 6 │ 6 │ 60 │ 60 │ 600 │ -│ 12 │ 12 │ 120 │ 120 │ 1200 │ -│ 18 │ 18 │ 180 │ 180 │ 1800 │ -└──────┴──────┴──────┴──────┴──────┘ -┌─t1.a─┬─t2.a─┬─t2.b─┬─t3.b─┬────c─┐ -│ 0 │ 0 │ 0 │ 0 │ 0 │ -│ 6 │ 6 │ 60 │ 60 │ 600 │ -│ 12 │ 12 │ 120 │ 120 │ 1200 │ -│ 18 │ 18 │ 180 │ 180 │ 1800 │ -└──────┴──────┴──────┴──────┴──────┘ + ┌─t1.a─┬─t2.a─┬─t2.b─┬─t3.b─┬────c─┐ +1. │ 0 │ 0 │ 0 │ 0 │ 0 │ +2. │ 6 │ 6 │ 60 │ 60 │ 600 │ +3. │ 12 │ 12 │ 120 │ 120 │ 1200 │ +4. │ 18 │ 18 │ 180 │ 180 │ 1800 │ + └──────┴──────┴──────┴──────┴──────┘ + ┌─t1.a─┬─t2.a─┬─t2.b─┬─t3.b─┬────c─┐ +1. │ 0 │ 0 │ 0 │ 0 │ 0 │ +2. │ 6 │ 6 │ 60 │ 60 │ 600 │ +3. │ 12 │ 12 │ 120 │ 120 │ 1200 │ +4. │ 18 │ 18 │ 180 │ 180 │ 1800 │ + └──────┴──────┴──────┴──────┴──────┘ 0 0 0 0 0 0 0 6 6 60 60 66 66 120 12 12 120 120 132 132 240 diff --git a/tests/queries/0_stateless/00820_multiple_joins_subquery_requires_alias.reference b/tests/queries/0_stateless/00820_multiple_joins_subquery_requires_alias.reference index 11755d6bc8b..8104f6e4490 100644 --- a/tests/queries/0_stateless/00820_multiple_joins_subquery_requires_alias.reference +++ b/tests/queries/0_stateless/00820_multiple_joins_subquery_requires_alias.reference @@ -5,9 +5,9 @@ 0 0 0 10 100 1000 20 200 2000 -┌─t1.a─┬─t2.a─┬─t2.b─┬─t3.b─┬─t3.c─┬─t5.a─┬─t5.b─┬─t5.c─┐ -│ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ -└──────┴──────┴──────┴──────┴──────┴──────┴──────┴──────┘ + ┌─t1.a─┬─t2.a─┬─t2.b─┬─t3.b─┬─t3.c─┬─t5.a─┬─t5.b─┬─t5.c─┐ +1. │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ 0 │ + └──────┴──────┴──────┴──────┴──────┴──────┴──────┴──────┘ 0 0 0 0 6 6 60 60 12 12 120 120 @@ -32,18 +32,18 @@ 6 6 60 60 12 12 120 120 18 18 180 180 -┌─t1.a─┬─t2.a─┬─t2.b─┬─t3.b─┬────c─┐ -│ 0 │ 0 │ 0 │ 0 │ 0 │ -│ 6 │ 6 │ 60 │ 60 │ 600 │ -│ 12 │ 12 │ 120 │ 120 │ 1200 │ -│ 18 │ 18 │ 180 │ 180 │ 1800 │ -└──────┴──────┴──────┴──────┴──────┘ -┌─t1.a─┬─t2.a─┬─t2.b─┬─t3.b─┬────c─┐ -│ 0 │ 0 │ 0 │ 0 │ 0 │ -│ 6 │ 6 │ 60 │ 60 │ 600 │ -│ 12 │ 12 │ 120 │ 120 │ 1200 │ -│ 18 │ 18 │ 180 │ 180 │ 1800 │ -└──────┴──────┴──────┴──────┴──────┘ + ┌─t1.a─┬─t2.a─┬─t2.b─┬─t3.b─┬────c─┐ +1. │ 0 │ 0 │ 0 │ 0 │ 0 │ +2. │ 6 │ 6 │ 60 │ 60 │ 600 │ +3. │ 12 │ 12 │ 120 │ 120 │ 1200 │ +4. │ 18 │ 18 │ 180 │ 180 │ 1800 │ + └──────┴──────┴──────┴──────┴──────┘ + ┌─t1.a─┬─t2.a─┬─t2.b─┬─t3.b─┬────c─┐ +1. │ 0 │ 0 │ 0 │ 0 │ 0 │ +2. │ 6 │ 6 │ 60 │ 60 │ 600 │ +3. │ 12 │ 12 │ 120 │ 120 │ 1200 │ +4. │ 18 │ 18 │ 180 │ 180 │ 1800 │ + └──────┴──────┴──────┴──────┴──────┘ 0 0 0 0 0 0 0 6 6 60 60 66 66 120 12 12 120 120 132 132 240 diff --git a/tests/queries/0_stateless/00847_multiple_join_same_column.reference b/tests/queries/0_stateless/00847_multiple_join_same_column.reference index 91bd62ca5a3..112289e1613 100644 --- a/tests/queries/0_stateless/00847_multiple_join_same_column.reference +++ b/tests/queries/0_stateless/00847_multiple_join_same_column.reference @@ -15,31 +15,31 @@ s.a: 0 s.b: 0 y.a: 0 y.b: 0 -┌─t.a─┬─s.b─┬─s.a─┬─s.b─┬─y.a─┬─y.b─┐ -│ 1 │ 1 │ 1 │ 1 │ 1 │ 1 │ -│ 2 │ 0 │ 0 │ 0 │ 0 │ 0 │ -└─────┴─────┴─────┴─────┴─────┴─────┘ -┌─t_a─┐ -│ 1 │ -│ 2 │ -└─────┘ -┌─t.a─┬─s_a─┐ -│ 1 │ 1 │ -│ 2 │ 0 │ -└─────┴─────┘ -┌─t.a─┬─t.a─┬─t_b─┐ -│ 1 │ 1 │ 1 │ -│ 2 │ 2 │ 2 │ -└─────┴─────┴─────┘ -┌─s.a─┬─s.a─┬─s_b─┬─s.b─┐ -│ 1 │ 1 │ 1 │ 1 │ -│ 0 │ 0 │ 0 │ 0 │ -└─────┴─────┴─────┴─────┘ -┌─y.a─┬─y.a─┬─y_b─┬─y.b─┐ -│ 1 │ 1 │ 1 │ 1 │ -│ 0 │ 0 │ 0 │ 0 │ -└─────┴─────┴─────┴─────┘ -┌─t.a─┬─t_a─┬─s.a─┬─s_a─┬─y.a─┬─y_a─┐ -│ 1 │ 1 │ 1 │ 1 │ 1 │ 1 │ -│ 2 │ 2 │ 0 │ 0 │ 0 │ 0 │ -└─────┴─────┴─────┴─────┴─────┴─────┘ + ┌─t.a─┬─s.b─┬─s.a─┬─s.b─┬─y.a─┬─y.b─┐ +1. │ 1 │ 1 │ 1 │ 1 │ 1 │ 1 │ +2. │ 2 │ 0 │ 0 │ 0 │ 0 │ 0 │ + └─────┴─────┴─────┴─────┴─────┴─────┘ + ┌─t_a─┐ +1. │ 1 │ +2. │ 2 │ + └─────┘ + ┌─t.a─┬─s_a─┐ +1. │ 1 │ 1 │ +2. │ 2 │ 0 │ + └─────┴─────┘ + ┌─t.a─┬─t.a─┬─t_b─┐ +1. │ 1 │ 1 │ 1 │ +2. │ 2 │ 2 │ 2 │ + └─────┴─────┴─────┘ + ┌─s.a─┬─s.a─┬─s_b─┬─s.b─┐ +1. │ 1 │ 1 │ 1 │ 1 │ +2. │ 0 │ 0 │ 0 │ 0 │ + └─────┴─────┴─────┴─────┘ + ┌─y.a─┬─y.a─┬─y_b─┬─y.b─┐ +1. │ 1 │ 1 │ 1 │ 1 │ +2. │ 0 │ 0 │ 0 │ 0 │ + └─────┴─────┴─────┴─────┘ + ┌─t.a─┬─t_a─┬─s.a─┬─s_a─┬─y.a─┬─y_a─┐ +1. │ 1 │ 1 │ 1 │ 1 │ 1 │ 1 │ +2. │ 2 │ 2 │ 0 │ 0 │ 0 │ 0 │ + └─────┴─────┴─────┴─────┴─────┴─────┘ diff --git a/tests/queries/0_stateless/01018_ambiguous_column.reference b/tests/queries/0_stateless/01018_ambiguous_column.reference index 308726fa184..ddc2ae1c572 100644 --- a/tests/queries/0_stateless/01018_ambiguous_column.reference +++ b/tests/queries/0_stateless/01018_ambiguous_column.reference @@ -7,10 +7,10 @@ 0 0 0 -┌─system.one.dummy─┬─A.dummy─┬─B.dummy─┐ -│ 0 │ 0 │ 0 │ -└──────────────────┴─────────┴─────────┘ -┌─A.dummy─┬─one.dummy─┬─two.dummy─┐ -│ 0 │ 0 │ 0 │ -└─────────┴───────────┴───────────┘ + ┌─system.one.dummy─┬─A.dummy─┬─B.dummy─┐ +1. │ 0 │ 0 │ 0 │ + └──────────────────┴─────────┴─────────┘ + ┌─A.dummy─┬─one.dummy─┬─two.dummy─┐ +1. │ 0 │ 0 │ 0 │ + └─────────┴───────────┴───────────┘ 0 diff --git a/tests/queries/0_stateless/01074_partial_revokes.reference b/tests/queries/0_stateless/01074_partial_revokes.reference index 9ffa2878ad4..b5baa93efb7 100644 --- a/tests/queries/0_stateless/01074_partial_revokes.reference +++ b/tests/queries/0_stateless/01074_partial_revokes.reference @@ -19,17 +19,17 @@ GRANT SELECT ON *.* TO test_user_01074 REVOKE SELECT ON db.* FROM test_user_01074 GRANT SELECT ON db.`table` TO test_user_01074 REVOKE SELECT(col1) ON db.`table` FROM test_user_01074 -┏━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━┳━━━━━━━━━━━━━┳━━━━━━━━━━┳━━━━━━━┳━━━━━━━━┳━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━┓ -┃ user_name  ┃ role_name ┃ access_type ┃ database ┃ table ┃ column ┃ is_partial_revoke ┃ grant_option ┃ -┡━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━╇━━━━━━━━━━━━━╇━━━━━━━━━━╇━━━━━━━╇━━━━━━━━╇━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━┩ -│ test_user_01074 │ ᴺᵁᴸᴸ │ SELECT │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 0 │ 0 │ -├─────────────────┼───────────┼─────────────┼──────────┼───────┼────────┼───────────────────┼──────────────┤ -│ test_user_01074 │ ᴺᵁᴸᴸ │ SELECT │ db │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 1 │ 0 │ -├─────────────────┼───────────┼─────────────┼──────────┼───────┼────────┼───────────────────┼──────────────┤ -│ test_user_01074 │ ᴺᵁᴸᴸ │ SELECT │ db │ table │ ᴺᵁᴸᴸ │ 0 │ 0 │ -├─────────────────┼───────────┼─────────────┼──────────┼───────┼────────┼───────────────────┼──────────────┤ -│ test_user_01074 │ ᴺᵁᴸᴸ │ SELECT │ db │ table │ col1 │ 1 │ 0 │ -└─────────────────┴───────────┴─────────────┴──────────┴───────┴────────┴───────────────────┴──────────────┘ + ┏━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━┳━━━━━━━━━━━━━┳━━━━━━━━━━┳━━━━━━━┳━━━━━━━━┳━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━┓ + ┃ user_name  ┃ role_name ┃ access_type ┃ database ┃ table ┃ column ┃ is_partial_revoke ┃ grant_option ┃ + ┡━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━╇━━━━━━━━━━━━━╇━━━━━━━━━━╇━━━━━━━╇━━━━━━━━╇━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━┩ +1. │ test_user_01074 │ ᴺᵁᴸᴸ │ SELECT │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 0 │ 0 │ + ├─────────────────┼───────────┼─────────────┼──────────┼───────┼────────┼───────────────────┼──────────────┤ +2. │ test_user_01074 │ ᴺᵁᴸᴸ │ SELECT │ db │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 1 │ 0 │ + ├─────────────────┼───────────┼─────────────┼──────────┼───────┼────────┼───────────────────┼──────────────┤ +3. │ test_user_01074 │ ᴺᵁᴸᴸ │ SELECT │ db │ table │ ᴺᵁᴸᴸ │ 0 │ 0 │ + ├─────────────────┼───────────┼─────────────┼──────────┼───────┼────────┼───────────────────┼──────────────┤ +4. │ test_user_01074 │ ᴺᵁᴸᴸ │ SELECT │ db │ table │ col1 │ 1 │ 0 │ + └─────────────────┴───────────┴─────────────┴──────────┴───────┴────────┴───────────────────┴──────────────┘ --cleanup --revoke 1 GRANT SELECT ON *.* TO test_user_01074 @@ -41,13 +41,13 @@ GRANT SELECT ON *.* TO test_user_01074 --grant option 1 GRANT SELECT ON *.* TO test_user_01074 WITH GRANT OPTION REVOKE GRANT OPTION FOR SELECT(col1) ON db.`table` FROM test_user_01074 -┏━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━┳━━━━━━━━━━━━━┳━━━━━━━━━━┳━━━━━━━┳━━━━━━━━┳━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━┓ -┃ user_name  ┃ role_name ┃ access_type ┃ database ┃ table ┃ column ┃ is_partial_revoke ┃ grant_option ┃ -┡━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━╇━━━━━━━━━━━━━╇━━━━━━━━━━╇━━━━━━━╇━━━━━━━━╇━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━┩ -│ test_user_01074 │ ᴺᵁᴸᴸ │ SELECT │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 0 │ 1 │ -├─────────────────┼───────────┼─────────────┼──────────┼───────┼────────┼───────────────────┼──────────────┤ -│ test_user_01074 │ ᴺᵁᴸᴸ │ SELECT │ db │ table │ col1 │ 1 │ 1 │ -└─────────────────┴───────────┴─────────────┴──────────┴───────┴────────┴───────────────────┴──────────────┘ + ┏━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━┳━━━━━━━━━━━━━┳━━━━━━━━━━┳━━━━━━━┳━━━━━━━━┳━━━━━━━━━━━━━━━━━━━┳━━━━━━━━━━━━━━┓ + ┃ user_name  ┃ role_name ┃ access_type ┃ database ┃ table ┃ column ┃ is_partial_revoke ┃ grant_option ┃ + ┡━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━╇━━━━━━━━━━━━━╇━━━━━━━━━━╇━━━━━━━╇━━━━━━━━╇━━━━━━━━━━━━━━━━━━━╇━━━━━━━━━━━━━━┩ +1. │ test_user_01074 │ ᴺᵁᴸᴸ │ SELECT │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ 0 │ 1 │ + ├─────────────────┼───────────┼─────────────┼──────────┼───────┼────────┼───────────────────┼──────────────┤ +2. │ test_user_01074 │ ᴺᵁᴸᴸ │ SELECT │ db │ table │ col1 │ 1 │ 1 │ + └─────────────────┴───────────┴─────────────┴──────────┴───────┴────────┴───────────────────┴──────────────┘ --cleanup --grant option 2 GRANT SELECT ON *.* TO test_user_01074 WITH GRANT OPTION diff --git a/tests/queries/0_stateless/01293_pretty_max_value_width.reference b/tests/queries/0_stateless/01293_pretty_max_value_width.reference index 9887169f7af..f9143a926c4 100644 --- a/tests/queries/0_stateless/01293_pretty_max_value_width.reference +++ b/tests/queries/0_stateless/01293_pretty_max_value_width.reference @@ -1,114 +1,114 @@ -┏━━━━━━━━┳━━━━━┓ -┃ x  ┃ y  ┃ -┡━━━━━━━━╇━━━━━┩ -│ привет │ мир │ -└────────┴─────┘ -┏━━━━━━━┳━━━━━┓ -┃ x  ┃ y  ┃ -┡━━━━━━━╇━━━━━┩ -│ приве⋯│ мир │ -└───────┴─────┘ -┌─x─────┬─y───┐ -│ приве⋯│ мир │ -└───────┴─────┘ - x y + ┏━━━━━━━━┳━━━━━┓ + ┃ x  ┃ y  ┃ + ┡━━━━━━━━╇━━━━━┩ +1. │ привет │ мир │ + └────────┴─────┘ + ┏━━━━━━━┳━━━━━┓ + ┃ x  ┃ y  ┃ + ┡━━━━━━━╇━━━━━┩ +1. │ приве⋯│ мир │ + └───────┴─────┘ + ┌─x─────┬─y───┐ +1. │ приве⋯│ мир │ + └───────┴─────┘ + x y - приве⋯ мир -┏━━━━━━━┳━━━━━━━┓ -┃ x  ┃ y  ┃ -┡━━━━━━━╇━━━━━━━┩ -│ приве⋯│ мир │ -├───────┼───────┤ -│ мир │ приве⋯│ -└───────┴───────┘ -┌─x─────┬─y─────┐ -│ приве⋯│ мир │ -│ мир │ приве⋯│ -└───────┴───────┘ - x y +1.  приве⋯ мир + ┏━━━━━━━┳━━━━━━━┓ + ┃ x  ┃ y  ┃ + ┡━━━━━━━╇━━━━━━━┩ +1. │ приве⋯│ мир │ + ├───────┼───────┤ +2. │ мир │ приве⋯│ + └───────┴───────┘ + ┌─x─────┬─y─────┐ +1. │ приве⋯│ мир │ +2. │ мир │ приве⋯│ + └───────┴───────┘ + x y - приве⋯ мир - мир приве⋯ -┏━━━━━━━━┳━━━━━┓ -┃ x  ┃ y  ┃ -┡━━━━━━━━╇━━━━━┩ -│ привет │ мир │ -└────────┴─────┘ -┌─x──────┬─y───┐ -│ привет │ мир │ -└────────┴─────┘ - x y +1.  приве⋯ мир +2.  мир приве⋯ + ┏━━━━━━━━┳━━━━━┓ + ┃ x  ┃ y  ┃ + ┡━━━━━━━━╇━━━━━┩ +1. │ привет │ мир │ + └────────┴─────┘ + ┌─x──────┬─y───┐ +1. │ привет │ мир │ + └────────┴─────┘ + x y - привет мир -┏━━━━━━━━┳━━━━━━━━┓ -┃ x  ┃ y  ┃ -┡━━━━━━━━╇━━━━━━━━┩ -│ привет │ мир │ -├────────┼────────┤ -│ мир │ привет │ -└────────┴────────┘ -┌─x──────┬─y──────┐ -│ привет │ мир │ -│ мир │ привет │ -└────────┴────────┘ - x y +1.  привет мир + ┏━━━━━━━━┳━━━━━━━━┓ + ┃ x  ┃ y  ┃ + ┡━━━━━━━━╇━━━━━━━━┩ +1. │ привет │ мир │ + ├────────┼────────┤ +2. │ мир │ привет │ + └────────┴────────┘ + ┌─x──────┬─y──────┐ +1. │ привет │ мир │ +2. │ мир │ привет │ + └────────┴────────┘ + x y - привет мир - мир привет -┏━━━┳━━━┓ -┃ x ┃ y ┃ -┡━━━╇━━━┩ -│ п⋯│ м⋯│ -└───┴───┘ -┌─x─┬─y─┐ -│ п⋯│ м⋯│ -└───┴───┘ - x y +1.  привет мир +2.  мир привет + ┏━━━┳━━━┓ + ┃ x ┃ y ┃ + ┡━━━╇━━━┩ +1. │ п⋯│ м⋯│ + └───┴───┘ + ┌─x─┬─y─┐ +1. │ п⋯│ м⋯│ + └───┴───┘ + x y - п⋯ м⋯ -┏━━━┳━━━┓ -┃ x ┃ y ┃ -┡━━━╇━━━┩ -│ п⋯│ м⋯│ -├───┼───┤ -│ м⋯│ п⋯│ -└───┴───┘ -┌─x─┬─y─┐ -│ п⋯│ м⋯│ -│ м⋯│ п⋯│ -└───┴───┘ - x y +1.  п⋯ м⋯ + ┏━━━┳━━━┓ + ┃ x ┃ y ┃ + ┡━━━╇━━━┩ +1. │ п⋯│ м⋯│ + ├───┼───┤ +2. │ м⋯│ п⋯│ + └───┴───┘ + ┌─x─┬─y─┐ +1. │ п⋯│ м⋯│ +2. │ м⋯│ п⋯│ + └───┴───┘ + x y - п⋯ м⋯ - м⋯ п⋯ -┏━━━┳━━━┓ -┃ x ┃ y ┃ -┡━━━╇━━━┩ -│ ⋯ │ ⋯ │ -└───┴───┘ -┌─x─┬─y─┐ -│ ⋯ │ ⋯ │ -└───┴───┘ - x y +1.  п⋯ м⋯ +2.  м⋯ п⋯ + ┏━━━┳━━━┓ + ┃ x ┃ y ┃ + ┡━━━╇━━━┩ +1. │ ⋯ │ ⋯ │ + └───┴───┘ + ┌─x─┬─y─┐ +1. │ ⋯ │ ⋯ │ + └───┴───┘ + x y - ⋯ ⋯ -┏━━━┳━━━┓ -┃ x ┃ y ┃ -┡━━━╇━━━┩ -│ ⋯ │ ⋯ │ -├───┼───┤ -│ ⋯ │ ⋯ │ -└───┴───┘ -┌─x─┬─y─┐ -│ ⋯ │ ⋯ │ -│ ⋯ │ ⋯ │ -└───┴───┘ - x y +1.  ⋯ ⋯ + ┏━━━┳━━━┓ + ┃ x ┃ y ┃ + ┡━━━╇━━━┩ +1. │ ⋯ │ ⋯ │ + ├───┼───┤ +2. │ ⋯ │ ⋯ │ + └───┴───┘ + ┌─x─┬─y─┐ +1. │ ⋯ │ ⋯ │ +2. │ ⋯ │ ⋯ │ + └───┴───┘ + x y - ⋯ ⋯ - ⋯ ⋯ -┏━━━┳━━━┓ -┃ x ┃ y ┃ -┡━━━╇━━━┩ -│ ⋯ │ ⋯ │ -└───┴───┘ +1.  ⋯ ⋯ +2.  ⋯ ⋯ + ┏━━━┳━━━┓ + ┃ x ┃ y ┃ + ┡━━━╇━━━┩ +1. │ ⋯ │ ⋯ │ + └───┴───┘ diff --git a/tests/queries/0_stateless/01351_parse_date_time_best_effort_us.reference b/tests/queries/0_stateless/01351_parse_date_time_best_effort_us.reference index 23bcb901557..62b3a40944a 100644 --- a/tests/queries/0_stateless/01351_parse_date_time_best_effort_us.reference +++ b/tests/queries/0_stateless/01351_parse_date_time_best_effort_us.reference @@ -1,40 +1,40 @@ parseDateTimeBestEffortUS - s a + s a - 1970/01/02 010203Z 1970-01-02 01:02:03 - 01-02-2001 UTC 2001-01-02 00:00:00 - 10.23.1990 1990-10-23 00:00:00 - 01-02-2017 03:04:05+1 2017-01-02 02:04:05 - 01/02/2017 03:04:05+300 2017-01-02 00:04:05 - 01.02.2017 03:04:05GMT 2017-01-02 03:04:05 - 01-02-2017 03:04:05 MSD 2017-01-01 23:04:05 - 01-02-2017 11:04:05 AM 2017-01-02 11:04:05 - 01-02-2017 11:04:05 PM 2017-01-02 23:04:05 - 01-02-2017 12:04:05 AM 2017-01-02 00:04:05 - 01-02-2017 12:04:05 PM 2017-01-02 12:04:05 - 01.02.17 03:04:05 MSD Feb 2017-02-01 23:04:05 - 01/02/2017 03:04:05 MSK 2017-01-02 00:04:05 - 12/13/2019 2019-12-13 00:00:00 - 13/12/2019 2019-12-13 00:00:00 - 03/04/2019 2019-03-04 00:00:00 + 1. 1970/01/02 010203Z 1970-01-02 01:02:03 + 2. 01-02-2001 UTC 2001-01-02 00:00:00 + 3. 10.23.1990 1990-10-23 00:00:00 + 4. 01-02-2017 03:04:05+1 2017-01-02 02:04:05 + 5. 01/02/2017 03:04:05+300 2017-01-02 00:04:05 + 6. 01.02.2017 03:04:05GMT 2017-01-02 03:04:05 + 7. 01-02-2017 03:04:05 MSD 2017-01-01 23:04:05 + 8. 01-02-2017 11:04:05 AM 2017-01-02 11:04:05 + 9. 01-02-2017 11:04:05 PM 2017-01-02 23:04:05 +10. 01-02-2017 12:04:05 AM 2017-01-02 00:04:05 +11. 01-02-2017 12:04:05 PM 2017-01-02 12:04:05 +12. 01.02.17 03:04:05 MSD Feb 2017-02-01 23:04:05 +13. 01/02/2017 03:04:05 MSK 2017-01-02 00:04:05 +14. 12/13/2019 2019-12-13 00:00:00 +15. 13/12/2019 2019-12-13 00:00:00 +16. 03/04/2019 2019-03-04 00:00:00 parseDateTimeBestEffortUSOrZero parseDateTimeBestEffortUSOrNull - s a b + s a b - 1970/01/02 010203Z 1970-01-02 01:02:03 1970-01-02 01:02:03 - 01-02-2001 UTC 2001-01-02 00:00:00 2001-01-02 00:00:00 - 10.23.1990 1990-10-23 00:00:00 1990-10-23 00:00:00 - 01-02-2017 03:04:05+1 2017-01-02 02:04:05 2017-01-02 02:04:05 - 01/02/2017 03:04:05+300 2017-01-02 00:04:05 2017-01-02 00:04:05 - 01.02.2017 03:04:05GMT 2017-01-02 03:04:05 2017-01-02 03:04:05 - 01-02-2017 03:04:05 MSD 2017-01-01 23:04:05 2017-01-01 23:04:05 - 01-02-2017 11:04:05 AM 2017-01-02 11:04:05 2017-01-02 11:04:05 - 01-02-2017 11:04:05 PM 2017-01-02 23:04:05 2017-01-02 23:04:05 - 01-02-2017 12:04:05 AM 2017-01-02 00:04:05 2017-01-02 00:04:05 - 01-02-2017 12:04:05 PM 2017-01-02 12:04:05 2017-01-02 12:04:05 - 01.02.17 03:04:05 MSD Feb 2017-02-01 23:04:05 2017-02-01 23:04:05 - 01/02/2017 03:04:05 MSK 2017-01-02 00:04:05 2017-01-02 00:04:05 - 12/13/2019 2019-12-13 00:00:00 2019-12-13 00:00:00 - 13/12/2019 2019-12-13 00:00:00 2019-12-13 00:00:00 - 03/04/2019 2019-03-04 00:00:00 2019-03-04 00:00:00 - 1970-01-01 00:00:00 ᴺᵁᴸᴸ - xyz 1970-01-01 00:00:00 ᴺᵁᴸᴸ + 1. 1970/01/02 010203Z 1970-01-02 01:02:03 1970-01-02 01:02:03 + 2. 01-02-2001 UTC 2001-01-02 00:00:00 2001-01-02 00:00:00 + 3. 10.23.1990 1990-10-23 00:00:00 1990-10-23 00:00:00 + 4. 01-02-2017 03:04:05+1 2017-01-02 02:04:05 2017-01-02 02:04:05 + 5. 01/02/2017 03:04:05+300 2017-01-02 00:04:05 2017-01-02 00:04:05 + 6. 01.02.2017 03:04:05GMT 2017-01-02 03:04:05 2017-01-02 03:04:05 + 7. 01-02-2017 03:04:05 MSD 2017-01-01 23:04:05 2017-01-01 23:04:05 + 8. 01-02-2017 11:04:05 AM 2017-01-02 11:04:05 2017-01-02 11:04:05 + 9. 01-02-2017 11:04:05 PM 2017-01-02 23:04:05 2017-01-02 23:04:05 +10. 01-02-2017 12:04:05 AM 2017-01-02 00:04:05 2017-01-02 00:04:05 +11. 01-02-2017 12:04:05 PM 2017-01-02 12:04:05 2017-01-02 12:04:05 +12. 01.02.17 03:04:05 MSD Feb 2017-02-01 23:04:05 2017-02-01 23:04:05 +13. 01/02/2017 03:04:05 MSK 2017-01-02 00:04:05 2017-01-02 00:04:05 +14. 12/13/2019 2019-12-13 00:00:00 2019-12-13 00:00:00 +15. 13/12/2019 2019-12-13 00:00:00 2019-12-13 00:00:00 +16. 03/04/2019 2019-03-04 00:00:00 2019-03-04 00:00:00 +17. 1970-01-01 00:00:00 ᴺᵁᴸᴸ +18. xyz 1970-01-01 00:00:00 ᴺᵁᴸᴸ diff --git a/tests/queries/0_stateless/01472_many_rows_in_totals.reference b/tests/queries/0_stateless/01472_many_rows_in_totals.reference index 92a9e4bbb53..6b0eaf3a6aa 100644 --- a/tests/queries/0_stateless/01472_many_rows_in_totals.reference +++ b/tests/queries/0_stateless/01472_many_rows_in_totals.reference @@ -1,21 +1,21 @@ -┏━━━┳━━━┓ -┃ g ┃ s ┃ -┡━━━╇━━━┩ -│ 0 │ 2 │ -├───┼───┤ -│ 0 │ 2 │ -├───┼───┤ -│ 1 │ 4 │ -├───┼───┤ -│ 1 │ 4 │ -└───┴───┘ + ┏━━━┳━━━┓ + ┃ g ┃ s ┃ + ┡━━━╇━━━┩ +1. │ 0 │ 2 │ + ├───┼───┤ +2. │ 0 │ 2 │ + ├───┼───┤ +3. │ 1 │ 4 │ + ├───┼───┤ +4. │ 1 │ 4 │ + └───┴───┘ Totals: -┏━━━┳━━━┓ -┃ g ┃ s ┃ -┡━━━╇━━━┩ -│ 0 │ 6 │ -└───┴───┘ + ┏━━━┳━━━┓ + ┃ g ┃ s ┃ + ┡━━━╇━━━┩ +1. │ 0 │ 6 │ + └───┴───┘ -- 0 2 0 2 diff --git a/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.reference b/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.reference index a270447ffc9..1dc350f0227 100644 --- a/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.reference +++ b/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.reference @@ -107,49 +107,49 @@ ┏━━━━━━━━┓ ┃ number ┃ ┡━━━━━━━━┩ - 1. │ 0 │ + 1. │ 0 │ ├────────┤ - 2. │ 1 │ + 2. │ 1 │ ├────────┤ - 3. │ 2 │ + 3. │ 2 │ ├────────┤ - 4. │ 3 │ + 4. │ 3 │ ├────────┤ - 5. │ 4 │ + 5. │ 4 │ ├────────┤ - 6. │ 5 │ + 6. │ 5 │ ├────────┤ - 7. │ 6 │ + 7. │ 6 │ ├────────┤ - 8. │ 7 │ + 8. │ 7 │ ├────────┤ - 9. │ 8 │ + 9. │ 8 │ ├────────┤ -10. │ 9 │ +10. │ 9 │ └────────┘ ┌─number─┐ - 1. │ 0 │ - 2. │ 1 │ - 3. │ 2 │ - 4. │ 3 │ - 5. │ 4 │ - 6. │ 5 │ - 7. │ 6 │ - 8. │ 7 │ - 9. │ 8 │ -10. │ 9 │ + 1. │ 0 │ + 2. │ 1 │ + 3. │ 2 │ + 4. │ 3 │ + 5. │ 4 │ + 6. │ 5 │ + 7. │ 6 │ + 8. │ 7 │ + 9. │ 8 │ +10. │ 9 │ └────────┘ ┌─number─┐ - 1. │ 0 │ - 2. │ 1 │ - 3. │ 2 │ - 4. │ 3 │ - 5. │ 4 │ - 6. │ 5 │ - 7. │ 6 │ - 8. │ 7 │ - 9. │ 8 │ -10. │ 9 │ + 1. │ 0 │ + 2. │ 1 │ + 3. │ 2 │ + 4. │ 3 │ + 5. │ 4 │ + 6. │ 5 │ + 7. │ 6 │ + 8. │ 7 │ + 9. │ 8 │ +10. │ 9 │ └────────┘ ┏━━━━━━━━┓ ┃ number ┃ @@ -200,35 +200,35 @@ 10. 9 number - 1. 0 - 2. 1 - 3. 2 - 4. 3 - 5. 4 - 6. 5 - 7. 6 - 8. 7 - 9. 8 -10. 9 + 1.  0 + 2.  1 + 3.  2 + 4.  3 + 5.  4 + 6.  5 + 7.  6 + 8.  7 + 9.  8 +10.  9 ┏━━━┓ ┃ a ┃ ┡━━━┩ -1. │ 1 │ +1. │ 1 │ └───┘ ┏━━━┓ ┃ a ┃ ┡━━━┩ -2. │ 2 │ +2. │ 2 │ └───┘ ┌─a─┐ -1. │ 1 │ +1. │ 1 │ └───┘ ┌─a─┐ -2. │ 2 │ +2. │ 2 │ └───┘ ┌─a─┐ -1. │ 1 │ -2. │ 2 │ +1. │ 1 │ +2. │ 2 │ └───┘ ┏━━━┓ ┃ a ┃ @@ -248,10 +248,10 @@ └───┘ a -1. 1 +1.  1 a -2. 2 +2.  2 a 1. 1 @@ -261,94 +261,94 @@ ┏━━━━━━━━┓ ┃ number ┃ ┡━━━━━━━━┩ -1. │ 0 │ +1. │ 0 │ └────────┘ ┏━━━━━━━━┓ ┃ number ┃ ┡━━━━━━━━┩ -2. │ 1 │ +2. │ 1 │ └────────┘ ┏━━━━━━━━┓ ┃ number ┃ ┡━━━━━━━━┩ -3. │ 2 │ +3. │ 2 │ └────────┘ ┏━━━━━━━━┓ ┃ number ┃ ┡━━━━━━━━┩ -4. │ 3 │ +4. │ 3 │ └────────┘ ┏━━━━━━━━┓ ┃ number ┃ ┡━━━━━━━━┩ -5. │ 4 │ +5. │ 4 │ └────────┘ ┏━━━━━━━━┓ ┃ number ┃ ┡━━━━━━━━┩ -6. │ 5 │ +6. │ 5 │ └────────┘ ┏━━━━━━━━┓ ┃ number ┃ ┡━━━━━━━━┩ -7. │ 6 │ +7. │ 6 │ └────────┘ ┏━━━━━━━━┓ ┃ number ┃ ┡━━━━━━━━┩ -8. │ 7 │ +8. │ 7 │ └────────┘ ┏━━━━━━━━┓ ┃ number ┃ ┡━━━━━━━━┩ -9. │ 8 │ +9. │ 8 │ └────────┘ ┏━━━━━━━━┓ ┃ number ┃ ┡━━━━━━━━┩ -10. │ 9 │ +10. │ 9 │ └────────┘ ┌─number─┐ -1. │ 0 │ +1. │ 0 │ └────────┘ ┌─number─┐ -2. │ 1 │ +2. │ 1 │ └────────┘ ┌─number─┐ -3. │ 2 │ +3. │ 2 │ └────────┘ ┌─number─┐ -4. │ 3 │ +4. │ 3 │ └────────┘ ┌─number─┐ -5. │ 4 │ +5. │ 4 │ └────────┘ ┌─number─┐ -6. │ 5 │ +6. │ 5 │ └────────┘ ┌─number─┐ -7. │ 6 │ +7. │ 6 │ └────────┘ ┌─number─┐ -8. │ 7 │ +8. │ 7 │ └────────┘ ┌─number─┐ -9. │ 8 │ +9. │ 8 │ └────────┘ ┌─number─┐ -10. │ 9 │ +10. │ 9 │ └────────┘ ┌─number─┐ - 1. │ 0 │ - 2. │ 1 │ - 3. │ 2 │ - 4. │ 3 │ - 5. │ 4 │ - 6. │ 5 │ - 7. │ 6 │ - 8. │ 7 │ - 9. │ 8 │ -10. │ 9 │ + 1. │ 0 │ + 2. │ 1 │ + 3. │ 2 │ + 4. │ 3 │ + 5. │ 4 │ + 6. │ 5 │ + 7. │ 6 │ + 8. │ 7 │ + 9. │ 8 │ +10. │ 9 │ └────────┘ ┏━━━━━━━━┓ ┃ number ┃ @@ -432,34 +432,34 @@ └────────┘ number -1. 0 +1.  0 number -2. 1 +2.  1 number -3. 2 +3.  2 number -4. 3 +4.  3 number -5. 4 +5.  4 number -6. 5 +6.  5 number -7. 6 +7.  6 number -8. 7 +8.  7 number -9. 8 +9.  8 number -10. 9 +10.  9 number 1. 0 diff --git a/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.sql b/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.sql index 3536b628ef2..0a09aef7fb2 100644 --- a/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.sql +++ b/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.sql @@ -1,4 +1,5 @@ SET output_format_pretty_color=1; +SET output_format_pretty_row_numbers=0; SELECT * FROM numbers(10) FORMAT Pretty; SELECT * FROM numbers(10) FORMAT PrettyCompact; SELECT * FROM numbers(10) FORMAT PrettyCompactMonoBlock; diff --git a/tests/queries/0_stateless/01553_settings_early_apply.reference b/tests/queries/0_stateless/01553_settings_early_apply.reference index 860d8d48c85..3306aaa734a 100644 --- a/tests/queries/0_stateless/01553_settings_early_apply.reference +++ b/tests/queries/0_stateless/01553_settings_early_apply.reference @@ -34,9 +34,9 @@ UInt64 0 - number + number - 0 +1. 0 0 1 2 diff --git a/tests/queries/0_stateless/01645_system_table_engines.reference b/tests/queries/0_stateless/01645_system_table_engines.reference index e19e6e035a6..e136700da74 100644 --- a/tests/queries/0_stateless/01645_system_table_engines.reference +++ b/tests/queries/0_stateless/01645_system_table_engines.reference @@ -1,4 +1,4 @@ -┌─name──────────────────────────┬─supports_settings─┬─supports_skipping_indices─┬─supports_projections─┬─supports_sort_order─┬─supports_ttl─┬─supports_replication─┬─supports_deduplication─┬─supports_parallel_insert─┐ -│ MergeTree │ 1 │ 1 │ 1 │ 1 │ 1 │ 0 │ 0 │ 1 │ -│ ReplicatedCollapsingMergeTree │ 1 │ 1 │ 1 │ 1 │ 1 │ 1 │ 1 │ 1 │ -└───────────────────────────────┴───────────────────┴───────────────────────────┴──────────────────────┴─────────────────────┴──────────────┴──────────────────────┴────────────────────────┴──────────────────────────┘ + ┌─name──────────────────────────┬─supports_settings─┬─supports_skipping_indices─┬─supports_projections─┬─supports_sort_order─┬─supports_ttl─┬─supports_replication─┬─supports_deduplication─┬─supports_parallel_insert─┐ +1. │ MergeTree │ 1 │ 1 │ 1 │ 1 │ 1 │ 0 │ 0 │ 1 │ +2. │ ReplicatedCollapsingMergeTree │ 1 │ 1 │ 1 │ 1 │ 1 │ 1 │ 1 │ 1 │ + └───────────────────────────────┴───────────────────┴───────────────────────────┴──────────────────────┴─────────────────────┴──────────────┴──────────────────────┴────────────────────────┴──────────────────────────┘ diff --git a/tests/queries/0_stateless/01656_ipv4_bad_formatting.reference b/tests/queries/0_stateless/01656_ipv4_bad_formatting.reference index a7b5c448f13..4cd84ff99fc 100644 --- a/tests/queries/0_stateless/01656_ipv4_bad_formatting.reference +++ b/tests/queries/0_stateless/01656_ipv4_bad_formatting.reference @@ -1,4 +1,4 @@ -┌─x───────────────┬─y───────────────┬──────────z─┐ -│ 1.1.1.1 │ 1.1.1.1 │ 16843009 │ -│ 255.255.255.255 │ 255.255.255.255 │ 4294967295 │ -└─────────────────┴─────────────────┴────────────┘ + ┌─x───────────────┬─y───────────────┬──────────z─┐ +1. │ 1.1.1.1 │ 1.1.1.1 │ 16843009 │ +2. │ 255.255.255.255 │ 255.255.255.255 │ 4294967295 │ + └─────────────────┴─────────────────┴────────────┘ diff --git a/tests/queries/0_stateless/01671_merge_join_and_constants.reference b/tests/queries/0_stateless/01671_merge_join_and_constants.reference index f82033ab3c7..1f904df6ae7 100644 --- a/tests/queries/0_stateless/01671_merge_join_and_constants.reference +++ b/tests/queries/0_stateless/01671_merge_join_and_constants.reference @@ -1,6 +1,6 @@ -┌─a─┬──────────b─┬─c─┬──────────d─┬─'0.10'─┬─c─┐ -│ a │ 2018-01-01 │ │ 1970-01-01 │ │ │ -│ b │ 2018-01-01 │ B │ 2018-01-01 │ 0.10 │ B │ -│ c │ 2018-01-01 │ C │ 2018-01-01 │ 0.10 │ C │ -└───┴────────────┴───┴────────────┴────────┴───┘ + ┌─a─┬──────────b─┬─c─┬──────────d─┬─'0.10'─┬─c─┐ +1. │ a │ 2018-01-01 │ │ 1970-01-01 │ │ │ +2. │ b │ 2018-01-01 │ B │ 2018-01-01 │ 0.10 │ B │ +3. │ c │ 2018-01-01 │ C │ 2018-01-01 │ 0.10 │ C │ + └───┴────────────┴───┴────────────┴────────┴───┘ \N \N \N \N 0 0 diff --git a/tests/queries/0_stateless/02026_describe_include_subcolumns.reference b/tests/queries/0_stateless/02026_describe_include_subcolumns.reference index ac114a03837..dec65f62748 100644 --- a/tests/queries/0_stateless/02026_describe_include_subcolumns.reference +++ b/tests/queries/0_stateless/02026_describe_include_subcolumns.reference @@ -1,33 +1,33 @@ -┌─name─┬─type──────────────────────────────────────────────────────────────────────┬─default_type─┬─default_expression─┬─comment─────────────────┬─codec_expression─┬─ttl_expression───────┐ -│ d │ Date │ │ │ │ │ │ -│ n │ Nullable(String) │ │ │ It is a nullable column │ │ │ -│ arr1 │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ -│ arr2 │ Array(Array(String)) │ │ │ │ │ d + toIntervalDay(1) │ -│ t │ Tuple( + ┌─name─┬─type──────────────────────────────────────────────────────────────────────┬─default_type─┬─default_expression─┬─comment─────────────────┬─codec_expression─┬─ttl_expression───────┐ +1. │ d │ Date │ │ │ │ │ │ +2. │ n │ Nullable(String) │ │ │ It is a nullable column │ │ │ +3. │ arr1 │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ +4. │ arr2 │ Array(Array(String)) │ │ │ │ │ d + toIntervalDay(1) │ +5. │ t │ Tuple( s String, a Array(Tuple( a UInt32, b UInt32))) │ │ │ │ ZSTD(1) │ │ -└──────┴───────────────────────────────────────────────────────────────────────────┴──────────────┴────────────────────┴─────────────────────────┴──────────────────┴──────────────────────┘ -┌─name───────┬─type──────────────────────────────────────────────────────────────────────┬─default_type─┬─default_expression─┬─comment─────────────────┬─codec_expression─┬─ttl_expression───────┬─is_subcolumn─┐ -│ d │ Date │ │ │ │ │ │ 0 │ -│ n │ Nullable(String) │ │ │ It is a nullable column │ │ │ 0 │ -│ arr1 │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ 0 │ -│ arr2 │ Array(Array(String)) │ │ │ │ │ d + toIntervalDay(1) │ 0 │ -│ t │ Tuple( + └──────┴───────────────────────────────────────────────────────────────────────────┴──────────────┴────────────────────┴─────────────────────────┴──────────────────┴──────────────────────┘ + ┌─name───────┬─type──────────────────────────────────────────────────────────────────────┬─default_type─┬─default_expression─┬─comment─────────────────┬─codec_expression─┬─ttl_expression───────┬─is_subcolumn─┐ + 1. │ d │ Date │ │ │ │ │ │ 0 │ + 2. │ n │ Nullable(String) │ │ │ It is a nullable column │ │ │ 0 │ + 3. │ arr1 │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ 0 │ + 4. │ arr2 │ Array(Array(String)) │ │ │ │ │ d + toIntervalDay(1) │ 0 │ + 5. │ t │ Tuple( s String, a Array(Tuple( a UInt32, b UInt32))) │ │ │ │ ZSTD(1) │ │ 0 │ -│ n.null │ UInt8 │ │ │ It is a nullable column │ │ │ 1 │ -│ arr1.size0 │ UInt64 │ │ │ │ │ │ 1 │ -│ arr2.size0 │ UInt64 │ │ │ │ │ d + toIntervalDay(1) │ 1 │ -│ arr2.size1 │ Array(UInt64) │ │ │ │ │ d + toIntervalDay(1) │ 1 │ -│ t.s │ String │ │ │ │ ZSTD(1) │ │ 1 │ -│ t.a │ Array(Tuple( + 6. │ n.null │ UInt8 │ │ │ It is a nullable column │ │ │ 1 │ + 7. │ arr1.size0 │ UInt64 │ │ │ │ │ │ 1 │ + 8. │ arr2.size0 │ UInt64 │ │ │ │ │ d + toIntervalDay(1) │ 1 │ + 9. │ arr2.size1 │ Array(UInt64) │ │ │ │ │ d + toIntervalDay(1) │ 1 │ +10. │ t.s │ String │ │ │ │ ZSTD(1) │ │ 1 │ +11. │ t.a │ Array(Tuple( a UInt32, b UInt32)) │ │ │ │ │ │ 1 │ -│ t.a.size0 │ UInt64 │ │ │ │ │ │ 1 │ -│ t.a.a │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ 1 │ -│ t.a.b │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ 1 │ -└────────────┴───────────────────────────────────────────────────────────────────────────┴──────────────┴────────────────────┴─────────────────────────┴──────────────────┴──────────────────────┴──────────────┘ +12. │ t.a.size0 │ UInt64 │ │ │ │ │ │ 1 │ +13. │ t.a.a │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ 1 │ +14. │ t.a.b │ Array(UInt32) │ │ │ │ ZSTD(1) │ │ 1 │ + └────────────┴───────────────────────────────────────────────────────────────────────────┴──────────────┴────────────────────┴─────────────────────────┴──────────────────┴──────────────────────┴──────────────┘ diff --git a/tests/queries/0_stateless/02152_bool_type.reference b/tests/queries/0_stateless/02152_bool_type.reference index a8c04f651e9..3d48302f941 100644 --- a/tests/queries/0_stateless/02152_bool_type.reference +++ b/tests/queries/0_stateless/02152_bool_type.reference @@ -32,11 +32,11 @@ Custom true Row 1: ────── CAST('true', 'Bool'): Custom true -┏━━━━━━━━━━━━━━━━━━━━━━┓ -┃ CAST('true', 'Bool') ┃ -┡━━━━━━━━━━━━━━━━━━━━━━┩ -│ Custom true │ -└──────────────────────┘ + ┏━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ CAST('true', 'Bool') ┃ + ┡━━━━━━━━━━━━━━━━━━━━━━┩ +1. │ Custom true │ + └──────────────────────┘ {"CAST('true', 'Bool')":true} 1 1 diff --git a/tests/queries/0_stateless/02375_pretty_formats.reference b/tests/queries/0_stateless/02375_pretty_formats.reference index 51b18f7eb09..e48646ff4bd 100644 --- a/tests/queries/0_stateless/02375_pretty_formats.reference +++ b/tests/queries/0_stateless/02375_pretty_formats.reference @@ -1,118 +1,118 @@ Pretty -┏━━━┳━━━┓ -┃ x ┃ y ┃ -┡━━━╇━━━┩ -│ 0 │ 1 │ -├───┼───┤ -│ 1 │ 2 │ -└───┴───┘ -┏━━━┳━━━┓ -┃ x ┃ y ┃ -┡━━━╇━━━┩ -│ 2 │ 3 │ -├───┼───┤ -│ 3 │ 4 │ -└───┴───┘ + ┏━━━┳━━━┓ + ┃ x ┃ y ┃ + ┡━━━╇━━━┩ +1. │ 0 │ 1 │ + ├───┼───┤ +2. │ 1 │ 2 │ + └───┴───┘ + ┏━━━┳━━━┓ + ┃ x ┃ y ┃ + ┡━━━╇━━━┩ +3. │ 2 │ 3 │ + ├───┼───┤ +4. │ 3 │ 4 │ + └───┴───┘ PrettyNoEscapes -┏━━━┳━━━┓ -┃ x ┃ y ┃ -┡━━━╇━━━┩ -│ 0 │ 1 │ -├───┼───┤ -│ 1 │ 2 │ -└───┴───┘ -┏━━━┳━━━┓ -┃ x ┃ y ┃ -┡━━━╇━━━┩ -│ 2 │ 3 │ -├───┼───┤ -│ 3 │ 4 │ -└───┴───┘ + ┏━━━┳━━━┓ + ┃ x ┃ y ┃ + ┡━━━╇━━━┩ +1. │ 0 │ 1 │ + ├───┼───┤ +2. │ 1 │ 2 │ + └───┴───┘ + ┏━━━┳━━━┓ + ┃ x ┃ y ┃ + ┡━━━╇━━━┩ +3. │ 2 │ 3 │ + ├───┼───┤ +4. │ 3 │ 4 │ + └───┴───┘ PrettyMonoBlock -┏━━━┳━━━┓ -┃ x ┃ y ┃ -┡━━━╇━━━┩ -│ 0 │ 1 │ -├───┼───┤ -│ 1 │ 2 │ -├───┼───┤ -│ 2 │ 3 │ -├───┼───┤ -│ 3 │ 4 │ -└───┴───┘ + ┏━━━┳━━━┓ + ┃ x ┃ y ┃ + ┡━━━╇━━━┩ +1. │ 0 │ 1 │ + ├───┼───┤ +2. │ 1 │ 2 │ + ├───┼───┤ +3. │ 2 │ 3 │ + ├───┼───┤ +4. │ 3 │ 4 │ + └───┴───┘ PrettyNoEscapesMonoBlock -┏━━━┳━━━┓ -┃ x ┃ y ┃ -┡━━━╇━━━┩ -│ 0 │ 1 │ -├───┼───┤ -│ 1 │ 2 │ -├───┼───┤ -│ 2 │ 3 │ -├───┼───┤ -│ 3 │ 4 │ -└───┴───┘ + ┏━━━┳━━━┓ + ┃ x ┃ y ┃ + ┡━━━╇━━━┩ +1. │ 0 │ 1 │ + ├───┼───┤ +2. │ 1 │ 2 │ + ├───┼───┤ +3. │ 2 │ 3 │ + ├───┼───┤ +4. │ 3 │ 4 │ + └───┴───┘ PrettyCompact -┌─x─┬─y─┐ -│ 0 │ 1 │ -│ 1 │ 2 │ -└───┴───┘ -┌─x─┬─y─┐ -│ 2 │ 3 │ -│ 3 │ 4 │ -└───┴───┘ + ┌─x─┬─y─┐ +1. │ 0 │ 1 │ +2. │ 1 │ 2 │ + └───┴───┘ + ┌─x─┬─y─┐ +3. │ 2 │ 3 │ +4. │ 3 │ 4 │ + └───┴───┘ PrettyCompactNoEscapes -┌─x─┬─y─┐ -│ 0 │ 1 │ -│ 1 │ 2 │ -└───┴───┘ -┌─x─┬─y─┐ -│ 2 │ 3 │ -│ 3 │ 4 │ -└───┴───┘ + ┌─x─┬─y─┐ +1. │ 0 │ 1 │ +2. │ 1 │ 2 │ + └───┴───┘ + ┌─x─┬─y─┐ +3. │ 2 │ 3 │ +4. │ 3 │ 4 │ + └───┴───┘ PrettyCompactMonoBlock -┌─x─┬─y─┐ -│ 0 │ 1 │ -│ 1 │ 2 │ -│ 2 │ 3 │ -│ 3 │ 4 │ -└───┴───┘ + ┌─x─┬─y─┐ +1. │ 0 │ 1 │ +2. │ 1 │ 2 │ +3. │ 2 │ 3 │ +4. │ 3 │ 4 │ + └───┴───┘ PrettyCompactNoEscapesMonoBlock -┌─x─┬─y─┐ -│ 0 │ 1 │ -│ 1 │ 2 │ -│ 2 │ 3 │ -│ 3 │ 4 │ -└───┴───┘ + ┌─x─┬─y─┐ +1. │ 0 │ 1 │ +2. │ 1 │ 2 │ +3. │ 2 │ 3 │ +4. │ 3 │ 4 │ + └───┴───┘ PrettySpace - x y + x y - 0 1 - 1 2 - x y +1.  0 1 +2.  1 2 + x y - 2 3 - 3 4 +3.  2 3 +4.  3 4 PrettySpaceNoEscapes - x y + x y - 0 1 - 1 2 - x y +1. 0 1 +2. 1 2 + x y - 2 3 - 3 4 +3. 2 3 +4. 3 4 PrettySpaceMonoBlock - x y + x y - 0 1 - 1 2 - 2 3 - 3 4 +1.  0 1 +2.  1 2 +3.  2 3 +4.  3 4 PrettySpaceNoEscapesMonoBlock - x y + x y - 0 1 - 1 2 - 2 3 - 3 4 +1. 0 1 +2. 1 2 +3. 2 3 +4. 3 4 diff --git a/tests/queries/0_stateless/02381_parseDateTime64BestEffortUS.reference b/tests/queries/0_stateless/02381_parseDateTime64BestEffortUS.reference index c9a13c97baf..469c523c578 100644 --- a/tests/queries/0_stateless/02381_parseDateTime64BestEffortUS.reference +++ b/tests/queries/0_stateless/02381_parseDateTime64BestEffortUS.reference @@ -1,10 +1,10 @@ parseDateTime64BestEffortUS - s a + s a - 01-02-1930 12:00:00 1930-01-02 12:00:00.000 - 12.02.1930 12:00:00 1930-12-02 12:00:00.000 - 13/02/1930 12:00:00 1930-02-13 12:00:00.000 - 02/25/1930 12:00:00 1930-02-25 12:00:00.000 +1. 01-02-1930 12:00:00 1930-01-02 12:00:00.000 +2. 12.02.1930 12:00:00 1930-12-02 12:00:00.000 +3. 13/02/1930 12:00:00 1930-02-13 12:00:00.000 +4. 02/25/1930 12:00:00 1930-02-25 12:00:00.000 parseDateTime64BestEffortUSOrNull \N diff --git a/tests/queries/0_stateless/02418_do_not_return_empty_blocks_from_ConvertingAggregatedToChunksTransform.reference b/tests/queries/0_stateless/02418_do_not_return_empty_blocks_from_ConvertingAggregatedToChunksTransform.reference index f2586c9c42a..9a54cd0ebaa 100644 --- a/tests/queries/0_stateless/02418_do_not_return_empty_blocks_from_ConvertingAggregatedToChunksTransform.reference +++ b/tests/queries/0_stateless/02418_do_not_return_empty_blocks_from_ConvertingAggregatedToChunksTransform.reference @@ -1,6 +1,6 @@ -┌─number─┐ -│ 42 │ -└────────┘ -┌─number─┐ -│ 42 │ -└────────┘ + ┌─number─┐ +1. │ 42 │ + └────────┘ + ┌─number─┐ +1. │ 42 │ + └────────┘ diff --git a/tests/queries/0_stateless/02790_sql_standard_fetch.reference b/tests/queries/0_stateless/02790_sql_standard_fetch.reference index ef578b526da..6ae56306797 100644 --- a/tests/queries/0_stateless/02790_sql_standard_fetch.reference +++ b/tests/queries/0_stateless/02790_sql_standard_fetch.reference @@ -1,36 +1,36 @@ -┌─name───────────┬─department─┬─salary─┐ -│ Frank │ it │ 120 │ -│ Henry or Irene │ it │ 104 │ -│ Henry or Irene │ it │ 104 │ -│ Alice │ sales │ 100 │ -│ Dave or Cindy │ sales │ 96 │ -└────────────────┴────────────┴────────┘ -┌─name───────────┬─department─┬─salary─┐ -│ Frank │ it │ 120 │ -│ Henry or Irene │ it │ 104 │ -│ Henry or Irene │ it │ 104 │ -│ Alice │ sales │ 100 │ -│ Dave or Cindy │ sales │ 96 │ -└────────────────┴────────────┴────────┘ -┌─name───────────┬─department─┬─salary─┐ -│ Frank │ it │ 120 │ -│ Henry or Irene │ it │ 104 │ -│ Henry or Irene │ it │ 104 │ -│ Alice │ sales │ 100 │ -│ Dave or Cindy │ sales │ 96 │ -│ Dave or Cindy │ sales │ 96 │ -└────────────────┴────────────┴────────┘ -┌─name──────────┬─department─┬─salary─┐ -│ Alice │ sales │ 100 │ -│ Dave or Cindy │ sales │ 96 │ -│ Dave or Cindy │ sales │ 96 │ -│ Grace │ it │ 90 │ -│ Emma │ it │ 84 │ -└───────────────┴────────────┴────────┘ -┌─name──────────┬─department─┬─salary─┐ -│ Alice │ sales │ 100 │ -│ Dave or Cindy │ sales │ 96 │ -│ Dave or Cindy │ sales │ 96 │ -│ Grace │ it │ 90 │ -│ Emma │ it │ 84 │ -└───────────────┴────────────┴────────┘ + ┌─name───────────┬─department─┬─salary─┐ +1. │ Frank │ it │ 120 │ +2. │ Henry or Irene │ it │ 104 │ +3. │ Henry or Irene │ it │ 104 │ +4. │ Alice │ sales │ 100 │ +5. │ Dave or Cindy │ sales │ 96 │ + └────────────────┴────────────┴────────┘ + ┌─name───────────┬─department─┬─salary─┐ +1. │ Frank │ it │ 120 │ +2. │ Henry or Irene │ it │ 104 │ +3. │ Henry or Irene │ it │ 104 │ +4. │ Alice │ sales │ 100 │ +5. │ Dave or Cindy │ sales │ 96 │ + └────────────────┴────────────┴────────┘ + ┌─name───────────┬─department─┬─salary─┐ +1. │ Frank │ it │ 120 │ +2. │ Henry or Irene │ it │ 104 │ +3. │ Henry or Irene │ it │ 104 │ +4. │ Alice │ sales │ 100 │ +5. │ Dave or Cindy │ sales │ 96 │ +6. │ Dave or Cindy │ sales │ 96 │ + └────────────────┴────────────┴────────┘ + ┌─name──────────┬─department─┬─salary─┐ +1. │ Alice │ sales │ 100 │ +2. │ Dave or Cindy │ sales │ 96 │ +3. │ Dave or Cindy │ sales │ 96 │ +4. │ Grace │ it │ 90 │ +5. │ Emma │ it │ 84 │ + └───────────────┴────────────┴────────┘ + ┌─name──────────┬─department─┬─salary─┐ +1. │ Alice │ sales │ 100 │ +2. │ Dave or Cindy │ sales │ 96 │ +3. │ Dave or Cindy │ sales │ 96 │ +4. │ Grace │ it │ 90 │ +5. │ Emma │ it │ 84 │ + └───────────────┴────────────┴────────┘ diff --git a/tests/queries/0_stateless/02793_implicit_pretty_format_settings.expect b/tests/queries/0_stateless/02793_implicit_pretty_format_settings.expect index 569cbc7330e..f9d3a2709d8 100755 --- a/tests/queries/0_stateless/02793_implicit_pretty_format_settings.expect +++ b/tests/queries/0_stateless/02793_implicit_pretty_format_settings.expect @@ -19,7 +19,7 @@ spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \ expect ":) " # Send a command -send -- "SELECT 1 SETTINGS output_format_pretty_row_numbers = 1\r" +send -- "SELECT 1 SETTINGS output_format_pretty_row_numbers = 1, output_format_pretty_color = 0\r" expect "1. │ 1 │" expect ":) " diff --git a/tests/queries/0_stateless/02798_explain_settings_not_applied_bug.reference b/tests/queries/0_stateless/02798_explain_settings_not_applied_bug.reference index ffb61b54f9a..3ad865b7a74 100644 --- a/tests/queries/0_stateless/02798_explain_settings_not_applied_bug.reference +++ b/tests/queries/0_stateless/02798_explain_settings_not_applied_bug.reference @@ -1,11 +1,11 @@ - explain + explain - (Expression) - ExpressionTransform - (Aggregating) - FinalizeAggregatedTransform - AggregatingInOrderTransform - (Expression) - ExpressionTransform - (ReadFromMergeTree) - MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1 +1.  (Expression) +2.  ExpressionTransform +3.  (Aggregating) +4.  FinalizeAggregatedTransform +5.  AggregatingInOrderTransform +6.  (Expression) +7.  ExpressionTransform +8.  (ReadFromMergeTree) +9.  MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1 diff --git a/tests/queries/0_stateless/02890_describe_table_options.reference b/tests/queries/0_stateless/02890_describe_table_options.reference index 05daade09d8..15ff1dc0f15 100644 --- a/tests/queries/0_stateless/02890_describe_table_options.reference +++ b/tests/queries/0_stateless/02890_describe_table_options.reference @@ -2,197 +2,197 @@ SET describe_compact_output = 0, describe_include_virtual_columns = 0, describe_include_subcolumns = 0; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name─┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┐ -│ id │ UInt64 │ │ │ index column │ │ │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ -│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ -└──────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┘ + ┌─name─┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┐ +1. │ id │ UInt64 │ │ │ index column │ │ │ +2. │ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ +3. │ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ + └──────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┘ DESCRIBE remote(test_shard_localhost, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name─┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┐ -│ id │ UInt64 │ │ │ index column │ │ │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ -│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ -└──────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┘ + ┌─name─┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┐ +1. │ id │ UInt64 │ │ │ index column │ │ │ +2. │ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ +3. │ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ + └──────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┘ SET describe_compact_output = 0, describe_include_virtual_columns = 0, describe_include_subcolumns = 1; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name──────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┐ -│ id │ UInt64 │ │ │ index column │ │ │ 0 │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ -│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ -│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ -│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ -│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ -└───────────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┘ + ┌─name──────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┐ +1. │ id │ UInt64 │ │ │ index column │ │ │ 0 │ +2. │ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ +3. │ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ +4. │ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ +5. │ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ +6. │ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ + └───────────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┘ DESCRIBE remote(test_shard_localhost, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name──────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┐ -│ id │ UInt64 │ │ │ index column │ │ │ 0 │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ -│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ -│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ -│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ -│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ -└───────────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┘ + ┌─name──────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┐ +1. │ id │ UInt64 │ │ │ index column │ │ │ 0 │ +2. │ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ +3. │ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ +4. │ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ +5. │ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ +6. │ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ + └───────────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┘ SET describe_compact_output = 0, describe_include_virtual_columns = 1, describe_include_subcolumns = 0; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name───────────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment─────────────────────────────────────────────────────────────────────────────────┬─codec_expression─┬─ttl_expression─┬─is_virtual─┐ -│ id │ UInt64 │ │ │ index column │ │ │ 0 │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ -│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ -│ _part │ LowCardinality(String) │ │ │ Name of part │ │ │ 1 │ -│ _part_index │ UInt64 │ │ │ Sequential index of the part in the query result │ │ │ 1 │ -│ _part_uuid │ UUID │ │ │ Unique part identifier (if enabled MergeTree setting assign_part_uuids) │ │ │ 1 │ -│ _partition_id │ LowCardinality(String) │ │ │ Name of partition │ │ │ 1 │ -│ _sample_factor │ Float64 │ │ │ Sample factor (from the query) │ │ │ 1 │ -│ _part_offset │ UInt64 │ │ │ Number of row in the part │ │ │ 1 │ -│ _row_exists │ UInt8 │ │ │ Persisted mask created by lightweight delete that show whether row exists or is deleted │ │ │ 1 │ -│ _block_number │ UInt64 │ │ │ Persisted original number of block that was assigned at insert │ Delta, LZ4 │ │ 1 │ -└────────────────┴───────────────────────────┴──────────────┴────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────┴──────────────────┴────────────────┴────────────┘ + ┌─name───────────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment─────────────────────────────────────────────────────────────────────────────────┬─codec_expression─┬─ttl_expression─┬─is_virtual─┐ + 1. │ id │ UInt64 │ │ │ index column │ │ │ 0 │ + 2. │ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ + 3. │ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ + 4. │ _part │ LowCardinality(String) │ │ │ Name of part │ │ │ 1 │ + 5. │ _part_index │ UInt64 │ │ │ Sequential index of the part in the query result │ │ │ 1 │ + 6. │ _part_uuid │ UUID │ │ │ Unique part identifier (if enabled MergeTree setting assign_part_uuids) │ │ │ 1 │ + 7. │ _partition_id │ LowCardinality(String) │ │ │ Name of partition │ │ │ 1 │ + 8. │ _sample_factor │ Float64 │ │ │ Sample factor (from the query) │ │ │ 1 │ + 9. │ _part_offset │ UInt64 │ │ │ Number of row in the part │ │ │ 1 │ +10. │ _row_exists │ UInt8 │ │ │ Persisted mask created by lightweight delete that show whether row exists or is deleted │ │ │ 1 │ +11. │ _block_number │ UInt64 │ │ │ Persisted original number of block that was assigned at insert │ Delta, LZ4 │ │ 1 │ + └────────────────┴───────────────────────────┴──────────────┴────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────┴──────────────────┴────────────────┴────────────┘ DESCRIBE remote(test_shard_localhost, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name───────────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment─────────────────────────────────────────────────────────────────────────────────┬─codec_expression─┬─ttl_expression─┬─is_virtual─┐ -│ id │ UInt64 │ │ │ index column │ │ │ 0 │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ -│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ -│ _part │ LowCardinality(String) │ │ │ Name of part │ │ │ 1 │ -│ _part_index │ UInt64 │ │ │ Sequential index of the part in the query result │ │ │ 1 │ -│ _part_uuid │ UUID │ │ │ Unique part identifier (if enabled MergeTree setting assign_part_uuids) │ │ │ 1 │ -│ _partition_id │ LowCardinality(String) │ │ │ Name of partition │ │ │ 1 │ -│ _sample_factor │ Float64 │ │ │ Sample factor (from the query) │ │ │ 1 │ -│ _part_offset │ UInt64 │ │ │ Number of row in the part │ │ │ 1 │ -│ _row_exists │ UInt8 │ │ │ Persisted mask created by lightweight delete that show whether row exists or is deleted │ │ │ 1 │ -│ _block_number │ UInt64 │ │ │ Persisted original number of block that was assigned at insert │ Delta, LZ4 │ │ 1 │ -│ _shard_num │ UInt32 │ │ │ Deprecated. Use function shardNum instead │ │ │ 1 │ -└────────────────┴───────────────────────────┴──────────────┴────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────┴──────────────────┴────────────────┴────────────┘ + ┌─name───────────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment─────────────────────────────────────────────────────────────────────────────────┬─codec_expression─┬─ttl_expression─┬─is_virtual─┐ + 1. │ id │ UInt64 │ │ │ index column │ │ │ 0 │ + 2. │ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ + 3. │ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ + 4. │ _part │ LowCardinality(String) │ │ │ Name of part │ │ │ 1 │ + 5. │ _part_index │ UInt64 │ │ │ Sequential index of the part in the query result │ │ │ 1 │ + 6. │ _part_uuid │ UUID │ │ │ Unique part identifier (if enabled MergeTree setting assign_part_uuids) │ │ │ 1 │ + 7. │ _partition_id │ LowCardinality(String) │ │ │ Name of partition │ │ │ 1 │ + 8. │ _sample_factor │ Float64 │ │ │ Sample factor (from the query) │ │ │ 1 │ + 9. │ _part_offset │ UInt64 │ │ │ Number of row in the part │ │ │ 1 │ +10. │ _row_exists │ UInt8 │ │ │ Persisted mask created by lightweight delete that show whether row exists or is deleted │ │ │ 1 │ +11. │ _block_number │ UInt64 │ │ │ Persisted original number of block that was assigned at insert │ Delta, LZ4 │ │ 1 │ +12. │ _shard_num │ UInt32 │ │ │ Deprecated. Use function shardNum instead │ │ │ 1 │ + └────────────────┴───────────────────────────┴──────────────┴────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────┴──────────────────┴────────────────┴────────────┘ SET describe_compact_output = 0, describe_include_virtual_columns = 1, describe_include_subcolumns = 1; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name───────────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment─────────────────────────────────────────────────────────────────────────────────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┬─is_virtual─┐ -│ id │ UInt64 │ │ │ index column │ │ │ 0 │ 0 │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ 0 │ -│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ 0 │ -│ _part │ LowCardinality(String) │ │ │ Name of part │ │ │ 0 │ 1 │ -│ _part_index │ UInt64 │ │ │ Sequential index of the part in the query result │ │ │ 0 │ 1 │ -│ _part_uuid │ UUID │ │ │ Unique part identifier (if enabled MergeTree setting assign_part_uuids) │ │ │ 0 │ 1 │ -│ _partition_id │ LowCardinality(String) │ │ │ Name of partition │ │ │ 0 │ 1 │ -│ _sample_factor │ Float64 │ │ │ Sample factor (from the query) │ │ │ 0 │ 1 │ -│ _part_offset │ UInt64 │ │ │ Number of row in the part │ │ │ 0 │ 1 │ -│ _row_exists │ UInt8 │ │ │ Persisted mask created by lightweight delete that show whether row exists or is deleted │ │ │ 0 │ 1 │ -│ _block_number │ UInt64 │ │ │ Persisted original number of block that was assigned at insert │ Delta, LZ4 │ │ 0 │ 1 │ -│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ 0 │ -│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ -│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ -└────────────────┴───────────────────────────┴──────────────┴────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────┴──────────────────┴────────────────┴──────────────┴────────────┘ + ┌─name───────────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment─────────────────────────────────────────────────────────────────────────────────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┬─is_virtual─┐ + 1. │ id │ UInt64 │ │ │ index column │ │ │ 0 │ 0 │ + 2. │ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ 0 │ + 3. │ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ 0 │ + 4. │ _part │ LowCardinality(String) │ │ │ Name of part │ │ │ 0 │ 1 │ + 5. │ _part_index │ UInt64 │ │ │ Sequential index of the part in the query result │ │ │ 0 │ 1 │ + 6. │ _part_uuid │ UUID │ │ │ Unique part identifier (if enabled MergeTree setting assign_part_uuids) │ │ │ 0 │ 1 │ + 7. │ _partition_id │ LowCardinality(String) │ │ │ Name of partition │ │ │ 0 │ 1 │ + 8. │ _sample_factor │ Float64 │ │ │ Sample factor (from the query) │ │ │ 0 │ 1 │ + 9. │ _part_offset │ UInt64 │ │ │ Number of row in the part │ │ │ 0 │ 1 │ +10. │ _row_exists │ UInt8 │ │ │ Persisted mask created by lightweight delete that show whether row exists or is deleted │ │ │ 0 │ 1 │ +11. │ _block_number │ UInt64 │ │ │ Persisted original number of block that was assigned at insert │ Delta, LZ4 │ │ 0 │ 1 │ +12. │ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ 0 │ +13. │ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ +14. │ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ + └────────────────┴───────────────────────────┴──────────────┴────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────┴──────────────────┴────────────────┴──────────────┴────────────┘ DESCRIBE remote(test_shard_localhost, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name───────────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment─────────────────────────────────────────────────────────────────────────────────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┬─is_virtual─┐ -│ id │ UInt64 │ │ │ index column │ │ │ 0 │ 0 │ -│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ 0 │ -│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ 0 │ -│ _part │ LowCardinality(String) │ │ │ Name of part │ │ │ 0 │ 1 │ -│ _part_index │ UInt64 │ │ │ Sequential index of the part in the query result │ │ │ 0 │ 1 │ -│ _part_uuid │ UUID │ │ │ Unique part identifier (if enabled MergeTree setting assign_part_uuids) │ │ │ 0 │ 1 │ -│ _partition_id │ LowCardinality(String) │ │ │ Name of partition │ │ │ 0 │ 1 │ -│ _sample_factor │ Float64 │ │ │ Sample factor (from the query) │ │ │ 0 │ 1 │ -│ _part_offset │ UInt64 │ │ │ Number of row in the part │ │ │ 0 │ 1 │ -│ _row_exists │ UInt8 │ │ │ Persisted mask created by lightweight delete that show whether row exists or is deleted │ │ │ 0 │ 1 │ -│ _block_number │ UInt64 │ │ │ Persisted original number of block that was assigned at insert │ Delta, LZ4 │ │ 0 │ 1 │ -│ _shard_num │ UInt32 │ │ │ Deprecated. Use function shardNum instead │ │ │ 0 │ 1 │ -│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ 0 │ -│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ -│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ -└────────────────┴───────────────────────────┴──────────────┴────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────┴──────────────────┴────────────────┴──────────────┴────────────┘ + ┌─name───────────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment─────────────────────────────────────────────────────────────────────────────────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┬─is_virtual─┐ + 1. │ id │ UInt64 │ │ │ index column │ │ │ 0 │ 0 │ + 2. │ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ 0 │ + 3. │ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ 0 │ + 4. │ _part │ LowCardinality(String) │ │ │ Name of part │ │ │ 0 │ 1 │ + 5. │ _part_index │ UInt64 │ │ │ Sequential index of the part in the query result │ │ │ 0 │ 1 │ + 6. │ _part_uuid │ UUID │ │ │ Unique part identifier (if enabled MergeTree setting assign_part_uuids) │ │ │ 0 │ 1 │ + 7. │ _partition_id │ LowCardinality(String) │ │ │ Name of partition │ │ │ 0 │ 1 │ + 8. │ _sample_factor │ Float64 │ │ │ Sample factor (from the query) │ │ │ 0 │ 1 │ + 9. │ _part_offset │ UInt64 │ │ │ Number of row in the part │ │ │ 0 │ 1 │ +10. │ _row_exists │ UInt8 │ │ │ Persisted mask created by lightweight delete that show whether row exists or is deleted │ │ │ 0 │ 1 │ +11. │ _block_number │ UInt64 │ │ │ Persisted original number of block that was assigned at insert │ Delta, LZ4 │ │ 0 │ 1 │ +12. │ _shard_num │ UInt32 │ │ │ Deprecated. Use function shardNum instead │ │ │ 0 │ 1 │ +13. │ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ 0 │ +14. │ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ +15. │ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ 0 │ + └────────────────┴───────────────────────────┴──────────────┴────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────┴──────────────────┴────────────────┴──────────────┴────────────┘ SET describe_compact_output = 1, describe_include_virtual_columns = 0, describe_include_subcolumns = 0; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name─┬─type──────────────────────┐ -│ id │ UInt64 │ -│ arr │ Array(UInt64) │ -│ t │ Tuple(a String, b UInt64) │ -└──────┴───────────────────────────┘ + ┌─name─┬─type──────────────────────┐ +1. │ id │ UInt64 │ +2. │ arr │ Array(UInt64) │ +3. │ t │ Tuple(a String, b UInt64) │ + └──────┴───────────────────────────┘ DESCRIBE remote(test_shard_localhost, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name─┬─type──────────────────────┐ -│ id │ UInt64 │ -│ arr │ Array(UInt64) │ -│ t │ Tuple(a String, b UInt64) │ -└──────┴───────────────────────────┘ + ┌─name─┬─type──────────────────────┐ +1. │ id │ UInt64 │ +2. │ arr │ Array(UInt64) │ +3. │ t │ Tuple(a String, b UInt64) │ + └──────┴───────────────────────────┘ SET describe_compact_output = 1, describe_include_virtual_columns = 0, describe_include_subcolumns = 1; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name──────┬─type──────────────────────┬─is_subcolumn─┐ -│ id │ UInt64 │ 0 │ -│ arr │ Array(UInt64) │ 0 │ -│ t │ Tuple(a String, b UInt64) │ 0 │ -│ arr.size0 │ UInt64 │ 1 │ -│ t.a │ String │ 1 │ -│ t.b │ UInt64 │ 1 │ -└───────────┴───────────────────────────┴──────────────┘ + ┌─name──────┬─type──────────────────────┬─is_subcolumn─┐ +1. │ id │ UInt64 │ 0 │ +2. │ arr │ Array(UInt64) │ 0 │ +3. │ t │ Tuple(a String, b UInt64) │ 0 │ +4. │ arr.size0 │ UInt64 │ 1 │ +5. │ t.a │ String │ 1 │ +6. │ t.b │ UInt64 │ 1 │ + └───────────┴───────────────────────────┴──────────────┘ DESCRIBE remote(test_shard_localhost, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name──────┬─type──────────────────────┬─is_subcolumn─┐ -│ id │ UInt64 │ 0 │ -│ arr │ Array(UInt64) │ 0 │ -│ t │ Tuple(a String, b UInt64) │ 0 │ -│ arr.size0 │ UInt64 │ 1 │ -│ t.a │ String │ 1 │ -│ t.b │ UInt64 │ 1 │ -└───────────┴───────────────────────────┴──────────────┘ + ┌─name──────┬─type──────────────────────┬─is_subcolumn─┐ +1. │ id │ UInt64 │ 0 │ +2. │ arr │ Array(UInt64) │ 0 │ +3. │ t │ Tuple(a String, b UInt64) │ 0 │ +4. │ arr.size0 │ UInt64 │ 1 │ +5. │ t.a │ String │ 1 │ +6. │ t.b │ UInt64 │ 1 │ + └───────────┴───────────────────────────┴──────────────┘ SET describe_compact_output = 1, describe_include_virtual_columns = 1, describe_include_subcolumns = 0; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name───────────┬─type──────────────────────┬─is_virtual─┐ -│ id │ UInt64 │ 0 │ -│ arr │ Array(UInt64) │ 0 │ -│ t │ Tuple(a String, b UInt64) │ 0 │ -│ _part │ LowCardinality(String) │ 1 │ -│ _part_index │ UInt64 │ 1 │ -│ _part_uuid │ UUID │ 1 │ -│ _partition_id │ LowCardinality(String) │ 1 │ -│ _sample_factor │ Float64 │ 1 │ -│ _part_offset │ UInt64 │ 1 │ -│ _row_exists │ UInt8 │ 1 │ -│ _block_number │ UInt64 │ 1 │ -└────────────────┴───────────────────────────┴────────────┘ + ┌─name───────────┬─type──────────────────────┬─is_virtual─┐ + 1. │ id │ UInt64 │ 0 │ + 2. │ arr │ Array(UInt64) │ 0 │ + 3. │ t │ Tuple(a String, b UInt64) │ 0 │ + 4. │ _part │ LowCardinality(String) │ 1 │ + 5. │ _part_index │ UInt64 │ 1 │ + 6. │ _part_uuid │ UUID │ 1 │ + 7. │ _partition_id │ LowCardinality(String) │ 1 │ + 8. │ _sample_factor │ Float64 │ 1 │ + 9. │ _part_offset │ UInt64 │ 1 │ +10. │ _row_exists │ UInt8 │ 1 │ +11. │ _block_number │ UInt64 │ 1 │ + └────────────────┴───────────────────────────┴────────────┘ DESCRIBE remote(test_shard_localhost, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name───────────┬─type──────────────────────┬─is_virtual─┐ -│ id │ UInt64 │ 0 │ -│ arr │ Array(UInt64) │ 0 │ -│ t │ Tuple(a String, b UInt64) │ 0 │ -│ _part │ LowCardinality(String) │ 1 │ -│ _part_index │ UInt64 │ 1 │ -│ _part_uuid │ UUID │ 1 │ -│ _partition_id │ LowCardinality(String) │ 1 │ -│ _sample_factor │ Float64 │ 1 │ -│ _part_offset │ UInt64 │ 1 │ -│ _row_exists │ UInt8 │ 1 │ -│ _block_number │ UInt64 │ 1 │ -│ _shard_num │ UInt32 │ 1 │ -└────────────────┴───────────────────────────┴────────────┘ + ┌─name───────────┬─type──────────────────────┬─is_virtual─┐ + 1. │ id │ UInt64 │ 0 │ + 2. │ arr │ Array(UInt64) │ 0 │ + 3. │ t │ Tuple(a String, b UInt64) │ 0 │ + 4. │ _part │ LowCardinality(String) │ 1 │ + 5. │ _part_index │ UInt64 │ 1 │ + 6. │ _part_uuid │ UUID │ 1 │ + 7. │ _partition_id │ LowCardinality(String) │ 1 │ + 8. │ _sample_factor │ Float64 │ 1 │ + 9. │ _part_offset │ UInt64 │ 1 │ +10. │ _row_exists │ UInt8 │ 1 │ +11. │ _block_number │ UInt64 │ 1 │ +12. │ _shard_num │ UInt32 │ 1 │ + └────────────────┴───────────────────────────┴────────────┘ SET describe_compact_output = 1, describe_include_virtual_columns = 1, describe_include_subcolumns = 1; DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes; -┌─name───────────┬─type──────────────────────┬─is_subcolumn─┬─is_virtual─┐ -│ id │ UInt64 │ 0 │ 0 │ -│ arr │ Array(UInt64) │ 0 │ 0 │ -│ t │ Tuple(a String, b UInt64) │ 0 │ 0 │ -│ _part │ LowCardinality(String) │ 0 │ 1 │ -│ _part_index │ UInt64 │ 0 │ 1 │ -│ _part_uuid │ UUID │ 0 │ 1 │ -│ _partition_id │ LowCardinality(String) │ 0 │ 1 │ -│ _sample_factor │ Float64 │ 0 │ 1 │ -│ _part_offset │ UInt64 │ 0 │ 1 │ -│ _row_exists │ UInt8 │ 0 │ 1 │ -│ _block_number │ UInt64 │ 0 │ 1 │ -│ arr.size0 │ UInt64 │ 1 │ 0 │ -│ t.a │ String │ 1 │ 0 │ -│ t.b │ UInt64 │ 1 │ 0 │ -└────────────────┴───────────────────────────┴──────────────┴────────────┘ + ┌─name───────────┬─type──────────────────────┬─is_subcolumn─┬─is_virtual─┐ + 1. │ id │ UInt64 │ 0 │ 0 │ + 2. │ arr │ Array(UInt64) │ 0 │ 0 │ + 3. │ t │ Tuple(a String, b UInt64) │ 0 │ 0 │ + 4. │ _part │ LowCardinality(String) │ 0 │ 1 │ + 5. │ _part_index │ UInt64 │ 0 │ 1 │ + 6. │ _part_uuid │ UUID │ 0 │ 1 │ + 7. │ _partition_id │ LowCardinality(String) │ 0 │ 1 │ + 8. │ _sample_factor │ Float64 │ 0 │ 1 │ + 9. │ _part_offset │ UInt64 │ 0 │ 1 │ +10. │ _row_exists │ UInt8 │ 0 │ 1 │ +11. │ _block_number │ UInt64 │ 0 │ 1 │ +12. │ arr.size0 │ UInt64 │ 1 │ 0 │ +13. │ t.a │ String │ 1 │ 0 │ +14. │ t.b │ UInt64 │ 1 │ 0 │ + └────────────────┴───────────────────────────┴──────────────┴────────────┘ DESCRIBE remote(test_shard_localhost, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes; -┌─name───────────┬─type──────────────────────┬─is_subcolumn─┬─is_virtual─┐ -│ id │ UInt64 │ 0 │ 0 │ -│ arr │ Array(UInt64) │ 0 │ 0 │ -│ t │ Tuple(a String, b UInt64) │ 0 │ 0 │ -│ _part │ LowCardinality(String) │ 0 │ 1 │ -│ _part_index │ UInt64 │ 0 │ 1 │ -│ _part_uuid │ UUID │ 0 │ 1 │ -│ _partition_id │ LowCardinality(String) │ 0 │ 1 │ -│ _sample_factor │ Float64 │ 0 │ 1 │ -│ _part_offset │ UInt64 │ 0 │ 1 │ -│ _row_exists │ UInt8 │ 0 │ 1 │ -│ _block_number │ UInt64 │ 0 │ 1 │ -│ _shard_num │ UInt32 │ 0 │ 1 │ -│ arr.size0 │ UInt64 │ 1 │ 0 │ -│ t.a │ String │ 1 │ 0 │ -│ t.b │ UInt64 │ 1 │ 0 │ -└────────────────┴───────────────────────────┴──────────────┴────────────┘ + ┌─name───────────┬─type──────────────────────┬─is_subcolumn─┬─is_virtual─┐ + 1. │ id │ UInt64 │ 0 │ 0 │ + 2. │ arr │ Array(UInt64) │ 0 │ 0 │ + 3. │ t │ Tuple(a String, b UInt64) │ 0 │ 0 │ + 4. │ _part │ LowCardinality(String) │ 0 │ 1 │ + 5. │ _part_index │ UInt64 │ 0 │ 1 │ + 6. │ _part_uuid │ UUID │ 0 │ 1 │ + 7. │ _partition_id │ LowCardinality(String) │ 0 │ 1 │ + 8. │ _sample_factor │ Float64 │ 0 │ 1 │ + 9. │ _part_offset │ UInt64 │ 0 │ 1 │ +10. │ _row_exists │ UInt8 │ 0 │ 1 │ +11. │ _block_number │ UInt64 │ 0 │ 1 │ +12. │ _shard_num │ UInt32 │ 0 │ 1 │ +13. │ arr.size0 │ UInt64 │ 1 │ 0 │ +14. │ t.a │ String │ 1 │ 0 │ +15. │ t.b │ UInt64 │ 1 │ 0 │ + └────────────────┴───────────────────────────┴──────────────┴────────────┘ diff --git a/tests/queries/0_stateless/02896_union_distinct_http_format.reference b/tests/queries/0_stateless/02896_union_distinct_http_format.reference index 3a68ab6dc9f..406c89021c1 100644 --- a/tests/queries/0_stateless/02896_union_distinct_http_format.reference +++ b/tests/queries/0_stateless/02896_union_distinct_http_format.reference @@ -1,7 +1,7 @@ -┌─1─┐ -│ 1 │ -└───┘ -┌─a─┐ -│ 1 │ -│ 2 │ -└───┘ + ┌─1─┐ +1. │ 1 │ + └───┘ + ┌─a─┐ +1. │ 1 │ +2. │ 2 │ + └───┘ diff --git a/tests/queries/0_stateless/02947_merge_tree_index_table_1.reference b/tests/queries/0_stateless/02947_merge_tree_index_table_1.reference index 0ce9de91293..d4397328dbc 100644 --- a/tests/queries/0_stateless/02947_merge_tree_index_table_1.reference +++ b/tests/queries/0_stateless/02947_merge_tree_index_table_1.reference @@ -18,30 +18,30 @@ 3 18 v324 4 14 v196 4 19 v361 -┌─part_name─┬─mark_number─┬─rows_in_granule─┬─a─┬──b─┐ -│ all_1_1_0 │ 0 │ 3 │ 0 │ 0 │ -│ all_1_1_0 │ 1 │ 3 │ 1 │ 6 │ -│ all_1_1_0 │ 2 │ 3 │ 3 │ 3 │ -│ all_1_1_0 │ 3 │ 1 │ 4 │ 9 │ -│ all_1_1_0 │ 4 │ 0 │ 4 │ 9 │ -│ all_2_2_0 │ 0 │ 3 │ 0 │ 10 │ -│ all_2_2_0 │ 1 │ 3 │ 1 │ 16 │ -│ all_2_2_0 │ 2 │ 3 │ 3 │ 13 │ -│ all_2_2_0 │ 3 │ 1 │ 4 │ 19 │ -│ all_2_2_0 │ 4 │ 0 │ 4 │ 19 │ -└───────────┴─────────────┴─────────────────┴───┴────┘ -┌─part_name─┬─mark_number─┬─rows_in_granule─┬─a─┬──b─┬─a.mark─┬─b.mark─┬─s.mark─┐ -│ all_1_1_0 │ 0 │ 3 │ 0 │ 0 │ (0,0) │ (0,0) │ (0,0) │ -│ all_1_1_0 │ 1 │ 3 │ 1 │ 6 │ (0,24) │ (0,24) │ (0,10) │ -│ all_1_1_0 │ 2 │ 3 │ 3 │ 3 │ (0,48) │ (0,48) │ (0,21) │ -│ all_1_1_0 │ 3 │ 1 │ 4 │ 9 │ (0,72) │ (0,72) │ (0,32) │ -│ all_1_1_0 │ 4 │ 0 │ 4 │ 9 │ (0,80) │ (0,80) │ (0,36) │ -│ all_2_2_0 │ 0 │ 3 │ 0 │ 10 │ (0,0) │ (0,0) │ (0,0) │ -│ all_2_2_0 │ 1 │ 3 │ 1 │ 16 │ (0,24) │ (0,24) │ (0,15) │ -│ all_2_2_0 │ 2 │ 3 │ 3 │ 13 │ (0,48) │ (0,48) │ (0,30) │ -│ all_2_2_0 │ 3 │ 1 │ 4 │ 19 │ (0,72) │ (0,72) │ (0,45) │ -│ all_2_2_0 │ 4 │ 0 │ 4 │ 19 │ (0,80) │ (0,80) │ (0,50) │ -└───────────┴─────────────┴─────────────────┴───┴────┴────────┴────────┴────────┘ + ┌─part_name─┬─mark_number─┬─rows_in_granule─┬─a─┬──b─┐ + 1. │ all_1_1_0 │ 0 │ 3 │ 0 │ 0 │ + 2. │ all_1_1_0 │ 1 │ 3 │ 1 │ 6 │ + 3. │ all_1_1_0 │ 2 │ 3 │ 3 │ 3 │ + 4. │ all_1_1_0 │ 3 │ 1 │ 4 │ 9 │ + 5. │ all_1_1_0 │ 4 │ 0 │ 4 │ 9 │ + 6. │ all_2_2_0 │ 0 │ 3 │ 0 │ 10 │ + 7. │ all_2_2_0 │ 1 │ 3 │ 1 │ 16 │ + 8. │ all_2_2_0 │ 2 │ 3 │ 3 │ 13 │ + 9. │ all_2_2_0 │ 3 │ 1 │ 4 │ 19 │ +10. │ all_2_2_0 │ 4 │ 0 │ 4 │ 19 │ + └───────────┴─────────────┴─────────────────┴───┴────┘ + ┌─part_name─┬─mark_number─┬─rows_in_granule─┬─a─┬──b─┬─a.mark─┬─b.mark─┬─s.mark─┐ + 1. │ all_1_1_0 │ 0 │ 3 │ 0 │ 0 │ (0,0) │ (0,0) │ (0,0) │ + 2. │ all_1_1_0 │ 1 │ 3 │ 1 │ 6 │ (0,24) │ (0,24) │ (0,10) │ + 3. │ all_1_1_0 │ 2 │ 3 │ 3 │ 3 │ (0,48) │ (0,48) │ (0,21) │ + 4. │ all_1_1_0 │ 3 │ 1 │ 4 │ 9 │ (0,72) │ (0,72) │ (0,32) │ + 5. │ all_1_1_0 │ 4 │ 0 │ 4 │ 9 │ (0,80) │ (0,80) │ (0,36) │ + 6. │ all_2_2_0 │ 0 │ 3 │ 0 │ 10 │ (0,0) │ (0,0) │ (0,0) │ + 7. │ all_2_2_0 │ 1 │ 3 │ 1 │ 16 │ (0,24) │ (0,24) │ (0,15) │ + 8. │ all_2_2_0 │ 2 │ 3 │ 3 │ 13 │ (0,48) │ (0,48) │ (0,30) │ + 9. │ all_2_2_0 │ 3 │ 1 │ 4 │ 19 │ (0,72) │ (0,72) │ (0,45) │ +10. │ all_2_2_0 │ 4 │ 0 │ 4 │ 19 │ (0,80) │ (0,80) │ (0,50) │ + └───────────┴─────────────┴─────────────────┴───┴────┴────────┴────────┴────────┘ 0 0 v0 0 4 v16 0 8 v64 @@ -62,23 +62,23 @@ 3 11 v121 3 15 v225 3 19 v361 -┌─part_name─┬─mark_number─┬─rows_in_granule─┬─a─┬──b─┐ -│ all_1_1_0 │ 0 │ 3 │ 0 │ 0 │ -│ all_1_1_0 │ 1 │ 3 │ 1 │ 1 │ -│ all_1_1_0 │ 2 │ 4 │ 2 │ 2 │ -│ all_1_1_0 │ 3 │ 0 │ 3 │ 7 │ -│ all_2_2_0 │ 0 │ 3 │ 0 │ 12 │ -│ all_2_2_0 │ 1 │ 3 │ 1 │ 17 │ -│ all_2_2_0 │ 2 │ 4 │ 2 │ 18 │ -│ all_2_2_0 │ 3 │ 0 │ 3 │ 19 │ -└───────────┴─────────────┴─────────────────┴───┴────┘ -┌─part_name─┬─mark_number─┬─rows_in_granule─┬─a─┬──b─┬─a.mark──┬─b.mark──┬─s.mark──┐ -│ all_1_1_0 │ 0 │ 3 │ 0 │ 0 │ (0,0) │ (35,0) │ (77,0) │ -│ all_1_1_0 │ 1 │ 3 │ 1 │ 1 │ (114,0) │ (153,0) │ (197,0) │ -│ all_1_1_0 │ 2 │ 4 │ 2 │ 2 │ (234,0) │ (281,0) │ (329,0) │ -│ all_1_1_0 │ 3 │ 0 │ 3 │ 7 │ (369,0) │ (369,0) │ (369,0) │ -│ all_2_2_0 │ 0 │ 3 │ 0 │ 12 │ (0,0) │ (38,0) │ (82,0) │ -│ all_2_2_0 │ 1 │ 3 │ 1 │ 17 │ (124,0) │ (168,0) │ (212,0) │ -│ all_2_2_0 │ 2 │ 4 │ 2 │ 18 │ (254,0) │ (297,0) │ (345,0) │ -│ all_2_2_0 │ 3 │ 0 │ 3 │ 19 │ (392,0) │ (392,0) │ (392,0) │ -└───────────┴─────────────┴─────────────────┴───┴────┴─────────┴─────────┴─────────┘ + ┌─part_name─┬─mark_number─┬─rows_in_granule─┬─a─┬──b─┐ +1. │ all_1_1_0 │ 0 │ 3 │ 0 │ 0 │ +2. │ all_1_1_0 │ 1 │ 3 │ 1 │ 1 │ +3. │ all_1_1_0 │ 2 │ 4 │ 2 │ 2 │ +4. │ all_1_1_0 │ 3 │ 0 │ 3 │ 7 │ +5. │ all_2_2_0 │ 0 │ 3 │ 0 │ 12 │ +6. │ all_2_2_0 │ 1 │ 3 │ 1 │ 17 │ +7. │ all_2_2_0 │ 2 │ 4 │ 2 │ 18 │ +8. │ all_2_2_0 │ 3 │ 0 │ 3 │ 19 │ + └───────────┴─────────────┴─────────────────┴───┴────┘ + ┌─part_name─┬─mark_number─┬─rows_in_granule─┬─a─┬──b─┬─a.mark──┬─b.mark──┬─s.mark──┐ +1. │ all_1_1_0 │ 0 │ 3 │ 0 │ 0 │ (0,0) │ (42,0) │ (96,0) │ +2. │ all_1_1_0 │ 1 │ 3 │ 1 │ 1 │ (141,0) │ (195,0) │ (249,0) │ +3. │ all_1_1_0 │ 2 │ 4 │ 2 │ 2 │ (294,0) │ (349,0) │ (404,0) │ +4. │ all_1_1_0 │ 3 │ 0 │ 3 │ 7 │ (452,0) │ (452,0) │ (452,0) │ +5. │ all_2_2_0 │ 0 │ 3 │ 0 │ 12 │ (0,0) │ (50,0) │ (104,0) │ +6. │ all_2_2_0 │ 1 │ 3 │ 1 │ 17 │ (153,0) │ (207,0) │ (261,0) │ +7. │ all_2_2_0 │ 2 │ 4 │ 2 │ 18 │ (310,0) │ (365,0) │ (420,0) │ +8. │ all_2_2_0 │ 3 │ 0 │ 3 │ 19 │ (474,0) │ (474,0) │ (474,0) │ + └───────────┴─────────────┴─────────────────┴───┴────┴─────────┴─────────┴─────────┘ diff --git a/tests/queries/0_stateless/02947_merge_tree_index_table_2.reference b/tests/queries/0_stateless/02947_merge_tree_index_table_2.reference index 7bfcb7b2822..5328ff18d12 100644 --- a/tests/queries/0_stateless/02947_merge_tree_index_table_2.reference +++ b/tests/queries/0_stateless/02947_merge_tree_index_table_2.reference @@ -13,21 +13,21 @@ │ all_3_3_0 │ 3 │ 1 │ 4 │ 9 │ 96 │ │ all_3_3_0 │ 4 │ 0 │ 4 │ 9 │ 96 │ └───────────┴─────────────┴─────────────────┴───┴───┴────────────────────────────┘ -┌─part_name─┬─mark_number─┬─rows_in_granule─┬─a─┬─b─┬─modulo(sipHash64(sp), 100)─┬─a.mark──┬─b.mark──┬─c.mark──────┬─sp.sparse.idx.mark─┬─sp.mark─┬─arr.size0.mark─┬─arr.dict.mark─┬─arr.mark─┬─n.size0.mark─┬─n%2Ec1.mark─┬─n%2Ec2.mark─┬─t%2Ec2.mark─┬─t%2Ec1.mark─┬─t.mark──────┬─column%2Ewith%2Edots.mark─┐ -│ all_1_1_0 │ 0 │ 3 │ 0 │ 0 │ 19 │ (0,0) │ (0,0) │ (NULL,NULL) │ (0,0) │ (0,0) │ (0,0) │ (0,8) │ (0,0) │ (0,0) │ (0,0) │ (0,0) │ (0,0) │ (0,0) │ (NULL,NULL) │ (0,0) │ -│ all_1_1_0 │ 1 │ 3 │ 1 │ 6 │ 19 │ (0,24) │ (0,24) │ (NULL,NULL) │ (0,9) │ (0,0) │ (0,24) │ (0,8) │ (0,22) │ (0,24) │ (0,36) │ (0,72) │ (0,24) │ (0,24) │ (NULL,NULL) │ (0,24) │ -│ all_1_1_0 │ 2 │ 3 │ 3 │ 3 │ 19 │ (0,48) │ (0,48) │ (NULL,NULL) │ (0,18) │ (0,0) │ (0,48) │ (0,8) │ (0,44) │ (0,48) │ (0,72) │ (0,144) │ (0,48) │ (0,48) │ (NULL,NULL) │ (0,48) │ -│ all_1_1_0 │ 3 │ 1 │ 4 │ 9 │ 19 │ (0,72) │ (0,72) │ (NULL,NULL) │ (0,27) │ (0,0) │ (0,72) │ (0,8) │ (0,66) │ (0,72) │ (0,108) │ (0,216) │ (0,72) │ (0,72) │ (NULL,NULL) │ (0,72) │ -│ all_1_1_0 │ 4 │ 0 │ 4 │ 9 │ 19 │ (0,80) │ (0,80) │ (NULL,NULL) │ (0,36) │ (0,0) │ (0,80) │ (0,25) │ (0,84) │ (0,80) │ (0,120) │ (0,240) │ (0,80) │ (0,80) │ (NULL,NULL) │ (0,80) │ -│ all_2_2_0 │ 0 │ 3 │ 0 │ 0 │ 96 │ (0,0) │ (42,0) │ (84,0) │ (NULL,NULL) │ (126,0) │ (NULL,NULL) │ (NULL,NULL) │ (165,0) │ (NULL,NULL) │ (232,0) │ (286,0) │ (NULL,NULL) │ (NULL,NULL) │ (342,0) │ (391,0) │ -│ all_2_2_0 │ 1 │ 2 │ 3 │ 3 │ 96 │ (433,0) │ (472,0) │ (511,0) │ (NULL,NULL) │ (550,0) │ (NULL,NULL) │ (NULL,NULL) │ (589,0) │ (NULL,NULL) │ (659,0) │ (717,0) │ (NULL,NULL) │ (NULL,NULL) │ (773,0) │ (817,0) │ -│ all_2_2_0 │ 2 │ 0 │ 4 │ 4 │ 96 │ (856,0) │ (856,0) │ (856,0) │ (NULL,NULL) │ (856,0) │ (NULL,NULL) │ (NULL,NULL) │ (856,0) │ (NULL,NULL) │ (856,0) │ (856,0) │ (NULL,NULL) │ (NULL,NULL) │ (856,0) │ (856,0) │ -│ all_3_3_0 │ 0 │ 3 │ 0 │ 0 │ 96 │ (0,0) │ (0,0) │ (0,0) │ (NULL,NULL) │ (0,0) │ (0,0) │ (0,8) │ (0,0) │ (0,0) │ (0,0) │ (0,0) │ (0,0) │ (0,0) │ (NULL,NULL) │ (0,0) │ -│ all_3_3_0 │ 1 │ 3 │ 1 │ 6 │ 96 │ (0,24) │ (0,24) │ (0,24) │ (NULL,NULL) │ (0,24) │ (0,24) │ (0,8) │ (0,22) │ (0,24) │ (0,36) │ (0,72) │ (0,24) │ (0,24) │ (NULL,NULL) │ (0,24) │ -│ all_3_3_0 │ 2 │ 3 │ 3 │ 3 │ 96 │ (0,48) │ (0,48) │ (0,48) │ (NULL,NULL) │ (0,48) │ (0,48) │ (0,8) │ (0,44) │ (0,48) │ (0,72) │ (0,144) │ (0,48) │ (0,48) │ (NULL,NULL) │ (0,48) │ -│ all_3_3_0 │ 3 │ 1 │ 4 │ 9 │ 96 │ (0,72) │ (0,72) │ (0,72) │ (NULL,NULL) │ (0,72) │ (0,72) │ (0,8) │ (0,66) │ (0,72) │ (0,108) │ (0,216) │ (0,72) │ (0,72) │ (NULL,NULL) │ (0,72) │ -│ all_3_3_0 │ 4 │ 0 │ 4 │ 9 │ 96 │ (0,80) │ (0,80) │ (0,80) │ (NULL,NULL) │ (0,80) │ (0,80) │ (0,25) │ (0,84) │ (0,80) │ (0,120) │ (0,240) │ (0,80) │ (0,80) │ (NULL,NULL) │ (0,80) │ -└───────────┴─────────────┴─────────────────┴───┴───┴────────────────────────────┴─────────┴─────────┴─────────────┴────────────────────┴─────────┴────────────────┴───────────────┴──────────┴──────────────┴─────────────┴─────────────┴─────────────┴─────────────┴─────────────┴───────────────────────────┘ +┌─part_name─┬─mark_number─┬─rows_in_granule─┬─a─┬─b─┬─modulo(sipHash64(sp), 100)─┬─a.mark───┬─b.mark───┬─c.mark──────┬─sp.sparse.idx.mark─┬─sp.mark──┬─arr.size0.mark─┬─arr.dict.mark─┬─arr.mark─┬─n.size0.mark─┬─n%2Ec1.mark─┬─n%2Ec2.mark─┬─t%2Ec2.mark─┬─t%2Ec1.mark─┬─t.mark──────┬─column%2Ewith%2Edots.mark─┐ +│ all_1_1_0 │ 0 │ 3 │ 0 │ 0 │ 19 │ (0,0) │ (0,0) │ (NULL,NULL) │ (0,0) │ (0,0) │ (0,0) │ (0,8) │ (0,0) │ (0,0) │ (0,0) │ (0,0) │ (0,0) │ (0,0) │ (NULL,NULL) │ (0,0) │ +│ all_1_1_0 │ 1 │ 3 │ 1 │ 6 │ 19 │ (0,24) │ (0,24) │ (NULL,NULL) │ (0,9) │ (0,0) │ (0,24) │ (0,8) │ (0,22) │ (0,24) │ (0,36) │ (0,72) │ (0,24) │ (0,24) │ (NULL,NULL) │ (0,24) │ +│ all_1_1_0 │ 2 │ 3 │ 3 │ 3 │ 19 │ (0,48) │ (0,48) │ (NULL,NULL) │ (0,18) │ (0,0) │ (0,48) │ (0,8) │ (0,44) │ (0,48) │ (0,72) │ (0,144) │ (0,48) │ (0,48) │ (NULL,NULL) │ (0,48) │ +│ all_1_1_0 │ 3 │ 1 │ 4 │ 9 │ 19 │ (0,72) │ (0,72) │ (NULL,NULL) │ (0,27) │ (0,0) │ (0,72) │ (0,8) │ (0,66) │ (0,72) │ (0,108) │ (0,216) │ (0,72) │ (0,72) │ (NULL,NULL) │ (0,72) │ +│ all_1_1_0 │ 4 │ 0 │ 4 │ 9 │ 19 │ (0,80) │ (0,80) │ (NULL,NULL) │ (0,36) │ (0,0) │ (0,80) │ (0,25) │ (0,84) │ (0,80) │ (0,120) │ (0,240) │ (0,80) │ (0,80) │ (NULL,NULL) │ (0,80) │ +│ all_2_2_0 │ 0 │ 3 │ 0 │ 0 │ 96 │ (0,0) │ (54,0) │ (108,0) │ (NULL,NULL) │ (162,0) │ (NULL,NULL) │ (NULL,NULL) │ (216,0) │ (NULL,NULL) │ (291,0) │ (355,0) │ (NULL,NULL) │ (NULL,NULL) │ (426,0) │ (484,0) │ +│ all_2_2_0 │ 1 │ 2 │ 3 │ 3 │ 96 │ (538,0) │ (588,0) │ (638,0) │ (NULL,NULL) │ (688,0) │ (NULL,NULL) │ (NULL,NULL) │ (738,0) │ (NULL,NULL) │ (808,0) │ (868,0) │ (NULL,NULL) │ (NULL,NULL) │ (931,0) │ (986,0) │ +│ all_2_2_0 │ 2 │ 0 │ 4 │ 4 │ 96 │ (1036,0) │ (1036,0) │ (1036,0) │ (NULL,NULL) │ (1036,0) │ (NULL,NULL) │ (NULL,NULL) │ (1036,0) │ (NULL,NULL) │ (1036,0) │ (1036,0) │ (NULL,NULL) │ (NULL,NULL) │ (1036,0) │ (1036,0) │ +│ all_3_3_0 │ 0 │ 3 │ 0 │ 0 │ 96 │ (0,0) │ (0,0) │ (0,0) │ (NULL,NULL) │ (0,0) │ (0,0) │ (0,8) │ (0,0) │ (0,0) │ (0,0) │ (0,0) │ (0,0) │ (0,0) │ (NULL,NULL) │ (0,0) │ +│ all_3_3_0 │ 1 │ 3 │ 1 │ 6 │ 96 │ (0,24) │ (0,24) │ (0,24) │ (NULL,NULL) │ (0,24) │ (0,24) │ (0,8) │ (0,22) │ (0,24) │ (0,36) │ (0,72) │ (0,24) │ (0,24) │ (NULL,NULL) │ (0,24) │ +│ all_3_3_0 │ 2 │ 3 │ 3 │ 3 │ 96 │ (0,48) │ (0,48) │ (0,48) │ (NULL,NULL) │ (0,48) │ (0,48) │ (0,8) │ (0,44) │ (0,48) │ (0,72) │ (0,144) │ (0,48) │ (0,48) │ (NULL,NULL) │ (0,48) │ +│ all_3_3_0 │ 3 │ 1 │ 4 │ 9 │ 96 │ (0,72) │ (0,72) │ (0,72) │ (NULL,NULL) │ (0,72) │ (0,72) │ (0,8) │ (0,66) │ (0,72) │ (0,108) │ (0,216) │ (0,72) │ (0,72) │ (NULL,NULL) │ (0,72) │ +│ all_3_3_0 │ 4 │ 0 │ 4 │ 9 │ 96 │ (0,80) │ (0,80) │ (0,80) │ (NULL,NULL) │ (0,80) │ (0,80) │ (0,25) │ (0,84) │ (0,80) │ (0,120) │ (0,240) │ (0,80) │ (0,80) │ (NULL,NULL) │ (0,80) │ +└───────────┴─────────────┴─────────────────┴───┴───┴────────────────────────────┴──────────┴──────────┴─────────────┴────────────────────┴──────────┴────────────────┴───────────────┴──────────┴──────────────┴─────────────┴─────────────┴─────────────┴─────────────┴─────────────┴───────────────────────────┘ part_name String mark_number UInt64 rows_in_granule UInt64 diff --git a/tests/queries/0_stateless/02998_pretty_format_print_readable_number_on_single_value.reference b/tests/queries/0_stateless/02998_pretty_format_print_readable_number_on_single_value.reference index 496e0337209..357b534cf00 100644 --- a/tests/queries/0_stateless/02998_pretty_format_print_readable_number_on_single_value.reference +++ b/tests/queries/0_stateless/02998_pretty_format_print_readable_number_on_single_value.reference @@ -1,360 +1,360 @@ -┏━━━━━━━━━┓ -┃ a ┃ -┡━━━━━━━━━┩ -│ 1000000 │ -└─────────┘ -┏━━━━━━━━━┓ -┃ a ┃ -┡━━━━━━━━━┩ -│ 1000000 │ -└─────────┘ -┏━━━━━━━━━┓ -┃ a ┃ -┡━━━━━━━━━┩ -│ 1000000 │ -└─────────┘ -┏━━━━━━━━━┓ -┃ a ┃ -┡━━━━━━━━━┩ -│ 1000000 │ -└─────────┘ -┌───────a─┐ -│ 1000000 │ -└─────────┘ -┌───────a─┐ -│ 1000000 │ -└─────────┘ -┌───────a─┐ -│ 1000000 │ -└─────────┘ -┌───────a─┐ -│ 1000000 │ -└─────────┘ - a - - 1000000 - a - - 1000000 - a - - 1000000 - a - - 1000000 -┏━━━━━━━━━┓ -┃ a ┃ -┡━━━━━━━━━┩ -│ 1000000 │ -- 1.00 million -└─────────┘ -┏━━━━━━━━━┓ -┃ a ┃ -┡━━━━━━━━━┩ -│ 1000000 │ -- 1.00 million -└─────────┘ -┏━━━━━━━━━┓ -┃ a ┃ -┡━━━━━━━━━┩ -│ 1000000 │ -- 1.00 million -└─────────┘ -┏━━━━━━━━━┓ -┃ a ┃ -┡━━━━━━━━━┩ -│ 1000000 │ -- 1.00 million -└─────────┘ -┌───────a─┐ -│ 1000000 │ -- 1.00 million -└─────────┘ -┌───────a─┐ -│ 1000000 │ -- 1.00 million -└─────────┘ -┌───────a─┐ -│ 1000000 │ -- 1.00 million -└─────────┘ -┌───────a─┐ -│ 1000000 │ -- 1.00 million -└─────────┘ - a - - 1000000 -- 1.00 million - a - - 1000000 -- 1.00 million - a - - 1000000 -- 1.00 million - a - - 1000000 -- 1.00 million -┏━━━━━━━━━┓ -┃ a ┃ -┡━━━━━━━━━┩ -│ 1000001 │ -- 1.00 million -└─────────┘ -┏━━━━━━━━━┓ -┃ a ┃ -┡━━━━━━━━━┩ -│ 1000001 │ -- 1.00 million -└─────────┘ -┏━━━━━━━━━┓ -┃ a ┃ -┡━━━━━━━━━┩ -│ 1000001 │ -- 1.00 million -└─────────┘ -┏━━━━━━━━━┓ -┃ a ┃ -┡━━━━━━━━━┩ -│ 1000001 │ -- 1.00 million -└─────────┘ -┌───────a─┐ -│ 1000001 │ -- 1.00 million -└─────────┘ -┌───────a─┐ -│ 1000001 │ -- 1.00 million -└─────────┘ -┌───────a─┐ -│ 1000001 │ -- 1.00 million -└─────────┘ -┌───────a─┐ -│ 1000001 │ -- 1.00 million -└─────────┘ - a - - 1000001 -- 1.00 million - a - - 1000001 -- 1.00 million - a - - 1000001 -- 1.00 million - a - - 1000001 -- 1.00 million -┏━━━━━━━━━━━━┓ -┃ a ┃ -┡━━━━━━━━━━━━┩ -│ 1000000000 │ -- 1.00 billion -└────────────┘ -┏━━━━━━━━━━━━┓ -┃ a ┃ -┡━━━━━━━━━━━━┩ -│ 1000000000 │ -- 1.00 billion -└────────────┘ -┏━━━━━━━━━━━━┓ -┃ a ┃ -┡━━━━━━━━━━━━┩ -│ 1000000000 │ -- 1.00 billion -└────────────┘ -┏━━━━━━━━━━━━┓ -┃ a ┃ -┡━━━━━━━━━━━━┩ -│ 1000000000 │ -- 1.00 billion -└────────────┘ -┌──────────a─┐ -│ 1000000000 │ -- 1.00 billion -└────────────┘ -┌──────────a─┐ -│ 1000000000 │ -- 1.00 billion -└────────────┘ -┌──────────a─┐ -│ 1000000000 │ -- 1.00 billion -└────────────┘ -┌──────────a─┐ -│ 1000000000 │ -- 1.00 billion -└────────────┘ + ┏━━━━━━━━━┓ + ┃ a ┃ + ┡━━━━━━━━━┩ +1. │ 1000000 │ + └─────────┘ + ┏━━━━━━━━━┓ + ┃ a ┃ + ┡━━━━━━━━━┩ +1. │ 1000000 │ + └─────────┘ + ┏━━━━━━━━━┓ + ┃ a ┃ + ┡━━━━━━━━━┩ +1. │ 1000000 │ + └─────────┘ + ┏━━━━━━━━━┓ + ┃ a ┃ + ┡━━━━━━━━━┩ +1. │ 1000000 │ + └─────────┘ + ┌───────a─┐ +1. │ 1000000 │ + └─────────┘ + ┌───────a─┐ +1. │ 1000000 │ + └─────────┘ + ┌───────a─┐ +1. │ 1000000 │ + └─────────┘ + ┌───────a─┐ +1. │ 1000000 │ + └─────────┘ a - 1000000000 -- 1.00 billion +1. 1000000 a - 1000000000 -- 1.00 billion +1. 1000000 a - 1000000000 -- 1.00 billion +1. 1000000 a - 1000000000 -- 1.00 billion -┏━━━━━━━━━━━━┳━━━━━━━━━━━━┓ -┃ a ┃ b ┃ -┡━━━━━━━━━━━━╇━━━━━━━━━━━━┩ -│ 1000000000 │ 1000000000 │ -└────────────┴────────────┘ -┏━━━━━━━━━━━━┳━━━━━━━━━━━━┓ -┃ a ┃ b ┃ -┡━━━━━━━━━━━━╇━━━━━━━━━━━━┩ -│ 1000000000 │ 1000000000 │ -└────────────┴────────────┘ -┏━━━━━━━━━━━━┳━━━━━━━━━━━━┓ -┃ a ┃ b ┃ -┡━━━━━━━━━━━━╇━━━━━━━━━━━━┩ -│ 1000000000 │ 1000000000 │ -└────────────┴────────────┘ -┏━━━━━━━━━━━━┳━━━━━━━━━━━━┓ -┃ a ┃ b ┃ -┡━━━━━━━━━━━━╇━━━━━━━━━━━━┩ -│ 1000000000 │ 1000000000 │ -└────────────┴────────────┘ -┌──────────a─┬──────────b─┐ -│ 1000000000 │ 1000000000 │ -└────────────┴────────────┘ -┌──────────a─┬──────────b─┐ -│ 1000000000 │ 1000000000 │ -└────────────┴────────────┘ -┌──────────a─┬──────────b─┐ -│ 1000000000 │ 1000000000 │ -└────────────┴────────────┘ -┌──────────a─┬──────────b─┐ -│ 1000000000 │ 1000000000 │ -└────────────┴────────────┘ - a b - - 1000000000 1000000000 - a b - - 1000000000 1000000000 - a b - - 1000000000 1000000000 - a b - - 1000000000 1000000000 -┏━━━━━━━━━━━━┓ -┃ a ┃ -┡━━━━━━━━━━━━┩ -│ 1000000000 │ -├────────────┤ -│ 1000000000 │ -└────────────┘ -┏━━━━━━━━━━━━┓ -┃ a ┃ -┡━━━━━━━━━━━━┩ -│ 1000000000 │ -├────────────┤ -│ 1000000000 │ -└────────────┘ -┏━━━━━━━━━━━━┓ -┃ a ┃ -┡━━━━━━━━━━━━┩ -│ 1000000000 │ -├────────────┤ -│ 1000000000 │ -└────────────┘ -┏━━━━━━━━━━━━┓ -┃ a ┃ -┡━━━━━━━━━━━━┩ -│ 1000000000 │ -├────────────┤ -│ 1000000000 │ -└────────────┘ -┌──────────a─┐ -│ 1000000000 │ -│ 1000000000 │ -└────────────┘ -┌──────────a─┐ -│ 1000000000 │ -│ 1000000000 │ -└────────────┘ -┌──────────a─┐ -│ 1000000000 │ -│ 1000000000 │ -└────────────┘ -┌──────────a─┐ -│ 1000000000 │ -│ 1000000000 │ -└────────────┘ +1. 1000000 + ┏━━━━━━━━━┓ + ┃ a ┃ + ┡━━━━━━━━━┩ +1. │ 1000000 │ -- 1.00 million + └─────────┘ + ┏━━━━━━━━━┓ + ┃ a ┃ + ┡━━━━━━━━━┩ +1. │ 1000000 │ -- 1.00 million + └─────────┘ + ┏━━━━━━━━━┓ + ┃ a ┃ + ┡━━━━━━━━━┩ +1. │ 1000000 │ -- 1.00 million + └─────────┘ + ┏━━━━━━━━━┓ + ┃ a ┃ + ┡━━━━━━━━━┩ +1. │ 1000000 │ -- 1.00 million + └─────────┘ + ┌───────a─┐ +1. │ 1000000 │ -- 1.00 million + └─────────┘ + ┌───────a─┐ +1. │ 1000000 │ -- 1.00 million + └─────────┘ + ┌───────a─┐ +1. │ 1000000 │ -- 1.00 million + └─────────┘ + ┌───────a─┐ +1. │ 1000000 │ -- 1.00 million + └─────────┘ a - 1000000000 - 1000000000 +1. 1000000 -- 1.00 million a - 1000000000 - 1000000000 +1. 1000000 -- 1.00 million a - 1000000000 - 1000000000 +1. 1000000 -- 1.00 million a - 1000000000 - 1000000000 -┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ CAST('2024-02-29', 'Date') ┃ -┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ 2024-02-29 │ -└────────────────────────────┘ -┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ CAST('2024-02-29', 'Date') ┃ -┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ 2024-02-29 │ -└────────────────────────────┘ -┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ CAST('2024-02-29', 'Date') ┃ -┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ 2024-02-29 │ -└────────────────────────────┘ -┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ CAST('2024-02-29', 'Date') ┃ -┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ 2024-02-29 │ -└────────────────────────────┘ -┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ CAST('2024-02-29', 'Date32') ┃ -┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ 2024-02-29 │ -└──────────────────────────────┘ -┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ CAST('2024-02-29', 'Date32') ┃ -┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ 2024-02-29 │ -└──────────────────────────────┘ -┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ CAST('2024-02-29', 'Date32') ┃ -┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ 2024-02-29 │ -└──────────────────────────────┘ -┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ CAST('2024-02-29', 'Date32') ┃ -┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ 2024-02-29 │ -└──────────────────────────────┘ -┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ CAST('2024-02-29 00:00:00', 'DateTime') ┃ -┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ 2024-02-29 00:00:00 │ -└─────────────────────────────────────────┘ -┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ CAST('2024-02-29 00:00:00', 'DateTime') ┃ -┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ 2024-02-29 00:00:00 │ -└─────────────────────────────────────────┘ -┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ CAST('2024-02-29 00:00:00', 'DateTime') ┃ -┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ 2024-02-29 00:00:00 │ -└─────────────────────────────────────────┘ -┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ CAST('2024-02-29 00:00:00', 'DateTime') ┃ -┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ 2024-02-29 00:00:00 │ -└─────────────────────────────────────────┘ -┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ CAST(CAST('2024-02-29 00:00:00', 'DateTime'), 'DateTime64') ┃ -┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ 2024-02-29 00:00:00.000 │ -└─────────────────────────────────────────────────────────────┘ -┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ CAST(CAST('2024-02-29 00:00:00', 'DateTime'), 'DateTime64') ┃ -┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ 2024-02-29 00:00:00.000 │ -└─────────────────────────────────────────────────────────────┘ -┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ CAST(CAST('2024-02-29 00:00:00', 'DateTime'), 'DateTime64') ┃ -┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ 2024-02-29 00:00:00.000 │ -└─────────────────────────────────────────────────────────────┘ -┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ -┃ CAST(CAST('2024-02-29 00:00:00', 'DateTime'), 'DateTime64') ┃ -┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ -│ 2024-02-29 00:00:00.000 │ -└─────────────────────────────────────────────────────────────┘ +1. 1000000 -- 1.00 million + ┏━━━━━━━━━┓ + ┃ a ┃ + ┡━━━━━━━━━┩ +1. │ 1000001 │ -- 1.00 million + └─────────┘ + ┏━━━━━━━━━┓ + ┃ a ┃ + ┡━━━━━━━━━┩ +1. │ 1000001 │ -- 1.00 million + └─────────┘ + ┏━━━━━━━━━┓ + ┃ a ┃ + ┡━━━━━━━━━┩ +1. │ 1000001 │ -- 1.00 million + └─────────┘ + ┏━━━━━━━━━┓ + ┃ a ┃ + ┡━━━━━━━━━┩ +1. │ 1000001 │ -- 1.00 million + └─────────┘ + ┌───────a─┐ +1. │ 1000001 │ -- 1.00 million + └─────────┘ + ┌───────a─┐ +1. │ 1000001 │ -- 1.00 million + └─────────┘ + ┌───────a─┐ +1. │ 1000001 │ -- 1.00 million + └─────────┘ + ┌───────a─┐ +1. │ 1000001 │ -- 1.00 million + └─────────┘ + a + +1. 1000001 -- 1.00 million + a + +1. 1000001 -- 1.00 million + a + +1. 1000001 -- 1.00 million + a + +1. 1000001 -- 1.00 million + ┏━━━━━━━━━━━━┓ + ┃ a ┃ + ┡━━━━━━━━━━━━┩ +1. │ 1000000000 │ -- 1.00 billion + └────────────┘ + ┏━━━━━━━━━━━━┓ + ┃ a ┃ + ┡━━━━━━━━━━━━┩ +1. │ 1000000000 │ -- 1.00 billion + └────────────┘ + ┏━━━━━━━━━━━━┓ + ┃ a ┃ + ┡━━━━━━━━━━━━┩ +1. │ 1000000000 │ -- 1.00 billion + └────────────┘ + ┏━━━━━━━━━━━━┓ + ┃ a ┃ + ┡━━━━━━━━━━━━┩ +1. │ 1000000000 │ -- 1.00 billion + └────────────┘ + ┌──────────a─┐ +1. │ 1000000000 │ -- 1.00 billion + └────────────┘ + ┌──────────a─┐ +1. │ 1000000000 │ -- 1.00 billion + └────────────┘ + ┌──────────a─┐ +1. │ 1000000000 │ -- 1.00 billion + └────────────┘ + ┌──────────a─┐ +1. │ 1000000000 │ -- 1.00 billion + └────────────┘ + a + +1. 1000000000 -- 1.00 billion + a + +1. 1000000000 -- 1.00 billion + a + +1. 1000000000 -- 1.00 billion + a + +1. 1000000000 -- 1.00 billion + ┏━━━━━━━━━━━━┳━━━━━━━━━━━━┓ + ┃ a ┃ b ┃ + ┡━━━━━━━━━━━━╇━━━━━━━━━━━━┩ +1. │ 1000000000 │ 1000000000 │ + └────────────┴────────────┘ + ┏━━━━━━━━━━━━┳━━━━━━━━━━━━┓ + ┃ a ┃ b ┃ + ┡━━━━━━━━━━━━╇━━━━━━━━━━━━┩ +1. │ 1000000000 │ 1000000000 │ + └────────────┴────────────┘ + ┏━━━━━━━━━━━━┳━━━━━━━━━━━━┓ + ┃ a ┃ b ┃ + ┡━━━━━━━━━━━━╇━━━━━━━━━━━━┩ +1. │ 1000000000 │ 1000000000 │ + └────────────┴────────────┘ + ┏━━━━━━━━━━━━┳━━━━━━━━━━━━┓ + ┃ a ┃ b ┃ + ┡━━━━━━━━━━━━╇━━━━━━━━━━━━┩ +1. │ 1000000000 │ 1000000000 │ + └────────────┴────────────┘ + ┌──────────a─┬──────────b─┐ +1. │ 1000000000 │ 1000000000 │ + └────────────┴────────────┘ + ┌──────────a─┬──────────b─┐ +1. │ 1000000000 │ 1000000000 │ + └────────────┴────────────┘ + ┌──────────a─┬──────────b─┐ +1. │ 1000000000 │ 1000000000 │ + └────────────┴────────────┘ + ┌──────────a─┬──────────b─┐ +1. │ 1000000000 │ 1000000000 │ + └────────────┴────────────┘ + a b + +1. 1000000000 1000000000 + a b + +1. 1000000000 1000000000 + a b + +1. 1000000000 1000000000 + a b + +1. 1000000000 1000000000 + ┏━━━━━━━━━━━━┓ + ┃ a ┃ + ┡━━━━━━━━━━━━┩ +1. │ 1000000000 │ + ├────────────┤ +2. │ 1000000000 │ + └────────────┘ + ┏━━━━━━━━━━━━┓ + ┃ a ┃ + ┡━━━━━━━━━━━━┩ +1. │ 1000000000 │ + ├────────────┤ +2. │ 1000000000 │ + └────────────┘ + ┏━━━━━━━━━━━━┓ + ┃ a ┃ + ┡━━━━━━━━━━━━┩ +1. │ 1000000000 │ + ├────────────┤ +2. │ 1000000000 │ + └────────────┘ + ┏━━━━━━━━━━━━┓ + ┃ a ┃ + ┡━━━━━━━━━━━━┩ +1. │ 1000000000 │ + ├────────────┤ +2. │ 1000000000 │ + └────────────┘ + ┌──────────a─┐ +1. │ 1000000000 │ +2. │ 1000000000 │ + └────────────┘ + ┌──────────a─┐ +1. │ 1000000000 │ +2. │ 1000000000 │ + └────────────┘ + ┌──────────a─┐ +1. │ 1000000000 │ +2. │ 1000000000 │ + └────────────┘ + ┌──────────a─┐ +1. │ 1000000000 │ +2. │ 1000000000 │ + └────────────┘ + a + +1. 1000000000 +2. 1000000000 + a + +1. 1000000000 +2. 1000000000 + a + +1. 1000000000 +2. 1000000000 + a + +1. 1000000000 +2. 1000000000 + ┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ CAST('2024-02-29', 'Date') ┃ + ┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +1. │ 2024-02-29 │ + └────────────────────────────┘ + ┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ CAST('2024-02-29', 'Date') ┃ + ┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +1. │ 2024-02-29 │ + └────────────────────────────┘ + ┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ CAST('2024-02-29', 'Date') ┃ + ┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +1. │ 2024-02-29 │ + └────────────────────────────┘ + ┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ CAST('2024-02-29', 'Date') ┃ + ┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +1. │ 2024-02-29 │ + └────────────────────────────┘ + ┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ CAST('2024-02-29', 'Date32') ┃ + ┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +1. │ 2024-02-29 │ + └──────────────────────────────┘ + ┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ CAST('2024-02-29', 'Date32') ┃ + ┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +1. │ 2024-02-29 │ + └──────────────────────────────┘ + ┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ CAST('2024-02-29', 'Date32') ┃ + ┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +1. │ 2024-02-29 │ + └──────────────────────────────┘ + ┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ CAST('2024-02-29', 'Date32') ┃ + ┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +1. │ 2024-02-29 │ + └──────────────────────────────┘ + ┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ CAST('2024-02-29 00:00:00', 'DateTime') ┃ + ┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +1. │ 2024-02-29 00:00:00 │ + └─────────────────────────────────────────┘ + ┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ CAST('2024-02-29 00:00:00', 'DateTime') ┃ + ┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +1. │ 2024-02-29 00:00:00 │ + └─────────────────────────────────────────┘ + ┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ CAST('2024-02-29 00:00:00', 'DateTime') ┃ + ┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +1. │ 2024-02-29 00:00:00 │ + └─────────────────────────────────────────┘ + ┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ CAST('2024-02-29 00:00:00', 'DateTime') ┃ + ┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +1. │ 2024-02-29 00:00:00 │ + └─────────────────────────────────────────┘ + ┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ CAST(CAST('2024-02-29 00:00:00', 'DateTime'), 'DateTime64') ┃ + ┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +1. │ 2024-02-29 00:00:00.000 │ + └─────────────────────────────────────────────────────────────┘ + ┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ CAST(CAST('2024-02-29 00:00:00', 'DateTime'), 'DateTime64') ┃ + ┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +1. │ 2024-02-29 00:00:00.000 │ + └─────────────────────────────────────────────────────────────┘ + ┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ CAST(CAST('2024-02-29 00:00:00', 'DateTime'), 'DateTime64') ┃ + ┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +1. │ 2024-02-29 00:00:00.000 │ + └─────────────────────────────────────────────────────────────┘ + ┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ CAST(CAST('2024-02-29 00:00:00', 'DateTime'), 'DateTime64') ┃ + ┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ +1. │ 2024-02-29 00:00:00.000 │ + └─────────────────────────────────────────────────────────────┘ From a3e7c73e9f5d19a2df1f4fca2245929b67f25f89 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 23 Mar 2024 20:02:38 +0100 Subject: [PATCH 068/102] Fix tests --- .../02947_merge_tree_index_table_1.reference | 16 +++++----- .../02947_merge_tree_index_table_1.sql | 6 ++-- .../02947_merge_tree_index_table_2.reference | 30 +++++++++---------- .../02947_merge_tree_index_table_2.sql | 2 ++ 4 files changed, 29 insertions(+), 25 deletions(-) diff --git a/tests/queries/0_stateless/02947_merge_tree_index_table_1.reference b/tests/queries/0_stateless/02947_merge_tree_index_table_1.reference index d4397328dbc..b897f6e50ed 100644 --- a/tests/queries/0_stateless/02947_merge_tree_index_table_1.reference +++ b/tests/queries/0_stateless/02947_merge_tree_index_table_1.reference @@ -73,12 +73,12 @@ 8. │ all_2_2_0 │ 3 │ 0 │ 3 │ 19 │ └───────────┴─────────────┴─────────────────┴───┴────┘ ┌─part_name─┬─mark_number─┬─rows_in_granule─┬─a─┬──b─┬─a.mark──┬─b.mark──┬─s.mark──┐ -1. │ all_1_1_0 │ 0 │ 3 │ 0 │ 0 │ (0,0) │ (42,0) │ (96,0) │ -2. │ all_1_1_0 │ 1 │ 3 │ 1 │ 1 │ (141,0) │ (195,0) │ (249,0) │ -3. │ all_1_1_0 │ 2 │ 4 │ 2 │ 2 │ (294,0) │ (349,0) │ (404,0) │ -4. │ all_1_1_0 │ 3 │ 0 │ 3 │ 7 │ (452,0) │ (452,0) │ (452,0) │ -5. │ all_2_2_0 │ 0 │ 3 │ 0 │ 12 │ (0,0) │ (50,0) │ (104,0) │ -6. │ all_2_2_0 │ 1 │ 3 │ 1 │ 17 │ (153,0) │ (207,0) │ (261,0) │ -7. │ all_2_2_0 │ 2 │ 4 │ 2 │ 18 │ (310,0) │ (365,0) │ (420,0) │ -8. │ all_2_2_0 │ 3 │ 0 │ 3 │ 19 │ (474,0) │ (474,0) │ (474,0) │ +1. │ all_1_1_0 │ 0 │ 3 │ 0 │ 0 │ (0,0) │ (35,0) │ (77,0) │ +2. │ all_1_1_0 │ 1 │ 3 │ 1 │ 1 │ (114,0) │ (153,0) │ (197,0) │ +3. │ all_1_1_0 │ 2 │ 4 │ 2 │ 2 │ (234,0) │ (281,0) │ (329,0) │ +4. │ all_1_1_0 │ 3 │ 0 │ 3 │ 7 │ (369,0) │ (369,0) │ (369,0) │ +5. │ all_2_2_0 │ 0 │ 3 │ 0 │ 12 │ (0,0) │ (38,0) │ (82,0) │ +6. │ all_2_2_0 │ 1 │ 3 │ 1 │ 17 │ (124,0) │ (168,0) │ (212,0) │ +7. │ all_2_2_0 │ 2 │ 4 │ 2 │ 18 │ (254,0) │ (297,0) │ (345,0) │ +8. │ all_2_2_0 │ 3 │ 0 │ 3 │ 19 │ (392,0) │ (392,0) │ (392,0) │ └───────────┴─────────────┴─────────────────┴───┴────┴─────────┴─────────┴─────────┘ diff --git a/tests/queries/0_stateless/02947_merge_tree_index_table_1.sql b/tests/queries/0_stateless/02947_merge_tree_index_table_1.sql index 412fd476413..84993de352b 100644 --- a/tests/queries/0_stateless/02947_merge_tree_index_table_1.sql +++ b/tests/queries/0_stateless/02947_merge_tree_index_table_1.sql @@ -1,6 +1,8 @@ +-- Tags: no-random-settings + DROP TABLE IF EXISTS t_merge_tree_index; -CREATE TABLE t_merge_tree_index (a UInt64, b UInt64, s String) +CREATE TABLE t_merge_tree_index (a UInt64 CODEC(LZ4), b UInt64 CODEC(LZ4), s String CODEC(LZ4)) ENGINE = MergeTree ORDER BY (a, b) SETTINGS index_granularity = 3, @@ -18,7 +20,7 @@ SELECT * FROM mergeTreeIndex(currentDatabase(), t_merge_tree_index, with_marks = DROP TABLE t_merge_tree_index; -CREATE TABLE t_merge_tree_index (a UInt64, b UInt64, s String) +CREATE TABLE t_merge_tree_index (a UInt64 CODEC(LZ4), b UInt64 CODEC(LZ4), s String CODEC(LZ4)) ENGINE = MergeTree ORDER BY (a, b) SETTINGS index_granularity = 3, diff --git a/tests/queries/0_stateless/02947_merge_tree_index_table_2.reference b/tests/queries/0_stateless/02947_merge_tree_index_table_2.reference index 5328ff18d12..7bfcb7b2822 100644 --- a/tests/queries/0_stateless/02947_merge_tree_index_table_2.reference +++ b/tests/queries/0_stateless/02947_merge_tree_index_table_2.reference @@ -13,21 +13,21 @@ │ all_3_3_0 │ 3 │ 1 │ 4 │ 9 │ 96 │ │ all_3_3_0 │ 4 │ 0 │ 4 │ 9 │ 96 │ └───────────┴─────────────┴─────────────────┴───┴───┴────────────────────────────┘ -┌─part_name─┬─mark_number─┬─rows_in_granule─┬─a─┬─b─┬─modulo(sipHash64(sp), 100)─┬─a.mark───┬─b.mark───┬─c.mark──────┬─sp.sparse.idx.mark─┬─sp.mark──┬─arr.size0.mark─┬─arr.dict.mark─┬─arr.mark─┬─n.size0.mark─┬─n%2Ec1.mark─┬─n%2Ec2.mark─┬─t%2Ec2.mark─┬─t%2Ec1.mark─┬─t.mark──────┬─column%2Ewith%2Edots.mark─┐ -│ all_1_1_0 │ 0 │ 3 │ 0 │ 0 │ 19 │ (0,0) │ (0,0) │ (NULL,NULL) │ (0,0) │ (0,0) │ (0,0) │ (0,8) │ (0,0) │ (0,0) │ (0,0) │ (0,0) │ (0,0) │ (0,0) │ (NULL,NULL) │ (0,0) │ -│ all_1_1_0 │ 1 │ 3 │ 1 │ 6 │ 19 │ (0,24) │ (0,24) │ (NULL,NULL) │ (0,9) │ (0,0) │ (0,24) │ (0,8) │ (0,22) │ (0,24) │ (0,36) │ (0,72) │ (0,24) │ (0,24) │ (NULL,NULL) │ (0,24) │ -│ all_1_1_0 │ 2 │ 3 │ 3 │ 3 │ 19 │ (0,48) │ (0,48) │ (NULL,NULL) │ (0,18) │ (0,0) │ (0,48) │ (0,8) │ (0,44) │ (0,48) │ (0,72) │ (0,144) │ (0,48) │ (0,48) │ (NULL,NULL) │ (0,48) │ -│ all_1_1_0 │ 3 │ 1 │ 4 │ 9 │ 19 │ (0,72) │ (0,72) │ (NULL,NULL) │ (0,27) │ (0,0) │ (0,72) │ (0,8) │ (0,66) │ (0,72) │ (0,108) │ (0,216) │ (0,72) │ (0,72) │ (NULL,NULL) │ (0,72) │ -│ all_1_1_0 │ 4 │ 0 │ 4 │ 9 │ 19 │ (0,80) │ (0,80) │ (NULL,NULL) │ (0,36) │ (0,0) │ (0,80) │ (0,25) │ (0,84) │ (0,80) │ (0,120) │ (0,240) │ (0,80) │ (0,80) │ (NULL,NULL) │ (0,80) │ -│ all_2_2_0 │ 0 │ 3 │ 0 │ 0 │ 96 │ (0,0) │ (54,0) │ (108,0) │ (NULL,NULL) │ (162,0) │ (NULL,NULL) │ (NULL,NULL) │ (216,0) │ (NULL,NULL) │ (291,0) │ (355,0) │ (NULL,NULL) │ (NULL,NULL) │ (426,0) │ (484,0) │ -│ all_2_2_0 │ 1 │ 2 │ 3 │ 3 │ 96 │ (538,0) │ (588,0) │ (638,0) │ (NULL,NULL) │ (688,0) │ (NULL,NULL) │ (NULL,NULL) │ (738,0) │ (NULL,NULL) │ (808,0) │ (868,0) │ (NULL,NULL) │ (NULL,NULL) │ (931,0) │ (986,0) │ -│ all_2_2_0 │ 2 │ 0 │ 4 │ 4 │ 96 │ (1036,0) │ (1036,0) │ (1036,0) │ (NULL,NULL) │ (1036,0) │ (NULL,NULL) │ (NULL,NULL) │ (1036,0) │ (NULL,NULL) │ (1036,0) │ (1036,0) │ (NULL,NULL) │ (NULL,NULL) │ (1036,0) │ (1036,0) │ -│ all_3_3_0 │ 0 │ 3 │ 0 │ 0 │ 96 │ (0,0) │ (0,0) │ (0,0) │ (NULL,NULL) │ (0,0) │ (0,0) │ (0,8) │ (0,0) │ (0,0) │ (0,0) │ (0,0) │ (0,0) │ (0,0) │ (NULL,NULL) │ (0,0) │ -│ all_3_3_0 │ 1 │ 3 │ 1 │ 6 │ 96 │ (0,24) │ (0,24) │ (0,24) │ (NULL,NULL) │ (0,24) │ (0,24) │ (0,8) │ (0,22) │ (0,24) │ (0,36) │ (0,72) │ (0,24) │ (0,24) │ (NULL,NULL) │ (0,24) │ -│ all_3_3_0 │ 2 │ 3 │ 3 │ 3 │ 96 │ (0,48) │ (0,48) │ (0,48) │ (NULL,NULL) │ (0,48) │ (0,48) │ (0,8) │ (0,44) │ (0,48) │ (0,72) │ (0,144) │ (0,48) │ (0,48) │ (NULL,NULL) │ (0,48) │ -│ all_3_3_0 │ 3 │ 1 │ 4 │ 9 │ 96 │ (0,72) │ (0,72) │ (0,72) │ (NULL,NULL) │ (0,72) │ (0,72) │ (0,8) │ (0,66) │ (0,72) │ (0,108) │ (0,216) │ (0,72) │ (0,72) │ (NULL,NULL) │ (0,72) │ -│ all_3_3_0 │ 4 │ 0 │ 4 │ 9 │ 96 │ (0,80) │ (0,80) │ (0,80) │ (NULL,NULL) │ (0,80) │ (0,80) │ (0,25) │ (0,84) │ (0,80) │ (0,120) │ (0,240) │ (0,80) │ (0,80) │ (NULL,NULL) │ (0,80) │ -└───────────┴─────────────┴─────────────────┴───┴───┴────────────────────────────┴──────────┴──────────┴─────────────┴────────────────────┴──────────┴────────────────┴───────────────┴──────────┴──────────────┴─────────────┴─────────────┴─────────────┴─────────────┴─────────────┴───────────────────────────┘ +┌─part_name─┬─mark_number─┬─rows_in_granule─┬─a─┬─b─┬─modulo(sipHash64(sp), 100)─┬─a.mark──┬─b.mark──┬─c.mark──────┬─sp.sparse.idx.mark─┬─sp.mark─┬─arr.size0.mark─┬─arr.dict.mark─┬─arr.mark─┬─n.size0.mark─┬─n%2Ec1.mark─┬─n%2Ec2.mark─┬─t%2Ec2.mark─┬─t%2Ec1.mark─┬─t.mark──────┬─column%2Ewith%2Edots.mark─┐ +│ all_1_1_0 │ 0 │ 3 │ 0 │ 0 │ 19 │ (0,0) │ (0,0) │ (NULL,NULL) │ (0,0) │ (0,0) │ (0,0) │ (0,8) │ (0,0) │ (0,0) │ (0,0) │ (0,0) │ (0,0) │ (0,0) │ (NULL,NULL) │ (0,0) │ +│ all_1_1_0 │ 1 │ 3 │ 1 │ 6 │ 19 │ (0,24) │ (0,24) │ (NULL,NULL) │ (0,9) │ (0,0) │ (0,24) │ (0,8) │ (0,22) │ (0,24) │ (0,36) │ (0,72) │ (0,24) │ (0,24) │ (NULL,NULL) │ (0,24) │ +│ all_1_1_0 │ 2 │ 3 │ 3 │ 3 │ 19 │ (0,48) │ (0,48) │ (NULL,NULL) │ (0,18) │ (0,0) │ (0,48) │ (0,8) │ (0,44) │ (0,48) │ (0,72) │ (0,144) │ (0,48) │ (0,48) │ (NULL,NULL) │ (0,48) │ +│ all_1_1_0 │ 3 │ 1 │ 4 │ 9 │ 19 │ (0,72) │ (0,72) │ (NULL,NULL) │ (0,27) │ (0,0) │ (0,72) │ (0,8) │ (0,66) │ (0,72) │ (0,108) │ (0,216) │ (0,72) │ (0,72) │ (NULL,NULL) │ (0,72) │ +│ all_1_1_0 │ 4 │ 0 │ 4 │ 9 │ 19 │ (0,80) │ (0,80) │ (NULL,NULL) │ (0,36) │ (0,0) │ (0,80) │ (0,25) │ (0,84) │ (0,80) │ (0,120) │ (0,240) │ (0,80) │ (0,80) │ (NULL,NULL) │ (0,80) │ +│ all_2_2_0 │ 0 │ 3 │ 0 │ 0 │ 96 │ (0,0) │ (42,0) │ (84,0) │ (NULL,NULL) │ (126,0) │ (NULL,NULL) │ (NULL,NULL) │ (165,0) │ (NULL,NULL) │ (232,0) │ (286,0) │ (NULL,NULL) │ (NULL,NULL) │ (342,0) │ (391,0) │ +│ all_2_2_0 │ 1 │ 2 │ 3 │ 3 │ 96 │ (433,0) │ (472,0) │ (511,0) │ (NULL,NULL) │ (550,0) │ (NULL,NULL) │ (NULL,NULL) │ (589,0) │ (NULL,NULL) │ (659,0) │ (717,0) │ (NULL,NULL) │ (NULL,NULL) │ (773,0) │ (817,0) │ +│ all_2_2_0 │ 2 │ 0 │ 4 │ 4 │ 96 │ (856,0) │ (856,0) │ (856,0) │ (NULL,NULL) │ (856,0) │ (NULL,NULL) │ (NULL,NULL) │ (856,0) │ (NULL,NULL) │ (856,0) │ (856,0) │ (NULL,NULL) │ (NULL,NULL) │ (856,0) │ (856,0) │ +│ all_3_3_0 │ 0 │ 3 │ 0 │ 0 │ 96 │ (0,0) │ (0,0) │ (0,0) │ (NULL,NULL) │ (0,0) │ (0,0) │ (0,8) │ (0,0) │ (0,0) │ (0,0) │ (0,0) │ (0,0) │ (0,0) │ (NULL,NULL) │ (0,0) │ +│ all_3_3_0 │ 1 │ 3 │ 1 │ 6 │ 96 │ (0,24) │ (0,24) │ (0,24) │ (NULL,NULL) │ (0,24) │ (0,24) │ (0,8) │ (0,22) │ (0,24) │ (0,36) │ (0,72) │ (0,24) │ (0,24) │ (NULL,NULL) │ (0,24) │ +│ all_3_3_0 │ 2 │ 3 │ 3 │ 3 │ 96 │ (0,48) │ (0,48) │ (0,48) │ (NULL,NULL) │ (0,48) │ (0,48) │ (0,8) │ (0,44) │ (0,48) │ (0,72) │ (0,144) │ (0,48) │ (0,48) │ (NULL,NULL) │ (0,48) │ +│ all_3_3_0 │ 3 │ 1 │ 4 │ 9 │ 96 │ (0,72) │ (0,72) │ (0,72) │ (NULL,NULL) │ (0,72) │ (0,72) │ (0,8) │ (0,66) │ (0,72) │ (0,108) │ (0,216) │ (0,72) │ (0,72) │ (NULL,NULL) │ (0,72) │ +│ all_3_3_0 │ 4 │ 0 │ 4 │ 9 │ 96 │ (0,80) │ (0,80) │ (0,80) │ (NULL,NULL) │ (0,80) │ (0,80) │ (0,25) │ (0,84) │ (0,80) │ (0,120) │ (0,240) │ (0,80) │ (0,80) │ (NULL,NULL) │ (0,80) │ +└───────────┴─────────────┴─────────────────┴───┴───┴────────────────────────────┴─────────┴─────────┴─────────────┴────────────────────┴─────────┴────────────────┴───────────────┴──────────┴──────────────┴─────────────┴─────────────┴─────────────┴─────────────┴─────────────┴───────────────────────────┘ part_name String mark_number UInt64 rows_in_granule UInt64 diff --git a/tests/queries/0_stateless/02947_merge_tree_index_table_2.sql b/tests/queries/0_stateless/02947_merge_tree_index_table_2.sql index f4fb2cec2dc..09c9a41e4d6 100644 --- a/tests/queries/0_stateless/02947_merge_tree_index_table_2.sql +++ b/tests/queries/0_stateless/02947_merge_tree_index_table_2.sql @@ -1,3 +1,5 @@ +-- Tags: no-random-settings + DROP TABLE IF EXISTS t_merge_tree_index; SET output_format_pretty_row_numbers = 0; From e195806c5b1899c55cc17c2116850cfeacfff46e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 23 Mar 2024 20:52:26 +0100 Subject: [PATCH 069/102] Add function getClientHTTPHeader, part 1 --- src/Interpreters/ClientInfo.cpp | 25 ++++++++++++++++++++++++- src/Interpreters/ClientInfo.h | 10 ++++++++++ src/Interpreters/Context.cpp | 6 ++---- src/Interpreters/Context.h | 2 +- src/Interpreters/Session.cpp | 12 +++--------- src/Interpreters/Session.h | 2 +- src/Server/HTTPHandler.cpp | 9 +-------- 7 files changed, 42 insertions(+), 24 deletions(-) diff --git a/src/Interpreters/ClientInfo.cpp b/src/Interpreters/ClientInfo.cpp index e4778edeb9c..ce1efb61cc0 100644 --- a/src/Interpreters/ClientInfo.cpp +++ b/src/Interpreters/ClientInfo.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -255,7 +256,29 @@ String toString(ClientInfo::Interface interface) return "TCP_INTERSERVER"; } - return std::format("Unknown {}!\n", static_cast(interface)); + return std::format("Unknown server interface ({}).", static_cast(interface)); +} + +void ClientInfo::setFromHTTPRequest(const Poco::Net::HTTPRequest & request) +{ + http_method = ClientInfo::HTTPMethod::UNKNOWN; + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET) + http_method = ClientInfo::HTTPMethod::GET; + else if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) + http_method = ClientInfo::HTTPMethod::POST; + + http_user_agent = request.get("User-Agent", ""); + http_referer = request.get("Referer", ""); + forwarded_for = request.get("X-Forwarded-For", ""); + + for (const auto & header : request) + { + /// These headers can contain authentication info and shouldn't be accessible by the user. + String key_lowercase = Poco::toLower(header.first); + if (key_lowercase.starts_with("x-clickhouse") || key_lowercase == "authentication") + continue; + http_headers[header.first] = header.second; + } } } diff --git a/src/Interpreters/ClientInfo.h b/src/Interpreters/ClientInfo.h index 70524333047..c2ed9f7ffa4 100644 --- a/src/Interpreters/ClientInfo.h +++ b/src/Interpreters/ClientInfo.h @@ -7,6 +7,12 @@ #include #include + +namespace Poco::Net +{ + class HTTPRequest; +} + namespace DB { @@ -93,6 +99,7 @@ public: HTTPMethod http_method = HTTPMethod::UNKNOWN; String http_user_agent; String http_referer; + std::unordered_map http_headers; /// For mysql and postgresql UInt64 connection_id = 0; @@ -135,6 +142,9 @@ public: /// Initialize parameters on client initiating query. void setInitialQuery(); + /// Initialize parameters related to HTTP request. + void setFromHTTPRequest(const Poco::Net::HTTPRequest & request); + bool clientVersionEquals(const ClientInfo & other, bool compare_patch) const; String getVersionStr() const; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index c6bc10f3d50..f86cb81c37f 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4640,11 +4640,9 @@ void Context::setClientConnectionId(uint32_t connection_id_) client_info.connection_id = connection_id_; } -void Context::setHTTPClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer) +void Context::setHTTPClientInfo(const Poco::Net::HTTPRequest & request) { - client_info.http_method = http_method; - client_info.http_user_agent = http_user_agent; - client_info.http_referer = http_referer; + client_info.setFromHTTPRequest(request); need_recalculate_access = true; } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index f7bf8282d8a..8601d09621f 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -642,7 +642,7 @@ public: void setClientInterface(ClientInfo::Interface interface); void setClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version); void setClientConnectionId(uint32_t connection_id); - void setHTTPClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer); + void setHTTPClientInfo(const Poco::Net::HTTPRequest & request); void setForwardedFor(const String & forwarded_for); void setQueryKind(ClientInfo::QueryKind query_kind); void setQueryKindInitial(); diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index b52f8a507e3..9f64380ab43 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -429,18 +429,12 @@ void Session::setClientConnectionId(uint32_t connection_id) prepared_client_info->connection_id = connection_id; } -void Session::setHTTPClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer) +void Session::setHTTPClientInfo(const Poco::Net::HTTPRequest & request) { if (session_context) - { - session_context->setHTTPClientInfo(http_method, http_user_agent, http_referer); - } + session_context->setHTTPClientInfo(request); else - { - prepared_client_info->http_method = http_method; - prepared_client_info->http_user_agent = http_user_agent; - prepared_client_info->http_referer = http_referer; - } + prepared_client_info->setFromHTTPRequest(request); } void Session::setForwardedFor(const String & forwarded_for) diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 334560a33c8..e6cb7ca20cd 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -65,7 +65,7 @@ public: void setClientInterface(ClientInfo::Interface interface); void setClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version); void setClientConnectionId(uint32_t connection_id); - void setHTTPClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer); + void setHTTPClientInfo(const Poco::Net::HTTPRequest & request); void setForwardedFor(const String & forwarded_for); void setQuotaClientKey(const String & quota_key); void setConnectionClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version); diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index c112eefec6c..fd9be999276 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -490,14 +490,7 @@ bool HTTPHandler::authenticateUser( /// Set client info. It will be used for quota accounting parameters in 'setUser' method. - ClientInfo::HTTPMethod http_method = ClientInfo::HTTPMethod::UNKNOWN; - if (request.getMethod() == HTTPServerRequest::HTTP_GET) - http_method = ClientInfo::HTTPMethod::GET; - else if (request.getMethod() == HTTPServerRequest::HTTP_POST) - http_method = ClientInfo::HTTPMethod::POST; - - session->setHTTPClientInfo(http_method, request.get("User-Agent", ""), request.get("Referer", "")); - session->setForwardedFor(request.get("X-Forwarded-For", "")); + session->setHTTPClientInfo(request); session->setQuotaClientKey(quota_key); /// Extract the last entry from comma separated list of forwarded_for addresses. From 965a3ac2f32f620acdbdbcf22dc645fbc18b8c20 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 23 Mar 2024 21:19:35 +0100 Subject: [PATCH 070/102] Add function getClientHTTPHeader, part 2 --- src/Core/Settings.h | 1 + src/Functions/getClientHTTPHeader.cpp | 94 +++++++++++++++++++++++++++ 2 files changed, 95 insertions(+) create mode 100644 src/Functions/getClientHTTPHeader.cpp diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d9071ef592b..9afa55e526e 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -895,6 +895,7 @@ class IColumn; M(Int64, ignore_cold_parts_seconds, 0, "Only available in ClickHouse Cloud. Exclude new data parts from SELECT queries until they're either pre-warmed (see cache_populated_by_fetch) or this many seconds old. Only for Replicated-/SharedMergeTree.", 0) \ M(Int64, prefer_warmed_unmerged_parts_seconds, 0, "Only available in ClickHouse Cloud. If a merged part is less than this many seconds old and is not pre-warmed (see cache_populated_by_fetch), but all its source parts are available and pre-warmed, SELECT queries will read from those parts instead. Only for ReplicatedMergeTree. Note that this only checks whether CacheWarmer processed the part; if the part was fetched into cache by something else, it'll still be considered cold until CacheWarmer gets to it; if it was warmed, then evicted from cache, it'll still be considered warm.", 0) \ M(Bool, iceberg_engine_ignore_schema_evolution, false, "Ignore schema evolution in Iceberg table engine and read all data using latest schema saved on table creation. Note that it can lead to incorrect result", 0) \ + M(Bool, allow_get_http_client_header, false, "Allow to use the function `getClientHTTPHeader` which lets to obtain a value of an the current HTTP request's header. It is not enabled by default for security reasons, because some headers, such as `Cookie`, could contain sensitive info. Note that the `X-ClickHouse-*` and `Authentication` headers are always restricted and cannot be obtained with this function.", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS. diff --git a/src/Functions/getClientHTTPHeader.cpp b/src/Functions/getClientHTTPHeader.cpp new file mode 100644 index 00000000000..081be6b14c9 --- /dev/null +++ b/src/Functions/getClientHTTPHeader.cpp @@ -0,0 +1,94 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int FUNCTION_NOT_ALLOWED; +} + +namespace +{ + +class FunctionGetClientHTTPHeader : public IFunction, WithContext +{ +private: + +public: + explicit FunctionGetClientHTTPHeader(ContextPtr context_) + : WithContext(context_) + { + if (!getContext()->getSettingsRef().allow_get_http_client_header) + throw Exception(ErrorCodes::FUNCTION_NOT_ALLOWED, "The function {} requires setting `allow_get_client_http_header` to be enabled.", getName()); + } + + String getName() const override { return "getClientHTTPHeader"; } + + bool useDefaultImplementationForConstants() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo &) const override { return false; } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isString(arguments[0])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The argument of function {} must be String", getName()); + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + const ClientInfo & client_info = getContext()->getClientInfo(); + + const auto & source = arguments[0].column; + auto result = result_type->createColumn(); + result->reserve(input_rows_count); + + for (size_t row = 0; row < input_rows_count; ++row) + { + Field header; + source->get(row, header); + if (auto it = client_info.http_headers.find(header.get()); it != client_info.http_headers.end()) + result->insert(it->second); + else + result->insertDefault(); + } + + return result; + } +}; + +} + +REGISTER_FUNCTION(GetClientHTTPHeader) +{ + factory.registerFunction("getClientHTTPHeader", + [](ContextPtr context) { return std::make_shared(context); }, + FunctionDocumentation{ + .description = R"( +Get the value of an HTTP header. + +If there no such header or the current request is not performed via the HTTP interface, the function returns an empty string. +Certain HTTP headers (e.g., `Authentication` and `X-ClickHouse-*`) are restricted. + +The function requires the setting `allow_get_client_http_header` to be enabled. +The setting is not enabled by default for security reasons, because some headers, such as `Cookie`, could contain sensitive info. +", + .syntax = "getClientHTTPHeader(name)", + .arguments = {{"name", "The HTTP header name (String)"}}, + .returned_value = "The value of the header (String).", + .categories{"Miscellaneous"}}); +} + +} From 78d335155231fd54b237cba88510c57f370978f7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 23 Mar 2024 21:48:28 +0100 Subject: [PATCH 071/102] Add function getClientHTTPHeader, part 3 --- .../functions/other-functions.md | 14 +++++++ src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 1 + src/Functions/getClientHTTPHeader.cpp | 8 +++- .../03021_get_client_http_header.reference | 26 +++++++++++++ .../03021_get_client_http_header.sh | 37 +++++++++++++++++++ 6 files changed, 85 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/03021_get_client_http_header.reference create mode 100755 tests/queries/0_stateless/03021_get_client_http_header.sh diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index e7fca31483a..42e24e7389a 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -3132,3 +3132,17 @@ Result: │ (616.2931945826209,108.8825,115.6175) │ └───────────────────────────────────────┘ ``` + +## getClientHTTPHeader + +Get the value of an HTTP header. + +If there is no such header or the current request is not performed via the HTTP interface, the function returns an empty string. +Certain HTTP headers (e.g., `Authentication` and `X-ClickHouse-*`) are restricted. + +The function requires the setting `allow_get_client_http_header` to be enabled. +The setting is not enabled by default for security reasons, because some headers, such as `Cookie`, could contain sensitive info. + +HTTP headers are case sensitive for this function. + +If the function is used in the context of a distributed query, it returns non-empty result only on the initiator node. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9afa55e526e..1c07872e95a 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -895,7 +895,7 @@ class IColumn; M(Int64, ignore_cold_parts_seconds, 0, "Only available in ClickHouse Cloud. Exclude new data parts from SELECT queries until they're either pre-warmed (see cache_populated_by_fetch) or this many seconds old. Only for Replicated-/SharedMergeTree.", 0) \ M(Int64, prefer_warmed_unmerged_parts_seconds, 0, "Only available in ClickHouse Cloud. If a merged part is less than this many seconds old and is not pre-warmed (see cache_populated_by_fetch), but all its source parts are available and pre-warmed, SELECT queries will read from those parts instead. Only for ReplicatedMergeTree. Note that this only checks whether CacheWarmer processed the part; if the part was fetched into cache by something else, it'll still be considered cold until CacheWarmer gets to it; if it was warmed, then evicted from cache, it'll still be considered warm.", 0) \ M(Bool, iceberg_engine_ignore_schema_evolution, false, "Ignore schema evolution in Iceberg table engine and read all data using latest schema saved on table creation. Note that it can lead to incorrect result", 0) \ - M(Bool, allow_get_http_client_header, false, "Allow to use the function `getClientHTTPHeader` which lets to obtain a value of an the current HTTP request's header. It is not enabled by default for security reasons, because some headers, such as `Cookie`, could contain sensitive info. Note that the `X-ClickHouse-*` and `Authentication` headers are always restricted and cannot be obtained with this function.", 0) \ + M(Bool, allow_get_client_http_header, false, "Allow to use the function `getClientHTTPHeader` which lets to obtain a value of an the current HTTP request's header. It is not enabled by default for security reasons, because some headers, such as `Cookie`, could contain sensitive info. Note that the `X-ClickHouse-*` and `Authentication` headers are always restricted and cannot be obtained with this function.", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS. diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index ee56b86d9f2..5f29aa1a7f7 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -105,6 +105,7 @@ static std::map sett {"keeper_retry_max_backoff_ms", 5000, 5000, "Max backoff timeout for general keeper operations"}, {"s3queue_allow_experimental_sharded_mode", false, false, "Enable experimental sharded mode of S3Queue table engine. It is experimental because it will be rewritten"}, {"merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability", 0.0, 0.0, "For testing of `PartsSplitter` - split read ranges into intersecting and non intersecting every time you read from MergeTree with the specified probability."}, + {"allow_get_client_http_header", false, false, "Introduced a new function."}, }}, {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, diff --git a/src/Functions/getClientHTTPHeader.cpp b/src/Functions/getClientHTTPHeader.cpp index 081be6b14c9..3879d241451 100644 --- a/src/Functions/getClientHTTPHeader.cpp +++ b/src/Functions/getClientHTTPHeader.cpp @@ -26,7 +26,7 @@ public: explicit FunctionGetClientHTTPHeader(ContextPtr context_) : WithContext(context_) { - if (!getContext()->getSettingsRef().allow_get_http_client_header) + if (!getContext()->getSettingsRef().allow_get_client_http_header) throw Exception(ErrorCodes::FUNCTION_NOT_ALLOWED, "The function {} requires setting `allow_get_client_http_header` to be enabled.", getName()); } @@ -79,11 +79,15 @@ REGISTER_FUNCTION(GetClientHTTPHeader) .description = R"( Get the value of an HTTP header. -If there no such header or the current request is not performed via the HTTP interface, the function returns an empty string. +If there is no such header or the current request is not performed via the HTTP interface, the function returns an empty string. Certain HTTP headers (e.g., `Authentication` and `X-ClickHouse-*`) are restricted. The function requires the setting `allow_get_client_http_header` to be enabled. The setting is not enabled by default for security reasons, because some headers, such as `Cookie`, could contain sensitive info. + +HTTP headers are case sensitive for this function. + +If the function is used in the context of a distributed query, it returns non-empty result only on the initiator node. ", .syntax = "getClientHTTPHeader(name)", .arguments = {{"name", "The HTTP header name (String)"}}, diff --git a/tests/queries/0_stateless/03021_get_client_http_header.reference b/tests/queries/0_stateless/03021_get_client_http_header.reference new file mode 100644 index 00000000000..b331692824f --- /dev/null +++ b/tests/queries/0_stateless/03021_get_client_http_header.reference @@ -0,0 +1,26 @@ +-- It works. +application/x-www-form-urlencoded +-- It supports non constant arguments. +application/x-www-form-urlencoded +*/* +-- Empty string for non-existent headers. +application/x-www-form-urlencoded + +-- I can use my own headers. +wtf +-- Some headers cannot be obtained. + + +-- The setting matters. +FUNCTION_NOT_ALLOWED +-- The setting is not enabled by default. +FUNCTION_NOT_ALLOWED +-- Are headers case-sentitive? +application/x-www-form-urlencoded + +-- Using it from non-HTTP does not make sense. + + +-- Does it work for distributed queries? (not yet, but maybe it will be needed later) +application/x-www-form-urlencoded + diff --git a/tests/queries/0_stateless/03021_get_client_http_header.sh b/tests/queries/0_stateless/03021_get_client_http_header.sh new file mode 100755 index 00000000000..605bc85d4bf --- /dev/null +++ b/tests/queries/0_stateless/03021_get_client_http_header.sh @@ -0,0 +1,37 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +echo "-- It works." +${CLICKHOUSE_CURL} "${CLICKHOUSE_URL}&allow_get_client_http_header=1" -d "SELECT getClientHTTPHeader('Content-Type')" + +echo "-- It supports non constant arguments." +${CLICKHOUSE_CURL} "${CLICKHOUSE_URL}&allow_get_client_http_header=1" -d "SELECT getClientHTTPHeader(arrayJoin(['Content-Type', 'Accept']))" + +echo "-- Empty string for non-existent headers." +${CLICKHOUSE_CURL} "${CLICKHOUSE_URL}&allow_get_client_http_header=1" -d "SELECT getClientHTTPHeader(arrayJoin(['Content-Type', 'Upyachka']))" + +echo "-- I can use my own headers." +${CLICKHOUSE_CURL} -H 'Upyachka: wtf' "${CLICKHOUSE_URL}&allow_get_client_http_header=1" -d "SELECT getClientHTTPHeader('Upyachka')" + +echo "-- Some headers cannot be obtained." +${CLICKHOUSE_CURL} -H 'X-ClickHouse-WTF: Secret' "${CLICKHOUSE_URL}&allow_get_client_http_header=1" -d "SELECT getClientHTTPHeader('X-ClickHouse-WTF')" +${CLICKHOUSE_CURL} -H 'Authentication: Secret' "${CLICKHOUSE_URL}&allow_get_client_http_header=1" -d "SELECT getClientHTTPHeader('Authentication')" + +echo "-- The setting matters." +${CLICKHOUSE_CURL} "${CLICKHOUSE_URL}&allow_get_client_http_header=0" -d "SELECT getClientHTTPHeader(arrayJoin(['Content-Type', 'Accept']))" | grep -o -F 'FUNCTION_NOT_ALLOWED' + +echo "-- The setting is not enabled by default." +${CLICKHOUSE_CURL} "${CLICKHOUSE_URL}" -d "SELECT getClientHTTPHeader(arrayJoin(['Content-Type', 'Accept']))" | grep -o -F 'FUNCTION_NOT_ALLOWED' + +echo "-- Are headers case-sentitive?" +${CLICKHOUSE_CURL} "${CLICKHOUSE_URL}&allow_get_client_http_header=1" -d "SELECT getClientHTTPHeader(arrayJoin(['Content-Type', 'content-type']))" + +echo "-- Using it from non-HTTP does not make sense." +${CLICKHOUSE_CLIENT} --allow_get_client_http_header true --query "SELECT getClientHTTPHeader('Host')" +${CLICKHOUSE_LOCAL} --allow_get_client_http_header true --query "SELECT getClientHTTPHeader('Host')" + +echo "-- Does it work for distributed queries? (not yet, but maybe it will be needed later)" +${CLICKHOUSE_CURL} "${CLICKHOUSE_URL}&allow_get_client_http_header=1" -d "SELECT getClientHTTPHeader(name) FROM clusterAllReplicas('test_cluster_one_shard_two_replicas', view(SELECT 'Content-Type' AS name))" From 78eb81825bf58541c60f8e29610cecb019a0b38b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 23 Mar 2024 23:43:35 +0100 Subject: [PATCH 072/102] Update tests --- tests/queries/0_stateless/00725_comment_columns_long.sql | 1 + .../00753_system_columns_and_system_tables_long.sql | 1 + tests/queries/0_stateless/01666_blns_long.sql | 2 +- .../0_stateless/02421_formats_with_totals_and_extremes.sql.j2 | 3 +-- 4 files changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/00725_comment_columns_long.sql b/tests/queries/0_stateless/00725_comment_columns_long.sql index 139f8ba006f..068f35b2e40 100644 --- a/tests/queries/0_stateless/00725_comment_columns_long.sql +++ b/tests/queries/0_stateless/00725_comment_columns_long.sql @@ -1,5 +1,6 @@ -- Tags: long, no-replicated-database -- Tag no-replicated-database: Unsupported type of ALTER query +SET output_format_pretty_row_numbers = 0; DROP TABLE IF EXISTS check_query_comment_column; diff --git a/tests/queries/0_stateless/00753_system_columns_and_system_tables_long.sql b/tests/queries/0_stateless/00753_system_columns_and_system_tables_long.sql index 51818228913..4613576cf4e 100644 --- a/tests/queries/0_stateless/00753_system_columns_and_system_tables_long.sql +++ b/tests/queries/0_stateless/00753_system_columns_and_system_tables_long.sql @@ -1,4 +1,5 @@ -- Tags: long, no-s3-storage, no-random-merge-tree-settings +SET output_format_pretty_row_numbers = 0; DROP TABLE IF EXISTS check_system_tables; diff --git a/tests/queries/0_stateless/01666_blns_long.sql b/tests/queries/0_stateless/01666_blns_long.sql index 01295b11138..d039c01c580 100644 --- a/tests/queries/0_stateless/01666_blns_long.sql +++ b/tests/queries/0_stateless/01666_blns_long.sql @@ -26,7 +26,7 @@ OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ -SET max_insert_threads = 0; +SET max_insert_threads = 0, output_format_pretty_row_numbers = 0; DROP TABLE IF EXISTS test; diff --git a/tests/queries/0_stateless/02421_formats_with_totals_and_extremes.sql.j2 b/tests/queries/0_stateless/02421_formats_with_totals_and_extremes.sql.j2 index f936501e72a..319d88f8d7c 100644 --- a/tests/queries/0_stateless/02421_formats_with_totals_and_extremes.sql.j2 +++ b/tests/queries/0_stateless/02421_formats_with_totals_and_extremes.sql.j2 @@ -1,6 +1,6 @@ -- Tags: no-fasttest -set output_format_pretty_color=1; +set output_format_pretty_color = 1, output_format_pretty_row_numbers = 0; set output_format_write_statistics=0; {% for format in ['CSV', 'TSV', 'XML', 'Vertical', 'Pretty', 'JSON', 'JSONCompact'] -%} @@ -20,4 +20,3 @@ select sum(number) from numbers(10) group by number % 2 with totals format {{ fo select ''; {% endfor -%} - From 1acdf691ad73d87066dc364b3fb9471a3196c77c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Mar 2024 00:59:23 +0100 Subject: [PATCH 073/102] Update test --- tests/queries/0_stateless/03022_highlight_digit_groups.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03022_highlight_digit_groups.sql b/tests/queries/0_stateless/03022_highlight_digit_groups.sql index 57341f228a9..c48a02e712f 100644 --- a/tests/queries/0_stateless/03022_highlight_digit_groups.sql +++ b/tests/queries/0_stateless/03022_highlight_digit_groups.sql @@ -1,3 +1,5 @@ +SET output_format_pretty_row_numbers = 0; + SELECT exp10(number) * (number % 2 ? 1 : -1) FROM numbers(30) FORMAT PrettySpace SETTINGS output_format_pretty_color = 1; SELECT exp10(number) FROM numbers(10) FORMAT PrettySpace SETTINGS output_format_pretty_color = 1, output_format_pretty_highlight_digit_groups = 0; From 5efe36bc7f0f5d32c5025cb021624284e723d238 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Mar 2024 21:37:21 +0100 Subject: [PATCH 074/102] Fix test --- .../queries/0_stateless/03021_get_client_http_header.reference | 2 +- tests/queries/0_stateless/03021_get_client_http_header.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03021_get_client_http_header.reference b/tests/queries/0_stateless/03021_get_client_http_header.reference index b331692824f..099bf1fa4e3 100644 --- a/tests/queries/0_stateless/03021_get_client_http_header.reference +++ b/tests/queries/0_stateless/03021_get_client_http_header.reference @@ -22,5 +22,5 @@ application/x-www-form-urlencoded -- Does it work for distributed queries? (not yet, but maybe it will be needed later) -application/x-www-form-urlencoded + diff --git a/tests/queries/0_stateless/03021_get_client_http_header.sh b/tests/queries/0_stateless/03021_get_client_http_header.sh index 605bc85d4bf..71486c725b4 100755 --- a/tests/queries/0_stateless/03021_get_client_http_header.sh +++ b/tests/queries/0_stateless/03021_get_client_http_header.sh @@ -34,4 +34,4 @@ ${CLICKHOUSE_CLIENT} --allow_get_client_http_header true --query "SELECT getClie ${CLICKHOUSE_LOCAL} --allow_get_client_http_header true --query "SELECT getClientHTTPHeader('Host')" echo "-- Does it work for distributed queries? (not yet, but maybe it will be needed later)" -${CLICKHOUSE_CURL} "${CLICKHOUSE_URL}&allow_get_client_http_header=1" -d "SELECT getClientHTTPHeader(name) FROM clusterAllReplicas('test_cluster_one_shard_two_replicas', view(SELECT 'Content-Type' AS name))" +${CLICKHOUSE_CURL} "${CLICKHOUSE_URL}&allow_get_client_http_header=1&prefer_localhost_replica=0" -d "SELECT getClientHTTPHeader(name) FROM clusterAllReplicas('test_cluster_one_shard_two_replicas', view(SELECT 'Content-Type' AS name))" From 65650809b4dc5e8888c5529f7f85828d1b60c5b7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Mar 2024 23:11:35 +0100 Subject: [PATCH 075/102] Double precision of geoDistance if the arguments are Float64 --- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.h | 1 + src/Functions/greatCircleDistance.cpp | 366 ++++++++++++++------------ src/IO/VarInt.h | 6 +- 4 files changed, 201 insertions(+), 173 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9b7c81ffd17..57a72799c23 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -896,6 +896,7 @@ class IColumn; M(Int64, ignore_cold_parts_seconds, 0, "Only available in ClickHouse Cloud. Exclude new data parts from SELECT queries until they're either pre-warmed (see cache_populated_by_fetch) or this many seconds old. Only for Replicated-/SharedMergeTree.", 0) \ M(Int64, prefer_warmed_unmerged_parts_seconds, 0, "Only available in ClickHouse Cloud. If a merged part is less than this many seconds old and is not pre-warmed (see cache_populated_by_fetch), but all its source parts are available and pre-warmed, SELECT queries will read from those parts instead. Only for ReplicatedMergeTree. Note that this only checks whether CacheWarmer processed the part; if the part was fetched into cache by something else, it'll still be considered cold until CacheWarmer gets to it; if it was warmed, then evicted from cache, it'll still be considered warm.", 0) \ M(Bool, iceberg_engine_ignore_schema_evolution, false, "Ignore schema evolution in Iceberg table engine and read all data using latest schema saved on table creation. Note that it can lead to incorrect result", 0) \ + M(Bool, geo_distance_returns_float64_on_float64_arguments, true, "If all four arguments to `geoDistance`, `greatCircleDistance`, `greatCircleAngle` functions are Float64, return Float64 and use double precision for internal calculations. In previous ClickHouse versions, the functions always returned Float32.", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS. diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 9bf0288ebc7..a2d947aa44b 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -113,6 +113,7 @@ static std::map sett {"output_format_parquet_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, {"output_format_orc_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, {"output_format_pretty_highlight_digit_groups", false, true, "If enabled and if output is a terminal, highlight every digit corresponding to the number of thousands, millions, etc. with underline."}, + {"geo_distance_returns_float64_on_float64_arguments", false, true, "Increase the default precision."}, }}, {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, diff --git a/src/Functions/greatCircleDistance.cpp b/src/Functions/greatCircleDistance.cpp index d1d1a101187..9d97f8560a8 100644 --- a/src/Functions/greatCircleDistance.cpp +++ b/src/Functions/greatCircleDistance.cpp @@ -42,121 +42,6 @@ namespace ErrorCodes namespace { -constexpr double PI = std::numbers::pi_v; -constexpr float PI_F = std::numbers::pi_v; - -constexpr float RAD_IN_DEG = static_cast(PI / 180.0); -constexpr float RAD_IN_DEG_HALF = static_cast(PI / 360.0); - -constexpr size_t COS_LUT_SIZE = 1024; // maxerr 0.00063% -constexpr float COS_LUT_SIZE_F = 1024.0f; // maxerr 0.00063% -constexpr size_t ASIN_SQRT_LUT_SIZE = 512; -constexpr size_t METRIC_LUT_SIZE = 1024; - -/** Earth radius in meters using WGS84 authalic radius. - * We use this value to be consistent with H3 library. - */ -constexpr float EARTH_RADIUS = 6371007.180918475f; -constexpr float EARTH_DIAMETER = 2 * EARTH_RADIUS; - - -float cos_lut[COS_LUT_SIZE + 1]; /// cos(x) table -float asin_sqrt_lut[ASIN_SQRT_LUT_SIZE + 1]; /// asin(sqrt(x)) * earth_diameter table - -float sphere_metric_lut[METRIC_LUT_SIZE + 1]; /// sphere metric, unitless: the distance in degrees for one degree across longitude depending on latitude -float sphere_metric_meters_lut[METRIC_LUT_SIZE + 1]; /// sphere metric: the distance in meters for one degree across longitude depending on latitude -float wgs84_metric_meters_lut[2 * (METRIC_LUT_SIZE + 1)]; /// ellipsoid metric: the distance in meters across one degree latitude/longitude depending on latitude - - -inline double sqr(double v) -{ - return v * v; -} - -inline float sqrf(float v) -{ - return v * v; -} - -void geodistInit() -{ - for (size_t i = 0; i <= COS_LUT_SIZE; ++i) - cos_lut[i] = static_cast(cos(2 * PI * i / COS_LUT_SIZE)); // [0, 2 * pi] -> [0, COS_LUT_SIZE] - - for (size_t i = 0; i <= ASIN_SQRT_LUT_SIZE; ++i) - asin_sqrt_lut[i] = static_cast(asin( - sqrt(static_cast(i) / ASIN_SQRT_LUT_SIZE))); // [0, 1] -> [0, ASIN_SQRT_LUT_SIZE] - - for (size_t i = 0; i <= METRIC_LUT_SIZE; ++i) - { - double latitude = i * (PI / METRIC_LUT_SIZE) - PI * 0.5; // [-pi / 2, pi / 2] -> [0, METRIC_LUT_SIZE] - - /// Squared metric coefficients (for the distance in meters) on a tangent plane, for latitude and longitude (in degrees), - /// depending on the latitude (in radians). - - /// https://github.com/mapbox/cheap-ruler/blob/master/index.js#L67 - wgs84_metric_meters_lut[i * 2] = static_cast(sqr(111132.09 - 566.05 * cos(2 * latitude) + 1.20 * cos(4 * latitude))); - wgs84_metric_meters_lut[i * 2 + 1] = static_cast(sqr(111415.13 * cos(latitude) - 94.55 * cos(3 * latitude) + 0.12 * cos(5 * latitude))); - - sphere_metric_meters_lut[i] = static_cast(sqr((EARTH_DIAMETER * PI / 360) * cos(latitude))); - - sphere_metric_lut[i] = static_cast(sqr(cos(latitude))); - } -} - -inline NO_SANITIZE_UNDEFINED size_t floatToIndex(float x) -{ - /// Implementation specific behaviour on overflow or infinite value. - return static_cast(x); -} - -inline float geodistDegDiff(float f) -{ - f = fabsf(f); - if (f > 180) - f = 360 - f; - return f; -} - -inline float geodistFastCos(float x) -{ - float y = fabsf(x) * (COS_LUT_SIZE_F / PI_F / 2.0f); - size_t i = floatToIndex(y); - y -= i; - i &= (COS_LUT_SIZE - 1); - return cos_lut[i] + (cos_lut[i + 1] - cos_lut[i]) * y; -} - -inline float geodistFastSin(float x) -{ - float y = fabsf(x) * (COS_LUT_SIZE_F / PI_F / 2.0f); - size_t i = floatToIndex(y); - y -= i; - i = (i - COS_LUT_SIZE / 4) & (COS_LUT_SIZE - 1); // cos(x - pi / 2) = sin(x), costable / 4 = pi / 2 - return cos_lut[i] + (cos_lut[i + 1] - cos_lut[i]) * y; -} - -/// fast implementation of asin(sqrt(x)) -/// max error in floats 0.00369%, in doubles 0.00072% -inline float geodistFastAsinSqrt(float x) -{ - if (x < 0.122f) - { - // distance under 4546 km, Taylor error under 0.00072% - float y = sqrtf(x); - return y + x * y * 0.166666666666666f + x * x * y * 0.075f + x * x * x * y * 0.044642857142857f; - } - if (x < 0.948f) - { - // distance under 17083 km, 512-entry LUT error under 0.00072% - x *= ASIN_SQRT_LUT_SIZE; - size_t i = floatToIndex(x); - return asin_sqrt_lut[i] + (asin_sqrt_lut[i + 1] - asin_sqrt_lut[i]) * (x - i); - } - return asinf(sqrtf(x)); // distance over 17083 km, just compute exact -} - - enum class Method { SPHERE_DEGREES, @@ -164,18 +49,123 @@ enum class Method WGS84_METERS, }; -} +constexpr size_t ASIN_SQRT_LUT_SIZE = 512; +constexpr size_t COS_LUT_SIZE = 1024; // maxerr 0.00063% +constexpr size_t METRIC_LUT_SIZE = 1024; + +template +struct Impl +{ + static constexpr T PI = std::numbers::pi_v; + static constexpr T RAD_IN_DEG = static_cast(PI / T(180.0)); + static constexpr T RAD_IN_DEG_HALF = static_cast(PI / T(360.0)); + + static constexpr T COS_LUT_SIZE_F = T(1024.0); + + /** Earth radius in meters using WGS84 authalic radius. + * We use this value to be consistent with H3 library. + */ + static constexpr T EARTH_RADIUS = T(6371007.180918475); + static constexpr T EARTH_DIAMETER = 2 * EARTH_RADIUS; + + T cos_lut[COS_LUT_SIZE + 1]; /// cos(x) table + T asin_sqrt_lut[ASIN_SQRT_LUT_SIZE + 1]; /// asin(sqrt(x)) * earth_diameter table + + T sphere_metric_lut[METRIC_LUT_SIZE + 1]; /// sphere metric, unitless: the distance in degrees for one degree across longitude depending on latitude + T sphere_metric_meters_lut[METRIC_LUT_SIZE + 1]; /// sphere metric: the distance in meters for one degree across longitude depending on latitude + T wgs84_metric_meters_lut[2 * (METRIC_LUT_SIZE + 1)]; /// ellipsoid metric: the distance in meters across one degree latitude/longitude depending on latitude + + static T sqr(T v) { return v * v; } + + Impl() + { + for (size_t i = 0; i <= COS_LUT_SIZE; ++i) + cos_lut[i] = std::cos(2 * PI * i / COS_LUT_SIZE); // [0, 2 * pi] -> [0, COS_LUT_SIZE] + + for (size_t i = 0; i <= ASIN_SQRT_LUT_SIZE; ++i) + asin_sqrt_lut[i] = std::asin(std::sqrt(static_cast(i) / ASIN_SQRT_LUT_SIZE)); // [0, 1] -> [0, ASIN_SQRT_LUT_SIZE] + + for (size_t i = 0; i <= METRIC_LUT_SIZE; ++i) + { + T latitude = i * (PI / METRIC_LUT_SIZE) - PI * T(0.5); // [-pi / 2, pi / 2] -> [0, METRIC_LUT_SIZE] + + /// Squared metric coefficients (for the distance in meters) on a tangent plane, for latitude and longitude (in degrees), + /// depending on the latitude (in radians). + + /// https://github.com/mapbox/cheap-ruler/blob/master/index.js#L67 + wgs84_metric_meters_lut[i * 2] = sqr(T(111132.09) - T(566.05) * std::cos(T(2.0) * latitude) + T(1.20) * std::cos(T(4.0) * latitude)); + wgs84_metric_meters_lut[i * 2 + 1] = sqr(T(111415.13) * std::cos(latitude) - T(94.55) * std::cos(T(3.0) * latitude) + T(0.12) * std::cos(T(5.0) * latitude)); + sphere_metric_meters_lut[i] = sqr((EARTH_DIAMETER * PI / 360) * std::cos(latitude)); + + sphere_metric_lut[i] = sqr(std::cos(latitude)); + } + } + + static inline NO_SANITIZE_UNDEFINED size_t toIndex(T x) + { + /// Implementation specific behaviour on overflow or infinite value. + return static_cast(x); + } + + static inline T degDiff(T f) + { + f = std::abs(f); + if (f > 180) + f = 360 - f; + return f; + } + + inline T fastCos(T x) + { + T y = std::abs(x) * (COS_LUT_SIZE_F / PI / T(2.0)); + size_t i = toIndex(y); + y -= i; + i &= (COS_LUT_SIZE - 1); + return cos_lut[i] + (cos_lut[i + 1] - cos_lut[i]) * y; + } + + inline T fastSin(T x) + { + T y = std::abs(x) * (COS_LUT_SIZE_F / PI / T(2.0)); + size_t i = toIndex(y); + y -= i; + i = (i - COS_LUT_SIZE / 4) & (COS_LUT_SIZE - 1); // cos(x - pi / 2) = sin(x), costable / 4 = pi / 2 + return cos_lut[i] + (cos_lut[i + 1] - cos_lut[i]) * y; + } + + /// fast implementation of asin(sqrt(x)) + /// max error in floats 0.00369%, in doubles 0.00072% + inline T fastAsinSqrt(T x) + { + if (x < T(0.122)) + { + // distance under 4546 km, Taylor error under 0.00072% + T y = std::sqrt(x); + return y + x * y * T(0.166666666666666) + x * x * y * T(0.075) + x * x * x * y * T(0.044642857142857); + } + if (x < T(0.948)) + { + // distance under 17083 km, 512-entry LUT error under 0.00072% + x *= ASIN_SQRT_LUT_SIZE; + size_t i = toIndex(x); + return asin_sqrt_lut[i] + (asin_sqrt_lut[i + 1] - asin_sqrt_lut[i]) * (x - i); + } + return std::asin(std::sqrt(x)); /// distance is over 17083 km, just compute exact + } +}; + +template Impl impl; DECLARE_MULTITARGET_CODE( namespace { -template -float distance(float lon1deg, float lat1deg, float lon2deg, float lat2deg) +template +T distance(T lon1deg, T lat1deg, T lon2deg, T lat2deg) { - float lat_diff = geodistDegDiff(lat1deg - lat2deg); - float lon_diff = geodistDegDiff(lon1deg - lon2deg); + T lat_diff = impl.degDiff(lat1deg - lat2deg); + T lon_diff = impl.degDiff(lon1deg - lon2deg); if (lon_diff < 13) { @@ -187,51 +177,51 @@ float distance(float lon1deg, float lat1deg, float lon2deg, float lat2deg) /// (Remember how a plane flies from Amsterdam to New York) /// But if longitude is close but latitude is different enough, there is no difference between meridian and great circle line. - float latitude_midpoint = (lat1deg + lat2deg + 180) * METRIC_LUT_SIZE / 360; // [-90, 90] degrees -> [0, METRIC_LUT_SIZE] indexes - size_t latitude_midpoint_index = floatToIndex(latitude_midpoint) & (METRIC_LUT_SIZE - 1); + T latitude_midpoint = (lat1deg + lat2deg + 180) * METRIC_LUT_SIZE / 360; // [-90, 90] degrees -> [0, METRIC_LUT_SIZE] indexes + size_t latitude_midpoint_index = impl.toIndex(latitude_midpoint) & (METRIC_LUT_SIZE - 1); /// This is linear interpolation between two table items at index "latitude_midpoint_index" and "latitude_midpoint_index + 1". - float k_lat{}; - float k_lon{}; + T k_lat{}; + T k_lon{}; if constexpr (method == Method::SPHERE_DEGREES) { k_lat = 1; - k_lon = sphere_metric_lut[latitude_midpoint_index] - + (sphere_metric_lut[latitude_midpoint_index + 1] - sphere_metric_lut[latitude_midpoint_index]) * (latitude_midpoint - latitude_midpoint_index); + k_lon = impl.sphere_metric_lut[latitude_midpoint_index] + + (impl.sphere_metric_lut[latitude_midpoint_index + 1] - impl.sphere_metric_lut[latitude_midpoint_index]) * (latitude_midpoint - latitude_midpoint_index); } else if constexpr (method == Method::SPHERE_METERS) { - k_lat = sqrf(EARTH_DIAMETER * PI_F / 360.0f); + k_lat = impl.sqr(impl.EARTH_DIAMETER * impl.PI / T(360.0)); - k_lon = sphere_metric_meters_lut[latitude_midpoint_index] - + (sphere_metric_meters_lut[latitude_midpoint_index + 1] - sphere_metric_meters_lut[latitude_midpoint_index]) * (latitude_midpoint - latitude_midpoint_index); + k_lon = impl.sphere_metric_meters_lut[latitude_midpoint_index] + + (impl.sphere_metric_meters_lut[latitude_midpoint_index + 1] - impl.sphere_metric_meters_lut[latitude_midpoint_index]) * (latitude_midpoint - latitude_midpoint_index); } else if constexpr (method == Method::WGS84_METERS) { - k_lat = wgs84_metric_meters_lut[latitude_midpoint_index * 2] - + (wgs84_metric_meters_lut[(latitude_midpoint_index + 1) * 2] - wgs84_metric_meters_lut[latitude_midpoint_index * 2]) * (latitude_midpoint - latitude_midpoint_index); + k_lat = impl.wgs84_metric_meters_lut[latitude_midpoint_index * 2] + + (impl.wgs84_metric_meters_lut[(latitude_midpoint_index + 1) * 2] - impl.wgs84_metric_meters_lut[latitude_midpoint_index * 2]) * (latitude_midpoint - latitude_midpoint_index); - k_lon = wgs84_metric_meters_lut[latitude_midpoint_index * 2 + 1] - + (wgs84_metric_meters_lut[(latitude_midpoint_index + 1) * 2 + 1] - wgs84_metric_meters_lut[latitude_midpoint_index * 2 + 1]) * (latitude_midpoint - latitude_midpoint_index); + k_lon = impl.wgs84_metric_meters_lut[latitude_midpoint_index * 2 + 1] + + (impl.wgs84_metric_meters_lut[(latitude_midpoint_index + 1) * 2 + 1] - impl.wgs84_metric_meters_lut[latitude_midpoint_index * 2 + 1]) * (latitude_midpoint - latitude_midpoint_index); } /// Metric on a tangent plane: it differs from Euclidean metric only by scale of coordinates. - return sqrtf(k_lat * lat_diff * lat_diff + k_lon * lon_diff * lon_diff); + return std::sqrt(k_lat * lat_diff * lat_diff + k_lon * lon_diff * lon_diff); } else { // points too far away; use haversine - float a = sqrf(geodistFastSin(lat_diff * RAD_IN_DEG_HALF)) - + geodistFastCos(lat1deg * RAD_IN_DEG) * geodistFastCos(lat2deg * RAD_IN_DEG) * sqrf(geodistFastSin(lon_diff * RAD_IN_DEG_HALF)); + T a = impl.sqr(impl.fastSin(lat_diff * impl.RAD_IN_DEG_HALF)) + + impl.fastCos(lat1deg * impl.RAD_IN_DEG) * impl.fastCos(lat2deg * impl.RAD_IN_DEG) * impl.sqr(impl.fastSin(lon_diff * impl.RAD_IN_DEG_HALF)); if constexpr (method == Method::SPHERE_DEGREES) - return (360.0f / PI_F) * geodistFastAsinSqrt(a); + return (T(360.0) / impl.PI) * impl.fastAsinSqrt(a); else - return EARTH_DIAMETER * geodistFastAsinSqrt(a); + return impl.EARTH_DIAMETER * impl.fastAsinSqrt(a); } } @@ -241,13 +231,24 @@ template class FunctionGeoDistance : public IFunction { public: - static constexpr auto name = - (method == Method::SPHERE_DEGREES) ? "greatCircleAngle" - : ((method == Method::SPHERE_METERS) ? "greatCircleDistance" - : "geoDistance"); + FunctionGeoDistance(ContextPtr context) + { + always_float32 = !context->getSettingsRef().geo_distance_returns_float64_on_float64_arguments; + } private: - String getName() const override { return name; } + bool always_float32; + + String getName() const override + { + if constexpr (method == Method::SPHERE_DEGREES) + return "greatCircleAngle"; + if constexpr (method == Method::SPHERE_METERS) + return "greatCircleDistance"; + else + return "geoDistance"; + } + size_t getNumberOfArguments() const override { return 4; } bool useDefaultImplementationForConstants() const override { return true; } @@ -255,22 +256,31 @@ private: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - for (const auto arg_idx : collections::range(0, arguments.size())) + bool has_float64 = false; + + for (size_t arg_idx = 0; arg_idx < 4; ++arg_idx) { - const auto * arg = arguments[arg_idx].get(); - if (!isNumber(WhichDataType(arg))) + WhichDataType which(arguments[arg_idx]); + + if (!isNumber(which)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument {} of function {}. " - "Must be numeric", arg->getName(), std::to_string(arg_idx + 1), getName()); + "Must be numeric", arguments[arg_idx]->getName(), std::to_string(arg_idx + 1), getName()); + + if (which.isFloat64()) + has_float64 = true; } - return std::make_shared(); + if (has_float64 && !always_float32) + return std::make_shared(); + else + return std::make_shared(); } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { - auto dst = ColumnVector::create(); - auto & dst_data = dst->getData(); - dst_data.resize(input_rows_count); + bool returns_float64 = WhichDataType(result_type).isFloat64(); + + auto dst = result_type->createColumn(); auto arguments_copy = arguments; for (auto & argument : arguments_copy) @@ -280,10 +290,24 @@ private: argument.type = result_type; } - const auto * col_lon1 = convertArgumentColumnToFloat32(arguments_copy, 0); - const auto * col_lat1 = convertArgumentColumnToFloat32(arguments_copy, 1); - const auto * col_lon2 = convertArgumentColumnToFloat32(arguments_copy, 2); - const auto * col_lat2 = convertArgumentColumnToFloat32(arguments_copy, 3); + if (returns_float64) + run(arguments_copy, dst, input_rows_count); + else + run(arguments_copy, dst, input_rows_count); + + return dst; + } + + template + void run(const ColumnsWithTypeAndName & arguments, MutableColumnPtr & dst, size_t input_rows_count) const + { + const auto * col_lon1 = convertArgumentColumn(arguments, 0); + const auto * col_lat1 = convertArgumentColumn(arguments, 1); + const auto * col_lon2 = convertArgumentColumn(arguments, 2); + const auto * col_lat2 = convertArgumentColumn(arguments, 3); + + auto & dst_data = assert_cast &>(*dst).getData(); + dst_data.resize(input_rows_count); for (size_t row_num = 0; row_num < input_rows_count; ++row_num) { @@ -291,20 +315,20 @@ private: col_lon1->getData()[row_num], col_lat1->getData()[row_num], col_lon2->getData()[row_num], col_lat2->getData()[row_num]); } - - return dst; } - const ColumnFloat32 * convertArgumentColumnToFloat32(const ColumnsWithTypeAndName & arguments, size_t argument_index) const + template + const ColumnVector * convertArgumentColumn(const ColumnsWithTypeAndName & arguments, size_t argument_index) const { - const auto * column_typed = checkAndGetColumn(arguments[argument_index].column.get()); + const auto * column_typed = checkAndGetColumn>(arguments[argument_index].column.get()); if (!column_typed) throw Exception( ErrorCodes::ILLEGAL_COLUMN, - "Illegal type {} of argument {} of function {}. Must be Float32.", + "Illegal type {} of argument {} of function {}. Must be {}.", arguments[argument_index].type->getName(), argument_index + 1, - getName()); + getName(), + TypeName); return column_typed; } @@ -316,18 +340,19 @@ template class FunctionGeoDistance : public TargetSpecific::Default::FunctionGeoDistance { public: - explicit FunctionGeoDistance(ContextPtr context) : selector(context) + explicit FunctionGeoDistance(ContextPtr context) + : TargetSpecific::Default::FunctionGeoDistance(context), selector(context) { selector.registerImplementation>(); + TargetSpecific::Default::FunctionGeoDistance>(context); #if USE_MULTITARGET_CODE selector.registerImplementation>(); + TargetSpecific::AVX::FunctionGeoDistance>(context); selector.registerImplementation>(); + TargetSpecific::AVX2::FunctionGeoDistance>(context); selector.registerImplementation>(); + TargetSpecific::AVX512F::FunctionGeoDistance>(context); #endif } @@ -345,12 +370,13 @@ private: ImplementationSelector selector; }; +} + REGISTER_FUNCTION(GeoDistance) { - geodistInit(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); + factory.registerFunction("greatCircleAngle", [](ContextPtr context) { return std::make_shared>(std::move(context)); }); + factory.registerFunction("greatCircleDistance", [](ContextPtr context) { return std::make_shared>(std::move(context)); }); + factory.registerFunction("geoDistance", [](ContextPtr context) { return std::make_shared>(std::move(context)); }); } } diff --git a/src/IO/VarInt.h b/src/IO/VarInt.h index 12a86ac55f6..9e72705341d 100644 --- a/src/IO/VarInt.h +++ b/src/IO/VarInt.h @@ -79,7 +79,7 @@ inline char * writeVarInt(Int64 x, char * ostr) return writeVarUInt(static_cast((x << 1) ^ (x >> 63)), ostr); } -namespace impl +namespace varint_impl { template @@ -106,8 +106,8 @@ inline void readVarUInt(UInt64 & x, ReadBuffer & istr) inline void readVarUInt(UInt64 & x, ReadBuffer & istr) { if (istr.buffer().end() - istr.position() >= 10) - return impl::readVarUInt(x, istr); - return impl::readVarUInt(x, istr); + return varint_impl::readVarUInt(x, istr); + return varint_impl::readVarUInt(x, istr); } inline void readVarUInt(UInt64 & x, std::istream & istr) From 06a9ac712645bb3275174152558e7b0680ee31a5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Mar 2024 23:23:13 +0100 Subject: [PATCH 076/102] Fix tests --- tests/integration/test_storage_kafka/test.py | 2 +- tests/integration/test_storage_rabbitmq/test.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 5bdea179449..081b15520a1 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -4452,7 +4452,7 @@ def test_block_based_formats_1(kafka_cluster): kafka_group_name = '{topic}', kafka_format = 'PrettySpace'; - INSERT INTO test.kafka SELECT number * 10 as key, number * 100 as value FROM numbers(5) settings max_block_size=2, optimize_trivial_insert_select=0, output_format_pretty_color=1; + INSERT INTO test.kafka SELECT number * 10 as key, number * 100 as value FROM numbers(5) settings max_block_size=2, optimize_trivial_insert_select=0, output_format_pretty_color=1, output_format_pretty_row_numbers=0; """ ) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 0f1c5eb17dd..1017e58e50c 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -3202,7 +3202,7 @@ def test_block_based_formats_1(rabbitmq_cluster): ) instance.query( - "INSERT INTO test.rabbitmq SELECT number * 10 as key, number * 100 as value FROM numbers(5) settings max_block_size=2, optimize_trivial_insert_select=0, output_format_pretty_color=1;" + "INSERT INTO test.rabbitmq SELECT number * 10 as key, number * 100 as value FROM numbers(5) settings max_block_size=2, optimize_trivial_insert_select=0, output_format_pretty_color=1, output_format_pretty_row_numbers=0;" ) insert_messages = [] From a2a90467ed6f0dac325c39bc0255412aa2855e4e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 25 Mar 2024 03:11:19 +0100 Subject: [PATCH 077/102] Better --- src/Functions/greatCircleDistance.cpp | 57 +++++++++---------- .../0_stateless/01043_geo_distance.sql | 2 + .../0_stateless/01678_great_circle_angle.sql | 2 + 3 files changed, 31 insertions(+), 30 deletions(-) diff --git a/src/Functions/greatCircleDistance.cpp b/src/Functions/greatCircleDistance.cpp index 9d97f8560a8..0f39cc648dd 100644 --- a/src/Functions/greatCircleDistance.cpp +++ b/src/Functions/greatCircleDistance.cpp @@ -53,51 +53,45 @@ constexpr size_t ASIN_SQRT_LUT_SIZE = 512; constexpr size_t COS_LUT_SIZE = 1024; // maxerr 0.00063% constexpr size_t METRIC_LUT_SIZE = 1024; +/// Earth radius in meters using WGS84 authalic radius. +/// We use this value to be consistent with H3 library. +constexpr double EARTH_RADIUS = 6371007.180918475; +constexpr double EARTH_DIAMETER = 2.0 * EARTH_RADIUS; +constexpr double PI = std::numbers::pi_v; + +template +T sqr(T v) { return v * v; } + template struct Impl { - static constexpr T PI = std::numbers::pi_v; - static constexpr T RAD_IN_DEG = static_cast(PI / T(180.0)); - static constexpr T RAD_IN_DEG_HALF = static_cast(PI / T(360.0)); - - static constexpr T COS_LUT_SIZE_F = T(1024.0); - - /** Earth radius in meters using WGS84 authalic radius. - * We use this value to be consistent with H3 library. - */ - static constexpr T EARTH_RADIUS = T(6371007.180918475); - static constexpr T EARTH_DIAMETER = 2 * EARTH_RADIUS; - T cos_lut[COS_LUT_SIZE + 1]; /// cos(x) table T asin_sqrt_lut[ASIN_SQRT_LUT_SIZE + 1]; /// asin(sqrt(x)) * earth_diameter table - T sphere_metric_lut[METRIC_LUT_SIZE + 1]; /// sphere metric, unitless: the distance in degrees for one degree across longitude depending on latitude T sphere_metric_meters_lut[METRIC_LUT_SIZE + 1]; /// sphere metric: the distance in meters for one degree across longitude depending on latitude T wgs84_metric_meters_lut[2 * (METRIC_LUT_SIZE + 1)]; /// ellipsoid metric: the distance in meters across one degree latitude/longitude depending on latitude - static T sqr(T v) { return v * v; } - Impl() { for (size_t i = 0; i <= COS_LUT_SIZE; ++i) - cos_lut[i] = std::cos(2 * PI * i / COS_LUT_SIZE); // [0, 2 * pi] -> [0, COS_LUT_SIZE] + cos_lut[i] = T(std::cos(2 * PI * static_cast(i) / COS_LUT_SIZE)); // [0, 2 * pi] -> [0, COS_LUT_SIZE] for (size_t i = 0; i <= ASIN_SQRT_LUT_SIZE; ++i) - asin_sqrt_lut[i] = std::asin(std::sqrt(static_cast(i) / ASIN_SQRT_LUT_SIZE)); // [0, 1] -> [0, ASIN_SQRT_LUT_SIZE] + asin_sqrt_lut[i] = T(std::asin(std::sqrt(static_cast(i) / ASIN_SQRT_LUT_SIZE))); // [0, 1] -> [0, ASIN_SQRT_LUT_SIZE] for (size_t i = 0; i <= METRIC_LUT_SIZE; ++i) { - T latitude = i * (PI / METRIC_LUT_SIZE) - PI * T(0.5); // [-pi / 2, pi / 2] -> [0, METRIC_LUT_SIZE] + double latitude = i * (PI / METRIC_LUT_SIZE) - PI * 0.5; // [-pi / 2, pi / 2] -> [0, METRIC_LUT_SIZE] /// Squared metric coefficients (for the distance in meters) on a tangent plane, for latitude and longitude (in degrees), /// depending on the latitude (in radians). /// https://github.com/mapbox/cheap-ruler/blob/master/index.js#L67 - wgs84_metric_meters_lut[i * 2] = sqr(T(111132.09) - T(566.05) * std::cos(T(2.0) * latitude) + T(1.20) * std::cos(T(4.0) * latitude)); - wgs84_metric_meters_lut[i * 2 + 1] = sqr(T(111415.13) * std::cos(latitude) - T(94.55) * std::cos(T(3.0) * latitude) + T(0.12) * std::cos(T(5.0) * latitude)); - sphere_metric_meters_lut[i] = sqr((EARTH_DIAMETER * PI / 360) * std::cos(latitude)); + wgs84_metric_meters_lut[i * 2] = T(sqr(111132.09 - 566.05 * std::cos(2.0 * latitude) + 1.20 * std::cos(4.0 * latitude))); + wgs84_metric_meters_lut[i * 2 + 1] = T(sqr(111415.13 * std::cos(latitude) - 94.55 * std::cos(3.0 * latitude) + 0.12 * std::cos(5.0 * latitude))); + sphere_metric_meters_lut[i] = T(sqr((EARTH_DIAMETER * PI / 360) * std::cos(latitude))); - sphere_metric_lut[i] = sqr(std::cos(latitude)); + sphere_metric_lut[i] = T(sqr(std::cos(latitude))); } } @@ -117,7 +111,7 @@ struct Impl inline T fastCos(T x) { - T y = std::abs(x) * (COS_LUT_SIZE_F / PI / T(2.0)); + T y = std::abs(x) * (T(COS_LUT_SIZE) / T(PI) / T(2.0)); size_t i = toIndex(y); y -= i; i &= (COS_LUT_SIZE - 1); @@ -126,7 +120,7 @@ struct Impl inline T fastSin(T x) { - T y = std::abs(x) * (COS_LUT_SIZE_F / PI / T(2.0)); + T y = std::abs(x) * (T(COS_LUT_SIZE) / T(PI) / T(2.0)); size_t i = toIndex(y); y -= i; i = (i - COS_LUT_SIZE / 4) & (COS_LUT_SIZE - 1); // cos(x - pi / 2) = sin(x), costable / 4 = pi / 2 @@ -194,7 +188,7 @@ T distance(T lon1deg, T lat1deg, T lon2deg, T lat2deg) } else if constexpr (method == Method::SPHERE_METERS) { - k_lat = impl.sqr(impl.EARTH_DIAMETER * impl.PI / T(360.0)); + k_lat = sqr(T(EARTH_DIAMETER) * T(PI) / T(360.0)); k_lon = impl.sphere_metric_meters_lut[latitude_midpoint_index] + (impl.sphere_metric_meters_lut[latitude_midpoint_index + 1] - impl.sphere_metric_meters_lut[latitude_midpoint_index]) * (latitude_midpoint - latitude_midpoint_index); @@ -213,15 +207,18 @@ T distance(T lon1deg, T lat1deg, T lon2deg, T lat2deg) } else { - // points too far away; use haversine + /// Points are too far away: use Haversine. - T a = impl.sqr(impl.fastSin(lat_diff * impl.RAD_IN_DEG_HALF)) - + impl.fastCos(lat1deg * impl.RAD_IN_DEG) * impl.fastCos(lat2deg * impl.RAD_IN_DEG) * impl.sqr(impl.fastSin(lon_diff * impl.RAD_IN_DEG_HALF)); + static constexpr T RAD_IN_DEG = T(PI / 180.0); + static constexpr T RAD_IN_DEG_HALF = T(PI / 360.0); + + T a = sqr(impl.fastSin(lat_diff * RAD_IN_DEG_HALF)) + + impl.fastCos(lat1deg * RAD_IN_DEG) * impl.fastCos(lat2deg * RAD_IN_DEG) * sqr(impl.fastSin(lon_diff * RAD_IN_DEG_HALF)); if constexpr (method == Method::SPHERE_DEGREES) - return (T(360.0) / impl.PI) * impl.fastAsinSqrt(a); + return (T(360.0) / T(PI)) * impl.fastAsinSqrt(a); else - return impl.EARTH_DIAMETER * impl.fastAsinSqrt(a); + return T(EARTH_DIAMETER) * impl.fastAsinSqrt(a); } } diff --git a/tests/queries/0_stateless/01043_geo_distance.sql b/tests/queries/0_stateless/01043_geo_distance.sql index c1fb29b9eb7..5897e1fae9b 100644 --- a/tests/queries/0_stateless/01043_geo_distance.sql +++ b/tests/queries/0_stateless/01043_geo_distance.sql @@ -1,3 +1,5 @@ +SET geo_distance_returns_float64_on_float64_arguments = 0; + SELECT greatCircleDistance(0., 0., 0., 1.); SELECT greatCircleDistance(0., 89., 0, 90.); diff --git a/tests/queries/0_stateless/01678_great_circle_angle.sql b/tests/queries/0_stateless/01678_great_circle_angle.sql index 124c7bfadf2..bcf0d751e1c 100644 --- a/tests/queries/0_stateless/01678_great_circle_angle.sql +++ b/tests/queries/0_stateless/01678_great_circle_angle.sql @@ -1,3 +1,5 @@ +SET geo_distance_returns_float64_on_float64_arguments = 0; + SELECT round(greatCircleAngle(0, 45, 0.1, 45.1), 4); SELECT round(greatCircleAngle(0, 45, 1, 45), 4); SELECT round(greatCircleAngle(0, 45, 1, 45.1), 4); From c7a48f601ae0c822371f38010543ee7f297e6142 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 25 Mar 2024 03:12:11 +0100 Subject: [PATCH 078/102] More tests --- src/Functions/greatCircleDistance.cpp | 1 - .../0_stateless/01043_geo_distance.reference | 8 ++++++++ tests/queries/0_stateless/01043_geo_distance.sql | 14 ++++++++++++++ .../0_stateless/01678_great_circle_angle.reference | 5 +++++ .../0_stateless/01678_great_circle_angle.sql | 9 +++++++++ 5 files changed, 36 insertions(+), 1 deletion(-) diff --git a/src/Functions/greatCircleDistance.cpp b/src/Functions/greatCircleDistance.cpp index 0f39cc648dd..1b5fe1ac35b 100644 --- a/src/Functions/greatCircleDistance.cpp +++ b/src/Functions/greatCircleDistance.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include diff --git a/tests/queries/0_stateless/01043_geo_distance.reference b/tests/queries/0_stateless/01043_geo_distance.reference index cd8a8a6dfe9..33e1285872e 100644 --- a/tests/queries/0_stateless/01043_geo_distance.reference +++ b/tests/queries/0_stateless/01043_geo_distance.reference @@ -6,3 +6,11 @@ 10007555 10007554 10001780 +111195.05197522942 +111195.05197522942 +110567.32686882635 +111699.2516454354 +10007554.677770648 +10007554.677770648 +10007554.677770648 +10001780.1 diff --git a/tests/queries/0_stateless/01043_geo_distance.sql b/tests/queries/0_stateless/01043_geo_distance.sql index 5897e1fae9b..cf877d05b9f 100644 --- a/tests/queries/0_stateless/01043_geo_distance.sql +++ b/tests/queries/0_stateless/01043_geo_distance.sql @@ -11,3 +11,17 @@ SELECT greatCircleDistance(0., 0., 0., 90.); SELECT geoDistance(0., 0., 90., 0.); SELECT geoDistance(0., 0., 0., 90.); + +SET geo_distance_returns_float64_on_float64_arguments = 1; + +SELECT greatCircleDistance(0., 0., 0., 1.); +SELECT greatCircleDistance(0., 89., 0, 90.); + +SELECT geoDistance(0., 0., 0., 1.); +SELECT geoDistance(0., 89., 0., 90.); + +SELECT greatCircleDistance(0., 0., 90., 0.); +SELECT greatCircleDistance(0., 0., 0., 90.); + +SELECT geoDistance(0., 0., 90., 0.); +SELECT geoDistance(0., 0., 0., 90.); diff --git a/tests/queries/0_stateless/01678_great_circle_angle.reference b/tests/queries/0_stateless/01678_great_circle_angle.reference index f3382476d4a..a409e1d84b8 100644 --- a/tests/queries/0_stateless/01678_great_circle_angle.reference +++ b/tests/queries/0_stateless/01678_great_circle_angle.reference @@ -3,3 +3,8 @@ 0.7135 10007555 10007554 +0.1224 +0.7071 +0.7135 +10007555 +10007554 diff --git a/tests/queries/0_stateless/01678_great_circle_angle.sql b/tests/queries/0_stateless/01678_great_circle_angle.sql index bcf0d751e1c..595622822f1 100644 --- a/tests/queries/0_stateless/01678_great_circle_angle.sql +++ b/tests/queries/0_stateless/01678_great_circle_angle.sql @@ -6,3 +6,12 @@ SELECT round(greatCircleAngle(0, 45, 1, 45.1), 4); SELECT round(greatCircleDistance(0, 0, 0, 90), 4); SELECT round(greatCircleDistance(0, 0, 90, 0), 4); + +SET geo_distance_returns_float64_on_float64_arguments = 1; + +SELECT round(greatCircleAngle(0, 45, 0.1, 45.1), 4); +SELECT round(greatCircleAngle(0, 45, 1, 45), 4); +SELECT round(greatCircleAngle(0, 45, 1, 45.1), 4); + +SELECT round(greatCircleDistance(0, 0, 0, 90), 4); +SELECT round(greatCircleDistance(0, 0, 90, 0), 4); From 8cf4c3bd3f263f5862571c99d2e89c79f39d4593 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 25 Mar 2024 03:28:37 +0100 Subject: [PATCH 079/102] Fix tidy --- src/Functions/getClientHTTPHeader.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Functions/getClientHTTPHeader.cpp b/src/Functions/getClientHTTPHeader.cpp index 3879d241451..ebd070a90b9 100644 --- a/src/Functions/getClientHTTPHeader.cpp +++ b/src/Functions/getClientHTTPHeader.cpp @@ -20,14 +20,12 @@ namespace class FunctionGetClientHTTPHeader : public IFunction, WithContext { -private: - public: explicit FunctionGetClientHTTPHeader(ContextPtr context_) : WithContext(context_) { if (!getContext()->getSettingsRef().allow_get_client_http_header) - throw Exception(ErrorCodes::FUNCTION_NOT_ALLOWED, "The function {} requires setting `allow_get_client_http_header` to be enabled.", getName()); + throw Exception(ErrorCodes::FUNCTION_NOT_ALLOWED, "The function getClientHTTPHeader requires setting `allow_get_client_http_header` to be enabled."); } String getName() const override { return "getClientHTTPHeader"; } From 271342a871cfcddfe65ca99331a8022351d117f5 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 4 Mar 2024 15:07:20 +0000 Subject: [PATCH 080/102] change column name from duration_ms to duration_us in system.zookeeper Signed-off-by: Duc Canh Le --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 22 +++++++++++----------- src/Interpreters/ZooKeeperLog.cpp | 4 ++-- src/Interpreters/ZooKeeperLog.h | 2 +- 3 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 8fd6e89dfd9..9f3752ede58 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -812,7 +812,7 @@ void ZooKeeper::receiveEvent() RequestInfo request_info; ZooKeeperResponsePtr response; - UInt64 elapsed_ms = 0; + UInt64 elapsed_us = 0; maybeInjectRecvFault(); @@ -875,8 +875,8 @@ void ZooKeeper::receiveEvent() CurrentMetrics::sub(CurrentMetrics::ZooKeeperRequest); } - elapsed_ms = std::chrono::duration_cast(clock::now() - request_info.time).count(); - ProfileEvents::increment(ProfileEvents::ZooKeeperWaitMicroseconds, elapsed_ms); + elapsed_us = std::chrono::duration_cast(clock::now() - request_info.time).count(); + ProfileEvents::increment(ProfileEvents::ZooKeeperWaitMicroseconds, elapsed_us); } try @@ -935,7 +935,7 @@ void ZooKeeper::receiveEvent() length, actual_length); } - logOperationIfNeeded(request_info.request, response, /* finalize= */ false, elapsed_ms); + logOperationIfNeeded(request_info.request, response, /* finalize= */ false, elapsed_us); } catch (...) { @@ -954,7 +954,7 @@ void ZooKeeper::receiveEvent() if (request_info.callback) request_info.callback(*response); - logOperationIfNeeded(request_info.request, response, /* finalize= */ false, elapsed_ms); + logOperationIfNeeded(request_info.request, response, /* finalize= */ false, elapsed_us); } catch (...) { @@ -1048,14 +1048,14 @@ void ZooKeeper::finalize(bool error_send, bool error_receive, const String & rea ? Error::ZCONNECTIONLOSS : Error::ZSESSIONEXPIRED; response->xid = request_info.request->xid; - UInt64 elapsed_ms = std::chrono::duration_cast(clock::now() - request_info.time).count(); + UInt64 elapsed_us = std::chrono::duration_cast(clock::now() - request_info.time).count(); if (request_info.callback) { try { request_info.callback(*response); - logOperationIfNeeded(request_info.request, response, true, elapsed_ms); + logOperationIfNeeded(request_info.request, response, true, elapsed_us); } catch (...) { @@ -1115,8 +1115,8 @@ void ZooKeeper::finalize(bool error_send, bool error_receive, const String & rea try { info.callback(*response); - UInt64 elapsed_ms = std::chrono::duration_cast(clock::now() - info.time).count(); - logOperationIfNeeded(info.request, response, true, elapsed_ms); + UInt64 elapsed_us = std::chrono::duration_cast(clock::now() - info.time).count(); + logOperationIfNeeded(info.request, response, true, elapsed_us); } catch (...) { @@ -1498,7 +1498,7 @@ void ZooKeeper::setZooKeeperLog(std::shared_ptr zk_log_) } #ifdef ZOOKEEPER_LOG -void ZooKeeper::logOperationIfNeeded(const ZooKeeperRequestPtr & request, const ZooKeeperResponsePtr & response, bool finalize, UInt64 elapsed_ms) +void ZooKeeper::logOperationIfNeeded(const ZooKeeperRequestPtr & request, const ZooKeeperResponsePtr & response, bool finalize, UInt64 elapsed_us) { auto maybe_zk_log = std::atomic_load(&zk_log); if (!maybe_zk_log) @@ -1536,7 +1536,7 @@ void ZooKeeper::logOperationIfNeeded(const ZooKeeperRequestPtr & request, const elem.event_time = event_time; elem.address = socket_address; elem.session_id = session_id; - elem.duration_ms = elapsed_ms; + elem.duration_us = elapsed_us; if (request) { elem.thread_id = request->thread_id; diff --git a/src/Interpreters/ZooKeeperLog.cpp b/src/Interpreters/ZooKeeperLog.cpp index 6f6d4568064..4821773bf98 100644 --- a/src/Interpreters/ZooKeeperLog.cpp +++ b/src/Interpreters/ZooKeeperLog.cpp @@ -131,7 +131,7 @@ ColumnsDescription ZooKeeperLogElement::getColumnsDescription() {"address", DataTypeFactory::instance().get("IPv6"), "IP address of ZooKeeper server that was used to make the request."}, {"port", std::make_shared(), "The port of ZooKeeper server that was used to make the request."}, {"session_id", std::make_shared(), "The session ID that the ZooKeeper server sets for each connection."}, - {"duration_ms", std::make_shared(), "The time taken by ZooKeeper to execute the request."}, + {"duration_us", std::make_shared(), "The time taken by ZooKeeper to execute the request."}, {"xid", std::make_shared(), "The ID of the request within the session. This is usually a sequential request number. It is the same for the request row and the paired response/finalize row."}, {"has_watch", std::make_shared(), "The request whether the watch has been set."}, @@ -183,7 +183,7 @@ void ZooKeeperLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insertData(IPv6ToBinary(address.host()).data(), 16); columns[i++]->insert(address.port()); columns[i++]->insert(session_id); - columns[i++]->insert(duration_ms); + columns[i++]->insert(duration_us); columns[i++]->insert(xid); columns[i++]->insert(has_watch); diff --git a/src/Interpreters/ZooKeeperLog.h b/src/Interpreters/ZooKeeperLog.h index 0d9dc618eb4..8187978bd57 100644 --- a/src/Interpreters/ZooKeeperLog.h +++ b/src/Interpreters/ZooKeeperLog.h @@ -28,7 +28,7 @@ struct ZooKeeperLogElement Poco::Net::SocketAddress address; Int64 session_id = 0; - UInt64 duration_ms = 0; + UInt64 duration_us = 0; /// Common request info Int32 xid = 0; From 29d51d71b7181a87f096b070151a8665f6529c80 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Mar 2024 03:47:52 +0300 Subject: [PATCH 081/102] Fix the name --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 22 +++++++++++----------- src/Interpreters/ZooKeeperLog.cpp | 2 +- src/Interpreters/ZooKeeperLog.h | 2 +- 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 9f3752ede58..55e33273d44 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -812,7 +812,7 @@ void ZooKeeper::receiveEvent() RequestInfo request_info; ZooKeeperResponsePtr response; - UInt64 elapsed_us = 0; + UInt64 elapsed_microseconds = 0; maybeInjectRecvFault(); @@ -875,8 +875,8 @@ void ZooKeeper::receiveEvent() CurrentMetrics::sub(CurrentMetrics::ZooKeeperRequest); } - elapsed_us = std::chrono::duration_cast(clock::now() - request_info.time).count(); - ProfileEvents::increment(ProfileEvents::ZooKeeperWaitMicroseconds, elapsed_us); + elapsed_microseconds = std::chrono::duration_cast(clock::now() - request_info.time).count(); + ProfileEvents::increment(ProfileEvents::ZooKeeperWaitMicroseconds, elapsed_microseconds); } try @@ -935,7 +935,7 @@ void ZooKeeper::receiveEvent() length, actual_length); } - logOperationIfNeeded(request_info.request, response, /* finalize= */ false, elapsed_us); + logOperationIfNeeded(request_info.request, response, /* finalize= */ false, elapsed_microseconds); } catch (...) { @@ -954,7 +954,7 @@ void ZooKeeper::receiveEvent() if (request_info.callback) request_info.callback(*response); - logOperationIfNeeded(request_info.request, response, /* finalize= */ false, elapsed_us); + logOperationIfNeeded(request_info.request, response, /* finalize= */ false, elapsed_microseconds); } catch (...) { @@ -1048,14 +1048,14 @@ void ZooKeeper::finalize(bool error_send, bool error_receive, const String & rea ? Error::ZCONNECTIONLOSS : Error::ZSESSIONEXPIRED; response->xid = request_info.request->xid; - UInt64 elapsed_us = std::chrono::duration_cast(clock::now() - request_info.time).count(); + UInt64 elapsed_microseconds = std::chrono::duration_cast(clock::now() - request_info.time).count(); if (request_info.callback) { try { request_info.callback(*response); - logOperationIfNeeded(request_info.request, response, true, elapsed_us); + logOperationIfNeeded(request_info.request, response, true, elapsed_microseconds); } catch (...) { @@ -1115,8 +1115,8 @@ void ZooKeeper::finalize(bool error_send, bool error_receive, const String & rea try { info.callback(*response); - UInt64 elapsed_us = std::chrono::duration_cast(clock::now() - info.time).count(); - logOperationIfNeeded(info.request, response, true, elapsed_us); + UInt64 elapsed_microseconds = std::chrono::duration_cast(clock::now() - info.time).count(); + logOperationIfNeeded(info.request, response, true, elapsed_microseconds); } catch (...) { @@ -1498,7 +1498,7 @@ void ZooKeeper::setZooKeeperLog(std::shared_ptr zk_log_) } #ifdef ZOOKEEPER_LOG -void ZooKeeper::logOperationIfNeeded(const ZooKeeperRequestPtr & request, const ZooKeeperResponsePtr & response, bool finalize, UInt64 elapsed_us) +void ZooKeeper::logOperationIfNeeded(const ZooKeeperRequestPtr & request, const ZooKeeperResponsePtr & response, bool finalize, UInt64 elapsed_microseconds) { auto maybe_zk_log = std::atomic_load(&zk_log); if (!maybe_zk_log) @@ -1536,7 +1536,7 @@ void ZooKeeper::logOperationIfNeeded(const ZooKeeperRequestPtr & request, const elem.event_time = event_time; elem.address = socket_address; elem.session_id = session_id; - elem.duration_us = elapsed_us; + elem.duration_us = elapsed_microseconds; if (request) { elem.thread_id = request->thread_id; diff --git a/src/Interpreters/ZooKeeperLog.cpp b/src/Interpreters/ZooKeeperLog.cpp index 4821773bf98..31a12a520f4 100644 --- a/src/Interpreters/ZooKeeperLog.cpp +++ b/src/Interpreters/ZooKeeperLog.cpp @@ -183,7 +183,7 @@ void ZooKeeperLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insertData(IPv6ToBinary(address.host()).data(), 16); columns[i++]->insert(address.port()); columns[i++]->insert(session_id); - columns[i++]->insert(duration_us); + columns[i++]->insert(elapsed_microseconds); columns[i++]->insert(xid); columns[i++]->insert(has_watch); diff --git a/src/Interpreters/ZooKeeperLog.h b/src/Interpreters/ZooKeeperLog.h index 8187978bd57..7afc1db39a6 100644 --- a/src/Interpreters/ZooKeeperLog.h +++ b/src/Interpreters/ZooKeeperLog.h @@ -28,7 +28,7 @@ struct ZooKeeperLogElement Poco::Net::SocketAddress address; Int64 session_id = 0; - UInt64 duration_us = 0; + UInt64 elapsed_microseconds = 0; /// Common request info Int32 xid = 0; From 3a5ed40eaa7b6b5db059efbf87569c616bdc789a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Mar 2024 18:45:03 +0100 Subject: [PATCH 082/102] Merge with master --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 55e33273d44..e367468d4b6 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -1536,7 +1536,7 @@ void ZooKeeper::logOperationIfNeeded(const ZooKeeperRequestPtr & request, const elem.event_time = event_time; elem.address = socket_address; elem.session_id = session_id; - elem.duration_us = elapsed_microseconds; + elem.elapsed_microseconds = elapsed_microseconds; if (request) { elem.thread_id = request->thread_id; From 4813014710fce195ae0e6834f94cb7e9ab6645ee Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Mar 2024 18:46:15 +0100 Subject: [PATCH 083/102] Merge with master --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 2 +- src/Interpreters/ZooKeeperLog.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index e367468d4b6..03d1b5a93d4 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -1536,7 +1536,7 @@ void ZooKeeper::logOperationIfNeeded(const ZooKeeperRequestPtr & request, const elem.event_time = event_time; elem.address = socket_address; elem.session_id = session_id; - elem.elapsed_microseconds = elapsed_microseconds; + elem.duration_microseconds = elapsed_microseconds; if (request) { elem.thread_id = request->thread_id; diff --git a/src/Interpreters/ZooKeeperLog.h b/src/Interpreters/ZooKeeperLog.h index 7afc1db39a6..d3868c01202 100644 --- a/src/Interpreters/ZooKeeperLog.h +++ b/src/Interpreters/ZooKeeperLog.h @@ -28,7 +28,7 @@ struct ZooKeeperLogElement Poco::Net::SocketAddress address; Int64 session_id = 0; - UInt64 elapsed_microseconds = 0; + UInt64 duration_microseconds = 0; /// Common request info Int32 xid = 0; From e3dd42131c4d1fb4222f3df7ed0cd0b3278ce481 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Mar 2024 19:01:29 +0100 Subject: [PATCH 084/102] Fix build --- src/Interpreters/ZooKeeperLog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ZooKeeperLog.cpp b/src/Interpreters/ZooKeeperLog.cpp index 31a12a520f4..a9fe2e01429 100644 --- a/src/Interpreters/ZooKeeperLog.cpp +++ b/src/Interpreters/ZooKeeperLog.cpp @@ -183,7 +183,7 @@ void ZooKeeperLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insertData(IPv6ToBinary(address.host()).data(), 16); columns[i++]->insert(address.port()); columns[i++]->insert(session_id); - columns[i++]->insert(elapsed_microseconds); + columns[i++]->insert(duration_microseconds); columns[i++]->insert(xid); columns[i++]->insert(has_watch); From 264b847f1a5bcd2784660a9400994526d94cda82 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Mar 2024 21:21:10 +0100 Subject: [PATCH 085/102] Fix build --- src/Common/ZooKeeper/ZooKeeperImpl.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index b5ab4f04b8e..cf331a03d06 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -343,7 +343,7 @@ private: void flushWriteBuffer(); ReadBuffer & getReadBuffer(); - void logOperationIfNeeded(const ZooKeeperRequestPtr & request, const ZooKeeperResponsePtr & response = nullptr, bool finalize = false, UInt64 elapsed_ms = 0); + void logOperationIfNeeded(const ZooKeeperRequestPtr & request, const ZooKeeperResponsePtr & response = nullptr, bool finalize = false, UInt64 elapsed_microseconds = 0); void initFeatureFlags(); From d488d4fac59cf808493d8cabe473751a4d290586 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 25 Mar 2024 01:52:08 +0100 Subject: [PATCH 086/102] Remove unused headers --- src/Interpreters/InterpreterSelectQuery.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 7c87dadfce6..6bbf03bb1e0 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -13,7 +13,6 @@ #include #include #include -#include #include #include @@ -73,7 +72,6 @@ #include #include -#include #include #include #include @@ -85,7 +83,6 @@ #include #include #include -#include #include #include #include From fffacf116c43cbcf7e4c25bee3f962eebf4051c3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 25 Mar 2024 03:36:31 +0100 Subject: [PATCH 087/102] Fix tests --- tests/queries/0_stateless/01158_zookeeper_log_long.reference | 2 +- tests/queries/0_stateless/01158_zookeeper_log_long.sql | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01158_zookeeper_log_long.reference b/tests/queries/0_stateless/01158_zookeeper_log_long.reference index 17c7ac84d1c..3a56c1a3d7b 100644 --- a/tests/queries/0_stateless/01158_zookeeper_log_long.reference +++ b/tests/queries/0_stateless/01158_zookeeper_log_long.reference @@ -34,5 +34,5 @@ Response 0 Error /test/01158/default/rmt/blocks/all_6308706741995381342_24957917 Response 0 Error /test/01158/default/rmt/block_numbers/all/block- 1 1 \N 0 2 ZRUNTIMEINCONSISTENCY \N \N 0 0 0 0 Request 0 Get /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 \N 0 0 \N \N \N 0 0 0 0 Response 0 Get /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 \N 0 0 ZOK \N \N 0 0 9 0 -duration_ms +duration_microseconds 1 diff --git a/tests/queries/0_stateless/01158_zookeeper_log_long.sql b/tests/queries/0_stateless/01158_zookeeper_log_long.sql index 1dd7578a3e1..55d4162fc48 100644 --- a/tests/queries/0_stateless/01158_zookeeper_log_long.sql +++ b/tests/queries/0_stateless/01158_zookeeper_log_long.sql @@ -42,5 +42,5 @@ order by xid, type, request_idx; drop table rmt sync; system flush logs; -select 'duration_ms'; -select count()>0 from system.zookeeper_log where path like '/test/01158/' || currentDatabase() || '/rmt%' and duration_ms > 0; +select 'duration_microseconds'; +select count()>0 from system.zookeeper_log where path like '/test/01158/' || currentDatabase() || '/rmt%' and duration_microseconds > 0; From cbf3b5ea024fbadff36e1d14c511ffcfdf96e80a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 25 Mar 2024 03:41:52 +0100 Subject: [PATCH 088/102] Fix error --- src/Interpreters/ZooKeeperLog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ZooKeeperLog.cpp b/src/Interpreters/ZooKeeperLog.cpp index a9fe2e01429..0d3063a569e 100644 --- a/src/Interpreters/ZooKeeperLog.cpp +++ b/src/Interpreters/ZooKeeperLog.cpp @@ -131,7 +131,7 @@ ColumnsDescription ZooKeeperLogElement::getColumnsDescription() {"address", DataTypeFactory::instance().get("IPv6"), "IP address of ZooKeeper server that was used to make the request."}, {"port", std::make_shared(), "The port of ZooKeeper server that was used to make the request."}, {"session_id", std::make_shared(), "The session ID that the ZooKeeper server sets for each connection."}, - {"duration_us", std::make_shared(), "The time taken by ZooKeeper to execute the request."}, + {"duration_microseconds", std::make_shared(), "The time taken by ZooKeeper to execute the request."}, {"xid", std::make_shared(), "The ID of the request within the session. This is usually a sequential request number. It is the same for the request row and the paired response/finalize row."}, {"has_watch", std::make_shared(), "The request whether the watch has been set."}, From 8e4c25e1c430f77fe5df4369be10c67906a5f61b Mon Sep 17 00:00:00 2001 From: LiuYuan <820738731@qq.com> Date: Fri, 22 Mar 2024 10:55:50 +0000 Subject: [PATCH 089/102] Fix RANGE frame is not supported for Nullable columns. --- src/Processors/Transforms/WindowTransform.cpp | 75 +++++++++++++++++++ .../01655_window_functions_null.reference | 5 ++ .../01655_window_functions_null.sql | 4 + 3 files changed, 84 insertions(+) create mode 100644 tests/queries/0_stateless/01655_window_functions_null.reference create mode 100644 tests/queries/0_stateless/01655_window_functions_null.sql diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 47b5b900400..02d2762dab3 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -172,6 +173,79 @@ static int compareValuesWithOffsetFloat(const IColumn * _compared_column, return result; } +// Helper macros to dispatch on type of the ORDER BY column +#define APPLY_FOR_ONE_NEST_TYPE(FUNCTION, TYPE) \ +else if (typeid_cast(nest_compared_column.get())) \ +{ \ + /* clang-tidy you're dumb, I can't put FUNCTION in braces here. */ \ + nest_compare_function = FUNCTION; /* NOLINT */ \ +} + +#define APPLY_FOR_NEST_TYPES(FUNCTION) \ +if (false) /* NOLINT */ \ +{ \ + /* Do nothing, a starter condition. */ \ +} \ +APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ +APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ +APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ +APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ +\ +APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ +APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ +APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ +APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ +APPLY_FOR_ONE_NEST_TYPE(FUNCTION, ColumnVector) \ +\ +APPLY_FOR_ONE_NEST_TYPE(FUNCTION##Float, ColumnVector) \ +APPLY_FOR_ONE_NEST_TYPE(FUNCTION##Float, ColumnVector) \ +\ +else \ +{ \ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, \ + "The RANGE OFFSET frame for '{}' ORDER BY nest column is not implemented", \ + demangle(typeid(nest_compared_column).name())); \ +} + +// A specialization of compareValuesWithOffset for nullable. +template +static int compareValuesWithOffsetNullable(const IColumn * _compared_column, + size_t compared_row, const IColumn * _reference_column, + size_t reference_row, + const Field & _offset, + bool offset_is_preceding) +{ + const auto * compared_column = assert_cast( + _compared_column); + const auto * reference_column = assert_cast( + _reference_column); + + if (compared_column->isNullAt(compared_row) && !reference_column->isNullAt(reference_row)) + { + return -1; + } + else if (compared_column->isNullAt(compared_row) && reference_column->isNullAt(reference_row)) + { + return 0; + } + else if (!compared_column->isNullAt(compared_row) && reference_column->isNullAt(reference_row)) + { + return 1; + } + + ColumnPtr nest_compared_column = compared_column->getNestedColumnPtr(); + ColumnPtr nest_reference_column = reference_column->getNestedColumnPtr(); + + std::function nest_compare_function; + APPLY_FOR_NEST_TYPES(compareValuesWithOffset) + return nest_compare_function(nest_compared_column.get(), compared_row, + nest_reference_column.get(), reference_row, _offset, offset_is_preceding); +} + // Helper macros to dispatch on type of the ORDER BY column #define APPLY_FOR_ONE_TYPE(FUNCTION, TYPE) \ else if (typeid_cast(column)) \ @@ -199,6 +273,7 @@ APPLY_FOR_ONE_TYPE(FUNCTION, ColumnVector) \ APPLY_FOR_ONE_TYPE(FUNCTION##Float, ColumnVector) \ APPLY_FOR_ONE_TYPE(FUNCTION##Float, ColumnVector) \ \ +APPLY_FOR_ONE_TYPE(FUNCTION##Nullable, ColumnNullable) \ else \ { \ throw Exception(ErrorCodes::NOT_IMPLEMENTED, \ diff --git a/tests/queries/0_stateless/01655_window_functions_null.reference b/tests/queries/0_stateless/01655_window_functions_null.reference new file mode 100644 index 00000000000..1cb15a4e849 --- /dev/null +++ b/tests/queries/0_stateless/01655_window_functions_null.reference @@ -0,0 +1,5 @@ +1 4 +1 4 +2 5 +3 3 +\N \N diff --git a/tests/queries/0_stateless/01655_window_functions_null.sql b/tests/queries/0_stateless/01655_window_functions_null.sql new file mode 100644 index 00000000000..a3971898adb --- /dev/null +++ b/tests/queries/0_stateless/01655_window_functions_null.sql @@ -0,0 +1,4 @@ +SELECT + number, + sum(number) OVER (ORDER BY number ASC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) AS sum +FROM values('number Nullable(Int8)', 1, 1, 2, 3, NULL) \ No newline at end of file From 4c9ea0075b3f399aef17622ce19c01721e7bdca7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 25 Mar 2024 09:49:59 +0100 Subject: [PATCH 090/102] Update documentation for SYSTEM FLUSH DISTRIBUTED SETTINGS Signed-off-by: Azat Khuzhin --- docs/en/sql-reference/statements/system.md | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index a128814f072..1dee2eac698 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -180,10 +180,16 @@ SYSTEM STOP DISTRIBUTED SENDS [db.] [ON CLUSTER cluster_ Forces ClickHouse to send data to cluster nodes synchronously. If any nodes are unavailable, ClickHouse throws an exception and stops query execution. You can retry the query until it succeeds, which will happen when all nodes are back online. +You can also override some settings via `SETTINGS` clause, this can be useful to avoid some temporary limitations, like `max_concurrent_queries_for_all_users` or `max_memory_usage`. + ``` sql -SYSTEM FLUSH DISTRIBUTED [db.] [ON CLUSTER cluster_name] +SYSTEM FLUSH DISTRIBUTED [db.] [ON CLUSTER cluster_name] [SETTINGS ...] ``` +:::note +Each pending block is stored in disk with settings from the initial INSERT query, so that is why sometimes you may want to override settings. +::: + ### START DISTRIBUTED SENDS Enables background data distribution when inserting data into distributed tables. From d859e4368f0be13f268b706ddfa3b4f4e682249d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 25 Mar 2024 10:12:10 +0100 Subject: [PATCH 091/102] doc: move ENABLE_LIBRARIES=OFF into proper section Signed-off-by: Azat Khuzhin --- docs/en/development/developer-instruction.md | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index e08096d8042..26ef1506b90 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -153,6 +153,18 @@ Upon successful build you get an executable file `ClickHouse//program ls -l programs/clickhouse +### Advanced Building Process {#advanced-building-process} + +#### Minimal Build {#minimal-build} + +If you are not interested in functionality provided by third-party libraries, you can further speed up the build using `cmake` options + +``` +cmake -DENABLE_LIBRARIES=OFF +``` + +In case of problems with any of the development options, you are on your own! + ## Running the Built Executable of ClickHouse {#running-the-built-executable-of-clickhouse} To run the server under the current user you need to navigate to `ClickHouse/programs/server/` (located outside of `build`) and run: @@ -250,10 +262,3 @@ Most probably some of the builds will fail at first times. This is due to the fa You can use GitHub integrated code browser [here](https://github.dev/ClickHouse/ClickHouse). Also, you can browse sources on [GitHub](https://github.com/ClickHouse/ClickHouse) as usual. - -If you are not interested in functionality provided by third-party libraries, you can further speed up the build using `cmake` options -``` --DENABLE_LIBRARIES=0 -``` - -In case of problems with any of the development options, you are on your own! From 62994ab0c1b9e4e5a07a3731e1ee4dd239304be9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 25 Mar 2024 10:12:32 +0100 Subject: [PATCH 092/102] doc: add a note how to disable Rust support Signed-off-by: Azat Khuzhin --- docs/en/development/developer-instruction.md | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index 26ef1506b90..42c7e5ac295 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -165,6 +165,14 @@ cmake -DENABLE_LIBRARIES=OFF In case of problems with any of the development options, you are on your own! +#### Rust support {#rust-support} + +Rust requires internet connection, in case you don't have it, you can disable Rust support: + +``` +cmake -DENABLE_RUST=OFF +``` + ## Running the Built Executable of ClickHouse {#running-the-built-executable-of-clickhouse} To run the server under the current user you need to navigate to `ClickHouse/programs/server/` (located outside of `build`) and run: From 78656c691b57dbdbc90bd398db3c631b9ad88fdc Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 25 Mar 2024 12:11:23 +0100 Subject: [PATCH 093/102] Revert "Fix flaky `02122_parallel_formatting_Template`" --- tests/queries/0_stateless/02122_parallel_formatting_Template.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02122_parallel_formatting_Template.sh b/tests/queries/0_stateless/02122_parallel_formatting_Template.sh index 86a4c017bbe..c7d2329ec14 100755 --- a/tests/queries/0_stateless/02122_parallel_formatting_Template.sh +++ b/tests/queries/0_stateless/02122_parallel_formatting_Template.sh @@ -21,7 +21,7 @@ $CLICKHOUSE_CLIENT -q "select number as x, number + 1 as y, concat('string: ', t diff $non_parallel_file $parallel_file -echo -ne '{prefix} \n${data}\n $$ suffix $$\n${totals}\n${min}\n${max}\n${rows:Quoted}\n${rows_before_limit:Quoted}\n' > $resultset_path +echo -ne '{prefix} \n${data}\n $$ suffix $$\n${totals}\n${min}\n${max}\n${rows:Quoted}\n${rows_before_limit:Quoted}\n${rows_read:Quoted}\n${bytes_read:Quoted}\n' > $resultset_path echo "Template-2" $CLICKHOUSE_CLIENT -q "select number as x, number + 1 as y, concat('string: ', toString(number)) as s from numbers(200000) group by number with totals order by number limit 190000 format Template $TEMPLATE_SETTINGS" --output_format_parallel_formatting=0 --extremes=1 > $non_parallel_file From d45a6fe9b9187e27405bfe90be840cf03b279aae Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 25 Mar 2024 15:46:55 +0100 Subject: [PATCH 094/102] Fix tidy --- src/Functions/greatCircleDistance.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/greatCircleDistance.cpp b/src/Functions/greatCircleDistance.cpp index 1b5fe1ac35b..01184f74b13 100644 --- a/src/Functions/greatCircleDistance.cpp +++ b/src/Functions/greatCircleDistance.cpp @@ -227,7 +227,7 @@ template class FunctionGeoDistance : public IFunction { public: - FunctionGeoDistance(ContextPtr context) + explicit FunctionGeoDistance(ContextPtr context) { always_float32 = !context->getSettingsRef().geo_distance_returns_float64_on_float64_arguments; } From 9d17f3106c60aa387be0318bc28c6a27b13c7bac Mon Sep 17 00:00:00 2001 From: Max K Date: Mon, 25 Mar 2024 17:53:01 +0100 Subject: [PATCH 095/102] Revert "Add --now option to enable and start the service" --- packages/clickhouse-server.postinstall | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/packages/clickhouse-server.postinstall b/packages/clickhouse-server.postinstall index 41d4405a790..d3b49db758f 100644 --- a/packages/clickhouse-server.postinstall +++ b/packages/clickhouse-server.postinstall @@ -36,7 +36,7 @@ if [ "$1" = configure ] || [ -n "$not_deb_os" ]; then fi /bin/systemctl daemon-reload - /bin/systemctl enable --now clickhouse-server + /bin/systemctl enable clickhouse-server else # If you downgrading to version older than 1.1.54336 run: systemctl disable clickhouse-server if [ -x "/etc/init.d/clickhouse-server" ]; then From 727aeb7e37d50c69f22941b77c517703a384060f Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Mon, 25 Mar 2024 16:39:50 +0000 Subject: [PATCH 096/102] CI: set timeout for install check #job_Install_packages_amd64 #job_style_check --- tests/ci/ci_config.py | 15 ++++++++------- tests/ci/ci_utils.py | 12 ++++++++++++ tests/ci/install_check.py | 6 +++++- 3 files changed, 25 insertions(+), 8 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 573976f1f84..b4fe6b516fc 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -220,7 +220,7 @@ class JobConfig: digest: DigestConfig = field(default_factory=DigestConfig) # will be triggered for the job if omited in CI workflow yml run_command: str = "" - # job timeout + # job timeout, seconds timeout: Optional[int] = None # sets number of batches for multi-batch job num_batches: int = 1 @@ -517,10 +517,11 @@ clickbench_test_params = { ), "run_command": 'clickbench.py "$CHECK_NAME"', } -install_test_params = { - "digest": install_check_digest, - "run_command": 'install_check.py "$CHECK_NAME"', -} +install_test_params = JobConfig( + digest=install_check_digest, + run_command='install_check.py "$CHECK_NAME"', + timeout=900, +) @dataclass @@ -1105,10 +1106,10 @@ CI_CONFIG = CIConfig( }, test_configs={ JobNames.INSTALL_TEST_AMD: TestConfig( - Build.PACKAGE_RELEASE, job_config=JobConfig(**install_test_params) # type: ignore + Build.PACKAGE_RELEASE, job_config=install_test_params ), JobNames.INSTALL_TEST_ARM: TestConfig( - Build.PACKAGE_AARCH64, job_config=JobConfig(**install_test_params) # type: ignore + Build.PACKAGE_AARCH64, job_config=install_test_params ), JobNames.STATEFUL_TEST_ASAN: TestConfig( Build.PACKAGE_ASAN, job_config=JobConfig(**stateful_test_common_params) # type: ignore diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index 2967ec2f309..97d42f9845b 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -1,5 +1,6 @@ from contextlib import contextmanager import os +import signal from typing import Any, List, Union, Iterator from pathlib import Path @@ -48,3 +49,14 @@ class GHActions: for line in lines: print(line) print("::endgroup::") + + +def set_job_timeout(): + def timeout_handler(_signum, _frame): + print("Timeout expired") + raise TimeoutError("Job's KILL_TIMEOUT expired") + + kill_timeout = int(os.getenv("KILL_TIMEOUT", "0")) + assert kill_timeout > 0, "kill timeout must be provided in KILL_TIMEOUT env" + signal.signal(signal.SIGALRM, timeout_handler) + signal.alarm(kill_timeout) diff --git a/tests/ci/install_check.py b/tests/ci/install_check.py index 4fc112c6d9f..71e2198f533 100644 --- a/tests/ci/install_check.py +++ b/tests/ci/install_check.py @@ -14,10 +14,11 @@ from build_download_helper import download_builds_filter from compress_files import compress_fast from docker_images_helper import DockerImage, pull_image, get_docker_image -from env_helper import REPORT_PATH, TEMP_PATH as TEMP +from env_helper import CI, REPORT_PATH, TEMP_PATH as TEMP from report import JobReport, TestResults, TestResult, FAILURE, FAIL, OK, SUCCESS from stopwatch import Stopwatch from tee_popen import TeePopen +from ci_utils import set_job_timeout RPM_IMAGE = "clickhouse/install-rpm-test" @@ -255,6 +256,9 @@ def main(): args = parse_args() + if CI: + set_job_timeout() + TEMP_PATH.mkdir(parents=True, exist_ok=True) LOGS_PATH.mkdir(parents=True, exist_ok=True) From 11bc6c3e6e7ba947abe52320edaf0c367751fc2f Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 25 Mar 2024 18:43:29 +0100 Subject: [PATCH 097/102] Test added --- .../03031_low_cardinality_logical_error.reference | 6 ++++++ .../03031_low_cardinality_logical_error.sql | 14 ++++++++++++++ 2 files changed, 20 insertions(+) create mode 100644 tests/queries/0_stateless/03031_low_cardinality_logical_error.reference create mode 100644 tests/queries/0_stateless/03031_low_cardinality_logical_error.sql diff --git a/tests/queries/0_stateless/03031_low_cardinality_logical_error.reference b/tests/queries/0_stateless/03031_low_cardinality_logical_error.reference new file mode 100644 index 00000000000..931811041f2 --- /dev/null +++ b/tests/queries/0_stateless/03031_low_cardinality_logical_error.reference @@ -0,0 +1,6 @@ +[] 0 ['2'] +['0'] 2 ['0'] +['0'] 2 ['0'] +['1'] 1 [] + +[] 3 [] diff --git a/tests/queries/0_stateless/03031_low_cardinality_logical_error.sql b/tests/queries/0_stateless/03031_low_cardinality_logical_error.sql new file mode 100644 index 00000000000..02ef0585b00 --- /dev/null +++ b/tests/queries/0_stateless/03031_low_cardinality_logical_error.sql @@ -0,0 +1,14 @@ +SELECT * +FROM ( + SELECT + ([toString(number % 2)] :: Array(LowCardinality(String))) AS item_id, + count() + FROM numbers(3) + GROUP BY item_id WITH TOTALS +) AS l FULL JOIN ( + SELECT + ([toString((number % 2) * 2)] :: Array(String)) AS item_id + FROM numbers(3) +) AS r +ON l.item_id = r.item_id +ORDER BY 1,2,3; From f9f830228a71225841b04d0eec39d51d399be116 Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Mon, 25 Mar 2024 16:08:53 +0000 Subject: [PATCH 098/102] CI: MQ support --- .github/workflows/pull_request.yml | 11 +++-------- tests/ci/ci.py | 27 ++++++--------------------- tests/ci/pr_info.py | 1 + 3 files changed, 10 insertions(+), 29 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 2dddde9aa14..ff0adee1443 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -6,6 +6,7 @@ env: PYTHONUNBUFFERED: 1 on: # yamllint disable-line rule:truthy + merge_group: pull_request: types: - synchronize @@ -29,6 +30,7 @@ jobs: fetch-depth: 0 # to get version filter: tree:0 - name: Labels check + if: ${{ github.event_name != 'merge_group' }} run: | cd "$GITHUB_WORKSPACE/tests/ci" python3 run_check.py @@ -56,16 +58,9 @@ jobs: echo 'EOF' } >> "$GITHUB_OUTPUT" - name: Re-create GH statuses for skipped jobs if any + if: ${{ github.event_name != 'merge_group' }} run: | python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --infile ${{ runner.temp }}/ci_run_data.json --update-gh-statuses - - name: Style check early - # hack to run style check before the docker build job if possible (style-check image not changed) - if: contains(fromJson(steps.runconfig.outputs.CI_DATA).jobs_data.jobs_to_do, 'Style check early') - run: | - DOCKER_TAG=$(echo '${{ toJson(fromJson(steps.runconfig.outputs.CI_DATA).docker_data.images) }}' | tr -d '\n') - export DOCKER_TAG=$DOCKER_TAG - python3 ./tests/ci/style_check.py --no-push - python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --infile ${{ runner.temp }}/ci_run_data.json --post --job-name 'Style check' BuildDockers: needs: [RunConfig] if: ${{ !failure() && !cancelled() && toJson(fromJson(needs.RunConfig.outputs.data).docker_data.missing_multi) != '[]' }} diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 514189a8b8a..a3cea281a56 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -1033,22 +1033,6 @@ def _print_results(result: Any, outfile: Optional[str], pretty: bool = False) -> raise AssertionError(f"Unexpected type for 'res': {type(result)}") -def _check_and_update_for_early_style_check(jobs_data: dict, docker_data: dict) -> None: - """ - This is temporary hack to start style check before docker build if possible - FIXME: need better solution to do style check as soon as possible and as fast as possible w/o dependency on docker job - """ - jobs_to_do = jobs_data.get("jobs_to_do", []) - docker_to_build = docker_data.get("missing_multi", []) - if ( - JobNames.STYLE_CHECK in jobs_to_do - and docker_to_build - and "clickhouse/style-test" not in docker_to_build - ): - index = jobs_to_do.index(JobNames.STYLE_CHECK) - jobs_to_do[index] = "Style check early" - - def _update_config_for_docs_only(jobs_data: dict) -> None: DOCS_CHECK_JOBS = [JobNames.DOCS_CHECK, JobNames.STYLE_CHECK] print(f"NOTE: Will keep only docs related jobs: [{DOCS_CHECK_JOBS}]") @@ -1306,6 +1290,12 @@ def _configure_jobs( if params["num_batches"] > 1: params["batches"] = list(requested_batches) + if pr_info.is_merge_queue(): + # FIXME: Quick support for MQ workflow which is only StyleCheck for now + jobs_to_do = [JobNames.STYLE_CHECK] + jobs_to_skip = [] + print(f"NOTE: This is Merge Queue CI: set jobs to do: [{jobs_to_do}]") + return { "digests": digests, "jobs_to_do": jobs_to_do, @@ -1752,11 +1742,6 @@ def main() -> int: else {} ) - # # FIXME: Early style check manipulates with job names might be not robust with await feature - # if pr_info.number != 0: - # # FIXME: it runs style check before docker build if possible (style-check images is not changed) - # # find a way to do style check always before docker build and others - # _check_and_update_for_early_style_check(jobs_data, docker_data) if not args.skip_jobs and pr_info.has_changes_in_documentation_only(): _update_config_for_docs_only(jobs_data) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 84f2db4002d..ddf59c49e1f 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -215,6 +215,7 @@ class PRInfo: .replace("{base}", base_sha) .replace("{head}", self.sha) ) + self.commit_html_url = f"{repo_prefix}/commits/{self.sha}" elif "commits" in github_event: self.event_type = EventType.PUSH From 03f8334ea1bd46839adb7ad45dabfb7fac30c042 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 25 Mar 2024 22:36:58 +0300 Subject: [PATCH 099/102] Remove SonarCloud (#61880) --- README.md | 1 - 1 file changed, 1 deletion(-) diff --git a/README.md b/README.md index e00ce42a60b..2b97bd25d70 100644 --- a/README.md +++ b/README.md @@ -28,7 +28,6 @@ curl https://clickhouse.com/ | sh * [Slack](https://clickhouse.com/slack) and [Telegram](https://telegram.me/clickhouse_en) allow chatting with ClickHouse users in real-time. * [Blog](https://clickhouse.com/blog/) contains various ClickHouse-related articles, as well as announcements and reports about events. * [Code Browser (github.dev)](https://github.dev/ClickHouse/ClickHouse) with syntax highlighting, powered by github.dev. -* [Static Analysis (SonarCloud)](https://sonarcloud.io/project/issues?resolved=false&id=ClickHouse_ClickHouse) proposes C++ quality improvements. * [Contacts](https://clickhouse.com/company/contact) can help to get your questions answered if there are any. ## Monthly Release & Community Call From cad66f3e688a759cad50ee9b69d1061c127c6ed6 Mon Sep 17 00:00:00 2001 From: pufit Date: Mon, 25 Mar 2024 20:03:02 -0400 Subject: [PATCH 100/102] Fix ALTER QUERY MODIFY SQL SECURITY (#61480) --- src/Databases/DatabasesCommon.cpp | 11 +++++++++++ src/Interpreters/InterpreterCreateQuery.cpp | 2 +- src/Parsers/ASTCreateQuery.cpp | 2 +- src/Parsers/CommonParsers.h | 1 + src/Parsers/ParserAlterQuery.cpp | 14 ++++++++++---- src/Storages/StorageInMemoryMetadata.cpp | 2 ++ ...create_view_with_sql_security_option.reference | 2 ++ .../02884_create_view_with_sql_security_option.sh | 15 +++++++-------- 8 files changed, 35 insertions(+), 14 deletions(-) diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index d40e5f98aaa..4dffb16e486 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -71,6 +71,17 @@ void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemo query->replace(ast_create_query.refresh_strategy, metadata.refresh); } + if (metadata.sql_security_type) + { + auto new_sql_security = std::make_shared(); + new_sql_security->type = metadata.sql_security_type; + + if (metadata.definer) + new_sql_security->definer = std::make_shared(*metadata.definer); + + ast_create_query.sql_security = std::move(new_sql_security); + } + /// MaterializedView, Dictionary are types of CREATE query without storage. if (ast_create_query.storage) { diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 8b777a758f6..d90d2446fca 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1881,7 +1881,7 @@ void InterpreterCreateQuery::addColumnsDescriptionToCreateQueryIfNecessary(ASTCr void InterpreterCreateQuery::processSQLSecurityOption(ContextPtr context_, ASTSQLSecurity & sql_security, bool is_attach, bool is_materialized_view) { /// If no SQL security is specified, apply default from default_*_view_sql_security setting. - if (!sql_security.type.has_value()) + if (!sql_security.type) { SQLSecurityType default_security; diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index de5eb40837f..0403dc33164 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -14,7 +14,7 @@ namespace DB void ASTSQLSecurity::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - if (!type.has_value()) + if (!type) return; if (definer || is_definer_current_user) diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index 545422aaf59..1a9fd7824d1 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -299,6 +299,7 @@ namespace DB MR_MACROS(MOD, "MOD") \ MR_MACROS(MODIFY_COLUMN, "MODIFY COLUMN") \ MR_MACROS(MODIFY_COMMENT, "MODIFY COMMENT") \ + MR_MACROS(MODIFY_DEFINER, "MODIFY DEFINER") \ MR_MACROS(MODIFY_ORDER_BY, "MODIFY ORDER BY") \ MR_MACROS(MODIFY_QUERY, "MODIFY QUERY") \ MR_MACROS(MODIFY_REFRESH, "MODIFY REFRESH") \ diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index c0c43381585..1baff45113b 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -41,6 +41,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_reset_setting(Keyword::RESET_SETTING); ParserKeyword s_modify_query(Keyword::MODIFY_QUERY); ParserKeyword s_modify_sql_security(Keyword::MODIFY_SQL_SECURITY); + ParserKeyword s_modify_definer(Keyword::MODIFY_DEFINER); ParserKeyword s_modify_refresh(Keyword::MODIFY_REFRESH); ParserKeyword s_add_index(Keyword::ADD_INDEX); @@ -862,11 +863,16 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected return false; command->type = ASTAlterCommand::MODIFY_QUERY; } - else if (s_modify_sql_security.ignore(pos, expected)) + else if (s_modify_sql_security.checkWithoutMoving(pos, expected)) { - /// This is a hack so we can reuse parser from create and don't have to write `MODIFY SQL SECURITY SQL SECURITY INVOKER` - --pos; - --pos; + s_modify.ignore(pos, expected); + if (!sql_security_p.parse(pos, command_sql_security, expected)) + return false; + command->type = ASTAlterCommand::MODIFY_SQL_SECURITY; + } + else if (s_modify_definer.checkWithoutMoving(pos, expected)) + { + s_modify.ignore(pos, expected); if (!sql_security_p.parse(pos, command_sql_security, expected)) return false; command->type = ASTAlterCommand::MODIFY_SQL_SECURITY; diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 8e5195d497f..1ac739f03fd 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -94,6 +94,8 @@ void StorageInMemoryMetadata::setSQLSecurity(const ASTSQLSecurity & sql_security { if (sql_security.definer) definer = sql_security.definer->toString(); + else + definer = std::nullopt; sql_security_type = sql_security.type; } diff --git a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference index 79728fadc04..6d9d1f07ec2 100644 --- a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference +++ b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference @@ -12,8 +12,10 @@ OK 2 2 OK +1 ===== MaterializedView ===== OK +1 0 0 OK diff --git a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh index a4ab3ed0024..bead7db8450 100755 --- a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh +++ b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh @@ -1,18 +1,17 @@ #!/usr/bin/env bash +# Tags: no-replicated-database CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -user1="user02884_1_$RANDOM$RANDOM" -user2="user02884_2_$RANDOM$RANDOM" -user3="user02884_3_$RANDOM$RANDOM" -db="db02884_$RANDOM$RANDOM" +user1="user02884_1_${CLICKHOUSE_DATABASE}_$RANDOM" +user2="user02884_2_${CLICKHOUSE_DATABASE}_$RANDOM" +user3="user02884_3_${CLICKHOUSE_DATABASE}_$RANDOM" +db=${CLICKHOUSE_DATABASE} ${CLICKHOUSE_CLIENT} --multiquery <&1 | grep -c "Not enough privileges") >= 1 )) && echo "OK" || echo "UNEXPECTED" +${CLICKHOUSE_CLIENT} --query "SHOW CREATE TABLE $db.test_view_10" | grep -c "SQL SECURITY INVOKER" echo "===== MaterializedView =====" @@ -136,6 +136,7 @@ ${CLICKHOUSE_CLIENT} --query "GRANT SELECT ON $db.test_mv_5 TO $user2" ${CLICKHOUSE_CLIENT} --query "ALTER TABLE $db.test_mv_5 MODIFY SQL SECURITY NONE" ${CLICKHOUSE_CLIENT} --user $user2 --query "SELECT * FROM $db.test_mv_5" +${CLICKHOUSE_CLIENT} --query "SHOW CREATE TABLE $db.test_mv_5" | grep -c "SQL SECURITY NONE" ${CLICKHOUSE_CLIENT} --query "GRANT SELECT ON $db.test_mv_1 TO $user2" ${CLICKHOUSE_CLIENT} --query "GRANT SELECT ON $db.test_mv_3 TO $user2" @@ -221,6 +222,4 @@ EOF ${CLICKHOUSE_CLIENT} --user $user2 --query "SELECT * FROM $db.test_mv_row_2" - -${CLICKHOUSE_CLIENT} --query "DROP DATABASE IF EXISTS $db;" ${CLICKHOUSE_CLIENT} --query "DROP USER IF EXISTS $user1, $user2, $user3"; From 94f137967732e8177570df5ac570f5566b17bf66 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 26 Mar 2024 04:02:32 +0100 Subject: [PATCH 101/102] Move KQL trash where it is supposed to be --- src/Parsers/CommonParsers.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/CommonParsers.h b/src/Parsers/CommonParsers.h index 1a9fd7824d1..49964b5c728 100644 --- a/src/Parsers/CommonParsers.h +++ b/src/Parsers/CommonParsers.h @@ -556,7 +556,7 @@ namespace DB MR_MACROS(SSH_KEY, "SSH_KEY") \ MR_MACROS(SSL_CERTIFICATE, "SSL_CERTIFICATE") \ MR_MACROS(STRICTLY_ASCENDING, "STRICTLY_ASCENDING") \ - MR_MACROS(WITH_ITEMINDEX, "with_itemindex") \ + MR_MACROS(WITH_ITEMINDEX, "WITH_ITEMINDEX") \ enum class Keyword : size_t { From 2a820a14c5eb2f9d999ea43883cbb4e83dae8a22 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 26 Mar 2024 05:49:21 +0100 Subject: [PATCH 102/102] Changelog for 24.3 --- CHANGELOG.md | 174 +++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 174 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 9df678d4b9a..9117289028d 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,10 +1,184 @@ ### Table of Contents +**[ClickHouse release v24.3 LTS, 2024-03-26](#243)**
**[ClickHouse release v24.2, 2024-02-29](#242)**
**[ClickHouse release v24.1, 2024-01-30](#241)**
**[Changelog for 2023](https://clickhouse.com/docs/en/whats-new/changelog/2023/)**
# 2024 Changelog +### ClickHouse release 24.3 LTS, 2024-03-26 + +#### Upgrade Notes +* The setting `allow_experimental_analyzer` is enabled by default and it switches the query analysis to a new implementation, which has better compatibility and feature completeness. The feature "analyzer" is considered beta instead of experimental. You can turn the old behavior by setting the `compatibility` to `24.2` or disabling the `allow_experimental_analyzer` setting. Watch the [video on YouTube](https://www.youtube.com/watch?v=zhrOYQpgvkk). +* ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. This is controlled by the settings, `output_format_parquet_string_as_string`, `output_format_orc_string_as_string`, `output_format_arrow_string_as_string`. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases. Parquet/ORC/Arrow supports many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools lack support for the faster `lz4` compression method, that's why we set `zstd` by default. This is controlled by the settings `output_format_parquet_compression_method`, `output_format_orc_compression_method`, and `output_format_arrow_compression_method`. We changed the default to `zstd` for Parquet and ORC, but not Arrow (it is emphasized for low-level usages). [#61817](https://github.com/ClickHouse/ClickHouse/pull/61817) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* In the new ClickHouse version, the functions `geoDistance`, `greatCircleDistance`, and `greatCircleAngle` will use 64-bit double precision floating point data type for internal calculations and return type if all the arguments are Float64. This closes [#58476](https://github.com/ClickHouse/ClickHouse/issues/58476). In previous versions, the function always used Float32. You can switch to the old behavior by setting `geo_distance_returns_float64_on_float64_arguments` to `false` or setting `compatibility` to `24.2` or earlier. [#61848](https://github.com/ClickHouse/ClickHouse/pull/61848) ([Alexey Milovidov](https://github.com/alexey-milovidov)). Co-authored with [Geet Patel](https://github.com/geetptl). +* The obsolete in-memory data parts have been deprecated since version 23.5 and have not been supported since version 23.10. Now the remaining code is removed. Continuation of [#55186](https://github.com/ClickHouse/ClickHouse/issues/55186) and [#45409](https://github.com/ClickHouse/ClickHouse/issues/45409). It is unlikely that you have used in-memory data parts because they were available only before version 23.5 and only when you enabled them manually by specifying the corresponding SETTINGS for a MergeTree table. To check if you have in-memory data parts, run the following query: `SELECT part_type, count() FROM system.parts GROUP BY part_type ORDER BY part_type`. To disable the usage of in-memory data parts, do `ALTER TABLE ... MODIFY SETTING min_bytes_for_compact_part = DEFAULT, min_rows_for_compact_part = DEFAULT`. Before upgrading from old ClickHouse releases, first check that you don't have in-memory data parts. If there are in-memory data parts, disable them first, then wait while there are no in-memory data parts and continue the upgrade. [#61127](https://github.com/ClickHouse/ClickHouse/pull/61127) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Changed the column name from `duration_ms` to `duration_microseconds` in the `system.zookeeper` table to reflect the reality that the duration is in the microsecond resolution. [#60774](https://github.com/ClickHouse/ClickHouse/pull/60774) ([Duc Canh Le](https://github.com/canhld94)). +* Reject incoming INSERT queries in case when query-level settings `async_insert` and `deduplicate_blocks_in_dependent_materialized_views` are enabled together. This behaviour is controlled by a setting `throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert` and enabled by default. This is a continuation of https://github.com/ClickHouse/ClickHouse/pull/59699 needed to unblock https://github.com/ClickHouse/ClickHouse/pull/59915. [#60888](https://github.com/ClickHouse/ClickHouse/pull/60888) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Utility `clickhouse-copier` is moved to a separate repository on GitHub: https://github.com/ClickHouse/copier. It is no longer included in the bundle but is still available as a separate download. This closes: [#60734](https://github.com/ClickHouse/ClickHouse/issues/60734) This closes: [#60540](https://github.com/ClickHouse/ClickHouse/issues/60540) This closes: [#60250](https://github.com/ClickHouse/ClickHouse/issues/60250) This closes: [#52917](https://github.com/ClickHouse/ClickHouse/issues/52917) This closes: [#51140](https://github.com/ClickHouse/ClickHouse/issues/51140) This closes: [#47517](https://github.com/ClickHouse/ClickHouse/issues/47517) This closes: [#47189](https://github.com/ClickHouse/ClickHouse/issues/47189) This closes: [#46598](https://github.com/ClickHouse/ClickHouse/issues/46598) This closes: [#40257](https://github.com/ClickHouse/ClickHouse/issues/40257) This closes: [#36504](https://github.com/ClickHouse/ClickHouse/issues/36504) This closes: [#35485](https://github.com/ClickHouse/ClickHouse/issues/35485) This closes: [#33702](https://github.com/ClickHouse/ClickHouse/issues/33702) This closes: [#26702](https://github.com/ClickHouse/ClickHouse/issues/26702). +* To increase compatibility with MySQL, the compatibility alias `locate` now accepts arguments `(needle, haystack[, start_pos])` by default. The previous behavior `(haystack, needle, [, start_pos])` can be restored by setting `function_locate_has_mysql_compatible_argument_order = 0`. [#61092](https://github.com/ClickHouse/ClickHouse/pull/61092) ([Robert Schulze](https://github.com/rschu1ze)). +* Forbid `SimpleAggregateFunction` in `ORDER BY` of `MergeTree` tables (like `AggregateFunction` is forbidden, but they are forbidden because they are not comparable) by default (use `allow_suspicious_primary_key` to allow them). [#61399](https://github.com/ClickHouse/ClickHouse/pull/61399) ([Azat Khuzhin](https://github.com/azat)). +* The `Ordinary` database engine is deprecated. You will receive a warning in clickhouse-client if your server is using it. This closes [#52229](https://github.com/ClickHouse/ClickHouse/issues/52229). [#56942](https://github.com/ClickHouse/ClickHouse/pull/56942) ([shabroo](https://github.com/shabroo)). + +#### New Feature +* Support reading and writing backups as `tar` (in addition to `zip`). [#59535](https://github.com/ClickHouse/ClickHouse/pull/59535) ([josh-hildred](https://github.com/josh-hildred)). +* Implemented support for S3 Express buckets. [#59965](https://github.com/ClickHouse/ClickHouse/pull/59965) ([Nikita Taranov](https://github.com/nickitat)). +* Allow to attach parts from a different disk (using copy instead of hard link). [#60112](https://github.com/ClickHouse/ClickHouse/pull/60112) ([Unalian](https://github.com/Unalian)). +* Size-capped `Memory` tables: controlled by their settings, `min_bytes_to_keep, max_bytes_to_keep, min_rows_to_keep` and `max_rows_to_keep`. [#60612](https://github.com/ClickHouse/ClickHouse/pull/60612) ([Jake Bamrah](https://github.com/JakeBamrah)). +* Separate limits on number of waiting and executing queries. Added new server setting `max_waiting_queries` that limits the number of queries waiting due to `async_load_databases`. Existing limits on number of executing queries no longer count waiting queries. [#61053](https://github.com/ClickHouse/ClickHouse/pull/61053) ([Sergei Trifonov](https://github.com/serxa)). +* Added a table `system.keywords` which contains all the keywords from parser. Mostly needed and will be used for better fuzzing and syntax highlighting. [#51808](https://github.com/ClickHouse/ClickHouse/pull/51808) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Add support for `ATTACH PARTITION ALL`. [#61107](https://github.com/ClickHouse/ClickHouse/pull/61107) ([Kirill Nikiforov](https://github.com/allmazz)). +* Add a new function, `getClientHTTPHeader`. This closes [#54665](https://github.com/ClickHouse/ClickHouse/issues/54665). Co-authored with @lingtaolf. [#61820](https://github.com/ClickHouse/ClickHouse/pull/61820) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add `generate_series` as a table function (compatibility alias for PostgreSQL to the existing `numbers` function). This function generates table with an arithmetic progression with natural numbers. [#59390](https://github.com/ClickHouse/ClickHouse/pull/59390) ([divanik](https://github.com/divanik)). +* A mode for `topK`/`topkWeighed` support mode, which return count of values and its error. [#54508](https://github.com/ClickHouse/ClickHouse/pull/54508) ([UnamedRus](https://github.com/UnamedRus)). +* Added function `toMillisecond` which returns the millisecond component for values of type`DateTime` or `DateTime64`. [#60281](https://github.com/ClickHouse/ClickHouse/pull/60281) ([Shaun Struwig](https://github.com/Blargian)). +* Allow configuring HTTP redirect handlers for clickhouse-server. For example, you can make `/` redirect to the Play UI. [#60390](https://github.com/ClickHouse/ClickHouse/pull/60390) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Performance Improvement +* Optimized function `dotProduct` to omit unnecessary and expensive memory copies. [#60928](https://github.com/ClickHouse/ClickHouse/pull/60928) ([Robert Schulze](https://github.com/rschu1ze)). +* 30x faster printing for 256-bit integers. [#61100](https://github.com/ClickHouse/ClickHouse/pull/61100) ([Raúl Marín](https://github.com/Algunenano)). +* If the table's primary key contains mostly useless columns, don't keep them in memory. This is controlled by a new setting `primary_key_ratio_of_unique_prefix_values_to_skip_suffix_columns` with the value `0.9` by default, which means: for a composite primary key, if a column changes its value for at least 0.9 of all the times, the next columns after it will be not loaded. [#60255](https://github.com/ClickHouse/ClickHouse/pull/60255) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Make all format names case insensitive, like Tsv, or TSV, or tsv, or even rowbinary. [#60420](https://github.com/ClickHouse/ClickHouse/pull/60420) ([豪肥肥](https://github.com/HowePa)). I appreciate if you will continue to write it correctly, e.g., `JSON` 😇, not `Json` 🤮, but we don't mind if you spell it as you prefer. +* Improve the performance of serialized aggregation method when involving multiple `Nullable` columns. [#55809](https://github.com/ClickHouse/ClickHouse/pull/55809) ([Amos Bird](https://github.com/amosbird)). +* Lazy build JSON's output to improve performance of ALL JOIN. [#58278](https://github.com/ClickHouse/ClickHouse/pull/58278) ([LiuNeng](https://github.com/liuneng1994)). +* Make HTTP/HTTPs connections with external services, such as AWS S3 reusable for all uses cases. Even when response is 3xx or 4xx. [#58845](https://github.com/ClickHouse/ClickHouse/pull/58845) ([Sema Checherinda](https://github.com/CheSema)). +* Improvements to aggregate functions `argMin` / `argMax` / `any` / `anyLast` / `anyHeavy`, as well as `ORDER BY {u8/u16/u32/u64/i8/i16/u32/i64) LIMIT 1` queries. [#58640](https://github.com/ClickHouse/ClickHouse/pull/58640) ([Raúl Marín](https://github.com/Algunenano)). +* Trivial optimization for column's filter. Peak memory can be reduced to 44% of the original in some cases. [#59698](https://github.com/ClickHouse/ClickHouse/pull/59698) ([李扬](https://github.com/taiyang-li)). +* Execute `multiIf` function in a columnar fashion when the result type's underlying type is a number. [#60384](https://github.com/ClickHouse/ClickHouse/pull/60384) ([李扬](https://github.com/taiyang-li)). +* Faster (almost 2x) mutexes. [#60823](https://github.com/ClickHouse/ClickHouse/pull/60823) ([Azat Khuzhin](https://github.com/azat)). +* Drain multiple connections in parallel when a distributed query is finishing. [#60845](https://github.com/ClickHouse/ClickHouse/pull/60845) ([lizhuoyu5](https://github.com/lzydmxy)). +* Optimize data movement between columns of a Nullable number or a Nullable string, which improves some micro-benchmarks. [#60846](https://github.com/ClickHouse/ClickHouse/pull/60846) ([李扬](https://github.com/taiyang-li)). +* Operations with the filesystem cache will suffer less from the lock contention. [#61066](https://github.com/ClickHouse/ClickHouse/pull/61066) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Optimize array join and other JOINs by preventing a wrong compiler's optimization. Close [#61074](https://github.com/ClickHouse/ClickHouse/issues/61074). [#61075](https://github.com/ClickHouse/ClickHouse/pull/61075) ([李扬](https://github.com/taiyang-li)). +* If a query with a syntax error contained `COLUMNS` matcher with a regular expression, the regular expression was compiled each time during the parser's backtracking, instead of being compiled once. This was a fundamental error. The compiled regexp was put to AST. But the letter A in AST means "abstract" which means it should not contain heavyweight objects. Parts of AST can be created and discarded during parsing, including a large number of backtracking. This leads to slowness on the parsing side and consequently allows DoS by a readonly user. But the main problem is that it prevents progress in fuzzers. [#61543](https://github.com/ClickHouse/ClickHouse/pull/61543) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add a new analyzer pass to optimize the IN operator for a single value. [#61564](https://github.com/ClickHouse/ClickHouse/pull/61564) ([LiuNeng](https://github.com/liuneng1994)). +* DNSResolver shuffles set of resolved IPs which is needed to uniformly utilize multiple endpoints of AWS S3. [#60965](https://github.com/ClickHouse/ClickHouse/pull/60965) ([Sema Checherinda](https://github.com/CheSema)). + +#### Experimental Feature +* Support parallel reading for Azure blob storage. This improves the performance of the experimental Azure object storage. [#61503](https://github.com/ClickHouse/ClickHouse/pull/61503) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Add asynchronous WriteBuffer for Azure blob storage similar to S3. This improves the performance of the experimental Azure object storage. [#59929](https://github.com/ClickHouse/ClickHouse/pull/59929) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Use managed identity for backups IO when using Azure Blob Storage. Add a setting to prevent ClickHouse from attempting to create a non-existent container, which requires permissions at the storage account level. [#61785](https://github.com/ClickHouse/ClickHouse/pull/61785) ([Daniel Pozo Escalona](https://github.com/danipozo)). +* Add a setting `parallel_replicas_allow_in_with_subquery = 1` which allows subqueries for IN work with parallel replicas. [#60950](https://github.com/ClickHouse/ClickHouse/pull/60950) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* A change for the "zero-copy" replication: all zero copy locks related to a table have to be dropped when the table is dropped. The directory which contains these locks has to be removed also. [#57575](https://github.com/ClickHouse/ClickHouse/pull/57575) ([Sema Checherinda](https://github.com/CheSema)). + +#### Improvement +* Use `MergeTree` as a default table engine. [#60524](https://github.com/ClickHouse/ClickHouse/pull/60524) ([Alexey Milovidov](https://github.com/alexey-milovidov)) +* Enable `output_format_pretty_row_numbers` by default. It is better for usability. [#61791](https://github.com/ClickHouse/ClickHouse/pull/61791) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* In the previous version, some numbers in Pretty formats were not pretty enough. [#61794](https://github.com/ClickHouse/ClickHouse/pull/61794) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* A long value in Pretty formats won't be cut if it is the single value in the resultset, such as in the result of the `SHOW CREATE TABLE` query. [#61795](https://github.com/ClickHouse/ClickHouse/pull/61795) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Similarly to `clickhouse-local`, `clickhouse-client` will accept the `--output-format` option as a synonym to the `--format` option. This closes [#59848](https://github.com/ClickHouse/ClickHouse/issues/59848). [#61797](https://github.com/ClickHouse/ClickHouse/pull/61797) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* If stdout is a terminal and the output format is not specified, `clickhouse-client` and similar tools will use `PrettyCompact` by default, similarly to the interactive mode. `clickhouse-client` and `clickhouse-local` will handle command line arguments for input and output formats in a unified fashion. This closes [#61272](https://github.com/ClickHouse/ClickHouse/issues/61272). [#61800](https://github.com/ClickHouse/ClickHouse/pull/61800) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Underscore digit groups in Pretty formats for better readability. This is controlled by a new setting, `output_format_pretty_highlight_digit_groups`. [#61802](https://github.com/ClickHouse/ClickHouse/pull/61802) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add ability to override initial INSERT settings via `SYSTEM FLUSH DISTRIBUTED`. [#61832](https://github.com/ClickHouse/ClickHouse/pull/61832) ([Azat Khuzhin](https://github.com/azat)). +* Enable processors profiling (time spent/in and out bytes for sorting, aggregation, ...) by default. [#61096](https://github.com/ClickHouse/ClickHouse/pull/61096) ([Azat Khuzhin](https://github.com/azat)). +* Support files without format extension in Filesystem database. [#60795](https://github.com/ClickHouse/ClickHouse/pull/60795) ([Kruglov Pavel](https://github.com/Avogar)). +* Added `none_only_active` mode for `distributed_ddl_output_mode` setting. [#60340](https://github.com/ClickHouse/ClickHouse/pull/60340) ([Alexander Tokmakov](https://github.com/tavplubix)). +* The advanced dashboard has slightly better colors for multi-line graphs. [#60391](https://github.com/ClickHouse/ClickHouse/pull/60391) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* The Advanced dashboard now has controls always visible on scrolling. This allows you to add a new chart without scrolling up. [#60692](https://github.com/ClickHouse/ClickHouse/pull/60692) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* While running the `MODIFY COLUMN` query for materialized views, check the inner table's structure to ensure every column exists. [#47427](https://github.com/ClickHouse/ClickHouse/pull/47427) ([sunny](https://github.com/sunny19930321)). +* String types and Enums can be used in the same context, such as: arrays, UNION queries, conditional expressions. This closes [#60726](https://github.com/ClickHouse/ClickHouse/issues/60726). [#60727](https://github.com/ClickHouse/ClickHouse/pull/60727) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Allow declaring Enums in the structure of external data for query processing (this is an immediate temporary table that you can provide for your query). [#57857](https://github.com/ClickHouse/ClickHouse/pull/57857) ([Duc Canh Le](https://github.com/canhld94)). +* Consider lightweight deleted rows when selecting parts to merge, so the disk size of the resulting part will be estimated better. [#58223](https://github.com/ClickHouse/ClickHouse/pull/58223) ([Zhuo Qiu](https://github.com/jewelzqiu)). +* Added comments for columns for more system tables. Continuation of https://github.com/ClickHouse/ClickHouse/pull/58356. [#59016](https://github.com/ClickHouse/ClickHouse/pull/59016) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Now we can use virtual columns in PREWHERE. It's worthwhile for non-const virtual columns like `_part_offset`. [#59033](https://github.com/ClickHouse/ClickHouse/pull/59033) ([Amos Bird](https://github.com/amosbird)). Improved overall usability of virtual columns. Now it is allowed to use virtual columns in `PREWHERE` (it's worthwhile for non-const virtual columns like `_part_offset`). Now a builtin documentation is available for virtual columns as a comment of column in `DESCRIBE` query with enabled setting `describe_include_virtual_columns`. [#60205](https://github.com/ClickHouse/ClickHouse/pull/60205) ([Anton Popov](https://github.com/CurtizJ)). +* Instead using a constant key, now object storage generates key for determining remove objects capability. [#59495](https://github.com/ClickHouse/ClickHouse/pull/59495) ([Sema Checherinda](https://github.com/CheSema)). +* Allow "local" as object storage type instead of "local_blob_storage". [#60165](https://github.com/ClickHouse/ClickHouse/pull/60165) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Parallel flush of pending INSERT blocks of Distributed engine on `DETACH`/server shutdown and `SYSTEM FLUSH DISTRIBUTED` (Parallelism will work only if you have multi-disk policy for a table (like everything in the Distributed engine right now)). [#60225](https://github.com/ClickHouse/ClickHouse/pull/60225) ([Azat Khuzhin](https://github.com/azat)). +* Add a setting to force read-through cache for merges. [#60308](https://github.com/ClickHouse/ClickHouse/pull/60308) ([Kseniia Sumarokova](https://github.com/kssenii)). +* An improvement for the MySQL compatibility protocol. The issue [#57598](https://github.com/ClickHouse/ClickHouse/issues/57598) mentions a variant behaviour regarding transaction handling. An issued COMMIT/ROLLBACK when no transaction is active is reported as an error contrary to MySQL behaviour. [#60338](https://github.com/ClickHouse/ClickHouse/pull/60338) ([PapaToemmsn](https://github.com/PapaToemmsn)). +* Function `substring` now has a new alias `byteSlice`. [#60494](https://github.com/ClickHouse/ClickHouse/pull/60494) ([Robert Schulze](https://github.com/rschu1ze)). +* Renamed server setting `dns_cache_max_size` to `dns_cache_max_entries` to reduce ambiguity. [#60500](https://github.com/ClickHouse/ClickHouse/pull/60500) ([Kirill Nikiforov](https://github.com/allmazz)). +* `SHOW INDEX | INDEXES | INDICES | KEYS` no longer sorts by the primary key columns (which was unintuitive). [#60514](https://github.com/ClickHouse/ClickHouse/pull/60514) ([Robert Schulze](https://github.com/rschu1ze)). +* Keeper improvement: abort during startup if an invalid snapshot is detected to avoid data loss. [#60537](https://github.com/ClickHouse/ClickHouse/pull/60537) ([Antonio Andelic](https://github.com/antonio2368)). +* Update tzdata to 2024a. [#60768](https://github.com/ClickHouse/ClickHouse/pull/60768) ([Raúl Marín](https://github.com/Algunenano)). +* Keeper improvement: support `leadership_expiry_ms` in Keeper's settings. [#60806](https://github.com/ClickHouse/ClickHouse/pull/60806) ([Brokenice0415](https://github.com/Brokenice0415)). +* Always infer exponential numbers in JSON formats regardless of the setting `input_format_try_infer_exponent_floats`. Add setting `input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects` that allows to use String type for ambiguous paths instead of an exception during named Tuples inference from JSON objects. [#60808](https://github.com/ClickHouse/ClickHouse/pull/60808) ([Kruglov Pavel](https://github.com/Avogar)). +* Add support for `START TRANSACTION` syntax typically used in MySQL syntax, resolving https://github.com/ClickHouse/ClickHouse/discussions/60865. [#60886](https://github.com/ClickHouse/ClickHouse/pull/60886) ([Zach Naimon](https://github.com/ArctypeZach)). +* Add a flag for the full-sorting merge join algorithm to treat null as biggest/smallest. So the behavior can be compitable with other SQL systems, like Apache Spark. [#60896](https://github.com/ClickHouse/ClickHouse/pull/60896) ([loudongfeng](https://github.com/loudongfeng)). +* Support detect output format by file exctension in `clickhouse-client` and `clickhouse-local`. [#61036](https://github.com/ClickHouse/ClickHouse/pull/61036) ([豪肥肥](https://github.com/HowePa)). +* Update memory limit in runtime when Linux's CGroups value changed. [#61049](https://github.com/ClickHouse/ClickHouse/pull/61049) ([Han Fei](https://github.com/hanfei1991)). +* Add the function `toUInt128OrZero`, which was missed by mistake (the mistake is related to https://github.com/ClickHouse/ClickHouse/pull/945). The compatibility aliases `FROM_UNIXTIME` and `DATE_FORMAT` (they are not ClickHouse-native and only exist for MySQL compatibility) have been made case insensitive, as expected for SQL-compatibility aliases. [#61114](https://github.com/ClickHouse/ClickHouse/pull/61114) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Improvements for the access checks, allowing to revoke of unpossessed rights in case the target user doesn't have the revoking grants either. Example: `GRANT SELECT ON *.* TO user1; REVOKE SELECT ON system.* FROM user1;`. [#61115](https://github.com/ClickHouse/ClickHouse/pull/61115) ([pufit](https://github.com/pufit)). +* Fix `has()` function with `Nullable` column (fixes [#60214](https://github.com/ClickHouse/ClickHouse/issues/60214)). [#61249](https://github.com/ClickHouse/ClickHouse/pull/61249) ([Mikhail Koviazin](https://github.com/mkmkme)). +* Now it's possible to specify the attribute `merge="true"` in config substitutions for subtrees ``. In case this attribute specified, clickhouse will merge subtree with existing configuration, otherwise default behavior is append new content to configuration. [#61299](https://github.com/ClickHouse/ClickHouse/pull/61299) ([alesapin](https://github.com/alesapin)). +* Add async metrics for virtual memory mappings: `VMMaxMapCount` & `VMNumMaps`. Closes [#60662](https://github.com/ClickHouse/ClickHouse/issues/60662). [#61354](https://github.com/ClickHouse/ClickHouse/pull/61354) ([Tuan Pham Anh](https://github.com/tuanpavn)). +* Use `temporary_files_codec` setting in all places where we create temporary data, for example external memory sorting and external memory GROUP BY. Before it worked only in `partial_merge` JOIN algorithm. [#61456](https://github.com/ClickHouse/ClickHouse/pull/61456) ([Maksim Kita](https://github.com/kitaisreal)). +* Add a new setting `max_parser_backtracks` which allows to limit the complexity of query parsing. [#61502](https://github.com/ClickHouse/ClickHouse/pull/61502) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Less contention during dynamic resize of the filesystem cache. [#61524](https://github.com/ClickHouse/ClickHouse/pull/61524) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Disallow sharded mode of StorageS3 queue, because it will be rewritten. [#61537](https://github.com/ClickHouse/ClickHouse/pull/61537) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fixed typo: from `use_leagcy_max_level` to `use_legacy_max_level`. [#61545](https://github.com/ClickHouse/ClickHouse/pull/61545) ([William Schoeffel](https://github.com/wiledusc)). +* Remove some duplicate entries in `system.blob_storage_log`. [#61622](https://github.com/ClickHouse/ClickHouse/pull/61622) ([YenchangChan](https://github.com/YenchangChan)). +* Added `current_user` function as a compatibility alias for MySQL. [#61770](https://github.com/ClickHouse/ClickHouse/pull/61770) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Fix inconsistent floating point aggregate function states in mixed x86-64 / ARM clusters [#60610](https://github.com/ClickHouse/ClickHouse/pull/60610) ([Harry Lee](https://github.com/HarryLeeIBM)). + +#### Build/Testing/Packaging Improvement +* The real-time query profiler now works on AArch64. In previous versions, it worked only when a program didn't spend time inside a syscall. [#60807](https://github.com/ClickHouse/ClickHouse/pull/60807) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* ClickHouse version has been added to docker labels. Closes [#54224](https://github.com/ClickHouse/ClickHouse/issues/54224). [#60949](https://github.com/ClickHouse/ClickHouse/pull/60949) ([Nikolay Monkov](https://github.com/nikmonkov)). +* Upgrade `prqlc` to 0.11.3. [#60616](https://github.com/ClickHouse/ClickHouse/pull/60616) ([Maximilian Roos](https://github.com/max-sixty)). +* Add generic query text fuzzer in `clickhouse-local`. [#61508](https://github.com/ClickHouse/ClickHouse/pull/61508) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Fix finished_mutations_to_keep=0 for MergeTree (as docs says 0 is to keep everything) [#60031](https://github.com/ClickHouse/ClickHouse/pull/60031) ([Azat Khuzhin](https://github.com/azat)). +* Something was wrong with the FINAL optimization, here is how the author describes it: "PartsSplitter invalid ranges for the same part". [#60041](https://github.com/ClickHouse/ClickHouse/pull/60041) ([Maksim Kita](https://github.com/kitaisreal)). +* Something was wrong with Apache Hive, which is experimental and not supported. [#60262](https://github.com/ClickHouse/ClickHouse/pull/60262) ([shanfengp](https://github.com/Aed-p)). +* An improvement for experimental parallel replicas: force reanalysis if parallel replicas changed [#60362](https://github.com/ClickHouse/ClickHouse/pull/60362) ([Raúl Marín](https://github.com/Algunenano)). +* Fix usage of plain metadata type with new disks configuration option [#60396](https://github.com/ClickHouse/ClickHouse/pull/60396) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Don't allow to set max_parallel_replicas to 0 as it doesn't make sense [#60430](https://github.com/ClickHouse/ClickHouse/pull/60430) ([Kruglov Pavel](https://github.com/Avogar)). +* Try to fix logical error 'Cannot capture column because it has incompatible type' in mapContainsKeyLike [#60451](https://github.com/ClickHouse/ClickHouse/pull/60451) ([Kruglov Pavel](https://github.com/Avogar)). +* Avoid calculation of scalar subqueries for CREATE TABLE. [#60464](https://github.com/ClickHouse/ClickHouse/pull/60464) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix deadlock in parallel parsing when lots of rows are skipped due to errors [#60516](https://github.com/ClickHouse/ClickHouse/pull/60516) ([Kruglov Pavel](https://github.com/Avogar)). +* Something was wrong with experimental KQL (Kusto) support: fix `max_query_size_for_kql_compound_operator`: [#60534](https://github.com/ClickHouse/ClickHouse/pull/60534) ([Yong Wang](https://github.com/kashwy)). +* Keeper fix: add timeouts when waiting for commit logs [#60544](https://github.com/ClickHouse/ClickHouse/pull/60544) ([Antonio Andelic](https://github.com/antonio2368)). +* Don't output number tips for date types [#60577](https://github.com/ClickHouse/ClickHouse/pull/60577) ([Raúl Marín](https://github.com/Algunenano)). +* Fix reading from MergeTree with non-deterministic functions in filter [#60586](https://github.com/ClickHouse/ClickHouse/pull/60586) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix logical error on bad compatibility setting value type [#60596](https://github.com/ClickHouse/ClickHouse/pull/60596) ([Kruglov Pavel](https://github.com/Avogar)). +* fix(prql): Robust panic handler [#60615](https://github.com/ClickHouse/ClickHouse/pull/60615) ([Maximilian Roos](https://github.com/max-sixty)). +* Fix `intDiv` for decimal and date arguments [#60672](https://github.com/ClickHouse/ClickHouse/pull/60672) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Fix: expand CTE in alter modify query [#60682](https://github.com/ClickHouse/ClickHouse/pull/60682) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix system.parts for non-Atomic/Ordinary database engine (i.e. Memory) [#60689](https://github.com/ClickHouse/ClickHouse/pull/60689) ([Azat Khuzhin](https://github.com/azat)). +* Fix "Invalid storage definition in metadata file" for parameterized views [#60708](https://github.com/ClickHouse/ClickHouse/pull/60708) ([Azat Khuzhin](https://github.com/azat)). +* Fix buffer overflow in CompressionCodecMultiple [#60731](https://github.com/ClickHouse/ClickHouse/pull/60731) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove nonsense from SQL/JSON [#60738](https://github.com/ClickHouse/ClickHouse/pull/60738) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove wrong assertion in aggregate function quantileGK [#60740](https://github.com/ClickHouse/ClickHouse/pull/60740) ([李扬](https://github.com/taiyang-li)). +* Fix insert-select + insert_deduplication_token bug by setting streams to 1 [#60745](https://github.com/ClickHouse/ClickHouse/pull/60745) ([Jordi Villar](https://github.com/jrdi)). +* Prevent setting custom metadata headers on unsupported multipart upload operations [#60748](https://github.com/ClickHouse/ClickHouse/pull/60748) ([Francisco J. Jurado Moreno](https://github.com/Beetelbrox)). +* Fix toStartOfInterval [#60763](https://github.com/ClickHouse/ClickHouse/pull/60763) ([Andrey Zvonov](https://github.com/zvonand)). +* Fix crash in arrayEnumerateRanked [#60764](https://github.com/ClickHouse/ClickHouse/pull/60764) ([Raúl Marín](https://github.com/Algunenano)). +* Fix crash when using input() in INSERT SELECT JOIN [#60765](https://github.com/ClickHouse/ClickHouse/pull/60765) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix crash with different allow_experimental_analyzer value in subqueries [#60770](https://github.com/ClickHouse/ClickHouse/pull/60770) ([Dmitry Novik](https://github.com/novikd)). +* Remove recursion when reading from S3 [#60849](https://github.com/ClickHouse/ClickHouse/pull/60849) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix possible stuck on error in HashedDictionaryParallelLoader [#60926](https://github.com/ClickHouse/ClickHouse/pull/60926) ([vdimir](https://github.com/vdimir)). +* Fix async RESTORE with Replicated database [#60934](https://github.com/ClickHouse/ClickHouse/pull/60934) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix deadlock in async inserts to `Log` tables via native protocol [#61055](https://github.com/ClickHouse/ClickHouse/pull/61055) ([Anton Popov](https://github.com/CurtizJ)). +* Fix lazy execution of default argument in dictGetOrDefault for RangeHashedDictionary [#61196](https://github.com/ClickHouse/ClickHouse/pull/61196) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix multiple bugs in groupArraySorted [#61203](https://github.com/ClickHouse/ClickHouse/pull/61203) ([Raúl Marín](https://github.com/Algunenano)). +* Fix Keeper reconfig for standalone binary [#61233](https://github.com/ClickHouse/ClickHouse/pull/61233) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix usage of session_token in S3 engine [#61234](https://github.com/ClickHouse/ClickHouse/pull/61234) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix possible incorrect result of aggregate function `uniqExact` [#61257](https://github.com/ClickHouse/ClickHouse/pull/61257) ([Anton Popov](https://github.com/CurtizJ)). +* Fix bugs in show database [#61269](https://github.com/ClickHouse/ClickHouse/pull/61269) ([Raúl Marín](https://github.com/Algunenano)). +* Fix logical error in RabbitMQ storage with MATERIALIZED columns [#61320](https://github.com/ClickHouse/ClickHouse/pull/61320) ([vdimir](https://github.com/vdimir)). +* Fix CREATE OR REPLACE DICTIONARY [#61356](https://github.com/ClickHouse/ClickHouse/pull/61356) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix ATTACH query with external ON CLUSTER [#61365](https://github.com/ClickHouse/ClickHouse/pull/61365) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix consecutive keys optimization for nullable keys [#61393](https://github.com/ClickHouse/ClickHouse/pull/61393) ([Anton Popov](https://github.com/CurtizJ)). +* fix issue of actions dag split [#61458](https://github.com/ClickHouse/ClickHouse/pull/61458) ([Raúl Marín](https://github.com/Algunenano)). +* Fix finishing a failed RESTORE [#61466](https://github.com/ClickHouse/ClickHouse/pull/61466) ([Vitaly Baranov](https://github.com/vitlibar)). +* Disable async_insert_use_adaptive_busy_timeout correctly with compatibility settings [#61468](https://github.com/ClickHouse/ClickHouse/pull/61468) ([Raúl Marín](https://github.com/Algunenano)). +* Allow queuing in restore pool [#61475](https://github.com/ClickHouse/ClickHouse/pull/61475) ([Nikita Taranov](https://github.com/nickitat)). +* Fix an inconsistency when reading system.parts using UUID. [#61479](https://github.com/ClickHouse/ClickHouse/pull/61479) ([Dan Wu](https://github.com/wudanzy)). +* Fix ALTER QUERY MODIFY SQL SECURITY [#61480](https://github.com/ClickHouse/ClickHouse/pull/61480) ([pufit](https://github.com/pufit)). +* Fix crash in window view [#61526](https://github.com/ClickHouse/ClickHouse/pull/61526) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix `repeat` with non native integers [#61527](https://github.com/ClickHouse/ClickHouse/pull/61527) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix client's `-s` argument [#61530](https://github.com/ClickHouse/ClickHouse/pull/61530) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix crash in arrayPartialReverseSort [#61539](https://github.com/ClickHouse/ClickHouse/pull/61539) ([Raúl Marín](https://github.com/Algunenano)). +* Fix string search with const position [#61547](https://github.com/ClickHouse/ClickHouse/pull/61547) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix addDays cause an error when used DateTime64 [#61561](https://github.com/ClickHouse/ClickHouse/pull/61561) ([Shuai li](https://github.com/loneylee)). +* Disallow LowCardinality input type for JSONExtract [#61617](https://github.com/ClickHouse/ClickHouse/pull/61617) ([Julia Kartseva](https://github.com/jkartseva)). +* Fix `system.part_log` for async insert with deduplication [#61620](https://github.com/ClickHouse/ClickHouse/pull/61620) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix a `Non-ready set` exception for system.parts. [#61666](https://github.com/ClickHouse/ClickHouse/pull/61666) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix actual_part_name for REPLACE_RANGE (`Entry actual part isn't empty yet`) [#61675](https://github.com/ClickHouse/ClickHouse/pull/61675) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix a sanitizer report in `multiSearchAllPositionsCaseInsensitiveUTF8` for incorrect UTF-8 [#61749](https://github.com/ClickHouse/ClickHouse/pull/61749) ([pufit](https://github.com/pufit)). +* Fix an observation that the RANGE frame is not supported for Nullable columns. [#61766](https://github.com/ClickHouse/ClickHouse/pull/61766) ([YuanLiu](https://github.com/ditgittube)). + ### ClickHouse release 24.2, 2024-02-29 #### Backward Incompatible Change