From f5c07b89383a1401797902a6acb3a5a300efa3fe Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 7 Aug 2024 09:43:13 +0000 Subject: [PATCH 001/103] Add test cases to 03217_datetime64_constant_to_ast --- ...03217_datetime64_constant_to_ast.reference | 3 +++ .../03217_datetime64_constant_to_ast.sql | 19 +++++++++++++++++++ 2 files changed, 22 insertions(+) diff --git a/tests/queries/0_stateless/03217_datetime64_constant_to_ast.reference b/tests/queries/0_stateless/03217_datetime64_constant_to_ast.reference index c20baa0d261..90120f85a71 100644 --- a/tests/queries/0_stateless/03217_datetime64_constant_to_ast.reference +++ b/tests/queries/0_stateless/03217_datetime64_constant_to_ast.reference @@ -1,2 +1,5 @@ 1970-01-01 00:00:01.000 1970-01-01 00:00:01.000 +1970-01-01 00:00:01.000 0 +1970-01-01 00:00:01.000 0 +1970-01-01 00:00:01.000 0 localhost diff --git a/tests/queries/0_stateless/03217_datetime64_constant_to_ast.sql b/tests/queries/0_stateless/03217_datetime64_constant_to_ast.sql index 63334a511c7..d01bb8d72f1 100644 --- a/tests/queries/0_stateless/03217_datetime64_constant_to_ast.sql +++ b/tests/queries/0_stateless/03217_datetime64_constant_to_ast.sql @@ -4,3 +4,22 @@ SET session_timezone = 'UTC'; SELECT toDateTime64('1970-01-01 00:00:01', 3) FROM remote('127.0.0.{1,2}', system, one) ; + +SELECT toDateTime64('1970-01-01 00:00:01', 3), dummy +FROM remote('127.0.0.{1,2}', system, one) +GROUP BY dummy +ORDER BY dummy +; + +SELECT materialize(toDateTime64('1970-01-01 00:00:01', 3)), dummy +FROM remote('127.0.0.{1,2}', system, one) +GROUP BY dummy +ORDER BY dummy +; + + +SELECT toDateTime64('1970-01-01 00:00:01', 3), sum(dummy), hostname() +FROM remote('127.0.0.{1,2}', system, one) +GROUP BY hostname() +ORDER BY ALL +; From cb0335446eb6b1ca2e452c9246893fe2053a7e1e Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 15 Aug 2024 16:55:47 +0100 Subject: [PATCH 002/103] impl --- src/Common/ProfileEvents.cpp | 2 + src/Core/ProtocolDefines.h | 3 +- src/Core/Settings.h | 2 +- .../IO/CachedOnDiskReadBufferFromFile.cpp | 2 + .../QueryPlan/ReadFromMergeTree.cpp | 9 +- src/Processors/QueryPlan/ReadFromRemote.cpp | 3 +- .../MergeTreeReadPoolParallelReplicas.cpp | 87 ++++++++++++++++++- .../MergeTreeReadPoolParallelReplicas.h | 3 +- ...rgeTreeReadPoolParallelReplicasInOrder.cpp | 17 ++-- .../MergeTree/MergeTreeSelectProcessor.h | 1 + .../ParallelReplicasReadingCoordinator.cpp | 16 ++-- .../ParallelReplicasReadingCoordinator.h | 3 +- src/Storages/MergeTree/RequestResponse.cpp | 10 ++- src/Storages/MergeTree/RequestResponse.h | 9 +- 14 files changed, 130 insertions(+), 37 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index ccdce7ff584..e5bad44ae93 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -329,6 +329,7 @@ The server successfully detected this situation and will download merged part fr M(ParallelReplicasReadAssignedMarks, "Sum across all replicas of how many of scheduled marks were assigned by consistent hash") \ M(ParallelReplicasReadUnassignedMarks, "Sum across all replicas of how many unassigned marks were scheduled") \ M(ParallelReplicasReadAssignedForStealingMarks, "Sum across all replicas of how many of scheduled marks were assigned for stealing by consistent hash") \ + M(ParallelReplicasReadMarks, "How many marks were read by the given replica") \ \ M(ParallelReplicasStealingByHashMicroseconds, "Time spent collecting segments meant for stealing by hash") \ M(ParallelReplicasProcessingPartsMicroseconds, "Time spent processing data parts") \ @@ -482,6 +483,7 @@ The server successfully detected this situation and will download merged part fr M(CachedReadBufferReadFromCacheMicroseconds, "Time reading from filesystem cache") \ M(CachedReadBufferReadFromSourceBytes, "Bytes read from filesystem cache source (from remote fs, etc)") \ M(CachedReadBufferReadFromCacheBytes, "Bytes read from filesystem cache") \ + M(CachedReadBufferPredownloadedBytes, "Bytes read from filesystem cache source. Cache segments are read from left to right as a whole, it might be that we need to predownload some part of the segment irrelevant for the current task just to get to the needed data") \ M(CachedReadBufferCacheWriteBytes, "Bytes written from source (remote fs, etc) to filesystem cache") \ M(CachedReadBufferCacheWriteMicroseconds, "Time spent writing data into filesystem cache") \ M(CachedReadBufferCreateBufferMicroseconds, "Prepare buffer time") \ diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index 02d54221ed3..dd37daadaff 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -33,7 +33,8 @@ static constexpr auto DBMS_MIN_REVISION_WITH_AGGREGATE_FUNCTIONS_VERSIONING = 54 static constexpr auto DBMS_CLUSTER_PROCESSING_PROTOCOL_VERSION = 1; -static constexpr auto DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION = 3; +static constexpr auto DBMS_MIN_SUPPORTED_PARALLEL_REPLICAS_PROTOCOL_VERSION = 3; +static constexpr auto DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION = 4; static constexpr auto DBMS_MIN_REVISION_WITH_PARALLEL_REPLICAS = 54453; static constexpr auto DBMS_MERGE_TREE_PART_INFO_VERSION = 1; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 0808e8eb49f..6a980a850c8 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -938,7 +938,7 @@ class IColumn; M(Bool, parallel_replicas_for_non_replicated_merge_tree, false, "If true, ClickHouse will use parallel replicas algorithm also for non-replicated MergeTree tables", 0) \ M(UInt64, parallel_replicas_min_number_of_rows_per_replica, 0, "Limit the number of replicas used in a query to (estimated rows to read / min_number_of_rows_per_replica). The max is still limited by 'max_parallel_replicas'", 0) \ M(Bool, parallel_replicas_prefer_local_join, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN.", 0) \ - M(UInt64, parallel_replicas_mark_segment_size, 128, "Parts virtually divided into segments to be distributed between replicas for parallel reading. This setting controls the size of these segments. Not recommended to change until you're absolutely sure in what you're doing", 0) \ + M(UInt64, parallel_replicas_mark_segment_size, 0, "Parts virtually divided into segments to be distributed between replicas for parallel reading. This setting controls the size of these segments. Not recommended to change until you're absolutely sure in what you're doing. Value should be in range [128; 16384]", 0) \ M(Bool, allow_archive_path_syntax, true, "File/S3 engines/table function will parse paths with '::' as ' :: ' if archive has correct extension", 0) \ \ M(Bool, allow_experimental_inverted_index, false, "If it is set to true, allow to use experimental inverted index.", 0) \ diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index b471f3fc58f..6363c40a9ad 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -28,6 +28,7 @@ extern const Event CachedReadBufferReadFromCacheMicroseconds; extern const Event CachedReadBufferCacheWriteMicroseconds; extern const Event CachedReadBufferReadFromSourceBytes; extern const Event CachedReadBufferReadFromCacheBytes; +extern const Event CachedReadBufferPredownloadedBytes; extern const Event CachedReadBufferCacheWriteBytes; extern const Event CachedReadBufferCreateBufferMicroseconds; @@ -644,6 +645,7 @@ void CachedOnDiskReadBufferFromFile::predownload(FileSegment & file_segment) size_t current_predownload_size = std::min(current_impl_buffer_size, bytes_to_predownload); ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromSourceBytes, current_impl_buffer_size); + ProfileEvents::increment(ProfileEvents::CachedReadBufferPredownloadedBytes, current_impl_buffer_size); bool continue_predownload = file_segment.reserve( current_predownload_size, settings.filesystem_cache_reserve_space_wait_lock_timeout_milliseconds); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 348019d7d10..768d1cbc639 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -52,6 +52,7 @@ #include #include +#include "Interpreters/Cluster.h" #include "config.h" using namespace DB; @@ -343,11 +344,11 @@ Pipe ReadFromMergeTree::readFromPoolParallelReplicas( { const auto & client_info = context->getClientInfo(); - auto extension = ParallelReadingExtension - { + auto extension = ParallelReadingExtension{ .all_callback = all_ranges_callback.value(), .callback = read_task_callback.value(), .number_of_current_replica = client_info.number_of_current_replica, + .total_nodes_count = context->getClusterForParallelReplicas()->getShardsInfo().begin()->getAllNodeCount(), }; /// We have a special logic for local replica. It has to read less data, because in some cases it should @@ -514,11 +515,11 @@ Pipe ReadFromMergeTree::readInOrder( if (is_parallel_reading_from_replicas) { const auto & client_info = context->getClientInfo(); - ParallelReadingExtension extension - { + ParallelReadingExtension extension{ .all_callback = all_ranges_callback.value(), .callback = read_task_callback.value(), .number_of_current_replica = client_info.number_of_current_replica, + .total_nodes_count = context->getClusterForParallelReplicas()->getShardsInfo().begin()->getAllNodeCount(), }; const auto multiplier = context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier; diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index cf11052cd59..9dd90d26487 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -436,8 +436,7 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder shuffled_pool = shard.pool->getShuffledPools(current_settings, priority_func); } - coordinator - = std::make_shared(max_replicas_to_use, current_settings.parallel_replicas_mark_segment_size); + coordinator = std::make_shared(max_replicas_to_use); for (size_t i=0; i < max_replicas_to_use; ++i) { diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp index 33eaf5a49bd..2736aab2603 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp @@ -1,6 +1,79 @@ -#include #include +#include +#include + +#include +#include +#include + + +namespace +{ + +size_t chooseSegmentSize( + LoggerPtr log, size_t mark_segment_size, size_t min_marks_per_task, size_t threads, size_t sum_marks, size_t number_of_replicas) +{ + /// Mark segment size determines the granularity of work distribution between replicas. + /// Namely, coordinator will take mark segments of size `mark_segment_size` granules, calculate hash of this segment and assign it to corresponding replica. + /// Small segments are good when we read a small random subset of a table, big - when we do full-scan over a large table. + /// With small segments there is a problem: consider a query like `select max(time) from wikistat`. Average size of `time` per granule is ~5KB. So when we + /// read 128 granules we still read only ~0.5MB of data. With default fs cache segment size of 4MB it means a lot of data will be downloaded and written + /// in cache for no reason. General case will look like this: + /// + /// +---------- useful data + /// v + /// +------+--+------+ + /// |------|++| | + /// |------|++| | + /// +------+--+------+ + /// ^ + /// predownloaded data -----------+ + /// + /// Having large segments solves all the problems in this case. Also bigger segments mean less requests (especially for big tables and full-scans). + /// These three values below chosen mostly intuitively. 128 granules is 1M rows - just a good starting point, 16384 seems to still make sense when reading + /// billions of rows and 1024 - is a reasonable point in between. We limit our choice to only these three options because when we change segment size + /// we essentially change distribution of data between replicas and of course we don't want to use simultaneously tens of different distributions, because + /// it would be a huge waste of cache space. + constexpr std::array borders{128, 1024, 16384}; + + LOG_DEBUG( + log, + "mark_segment_size={}, min_marks_per_task*threads={}, sum_marks/number_of_replicas^2={}", + mark_segment_size, + min_marks_per_task * threads, + sum_marks / number_of_replicas / number_of_replicas); + + /// Here we take max of three numbers: + /// * user provided setting (0 by default) + /// * (min_marks_per_task * threads) = the number of marks we request from the coordinator each time - there is no point to have segments smaller than one unit of work for a replica + /// * (sum_marks / number_of_replicas^2) - we use consistent hashing for work distribution (including work stealing). If we have a really slow replica + /// everything up to (1/number_of_replicas) portion of its work will be stolen by other replicas. And it owns (1/number_of_replicas) share of total number of marks. + /// Aslo important to note here that sum_marks is calculated after PK analysis, it means in particular that different segment sizes might be used for the + /// same table for different queries (it is intentional). + mark_segment_size = std::max({mark_segment_size, min_marks_per_task * threads, sum_marks / number_of_replicas / number_of_replicas}); + + /// Squeeze the value to the borders. + mark_segment_size = std::clamp(mark_segment_size, borders.front(), borders.back()); + /// After we calculated a hopefully good value for segment_size let's just find the maximal border that is not bigger than the chosen value. + for (auto border : borders | std::views::reverse) + { + if (mark_segment_size >= border) + { + LOG_DEBUG(log, "Chosen segment size: {}", border); + return border; + } + } + + UNREACHABLE(); +} + +} + +namespace ProfileEvents +{ +extern const Event ParallelReplicasReadMarks; +} namespace DB { @@ -34,12 +107,19 @@ MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas( , extension(std::move(extension_)) , coordination_mode(CoordinationMode::Default) , min_marks_per_task(pool_settings.min_marks_for_concurrent_read) + , mark_segment_size(chooseSegmentSize( + log, + context_->getSettingsRef().parallel_replicas_mark_segment_size, + min_marks_per_task, + pool_settings.threads, + pool_settings.sum_marks, + extension.total_nodes_count)) { for (const auto & info : per_part_infos) min_marks_per_task = std::max(min_marks_per_task, info->min_marks_per_task); - extension.all_callback( - InitialAllRangesAnnouncement(coordination_mode, parts_ranges.getDescriptions(), extension.number_of_current_replica)); + extension.all_callback(InitialAllRangesAnnouncement( + coordination_mode, parts_ranges.getDescriptions(), extension.number_of_current_replica, mark_segment_size)); } MergeTreeReadTaskPtr MergeTreeReadPoolParallelReplicas::getTask(size_t /*task_idx*/, MergeTreeReadTask * previous_task) @@ -104,6 +184,7 @@ MergeTreeReadTaskPtr MergeTreeReadPoolParallelReplicas::getTask(size_t /*task_id if (current_task.ranges.empty()) buffered_ranges.pop_front(); + ProfileEvents::increment(ProfileEvents::ParallelReplicasReadMarks, current_sum_marks); return createTask(per_part_infos[part_idx], std::move(ranges_to_read), previous_task); } diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h index 6ba63cc2c9a..1c79bd736bd 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h @@ -30,12 +30,13 @@ public: private: mutable std::mutex mutex; + LoggerPtr log = getLogger("MergeTreeReadPoolParallelReplicas"); const ParallelReadingExtension extension; const CoordinationMode coordination_mode; size_t min_marks_per_task{0}; + size_t mark_segment_size{0}; RangesInDataPartsDescription buffered_ranges; bool no_more_tasks_available{false}; - LoggerPtr log = getLogger("MergeTreeReadPoolParallelReplicas"); }; } diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp index 6b5cf978423..e0cb88c209a 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp @@ -1,5 +1,10 @@ #include +namespace ProfileEvents +{ +extern const Event ParallelReplicasReadMarks; +} + namespace DB { @@ -43,11 +48,8 @@ MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrd for (const auto & part : parts_ranges) buffered_tasks.push_back({part.data_part->info, MarkRanges{}}); - extension.all_callback(InitialAllRangesAnnouncement( - mode, - parts_ranges.getDescriptions(), - extension.number_of_current_replica - )); + extension.all_callback( + InitialAllRangesAnnouncement(mode, parts_ranges.getDescriptions(), extension.number_of_current_replica, /*mark_segment_size_=*/0)); } MergeTreeReadTaskPtr MergeTreeReadPoolParallelReplicasInOrder::getTask(size_t task_idx, MergeTreeReadTask * previous_task) @@ -68,13 +70,14 @@ MergeTreeReadTaskPtr MergeTreeReadPoolParallelReplicasInOrder::getTask(size_t ta { auto result = std::move(desc.ranges); desc.ranges = MarkRanges{}; + ProfileEvents::increment(ProfileEvents::ParallelReplicasReadMarks, desc.ranges.getNumberOfMarks()); return result; } } return std::nullopt; }; - if (auto result = get_from_buffer(); result) + if (auto result = get_from_buffer()) return createTask(per_part_infos[task_idx], std::move(*result), previous_task); if (no_more_tasks) @@ -97,7 +100,7 @@ MergeTreeReadTaskPtr MergeTreeReadPoolParallelReplicasInOrder::getTask(size_t ta std::move(new_ranges.begin(), new_ranges.end(), std::back_inserter(old_ranges)); } - if (auto result = get_from_buffer(); result) + if (auto result = get_from_buffer()) return createTask(per_part_infos[task_idx], std::move(*result), previous_task); return nullptr; diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.h b/src/Storages/MergeTree/MergeTreeSelectProcessor.h index 7a9cebbcb2e..e20427dbff0 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.h @@ -27,6 +27,7 @@ struct ParallelReadingExtension MergeTreeAllRangesCallback all_callback; MergeTreeReadTaskCallback callback; size_t number_of_current_replica{0}; + size_t total_nodes_count{0}; }; /// Base class for MergeTreeThreadSelectAlgorithm and MergeTreeSelectAlgorithm diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index f46b4de10b7..4a1b6d0bada 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -211,14 +211,11 @@ using PartRefs = std::deque; class DefaultCoordinator : public ParallelReplicasReadingCoordinator::ImplInterface { public: - explicit DefaultCoordinator(size_t replicas_count_, size_t mark_segment_size_) + explicit DefaultCoordinator(size_t replicas_count_) : ParallelReplicasReadingCoordinator::ImplInterface(replicas_count_) - , mark_segment_size(mark_segment_size_) , replica_status(replicas_count_) , distribution_by_hash_queue(replicas_count_) { - if (mark_segment_size == 0) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Zero value provided for `mark_segment_size`"); } ~DefaultCoordinator() override; @@ -231,7 +228,7 @@ public: private: /// This many granules will represent a single segment of marks that will be assigned to a replica - const size_t mark_segment_size{0}; + size_t mark_segment_size{0}; bool state_initialized{false}; size_t finished_replicas{0}; @@ -393,6 +390,10 @@ void DefaultCoordinator::initializeReadingState(InitialAllRangesAnnouncement ann state_initialized = true; source_replica_for_parts_snapshot = announcement.replica_num; + mark_segment_size = announcement.mark_segment_size; + if (mark_segment_size == 0) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Zero value provided for `mark_segment_size`"); + LOG_DEBUG(log, "Reading state is fully initialized: {}", fmt::join(all_parts_to_read, "; ")); } @@ -1043,7 +1044,7 @@ void ParallelReplicasReadingCoordinator::initialize(CoordinationMode mode) switch (mode) { case CoordinationMode::Default: - pimpl = std::make_unique(replicas_count, mark_segment_size); + pimpl = std::make_unique(replicas_count); break; case CoordinationMode::WithOrder: pimpl = std::make_unique>(replicas_count); @@ -1060,8 +1061,7 @@ void ParallelReplicasReadingCoordinator::initialize(CoordinationMode mode) pimpl->markReplicaAsUnavailable(replica); } -ParallelReplicasReadingCoordinator::ParallelReplicasReadingCoordinator(size_t replicas_count_, size_t mark_segment_size_) - : replicas_count(replicas_count_), mark_segment_size(mark_segment_size_) +ParallelReplicasReadingCoordinator::ParallelReplicasReadingCoordinator(size_t replicas_count_) : replicas_count(replicas_count_) { } diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h index 8b463fda395..ad51d20f553 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h @@ -15,7 +15,7 @@ class ParallelReplicasReadingCoordinator public: class ImplInterface; - explicit ParallelReplicasReadingCoordinator(size_t replicas_count_, size_t mark_segment_size_ = 0); + explicit ParallelReplicasReadingCoordinator(size_t replicas_count_); ~ParallelReplicasReadingCoordinator(); void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement); @@ -35,7 +35,6 @@ private: std::mutex mutex; const size_t replicas_count{0}; - size_t mark_segment_size{0}; std::unique_ptr pimpl; ProgressCallback progress_callback; // store the callback only to bypass it to coordinator implementation std::set replicas_used; diff --git a/src/Storages/MergeTree/RequestResponse.cpp b/src/Storages/MergeTree/RequestResponse.cpp index 2ce0e20dcd2..1b0ad3cca48 100644 --- a/src/Storages/MergeTree/RequestResponse.cpp +++ b/src/Storages/MergeTree/RequestResponse.cpp @@ -126,6 +126,7 @@ void InitialAllRangesAnnouncement::serialize(WriteBuffer & out) const writeIntBinary(mode, out); description.serialize(out); writeIntBinary(replica_num, out); + writeIntBinary(mark_segment_size, out); } @@ -156,10 +157,15 @@ InitialAllRangesAnnouncement InitialAllRangesAnnouncement::deserialize(ReadBuffe description.deserialize(in); readIntBinary(replica_num, in); - return InitialAllRangesAnnouncement { + size_t mark_segment_size = 128; + if (version >= 4) + readIntBinary(mark_segment_size, in); + + return InitialAllRangesAnnouncement{ mode, description, - replica_num + replica_num, + mark_segment_size, }; } diff --git a/src/Storages/MergeTree/RequestResponse.h b/src/Storages/MergeTree/RequestResponse.h index 5f5516a6804..17518adf833 100644 --- a/src/Storages/MergeTree/RequestResponse.h +++ b/src/Storages/MergeTree/RequestResponse.h @@ -93,17 +93,14 @@ struct InitialAllRangesAnnouncement /// No default constructor, you must initialize all fields at once. InitialAllRangesAnnouncement( - CoordinationMode mode_, - RangesInDataPartsDescription description_, - size_t replica_num_) - : mode(mode_) - , description(description_) - , replica_num(replica_num_) + CoordinationMode mode_, RangesInDataPartsDescription description_, size_t replica_num_, size_t mark_segment_size_) + : mode(mode_), description(description_), replica_num(replica_num_), mark_segment_size(mark_segment_size_) {} CoordinationMode mode; RangesInDataPartsDescription description; size_t replica_num; + size_t mark_segment_size; void serialize(WriteBuffer & out) const; String describe(); From 891f9c5358dce8c572848fdb24513c0e56629bd8 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 15 Aug 2024 18:44:31 +0100 Subject: [PATCH 003/103] fix typo --- src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp index 2736aab2603..87fc1c12ddd 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp @@ -49,7 +49,7 @@ size_t chooseSegmentSize( /// * (min_marks_per_task * threads) = the number of marks we request from the coordinator each time - there is no point to have segments smaller than one unit of work for a replica /// * (sum_marks / number_of_replicas^2) - we use consistent hashing for work distribution (including work stealing). If we have a really slow replica /// everything up to (1/number_of_replicas) portion of its work will be stolen by other replicas. And it owns (1/number_of_replicas) share of total number of marks. - /// Aslo important to note here that sum_marks is calculated after PK analysis, it means in particular that different segment sizes might be used for the + /// Also important to note here that sum_marks is calculated after PK analysis, it means in particular that different segment sizes might be used for the /// same table for different queries (it is intentional). mark_segment_size = std::max({mark_segment_size, min_marks_per_task * threads, sum_marks / number_of_replicas / number_of_replicas}); From 80d985a690d66621dd994b9e8066788b16cfe044 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 15 Aug 2024 19:11:23 +0100 Subject: [PATCH 004/103] add setting change --- src/Core/SettingsChangesHistory.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 20a8721c10e..0c2f9190553 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -88,6 +88,7 @@ static std::initializer_list Date: Thu, 15 Aug 2024 18:14:07 +0100 Subject: [PATCH 005/103] add bw-compatibility test --- .../configs/clusters.xml | 20 ++++++ .../test_parallel_replicas_protocol.py | 64 +++++++++++++++++++ 2 files changed, 84 insertions(+) create mode 100644 tests/integration/test_backward_compatibility/configs/clusters.xml create mode 100644 tests/integration/test_backward_compatibility/test_parallel_replicas_protocol.py diff --git a/tests/integration/test_backward_compatibility/configs/clusters.xml b/tests/integration/test_backward_compatibility/configs/clusters.xml new file mode 100644 index 00000000000..ac773152df9 --- /dev/null +++ b/tests/integration/test_backward_compatibility/configs/clusters.xml @@ -0,0 +1,20 @@ + + + + + + node0 + 9000 + + + node1 + 9000 + + + node2 + 9000 + + + + + diff --git a/tests/integration/test_backward_compatibility/test_parallel_replicas_protocol.py b/tests/integration/test_backward_compatibility/test_parallel_replicas_protocol.py new file mode 100644 index 00000000000..6f97df95876 --- /dev/null +++ b/tests/integration/test_backward_compatibility/test_parallel_replicas_protocol.py @@ -0,0 +1,64 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + + +cluster = ClickHouseCluster(__file__) +cluster_name = "parallel_replicas" +nodes = [ + cluster.add_instance( + f"node{num}", + main_configs=["configs/clusters.xml"], + with_zookeeper=False, + image="clickhouse/clickhouse-server", + tag="23.11", + stay_alive=True, + use_old_analyzer=True, + with_installed_binary=True, + ) + for num in range(2) +] + [ + cluster.add_instance( + "node2", + main_configs=["configs/clusters.xml"], + with_zookeeper=False, + use_old_analyzer=True, + ) +] + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_backward_compatability(start_cluster): + for node in nodes: + node.query("create table t (a UInt64) engine = MergeTree order by tuple()") + node.query("insert into t select number % 100000 from numbers_mt(1000000)") + + # all we want is the query to run without errors + for node in nodes: + assert ( + node.query( + """ + select sum(a) + from t + """, + settings={ + "cluster_for_parallel_replicas": "parallel_replicas", + "max_parallel_replicas": 3, + "allow_experimental_parallel_reading_from_replicas": 1, + "parallel_replicas_for_non_replicated_merge_tree": 1, + }, + ) + == "49999500000\n" + ) + + for node in nodes: + node.query("drop table t") From 628a4300ba5d492cf1ae0b4b077459cd2a37f1be Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sun, 18 Aug 2024 16:53:00 +0100 Subject: [PATCH 006/103] fix --- src/Client/Connection.cpp | 2 +- src/Core/ProtocolDefines.h | 4 +- src/Server/TCPHandler.cpp | 2 +- src/Storages/MergeTree/RequestResponse.cpp | 63 +++++++++++-------- src/Storages/MergeTree/RequestResponse.h | 4 +- .../test_parallel_replicas_protocol.py | 2 +- 6 files changed, 43 insertions(+), 34 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 07f4bf19f05..4f19fece5ef 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -1330,7 +1330,7 @@ ParallelReadRequest Connection::receiveParallelReadRequest() const InitialAllRangesAnnouncement Connection::receiveInitialParallelReadAnnouncement() const { - return InitialAllRangesAnnouncement::deserialize(*in); + return InitialAllRangesAnnouncement::deserialize(*in, server_revision); } diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index dd37daadaff..5765704e8ac 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -33,8 +33,6 @@ static constexpr auto DBMS_MIN_REVISION_WITH_AGGREGATE_FUNCTIONS_VERSIONING = 54 static constexpr auto DBMS_CLUSTER_PROCESSING_PROTOCOL_VERSION = 1; -static constexpr auto DBMS_MIN_SUPPORTED_PARALLEL_REPLICAS_PROTOCOL_VERSION = 3; -static constexpr auto DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION = 4; static constexpr auto DBMS_MIN_REVISION_WITH_PARALLEL_REPLICAS = 54453; static constexpr auto DBMS_MERGE_TREE_PART_INFO_VERSION = 1; @@ -84,6 +82,8 @@ static constexpr auto DBMS_MIN_REVISION_WITH_SYSTEM_KEYWORDS_TABLE = 54468; static constexpr auto DBMS_MIN_REVISION_WITH_ROWS_BEFORE_AGGREGATION = 54469; +static constexpr auto DBMS_MIN_REVISION_WITH_ADAPTIVE_MARK_SEGMENT_FOR_PARALLEL_REPLICAS = 54470; + /// Version of ClickHouse TCP protocol. /// /// Should be incremented manually on protocol changes. diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 679f72b85ff..4a017b2ae40 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1221,7 +1221,7 @@ void TCPHandler::sendReadTaskRequestAssumeLocked() void TCPHandler::sendMergeTreeAllRangesAnnouncementAssumeLocked(InitialAllRangesAnnouncement announcement) { writeVarUInt(Protocol::Server::MergeTreeAllRangesAnnouncement, *out); - announcement.serialize(*out); + announcement.serialize(*out, client_tcp_protocol_version); out->next(); } diff --git a/src/Storages/MergeTree/RequestResponse.cpp b/src/Storages/MergeTree/RequestResponse.cpp index 1b0ad3cca48..5e7e392384f 100644 --- a/src/Storages/MergeTree/RequestResponse.cpp +++ b/src/Storages/MergeTree/RequestResponse.cpp @@ -2,10 +2,10 @@ #include #include -#include +#include #include #include -#include +#include #include @@ -20,20 +20,21 @@ namespace ErrorCodes namespace { - CoordinationMode validateAndGet(uint8_t candidate) - { - if (candidate <= static_cast(CoordinationMode::MAX)) - return static_cast(candidate); +constexpr UInt64 DEPRECATED_FIELD_PARALLEL_REPLICAS_PROTOCOL_VERSION = 3; - throw Exception(ErrorCodes::UNKNOWN_ELEMENT_OF_ENUM, "Unknown reading mode: {}", candidate); - } +CoordinationMode validateAndGet(uint8_t candidate) +{ + if (candidate <= static_cast(CoordinationMode::MAX)) + return static_cast(candidate); + + throw Exception(ErrorCodes::UNKNOWN_ELEMENT_OF_ENUM, "Unknown reading mode: {}", candidate); +} } void ParallelReadRequest::serialize(WriteBuffer & out) const { - UInt64 version = DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION; /// Must be the first - writeIntBinary(version, out); + writeIntBinary(DEPRECATED_FIELD_PARALLEL_REPLICAS_PROTOCOL_VERSION, out); writeIntBinary(mode, out); writeIntBinary(replica_num, out); @@ -55,10 +56,13 @@ ParallelReadRequest ParallelReadRequest::deserialize(ReadBuffer & in) { UInt64 version; readIntBinary(version, in); - if (version != DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION) - throw Exception(ErrorCodes::UNKNOWN_PROTOCOL, "Protocol versions for parallel reading "\ + if (version != DEPRECATED_FIELD_PARALLEL_REPLICAS_PROTOCOL_VERSION) + throw Exception( + ErrorCodes::UNKNOWN_PROTOCOL, + "Protocol versions for parallel reading " "from replicas differ. Got: {}, supported version: {}", - version, DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION); + version, + DEPRECATED_FIELD_PARALLEL_REPLICAS_PROTOCOL_VERSION); CoordinationMode mode; size_t replica_num; @@ -90,9 +94,8 @@ void ParallelReadRequest::merge(ParallelReadRequest & other) void ParallelReadResponse::serialize(WriteBuffer & out) const { - UInt64 version = DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION; /// Must be the first - writeIntBinary(version, out); + writeIntBinary(DEPRECATED_FIELD_PARALLEL_REPLICAS_PROTOCOL_VERSION, out); writeBoolText(finish, out); description.serialize(out); @@ -107,26 +110,29 @@ void ParallelReadResponse::deserialize(ReadBuffer & in) { UInt64 version; readIntBinary(version, in); - if (version != DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION) - throw Exception(ErrorCodes::UNKNOWN_PROTOCOL, "Protocol versions for parallel reading " \ + if (version != DEPRECATED_FIELD_PARALLEL_REPLICAS_PROTOCOL_VERSION) + throw Exception( + ErrorCodes::UNKNOWN_PROTOCOL, + "Protocol versions for parallel reading " "from replicas differ. Got: {}, supported version: {}", - version, DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION); + version, + DEPRECATED_FIELD_PARALLEL_REPLICAS_PROTOCOL_VERSION); readBoolText(finish, in); description.deserialize(in); } -void InitialAllRangesAnnouncement::serialize(WriteBuffer & out) const +void InitialAllRangesAnnouncement::serialize(WriteBuffer & out, UInt64 client_protocol_revision) const { - UInt64 version = DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION; /// Must be the first - writeIntBinary(version, out); + writeIntBinary(DEPRECATED_FIELD_PARALLEL_REPLICAS_PROTOCOL_VERSION, out); writeIntBinary(mode, out); description.serialize(out); writeIntBinary(replica_num, out); - writeIntBinary(mark_segment_size, out); + if (client_protocol_revision >= DBMS_MIN_REVISION_WITH_ADAPTIVE_MARK_SEGMENT_FOR_PARALLEL_REPLICAS) + writeIntBinary(mark_segment_size, out); } @@ -138,14 +144,17 @@ String InitialAllRangesAnnouncement::describe() return result; } -InitialAllRangesAnnouncement InitialAllRangesAnnouncement::deserialize(ReadBuffer & in) +InitialAllRangesAnnouncement InitialAllRangesAnnouncement::deserialize(ReadBuffer & in, UInt64 client_protocol_revision) { UInt64 version; readIntBinary(version, in); - if (version != DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION) - throw Exception(ErrorCodes::UNKNOWN_PROTOCOL, "Protocol versions for parallel reading " \ + if (version != DEPRECATED_FIELD_PARALLEL_REPLICAS_PROTOCOL_VERSION) + throw Exception( + ErrorCodes::UNKNOWN_PROTOCOL, + "Protocol versions for parallel reading " "from replicas differ. Got: {}, supported version: {}", - version, DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION); + version, + DEPRECATED_FIELD_PARALLEL_REPLICAS_PROTOCOL_VERSION); CoordinationMode mode; RangesInDataPartsDescription description; @@ -158,7 +167,7 @@ InitialAllRangesAnnouncement InitialAllRangesAnnouncement::deserialize(ReadBuffe readIntBinary(replica_num, in); size_t mark_segment_size = 128; - if (version >= 4) + if (client_protocol_revision >= DBMS_MIN_REVISION_WITH_ADAPTIVE_MARK_SEGMENT_FOR_PARALLEL_REPLICAS) readIntBinary(mark_segment_size, in); return InitialAllRangesAnnouncement{ diff --git a/src/Storages/MergeTree/RequestResponse.h b/src/Storages/MergeTree/RequestResponse.h index 17518adf833..fcb6147c087 100644 --- a/src/Storages/MergeTree/RequestResponse.h +++ b/src/Storages/MergeTree/RequestResponse.h @@ -102,9 +102,9 @@ struct InitialAllRangesAnnouncement size_t replica_num; size_t mark_segment_size; - void serialize(WriteBuffer & out) const; + void serialize(WriteBuffer & out, UInt64 client_protocol_revision) const; String describe(); - static InitialAllRangesAnnouncement deserialize(ReadBuffer & in); + static InitialAllRangesAnnouncement deserialize(ReadBuffer & i, UInt64 client_protocol_revisionn); }; diff --git a/tests/integration/test_backward_compatibility/test_parallel_replicas_protocol.py b/tests/integration/test_backward_compatibility/test_parallel_replicas_protocol.py index 6f97df95876..e1b9049ef5d 100644 --- a/tests/integration/test_backward_compatibility/test_parallel_replicas_protocol.py +++ b/tests/integration/test_backward_compatibility/test_parallel_replicas_protocol.py @@ -11,7 +11,7 @@ nodes = [ main_configs=["configs/clusters.xml"], with_zookeeper=False, image="clickhouse/clickhouse-server", - tag="23.11", + tag="23.11", # earlier versions lead to "Not found column sum(a) in block." exception 🤷 stay_alive=True, use_old_analyzer=True, with_installed_binary=True, From 30229a3bfdd9ca0e827de0f741b9ff1d9553203d Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sun, 18 Aug 2024 17:44:16 +0100 Subject: [PATCH 007/103] better --- src/Core/ProtocolDefines.h | 4 +-- .../QueryPlan/ReadFromMergeTree.cpp | 26 ++++++++++--------- .../MergeTreeReadPoolParallelReplicas.cpp | 8 +++--- src/Storages/MergeTree/RequestResponse.cpp | 4 +++ src/Storages/MergeTree/RequestResponse.h | 2 +- 5 files changed, 25 insertions(+), 19 deletions(-) diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index 55c87976355..5acc07e70b7 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -94,6 +94,4 @@ static constexpr auto DBMS_MIN_REVISION_WITH_ADAPTIVE_MARK_SEGMENT_FOR_PARALLEL_ /// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION, /// later is just a number for server version (one number instead of commit SHA) /// for simplicity (sometimes it may be more convenient in some use cases). -static constexpr auto DBMS_TCP_PROTOCOL_VERSION = 54470; - -} +static constexpr auto DBMS_TCP_PROTOCOL_VERSION = 54471; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 5d3a676c959..a52a36daf7d 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1,6 +1,8 @@ #include +#include #include +#include #include #include #include @@ -8,6 +10,8 @@ #include #include #include +#include +#include #include #include #include @@ -16,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -24,10 +29,11 @@ #include #include #include -#include #include -#include +#include +#include #include +#include #include #include #include @@ -41,18 +47,12 @@ #include #include #include -#include -#include -#include -#include -#include #include #include #include #include -#include "Interpreters/Cluster.h" #include "config.h" using namespace DB; @@ -344,11 +344,12 @@ Pipe ReadFromMergeTree::readFromPoolParallelReplicas( { const auto & client_info = context->getClientInfo(); - auto extension = ParallelReadingExtension{ + auto extension = ParallelReadingExtension + { .all_callback = all_ranges_callback.value(), .callback = read_task_callback.value(), .number_of_current_replica = client_info.number_of_current_replica, - .total_nodes_count = context->getClusterForParallelReplicas()->getShardsInfo().begin()->getAllNodeCount(), + .total_nodes_count = context->getClusterForParallelReplicas()->getShardsInfo().at(0).getAllNodeCount(), }; /// We have a special logic for local replica. It has to read less data, because in some cases it should @@ -523,11 +524,12 @@ Pipe ReadFromMergeTree::readInOrder( if (is_parallel_reading_from_replicas) { const auto & client_info = context->getClientInfo(); - ParallelReadingExtension extension{ + ParallelReadingExtension extension + { .all_callback = all_ranges_callback.value(), .callback = read_task_callback.value(), .number_of_current_replica = client_info.number_of_current_replica, - .total_nodes_count = context->getClusterForParallelReplicas()->getShardsInfo().begin()->getAllNodeCount(), + .total_nodes_count = context->getClusterForParallelReplicas()->getShardsInfo().at(0).getAllNodeCount(), }; auto multiplier = context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier; diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp index 87fc1c12ddd..fc982fc0249 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp @@ -44,14 +44,16 @@ size_t chooseSegmentSize( min_marks_per_task * threads, sum_marks / number_of_replicas / number_of_replicas); - /// Here we take max of three numbers: - /// * user provided setting (0 by default) + /// Here we take max of two numbers: /// * (min_marks_per_task * threads) = the number of marks we request from the coordinator each time - there is no point to have segments smaller than one unit of work for a replica /// * (sum_marks / number_of_replicas^2) - we use consistent hashing for work distribution (including work stealing). If we have a really slow replica /// everything up to (1/number_of_replicas) portion of its work will be stolen by other replicas. And it owns (1/number_of_replicas) share of total number of marks. /// Also important to note here that sum_marks is calculated after PK analysis, it means in particular that different segment sizes might be used for the /// same table for different queries (it is intentional). - mark_segment_size = std::max({mark_segment_size, min_marks_per_task * threads, sum_marks / number_of_replicas / number_of_replicas}); + /// + /// Positive `mark_segment_size` means it is a user provided value, we have to preserve it. + if (mark_segment_size == 0) + mark_segment_size = std::max(min_marks_per_task * threads, sum_marks / number_of_replicas / number_of_replicas); /// Squeeze the value to the borders. mark_segment_size = std::clamp(mark_segment_size, borders.front(), borders.back()); diff --git a/src/Storages/MergeTree/RequestResponse.cpp b/src/Storages/MergeTree/RequestResponse.cpp index 5e7e392384f..48ff6ebccfd 100644 --- a/src/Storages/MergeTree/RequestResponse.cpp +++ b/src/Storages/MergeTree/RequestResponse.cpp @@ -20,6 +20,10 @@ namespace ErrorCodes namespace { +/// Previously we had a separate protocol version number for parallel replicas. +/// But we didn't maintain backward compatibility and every protocol change was breaking. +/// Now we have to support at least minimal tail of the previous versions and the implementation +/// is based on the common tcp protocol version as in all other places. constexpr UInt64 DEPRECATED_FIELD_PARALLEL_REPLICAS_PROTOCOL_VERSION = 3; CoordinationMode validateAndGet(uint8_t candidate) diff --git a/src/Storages/MergeTree/RequestResponse.h b/src/Storages/MergeTree/RequestResponse.h index fcb6147c087..da2fa2795a3 100644 --- a/src/Storages/MergeTree/RequestResponse.h +++ b/src/Storages/MergeTree/RequestResponse.h @@ -94,7 +94,7 @@ struct InitialAllRangesAnnouncement InitialAllRangesAnnouncement( CoordinationMode mode_, RangesInDataPartsDescription description_, size_t replica_num_, size_t mark_segment_size_) - : mode(mode_), description(description_), replica_num(replica_num_), mark_segment_size(mark_segment_size_) + : mode(mode_), description(std::move(description_)), replica_num(replica_num_), mark_segment_size(mark_segment_size_) {} CoordinationMode mode; From c252b3c8b05d85acb355054543d0b7f78d171af0 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sun, 18 Aug 2024 18:29:48 +0100 Subject: [PATCH 008/103] fix build --- src/Core/ProtocolDefines.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index 5acc07e70b7..f80ddc646bb 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -95,3 +95,5 @@ static constexpr auto DBMS_MIN_REVISION_WITH_ADAPTIVE_MARK_SEGMENT_FOR_PARALLEL_ /// later is just a number for server version (one number instead of commit SHA) /// for simplicity (sometimes it may be more convenient in some use cases). static constexpr auto DBMS_TCP_PROTOCOL_VERSION = 54471; + +} From 165d08f088d614dd835ee65a2d70f4e1fb0a27e2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Aug 2024 05:53:59 +0200 Subject: [PATCH 009/103] Fix bad exception messages --- .../MergeTree/ReplicatedMergeTreePartCheckThread.cpp | 2 +- src/Storages/MergeTree/checkDataPart.cpp | 2 +- tests/integration/test_broken_projections/test.py | 12 ++++++------ 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index dc242a7b084..8877ebff6a1 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -391,7 +391,7 @@ ReplicatedCheckResult ReplicatedMergeTreePartCheckThread::checkPartImpl(const St { WriteBufferFromOwnString wb; message = PreformattedMessage::create( - "Part {} has a broken projections. It will be ignored. Broken projections info: {}", + "Part `{}` has broken projections. It will be ignored. Broken projections info: {}", part_name, getCurrentExceptionMessage(true)); LOG_DEBUG(log, message); result.action = ReplicatedCheckResult::DoNothing; diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 3a22daa0011..4a16e0257a0 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -323,7 +323,7 @@ static IMergeTreeDataPart::Checksums checkDataPart( broken_projections_message += "\n"; broken_projections_message += fmt::format( - "Part {} has a broken projection {} (error: {})", + "Part `{}` has broken projection `{}` (error: {})", data_part->name, name, exception_message); } diff --git a/tests/integration/test_broken_projections/test.py b/tests/integration/test_broken_projections/test.py index 578ff42369c..15ed7a0bf3c 100644 --- a/tests/integration/test_broken_projections/test.py +++ b/tests/integration/test_broken_projections/test.py @@ -405,7 +405,7 @@ def test_materialize_broken_projection(cluster): assert "NO_FILE_IN_DATA_PART" in get_broken_projections_info( node, table_name, part="all_1_1_0", projection="proj1" ) - assert "Part all_1_1_0 has a broken projection proj1" in check_table_full( + assert "Part `all_1_1_0` has broken projection `proj1`" in check_table_full( node, table_name ) @@ -415,13 +415,13 @@ def test_materialize_broken_projection(cluster): assert "FILE_DOESNT_EXIST" in get_broken_projections_info( node, table_name, part="all_1_1_0", projection="proj2" ) - assert "Part all_1_1_0 has a broken projection proj2" in check_table_full( + assert "Part `all_1_1_0` has broken projection `proj2`" in check_table_full( node, table_name ) materialize_projection(node, table_name, "proj1") - assert "has a broken projection" not in check_table_full(node, table_name) + assert "has broken projection" not in check_table_full(node, table_name) def test_broken_ignored_replicated(cluster): @@ -443,13 +443,13 @@ def test_broken_ignored_replicated(cluster): check(node, table_name2, 1) break_projection(node, table_name, "proj1", "all_0_0_0", "data") - assert "Part all_0_0_0 has a broken projection proj1" in check_table_full( + assert "Part `all_0_0_0` has broken projection `proj1`" in check_table_full( node, table_name ) break_part(node, table_name, "all_0_0_0") node.query(f"SYSTEM SYNC REPLICA {table_name}") - assert "has a broken projection" not in check_table_full(node, table_name) + assert "has broken projection" not in check_table_full(node, table_name) def get_random_string(string_length=8): @@ -528,7 +528,7 @@ def test_broken_projections_in_backups_3(cluster): check(node, table_name, 1) break_projection(node, table_name, "proj1", "all_1_1_0", "part") - assert "Part all_1_1_0 has a broken projection proj1" in check_table_full( + assert "Part `all_1_1_0` has broken projection `proj1`" in check_table_full( node, table_name ) assert "FILE_DOESNT_EXIST" in get_broken_projections_info( From dbd4ee44edd8476ddf9c8745696f2295849b3676 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 20 Aug 2024 14:09:14 +0800 Subject: [PATCH 010/103] enable dict encoding in orc writer --- src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp index 4a7a23158ff..3356adb5315 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp @@ -78,7 +78,9 @@ void ORCOutputStream::write(const void* buf, size_t length) } ORCBlockOutputFormat::ORCBlockOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_) - : IOutputFormat(header_, out_), format_settings{format_settings_}, output_stream(out_) + : IOutputFormat(header_, out_) + , format_settings{format_settings_} + , output_stream(out_) { for (const auto & type : header_.getDataTypes()) data_types.push_back(recursiveRemoveLowCardinality(type)); @@ -565,6 +567,7 @@ void ORCBlockOutputFormat::prepareWriter() schema = orc::createStructType(); options.setCompression(getORCCompression(format_settings.orc.output_compression_method)); options.setRowIndexStride(format_settings.orc.output_row_index_stride); + options.setDictionaryKeySizeThreshold(format_settings.orc.output_dictionary_key_size_threshold); size_t columns_count = header.columns(); for (size_t i = 0; i != columns_count; ++i) schema->addStructField(header.safeGetByPosition(i).name, getORCType(recursiveRemoveLowCardinality(data_types[i]))); From 03ab6252653ddcb70d1fef0a340bfb52a8d63f89 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 20 Aug 2024 15:47:26 +0800 Subject: [PATCH 011/103] enable string dict encoding in orc output format --- src/Core/Settings.h | 1 + src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + 3 files changed, 3 insertions(+) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d8837d26e54..3c507bc064f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1265,6 +1265,7 @@ class IColumn; M(Bool, output_format_orc_string_as_string, true, "Use ORC String type instead of Binary for String columns", 0) \ M(ORCCompression, output_format_orc_compression_method, "zstd", "Compression method for ORC output format. Supported codecs: lz4, snappy, zlib, zstd, none (uncompressed)", 0) \ M(UInt64, output_format_orc_row_index_stride, 10'000, "Target row index stride in ORC output format", 0) \ + M(Double, output_format_orc_dictionary_key_size_threshold, 0.0, "For a string column in ORC output format, if the number of distinct values is greater than this fraction of the total number of non-null rows, turn off dictionary encoding. Otherwise dictionary encoding is enabled", 0) \ \ M(CapnProtoEnumComparingMode, format_capn_proto_enum_comparising_mode, FormatSettings::CapnProtoEnumComparingMode::BY_VALUES, "How to map ClickHouse Enum and CapnProto Enum", 0) \ \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 865b6e6f3f1..479d7a3f029 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -243,6 +243,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.orc.output_string_as_string = settings.output_format_orc_string_as_string; format_settings.orc.output_compression_method = settings.output_format_orc_compression_method; format_settings.orc.output_row_index_stride = settings.output_format_orc_row_index_stride; + format_settings.orc.output_dictionary_key_size_threshold = settings.output_format_orc_dictionary_key_size_threshold; format_settings.orc.use_fast_decoder = settings.input_format_orc_use_fast_decoder; format_settings.orc.filter_push_down = settings.input_format_orc_filter_push_down; format_settings.orc.reader_time_zone_name = settings.input_format_orc_reader_time_zone_name; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index da25da74134..06535bc07a3 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -413,6 +413,7 @@ struct FormatSettings bool filter_push_down = true; UInt64 output_row_index_stride = 10'000; String reader_time_zone_name = "GMT"; + double output_dictionary_key_size_threshold = 0.0; } orc{}; /// For capnProto format we should determine how to From 21e64f2aa98fbf381971ef855d6de11a7e062c39 Mon Sep 17 00:00:00 2001 From: "Sergey (Finn) Gnezdilov" <72432693+Sergey2Gnezdilov@users.noreply.github.com> Date: Tue, 20 Aug 2024 11:33:13 +0300 Subject: [PATCH 012/103] Update nats.md "kafka_handle_error_mode" fixed on "nats_handle_error_mode" --- docs/en/engines/table-engines/integrations/nats.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/nats.md b/docs/en/engines/table-engines/integrations/nats.md index 78ce537224c..2ea2fdb01ae 100644 --- a/docs/en/engines/table-engines/integrations/nats.md +++ b/docs/en/engines/table-engines/integrations/nats.md @@ -167,7 +167,7 @@ If you want to change the target table by using `ALTER`, we recommend disabling - `_subject` - NATS message subject. Data type: `String`. -Additional virtual columns when `kafka_handle_error_mode='stream'`: +Additional virtual columns when `nats_handle_error_mode='stream'`: - `_raw_message` - Raw message that couldn't be parsed successfully. Data type: `Nullable(String)`. - `_error` - Exception message happened during failed parsing. Data type: `Nullable(String)`. From b0a0988c5b324a49792b5dd201f7f9dec95aa831 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 22 Aug 2024 10:46:44 +0800 Subject: [PATCH 013/103] change as request --- src/Core/SettingsChangesHistory.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 2415323b4a0..b9c18c88652 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -94,7 +94,8 @@ static std::initializer_list Date: Thu, 22 Aug 2024 09:03:02 +0000 Subject: [PATCH 014/103] Proper ErrorCodes --- .../ObjectStorage/HDFS/ReadBufferFromHDFS.cpp | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp index bf6f9db722c..d3379b90163 100644 --- a/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp @@ -22,12 +22,12 @@ namespace DB namespace ErrorCodes { - extern const int NETWORK_ERROR; - extern const int CANNOT_OPEN_FILE; - extern const int CANNOT_SEEK_THROUGH_FILE; - extern const int SEEK_POSITION_OUT_OF_BOUND; - extern const int LOGICAL_ERROR; - extern const int UNKNOWN_FILE_SIZE; +extern const int HDFS_ERROR; +extern const int CANNOT_OPEN_FILE; +extern const int CANNOT_SEEK_THROUGH_FILE; +extern const int SEEK_POSITION_OUT_OF_BOUND; +extern const int LOGICAL_ERROR; +extern const int UNKNOWN_FILE_SIZE; } @@ -135,9 +135,12 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory Date: Fri, 23 Aug 2024 16:36:27 +0200 Subject: [PATCH 015/103] Fix zero copy bug with encrypted disk and UNFREEZE When running UNFREEZE with encrypted disk, zookeeper metadata would be erroneously removed here src/Storages/StorageReplicatedMergeTree.cpp#L10418. --- src/Disks/DiskEncrypted.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index f06f5ba8e17..9818c284009 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -350,6 +350,11 @@ public: return delegate; } + UInt32 getRefCount(const String & path) const override { + auto wrapped_path = wrappedPath(path); + return delegate->getRefCount(wrapped_path); + } + #if USE_AWS_S3 std::shared_ptr getS3StorageClient() const override { From 1011f8ef9c2511654937c96b7bea7693a4c9715f Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 26 Aug 2024 14:45:41 +0800 Subject: [PATCH 016/103] add uts about orc string encode --- src/Common/CMakeLists.txt | 4 +- .../benchmarks/orc_string_dictionary.cpp | 302 ++++++++++++++++++ 2 files changed, 304 insertions(+), 2 deletions(-) create mode 100644 src/Common/benchmarks/orc_string_dictionary.cpp diff --git a/src/Common/CMakeLists.txt b/src/Common/CMakeLists.txt index d4802c28f53..7cc8dd25cd9 100644 --- a/src/Common/CMakeLists.txt +++ b/src/Common/CMakeLists.txt @@ -1,6 +1,6 @@ -if (ENABLE_BENCHMARKS) +#if (ENABLE_BENCHMARKS) add_subdirectory(benchmarks) -endif() +#endif() if (ENABLE_EXAMPLES) add_subdirectory(examples) diff --git a/src/Common/benchmarks/orc_string_dictionary.cpp b/src/Common/benchmarks/orc_string_dictionary.cpp new file mode 100644 index 00000000000..bde7d03e864 --- /dev/null +++ b/src/Common/benchmarks/orc_string_dictionary.cpp @@ -0,0 +1,302 @@ +#include +#include +#include + +class OldSortedStringDictionary +{ +public: + struct DictEntry + { + DictEntry(const char * str, size_t len) : data(str), length(len) { } + const char * data; + size_t length; + }; + + OldSortedStringDictionary() : totalLength(0) { } + + // insert a new string into dictionary, return its insertion order + size_t insert(const char * str, size_t len); + + // reorder input index buffer from insertion order to dictionary order + void reorder(std::vector & idxBuffer) const; + + // get dict entries in insertion order + void getEntriesInInsertionOrder(std::vector &) const; + + size_t size() const; + + // return total length of strings in the dictioanry + uint64_t length() const; + + void clear(); + + // store indexes of insertion order in the dictionary for not-null rows + std::vector idxInDictBuffer; + +private: + struct LessThan + { + bool operator()(const DictEntry & left, const DictEntry & right) const + { + int ret = memcmp(left.data, right.data, std::min(left.length, right.length)); + if (ret != 0) + { + return ret < 0; + } + return left.length < right.length; + } + }; + + std::map dict; + std::vector> data; + uint64_t totalLength; +}; + +// insert a new string into dictionary, return its insertion order +size_t OldSortedStringDictionary::insert(const char * str, size_t len) +{ + auto ret = dict.insert({DictEntry(str, len), dict.size()}); + if (ret.second) + { + // make a copy to internal storage + data.push_back(std::vector(len)); + memcpy(data.back().data(), str, len); + // update dictionary entry to link pointer to internal storage + DictEntry * entry = const_cast(&(ret.first->first)); + entry->data = data.back().data(); + totalLength += len; + } + return ret.first->second; +} + +/** + * Reorder input index buffer from insertion order to dictionary order + * + * We require this function because string values are buffered by indexes + * in their insertion order. Until the entire dictionary is complete can + * we get their sorted indexes in the dictionary in that ORC specification + * demands dictionary should be ordered. Therefore this function transforms + * the indexes from insertion order to dictionary value order for final + * output. + */ +void OldSortedStringDictionary::reorder(std::vector & idxBuffer) const +{ + // iterate the dictionary to get mapping from insertion order to value order + std::vector mapping(dict.size()); + size_t dictIdx = 0; + for (auto it = dict.cbegin(); it != dict.cend(); ++it) + { + mapping[it->second] = dictIdx++; + } + + // do the transformation + for (size_t i = 0; i != idxBuffer.size(); ++i) + { + idxBuffer[i] = static_cast(mapping[static_cast(idxBuffer[i])]); + } +} + +// get dict entries in insertion order +void OldSortedStringDictionary::getEntriesInInsertionOrder(std::vector & entries) const +{ + entries.resize(dict.size()); + for (auto it = dict.cbegin(); it != dict.cend(); ++it) + { + entries[it->second] = &(it->first); + } +} + +// return count of entries +size_t OldSortedStringDictionary::size() const +{ + return dict.size(); +} + +// return total length of strings in the dictioanry +uint64_t OldSortedStringDictionary::length() const +{ + return totalLength; +} + +void OldSortedStringDictionary::clear() +{ + totalLength = 0; + data.clear(); + dict.clear(); +} + + +/** + * Implementation of increasing sorted string dictionary + */ +class NewSortedStringDictionary +{ +public: + struct DictEntry + { + DictEntry(const char * str, size_t len) : data(str), length(len) { } + const char * data; + size_t length; + }; + + struct DictEntryWithIndex + { + DictEntryWithIndex(const char * str, size_t len, size_t index_) : entry(str, len), index(index_) { } + DictEntry entry; + size_t index; + }; + + NewSortedStringDictionary() : totalLength_(0) { } + + // insert a new string into dictionary, return its insertion order + size_t insert(const char * str, size_t len); + + // reorder input index buffer from insertion order to dictionary order + void reorder(std::vector & idxBuffer) const; + + // get dict entries in insertion order + void getEntriesInInsertionOrder(std::vector &) const; + + // return count of entries + size_t size() const; + + // return total length of strings in the dictioanry + uint64_t length() const; + + void clear(); + + // store indexes of insertion order in the dictionary for not-null rows + std::vector idxInDictBuffer; + +private: + struct LessThan + { + bool operator()(const DictEntryWithIndex & l, const DictEntryWithIndex & r) + { + const auto & left = l.entry; + const auto & right = r.entry; + int ret = memcmp(left.data, right.data, std::min(left.length, right.length)); + if (ret != 0) + { + return ret < 0; + } + return left.length < right.length; + } + }; + + mutable std::vector flatDict_; + std::unordered_map keyToIndex; + uint64_t totalLength_; +}; + +// insert a new string into dictionary, return its insertion order +size_t NewSortedStringDictionary::insert(const char * str, size_t len) +{ + size_t index = flatDict_.size(); + auto ret = keyToIndex.emplace(std::string(str, len), index); + if (ret.second) + { + flatDict_.emplace_back(ret.first->first.data(), ret.first->first.size(), index); + totalLength_ += len; + } + return ret.first->second; +} + +/** + * Reorder input index buffer from insertion order to dictionary order + * + * We require this function because string values are buffered by indexes + * in their insertion order. Until the entire dictionary is complete can + * we get their sorted indexes in the dictionary in that ORC specification + * demands dictionary should be ordered. Therefore this function transforms + * the indexes from insertion order to dictionary value order for final + * output. + */ +void NewSortedStringDictionary::reorder(std::vector & idxBuffer) const +{ + // iterate the dictionary to get mapping from insertion order to value order + std::vector mapping(flatDict_.size()); + for (size_t i = 0; i < flatDict_.size(); ++i) + { + mapping[flatDict_[i].index] = i; + } + + // do the transformation + for (size_t i = 0; i != idxBuffer.size(); ++i) + { + idxBuffer[i] = static_cast(mapping[static_cast(idxBuffer[i])]); + } +} + +// get dict entries in insertion order +void NewSortedStringDictionary::getEntriesInInsertionOrder(std::vector & entries) const +{ + std::sort( + flatDict_.begin(), + flatDict_.end(), + [](const DictEntryWithIndex & left, const DictEntryWithIndex & right) { return left.index < right.index; }); + + entries.resize(flatDict_.size()); + for (size_t i = 0; i < flatDict_.size(); ++i) + { + entries[i] = &(flatDict_[i].entry); + } +} + +// return count of entries +size_t NewSortedStringDictionary::size() const +{ + return flatDict_.size(); +} + +// return total length of strings in the dictioanry +uint64_t NewSortedStringDictionary::length() const +{ + return totalLength_; +} + +void NewSortedStringDictionary::clear() +{ + totalLength_ = 0; + keyToIndex.clear(); + flatDict_.clear(); +} + +static std::vector mockStrings() +{ + std::vector res(1000000); + for (auto & s : res) + { + s = "test string dictionary " + std::to_string(rand() % 100); + } + return res; +} + +template +static NO_INLINE std::unique_ptr createAndWriteStringDictionary(const std::vector & strs) +{ + auto dict = std::make_unique(); + for (const auto & str : strs) + { + auto index = dict->insert(str.data(), str.size()); + dict->idxInDictBuffer.push_back(index); + } + dict->reorder(dict->idxInDictBuffer); + + return dict; +} + +template +static void BM_writeStringDictionary(benchmark::State & state) +{ + auto strs = mockStrings(); + for (auto _ : state) + { + auto dict = createAndWriteStringDictionary(strs); + benchmark::DoNotOptimize(dict); + } +} + +BENCHMARK_TEMPLATE(BM_writeStringDictionary, OldSortedStringDictionary); +BENCHMARK_TEMPLATE(BM_writeStringDictionary, NewSortedStringDictionary); + From d6df83d561b7d123d1bc854179a7c5435db7081c Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 26 Aug 2024 14:57:51 +0800 Subject: [PATCH 017/103] add uts about orc string encode --- src/Common/benchmarks/CMakeLists.txt | 7 ++++++- .../benchmarks/orc_string_dictionary.cpp | 19 ++++++++++++++----- 2 files changed, 20 insertions(+), 6 deletions(-) diff --git a/src/Common/benchmarks/CMakeLists.txt b/src/Common/benchmarks/CMakeLists.txt index 57ed837db8b..690ced1eb88 100644 --- a/src/Common/benchmarks/CMakeLists.txt +++ b/src/Common/benchmarks/CMakeLists.txt @@ -1,4 +1,4 @@ -clickhouse_add_executable(integer_hash_tables_and_hashes integer_hash_tables_and_hashes.cpp) +clickhouse_add_executable(integer_hash_tables_and_hashes integer_hash_tables_and_hashes.cpp orc_string_dictionary.cpp) target_link_libraries (integer_hash_tables_and_hashes PRIVATE ch_contrib::gbenchmark_all dbms @@ -7,3 +7,8 @@ target_link_libraries (integer_hash_tables_and_hashes PRIVATE ch_contrib::wyhash ch_contrib::farmhash ch_contrib::xxHash) + +clickhouse_add_executable(orc_string_dictionary orc_string_dictionary.cpp) +target_link_libraries (orc_string_dictionary PRIVATE + ch_contrib::gbenchmark_all + dbms) diff --git a/src/Common/benchmarks/orc_string_dictionary.cpp b/src/Common/benchmarks/orc_string_dictionary.cpp index bde7d03e864..7d4a3b1ab56 100644 --- a/src/Common/benchmarks/orc_string_dictionary.cpp +++ b/src/Common/benchmarks/orc_string_dictionary.cpp @@ -262,12 +262,13 @@ void NewSortedStringDictionary::clear() flatDict_.clear(); } +template static std::vector mockStrings() { std::vector res(1000000); for (auto & s : res) { - s = "test string dictionary " + std::to_string(rand() % 100); + s = "test string dictionary " + std::to_string(rand() % cardinality); } return res; } @@ -286,10 +287,10 @@ static NO_INLINE std::unique_ptr createAndWriteStringDictionary( return dict; } -template +template static void BM_writeStringDictionary(benchmark::State & state) { - auto strs = mockStrings(); + auto strs = mockStrings(); for (auto _ : state) { auto dict = createAndWriteStringDictionary(strs); @@ -297,6 +298,14 @@ static void BM_writeStringDictionary(benchmark::State & state) } } -BENCHMARK_TEMPLATE(BM_writeStringDictionary, OldSortedStringDictionary); -BENCHMARK_TEMPLATE(BM_writeStringDictionary, NewSortedStringDictionary); +BENCHMARK_TEMPLATE(BM_writeStringDictionary, OldSortedStringDictionary, 10); +BENCHMARK_TEMPLATE(BM_writeStringDictionary, NewSortedStringDictionary, 10); +BENCHMARK_TEMPLATE(BM_writeStringDictionary, OldSortedStringDictionary, 100); +BENCHMARK_TEMPLATE(BM_writeStringDictionary, NewSortedStringDictionary, 100); +BENCHMARK_TEMPLATE(BM_writeStringDictionary, OldSortedStringDictionary, 1000); +BENCHMARK_TEMPLATE(BM_writeStringDictionary, NewSortedStringDictionary, 1000); +BENCHMARK_TEMPLATE(BM_writeStringDictionary, OldSortedStringDictionary, 10000); +BENCHMARK_TEMPLATE(BM_writeStringDictionary, NewSortedStringDictionary, 10000); +BENCHMARK_TEMPLATE(BM_writeStringDictionary, OldSortedStringDictionary, 100000); +BENCHMARK_TEMPLATE(BM_writeStringDictionary, NewSortedStringDictionary, 100000); From 7aaa0289e157f90441a837544efd485450c032d7 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 26 Aug 2024 14:58:57 +0800 Subject: [PATCH 018/103] revert files --- src/Common/CMakeLists.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/CMakeLists.txt b/src/Common/CMakeLists.txt index 7cc8dd25cd9..d4802c28f53 100644 --- a/src/Common/CMakeLists.txt +++ b/src/Common/CMakeLists.txt @@ -1,6 +1,6 @@ -#if (ENABLE_BENCHMARKS) +if (ENABLE_BENCHMARKS) add_subdirectory(benchmarks) -#endif() +endif() if (ENABLE_EXAMPLES) add_subdirectory(examples) From aa4688a98203adb24ad3162a0afebb5e6be7be35 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 27 Aug 2024 12:25:22 +0800 Subject: [PATCH 019/103] fix style --- src/Common/benchmarks/orc_string_dictionary.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Common/benchmarks/orc_string_dictionary.cpp b/src/Common/benchmarks/orc_string_dictionary.cpp index 7d4a3b1ab56..a542ed2182b 100644 --- a/src/Common/benchmarks/orc_string_dictionary.cpp +++ b/src/Common/benchmarks/orc_string_dictionary.cpp @@ -25,7 +25,7 @@ public: size_t size() const; - // return total length of strings in the dictioanry + // return total length of strings in the dictionary uint64_t length() const; void clear(); @@ -112,7 +112,7 @@ size_t OldSortedStringDictionary::size() const return dict.size(); } -// return total length of strings in the dictioanry +// return total length of strings in the dictionary uint64_t OldSortedStringDictionary::length() const { return totalLength; @@ -160,7 +160,7 @@ public: // return count of entries size_t size() const; - // return total length of strings in the dictioanry + // return total length of strings in the dictionary uint64_t length() const; void clear(); @@ -249,7 +249,7 @@ size_t NewSortedStringDictionary::size() const return flatDict_.size(); } -// return total length of strings in the dictioanry +// return total length of strings in the dictionary uint64_t NewSortedStringDictionary::length() const { return totalLength_; From ae582120aee9065d547c49c1bd13e76d7afacc84 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 28 Aug 2024 20:56:33 +0800 Subject: [PATCH 020/103] change as request --- src/Core/SettingsChangesHistory.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index b9c18c88652..803f19b894a 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -71,6 +71,7 @@ static std::initializer_list Date: Tue, 3 Sep 2024 16:10:46 +0300 Subject: [PATCH 021/103] Allow custom settings in database engine --- src/Databases/DatabaseFactory.cpp | 21 +++++++---------- src/Databases/DatabaseFactory.h | 23 ++++++++++++++++--- src/Databases/DatabaseFilesystem.cpp | 2 +- src/Databases/DatabaseHDFS.cpp | 2 +- src/Databases/DatabaseLazy.cpp | 2 +- src/Databases/DatabaseReplicated.cpp | 2 +- src/Databases/DatabaseS3.cpp | 2 +- .../MySQL/DatabaseMaterializedMySQL.cpp | 10 ++++++-- src/Databases/MySQL/DatabaseMySQL.cpp | 2 +- .../DatabaseMaterializedPostgreSQL.cpp | 6 ++++- .../PostgreSQL/DatabasePostgreSQL.cpp | 2 +- src/Databases/SQLite/DatabaseSQLite.cpp | 2 +- 12 files changed, 49 insertions(+), 27 deletions(-) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 05a5e057c55..358cdccf8c5 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -66,28 +66,23 @@ void validate(const ASTCreateQuery & create_query) { auto * storage = create_query.storage; - /// Check engine may have arguments - static const std::unordered_set engines_with_arguments{"MySQL", "MaterializeMySQL", "MaterializedMySQL", - "Lazy", "Replicated", "PostgreSQL", "MaterializedPostgreSQL", "SQLite", "Filesystem", "S3", "HDFS"}; - const String & engine_name = storage->engine->name; - bool engine_may_have_arguments = engines_with_arguments.contains(engine_name); + const EngineFeatures & engine_features = database_engines.at(engine_name).features; - if (storage->engine->arguments && !engine_may_have_arguments) + /// Check engine may have arguments + if (storage->engine->arguments && !engine_features.supports_arguments) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine `{}` cannot have arguments", engine_name); /// Check engine may have settings - bool may_have_settings = endsWith(engine_name, "MySQL") || engine_name == "Replicated" || engine_name == "MaterializedPostgreSQL"; bool has_unexpected_element = storage->engine->parameters || storage->partition_by || storage->primary_key || storage->order_by || storage->sample_by; - if (has_unexpected_element || (!may_have_settings && storage->settings)) + if (has_unexpected_element || (!engine_features.supports_settings && storage->settings)) throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_AST, "Database engine `{}` cannot have parameters, primary_key, order_by, sample_by, settings", engine_name); /// Check engine with table overrides - static const std::unordered_set engines_with_table_overrides{"MaterializeMySQL", "MaterializedMySQL", "MaterializedPostgreSQL"}; - if (create_query.table_overrides && !engines_with_table_overrides.contains(engine_name)) + if (create_query.table_overrides && !engine_features.supports_table_overrides) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine `{}` cannot have table overrides", engine_name); } @@ -121,9 +116,9 @@ DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & m return impl; } -void DatabaseFactory::registerDatabase(const std::string & name, CreatorFn creator_fn) +void DatabaseFactory::registerDatabase(const std::string & name, CreatorFn creator_fn, EngineFeatures features) { - if (!database_engines.emplace(name, std::move(creator_fn)).second) + if (!database_engines.emplace(name, Creator{std::move(creator_fn), features}).second) throw Exception(ErrorCodes::LOGICAL_ERROR, "DatabaseFactory: the database engine name '{}' is not unique", name); } @@ -154,7 +149,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String .context = context}; // creator_fn creates and returns a DatabasePtr with the supplied arguments - auto creator_fn = database_engines.at(engine_name); + auto creator_fn = database_engines.at(engine_name).creator_fn; return creator_fn(arguments); } diff --git a/src/Databases/DatabaseFactory.h b/src/Databases/DatabaseFactory.h index 494c9e0076e..36275820cbe 100644 --- a/src/Databases/DatabaseFactory.h +++ b/src/Databases/DatabaseFactory.h @@ -43,13 +43,30 @@ public: ContextPtr & context; }; - DatabasePtr get(const ASTCreateQuery & create, const String & metadata_path, ContextPtr context); + struct EngineFeatures + { + bool supports_arguments = false; + bool supports_settings = false; + bool supports_table_overrides = false; + }; using CreatorFn = std::function; - using DatabaseEngines = std::unordered_map; + struct Creator + { + CreatorFn creator_fn; + EngineFeatures features; + }; - void registerDatabase(const std::string & name, CreatorFn creator_fn); + DatabasePtr get(const ASTCreateQuery & create, const String & metadata_path, ContextPtr context); + + using DatabaseEngines = std::unordered_map; + + void registerDatabase(const std::string & name, CreatorFn creator_fn, EngineFeatures features = EngineFeatures{ + supports_arguments = false, + supports_parameters = false, + supports_table_overrides = false, + }); const DatabaseEngines & getDatabaseEngines() const { return database_engines; } diff --git a/src/Databases/DatabaseFilesystem.cpp b/src/Databases/DatabaseFilesystem.cpp index 31701e665a1..4b50e79da4a 100644 --- a/src/Databases/DatabaseFilesystem.cpp +++ b/src/Databases/DatabaseFilesystem.cpp @@ -257,6 +257,6 @@ void registerDatabaseFilesystem(DatabaseFactory & factory) return std::make_shared(args.database_name, init_path, args.context); }; - factory.registerDatabase("Filesystem", create_fn); + factory.registerDatabase("Filesystem", create_fn, {.supports_arguments = true}); } } diff --git a/src/Databases/DatabaseHDFS.cpp b/src/Databases/DatabaseHDFS.cpp index 7fa67a5678e..ceca2666e49 100644 --- a/src/Databases/DatabaseHDFS.cpp +++ b/src/Databases/DatabaseHDFS.cpp @@ -253,7 +253,7 @@ void registerDatabaseHDFS(DatabaseFactory & factory) return std::make_shared(args.database_name, source_url, args.context); }; - factory.registerDatabase("HDFS", create_fn); + factory.registerDatabase("HDFS", create_fn, {.supports_arguments = true}); } } // DB diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index 2ccdd8510a8..0a4b02c4917 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -398,6 +398,6 @@ void registerDatabaseLazy(DatabaseFactory & factory) cache_expiration_time_seconds, args.context); }; - factory.registerDatabase("Lazy", create_fn); + factory.registerDatabase("Lazy", create_fn, {.supports_arguments = true}); } } diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 8e3378bcc12..53064703c2c 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -1949,6 +1949,6 @@ void registerDatabaseReplicated(DatabaseFactory & factory) replica_name, std::move(database_replicated_settings), args.context); }; - factory.registerDatabase("Replicated", create_fn); + factory.registerDatabase("Replicated", create_fn, {.supports_arguments = true, .supports_settings = true}); } } diff --git a/src/Databases/DatabaseS3.cpp b/src/Databases/DatabaseS3.cpp index 2b2d978a846..d80cc6d0953 100644 --- a/src/Databases/DatabaseS3.cpp +++ b/src/Databases/DatabaseS3.cpp @@ -326,7 +326,7 @@ void registerDatabaseS3(DatabaseFactory & factory) return std::make_shared(args.database_name, config, args.context); }; - factory.registerDatabase("S3", create_fn); + factory.registerDatabase("S3", create_fn, {.supports_arguments = true}); } } #endif diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp index 2f5477a6b9d..50c7a5bf588 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp @@ -290,8 +290,14 @@ void registerDatabaseMaterializedMySQL(DatabaseFactory & factory) binlog_client, std::move(materialize_mode_settings)); }; - factory.registerDatabase("MaterializeMySQL", create_fn); - factory.registerDatabase("MaterializedMySQL", create_fn); + + DatabaseFactory::Features features{ + .supports_arguments = true, + .supports_settings = true, + .supports_table_overrides = true, + } + factory.registerDatabase("MaterializeMySQL", create_fn, features); + factory.registerDatabase("MaterializedMySQL", create_fn, features); } } diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index 7aa29018f4d..3b72f2aeae5 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -584,7 +584,7 @@ void registerDatabaseMySQL(DatabaseFactory & factory) throw Exception(ErrorCodes::CANNOT_CREATE_DATABASE, "Cannot create MySQL database, because {}", exception_message); } }; - factory.registerDatabase("MySQL", create_fn); + factory.registerDatabase("MySQL", create_fn, {.supports_arguments = true, .supports_settings = true}); } } diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index 6b0548b85c7..ed62398e594 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -546,7 +546,11 @@ void registerDatabaseMaterializedPostgreSQL(DatabaseFactory & factory) args.database_name, configuration.database, connection_info, std::move(postgresql_replica_settings)); }; - factory.registerDatabase("MaterializedPostgreSQL", create_fn); + factory.registerDatabase("MaterializedPostgreSQL", create_fn, { + .supports_arguments = true, + .supports_settings = true, + .supports_table_overrides = true, + }); } } diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 032fc33ea16..0eafd1c3b5b 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -558,7 +558,7 @@ void registerDatabasePostgreSQL(DatabaseFactory & factory) pool, use_table_cache); }; - factory.registerDatabase("PostgreSQL", create_fn); + factory.registerDatabase("PostgreSQL", create_fn, {.supports_arguments = true}); } } diff --git a/src/Databases/SQLite/DatabaseSQLite.cpp b/src/Databases/SQLite/DatabaseSQLite.cpp index 471730fce29..5af9eb1920e 100644 --- a/src/Databases/SQLite/DatabaseSQLite.cpp +++ b/src/Databases/SQLite/DatabaseSQLite.cpp @@ -220,7 +220,7 @@ void registerDatabaseSQLite(DatabaseFactory & factory) return std::make_shared(args.context, engine_define, args.create_query.attach, database_path); }; - factory.registerDatabase("SQLite", create_fn); + factory.registerDatabase("SQLite", create_fn, {.supports_arguments = true}); } } From e874c6e1de7a8b1bc92fd1e60dcdbc53caac3ded Mon Sep 17 00:00:00 2001 From: NikBarykin Date: Tue, 3 Sep 2024 18:58:39 +0300 Subject: [PATCH 022/103] Fix typo --- src/Databases/DatabaseFactory.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseFactory.h b/src/Databases/DatabaseFactory.h index 36275820cbe..09ebd2bb2b2 100644 --- a/src/Databases/DatabaseFactory.h +++ b/src/Databases/DatabaseFactory.h @@ -64,7 +64,7 @@ public: void registerDatabase(const std::string & name, CreatorFn creator_fn, EngineFeatures features = EngineFeatures{ supports_arguments = false, - supports_parameters = false, + supports_settings = false, supports_table_overrides = false, }); From 83854cf293bf52376aa9aa5359fa606f4ccd713e Mon Sep 17 00:00:00 2001 From: NikBarykin Date: Tue, 3 Sep 2024 19:13:05 +0300 Subject: [PATCH 023/103] Make method of DatabaseFactory --- src/Databases/DatabaseFactory.cpp | 5 +---- src/Databases/DatabaseFactory.h | 10 +++++++--- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 358cdccf8c5..d97474bd245 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -59,10 +59,7 @@ void cckMetadataPathForOrdinary(const ASTCreateQuery & create, const String & me } -/// validate validates the database engine that's specified in the create query for -/// engine arguments, settings and table overrides. -void validate(const ASTCreateQuery & create_query) - +void DatabaseFactory::validate(const ASTCreateQuery & create_query) const { auto * storage = create_query.storage; diff --git a/src/Databases/DatabaseFactory.h b/src/Databases/DatabaseFactory.h index 09ebd2bb2b2..ede4394e435 100644 --- a/src/Databases/DatabaseFactory.h +++ b/src/Databases/DatabaseFactory.h @@ -63,9 +63,9 @@ public: using DatabaseEngines = std::unordered_map; void registerDatabase(const std::string & name, CreatorFn creator_fn, EngineFeatures features = EngineFeatures{ - supports_arguments = false, - supports_settings = false, - supports_table_overrides = false, + .supports_arguments = false, + .supports_settings = false, + .supports_table_overrides = false, }); const DatabaseEngines & getDatabaseEngines() const { return database_engines; } @@ -82,6 +82,10 @@ private: DatabaseEngines database_engines; DatabasePtr getImpl(const ASTCreateQuery & create, const String & metadata_path, ContextPtr context); + + /// validate validates the database engine that's specified in the create query for + /// engine arguments, settings and table overrides. + void validate(const ASTCreateQuery & create_query) const; }; } From 5f5acd3c4480c3f971b1ff33fc779bb767c2c0fc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 5 Sep 2024 13:22:45 +0000 Subject: [PATCH 024/103] Refactor ArrayJoin step. --- src/Functions/FunctionStringOrArrayToT.h | 3 +- src/Functions/array/arrayResize.cpp | 176 ++++++++---------- src/Functions/array/emptyArrayToSingle.cpp | 39 +--- src/Functions/array/length.cpp | 60 +----- src/Interpreters/ArrayJoinAction.cpp | 63 +++++-- src/Interpreters/ArrayJoinAction.h | 6 +- src/Interpreters/ExpressionAnalyzer.cpp | 5 +- src/Interpreters/InterpreterSelectQuery.cpp | 8 +- src/Planner/PlannerJoinTree.cpp | 10 +- src/Processors/QueryPlan/ArrayJoinStep.cpp | 20 +- src/Processors/QueryPlan/ArrayJoinStep.h | 10 +- .../Optimizations/filterPushDown.cpp | 3 +- .../Optimizations/liftUpArrayJoin.cpp | 4 +- .../Optimizations/optimizeReadInOrder.cpp | 4 +- .../Transforms/ArrayJoinTransform.cpp | 6 +- .../Transforms/ArrayJoinTransform.h | 2 +- 16 files changed, 185 insertions(+), 234 deletions(-) diff --git a/src/Functions/FunctionStringOrArrayToT.h b/src/Functions/FunctionStringOrArrayToT.h index 40f780d82a8..cd98e0f5875 100644 --- a/src/Functions/FunctionStringOrArrayToT.h +++ b/src/Functions/FunctionStringOrArrayToT.h @@ -27,7 +27,8 @@ class FunctionStringOrArrayToT : public IFunction { public: static constexpr auto name = Name::name; - static FunctionPtr create(ContextPtr) + static FunctionPtr create(ContextPtr) { return createImpl(); } + static FunctionPtr createImpl() { return std::make_shared(); } diff --git a/src/Functions/array/arrayResize.cpp b/src/Functions/array/arrayResize.cpp index 8f4ea69fc5d..fe928f22d38 100644 --- a/src/Functions/array/arrayResize.cpp +++ b/src/Functions/array/arrayResize.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -21,117 +21,99 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } -class FunctionArrayResize : public IFunction +DataTypePtr FunctionArrayResize::getReturnTypeImpl(const DataTypes & arguments) const { -public: - static constexpr auto name = "arrayResize"; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } + const size_t number_of_arguments = arguments.size(); - String getName() const override { return name; } + if (number_of_arguments < 2 || number_of_arguments > 3) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be 2 or 3", + getName(), number_of_arguments); - bool isVariadic() const override { return true; } - size_t getNumberOfArguments() const override { return 0; } + if (arguments[0]->onlyNull()) + return arguments[0]; - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + const auto * array_type = typeid_cast(arguments[0].get()); + if (!array_type) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "First argument for function {} must be an array but it has type {}.", + getName(), arguments[0]->getName()); - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + if (WhichDataType(array_type->getNestedType()).isNothing()) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function {} cannot resize {}", getName(), array_type->getName()); + + if (!isInteger(removeNullable(arguments[1])) && !arguments[1]->onlyNull()) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Argument {} for function {} must be integer but it has type {}.", + toString(1), getName(), arguments[1]->getName()); + + if (number_of_arguments == 2) + return arguments[0]; + else /* if (number_of_arguments == 3) */ + return std::make_shared(getLeastSupertype(DataTypes{array_type->getNestedType(), arguments[2]})); +} + +ColumnPtr FunctionArrayResize::executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type, size_t input_rows_count) const +{ + if (return_type->onlyNull()) + return return_type->createColumnConstWithDefaultValue(input_rows_count); + + auto result_column = return_type->createColumn(); + + auto array_column = arguments[0].column; + auto size_column = arguments[1].column; + + if (!arguments[0].type->equals(*return_type)) + array_column = castColumn(arguments[0], return_type); + + const DataTypePtr & return_nested_type = typeid_cast(*return_type).getNestedType(); + size_t size = array_column->size(); + + ColumnPtr appended_column; + if (arguments.size() == 3) { - const size_t number_of_arguments = arguments.size(); + appended_column = arguments[2].column; + if (!arguments[2].type->equals(*return_nested_type)) + appended_column = castColumn(arguments[2], return_nested_type); + } + else + appended_column = return_nested_type->createColumnConstWithDefaultValue(size); - if (number_of_arguments < 2 || number_of_arguments > 3) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, should be 2 or 3", - getName(), number_of_arguments); + std::unique_ptr array_source; + std::unique_ptr value_source; - if (arguments[0]->onlyNull()) - return arguments[0]; + bool is_const = false; - const auto * array_type = typeid_cast(arguments[0].get()); - if (!array_type) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "First argument for function {} must be an array but it has type {}.", - getName(), arguments[0]->getName()); - - if (WhichDataType(array_type->getNestedType()).isNothing()) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function {} cannot resize {}", getName(), array_type->getName()); - - if (!isInteger(removeNullable(arguments[1])) && !arguments[1]->onlyNull()) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Argument {} for function {} must be integer but it has type {}.", - toString(1), getName(), arguments[1]->getName()); - - if (number_of_arguments == 2) - return arguments[0]; - else /* if (number_of_arguments == 3) */ - return std::make_shared(getLeastSupertype(DataTypes{array_type->getNestedType(), arguments[2]})); + if (const auto * const_array_column = typeid_cast(array_column.get())) + { + is_const = true; + array_column = const_array_column->getDataColumnPtr(); } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type, size_t input_rows_count) const override + if (const auto * argument_column_array = typeid_cast(array_column.get())) + array_source = GatherUtils::createArraySource(*argument_column_array, is_const, size); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "First arguments for function {} must be array.", getName()); + + + bool is_appended_const = false; + if (const auto * const_appended_column = typeid_cast(appended_column.get())) { - if (return_type->onlyNull()) - return return_type->createColumnConstWithDefaultValue(input_rows_count); - - auto result_column = return_type->createColumn(); - - auto array_column = arguments[0].column; - auto size_column = arguments[1].column; - - if (!arguments[0].type->equals(*return_type)) - array_column = castColumn(arguments[0], return_type); - - const DataTypePtr & return_nested_type = typeid_cast(*return_type).getNestedType(); - size_t size = array_column->size(); - - ColumnPtr appended_column; - if (arguments.size() == 3) - { - appended_column = arguments[2].column; - if (!arguments[2].type->equals(*return_nested_type)) - appended_column = castColumn(arguments[2], return_nested_type); - } - else - appended_column = return_nested_type->createColumnConstWithDefaultValue(size); - - std::unique_ptr array_source; - std::unique_ptr value_source; - - bool is_const = false; - - if (const auto * const_array_column = typeid_cast(array_column.get())) - { - is_const = true; - array_column = const_array_column->getDataColumnPtr(); - } - - if (const auto * argument_column_array = typeid_cast(array_column.get())) - array_source = GatherUtils::createArraySource(*argument_column_array, is_const, size); - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "First arguments for function {} must be array.", getName()); - - - bool is_appended_const = false; - if (const auto * const_appended_column = typeid_cast(appended_column.get())) - { - is_appended_const = true; - appended_column = const_appended_column->getDataColumnPtr(); - } - - value_source = GatherUtils::createValueSource(*appended_column, is_appended_const, size); - - auto sink = GatherUtils::createArraySink(typeid_cast(*result_column), size); - - if (isColumnConst(*size_column)) - GatherUtils::resizeConstantSize(*array_source, *value_source, *sink, size_column->getInt(0)); - else - GatherUtils::resizeDynamicSize(*array_source, *value_source, *sink, *size_column); - - return result_column; + is_appended_const = true; + appended_column = const_appended_column->getDataColumnPtr(); } - bool useDefaultImplementationForConstants() const override { return true; } - bool useDefaultImplementationForNulls() const override { return false; } -}; + value_source = GatherUtils::createValueSource(*appended_column, is_appended_const, size); + auto sink = GatherUtils::createArraySink(typeid_cast(*result_column), size); + + if (isColumnConst(*size_column)) + GatherUtils::resizeConstantSize(*array_source, *value_source, *sink, size_column->getInt(0)); + else + GatherUtils::resizeDynamicSize(*array_source, *value_source, *sink, *size_column); + + return result_column; +} REGISTER_FUNCTION(ArrayResize) { diff --git a/src/Functions/array/emptyArrayToSingle.cpp b/src/Functions/array/emptyArrayToSingle.cpp index 2071abf9911..5699a4024a1 100644 --- a/src/Functions/array/emptyArrayToSingle.cpp +++ b/src/Functions/array/emptyArrayToSingle.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -20,35 +20,6 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } - -/** emptyArrayToSingle(arr) - replace empty arrays with arrays of one element with a default value. - */ -class FunctionEmptyArrayToSingle : public IFunction -{ -public: - static constexpr auto name = "emptyArrayToSingle"; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } - - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 1; } - bool useDefaultImplementationForConstants() const override { return true; } - bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - const DataTypeArray * array_type = checkAndGetDataType(arguments[0].get()); - if (!array_type) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Argument for function {} must be array.", getName()); - - return arguments[0]; - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override; -}; - - namespace { namespace FunctionEmptyArrayToSingleImpl @@ -366,6 +337,14 @@ namespace } } +DataTypePtr FunctionEmptyArrayToSingle::getReturnTypeImpl(const DataTypes & arguments) const +{ + const DataTypeArray * array_type = checkAndGetDataType(arguments[0].get()); + if (!array_type) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Argument for function {} must be array.", getName()); + + return arguments[0]; +} ColumnPtr FunctionEmptyArrayToSingle::executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const { diff --git a/src/Functions/array/length.cpp b/src/Functions/array/length.cpp index 760506194fa..949a5441e58 100644 --- a/src/Functions/array/length.cpp +++ b/src/Functions/array/length.cpp @@ -1,65 +1,7 @@ -#include -#include -#include - +#include namespace DB { -namespace ErrorCodes -{ - extern const int ILLEGAL_TYPE_OF_ARGUMENT; -} - -/** Calculates the length of a string in bytes. - */ -struct LengthImpl -{ - static constexpr auto is_fixed_to_constant = true; - - static void vector(const ColumnString::Chars & /*data*/, const ColumnString::Offsets & offsets, PaddedPODArray & res, size_t input_rows_count) - { - for (size_t i = 0; i < input_rows_count; ++i) - res[i] = offsets[i] - 1 - offsets[i - 1]; - } - - static void vectorFixedToConstant(const ColumnString::Chars & /*data*/, size_t n, UInt64 & res, size_t) - { - res = n; - } - - static void vectorFixedToVector(const ColumnString::Chars & /*data*/, size_t /*n*/, PaddedPODArray & /*res*/, size_t) - { - } - - static void array(const ColumnString::Offsets & offsets, PaddedPODArray & res, size_t input_rows_count) - { - for (size_t i = 0; i < input_rows_count; ++i) - res[i] = offsets[i] - offsets[i - 1]; - } - - [[noreturn]] static void uuid(const ColumnUUID::Container &, size_t &, PaddedPODArray &, size_t) - { - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Cannot apply function length to UUID argument"); - } - - [[noreturn]] static void ipv6(const ColumnIPv6::Container &, size_t &, PaddedPODArray &, size_t) - { - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Cannot apply function length to IPv6 argument"); - } - - [[noreturn]] static void ipv4(const ColumnIPv4::Container &, size_t &, PaddedPODArray &, size_t) - { - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Cannot apply function length to IPv4 argument"); - } -}; - - -struct NameLength -{ - static constexpr auto name = "length"; -}; - -using FunctionLength = FunctionStringOrArrayToT; REGISTER_FUNCTION(Length) { diff --git a/src/Interpreters/ArrayJoinAction.cpp b/src/Interpreters/ArrayJoinAction.cpp index df7a0b48057..802d38b0c03 100644 --- a/src/Interpreters/ArrayJoinAction.cpp +++ b/src/Interpreters/ArrayJoinAction.cpp @@ -6,6 +6,9 @@ #include #include #include +#include +#include +#include #include #include @@ -59,26 +62,27 @@ ColumnWithTypeAndName convertArrayJoinColumn(const ColumnWithTypeAndName & src_c return array_col; } -ArrayJoinAction::ArrayJoinAction(const NameSet & array_joined_columns_, bool array_join_is_left, ContextPtr context) +ArrayJoinAction::ArrayJoinAction(const NameSet & array_joined_columns_, bool is_left_, bool is_unaligned_, size_t max_block_size_) : columns(array_joined_columns_) - , is_left(array_join_is_left) - , is_unaligned(context->getSettingsRef().enable_unaligned_array_join) - , max_block_size(context->getSettingsRef().max_block_size) + , is_left(is_left_) + , is_unaligned(is_unaligned_) + , max_block_size(max_block_size_) + // , is_unaligned(context->getSettingsRef().enable_unaligned_array_join) + // , max_block_size(context->getSettingsRef().max_block_size) { if (columns.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "No arrays to join"); if (is_unaligned) { - function_length = FunctionFactory::instance().get("length", context); - function_greatest = FunctionFactory::instance().get("greatest", context); - function_array_resize = FunctionFactory::instance().get("arrayResize", context); + function_length = std::make_unique(FunctionLength::createImpl()); + function_array_resize = std::make_unique(FunctionArrayResize::createImpl()); } else if (is_left) - function_builder = FunctionFactory::instance().get("emptyArrayToSingle", context); + function_builder = std::make_unique(FunctionEmptyArrayToSingle::createImpl()); } -void ArrayJoinAction::prepare(ColumnsWithTypeAndName & sample) const +void ArrayJoinAction::prepare(const NameSet & columns, ColumnsWithTypeAndName & sample) { for (auto & current : sample) { @@ -103,6 +107,35 @@ ArrayJoinResultIteratorPtr ArrayJoinAction::execute(Block block) return std::make_unique(this, std::move(block)); } +static void updateMaxLength(ColumnUInt64 & max_length, UInt64 length) +{ + for (auto & value : max_length.getData()) + value = std::max(value, length); +} + +static void updateMaxLength(ColumnUInt64 & max_length, const IColumn & length) +{ + if (const auto * length_const = typeid_cast(&length)) + { + updateMaxLength(max_length, length_const->getUInt(0)); + return; + } + + const auto * length_uint64 = typeid_cast(&length); + if (!length_uint64) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected UInt64 for array length, got {}", length.getName()); + + auto & max_lenght_data = max_length.getData(); + const auto & length_data = length_uint64->getData(); + size_t num_rows = max_lenght_data.size(); + if (num_rows != length_data.size()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Different columns sizes in ARRAY JOIN: {} and {}", num_rows, length_data.size()); + + for (size_t row = 0; row < num_rows; ++row) + max_lenght_data[row] = std::max(max_lenght_data[row], length_data[row]); +} ArrayJoinResultIterator::ArrayJoinResultIterator(const ArrayJoinAction * array_join_, Block block_) : array_join(array_join_), block(std::move(block_)), total_rows(block.rows()), current_row(0) @@ -111,7 +144,6 @@ ArrayJoinResultIterator::ArrayJoinResultIterator(const ArrayJoinAction * array_j bool is_unaligned = array_join->is_unaligned; bool is_left = array_join->is_left; const auto & function_length = array_join->function_length; - const auto & function_greatest = array_join->function_greatest; const auto & function_array_resize = array_join->function_array_resize; const auto & function_builder = array_join->function_builder; @@ -125,11 +157,7 @@ ArrayJoinResultIterator::ArrayJoinResultIterator(const ArrayJoinAction * array_j /// Resize all array joined columns to the longest one, (at least 1 if LEFT ARRAY JOIN), padded with default values. auto rows = block.rows(); auto uint64 = std::make_shared(); - ColumnWithTypeAndName column_of_max_length{{}, uint64, {}}; - if (is_left) - column_of_max_length = ColumnWithTypeAndName(uint64->createColumnConst(rows, 1u), uint64, {}); - else - column_of_max_length = ColumnWithTypeAndName(uint64->createColumnConst(rows, 0u), uint64, {}); + auto max_length = ColumnUInt64::create(rows, (is_left ? 1u : 0u)); for (const auto & name : columns) { @@ -138,11 +166,10 @@ ArrayJoinResultIterator::ArrayJoinResultIterator(const ArrayJoinAction * array_j ColumnWithTypeAndName array_col = convertArrayJoinColumn(src_col); ColumnsWithTypeAndName tmp_block{array_col}; //, {{}, uint64, {}}}; auto len_col = function_length->build(tmp_block)->execute(tmp_block, uint64, rows); - - ColumnsWithTypeAndName tmp_block2{column_of_max_length, {len_col, uint64, {}}}; - column_of_max_length.column = function_greatest->build(tmp_block2)->execute(tmp_block2, uint64, rows); + updateMaxLength(*max_length, *len_col); } + ColumnWithTypeAndName column_of_max_length{std::move(max_length), uint64, {}}; for (const auto & name : columns) { auto & src_col = block.getByName(name); diff --git a/src/Interpreters/ArrayJoinAction.h b/src/Interpreters/ArrayJoinAction.h index 603f22ef245..287eabaac65 100644 --- a/src/Interpreters/ArrayJoinAction.h +++ b/src/Interpreters/ArrayJoinAction.h @@ -33,14 +33,14 @@ public: /// For unaligned [LEFT] ARRAY JOIN FunctionOverloadResolverPtr function_length; - FunctionOverloadResolverPtr function_greatest; + //FunctionOverloadResolverPtr function_greatest; FunctionOverloadResolverPtr function_array_resize; /// For LEFT ARRAY JOIN. FunctionOverloadResolverPtr function_builder; - ArrayJoinAction(const NameSet & array_joined_columns_, bool array_join_is_left, ContextPtr context); - void prepare(ColumnsWithTypeAndName & sample) const; + ArrayJoinAction(const NameSet & array_joined_columns_, bool is_left_, bool is_unaligned_, size_t max_block_size_); + static void prepare(const NameSet & columns, ColumnsWithTypeAndName & sample); ArrayJoinResultIteratorPtr execute(Block block); }; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 7063b2162a0..3315f4a67b2 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -215,7 +215,7 @@ NamesAndTypesList ExpressionAnalyzer::getColumnsAfterArrayJoin(ActionsDAG & acti auto array_join = addMultipleArrayJoinAction(actions, is_array_join_left); auto sample_columns = actions.getResultColumns(); - array_join->prepare(sample_columns); + ArrayJoinAction::prepare(array_join->columns, sample_columns); actions = ActionsDAG(sample_columns); NamesAndTypesList new_columns_after_array_join; @@ -905,7 +905,8 @@ ArrayJoinActionPtr ExpressionAnalyzer::addMultipleArrayJoinAction(ActionsDAG & a result_columns.insert(result_source.first); } - return std::make_shared(result_columns, array_join_is_left, getContext()); + const auto & query_settings = getContext()->getSettingsRef(); + return std::make_shared(result_columns, array_join_is_left, query_settings.enable_unaligned_array_join, query_settings.max_block_size); } ArrayJoinActionPtr SelectQueryExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, ActionsAndProjectInputsFlagPtr & before_array_join, bool only_types) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index ca0e84a5267..1bc48b9026a 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -86,6 +86,7 @@ #include #include #include +#include #include #include #include @@ -1676,7 +1677,12 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

(query_plan.getCurrentDataStream(), expressions.array_join); + = std::make_unique( + query_plan.getCurrentDataStream(), + expressions.array_join->columns, + expressions.array_join->is_left, + expressions.array_join->is_unaligned, + expressions.array_join->max_block_size); array_join_step->setStepDescription("ARRAY JOIN"); query_plan.addStep(std::move(array_join_step)); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index bc31af32a20..1b2a6625828 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1707,8 +1707,14 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_ drop_unused_columns_before_array_join_transform_step->setStepDescription("DROP unused columns before ARRAY JOIN"); plan.addStep(std::move(drop_unused_columns_before_array_join_transform_step)); - auto array_join_action = std::make_shared(array_join_column_names, array_join_node.isLeft(), planner_context->getQueryContext()); - auto array_join_step = std::make_unique(plan.getCurrentDataStream(), std::move(array_join_action)); + const auto & settings = planner_context->getQueryContext()->getSettingsRef(); + auto array_join_step = std::make_unique( + plan.getCurrentDataStream(), + std::move(array_join_column_names), + array_join_node.isLeft(), + settings.enable_unaligned_array_join, + settings.max_block_size); + array_join_step->setStepDescription("ARRAY JOIN"); plan.addStep(std::move(array_join_step)); diff --git a/src/Processors/QueryPlan/ArrayJoinStep.cpp b/src/Processors/QueryPlan/ArrayJoinStep.cpp index 23a0a756f0d..aa721e138cf 100644 --- a/src/Processors/QueryPlan/ArrayJoinStep.cpp +++ b/src/Processors/QueryPlan/ArrayJoinStep.cpp @@ -24,23 +24,27 @@ static ITransformingStep::Traits getTraits() }; } -ArrayJoinStep::ArrayJoinStep(const DataStream & input_stream_, ArrayJoinActionPtr array_join_) +ArrayJoinStep::ArrayJoinStep(const DataStream & input_stream_, NameSet columns_, bool is_left_, bool is_unaligned_, size_t max_block_size_) : ITransformingStep( input_stream_, - ArrayJoinTransform::transformHeader(input_stream_.header, array_join_), + ArrayJoinTransform::transformHeader(input_stream_.header, columns_), getTraits()) - , array_join(std::move(array_join_)) + , columns(std::move(columns_)) + , is_left(is_left_) + , is_unaligned(is_unaligned_) + , max_block_size(max_block_size_) { } void ArrayJoinStep::updateOutputStream() { output_stream = createOutputStream( - input_streams.front(), ArrayJoinTransform::transformHeader(input_streams.front().header, array_join), getDataStreamTraits()); + input_streams.front(), ArrayJoinTransform::transformHeader(input_streams.front().header, columns), getDataStreamTraits()); } void ArrayJoinStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { + auto array_join = std::make_shared(columns, is_left, is_unaligned, max_block_size); pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) { bool on_totals = stream_type == QueryPipelineBuilder::StreamType::Totals; @@ -53,8 +57,8 @@ void ArrayJoinStep::describeActions(FormatSettings & settings) const String prefix(settings.offset, ' '); bool first = true; - settings.out << prefix << (array_join->is_left ? "LEFT " : "") << "ARRAY JOIN "; - for (const auto & column : array_join->columns) + settings.out << prefix << (is_left ? "LEFT " : "") << "ARRAY JOIN "; + for (const auto & column : columns) { if (!first) settings.out << ", "; @@ -68,10 +72,10 @@ void ArrayJoinStep::describeActions(FormatSettings & settings) const void ArrayJoinStep::describeActions(JSONBuilder::JSONMap & map) const { - map.add("Left", array_join->is_left); + map.add("Left", is_left); auto columns_array = std::make_unique(); - for (const auto & column : array_join->columns) + for (const auto & column : columns) columns_array->add(column); map.add("Columns", std::move(columns_array)); diff --git a/src/Processors/QueryPlan/ArrayJoinStep.h b/src/Processors/QueryPlan/ArrayJoinStep.h index 2d9b2ebd0c8..3f2eacc3159 100644 --- a/src/Processors/QueryPlan/ArrayJoinStep.h +++ b/src/Processors/QueryPlan/ArrayJoinStep.h @@ -10,7 +10,7 @@ using ArrayJoinActionPtr = std::shared_ptr; class ArrayJoinStep : public ITransformingStep { public: - explicit ArrayJoinStep(const DataStream & input_stream_, ArrayJoinActionPtr array_join_); + ArrayJoinStep(const DataStream & input_stream_, NameSet columns_, bool is_left_, bool is_unaligned_, size_t max_block_size_); String getName() const override { return "ArrayJoin"; } void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; @@ -18,12 +18,16 @@ public: void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(FormatSettings & settings) const override; - const ArrayJoinActionPtr & arrayJoin() const { return array_join; } + const NameSet & getColumns() const { return columns; } + bool isLeft() const { return is_left; } private: void updateOutputStream() override; - ArrayJoinActionPtr array_join; + NameSet columns; + bool is_left = false; + bool is_unaligned = false; + size_t max_block_size = DEFAULT_BLOCK_SIZE; }; } diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index b71326ff75b..e4a292394f3 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -520,8 +520,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (auto * array_join = typeid_cast(child.get())) { - const auto & array_join_actions = array_join->arrayJoin(); - const auto & keys = array_join_actions->columns; + const auto & keys = array_join->getColumns(); const auto & array_join_header = array_join->getInputStreams().front().header; Names allowed_inputs; diff --git a/src/Processors/QueryPlan/Optimizations/liftUpArrayJoin.cpp b/src/Processors/QueryPlan/Optimizations/liftUpArrayJoin.cpp index 0d4f2330119..8866bb99cbe 100644 --- a/src/Processors/QueryPlan/Optimizations/liftUpArrayJoin.cpp +++ b/src/Processors/QueryPlan/Optimizations/liftUpArrayJoin.cpp @@ -24,11 +24,11 @@ size_t tryLiftUpArrayJoin(QueryPlan::Node * parent_node, QueryPlan::Nodes & node if (!(expression_step || filter_step) || !array_join_step) return 0; - const auto & array_join = array_join_step->arrayJoin(); + const auto & array_join_columns = array_join_step->getColumns(); const auto & expression = expression_step ? expression_step->getExpression() : filter_step->getExpression(); - auto split_actions = expression.splitActionsBeforeArrayJoin(array_join->columns); + auto split_actions = expression.splitActionsBeforeArrayJoin(array_join_columns); /// No actions can be moved before ARRAY JOIN. if (split_actions.first.trivial()) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index 9dc4992cddd..450900539b5 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -230,10 +230,10 @@ void buildSortingDAG(QueryPlan::Node & node, std::optional & dag, Fi { /// Should ignore limit because ARRAY JOIN can reduce the number of rows in case of empty array. /// But in case of LEFT ARRAY JOIN the result number of rows is always bigger. - if (!array_join->arrayJoin()->is_left) + if (!array_join->isLeft()) limit = 0; - const auto & array_joined_columns = array_join->arrayJoin()->columns; + const auto & array_joined_columns = array_join->getColumns(); if (dag) { diff --git a/src/Processors/Transforms/ArrayJoinTransform.cpp b/src/Processors/Transforms/ArrayJoinTransform.cpp index 1304434d74e..bd436cbe408 100644 --- a/src/Processors/Transforms/ArrayJoinTransform.cpp +++ b/src/Processors/Transforms/ArrayJoinTransform.cpp @@ -10,10 +10,10 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -Block ArrayJoinTransform::transformHeader(Block header, const ArrayJoinActionPtr & array_join) +Block ArrayJoinTransform::transformHeader(Block header, const NameSet & array_join_columns) { auto columns = header.getColumnsWithTypeAndName(); - array_join->prepare(columns); + ArrayJoinAction::prepare(array_join_columns, columns); Block res{std::move(columns)}; res.setColumns(res.mutateColumns()); return res; @@ -23,7 +23,7 @@ ArrayJoinTransform::ArrayJoinTransform( const Block & header_, ArrayJoinActionPtr array_join_, bool /*on_totals_*/) - : IInflatingTransform(header_, transformHeader(header_, array_join_)) + : IInflatingTransform(header_, transformHeader(header_, array_join_->columns)) , array_join(std::move(array_join_)) { /// TODO diff --git a/src/Processors/Transforms/ArrayJoinTransform.h b/src/Processors/Transforms/ArrayJoinTransform.h index 4219135982d..386b9d6616b 100644 --- a/src/Processors/Transforms/ArrayJoinTransform.h +++ b/src/Processors/Transforms/ArrayJoinTransform.h @@ -22,7 +22,7 @@ public: String getName() const override { return "ArrayJoinTransform"; } - static Block transformHeader(Block header, const ArrayJoinActionPtr & array_join); + static Block transformHeader(Block header, const NameSet & array_join_columns); protected: void consume(Chunk chunk) override; From 03ac70f98848b60935b54b859cc5a817f8a6b39d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 5 Sep 2024 14:41:06 +0000 Subject: [PATCH 025/103] Fising build. --- src/Functions/array/arrayResize.h | 28 ++++++++++ src/Functions/array/emptyArrayToSingle.h | 28 ++++++++++ src/Functions/array/length.h | 66 ++++++++++++++++++++++++ 3 files changed, 122 insertions(+) create mode 100644 src/Functions/array/arrayResize.h create mode 100644 src/Functions/array/emptyArrayToSingle.h create mode 100644 src/Functions/array/length.h diff --git a/src/Functions/array/arrayResize.h b/src/Functions/array/arrayResize.h new file mode 100644 index 00000000000..b6a8ccbbc3c --- /dev/null +++ b/src/Functions/array/arrayResize.h @@ -0,0 +1,28 @@ +#pragma once +#include +#include + +namespace DB +{ + +class FunctionArrayResize : public IFunction +{ +public: + static constexpr auto name = "arrayResize"; + static FunctionPtr createImpl() { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return createImpl(); } + + String getName() const override { return name; } + + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type, size_t input_rows_count) const override; + bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForNulls() const override { return false; } +}; + +} diff --git a/src/Functions/array/emptyArrayToSingle.h b/src/Functions/array/emptyArrayToSingle.h new file mode 100644 index 00000000000..693c776184c --- /dev/null +++ b/src/Functions/array/emptyArrayToSingle.h @@ -0,0 +1,28 @@ +#include +#include + +namespace DB +{ + +/** emptyArrayToSingle(arr) - replace empty arrays with arrays of one element with a default value. + */ +class FunctionEmptyArrayToSingle : public IFunction +{ +public: + static constexpr auto name = "emptyArrayToSingle"; + static FunctionPtr createImpl() { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return createImpl(); } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 1; } + bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override; +}; + +} diff --git a/src/Functions/array/length.h b/src/Functions/array/length.h new file mode 100644 index 00000000000..2ecab76e0f8 --- /dev/null +++ b/src/Functions/array/length.h @@ -0,0 +1,66 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +/** Calculates the length of a string in bytes. + */ +struct LengthImpl +{ + static constexpr auto is_fixed_to_constant = true; + + static void vector(const ColumnString::Chars & /*data*/, const ColumnString::Offsets & offsets, PaddedPODArray & res, size_t input_rows_count) + { + for (size_t i = 0; i < input_rows_count; ++i) + res[i] = offsets[i] - 1 - offsets[i - 1]; + } + + static void vectorFixedToConstant(const ColumnString::Chars & /*data*/, size_t n, UInt64 & res, size_t) + { + res = n; + } + + static void vectorFixedToVector(const ColumnString::Chars & /*data*/, size_t /*n*/, PaddedPODArray & /*res*/, size_t) + { + } + + static void array(const ColumnString::Offsets & offsets, PaddedPODArray & res, size_t input_rows_count) + { + for (size_t i = 0; i < input_rows_count; ++i) + res[i] = offsets[i] - offsets[i - 1]; + } + + [[noreturn]] static void uuid(const ColumnUUID::Container &, size_t &, PaddedPODArray &, size_t) + { + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Cannot apply function length to UUID argument"); + } + + [[noreturn]] static void ipv6(const ColumnIPv6::Container &, size_t &, PaddedPODArray &, size_t) + { + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Cannot apply function length to IPv6 argument"); + } + + [[noreturn]] static void ipv4(const ColumnIPv4::Container &, size_t &, PaddedPODArray &, size_t) + { + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Cannot apply function length to IPv4 argument"); + } +}; + + +struct NameLength +{ + static constexpr auto name = "length"; +}; + +using FunctionLength = FunctionStringOrArrayToT; + +} From fb8999a8857c8ab799aac1e1123b69f275823749 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 5 Sep 2024 14:44:31 +0000 Subject: [PATCH 026/103] Remove commented code. --- src/Interpreters/ArrayJoinAction.cpp | 2 -- src/Interpreters/ArrayJoinAction.h | 1 - 2 files changed, 3 deletions(-) diff --git a/src/Interpreters/ArrayJoinAction.cpp b/src/Interpreters/ArrayJoinAction.cpp index 802d38b0c03..5124c01f8f3 100644 --- a/src/Interpreters/ArrayJoinAction.cpp +++ b/src/Interpreters/ArrayJoinAction.cpp @@ -67,8 +67,6 @@ ArrayJoinAction::ArrayJoinAction(const NameSet & array_joined_columns_, bool is_ , is_left(is_left_) , is_unaligned(is_unaligned_) , max_block_size(max_block_size_) - // , is_unaligned(context->getSettingsRef().enable_unaligned_array_join) - // , max_block_size(context->getSettingsRef().max_block_size) { if (columns.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "No arrays to join"); diff --git a/src/Interpreters/ArrayJoinAction.h b/src/Interpreters/ArrayJoinAction.h index 287eabaac65..b296467c1e0 100644 --- a/src/Interpreters/ArrayJoinAction.h +++ b/src/Interpreters/ArrayJoinAction.h @@ -33,7 +33,6 @@ public: /// For unaligned [LEFT] ARRAY JOIN FunctionOverloadResolverPtr function_length; - //FunctionOverloadResolverPtr function_greatest; FunctionOverloadResolverPtr function_array_resize; /// For LEFT ARRAY JOIN. From d23145fd19c02721a605127725d8123a7a1cda7d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 5 Sep 2024 16:59:14 +0200 Subject: [PATCH 027/103] Update emptyArrayToSingle.h --- src/Functions/array/emptyArrayToSingle.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Functions/array/emptyArrayToSingle.h b/src/Functions/array/emptyArrayToSingle.h index 693c776184c..f8b67101471 100644 --- a/src/Functions/array/emptyArrayToSingle.h +++ b/src/Functions/array/emptyArrayToSingle.h @@ -1,3 +1,4 @@ +#pragma once #include #include From 92507d993844862bc7371a1420487ff46db08be0 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 5 Sep 2024 17:10:12 +0200 Subject: [PATCH 028/103] Update nats.md --- docs/en/engines/table-engines/integrations/nats.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/nats.md b/docs/en/engines/table-engines/integrations/nats.md index 2ea2fdb01ae..806437571f7 100644 --- a/docs/en/engines/table-engines/integrations/nats.md +++ b/docs/en/engines/table-engines/integrations/nats.md @@ -112,7 +112,7 @@ Example: ``` The NATS server configuration can be added using the ClickHouse config file. - More specifically you can add Redis password for NATS engine: +More specifically you can add Redis password for NATS engine: ``` xml From fdbf8e71ab053debb32500f5e26740998c960799 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 6 Sep 2024 10:29:47 +0000 Subject: [PATCH 029/103] Stable explain --- src/Interpreters/ActionsDAG.cpp | 5 +++-- src/Interpreters/ActionsDAG.h | 2 +- src/Interpreters/ArrayJoin.h | 13 ++++++++++++ src/Interpreters/ArrayJoinAction.cpp | 10 +++++++-- src/Interpreters/ArrayJoinAction.h | 3 ++- src/Interpreters/ExpressionActions.cpp | 10 ++++----- src/Interpreters/ExpressionActions.h | 8 +++---- src/Interpreters/ExpressionAnalyzer.cpp | 19 +++++++++-------- src/Interpreters/ExpressionAnalyzer.h | 6 +++--- src/Interpreters/InterpreterSelectQuery.cpp | 7 +++---- src/Planner/PlannerJoinTree.cpp | 8 +++---- src/Processors/QueryPlan/ArrayJoinStep.cpp | 21 +++++++++---------- src/Processors/QueryPlan/ArrayJoinStep.h | 10 ++++----- .../Optimizations/filterPushDown.cpp | 4 +++- .../Optimizations/optimizeReadInOrder.cpp | 4 +++- .../Transforms/ArrayJoinTransform.cpp | 10 +++++++-- .../Transforms/ArrayJoinTransform.h | 2 +- 17 files changed, 85 insertions(+), 57 deletions(-) create mode 100644 src/Interpreters/ArrayJoin.h diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 2a594839c6a..45263c8620a 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -2028,8 +2028,9 @@ ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split return {std::move(first_actions), std::move(second_actions), std::move(split_nodes_mapping)}; } -ActionsDAG::SplitResult ActionsDAG::splitActionsBeforeArrayJoin(const NameSet & array_joined_columns) const +ActionsDAG::SplitResult ActionsDAG::splitActionsBeforeArrayJoin(const Names & array_joined_columns) const { + std::unordered_set array_joined_columns_set(array_joined_columns.begin(), array_joined_columns.end()); struct Frame { const Node * node = nullptr; @@ -2072,7 +2073,7 @@ ActionsDAG::SplitResult ActionsDAG::splitActionsBeforeArrayJoin(const NameSet & if (cur.next_child_to_visit == cur.node->children.size()) { bool depend_on_array_join = false; - if (cur.node->type == ActionType::INPUT && array_joined_columns.contains(cur.node->result_name)) + if (cur.node->type == ActionType::INPUT && array_joined_columns_set.contains(cur.node->result_name)) depend_on_array_join = true; for (const auto * child : cur.node->children) diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index ee2b3fbf4f2..9b1842c83f8 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -340,7 +340,7 @@ public: SplitResult split(std::unordered_set split_nodes, bool create_split_nodes_mapping = false, bool avoid_duplicate_inputs = false) const; /// Splits actions into two parts. Returned first half may be swapped with ARRAY JOIN. - SplitResult splitActionsBeforeArrayJoin(const NameSet & array_joined_columns) const; + SplitResult splitActionsBeforeArrayJoin(const Names & array_joined_columns) const; /// Splits actions into two parts. First part has minimal size sufficient for calculation of column_name. /// Outputs of initial actions must contain column_name. diff --git a/src/Interpreters/ArrayJoin.h b/src/Interpreters/ArrayJoin.h new file mode 100644 index 00000000000..6a2bbef79db --- /dev/null +++ b/src/Interpreters/ArrayJoin.h @@ -0,0 +1,13 @@ +#pragma once +#include + +namespace DB +{ + +struct ArrayJoin +{ + Names columns; + bool is_left = false; +}; + +} diff --git a/src/Interpreters/ArrayJoinAction.cpp b/src/Interpreters/ArrayJoinAction.cpp index 5124c01f8f3..12ed64cb0eb 100644 --- a/src/Interpreters/ArrayJoinAction.cpp +++ b/src/Interpreters/ArrayJoinAction.cpp @@ -62,8 +62,8 @@ ColumnWithTypeAndName convertArrayJoinColumn(const ColumnWithTypeAndName & src_c return array_col; } -ArrayJoinAction::ArrayJoinAction(const NameSet & array_joined_columns_, bool is_left_, bool is_unaligned_, size_t max_block_size_) - : columns(array_joined_columns_) +ArrayJoinAction::ArrayJoinAction(const Names & columns_, bool is_left_, bool is_unaligned_, size_t max_block_size_) + : columns(columns_.begin(), columns_.end()) , is_left(is_left_) , is_unaligned(is_unaligned_) , max_block_size(max_block_size_) @@ -80,6 +80,12 @@ ArrayJoinAction::ArrayJoinAction(const NameSet & array_joined_columns_, bool is_ function_builder = std::make_unique(FunctionEmptyArrayToSingle::createImpl()); } +void ArrayJoinAction::prepare(const Names & columns, ColumnsWithTypeAndName & sample) +{ + NameSet columns_set(columns.begin(), columns.end()); + return prepare(columns_set, sample); +} + void ArrayJoinAction::prepare(const NameSet & columns, ColumnsWithTypeAndName & sample) { for (auto & current : sample) diff --git a/src/Interpreters/ArrayJoinAction.h b/src/Interpreters/ArrayJoinAction.h index b296467c1e0..b76822e6b71 100644 --- a/src/Interpreters/ArrayJoinAction.h +++ b/src/Interpreters/ArrayJoinAction.h @@ -38,8 +38,9 @@ public: /// For LEFT ARRAY JOIN. FunctionOverloadResolverPtr function_builder; - ArrayJoinAction(const NameSet & array_joined_columns_, bool is_left_, bool is_unaligned_, size_t max_block_size_); + ArrayJoinAction(const Names & columns_, bool is_left_, bool is_unaligned_, size_t max_block_size_); static void prepare(const NameSet & columns, ColumnsWithTypeAndName & sample); + static void prepare(const Names & columns, ColumnsWithTypeAndName & sample); ArrayJoinResultIteratorPtr execute(Block block); }; diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 4c313b3c9a8..edf419d404e 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -1059,16 +1059,16 @@ std::string ExpressionActionsChain::dumpChain() const return ss.str(); } -ExpressionActionsChain::ArrayJoinStep::ArrayJoinStep(ArrayJoinActionPtr array_join_, ColumnsWithTypeAndName required_columns_) +ExpressionActionsChain::ArrayJoinStep::ArrayJoinStep(const Names & array_join_columns_, ColumnsWithTypeAndName required_columns_) : Step({}) - , array_join(std::move(array_join_)) + , array_join_columns(array_join_columns_.begin(), array_join_columns_.end()) , result_columns(std::move(required_columns_)) { for (auto & column : result_columns) { required_columns.emplace_back(NameAndTypePair(column.name, column.type)); - if (array_join->columns.contains(column.name)) + if (array_join_columns.contains(column.name)) { const auto & array = getArrayJoinDataType(column.type); column.type = array->getNestedType(); @@ -1085,12 +1085,12 @@ void ExpressionActionsChain::ArrayJoinStep::finalize(const NameSet & required_ou for (const auto & column : result_columns) { - if (array_join->columns.contains(column.name) || required_output_.contains(column.name)) + if (array_join_columns.contains(column.name) || required_output_.contains(column.name)) new_result_columns.emplace_back(column); } for (const auto & column : required_columns) { - if (array_join->columns.contains(column.name) || required_output_.contains(column.name)) + if (array_join_columns.contains(column.name) || required_output_.contains(column.name)) new_required_columns.emplace_back(column); } diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index 7652fe49eab..539c7c8d141 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -22,9 +23,6 @@ class TableJoin; class IJoin; using JoinPtr = std::shared_ptr; -class ArrayJoinAction; -using ArrayJoinActionPtr = std::shared_ptr; - class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; @@ -223,11 +221,11 @@ struct ExpressionActionsChain : WithContext struct ArrayJoinStep : public Step { - ArrayJoinActionPtr array_join; + const NameSet array_join_columns; NamesAndTypesList required_columns; ColumnsWithTypeAndName result_columns; - ArrayJoinStep(ArrayJoinActionPtr array_join_, ColumnsWithTypeAndName required_columns_); + ArrayJoinStep(const Names & array_join_columns_, ColumnsWithTypeAndName required_columns_); NamesAndTypesList getRequiredColumns() const override { return required_columns; } ColumnsWithTypeAndName getResultColumns() const override { return result_columns; } diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 3315f4a67b2..1235ebb0871 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -215,7 +215,7 @@ NamesAndTypesList ExpressionAnalyzer::getColumnsAfterArrayJoin(ActionsDAG & acti auto array_join = addMultipleArrayJoinAction(actions, is_array_join_left); auto sample_columns = actions.getResultColumns(); - ArrayJoinAction::prepare(array_join->columns, sample_columns); + ArrayJoinAction::prepare(array_join.columns, sample_columns); actions = ActionsDAG(sample_columns); NamesAndTypesList new_columns_after_array_join; @@ -889,9 +889,11 @@ const ASTSelectQuery * SelectQueryExpressionAnalyzer::getAggregatingQuery() cons } /// "Big" ARRAY JOIN. -ArrayJoinActionPtr ExpressionAnalyzer::addMultipleArrayJoinAction(ActionsDAG & actions, bool array_join_is_left) const +ArrayJoin ExpressionAnalyzer::addMultipleArrayJoinAction(ActionsDAG & actions, bool array_join_is_left) const { - NameSet result_columns; + Names result_columns; + result_columns.reserve(syntax->array_join_result_to_source.size()); + for (const auto & result_source : syntax->array_join_result_to_source) { /// Assign new names to columns, if needed. @@ -902,20 +904,19 @@ ArrayJoinActionPtr ExpressionAnalyzer::addMultipleArrayJoinAction(ActionsDAG & a } /// Make ARRAY JOIN (replace arrays with their insides) for the columns in these new names. - result_columns.insert(result_source.first); + result_columns.push_back(result_source.first); } - const auto & query_settings = getContext()->getSettingsRef(); - return std::make_shared(result_columns, array_join_is_left, query_settings.enable_unaligned_array_join, query_settings.max_block_size); + return {std::move(result_columns), array_join_is_left}; } -ArrayJoinActionPtr SelectQueryExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, ActionsAndProjectInputsFlagPtr & before_array_join, bool only_types) +std::optional SelectQueryExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, ActionsAndProjectInputsFlagPtr & before_array_join, bool only_types) { const auto * select_query = getSelectQuery(); auto [array_join_expression_list, is_array_join_left] = select_query->arrayJoinExpressionList(); if (!array_join_expression_list) - return nullptr; + return {}; ExpressionActionsChain::Step & step = chain.lastStep(sourceColumns()); @@ -924,7 +925,7 @@ ArrayJoinActionPtr SelectQueryExpressionAnalyzer::appendArrayJoin(ExpressionActi auto array_join = addMultipleArrayJoinAction(step.actions()->dag, is_array_join_left); before_array_join = chain.getLastActions(); - chain.steps.push_back(std::make_unique(array_join, step.getResultColumns())); + chain.steps.push_back(std::make_unique(array_join.columns, step.getResultColumns())); chain.addStep(); diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index dc038e10594..483be5cc4c4 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -174,7 +174,7 @@ protected: /// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables. void initGlobalSubqueriesAndExternalTables(bool do_global, bool is_explain); - ArrayJoinActionPtr addMultipleArrayJoinAction(ActionsDAG & actions, bool is_left) const; + ArrayJoin addMultipleArrayJoinAction(ActionsDAG & actions, bool is_left) const; void getRootActions(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAG & actions, bool only_consts = false); @@ -234,7 +234,7 @@ struct ExpressionAnalysisResult bool use_grouping_set_key = false; ActionsAndProjectInputsFlagPtr before_array_join; - ArrayJoinActionPtr array_join; + std::optional array_join; ActionsAndProjectInputsFlagPtr before_join; ActionsAndProjectInputsFlagPtr converting_join_columns; JoinPtr join; @@ -388,7 +388,7 @@ private: */ /// Before aggregation: - ArrayJoinActionPtr appendArrayJoin(ExpressionActionsChain & chain, ActionsAndProjectInputsFlagPtr & before_array_join, bool only_types); + std::optional appendArrayJoin(ExpressionActionsChain & chain, ActionsAndProjectInputsFlagPtr & before_array_join, bool only_types); bool appendJoinLeftKeys(ExpressionActionsChain & chain, bool only_types); JoinPtr appendJoin(ExpressionActionsChain & chain, ActionsAndProjectInputsFlagPtr & converting_join_columns); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 1bc48b9026a..b7359b85079 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1679,10 +1679,9 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( query_plan.getCurrentDataStream(), - expressions.array_join->columns, - expressions.array_join->is_left, - expressions.array_join->is_unaligned, - expressions.array_join->max_block_size); + *expressions.array_join, + settings.enable_unaligned_array_join, + settings.max_block_size); array_join_step->setStepDescription("ARRAY JOIN"); query_plan.addStep(std::move(array_join_step)); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 1b2a6625828..0280ebc7a90 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -1654,11 +1654,12 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_ PlannerActionsVisitor actions_visitor(planner_context); std::unordered_set array_join_expressions_output_nodes; - NameSet array_join_column_names; + Names array_join_column_names; + array_join_column_names.reserve(array_join_node.getJoinExpressions().getNodes().size()); for (auto & array_join_expression : array_join_node.getJoinExpressions().getNodes()) { const auto & array_join_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(array_join_expression); - array_join_column_names.insert(array_join_column_identifier); + array_join_column_names.push_back(array_join_column_identifier); auto & array_join_expression_column = array_join_expression->as(); auto expression_dag_index_nodes = actions_visitor.visit(array_join_action_dag, array_join_expression_column.getExpressionOrThrow()); @@ -1710,8 +1711,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_ const auto & settings = planner_context->getQueryContext()->getSettingsRef(); auto array_join_step = std::make_unique( plan.getCurrentDataStream(), - std::move(array_join_column_names), - array_join_node.isLeft(), + ArrayJoin{std::move(array_join_column_names), array_join_node.isLeft()}, settings.enable_unaligned_array_join, settings.max_block_size); diff --git a/src/Processors/QueryPlan/ArrayJoinStep.cpp b/src/Processors/QueryPlan/ArrayJoinStep.cpp index aa721e138cf..94cb6ae2ee5 100644 --- a/src/Processors/QueryPlan/ArrayJoinStep.cpp +++ b/src/Processors/QueryPlan/ArrayJoinStep.cpp @@ -24,13 +24,12 @@ static ITransformingStep::Traits getTraits() }; } -ArrayJoinStep::ArrayJoinStep(const DataStream & input_stream_, NameSet columns_, bool is_left_, bool is_unaligned_, size_t max_block_size_) +ArrayJoinStep::ArrayJoinStep(const DataStream & input_stream_, ArrayJoin array_join_, bool is_unaligned_, size_t max_block_size_) : ITransformingStep( input_stream_, - ArrayJoinTransform::transformHeader(input_stream_.header, columns_), + ArrayJoinTransform::transformHeader(input_stream_.header, array_join_.columns), getTraits()) - , columns(std::move(columns_)) - , is_left(is_left_) + , array_join(std::move(array_join_)) , is_unaligned(is_unaligned_) , max_block_size(max_block_size_) { @@ -39,16 +38,16 @@ ArrayJoinStep::ArrayJoinStep(const DataStream & input_stream_, NameSet columns_, void ArrayJoinStep::updateOutputStream() { output_stream = createOutputStream( - input_streams.front(), ArrayJoinTransform::transformHeader(input_streams.front().header, columns), getDataStreamTraits()); + input_streams.front(), ArrayJoinTransform::transformHeader(input_streams.front().header, array_join.columns), getDataStreamTraits()); } void ArrayJoinStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - auto array_join = std::make_shared(columns, is_left, is_unaligned, max_block_size); + auto array_join_actions = std::make_shared(array_join.columns, array_join.is_left, is_unaligned, max_block_size); pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) { bool on_totals = stream_type == QueryPipelineBuilder::StreamType::Totals; - return std::make_shared(header, array_join, on_totals); + return std::make_shared(header, array_join_actions, on_totals); }); } @@ -57,8 +56,8 @@ void ArrayJoinStep::describeActions(FormatSettings & settings) const String prefix(settings.offset, ' '); bool first = true; - settings.out << prefix << (is_left ? "LEFT " : "") << "ARRAY JOIN "; - for (const auto & column : columns) + settings.out << prefix << (array_join.is_left ? "LEFT " : "") << "ARRAY JOIN "; + for (const auto & column : array_join.columns) { if (!first) settings.out << ", "; @@ -72,10 +71,10 @@ void ArrayJoinStep::describeActions(FormatSettings & settings) const void ArrayJoinStep::describeActions(JSONBuilder::JSONMap & map) const { - map.add("Left", is_left); + map.add("Left", array_join.is_left); auto columns_array = std::make_unique(); - for (const auto & column : columns) + for (const auto & column : array_join.columns) columns_array->add(column); map.add("Columns", std::move(columns_array)); diff --git a/src/Processors/QueryPlan/ArrayJoinStep.h b/src/Processors/QueryPlan/ArrayJoinStep.h index 3f2eacc3159..1a049d5805e 100644 --- a/src/Processors/QueryPlan/ArrayJoinStep.h +++ b/src/Processors/QueryPlan/ArrayJoinStep.h @@ -1,5 +1,6 @@ #pragma once #include +#include namespace DB { @@ -10,7 +11,7 @@ using ArrayJoinActionPtr = std::shared_ptr; class ArrayJoinStep : public ITransformingStep { public: - ArrayJoinStep(const DataStream & input_stream_, NameSet columns_, bool is_left_, bool is_unaligned_, size_t max_block_size_); + ArrayJoinStep(const DataStream & input_stream_, ArrayJoin array_join_, bool is_unaligned_, size_t max_block_size_); String getName() const override { return "ArrayJoin"; } void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; @@ -18,14 +19,13 @@ public: void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(FormatSettings & settings) const override; - const NameSet & getColumns() const { return columns; } - bool isLeft() const { return is_left; } + const Names & getColumns() const { return array_join.columns; } + bool isLeft() const { return array_join.is_left; } private: void updateOutputStream() override; - NameSet columns; - bool is_left = false; + ArrayJoin array_join; bool is_unaligned = false; size_t max_block_size = DEFAULT_BLOCK_SIZE; }; diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index e4a292394f3..63ea8880cca 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -521,11 +521,13 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (auto * array_join = typeid_cast(child.get())) { const auto & keys = array_join->getColumns(); + std::unordered_set keys_set(keys.begin(), keys.end()); + const auto & array_join_header = array_join->getInputStreams().front().header; Names allowed_inputs; for (const auto & column : array_join_header) - if (!keys.contains(column.name)) + if (!keys_set.contains(column.name)) allowed_inputs.push_back(column.name); if (auto updated_steps = tryAddNewFilterStep(parent_node, nodes, allowed_inputs)) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index 450900539b5..f6e1794dc3f 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -237,6 +237,8 @@ void buildSortingDAG(QueryPlan::Node & node, std::optional & dag, Fi if (dag) { + std::unordered_set keys_set(array_joined_columns.begin(), array_joined_columns.end()); + /// Remove array joined columns from outputs. /// Types are changed after ARRAY JOIN, and we can't use this columns anyway. ActionsDAG::NodeRawConstPtrs outputs; @@ -244,7 +246,7 @@ void buildSortingDAG(QueryPlan::Node & node, std::optional & dag, Fi for (const auto & output : dag->getOutputs()) { - if (!array_joined_columns.contains(output->result_name)) + if (!keys_set.contains(output->result_name)) outputs.push_back(output); } diff --git a/src/Processors/Transforms/ArrayJoinTransform.cpp b/src/Processors/Transforms/ArrayJoinTransform.cpp index bd436cbe408..ec1d77e1a1b 100644 --- a/src/Processors/Transforms/ArrayJoinTransform.cpp +++ b/src/Processors/Transforms/ArrayJoinTransform.cpp @@ -10,7 +10,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -Block ArrayJoinTransform::transformHeader(Block header, const NameSet & array_join_columns) +template +Block transformHeaderImpl(Block header, const Container & array_join_columns) { auto columns = header.getColumnsWithTypeAndName(); ArrayJoinAction::prepare(array_join_columns, columns); @@ -19,11 +20,16 @@ Block ArrayJoinTransform::transformHeader(Block header, const NameSet & array_jo return res; } +Block ArrayJoinTransform::transformHeader(Block header, const Names & array_join_columns) +{ + return transformHeaderImpl(std::move(header), array_join_columns); +} + ArrayJoinTransform::ArrayJoinTransform( const Block & header_, ArrayJoinActionPtr array_join_, bool /*on_totals_*/) - : IInflatingTransform(header_, transformHeader(header_, array_join_->columns)) + : IInflatingTransform(header_, transformHeaderImpl(header_, array_join_->columns)) , array_join(std::move(array_join_)) { /// TODO diff --git a/src/Processors/Transforms/ArrayJoinTransform.h b/src/Processors/Transforms/ArrayJoinTransform.h index 386b9d6616b..9ade337e676 100644 --- a/src/Processors/Transforms/ArrayJoinTransform.h +++ b/src/Processors/Transforms/ArrayJoinTransform.h @@ -22,7 +22,7 @@ public: String getName() const override { return "ArrayJoinTransform"; } - static Block transformHeader(Block header, const NameSet & array_join_columns); + static Block transformHeader(Block header, const Names & array_join_columns); protected: void consume(Chunk chunk) override; From ee304c7fc3d6361bebc8da0736dbb2fe6c491d17 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 6 Sep 2024 16:02:47 +0000 Subject: [PATCH 030/103] Fix tidy --- src/Interpreters/ArrayJoinAction.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ArrayJoinAction.cpp b/src/Interpreters/ArrayJoinAction.cpp index 12ed64cb0eb..f49b3af9c58 100644 --- a/src/Interpreters/ArrayJoinAction.cpp +++ b/src/Interpreters/ArrayJoinAction.cpp @@ -83,7 +83,7 @@ ArrayJoinAction::ArrayJoinAction(const Names & columns_, bool is_left_, bool is_ void ArrayJoinAction::prepare(const Names & columns, ColumnsWithTypeAndName & sample) { NameSet columns_set(columns.begin(), columns.end()); - return prepare(columns_set, sample); + prepare(columns_set, sample); } void ArrayJoinAction::prepare(const NameSet & columns, ColumnsWithTypeAndName & sample) From 8cdc10cf656791ab88093cddb0cb07bb4209492a Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 9 Sep 2024 18:11:03 +0100 Subject: [PATCH 031/103] fix settings changes --- src/Core/SettingsChangesHistory.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 0a4f31522ba..bd701af39b0 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -76,7 +76,8 @@ static std::initializer_list Date: Tue, 10 Sep 2024 12:07:44 +0100 Subject: [PATCH 032/103] fix --- src/Core/SettingsChangesHistory.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index a5cbc8f1ec6..d19ca031c25 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -78,7 +78,6 @@ static std::initializer_list Date: Tue, 10 Sep 2024 17:22:19 +0200 Subject: [PATCH 033/103] Add test for SYSTEM UNFREEZE with zero_copy --- src/Disks/DiskEncrypted.h | 3 +- .../test_zero_copy_unfreeze/__init__.py | 0 .../configs/storage_conf.xml | 42 ++++++++++++++ .../test_zero_copy_unfreeze/test.py | 57 +++++++++++++++++++ 4 files changed, 101 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_zero_copy_unfreeze/__init__.py create mode 100644 tests/integration/test_zero_copy_unfreeze/configs/storage_conf.xml create mode 100644 tests/integration/test_zero_copy_unfreeze/test.py diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index 9818c284009..95bb5ac2a11 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -350,7 +350,8 @@ public: return delegate; } - UInt32 getRefCount(const String & path) const override { + UInt32 getRefCount(const String & path) const override + { auto wrapped_path = wrappedPath(path); return delegate->getRefCount(wrapped_path); } diff --git a/tests/integration/test_zero_copy_unfreeze/__init__.py b/tests/integration/test_zero_copy_unfreeze/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_zero_copy_unfreeze/configs/storage_conf.xml b/tests/integration/test_zero_copy_unfreeze/configs/storage_conf.xml new file mode 100644 index 00000000000..683ff7055a4 --- /dev/null +++ b/tests/integration/test_zero_copy_unfreeze/configs/storage_conf.xml @@ -0,0 +1,42 @@ + + + test + + + + + + s3 + http://minio1:9001/root/data/ + minio + minio123 + + + encrypted + s3 + 9bb21f41535d8e89098f3f23a62c661e + + + + + +

+ s3 +
+ + + + +
+ s3 +
+
+
+ + + + true + false + + true + diff --git a/tests/integration/test_zero_copy_unfreeze/test.py b/tests/integration/test_zero_copy_unfreeze/test.py new file mode 100644 index 00000000000..a64ad7c8b36 --- /dev/null +++ b/tests/integration/test_zero_copy_unfreeze/test.py @@ -0,0 +1,57 @@ +from collections.abc import Iterable +import pytest + +from helpers.cluster import ClickHouseCluster, ClickHouseInstance + +cluster = ClickHouseCluster(__file__) + + +@pytest.fixture(scope="module") +def started_cluster() -> Iterable[ClickHouseCluster]: + try: + cluster.add_instance( + "node1", + main_configs=["configs/storage_conf.xml"], + with_minio=True, + with_zookeeper=True, + ) + cluster.start() + + yield cluster + finally: + cluster.shutdown() + + +@pytest.mark.parametrize("storage_policy", ["s3", "encrypted"]) +def test_unfreeze(storage_policy: str, started_cluster: ClickHouseCluster) -> None: + node1: ClickHouseInstance = started_cluster.instances["node1"] + node1.query( + f"""\ +CREATE TABLE test1 (a Int) +ENGINE = ReplicatedMergeTree('/clickhouse-tables/test1/{storage_policy}', 'r1') +ORDER BY a +SETTINGS storage_policy = '{storage_policy}' +""" + ) + + node1.query( + """\ +INSERT INTO test1 +SELECT * +FROM system.numbers +LIMIT 20 +""" + ) + + node1.query("ALTER TABLE test1 FREEZE WITH NAME 'test'") + node1.query("SYSTEM UNFREEZE WITH NAME 'test'") + uuid = node1.query("SELECT uuid FROM system.tables WHERE name = 'test1'").strip() + # ensure that zero copy lock parent still exists + kazoo = started_cluster.get_kazoo_client("zoo1") + part_path = f"/clickhouse/zero_copy/zero_copy_s3/{uuid}/all_0_0_0/" + children: list[str] = kazoo.get_children(part_path) + assert len(children) == 1 + part_name = children[0] + assert len(kazoo.get_children(part_path + part_name)) == 1 + assert node1.query("SELECT count() FROM test1").strip() == "20" + node1.query("DROP TABLE test1") From 8d5d7dd83a68188ed3064b817c8d50a9e054e42e Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 10 Sep 2024 17:18:27 +0100 Subject: [PATCH 034/103] fix wording --- src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp index bee73a38128..452f11085e3 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp @@ -47,7 +47,7 @@ size_t chooseSegmentSize( /// Here we take max of two numbers: /// * (min_marks_per_task * threads) = the number of marks we request from the coordinator each time - there is no point to have segments smaller than one unit of work for a replica /// * (sum_marks / number_of_replicas^2) - we use consistent hashing for work distribution (including work stealing). If we have a really slow replica - /// everything up to (1/number_of_replicas) portion of its work will be stolen by other replicas. And it owns (1/number_of_replicas) share of total number of marks. + /// everything except (1/number_of_replicas) portion of its work will be stolen by other replicas. And it owns (1/number_of_replicas) share of total number of marks. /// Also important to note here that sum_marks is calculated after PK analysis, it means in particular that different segment sizes might be used for the /// same table for different queries (it is intentional). /// From f378047f30a22e67b8fca20ab2338009463da133 Mon Sep 17 00:00:00 2001 From: Joe Lynch Date: Wed, 11 Sep 2024 16:02:36 +0200 Subject: [PATCH 035/103] Properly clean up --- tests/integration/test_zero_copy_unfreeze/test.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_zero_copy_unfreeze/test.py b/tests/integration/test_zero_copy_unfreeze/test.py index a64ad7c8b36..9c1902c97ee 100644 --- a/tests/integration/test_zero_copy_unfreeze/test.py +++ b/tests/integration/test_zero_copy_unfreeze/test.py @@ -28,7 +28,7 @@ def test_unfreeze(storage_policy: str, started_cluster: ClickHouseCluster) -> No node1.query( f"""\ CREATE TABLE test1 (a Int) -ENGINE = ReplicatedMergeTree('/clickhouse-tables/test1/{storage_policy}', 'r1') +ENGINE = ReplicatedMergeTree('/clickhouse-tables/test1', 'r1') ORDER BY a SETTINGS storage_policy = '{storage_policy}' """ @@ -55,3 +55,4 @@ LIMIT 20 assert len(kazoo.get_children(part_path + part_name)) == 1 assert node1.query("SELECT count() FROM test1").strip() == "20" node1.query("DROP TABLE test1") + node1.query("SYSTEM DROP REPLICA 'r1' FROM ZKPATH '/clickhouse-tables/test1'") From fc83c1c7a2200ecc5b571d931415a7af16d1865b Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 11 Sep 2024 20:20:18 +0100 Subject: [PATCH 036/103] use final task size in segment size calculation --- .../MergeTreeReadPoolParallelReplicas.cpp | 20 +++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp index 452f11085e3..71d89f9950a 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp @@ -70,6 +70,17 @@ size_t chooseSegmentSize( UNREACHABLE(); } +size_t getMinMarksPerTask(size_t min_marks_per_task, const std::vector & per_part_infos) +{ + for (const auto & info : per_part_infos) + min_marks_per_task = std::max(min_marks_per_task, info->min_marks_per_task); + + if (min_marks_per_task == 0) + throw DB::Exception( + DB::ErrorCodes::BAD_ARGUMENTS, "Chosen number of marks to read is zero (likely because of weird interference of settings)"); + + return min_marks_per_task; +} } namespace ProfileEvents @@ -111,7 +122,7 @@ MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas( context_) , extension(std::move(extension_)) , coordination_mode(CoordinationMode::Default) - , min_marks_per_task(pool_settings.min_marks_for_concurrent_read) + , min_marks_per_task(getMinMarksPerTask(pool_settings.min_marks_for_concurrent_read, per_part_infos)) , mark_segment_size(chooseSegmentSize( log, context_->getSettingsRef().parallel_replicas_mark_segment_size, @@ -120,13 +131,6 @@ MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas( pool_settings.sum_marks, extension.total_nodes_count)) { - for (const auto & info : per_part_infos) - min_marks_per_task = std::max(min_marks_per_task, info->min_marks_per_task); - - if (min_marks_per_task == 0) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, "Chosen number of marks to read is zero (likely because of weird interference of settings)"); - extension.all_callback(InitialAllRangesAnnouncement( coordination_mode, parts_ranges.getDescriptions(), extension.number_of_current_replica, mark_segment_size)); } From 21bd47f09e1823b8812b71b34dd29b93ab871e63 Mon Sep 17 00:00:00 2001 From: marco-vb Date: Wed, 11 Sep 2024 17:17:15 +0000 Subject: [PATCH 037/103] Add settings min_free_disk_bytes_to_throw_insert and min_free_disk_ratio_to_throw_insert and update documentation. --- .../settings/merge-tree-settings.md | 20 +++++++++++++++++++ src/Core/Settings.h | 2 ++ src/Storages/MergeTree/MergeTreeSettings.h | 2 ++ 3 files changed, 24 insertions(+) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index a13aacc76e6..376c1c66ad5 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -156,6 +156,26 @@ Default value: 1000. ClickHouse artificially executes `INSERT` longer (adds ‘sleep’) so that the background merge process can merge parts faster than they are added. +## min_free_disk_bytes_to_throw_insert {#min_free_disk_bytes_to_throw_insert} + +The minimum number of bytes that should be free in disk space in order to insert data. If the number of available free bytes - `keep_free_space_bytes` is less than `min_free_disk_bytes_to_throw_insert` then an exception is thrown and the insert is not executed. Note that this setting does not take into account the amount of data that will be written by the `INSERT` operation. + +Possible values: + +- Any positive integer. + +Default value: 0 bytes. + +## min_free_disk_ratio_to_throw_insert {#min_free_disk_ratio_to_throw_insert} + +The minimum free to total disk space ratio to perform an `INSERT`. The free space is calculated by subtracting `keep_free_space_bytes` from the total available space in disk. + +Possible values: + +- Float, 0.0 - 1.0 + +Default value: 0.0 + ## inactive_parts_to_throw_insert {#inactive-parts-to-throw-insert} If the number of inactive parts in a single partition more than the `inactive_parts_to_throw_insert` value, `INSERT` is interrupted with the "Too many inactive parts (N). Parts cleaning are processing significantly slower than inserts" exception. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 493752fc3fe..86522e6c378 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -343,6 +343,8 @@ class IColumn; M(Int64, distributed_ddl_task_timeout, 180, "Timeout for DDL query responses from all hosts in cluster. If a ddl request has not been performed on all hosts, a response will contain a timeout error and a request will be executed in an async mode. Negative value means infinite. Zero means async mode.", 0) \ M(Milliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.", 0) \ M(Milliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.", 0) \ + M(UInt64, min_free_disk_bytes_to_throw_insert, 0, "Minimum free disk space bytes to throw an insert.", 0) \ + M(Double, min_free_disk_ratio_to_throw_insert, 0.0, "Minimum free disk space ratio to throw an insert.", 0) \ \ M(Bool, final, false, "Query with the FINAL modifier by default. If the engine does not support final, it does not have any effect. On queries with multiple tables final is applied only on those that support it. It also works on distributed tables", 0) \ \ diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 0769b60dc6b..02ba56f6e9a 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -99,6 +99,8 @@ struct Settings; M(Bool, add_implicit_sign_column_constraint_for_collapsing_engine, false, "If true, add implicit constraint for sign column for CollapsingMergeTree engine.", 0) \ M(Milliseconds, sleep_before_commit_local_part_in_replicated_table_ms, 0, "For testing. Do not change it.", 0) \ M(Bool, optimize_row_order, false, "Allow reshuffling of rows during part inserts and merges to improve the compressibility of the new part", 0) \ + M(UInt64, min_free_disk_bytes_to_throw_insert, 0, "Minimum free disk space bytes to throw an insert.", 0) \ + M(Double, min_free_disk_ratio_to_throw_insert, 0.0, "Minimum free disk space ratio to throw an insert.", 0) \ \ /* Part removal settings. */ \ M(UInt64, simultaneous_parts_removal_limit, 0, "Maximum number of parts to remove during one CleanupThread iteration (0 means unlimited).", 0) \ From 7d36f3b7646b56b5e7abc54df6e3a2c305023db0 Mon Sep 17 00:00:00 2001 From: marco-vb Date: Thu, 12 Sep 2024 09:53:07 +0000 Subject: [PATCH 038/103] Implemented checks for new settings. --- .../MergeTree/MergeTreeDataWriter.cpp | 20 ++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index f29d715e791..fa280e6080a 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include #include @@ -688,8 +689,25 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( MergeTreeDataPartType part_type; /// Size of part would not be greater than block.bytes() + epsilon size_t expected_size = block.bytes(); + + // If not temporary insert try to reserve respecting min free disk bytes + size_t reserve_extra = 0; + + if (!is_temp) + { + const auto context = CurrentThread::getQueryContext(); + const auto * settings = context ? &context->getSettingsRef() : nullptr; + + const UInt64 min_bytes = settings->min_free_disk_bytes_to_throw_insert; + const Float64 min_ratio = settings->min_free_disk_ratio_to_throw_insert; + + const auto total_disk_space = parent_part->getDataPartStorage().calculateTotalSizeOnDisk(); + const UInt64 min_bytes_from_ratio = static_cast(min_ratio * total_disk_space); + reserve_extra = std::min(min_bytes, min_bytes_from_ratio); + } + // just check if there is enough space on parent volume - MergeTreeData::reserveSpace(expected_size, parent_part->getDataPartStorage()); + MergeTreeData::reserveSpace(expected_size + reserve_extra, parent_part->getDataPartStorage()); part_type = data.choosePartFormatOnDisk(expected_size, block.rows()).part_type; auto new_data_part = parent_part->getProjectionPartBuilder(part_name, is_temp).withPartType(part_type).build(); From 99ede620bec26b9b3922abc3225794a1afb3dda7 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 12 Sep 2024 15:24:25 +0200 Subject: [PATCH 039/103] Add `kill_ci_runner` to ci_utils, will allow restarts --- tests/ci/ci_utils.py | 33 ++++++++++++++++++++++++++++++++- 1 file changed, 32 insertions(+), 1 deletion(-) diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index 8b60f61b006..86fa1c008c9 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -1,4 +1,5 @@ import json +import logging import os import re import subprocess @@ -6,10 +7,12 @@ import sys import time from contextlib import contextmanager from pathlib import Path -from typing import Any, Iterator, List, Union, Optional, Sequence +from typing import Any, Dict, Iterator, List, Optional, Sequence, Tuple, Union import requests +logger = logging.getLogger(__name__) + class Envs: GITHUB_REPOSITORY = os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse") @@ -36,6 +39,34 @@ def cd(path: Union[Path, str]) -> Iterator[None]: os.chdir(oldpwd) +def kill_ci_runner(message: str) -> None: + """The function to kill the current process with all parents when it's possible. + Works only when run with the set `CI` environment""" + if not os.getenv("CI", ""): # cycle import env_helper + logger.info("Running outside the CI, won't kill the runner") + return + print(f"::error::{message}") + + def get_ppid_name(pid: int) -> Tuple[int, str]: + # Avoid using psutil, it's not in stdlib + stats = Path(f"/proc/{pid}/stat").read_text(encoding="utf-8").split() + return int(stats[3]), stats[1] + + pid = os.getpid() + pids = {} # type: Dict[str, str] + while pid: + ppid, name = get_ppid_name(pid) + pids[str(pid)] = name + pid = ppid + logger.error( + "Sleeping 5 seconds and killing all possible processes from following:\n %s", + "\n ".join(f"{p}: {n}" for p, n in pids.items()), + ) + time.sleep(5) + # The current process will be killed too + subprocess.run(f"kill -9 {' '.join(pids.keys())}", check=False, shell=True) + + class GH: class ActionsNames: RunConfig = "RunConfig" From 8d5babf65fe78449e1bf3e3979f3fb7d09d18708 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 12 Sep 2024 15:26:21 +0200 Subject: [PATCH 040/103] Kill the runner process if integration tests fail to pre-pull --- tests/ci/integration_tests_runner.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index 6405492cd9c..21ca1649dc9 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -19,11 +19,12 @@ from collections import defaultdict from itertools import chain from typing import Any, Dict, Optional +from ci_utils import kill_ci_runner from env_helper import IS_CI from integration_test_images import IMAGES -from tee_popen import TeePopen from report import JOB_TIMEOUT_TEST_NAME from stopwatch import Stopwatch +from tee_popen import TeePopen MAX_RETRY = 1 NUM_WORKERS = 5 @@ -332,7 +333,9 @@ class ClickhouseIntegrationTestsRunner: except subprocess.CalledProcessError as err: logging.info("docker-compose pull failed: %s", str(err)) continue - logging.error("Pulling images failed for 5 attempts. Will fail the worker.") + message = "Pulling images failed for 5 attempts. Will fail the worker." + logging.error(message) + kill_ci_runner(message) # We pass specific retcode to to ci/integration_test_check.py to skip status reporting and restart job sys.exit(13) From 562c23eac6b73199529aad5d85a7b81aca853376 Mon Sep 17 00:00:00 2001 From: marco-vb Date: Thu, 12 Sep 2024 13:28:49 +0000 Subject: [PATCH 041/103] Add new settings to settings change history. --- src/Core/SettingsChangesHistory.cpp | 2 ++ src/Storages/MergeTree/MergeTreeDataWriter.cpp | 1 + 2 files changed, 3 insertions(+) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index fae8c25f5ed..ed36d69fba1 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -80,6 +80,8 @@ static std::initializer_listgetDataPartStorage()); part_type = data.choosePartFormatOnDisk(expected_size, block.rows()).part_type; From 4a9b376e2a9764491eb8b99b0a03d8b8afe97053 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 12 Sep 2024 14:01:26 +0000 Subject: [PATCH 042/103] Fix typo --- contrib/libpqxx-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libpqxx-cmake/CMakeLists.txt b/contrib/libpqxx-cmake/CMakeLists.txt index a3317404f95..fa2d01e161f 100644 --- a/contrib/libpqxx-cmake/CMakeLists.txt +++ b/contrib/libpqxx-cmake/CMakeLists.txt @@ -1,4 +1,4 @@ -option(ENABLE_LIBPQXX "Enalbe libpqxx" ${ENABLE_LIBRARIES}) +option(ENABLE_LIBPQXX "Enable libpqxx" ${ENABLE_LIBRARIES}) if (NOT ENABLE_LIBPQXX) message(STATUS "Not using libpqxx") From bde54b96f7e9118a1d73b736f34bd293341fad88 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 12 Sep 2024 14:03:22 +0000 Subject: [PATCH 043/103] Move ENABLE_LIBPQXX in a central place --- contrib/CMakeLists.txt | 9 +++++++-- contrib/libpq-cmake/CMakeLists.txt | 4 ---- contrib/libpqxx-cmake/CMakeLists.txt | 7 ------- 3 files changed, 7 insertions(+), 13 deletions(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index d7489bc5c0e..7f980cef76d 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -145,8 +145,13 @@ add_contrib (isa-l-cmake isa-l) add_contrib (libhdfs3-cmake libhdfs3) # requires: google-protobuf, krb5, isa-l add_contrib (hive-metastore-cmake hive-metastore) # requires: thrift, avro, arrow, libhdfs3 add_contrib (cppkafka-cmake cppkafka) -add_contrib (libpqxx-cmake libpqxx) -add_contrib (libpq-cmake libpq) + +option(ENABLE_LIBPQXX "Enable PostgreSQL" ${ENABLE_LIBRARIES}) +if (ENABLE_LIBPQXX) + add_contrib (libpqxx-cmake libpqxx) + add_contrib (libpq-cmake libpq) +endif() + add_contrib (rocksdb-cmake rocksdb) # requires: jemalloc, snappy, zlib, lz4, zstd, liburing add_contrib (nuraft-cmake NuRaft) add_contrib (fast_float-cmake fast_float) diff --git a/contrib/libpq-cmake/CMakeLists.txt b/contrib/libpq-cmake/CMakeLists.txt index 246e19593f6..fd5b6916ca2 100644 --- a/contrib/libpq-cmake/CMakeLists.txt +++ b/contrib/libpq-cmake/CMakeLists.txt @@ -1,7 +1,3 @@ -if (NOT ENABLE_LIBPQXX) - return() -endif() - set(LIBPQ_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/libpq") set(SRCS diff --git a/contrib/libpqxx-cmake/CMakeLists.txt b/contrib/libpqxx-cmake/CMakeLists.txt index fa2d01e161f..18f19ebf0f1 100644 --- a/contrib/libpqxx-cmake/CMakeLists.txt +++ b/contrib/libpqxx-cmake/CMakeLists.txt @@ -1,10 +1,3 @@ -option(ENABLE_LIBPQXX "Enable libpqxx" ${ENABLE_LIBRARIES}) - -if (NOT ENABLE_LIBPQXX) - message(STATUS "Not using libpqxx") - return() -endif() - set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/libpqxx") set (SRCS From 1e3bc6d359453ef356d3d2c19af92414dee68ef9 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 12 Sep 2024 15:15:57 +0100 Subject: [PATCH 044/103] log mark_segment_size on initiator --- src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 6d0ba879881..1955501b4ba 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -394,7 +394,7 @@ void DefaultCoordinator::initializeReadingState(InitialAllRangesAnnouncement ann if (mark_segment_size == 0) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Zero value provided for `mark_segment_size`"); - LOG_DEBUG(log, "Reading state is fully initialized: {}", fmt::join(all_parts_to_read, "; ")); + LOG_DEBUG(log, "Reading state is fully initialized: {}, mark_segment_size: {}", fmt::join(all_parts_to_read, "; "), mark_segment_size); } void DefaultCoordinator::markReplicaAsUnavailable(size_t replica_number) From 16f93ea1b34b7c4010e88e3a212a98f49e127c8f Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 12 Sep 2024 15:40:51 +0100 Subject: [PATCH 045/103] revive separate protocol versioning for PRs --- src/Client/Connection.cpp | 9 ++- src/Client/Connection.h | 1 + src/Core/ProtocolDefines.h | 4 +- src/Server/TCPHandler.cpp | 9 ++- src/Server/TCPHandler.h | 1 + src/Storages/MergeTree/RequestResponse.cpp | 83 +++++++++++----------- src/Storages/MergeTree/RequestResponse.h | 8 +-- 7 files changed, 63 insertions(+), 52 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 61022220b61..8a1c7d3988a 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -455,6 +455,9 @@ void Connection::sendAddendum() writeStringBinary(proto_recv_chunked, *out); } + if (server_revision >= DBMS_MIN_REVISION_WITH_VERSIONED_PARALLEL_REPLICAS_PROTOCOL) + writeVarUInt(DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION, *out); + out->next(); } @@ -525,6 +528,8 @@ void Connection::receiveHello(const Poco::Timespan & handshake_timeout) readVarUInt(server_version_major, *in); readVarUInt(server_version_minor, *in); readVarUInt(server_revision, *in); + if (server_revision >= DBMS_MIN_REVISION_WITH_VERSIONED_PARALLEL_REPLICAS_PROTOCOL) + readVarUInt(server_parallel_replicas_protocol_version, *in); if (server_revision >= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE) readStringBinary(server_timezone, *in); if (server_revision >= DBMS_MIN_REVISION_WITH_SERVER_DISPLAY_NAME) @@ -959,7 +964,7 @@ void Connection::sendReadTaskResponse(const String & response) void Connection::sendMergeTreeReadTaskResponse(const ParallelReadResponse & response) { writeVarUInt(Protocol::Client::MergeTreeReadTaskResponse, *out); - response.serialize(*out); + response.serialize(*out, server_parallel_replicas_protocol_version); out->finishChunk(); out->next(); } @@ -1413,7 +1418,7 @@ ParallelReadRequest Connection::receiveParallelReadRequest() const InitialAllRangesAnnouncement Connection::receiveInitialParallelReadAnnouncement() const { - return InitialAllRangesAnnouncement::deserialize(*in, server_revision); + return InitialAllRangesAnnouncement::deserialize(*in, server_parallel_replicas_protocol_version); } diff --git a/src/Client/Connection.h b/src/Client/Connection.h index ed84bc51318..e09d913f1ba 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -210,6 +210,7 @@ private: UInt64 server_version_minor = 0; UInt64 server_version_patch = 0; UInt64 server_revision = 0; + UInt64 server_parallel_replicas_protocol_version = 0; String server_timezone; String server_display_name; diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index f80ddc646bb..49c6fc1dde6 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -33,6 +33,8 @@ static constexpr auto DBMS_MIN_REVISION_WITH_AGGREGATE_FUNCTIONS_VERSIONING = 54 static constexpr auto DBMS_CLUSTER_PROCESSING_PROTOCOL_VERSION = 1; +static constexpr auto DBMS_MIN_SUPPORTED_PARALLEL_REPLICAS_PROTOCOL_VERSION = 3; +static constexpr auto DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION = 4; static constexpr auto DBMS_MIN_REVISION_WITH_PARALLEL_REPLICAS = 54453; static constexpr auto DBMS_MERGE_TREE_PART_INFO_VERSION = 1; @@ -85,7 +87,7 @@ static constexpr auto DBMS_MIN_REVISION_WITH_ROWS_BEFORE_AGGREGATION = 54469; /// Packets size header static constexpr auto DBMS_MIN_PROTOCOL_VERSION_WITH_CHUNKED_PACKETS = 54470; -static constexpr auto DBMS_MIN_REVISION_WITH_ADAPTIVE_MARK_SEGMENT_FOR_PARALLEL_REPLICAS = 54471; +static constexpr auto DBMS_MIN_REVISION_WITH_VERSIONED_PARALLEL_REPLICAS_PROTOCOL = 54471; /// Version of ClickHouse TCP protocol. /// diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 4df6e7fb7c3..13573ce6db0 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1270,7 +1270,7 @@ void TCPHandler::sendReadTaskRequestAssumeLocked() void TCPHandler::sendMergeTreeAllRangesAnnouncementAssumeLocked(InitialAllRangesAnnouncement announcement) { writeVarUInt(Protocol::Server::MergeTreeAllRangesAnnouncement, *out); - announcement.serialize(*out, client_tcp_protocol_version); + announcement.serialize(*out, client_parallel_replicas_protocol_version); out->finishChunk(); out->next(); @@ -1280,7 +1280,7 @@ void TCPHandler::sendMergeTreeAllRangesAnnouncementAssumeLocked(InitialAllRanges void TCPHandler::sendMergeTreeReadTaskRequestAssumeLocked(ParallelReadRequest request) { writeVarUInt(Protocol::Server::MergeTreeReadTaskRequest, *out); - request.serialize(*out); + request.serialize(*out, client_parallel_replicas_protocol_version); out->finishChunk(); out->next(); @@ -1652,6 +1652,9 @@ void TCPHandler::receiveAddendum() readStringBinary(proto_send_chunked_cl, *in); readStringBinary(proto_recv_chunked_cl, *in); } + + if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_VERSIONED_PARALLEL_REPLICAS_PROTOCOL) + readVarUInt(client_parallel_replicas_protocol_version, *in); } @@ -1679,6 +1682,8 @@ void TCPHandler::sendHello() writeVarUInt(VERSION_MAJOR, *out); writeVarUInt(VERSION_MINOR, *out); writeVarUInt(DBMS_TCP_PROTOCOL_VERSION, *out); + if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_VERSIONED_PARALLEL_REPLICAS_PROTOCOL) + writeVarUInt(DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION, *out); if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE) writeStringBinary(DateLUT::instance().getTimeZone(), *out); if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_DISPLAY_NAME) diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index dca40e98920..3b6e0059a30 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -188,6 +188,7 @@ private: UInt64 client_version_minor = 0; UInt64 client_version_patch = 0; UInt32 client_tcp_protocol_version = 0; + UInt32 client_parallel_replicas_protocol_version = 0; String proto_send_chunked_cl = "notchunked"; String proto_recv_chunked_cl = "notchunked"; String quota_key; diff --git a/src/Storages/MergeTree/RequestResponse.cpp b/src/Storages/MergeTree/RequestResponse.cpp index 48ff6ebccfd..04d7b23513a 100644 --- a/src/Storages/MergeTree/RequestResponse.cpp +++ b/src/Storages/MergeTree/RequestResponse.cpp @@ -14,18 +14,12 @@ namespace DB namespace ErrorCodes { - extern const int UNKNOWN_PROTOCOL; - extern const int UNKNOWN_ELEMENT_OF_ENUM; +extern const int UNKNOWN_PROTOCOL; +extern const int UNKNOWN_ELEMENT_OF_ENUM; } namespace { -/// Previously we had a separate protocol version number for parallel replicas. -/// But we didn't maintain backward compatibility and every protocol change was breaking. -/// Now we have to support at least minimal tail of the previous versions and the implementation -/// is based on the common tcp protocol version as in all other places. -constexpr UInt64 DEPRECATED_FIELD_PARALLEL_REPLICAS_PROTOCOL_VERSION = 3; - CoordinationMode validateAndGet(uint8_t candidate) { if (candidate <= static_cast(CoordinationMode::MAX)) @@ -35,10 +29,15 @@ CoordinationMode validateAndGet(uint8_t candidate) } } -void ParallelReadRequest::serialize(WriteBuffer & out) const +void ParallelReadRequest::serialize(WriteBuffer & out, UInt64 initiator_protocol_version) const { - /// Must be the first - writeIntBinary(DEPRECATED_FIELD_PARALLEL_REPLICAS_PROTOCOL_VERSION, out); + /// Previously we didn't maintain backward compatibility and every change was breaking. + /// Particularly, we had an equality check for the version. To work around that code + /// in previous server versions we now have to lie to them about the version. + const UInt64 version = initiator_protocol_version >= DBMS_MIN_REVISION_WITH_VERSIONED_PARALLEL_REPLICAS_PROTOCOL + ? DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION + : DBMS_MIN_SUPPORTED_PARALLEL_REPLICAS_PROTOCOL_VERSION; + writeIntBinary(version, out); writeIntBinary(mode, out); writeIntBinary(replica_num, out); @@ -60,13 +59,12 @@ ParallelReadRequest ParallelReadRequest::deserialize(ReadBuffer & in) { UInt64 version; readIntBinary(version, in); - if (version != DEPRECATED_FIELD_PARALLEL_REPLICAS_PROTOCOL_VERSION) + if (version < DBMS_MIN_SUPPORTED_PARALLEL_REPLICAS_PROTOCOL_VERSION) throw Exception( ErrorCodes::UNKNOWN_PROTOCOL, - "Protocol versions for parallel reading " - "from replicas differ. Got: {}, supported version: {}", + "Parallel replicas protocol version is too old. Got: {}, min supported version: {}", version, - DEPRECATED_FIELD_PARALLEL_REPLICAS_PROTOCOL_VERSION); + DBMS_MIN_SUPPORTED_PARALLEL_REPLICAS_PROTOCOL_VERSION); CoordinationMode mode; size_t replica_num; @@ -80,12 +78,7 @@ ParallelReadRequest ParallelReadRequest::deserialize(ReadBuffer & in) readIntBinary(min_number_of_marks, in); description.deserialize(in); - return ParallelReadRequest( - mode, - replica_num, - min_number_of_marks, - std::move(description) - ); + return ParallelReadRequest(mode, replica_num, min_number_of_marks, std::move(description)); } void ParallelReadRequest::merge(ParallelReadRequest & other) @@ -96,10 +89,16 @@ void ParallelReadRequest::merge(ParallelReadRequest & other) description.merge(other.description); } -void ParallelReadResponse::serialize(WriteBuffer & out) const +void ParallelReadResponse::serialize(WriteBuffer & out, UInt64 replica_protocol_version) const { + /// Previously we didn't maintain backward compatibility and every change was breaking. + /// Particularly, we had an equality check for the version. To work around that code + /// in previous server versions we now have to lie to them about the version. + UInt64 version = replica_protocol_version >= DBMS_MIN_REVISION_WITH_VERSIONED_PARALLEL_REPLICAS_PROTOCOL + ? DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION + : DBMS_MIN_SUPPORTED_PARALLEL_REPLICAS_PROTOCOL_VERSION; /// Must be the first - writeIntBinary(DEPRECATED_FIELD_PARALLEL_REPLICAS_PROTOCOL_VERSION, out); + writeIntBinary(version, out); writeBoolText(finish, out); description.serialize(out); @@ -114,28 +113,32 @@ void ParallelReadResponse::deserialize(ReadBuffer & in) { UInt64 version; readIntBinary(version, in); - if (version != DEPRECATED_FIELD_PARALLEL_REPLICAS_PROTOCOL_VERSION) + if (version < DBMS_MIN_SUPPORTED_PARALLEL_REPLICAS_PROTOCOL_VERSION) throw Exception( ErrorCodes::UNKNOWN_PROTOCOL, - "Protocol versions for parallel reading " - "from replicas differ. Got: {}, supported version: {}", + "Parallel replicas protocol version is too old. Got: {}, min supported version: {}", version, - DEPRECATED_FIELD_PARALLEL_REPLICAS_PROTOCOL_VERSION); + DBMS_MIN_SUPPORTED_PARALLEL_REPLICAS_PROTOCOL_VERSION); readBoolText(finish, in); description.deserialize(in); } -void InitialAllRangesAnnouncement::serialize(WriteBuffer & out, UInt64 client_protocol_revision) const +void InitialAllRangesAnnouncement::serialize(WriteBuffer & out, UInt64 initiator_protocol_version) const { - /// Must be the first - writeIntBinary(DEPRECATED_FIELD_PARALLEL_REPLICAS_PROTOCOL_VERSION, out); + /// Previously we didn't maintain backward compatibility and every change was breaking. + /// Particularly, we had an equality check for the version. To work around that code + /// in previous server versions we now have to lie to them about the version. + UInt64 version = initiator_protocol_version >= DBMS_MIN_REVISION_WITH_VERSIONED_PARALLEL_REPLICAS_PROTOCOL + ? DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION + : DBMS_MIN_SUPPORTED_PARALLEL_REPLICAS_PROTOCOL_VERSION; + writeIntBinary(version, out); writeIntBinary(mode, out); description.serialize(out); writeIntBinary(replica_num, out); - if (client_protocol_revision >= DBMS_MIN_REVISION_WITH_ADAPTIVE_MARK_SEGMENT_FOR_PARALLEL_REPLICAS) + if (initiator_protocol_version >= DBMS_MIN_REVISION_WITH_VERSIONED_PARALLEL_REPLICAS_PROTOCOL) writeIntBinary(mark_segment_size, out); } @@ -148,17 +151,16 @@ String InitialAllRangesAnnouncement::describe() return result; } -InitialAllRangesAnnouncement InitialAllRangesAnnouncement::deserialize(ReadBuffer & in, UInt64 client_protocol_revision) +InitialAllRangesAnnouncement InitialAllRangesAnnouncement::deserialize(ReadBuffer & in, UInt64 replica_protocol_version) { UInt64 version; readIntBinary(version, in); - if (version != DEPRECATED_FIELD_PARALLEL_REPLICAS_PROTOCOL_VERSION) + if (version < DBMS_MIN_SUPPORTED_PARALLEL_REPLICAS_PROTOCOL_VERSION) throw Exception( ErrorCodes::UNKNOWN_PROTOCOL, - "Protocol versions for parallel reading " - "from replicas differ. Got: {}, supported version: {}", + "Parallel replicas protocol version is too old. Got: {}, min supported version: {}", version, - DEPRECATED_FIELD_PARALLEL_REPLICAS_PROTOCOL_VERSION); + DBMS_MIN_SUPPORTED_PARALLEL_REPLICAS_PROTOCOL_VERSION); CoordinationMode mode; RangesInDataPartsDescription description; @@ -171,15 +173,10 @@ InitialAllRangesAnnouncement InitialAllRangesAnnouncement::deserialize(ReadBuffe readIntBinary(replica_num, in); size_t mark_segment_size = 128; - if (client_protocol_revision >= DBMS_MIN_REVISION_WITH_ADAPTIVE_MARK_SEGMENT_FOR_PARALLEL_REPLICAS) + if (replica_protocol_version >= DBMS_MIN_REVISION_WITH_VERSIONED_PARALLEL_REPLICAS_PROTOCOL) readIntBinary(mark_segment_size, in); - return InitialAllRangesAnnouncement{ - mode, - description, - replica_num, - mark_segment_size, - }; + return InitialAllRangesAnnouncement{mode, description, replica_num, mark_segment_size}; } } diff --git a/src/Storages/MergeTree/RequestResponse.h b/src/Storages/MergeTree/RequestResponse.h index da2fa2795a3..96b65c45bfa 100644 --- a/src/Storages/MergeTree/RequestResponse.h +++ b/src/Storages/MergeTree/RequestResponse.h @@ -63,7 +63,7 @@ struct ParallelReadRequest /// Contains only data part names without mark ranges. RangesInDataPartsDescription description; - void serialize(WriteBuffer & out) const; + void serialize(WriteBuffer & out, UInt64 initiator_protocol_version) const; String describe() const; static ParallelReadRequest deserialize(ReadBuffer & in); void merge(ParallelReadRequest & other); @@ -78,7 +78,7 @@ struct ParallelReadResponse bool finish{false}; RangesInDataPartsDescription description; - void serialize(WriteBuffer & out) const; + void serialize(WriteBuffer & out, UInt64 replica_protocol_version) const; String describe() const; void deserialize(ReadBuffer & in); }; @@ -102,9 +102,9 @@ struct InitialAllRangesAnnouncement size_t replica_num; size_t mark_segment_size; - void serialize(WriteBuffer & out, UInt64 client_protocol_revision) const; + void serialize(WriteBuffer & out, UInt64 initiator_protocol_version) const; String describe(); - static InitialAllRangesAnnouncement deserialize(ReadBuffer & i, UInt64 client_protocol_revisionn); + static InitialAllRangesAnnouncement deserialize(ReadBuffer & i, UInt64 replica_protocol_version); }; From 877002f6895bfe81df1068fb365f90a0380ef2d2 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 12 Sep 2024 14:41:21 +0000 Subject: [PATCH 046/103] 3% more aesthetic build description --- contrib/libpq-cmake/CMakeLists.txt | 96 +++++++++++++++--------------- 1 file changed, 48 insertions(+), 48 deletions(-) diff --git a/contrib/libpq-cmake/CMakeLists.txt b/contrib/libpq-cmake/CMakeLists.txt index fd5b6916ca2..b2c3838c232 100644 --- a/contrib/libpq-cmake/CMakeLists.txt +++ b/contrib/libpq-cmake/CMakeLists.txt @@ -1,56 +1,56 @@ set(LIBPQ_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/libpq") set(SRCS - "${LIBPQ_SOURCE_DIR}/fe-auth.c" - "${LIBPQ_SOURCE_DIR}/fe-auth-scram.c" - "${LIBPQ_SOURCE_DIR}/fe-connect.c" - "${LIBPQ_SOURCE_DIR}/fe-exec.c" - "${LIBPQ_SOURCE_DIR}/fe-lobj.c" - "${LIBPQ_SOURCE_DIR}/fe-misc.c" - "${LIBPQ_SOURCE_DIR}/fe-print.c" - "${LIBPQ_SOURCE_DIR}/fe-trace.c" - "${LIBPQ_SOURCE_DIR}/fe-protocol3.c" - "${LIBPQ_SOURCE_DIR}/fe-secure.c" - "${LIBPQ_SOURCE_DIR}/fe-secure-common.c" - "${LIBPQ_SOURCE_DIR}/fe-secure-openssl.c" - "${LIBPQ_SOURCE_DIR}/legacy-pqsignal.c" - "${LIBPQ_SOURCE_DIR}/libpq-events.c" - "${LIBPQ_SOURCE_DIR}/pqexpbuffer.c" + "${LIBPQ_SOURCE_DIR}/fe-auth.c" + "${LIBPQ_SOURCE_DIR}/fe-auth-scram.c" + "${LIBPQ_SOURCE_DIR}/fe-connect.c" + "${LIBPQ_SOURCE_DIR}/fe-exec.c" + "${LIBPQ_SOURCE_DIR}/fe-lobj.c" + "${LIBPQ_SOURCE_DIR}/fe-misc.c" + "${LIBPQ_SOURCE_DIR}/fe-print.c" + "${LIBPQ_SOURCE_DIR}/fe-trace.c" + "${LIBPQ_SOURCE_DIR}/fe-protocol3.c" + "${LIBPQ_SOURCE_DIR}/fe-secure.c" + "${LIBPQ_SOURCE_DIR}/fe-secure-common.c" + "${LIBPQ_SOURCE_DIR}/fe-secure-openssl.c" + "${LIBPQ_SOURCE_DIR}/legacy-pqsignal.c" + "${LIBPQ_SOURCE_DIR}/libpq-events.c" + "${LIBPQ_SOURCE_DIR}/pqexpbuffer.c" - "${LIBPQ_SOURCE_DIR}/common/scram-common.c" - "${LIBPQ_SOURCE_DIR}/common/sha2.c" - "${LIBPQ_SOURCE_DIR}/common/sha1.c" - "${LIBPQ_SOURCE_DIR}/common/md5.c" - "${LIBPQ_SOURCE_DIR}/common/md5_common.c" - "${LIBPQ_SOURCE_DIR}/common/hmac_openssl.c" - "${LIBPQ_SOURCE_DIR}/common/cryptohash.c" - "${LIBPQ_SOURCE_DIR}/common/saslprep.c" - "${LIBPQ_SOURCE_DIR}/common/unicode_norm.c" - "${LIBPQ_SOURCE_DIR}/common/ip.c" - "${LIBPQ_SOURCE_DIR}/common/jsonapi.c" - "${LIBPQ_SOURCE_DIR}/common/wchar.c" - "${LIBPQ_SOURCE_DIR}/common/base64.c" - "${LIBPQ_SOURCE_DIR}/common/link-canary.c" - "${LIBPQ_SOURCE_DIR}/common/fe_memutils.c" - "${LIBPQ_SOURCE_DIR}/common/string.c" - "${LIBPQ_SOURCE_DIR}/common/pg_get_line.c" - "${LIBPQ_SOURCE_DIR}/common/stringinfo.c" - "${LIBPQ_SOURCE_DIR}/common/psprintf.c" - "${LIBPQ_SOURCE_DIR}/common/encnames.c" - "${LIBPQ_SOURCE_DIR}/common/logging.c" + "${LIBPQ_SOURCE_DIR}/common/scram-common.c" + "${LIBPQ_SOURCE_DIR}/common/sha2.c" + "${LIBPQ_SOURCE_DIR}/common/sha1.c" + "${LIBPQ_SOURCE_DIR}/common/md5.c" + "${LIBPQ_SOURCE_DIR}/common/md5_common.c" + "${LIBPQ_SOURCE_DIR}/common/hmac_openssl.c" + "${LIBPQ_SOURCE_DIR}/common/cryptohash.c" + "${LIBPQ_SOURCE_DIR}/common/saslprep.c" + "${LIBPQ_SOURCE_DIR}/common/unicode_norm.c" + "${LIBPQ_SOURCE_DIR}/common/ip.c" + "${LIBPQ_SOURCE_DIR}/common/jsonapi.c" + "${LIBPQ_SOURCE_DIR}/common/wchar.c" + "${LIBPQ_SOURCE_DIR}/common/base64.c" + "${LIBPQ_SOURCE_DIR}/common/link-canary.c" + "${LIBPQ_SOURCE_DIR}/common/fe_memutils.c" + "${LIBPQ_SOURCE_DIR}/common/string.c" + "${LIBPQ_SOURCE_DIR}/common/pg_get_line.c" + "${LIBPQ_SOURCE_DIR}/common/stringinfo.c" + "${LIBPQ_SOURCE_DIR}/common/psprintf.c" + "${LIBPQ_SOURCE_DIR}/common/encnames.c" + "${LIBPQ_SOURCE_DIR}/common/logging.c" - "${LIBPQ_SOURCE_DIR}/port/snprintf.c" - "${LIBPQ_SOURCE_DIR}/port/strlcpy.c" - "${LIBPQ_SOURCE_DIR}/port/strerror.c" - "${LIBPQ_SOURCE_DIR}/port/inet_net_ntop.c" - "${LIBPQ_SOURCE_DIR}/port/getpeereid.c" - "${LIBPQ_SOURCE_DIR}/port/chklocale.c" - "${LIBPQ_SOURCE_DIR}/port/noblock.c" - "${LIBPQ_SOURCE_DIR}/port/pg_strong_random.c" - "${LIBPQ_SOURCE_DIR}/port/pgstrcasecmp.c" - "${LIBPQ_SOURCE_DIR}/port/thread.c" - "${LIBPQ_SOURCE_DIR}/port/path.c" - ) + "${LIBPQ_SOURCE_DIR}/port/snprintf.c" + "${LIBPQ_SOURCE_DIR}/port/strlcpy.c" + "${LIBPQ_SOURCE_DIR}/port/strerror.c" + "${LIBPQ_SOURCE_DIR}/port/inet_net_ntop.c" + "${LIBPQ_SOURCE_DIR}/port/getpeereid.c" + "${LIBPQ_SOURCE_DIR}/port/chklocale.c" + "${LIBPQ_SOURCE_DIR}/port/noblock.c" + "${LIBPQ_SOURCE_DIR}/port/pg_strong_random.c" + "${LIBPQ_SOURCE_DIR}/port/pgstrcasecmp.c" + "${LIBPQ_SOURCE_DIR}/port/thread.c" + "${LIBPQ_SOURCE_DIR}/port/path.c" +) add_library(_libpq ${SRCS}) From beffb92411ad726e2b2302934f0c37b9223359c4 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 12 Sep 2024 14:51:59 +0000 Subject: [PATCH 047/103] Keep original order of conditions during move to prewhere --- .../MergeTree/MergeTreeWhereOptimizer.cpp | 21 +++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index f0c26c302e1..76a02bbd2c4 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -361,11 +361,23 @@ std::optional MergeTreeWhereOptimizer:: UInt64 total_size_of_moved_conditions = 0; UInt64 total_number_of_moved_columns = 0; + /// Remember positions of conditions in where_conditions list + /// to keep original order of conditions in prewhere_conditions while moving. + std::unordered_map condition_positions; + size_t position= 0; + for (const auto & condition : where_conditions) + condition_positions[&condition] = position++; + /// Move condition and all other conditions depend on the same set of columns. auto move_condition = [&](Conditions::iterator cond_it) { LOG_TRACE(log, "Condition {} moved to PREWHERE", cond_it->node.getColumnName()); - prewhere_conditions.splice(prewhere_conditions.end(), where_conditions, cond_it); + /// Keep the original order of conditions in prewhere_conditions. + position = condition_positions[&(*cond_it)]; + auto prewhere_it = prewhere_conditions.begin(); + while (condition_positions[&(*prewhere_it)] < position && prewhere_it != prewhere_conditions.end()) + ++prewhere_it; + prewhere_conditions.splice(prewhere_it, where_conditions, cond_it); total_size_of_moved_conditions += cond_it->columns_size; total_number_of_moved_columns += cond_it->table_columns.size(); @@ -375,7 +387,12 @@ std::optional MergeTreeWhereOptimizer:: if (jt->viable && jt->columns_size == cond_it->columns_size && jt->table_columns == cond_it->table_columns) { LOG_TRACE(log, "Condition {} moved to PREWHERE", jt->node.getColumnName()); - prewhere_conditions.splice(prewhere_conditions.end(), where_conditions, jt++); + /// Keep the original order of conditions in prewhere_conditions. + position = condition_positions[&(*jt)]; + prewhere_it = prewhere_conditions.begin(); + while (condition_positions[&(*prewhere_it)] < position && prewhere_it != prewhere_conditions.end()) + ++prewhere_it; + prewhere_conditions.splice(prewhere_it, where_conditions, jt++); } else { From 51d770fa7aeacbe0a10e7f48c86fdeec8ab214e1 Mon Sep 17 00:00:00 2001 From: 1on Date: Fri, 30 Aug 2024 16:47:41 +0300 Subject: [PATCH 048/103] Ability to limit columns for tables in MaterializedPostgreSQL --- .../materialized-postgresql.md | 6 + .../fetchPostgreSQLTableStructure.cpp | 19 ++- .../fetchPostgreSQLTableStructure.h | 2 +- .../PostgreSQLReplicationHandler.cpp | 160 ++++++++++++++---- .../PostgreSQL/PostgreSQLReplicationHandler.h | 5 +- tests/integration/helpers/postgres_utility.py | 3 +- .../test.py | 104 ++++++++++++ 7 files changed, 259 insertions(+), 40 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index 3aa6dd01ea3..97185f35e1e 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -155,6 +155,12 @@ Replication of [**TOAST**](https://www.postgresql.org/docs/9.5/storage-toast.htm Sets a comma-separated list of PostgreSQL database tables, which will be replicated via [MaterializedPostgreSQL](../../engines/database-engines/materialized-postgresql.md) database engine. + Each table can have subset of replicated columns in brackets. If subset of columns is omitted, then all columns for table will be replicated. + + ``` sql + materialized_postgresql_tables_list = 'table1(co1, col2),table2,table3(co3, col5, col7) + ``` + Default value: empty list — means whole PostgreSQL database will be replicated. ### `materialized_postgresql_schema` {#materialized-postgresql-schema} diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index b9fd9c325f8..45fd52f27ab 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include #include @@ -292,7 +293,7 @@ PostgreSQLTableStructure::ColumnsInfoPtr readNamesAndTypesList( template PostgreSQLTableStructure fetchPostgreSQLTableStructure( - T & tx, const String & postgres_table, const String & postgres_schema, bool use_nulls, bool with_primary_key, bool with_replica_identity_index) + T & tx, const String & postgres_table, const String & postgres_schema, bool use_nulls, bool with_primary_key, bool with_replica_identity_index, const Strings & columns) { PostgreSQLTableStructure table; @@ -302,6 +303,10 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( ? " AND relnamespace = (SELECT oid FROM pg_namespace WHERE nspname = 'public')" : fmt::format(" AND relnamespace = (SELECT oid FROM pg_namespace WHERE nspname = {})", quoteString(postgres_schema)); + std::string columns_part; + if (!columns.empty()) + columns_part = fmt::format(" AND attname IN ('{}')", boost::algorithm::join(columns, "','")); + std::string query = fmt::format( "SELECT attname AS name, " /// column name "format_type(atttypid, atttypmod) AS type, " /// data type @@ -312,9 +317,9 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( "attnum as att_num, " "attgenerated as generated " /// if column has GENERATED "FROM pg_attribute " - "WHERE attrelid = (SELECT oid FROM pg_class WHERE {}) " + "WHERE attrelid = (SELECT oid FROM pg_class WHERE {}) {}" "AND NOT attisdropped AND attnum > 0 " - "ORDER BY attnum ASC", where); + "ORDER BY attnum ASC", where, columns_part); auto postgres_table_with_schema = postgres_schema.empty() ? postgres_table : doubleQuoteString(postgres_schema) + '.' + doubleQuoteString(postgres_table); table.physical_columns = readNamesAndTypesList(tx, postgres_table_with_schema, query, use_nulls, false); @@ -415,7 +420,7 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( PostgreSQLTableStructure fetchPostgreSQLTableStructure(pqxx::connection & connection, const String & postgres_table, const String & postgres_schema, bool use_nulls) { pqxx::ReadTransaction tx(connection); - auto result = fetchPostgreSQLTableStructure(tx, postgres_table, postgres_schema, use_nulls, false, false); + auto result = fetchPostgreSQLTableStructure(tx, postgres_table, postgres_schema, use_nulls, false, false, {}); tx.commit(); return result; } @@ -433,17 +438,17 @@ std::set fetchPostgreSQLTablesList(pqxx::connection & connection, const template PostgreSQLTableStructure fetchPostgreSQLTableStructure( pqxx::ReadTransaction & tx, const String & postgres_table, const String & postgres_schema, - bool use_nulls, bool with_primary_key, bool with_replica_identity_index); + bool use_nulls, bool with_primary_key, bool with_replica_identity_index, const Strings & columns); template PostgreSQLTableStructure fetchPostgreSQLTableStructure( pqxx::ReplicationTransaction & tx, const String & postgres_table, const String & postgres_schema, - bool use_nulls, bool with_primary_key, bool with_replica_identity_index); + bool use_nulls, bool with_primary_key, bool with_replica_identity_index, const Strings & columns); template PostgreSQLTableStructure fetchPostgreSQLTableStructure( pqxx::nontransaction & tx, const String & postgres_table, const String & postrges_schema, - bool use_nulls, bool with_primary_key, bool with_replica_identity_index); + bool use_nulls, bool with_primary_key, bool with_replica_identity_index, const Strings & columns); std::set fetchPostgreSQLTablesList(pqxx::work & tx, const String & postgres_schema); diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h index 25ece6909fd..6f7bae44c35 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h @@ -48,7 +48,7 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( template PostgreSQLTableStructure fetchPostgreSQLTableStructure( T & tx, const String & postgres_table, const String & postgres_schema, bool use_nulls = true, - bool with_primary_key = false, bool with_replica_identity_index = false); + bool with_primary_key = false, bool with_replica_identity_index = false, const Strings & columns = {}); template std::set fetchPostgreSQLTablesList(T & tx, const String & postgres_schema); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 01f78673ed8..2fe1fb5905a 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -348,11 +348,10 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) auto * materialized_storage = storage->as (); try { - auto [postgres_table_schema, postgres_table_name] = getSchemaAndTableName(table_name); - auto table_structure = fetchPostgreSQLTableStructure(tx, postgres_table_name, postgres_table_schema, true, true, true); - if (!table_structure.physical_columns) + auto table_structure = fetchTableStructure(tx, table_name); + if (!table_structure->physical_columns) throw Exception(ErrorCodes::LOGICAL_ERROR, "No columns"); - auto storage_info = StorageInfo(materialized_storage->getNested(), table_structure.physical_columns->attributes); + auto storage_info = StorageInfo(materialized_storage->getNested(), table_structure->physical_columns->attributes); nested_storages.emplace(table_name, std::move(storage_info)); } catch (Exception & e) @@ -399,9 +398,7 @@ ASTPtr PostgreSQLReplicationHandler::getCreateNestedTableQuery(StorageMaterializ postgres::Connection connection(connection_info); pqxx::nontransaction tx(connection.getRef()); - auto [postgres_table_schema, postgres_table_name] = getSchemaAndTableName(table_name); - auto table_structure = std::make_unique(fetchPostgreSQLTableStructure(tx, postgres_table_name, postgres_table_schema, true, true, true)); - + auto table_structure = fetchTableStructure(tx, table_name); auto table_override = tryGetTableOverride(current_database_name, table_name); return storage->getCreateNestedTableQuery(std::move(table_structure), table_override ? table_override->as() : nullptr); } @@ -415,16 +412,35 @@ StorageInfo PostgreSQLReplicationHandler::loadFromSnapshot(postgres::Connection std::string query_str = fmt::format("SET TRANSACTION SNAPSHOT '{}'", snapshot_name); tx->exec(query_str); - auto table_structure = fetchTableStructure(*tx, table_name); + PostgreSQLTableStructurePtr table_structure; + try + { + table_structure = fetchTableStructure(*tx, table_name); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + table_structure = std::make_unique(); + } if (!table_structure->physical_columns) throw Exception(ErrorCodes::LOGICAL_ERROR, "No table attributes"); auto table_attributes = table_structure->physical_columns->attributes; + auto columns = getTableAllowedColumns(table_name); /// Load from snapshot, which will show table state before creation of replication slot. /// Already connected to needed database, no need to add it to query. auto quoted_name = doubleQuoteWithSchema(table_name); - query_str = fmt::format("SELECT * FROM ONLY {}", quoted_name); + if (columns.empty()) + query_str = fmt::format("SELECT * FROM ONLY {}", quoted_name); + else + { + /// We should not use columns list from getTableAllowedColumns because it may have broken columns order + Strings allowed_columns; + for (const auto & column : table_structure->physical_columns->columns) + allowed_columns.push_back(column.name); + query_str = fmt::format("SELECT {} FROM ONLY {}", boost::algorithm::join(allowed_columns, ","), quoted_name); + } LOG_DEBUG(log, "Loading PostgreSQL table {}.{}", postgres_database, quoted_name); @@ -700,6 +716,37 @@ void PostgreSQLReplicationHandler::setSetting(const SettingChange & setting) } +/// Allowed columns for table from materialized_postgresql_tables_list setting +Strings PostgreSQLReplicationHandler::getTableAllowedColumns(const std::string & table_name) const +{ + Strings result; + if (tables_list.empty()) + return result; + + size_t table_pos = tables_list.find(table_name); + if (table_pos == std::string::npos) + { + return result; + } + + if (table_pos + table_name.length() + 1 > tables_list.length()) + { + return result; + } + String column_list = tables_list.substr(table_pos + table_name.length() + 1); + column_list.erase(std::remove(column_list.begin(), column_list.end(), '"'), column_list.end()); + boost::trim(column_list); + if (column_list.empty() || column_list[0] != '(') + return result; + + size_t end_bracket_pos = column_list.find(')'); + column_list = column_list.substr(1, end_bracket_pos - 1); + splitInto<','>(result, column_list); + + return result; +} + + void PostgreSQLReplicationHandler::shutdownFinal() { try @@ -749,11 +796,27 @@ std::set PostgreSQLReplicationHandler::fetchRequiredTables() Strings expected_tables; if (!tables_list.empty()) { - splitInto<','>(expected_tables, tables_list); - if (expected_tables.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse tables list: {}", tables_list); - for (auto & table_name : expected_tables) - boost::trim(table_name); + /// Removing columns `table(col1, col2)` from tables_list + String cleared_tables_list = tables_list; + while (true) + { + size_t start_bracket_pos = cleared_tables_list.find('('); + size_t end_bracket_pos = cleared_tables_list.find(')'); + if (start_bracket_pos == std::string::npos || end_bracket_pos == std::string::npos) + { + break; + } + cleared_tables_list = cleared_tables_list.substr(0, start_bracket_pos) + cleared_tables_list.substr(end_bracket_pos + 1); + } + + splitInto<','>(expected_tables, cleared_tables_list); + if (expected_tables.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse tables list: {}", tables_list); + + for (auto & table_name : expected_tables) + { + boost::trim(table_name); + } } /// Try to fetch tables list from publication if there is not tables list. @@ -864,18 +927,50 @@ std::set PostgreSQLReplicationHandler::fetchRequiredTables() /// `schema1.table1, schema2.table2, ...` -> `"schema1"."table1", "schema2"."table2", ...` /// or /// `table1, table2, ...` + setting `schema` -> `"schema"."table1", "schema"."table2", ...` + /// or + /// `table1, table2(id,name), ...` + setting `schema` -> `"schema"."table1", "schema"."table2"("id","name"), ...` if (!tables_list.empty()) { - Strings tables_names; - splitInto<','>(tables_names, tables_list); - if (tables_names.empty()) + Strings parts; + splitInto<','>(parts, tables_list); + if (parts.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Empty list of tables"); + bool is_column = false; WriteBufferFromOwnString buf; - for (auto & table_name : tables_names) + for (auto & part : parts) { - boost::trim(table_name); - buf << doubleQuoteWithSchema(table_name); + boost::trim(part); + + size_t bracket_pos = part.find('('); + if (bracket_pos != std::string::npos) + { + is_column = true; + std::string table_name = part.substr(0, bracket_pos); + boost::trim(table_name); + buf << doubleQuoteWithSchema(table_name); + + part = part.substr(bracket_pos + 1); + boost::trim(part); + buf << '('; + buf << doubleQuoteString(part); + } + else if (part.back() == ')') + { + is_column = false; + part = part.substr(0, part.size() - 1); + boost::trim(part); + buf << doubleQuoteString(part); + buf << ')'; + } + else if (is_column) + { + buf << doubleQuoteString(part); + } + else + { + buf << doubleQuoteWithSchema(part); + } buf << ","; } tables_list = buf.str(); @@ -902,23 +997,28 @@ std::set PostgreSQLReplicationHandler::fetchTablesFromPublication(pqxx:: } +template PostgreSQLTableStructurePtr PostgreSQLReplicationHandler::fetchTableStructure( - pqxx::ReplicationTransaction & tx, const std::string & table_name) const + T & tx, const std::string & table_name) const { PostgreSQLTableStructure structure; - try - { - auto [schema, table] = getSchemaAndTableName(table_name); - structure = fetchPostgreSQLTableStructure(tx, table, schema, true, true, true); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } + auto [schema, table] = getSchemaAndTableName(table_name); + structure = fetchPostgreSQLTableStructure(tx, table, schema, true, true, true, getTableAllowedColumns(table_name)); return std::make_unique(std::move(structure)); } +template +PostgreSQLTableStructurePtr PostgreSQLReplicationHandler::fetchTableStructure( + pqxx::ReadTransaction & tx, const std::string & table_name) const; + +template +PostgreSQLTableStructurePtr PostgreSQLReplicationHandler::fetchTableStructure( + pqxx::ReplicationTransaction & tx, const std::string & table_name) const; + +template +PostgreSQLTableStructurePtr PostgreSQLReplicationHandler::fetchTableStructure( + pqxx::nontransaction & tx, const std::string & table_name) const; void PostgreSQLReplicationHandler::addTableToReplication(StorageMaterializedPostgreSQL * materialized_storage, const String & postgres_table_name) { diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 5c519053d84..8257f92ae1f 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -57,6 +57,8 @@ public: void setSetting(const SettingChange & setting); + Strings getTableAllowedColumns(const std::string & table_name) const; + void cleanupFunc(); private: @@ -94,7 +96,8 @@ private: StorageInfo loadFromSnapshot(postgres::Connection & connection, std::string & snapshot_name, const String & table_name, StorageMaterializedPostgreSQL * materialized_storage); - PostgreSQLTableStructurePtr fetchTableStructure(pqxx::ReplicationTransaction & tx, const String & table_name) const; + template + PostgreSQLTableStructurePtr fetchTableStructure(T & tx, const String & table_name) const; String doubleQuoteWithSchema(const String & table_name) const; diff --git a/tests/integration/helpers/postgres_utility.py b/tests/integration/helpers/postgres_utility.py index 468c3b3bb63..76dddd7d0cf 100644 --- a/tests/integration/helpers/postgres_utility.py +++ b/tests/integration/helpers/postgres_utility.py @@ -359,6 +359,7 @@ def check_tables_are_synchronized( postgres_database="postgres_database", materialized_database="test_database", schema_name="", + columns=["*"], ): assert_nested_table_is_created( instance, table_name, materialized_database, schema_name @@ -374,7 +375,7 @@ def check_tables_are_synchronized( result_query = f"select * from {table_path} order by {order_by};" expected = instance.query( - f"select * from `{postgres_database}`.`{table_name}` order by {order_by};" + f"select {','.join(columns)} from `{postgres_database}`.`{table_name}` order by {order_by};" ) result = instance.query(result_query) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 7fdd17625a9..e64c9eb9d1e 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -1141,6 +1141,110 @@ def test_dependent_loading(started_cluster): instance.query(f"DROP TABLE {table} SYNC") +def test_partial_table(started_cluster): + table = "test_partial_table" + + pg_manager.create_postgres_table( + table, + "", + f"""CREATE TABLE {table} ( + key integer PRIMARY KEY, + x integer DEFAULT 0, + y integer, + z text DEFAULT 'z'); + """, + ) + pg_manager.execute(f"insert into {table} (key, x, z) values (1,1,'a');") + pg_manager.execute(f"insert into {table} (key, x, z) values (2,2,'b');") + + pg_manager.create_materialized_db( + ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + settings=[ + f"materialized_postgresql_tables_list = '{table}(z, key)'", + "materialized_postgresql_backoff_min_ms = 100", + "materialized_postgresql_backoff_max_ms = 100", + ], + ) + check_tables_are_synchronized( + instance, + table, + postgres_database=pg_manager.get_default_database(), + columns=["key", "z"], + ) + + pg_manager.execute(f"insert into {table} (key, x, z) values (3,3,'c');") + pg_manager.execute(f"insert into {table} (key, x, z) values (4,4,'d');") + + check_tables_are_synchronized( + instance, + table, + postgres_database=pg_manager.get_default_database(), + columns=["key", "z"], + ) + + +def test_partial_and_full_table(started_cluster): + table = "test_partial_and_full_table" + + pg_manager.create_postgres_table( + table, + "", + f"""CREATE TABLE {table}1 ( + key integer PRIMARY KEY, + x integer DEFAULT 0, + y integer, + z text DEFAULT 'z'); + """, + ) + pg_manager.execute(f"insert into {table}1 (key, x, y, z) values (1,1,1,'1');") + pg_manager.execute(f"insert into {table}1 (key, x, y, z) values (2,2,2,'2');") + pg_manager.create_postgres_table( + table, + "", + f"""CREATE TABLE {table}2 ( + key integer PRIMARY KEY, + x integer DEFAULT 0, + y integer, + z text DEFAULT 'z'); + """, + ) + pg_manager.execute(f"insert into {table}2 (key, x, y, z) values (3,3,3,'3');") + pg_manager.execute(f"insert into {table}2 (key, x, y, z) values (4,4,4,'4');") + + pg_manager.create_materialized_db( + ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + settings=[ + f"materialized_postgresql_tables_list = '{table}1(key, x, z), {table}2'", + "materialized_postgresql_backoff_min_ms = 100", + "materialized_postgresql_backoff_max_ms = 100", + ], + ) + check_tables_are_synchronized( + instance, + f"{table}1", + postgres_database=pg_manager.get_default_database(), + columns=["key", "x", "z"], + ) + check_tables_are_synchronized( + instance, f"{table}2", postgres_database=pg_manager.get_default_database() + ) + + pg_manager.execute(f"insert into {table}1 (key, x, z) values (3,3,'3');") + pg_manager.execute(f"insert into {table}2 (key, x, z) values (5,5,'5');") + + check_tables_are_synchronized( + instance, + f"{table}1", + postgres_database=pg_manager.get_default_database(), + columns=["key", "x", "z"], + ) + check_tables_are_synchronized( + instance, f"{table}2", postgres_database=pg_manager.get_default_database() + ) + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From 401a3d09317d4cbe401d64cdf50c056a5c08e63a Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 12 Sep 2024 15:10:29 +0000 Subject: [PATCH 049/103] Add test --- .../0_stateless/03231_prewhere_conditions_order.reference | 1 + .../queries/0_stateless/03231_prewhere_conditions_order.sql | 6 ++++++ 2 files changed, 7 insertions(+) create mode 100644 tests/queries/0_stateless/03231_prewhere_conditions_order.reference create mode 100644 tests/queries/0_stateless/03231_prewhere_conditions_order.sql diff --git a/tests/queries/0_stateless/03231_prewhere_conditions_order.reference b/tests/queries/0_stateless/03231_prewhere_conditions_order.reference new file mode 100644 index 00000000000..bb14c5f88f2 --- /dev/null +++ b/tests/queries/0_stateless/03231_prewhere_conditions_order.reference @@ -0,0 +1 @@ +1 [0,1] [0,1] diff --git a/tests/queries/0_stateless/03231_prewhere_conditions_order.sql b/tests/queries/0_stateless/03231_prewhere_conditions_order.sql new file mode 100644 index 00000000000..acaba12684c --- /dev/null +++ b/tests/queries/0_stateless/03231_prewhere_conditions_order.sql @@ -0,0 +1,6 @@ +drop table if exists test; +create table test (x UInt32, arr1 Array(UInt32), arr2 Array(UInt32)) engine=MergeTree order by x; +insert into test values (1, [0, 1], [0, 1]), (2, [0], [0, 1]); +select * from test where x == 1 and arrayExists((x1, x2) -> (x1 == x2), arr1, arr2); +drop table test; + From 0bb3967d143a28e994cd7352ef6d0f202a0df1da Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 12 Sep 2024 14:45:25 +0000 Subject: [PATCH 050/103] Remove obsolete target_include_directories (/config does not exist) --- contrib/libpq-cmake/CMakeLists.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/contrib/libpq-cmake/CMakeLists.txt b/contrib/libpq-cmake/CMakeLists.txt index b2c3838c232..be1e88b6cc2 100644 --- a/contrib/libpq-cmake/CMakeLists.txt +++ b/contrib/libpq-cmake/CMakeLists.txt @@ -60,7 +60,6 @@ add_definitions(-DHAVE_HMAC_CTX_FREE) target_include_directories (_libpq SYSTEM PUBLIC ${LIBPQ_SOURCE_DIR}) target_include_directories (_libpq SYSTEM PUBLIC "${LIBPQ_SOURCE_DIR}/include") -target_include_directories (_libpq SYSTEM PRIVATE "${LIBPQ_SOURCE_DIR}/configs") # NOTE: this is a dirty hack to avoid and instead pg_config.h should be shipped # for different OS'es like for jemalloc, not one generic for all OS'es like From e2bfce66dda4fef55b546ea4735c37a396e75a2d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 12 Sep 2024 14:38:28 +0000 Subject: [PATCH 051/103] Add postgres as a submodule --- .gitmodules | 3 +++ contrib/postgres | 1 + 2 files changed, 4 insertions(+) create mode 160000 contrib/postgres diff --git a/.gitmodules b/.gitmodules index 53ebde0cd3b..b437e06d2ac 100644 --- a/.gitmodules +++ b/.gitmodules @@ -369,3 +369,6 @@ [submodule "contrib/numactl"] path = contrib/numactl url = https://github.com/ClickHouse/numactl.git +[submodule "contrib/postgres"] + path = contrib/postgres + url = https://github.com/ClickHouse/postgres.git diff --git a/contrib/postgres b/contrib/postgres new file mode 160000 index 00000000000..4eada203a5a --- /dev/null +++ b/contrib/postgres @@ -0,0 +1 @@ +Subproject commit 4eada203a5a871f893afe3eb3e07eea5de1aa642 From 4963ab603cdf003edb797ec3fc163900ee3d28fa Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 12 Sep 2024 14:39:24 +0000 Subject: [PATCH 052/103] Switch Postgres to 2f7bae2f92, adjust build description, delete libqp Based on the code state of July 2021 which Kseniia copied over here: https://github.com/ClickHouse/libpq/pull/5 (found out the hard way) --- .gitmodules | 3 - contrib/CMakeLists.txt | 2 +- contrib/libpq | 1 - contrib/libpq-cmake/CMakeLists.txt | 73 -- contrib/postgres | 2 +- contrib/postgres-cmake/CMakeLists.txt | 78 ++ contrib/postgres-cmake/pg_config.h | 925 +++++++++++++++++++++++ contrib/postgres-cmake/pg_config_ext.h | 7 + contrib/postgres-cmake/pg_config_os.h | 34 + contrib/postgres-cmake/pg_config_paths.h | 12 + contrib/postgres-cmake/utils/errcodes.h | 0 11 files changed, 1058 insertions(+), 79 deletions(-) delete mode 160000 contrib/libpq delete mode 100644 contrib/libpq-cmake/CMakeLists.txt create mode 100644 contrib/postgres-cmake/CMakeLists.txt create mode 100644 contrib/postgres-cmake/pg_config.h create mode 100644 contrib/postgres-cmake/pg_config_ext.h create mode 100644 contrib/postgres-cmake/pg_config_os.h create mode 100644 contrib/postgres-cmake/pg_config_paths.h create mode 100644 contrib/postgres-cmake/utils/errcodes.h diff --git a/.gitmodules b/.gitmodules index b437e06d2ac..3aa2e4e8ea9 100644 --- a/.gitmodules +++ b/.gitmodules @@ -170,9 +170,6 @@ [submodule "contrib/fast_float"] path = contrib/fast_float url = https://github.com/fastfloat/fast_float -[submodule "contrib/libpq"] - path = contrib/libpq - url = https://github.com/ClickHouse/libpq [submodule "contrib/NuRaft"] path = contrib/NuRaft url = https://github.com/ClickHouse/NuRaft diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 7f980cef76d..c36ace61396 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -148,8 +148,8 @@ add_contrib (cppkafka-cmake cppkafka) option(ENABLE_LIBPQXX "Enable PostgreSQL" ${ENABLE_LIBRARIES}) if (ENABLE_LIBPQXX) + add_contrib (postgres-cmake postgres) add_contrib (libpqxx-cmake libpqxx) - add_contrib (libpq-cmake libpq) endif() add_contrib (rocksdb-cmake rocksdb) # requires: jemalloc, snappy, zlib, lz4, zstd, liburing diff --git a/contrib/libpq b/contrib/libpq deleted file mode 160000 index 2446f2c8565..00000000000 --- a/contrib/libpq +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 2446f2c85650b56df9d4ebc4c2ea7f4b01beee57 diff --git a/contrib/libpq-cmake/CMakeLists.txt b/contrib/libpq-cmake/CMakeLists.txt deleted file mode 100644 index be1e88b6cc2..00000000000 --- a/contrib/libpq-cmake/CMakeLists.txt +++ /dev/null @@ -1,73 +0,0 @@ -set(LIBPQ_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/libpq") - -set(SRCS - "${LIBPQ_SOURCE_DIR}/fe-auth.c" - "${LIBPQ_SOURCE_DIR}/fe-auth-scram.c" - "${LIBPQ_SOURCE_DIR}/fe-connect.c" - "${LIBPQ_SOURCE_DIR}/fe-exec.c" - "${LIBPQ_SOURCE_DIR}/fe-lobj.c" - "${LIBPQ_SOURCE_DIR}/fe-misc.c" - "${LIBPQ_SOURCE_DIR}/fe-print.c" - "${LIBPQ_SOURCE_DIR}/fe-trace.c" - "${LIBPQ_SOURCE_DIR}/fe-protocol3.c" - "${LIBPQ_SOURCE_DIR}/fe-secure.c" - "${LIBPQ_SOURCE_DIR}/fe-secure-common.c" - "${LIBPQ_SOURCE_DIR}/fe-secure-openssl.c" - "${LIBPQ_SOURCE_DIR}/legacy-pqsignal.c" - "${LIBPQ_SOURCE_DIR}/libpq-events.c" - "${LIBPQ_SOURCE_DIR}/pqexpbuffer.c" - - "${LIBPQ_SOURCE_DIR}/common/scram-common.c" - "${LIBPQ_SOURCE_DIR}/common/sha2.c" - "${LIBPQ_SOURCE_DIR}/common/sha1.c" - "${LIBPQ_SOURCE_DIR}/common/md5.c" - "${LIBPQ_SOURCE_DIR}/common/md5_common.c" - "${LIBPQ_SOURCE_DIR}/common/hmac_openssl.c" - "${LIBPQ_SOURCE_DIR}/common/cryptohash.c" - "${LIBPQ_SOURCE_DIR}/common/saslprep.c" - "${LIBPQ_SOURCE_DIR}/common/unicode_norm.c" - "${LIBPQ_SOURCE_DIR}/common/ip.c" - "${LIBPQ_SOURCE_DIR}/common/jsonapi.c" - "${LIBPQ_SOURCE_DIR}/common/wchar.c" - "${LIBPQ_SOURCE_DIR}/common/base64.c" - "${LIBPQ_SOURCE_DIR}/common/link-canary.c" - "${LIBPQ_SOURCE_DIR}/common/fe_memutils.c" - "${LIBPQ_SOURCE_DIR}/common/string.c" - "${LIBPQ_SOURCE_DIR}/common/pg_get_line.c" - "${LIBPQ_SOURCE_DIR}/common/stringinfo.c" - "${LIBPQ_SOURCE_DIR}/common/psprintf.c" - "${LIBPQ_SOURCE_DIR}/common/encnames.c" - "${LIBPQ_SOURCE_DIR}/common/logging.c" - - "${LIBPQ_SOURCE_DIR}/port/snprintf.c" - "${LIBPQ_SOURCE_DIR}/port/strlcpy.c" - "${LIBPQ_SOURCE_DIR}/port/strerror.c" - "${LIBPQ_SOURCE_DIR}/port/inet_net_ntop.c" - "${LIBPQ_SOURCE_DIR}/port/getpeereid.c" - "${LIBPQ_SOURCE_DIR}/port/chklocale.c" - "${LIBPQ_SOURCE_DIR}/port/noblock.c" - "${LIBPQ_SOURCE_DIR}/port/pg_strong_random.c" - "${LIBPQ_SOURCE_DIR}/port/pgstrcasecmp.c" - "${LIBPQ_SOURCE_DIR}/port/thread.c" - "${LIBPQ_SOURCE_DIR}/port/path.c" -) - -add_library(_libpq ${SRCS}) - -add_definitions(-DHAVE_BIO_METH_NEW) -add_definitions(-DHAVE_HMAC_CTX_NEW) -add_definitions(-DHAVE_HMAC_CTX_FREE) - -target_include_directories (_libpq SYSTEM PUBLIC ${LIBPQ_SOURCE_DIR}) -target_include_directories (_libpq SYSTEM PUBLIC "${LIBPQ_SOURCE_DIR}/include") - -# NOTE: this is a dirty hack to avoid and instead pg_config.h should be shipped -# for different OS'es like for jemalloc, not one generic for all OS'es like -# now. -if (OS_DARWIN OR OS_FREEBSD OR USE_MUSL) - target_compile_definitions(_libpq PRIVATE -DSTRERROR_R_INT=1) -endif() - -target_link_libraries (_libpq PRIVATE OpenSSL::SSL) - -add_library(ch_contrib::libpq ALIAS _libpq) diff --git a/contrib/postgres b/contrib/postgres index 4eada203a5a..f0740b4eb82 160000 --- a/contrib/postgres +++ b/contrib/postgres @@ -1 +1 @@ -Subproject commit 4eada203a5a871f893afe3eb3e07eea5de1aa642 +Subproject commit f0740b4eb8210cf3fd44de078ff9692d0b09bdb5 diff --git a/contrib/postgres-cmake/CMakeLists.txt b/contrib/postgres-cmake/CMakeLists.txt new file mode 100644 index 00000000000..644e6530bbd --- /dev/null +++ b/contrib/postgres-cmake/CMakeLists.txt @@ -0,0 +1,78 @@ +# Build description for libpq which is part of the PostgreSQL sources + +set(POSTGRES_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/postgres") +set(LIBPQ_SOURCE_DIR "${POSTGRES_SOURCE_DIR}/src/interfaces/libpq") +set(LIBPQ_CMAKE_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/postgres-cmake") + +set(SRCS + "${LIBPQ_SOURCE_DIR}/fe-auth.c" + "${LIBPQ_SOURCE_DIR}/fe-auth-scram.c" + "${LIBPQ_SOURCE_DIR}/fe-connect.c" + "${LIBPQ_SOURCE_DIR}/fe-exec.c" + "${LIBPQ_SOURCE_DIR}/fe-lobj.c" + "${LIBPQ_SOURCE_DIR}/fe-misc.c" + "${LIBPQ_SOURCE_DIR}/fe-print.c" + "${LIBPQ_SOURCE_DIR}/fe-trace.c" + "${LIBPQ_SOURCE_DIR}/fe-protocol3.c" + "${LIBPQ_SOURCE_DIR}/fe-secure.c" + "${LIBPQ_SOURCE_DIR}/fe-secure-common.c" + "${LIBPQ_SOURCE_DIR}/fe-secure-openssl.c" + "${LIBPQ_SOURCE_DIR}/legacy-pqsignal.c" + "${LIBPQ_SOURCE_DIR}/libpq-events.c" + "${LIBPQ_SOURCE_DIR}/pqexpbuffer.c" + + "${POSTGRES_SOURCE_DIR}/src/common/scram-common.c" + "${POSTGRES_SOURCE_DIR}/src/common/sha2.c" + "${POSTGRES_SOURCE_DIR}/src/common/sha1.c" + "${POSTGRES_SOURCE_DIR}/src/common/md5.c" + "${POSTGRES_SOURCE_DIR}/src/common/md5_common.c" + "${POSTGRES_SOURCE_DIR}/src/common/hmac_openssl.c" + "${POSTGRES_SOURCE_DIR}/src/common/cryptohash.c" + "${POSTGRES_SOURCE_DIR}/src/common/saslprep.c" + "${POSTGRES_SOURCE_DIR}/src/common/unicode_norm.c" + "${POSTGRES_SOURCE_DIR}/src/common/ip.c" + "${POSTGRES_SOURCE_DIR}/src/common/jsonapi.c" + "${POSTGRES_SOURCE_DIR}/src/common/wchar.c" + "${POSTGRES_SOURCE_DIR}/src/common/base64.c" + "${POSTGRES_SOURCE_DIR}/src/common/link-canary.c" + "${POSTGRES_SOURCE_DIR}/src/common/fe_memutils.c" + "${POSTGRES_SOURCE_DIR}/src/common/string.c" + "${POSTGRES_SOURCE_DIR}/src/common/pg_get_line.c" + "${POSTGRES_SOURCE_DIR}/src/common/stringinfo.c" + "${POSTGRES_SOURCE_DIR}/src/common/psprintf.c" + "${POSTGRES_SOURCE_DIR}/src/common/encnames.c" + "${POSTGRES_SOURCE_DIR}/src/common/logging.c" + + "${POSTGRES_SOURCE_DIR}/src/port/snprintf.c" + "${POSTGRES_SOURCE_DIR}/src/port/strlcpy.c" + "${POSTGRES_SOURCE_DIR}/src/port/strerror.c" + "${POSTGRES_SOURCE_DIR}/src/port/inet_net_ntop.c" + "${POSTGRES_SOURCE_DIR}/src/port/getpeereid.c" + "${POSTGRES_SOURCE_DIR}/src/port/chklocale.c" + "${POSTGRES_SOURCE_DIR}/src/port/noblock.c" + "${POSTGRES_SOURCE_DIR}/src/port/pg_strong_random.c" + "${POSTGRES_SOURCE_DIR}/src/port/pgstrcasecmp.c" + "${POSTGRES_SOURCE_DIR}/src/port/thread.c" + "${POSTGRES_SOURCE_DIR}/src/port/path.c" +) + +add_library(_libpq ${SRCS}) + +add_definitions(-DHAVE_BIO_METH_NEW) +add_definitions(-DHAVE_HMAC_CTX_NEW) +add_definitions(-DHAVE_HMAC_CTX_FREE) + +target_include_directories (_libpq SYSTEM PUBLIC ${LIBPQ_SOURCE_DIR}) +target_include_directories (_libpq SYSTEM PUBLIC "${POSTGRES_SOURCE_DIR}/src/include") +target_include_directories (_libpq SYSTEM PUBLIC "${LIBPQ_CMAKE_SOURCE_DIR}") # pre-generated headers + +# NOTE: this is a dirty hack to avoid and instead pg_config.h should be shipped +# for different OS'es like for jemalloc, not one generic for all OS'es like +# now. +if (OS_DARWIN OR OS_FREEBSD OR USE_MUSL) + target_compile_definitions(_libpq PRIVATE -DSTRERROR_R_INT=1) +endif() + +target_link_libraries (_libpq PRIVATE OpenSSL::SSL) + +add_library(ch_contrib::libpq ALIAS _libpq) diff --git a/contrib/postgres-cmake/pg_config.h b/contrib/postgres-cmake/pg_config.h new file mode 100644 index 00000000000..a812e7f0776 --- /dev/null +++ b/contrib/postgres-cmake/pg_config.h @@ -0,0 +1,925 @@ +/* src/include/pg_config.h. Generated from pg_config.h.in by configure. */ +/* src/include/pg_config.h.in. Generated from configure.in by autoheader. */ + +/* Define to the type of arg 1 of 'accept' */ +#define ACCEPT_TYPE_ARG1 int + +/* Define to the type of arg 2 of 'accept' */ +#define ACCEPT_TYPE_ARG2 struct sockaddr * + +/* Define to the type of arg 3 of 'accept' */ +#define ACCEPT_TYPE_ARG3 size_t + +/* Define to the return type of 'accept' */ +#define ACCEPT_TYPE_RETURN int + +/* Define if building universal (internal helper macro) */ +/* #undef AC_APPLE_UNIVERSAL_BUILD */ + +/* The normal alignment of `double', in bytes. */ +#define ALIGNOF_DOUBLE 4 + +/* The normal alignment of `int', in bytes. */ +#define ALIGNOF_INT 4 + +/* The normal alignment of `long', in bytes. */ +#define ALIGNOF_LONG 4 + +/* The normal alignment of `long long int', in bytes. */ +#define ALIGNOF_LONG_LONG_INT 4 + +/* The normal alignment of `short', in bytes. */ +#define ALIGNOF_SHORT 2 + +/* Size of a disk block --- this also limits the size of a tuple. You can set + it bigger if you need bigger tuples (although TOAST should reduce the need + to have large tuples, since fields can be spread across multiple tuples). + BLCKSZ must be a power of 2. The maximum possible value of BLCKSZ is + currently 2^15 (32768). This is determined by the 15-bit widths of the + lp_off and lp_len fields in ItemIdData (see include/storage/itemid.h). + Changing BLCKSZ requires an initdb. */ +#define BLCKSZ 8192 + +/* Define to the default TCP port number on which the server listens and to + which clients will try to connect. This can be overridden at run-time, but + it's convenient if your clients have the right default compiled in. + (--with-pgport=PORTNUM) */ +#define DEF_PGPORT 5432 + +/* Define to the default TCP port number as a string constant. */ +#define DEF_PGPORT_STR "5432" + +/* Define to build with GSSAPI support. (--with-gssapi) */ +//#define ENABLE_GSS 0 + +/* Define to 1 if you want National Language Support. (--enable-nls) */ +/* #undef ENABLE_NLS */ + +/* Define to 1 to build client libraries as thread-safe code. + (--enable-thread-safety) */ +#define ENABLE_THREAD_SAFETY 1 + +/* Define to nothing if C supports flexible array members, and to 1 if it does + not. That way, with a declaration like `struct s { int n; double + d[FLEXIBLE_ARRAY_MEMBER]; };', the struct hack can be used with pre-C99 + compilers. When computing the size of such an object, don't use 'sizeof + (struct s)' as it overestimates the size. Use 'offsetof (struct s, d)' + instead. Don't use 'offsetof (struct s, d[0])', as this doesn't work with + MSVC and with C++ compilers. */ +#define FLEXIBLE_ARRAY_MEMBER /**/ + +/* float4 values are passed by value if 'true', by reference if 'false' */ +#define FLOAT4PASSBYVAL true + +/* float8, int8, and related values are passed by value if 'true', by + reference if 'false' */ +#define FLOAT8PASSBYVAL false + +/* Define to 1 if gettimeofday() takes only 1 argument. */ +/* #undef GETTIMEOFDAY_1ARG */ + +#ifdef GETTIMEOFDAY_1ARG +# define gettimeofday(a,b) gettimeofday(a) +#endif + +/* Define to 1 if you have the `append_history' function. */ +/* #undef HAVE_APPEND_HISTORY */ + +/* Define to 1 if you want to use atomics if available. */ +#define HAVE_ATOMICS 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_ATOMIC_H */ + +/* Define to 1 if you have the `cbrt' function. */ +#define HAVE_CBRT 1 + +/* Define to 1 if you have the `class' function. */ +/* #undef HAVE_CLASS */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_CRTDEFS_H */ + +/* Define to 1 if you have the `crypt' function. */ +#define HAVE_CRYPT 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_CRYPT_H 1 + +/* Define to 1 if you have the declaration of `fdatasync', and to 0 if you + don't. */ +#define HAVE_DECL_FDATASYNC 1 + +/* Define to 1 if you have the declaration of `F_FULLFSYNC', and to 0 if you + don't. */ +#define HAVE_DECL_F_FULLFSYNC 0 + +/* Define to 1 if you have the declaration of `posix_fadvise', and to 0 if you + don't. */ +#define HAVE_DECL_POSIX_FADVISE 1 + +/* Define to 1 if you have the declaration of `snprintf', and to 0 if you + don't. */ +#define HAVE_DECL_SNPRINTF 1 + +/* Define to 1 if you have the declaration of `strlcat', and to 0 if you + don't. */ +#if OS_DARWIN +#define HAVE_DECL_STRLCAT 1 +#endif + +/* Define to 1 if you have the declaration of `strlcpy', and to 0 if you + don't. */ +#if OS_DARWIN +#define HAVE_DECL_STRLCPY 1 +#endif + +/* Define to 1 if you have the declaration of `sys_siglist', and to 0 if you + don't. */ +#define HAVE_DECL_SYS_SIGLIST 1 + +/* Define to 1 if you have the declaration of `vsnprintf', and to 0 if you + don't. */ +#define HAVE_DECL_VSNPRINTF 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_DLD_H */ + +/* Define to 1 if you have the `dlopen' function. */ +#define HAVE_DLOPEN 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_EDITLINE_HISTORY_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_EDITLINE_READLINE_H 1 + +/* Define to 1 if you have the `fdatasync' function. */ +#define HAVE_FDATASYNC 1 + +/* Define to 1 if you have the `fls' function. */ +/* #undef HAVE_FLS */ + +/* Define to 1 if you have the `fpclass' function. */ +/* #undef HAVE_FPCLASS */ + +/* Define to 1 if you have the `fp_class' function. */ +/* #undef HAVE_FP_CLASS */ + +/* Define to 1 if you have the `fp_class_d' function. */ +/* #undef HAVE_FP_CLASS_D */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_FP_CLASS_H */ + +/* Define to 1 if fseeko (and presumably ftello) exists and is declared. */ +#define HAVE_FSEEKO 1 + +/* Define to 1 if your compiler understands __func__. */ +#define HAVE_FUNCNAME__FUNC 1 + +/* Define to 1 if your compiler understands __FUNCTION__. */ +/* #undef HAVE_FUNCNAME__FUNCTION */ + +/* Define to 1 if you have __atomic_compare_exchange_n(int *, int *, int). */ +/* #undef HAVE_GCC__ATOMIC_INT32_CAS */ + +/* Define to 1 if you have __atomic_compare_exchange_n(int64 *, int *, int64). + */ +/* #undef HAVE_GCC__ATOMIC_INT64_CAS */ + +/* Define to 1 if you have __sync_lock_test_and_set(char *) and friends. */ +#define HAVE_GCC__SYNC_CHAR_TAS 1 + +/* Define to 1 if you have __sync_compare_and_swap(int *, int, int). */ +/* #undef HAVE_GCC__SYNC_INT32_CAS */ + +/* Define to 1 if you have __sync_lock_test_and_set(int *) and friends. */ +#define HAVE_GCC__SYNC_INT32_TAS 1 + +/* Define to 1 if you have __sync_compare_and_swap(int64 *, int64, int64). */ +/* #undef HAVE_GCC__SYNC_INT64_CAS */ + +/* Define to 1 if you have the `getaddrinfo' function. */ +#define HAVE_GETADDRINFO 1 + +/* Define to 1 if you have the `gethostbyname_r' function. */ +#define HAVE_GETHOSTBYNAME_R 1 + +/* Define to 1 if you have the `getifaddrs' function. */ +#define HAVE_GETIFADDRS 1 + +/* Define to 1 if you have the `getopt' function. */ +#define HAVE_GETOPT 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_GETOPT_H 1 + +/* Define to 1 if you have the `getopt_long' function. */ +#define HAVE_GETOPT_LONG 1 + +/* Define to 1 if you have the `getpeereid' function. */ +/* #undef HAVE_GETPEEREID */ + +/* Define to 1 if you have the `getpeerucred' function. */ +/* #undef HAVE_GETPEERUCRED */ + +/* Define to 1 if you have the `getpwuid_r' function. */ +#define HAVE_GETPWUID_R 1 + +/* Define to 1 if you have the `getrlimit' function. */ +#define HAVE_GETRLIMIT 1 + +/* Define to 1 if you have the `getrusage' function. */ +#define HAVE_GETRUSAGE 1 + +/* Define to 1 if you have the `gettimeofday' function. */ +/* #undef HAVE_GETTIMEOFDAY */ + +/* Define to 1 if you have the header file. */ +//#define HAVE_GSSAPI_GSSAPI_H 0 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_GSSAPI_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_HISTORY_H */ + +/* Define to 1 if you have the `history_truncate_file' function. */ +#define HAVE_HISTORY_TRUNCATE_FILE 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_IEEEFP_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_IFADDRS_H 1 + +/* Define to 1 if you have the `inet_aton' function. */ +#define HAVE_INET_ATON 1 + +/* Define to 1 if the system has the type `int64'. */ +/* #undef HAVE_INT64 */ + +/* Define to 1 if the system has the type `int8'. */ +/* #undef HAVE_INT8 */ + +/* Define to 1 if the system has the type `intptr_t'. */ +#define HAVE_INTPTR_T 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_INTTYPES_H 1 + +/* Define to 1 if you have the global variable 'int opterr'. */ +#define HAVE_INT_OPTERR 1 + +/* Define to 1 if you have the global variable 'int optreset'. */ +/* #undef HAVE_INT_OPTRESET */ + +/* Define to 1 if you have the global variable 'int timezone'. */ +#define HAVE_INT_TIMEZONE 1 + +/* Define to 1 if you have support for IPv6. */ +#define HAVE_IPV6 1 + +/* Define to 1 if you have isinf(). */ +#define HAVE_ISINF 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_LANGINFO_H 1 + +/* Define to 1 if you have the header file. */ +//#define HAVE_LDAP_H 0 + +/* Define to 1 if you have the `crypto' library (-lcrypto). */ +#define HAVE_LIBCRYPTO 1 + +/* Define to 1 if you have the `ldap' library (-lldap). */ +//#define HAVE_LIBLDAP 0 + +/* Define to 1 if you have the `ldap_r' library (-lldap_r). */ +#define HAVE_LIBLDAP_R 0 + +/* Define to 1 if you have the `m' library (-lm). */ +#define HAVE_LIBM 1 + +/* Define to 1 if you have the `pam' library (-lpam). */ +#define HAVE_LIBPAM 1 + +/* Define if you have a function readline library */ +#define HAVE_LIBREADLINE 1 + +/* Define to 1 if you have the `selinux' library (-lselinux). */ +/* #undef HAVE_LIBSELINUX */ + +/* Define to 1 if you have the `ssl' library (-lssl). */ +#define HAVE_LIBSSL 0 + +/* Define to 1 if you have the `wldap32' library (-lwldap32). */ +/* #undef HAVE_LIBWLDAP32 */ + +/* Define to 1 if you have the `xml2' library (-lxml2). */ +#define HAVE_LIBXML2 1 + +/* Define to 1 if you have the `xslt' library (-lxslt). */ +#define HAVE_LIBXSLT 1 + +/* Define to 1 if you have the `z' library (-lz). */ +#define HAVE_LIBZ 1 + +/* Define to 1 if constants of type 'long long int' should have the suffix LL. + */ +#define HAVE_LL_CONSTANTS 1 + +/* Define to 1 if the system has the type `locale_t'. */ +#define HAVE_LOCALE_T 1 + +/* Define to 1 if `long int' works and is 64 bits. */ +/* #undef HAVE_LONG_INT_64 */ + +/* Define to 1 if the system has the type `long long int'. */ +#define HAVE_LONG_LONG_INT 1 + +/* Define to 1 if `long long int' works and is 64 bits. */ +#define HAVE_LONG_LONG_INT_64 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_MBARRIER_H */ + +/* Define to 1 if you have the `mbstowcs_l' function. */ +/* #undef HAVE_MBSTOWCS_L */ + +/* Define to 1 if you have the `memmove' function. */ +#define HAVE_MEMMOVE 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_MEMORY_H 1 + +/* Define to 1 if the system has the type `MINIDUMP_TYPE'. */ +/* #undef HAVE_MINIDUMP_TYPE */ + +/* Define to 1 if you have the `mkdtemp' function. */ +#define HAVE_MKDTEMP 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_NETINET_IN_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_NETINET_TCP_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_NET_IF_H 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_OSSP_UUID_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_PAM_PAM_APPL_H */ + +/* Define to 1 if you have the `poll' function. */ +#define HAVE_POLL 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_POLL_H 1 + +/* Define to 1 if you have the `posix_fadvise' function. */ +#define HAVE_POSIX_FADVISE 1 + +/* Define to 1 if you have the POSIX signal interface. */ +#define HAVE_POSIX_SIGNALS 1 + +/* Define to 1 if the assembler supports PPC's LWARX mutex hint bit. */ +/* #undef HAVE_PPC_LWARX_MUTEX_HINT */ + +/* Define to 1 if you have the `pstat' function. */ +/* #undef HAVE_PSTAT */ + +/* Define to 1 if the PS_STRINGS thing exists. */ +/* #undef HAVE_PS_STRINGS */ + +/* Define to 1 if you have the `pthread_is_threaded_np' function. */ +/* #undef HAVE_PTHREAD_IS_THREADED_NP */ + +/* Define to 1 if you have the header file. */ +#define HAVE_PWD_H 1 + +/* Define to 1 if you have the `random' function. */ +#define HAVE_RANDOM 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_READLINE_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_READLINE_HISTORY_H 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_READLINE_READLINE_H */ + +/* Define to 1 if you have the `readlink' function. */ +#define HAVE_READLINK 1 + +/* Define to 1 if you have the `rint' function. */ +#define HAVE_RINT 1 + +/* Define to 1 if you have the global variable + 'rl_completion_append_character'. */ +/* #undef HAVE_RL_COMPLETION_APPEND_CHARACTER */ + +/* Define to 1 if you have the `rl_completion_matches' function. */ +#define HAVE_RL_COMPLETION_MATCHES 1 + +/* Define to 1 if you have the `rl_filename_completion_function' function. */ +#define HAVE_RL_FILENAME_COMPLETION_FUNCTION 1 + +/* Define to 1 if you have the `rl_reset_screen_size' function. */ +/* #undef HAVE_RL_RESET_SCREEN_SIZE */ + +/* Define to 1 if you have the header file. */ +#define HAVE_SECURITY_PAM_APPL_H 1 + +/* Define to 1 if you have the `setproctitle' function. */ +/* #undef HAVE_SETPROCTITLE */ + +/* Define to 1 if you have the `setsid' function. */ +#define HAVE_SETSID 1 + +/* Define to 1 if you have the `shm_open' function. */ +#define HAVE_SHM_OPEN 1 + +/* Define to 1 if you have the `sigprocmask' function. */ +#define HAVE_SIGPROCMASK 1 + +/* Define to 1 if you have sigsetjmp(). */ +#define HAVE_SIGSETJMP 1 + +/* Define to 1 if the system has the type `sig_atomic_t'. */ +#define HAVE_SIG_ATOMIC_T 1 + +/* Define to 1 if you have the `snprintf' function. */ +#define HAVE_SNPRINTF 1 + +/* Define to 1 if you have spinlocks. */ +#define HAVE_SPINLOCKS 1 + +/* Define to 1 if you have the `srandom' function. */ +#define HAVE_SRANDOM 1 + +/* Define to 1 if you have the `SSL_get_current_compression' function. */ +#define HAVE_SSL_GET_CURRENT_COMPRESSION 0 + +/* Define to 1 if you have the header file. */ +#define HAVE_STDINT_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_STDLIB_H 1 + +/* Define to 1 if you have the `strerror' function. */ +#define HAVE_STRERROR 1 + +/* Define to 1 if you have the `strerror_r' function. */ +#define HAVE_STRERROR_R 1 + +/* Define to 1 if you have the header file. */ +//#define HAVE_STRINGS_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_STRING_H 1 + +/* Define to 1 if you have the `strlcat' function. */ +/* #undef HAVE_STRLCAT */ + +/* Define to 1 if you have the `strlcpy' function. */ +/* #undef HAVE_STRLCPY */ + +/* Define to 1 if you have the `strtoll' function. */ +#define HAVE_STRTOLL 1 + +#if (!OS_DARWIN) +#define HAVE_STRCHRNUL 1 +#endif + +/* Define to 1 if you have the `strtoq' function. */ +/* #undef HAVE_STRTOQ */ + +/* Define to 1 if you have the `strtoull' function. */ +#define HAVE_STRTOULL 1 + +/* Define to 1 if you have the `strtouq' function. */ +/* #undef HAVE_STRTOUQ */ + +/* Define to 1 if the system has the type `struct addrinfo'. */ +#define HAVE_STRUCT_ADDRINFO 1 + +/* Define to 1 if the system has the type `struct cmsgcred'. */ +/* #undef HAVE_STRUCT_CMSGCRED */ + +/* Define to 1 if the system has the type `struct option'. */ +#define HAVE_STRUCT_OPTION 1 + +/* Define to 1 if `sa_len' is a member of `struct sockaddr'. */ +/* #undef HAVE_STRUCT_SOCKADDR_SA_LEN */ + +/* Define to 1 if the system has the type `struct sockaddr_storage'. */ +#define HAVE_STRUCT_SOCKADDR_STORAGE 1 + +/* Define to 1 if `ss_family' is a member of `struct sockaddr_storage'. */ +#define HAVE_STRUCT_SOCKADDR_STORAGE_SS_FAMILY 1 + +/* Define to 1 if `ss_len' is a member of `struct sockaddr_storage'. */ +/* #undef HAVE_STRUCT_SOCKADDR_STORAGE_SS_LEN */ + +/* Define to 1 if `__ss_family' is a member of `struct sockaddr_storage'. */ +/* #undef HAVE_STRUCT_SOCKADDR_STORAGE___SS_FAMILY */ + +/* Define to 1 if `__ss_len' is a member of `struct sockaddr_storage'. */ +/* #undef HAVE_STRUCT_SOCKADDR_STORAGE___SS_LEN */ + +/* Define to 1 if `tm_zone' is a member of `struct tm'. */ +#define HAVE_STRUCT_TM_TM_ZONE 1 + +/* Define to 1 if you have the `symlink' function. */ +#define HAVE_SYMLINK 1 + +/* Define to 1 if you have the `sync_file_range' function. */ +/* #undef HAVE_SYNC_FILE_RANGE */ + +/* Define to 1 if you have the syslog interface. */ +#define HAVE_SYSLOG 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_IOCTL_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_IPC_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_POLL_H 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SYS_PSTAT_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_RESOURCE_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_SELECT_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_SEM_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_SHM_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_SOCKET_H 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SYS_SOCKIO_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_STAT_H 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SYS_TAS_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_TIME_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_TYPES_H 1 + +/* Define to 1 if you have the header file. */ +#if (OS_DARWIN || OS_FREEBSD) +#define HAVE_SYS_UCRED_H 1 +#endif + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_UN_H 1 +#define _GNU_SOURCE 1 /* Needed for glibc struct ucred */ + +/* Define to 1 if you have the header file. */ +#define HAVE_TERMIOS_H 1 + +/* Define to 1 if your `struct tm' has `tm_zone'. Deprecated, use + `HAVE_STRUCT_TM_TM_ZONE' instead. */ +#define HAVE_TM_ZONE 1 + +/* Define to 1 if you have the `towlower' function. */ +#define HAVE_TOWLOWER 1 + +/* Define to 1 if you have the external array `tzname'. */ +#define HAVE_TZNAME 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_UCRED_H */ + +/* Define to 1 if the system has the type `uint64'. */ +/* #undef HAVE_UINT64 */ + +/* Define to 1 if the system has the type `uint8'. */ +/* #undef HAVE_UINT8 */ + +/* Define to 1 if the system has the type `uintptr_t'. */ +#define HAVE_UINTPTR_T 1 + +/* Define to 1 if the system has the type `union semun'. */ +/* #undef HAVE_UNION_SEMUN */ + +/* Define to 1 if you have the header file. */ +#define HAVE_UNISTD_H 1 + +/* Define to 1 if you have unix sockets. */ +#define HAVE_UNIX_SOCKETS 1 + +/* Define to 1 if you have the `unsetenv' function. */ +#define HAVE_UNSETENV 1 + +/* Define to 1 if the system has the type `unsigned long long int'. */ +#define HAVE_UNSIGNED_LONG_LONG_INT 1 + +/* Define to 1 if you have the `utime' function. */ +#define HAVE_UTIME 1 + +/* Define to 1 if you have the `utimes' function. */ +#define HAVE_UTIMES 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_UTIME_H 1 + +/* Define to 1 if you have BSD UUID support. */ +/* #undef HAVE_UUID_BSD */ + +/* Define to 1 if you have E2FS UUID support. */ +/* #undef HAVE_UUID_E2FS */ + +/* Define to 1 if you have the header file. */ +#define HAVE_UUID_H 1 + +/* Define to 1 if you have OSSP UUID support. */ +#define HAVE_UUID_OSSP 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_UUID_UUID_H */ + +/* Define to 1 if you have the `vsnprintf' function. */ +#define HAVE_VSNPRINTF 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_WCHAR_H 1 + +/* Define to 1 if you have the `wcstombs' function. */ +#define HAVE_WCSTOMBS 1 + +/* Define to 1 if you have the `wcstombs_l' function. */ +/* #undef HAVE_WCSTOMBS_L */ + +/* Define to 1 if you have the header file. */ +#define HAVE_WCTYPE_H 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_WINLDAP_H */ + +/* Define to 1 if your compiler understands __builtin_bswap32. */ +/* #undef HAVE__BUILTIN_BSWAP32 */ + +/* Define to 1 if your compiler understands __builtin_constant_p. */ +#define HAVE__BUILTIN_CONSTANT_P 1 + +/* Define to 1 if your compiler understands __builtin_types_compatible_p. */ +#define HAVE__BUILTIN_TYPES_COMPATIBLE_P 1 + +/* Define to 1 if your compiler understands __builtin_unreachable. */ +/* #undef HAVE__BUILTIN_UNREACHABLE */ + +/* Define to 1 if you have __cpuid. */ +/* #undef HAVE__CPUID */ + +/* Define to 1 if you have __get_cpuid. */ +/* #undef HAVE__GET_CPUID */ + +/* Define to 1 if your compiler understands _Static_assert. */ +/* #undef HAVE__STATIC_ASSERT */ + +/* Define to 1 if your compiler understands __VA_ARGS__ in macros. */ +#define HAVE__VA_ARGS 1 + +/* Define to the appropriate snprintf length modifier for 64-bit ints. */ +#define INT64_MODIFIER "ll" + +/* Define to 1 if `locale_t' requires . */ +/* #undef LOCALE_T_IN_XLOCALE */ + +/* Define as the maximum alignment requirement of any C data type. */ +#define MAXIMUM_ALIGNOF 4 + +/* Define bytes to use libc memset(). */ +#define MEMSET_LOOP_LIMIT 1024 + +/* Define to the address where bug reports for this package should be sent. */ +#define PACKAGE_BUGREPORT "pgsql-bugs@postgresql.org" + +/* Define to the full name of this package. */ +#define PACKAGE_NAME "PostgreSQL" + +/* Define to the full name and version of this package. */ +#define PACKAGE_STRING "PostgreSQL 9.5.4" + +/* Define to the one symbol short name of this package. */ +#define PACKAGE_TARNAME "postgresql" + +/* Define to the home page for this package. */ +#define PACKAGE_URL "" + +/* Define to the version of this package. */ +#define PACKAGE_VERSION "9.5.4" + +/* Define to the name of a signed 128-bit integer type. */ +/* #undef PG_INT128_TYPE */ + +/* Define to the name of a signed 64-bit integer type. */ +#define PG_INT64_TYPE long long int + +/* Define to the name of the default PostgreSQL service principal in Kerberos + (GSSAPI). (--with-krb-srvnam=NAME) */ +#define PG_KRB_SRVNAM "postgres" + +/* PostgreSQL major version as a string */ +#define PG_MAJORVERSION "9.5" + +/* Define to gnu_printf if compiler supports it, else printf. */ +#define PG_PRINTF_ATTRIBUTE printf + +/* Define to 1 if "static inline" works without unwanted warnings from + compilations where static inline functions are defined but not called. */ +#define PG_USE_INLINE 1 + +/* PostgreSQL version as a string */ +#define PG_VERSION "9.5.4" + +/* PostgreSQL version as a number */ +#define PG_VERSION_NUM 90504 + +/* A string containing the version number, platform, and C compiler */ +#define PG_VERSION_STR "PostgreSQL 9.5.4 on i686-pc-linux-gnu, compiled by gcc (GCC) 4.1.2 20080704 (Red Hat 4.1.2-55), 32-bit" + +/* Define to 1 to allow profiling output to be saved separately for each + process. */ +/* #undef PROFILE_PID_DIR */ + +/* RELSEG_SIZE is the maximum number of blocks allowed in one disk file. Thus, + the maximum size of a single file is RELSEG_SIZE * BLCKSZ; relations bigger + than that are divided into multiple files. RELSEG_SIZE * BLCKSZ must be + less than your OS' limit on file size. This is often 2 GB or 4GB in a + 32-bit operating system, unless you have large file support enabled. By + default, we make the limit 1 GB to avoid any possible integer-overflow + problems within the OS. A limit smaller than necessary only means we divide + a large relation into more chunks than necessary, so it seems best to err + in the direction of a small limit. A power-of-2 value is recommended to + save a few cycles in md.c, but is not absolutely required. Changing + RELSEG_SIZE requires an initdb. */ +#define RELSEG_SIZE 131072 + +/* The size of `long', as computed by sizeof. */ +#define SIZEOF_LONG 4 + +/* The size of `off_t', as computed by sizeof. */ +#define SIZEOF_OFF_T 8 + +/* The size of `size_t', as computed by sizeof. */ +#define SIZEOF_SIZE_T 4 + +/* The size of `void *', as computed by sizeof. */ +#define SIZEOF_VOID_P 4 + +/* Define to 1 if you have the ANSI C header files. */ +#define STDC_HEADERS 1 + +/* Define to 1 if strerror_r() returns a int. */ +/* #undef STRERROR_R_INT */ + +/* Define to 1 if your declares `struct tm'. */ +/* #undef TM_IN_SYS_TIME */ + +/* Define to 1 to build with assertion checks. (--enable-cassert) */ +/* #undef USE_ASSERT_CHECKING */ + +/* Define to 1 to build with Bonjour support. (--with-bonjour) */ +/* #undef USE_BONJOUR */ + +/* Define to 1 if you want float4 values to be passed by value. + (--enable-float4-byval) */ +#define USE_FLOAT4_BYVAL 1 + +/* Define to 1 if you want float8, int8, etc values to be passed by value. + (--enable-float8-byval) */ +/* #undef USE_FLOAT8_BYVAL */ + +/* Define to 1 if you want 64-bit integer timestamp and interval support. + (--enable-integer-datetimes) */ +#define USE_INTEGER_DATETIMES 1 + +/* Define to 1 to build with LDAP support. (--with-ldap) */ +//#define USE_LDAP 0 + +/* Define to 1 to build with XML support. (--with-libxml) */ +#define USE_LIBXML 1 + +/* Define to 1 to use XSLT support when building contrib/xml2. + (--with-libxslt) */ +#define USE_LIBXSLT 1 + +/* Define to select named POSIX semaphores. */ +/* #undef USE_NAMED_POSIX_SEMAPHORES */ + +/* Define to build with OpenSSL support. (--with-openssl) */ +#define USE_OPENSSL 0 + +#define USE_OPENSSL_RANDOM 0 + +#define HAVE_BIO_GET_DATA 0 + +#define FRONTEND 1 + +/* Define to 1 to build with PAM support. (--with-pam) */ +#define USE_PAM 1 + +/* Use replacement snprintf() functions. */ +/* #undef USE_REPL_SNPRINTF */ + +/* Define to 1 to use Intel SSE 4.2 CRC instructions with a runtime check. */ +#define USE_SLICING_BY_8_CRC32C 1 + +/* Define to 1 use Intel SSE 4.2 CRC instructions. */ +/* #undef USE_SSE42_CRC32C */ + +/* Define to 1 to use Intel SSSE 4.2 CRC instructions with a runtime check. */ +/* #undef USE_SSE42_CRC32C_WITH_RUNTIME_CHECK */ + +/* Define to select SysV-style semaphores. */ +#define USE_SYSV_SEMAPHORES 1 + +/* Define to select SysV-style shared memory. */ +#define USE_SYSV_SHARED_MEMORY 1 + +/* Define to select unnamed POSIX semaphores. */ +/* #undef USE_UNNAMED_POSIX_SEMAPHORES */ + +/* Define to select Win32-style semaphores. */ +/* #undef USE_WIN32_SEMAPHORES */ + +/* Define to select Win32-style shared memory. */ +/* #undef USE_WIN32_SHARED_MEMORY */ + +/* Define to 1 if `wcstombs_l' requires . */ +/* #undef WCSTOMBS_L_IN_XLOCALE */ + +/* Define WORDS_BIGENDIAN to 1 if your processor stores words with the most + significant byte first (like Motorola and SPARC, unlike Intel). */ +#if defined AC_APPLE_UNIVERSAL_BUILD +# if defined __BIG_ENDIAN__ +# define WORDS_BIGENDIAN 1 +# endif +#else +# ifndef WORDS_BIGENDIAN +/* # undef WORDS_BIGENDIAN */ +# endif +#endif + +/* Size of a WAL file block. This need have no particular relation to BLCKSZ. + XLOG_BLCKSZ must be a power of 2, and if your system supports O_DIRECT I/O, + XLOG_BLCKSZ must be a multiple of the alignment requirement for direct-I/O + buffers, else direct I/O may fail. Changing XLOG_BLCKSZ requires an initdb. + */ +#define XLOG_BLCKSZ 8192 + +/* XLOG_SEG_SIZE is the size of a single WAL file. This must be a power of 2 + and larger than XLOG_BLCKSZ (preferably, a great deal larger than + XLOG_BLCKSZ). Changing XLOG_SEG_SIZE requires an initdb. */ +#define XLOG_SEG_SIZE (16 * 1024 * 1024) + + + +/* Number of bits in a file offset, on hosts where this is settable. */ +#define _FILE_OFFSET_BITS 64 + +/* Define to 1 to make fseeko visible on some hosts (e.g. glibc 2.2). */ +/* #undef _LARGEFILE_SOURCE */ + +/* Define for large files, on AIX-style hosts. */ +/* #undef _LARGE_FILES */ + +/* Define to `__inline__' or `__inline' if that's what the C compiler + calls it, or to nothing if 'inline' is not supported under any name. */ +#ifndef __cplusplus +/* #undef inline */ +#endif + +/* Define to the type of a signed integer type wide enough to hold a pointer, + if such a type exists, and if the system does not define it. */ +/* #undef intptr_t */ + +/* Define to empty if the C compiler does not understand signed types. */ +/* #undef signed */ + +/* Define to the type of an unsigned integer type wide enough to hold a + pointer, if such a type exists, and if the system does not define it. */ +/* #undef uintptr_t */ diff --git a/contrib/postgres-cmake/pg_config_ext.h b/contrib/postgres-cmake/pg_config_ext.h new file mode 100644 index 00000000000..c5401355766 --- /dev/null +++ b/contrib/postgres-cmake/pg_config_ext.h @@ -0,0 +1,7 @@ +/* + * * src/include/pg_config_ext.h.in. This is generated manually, not by + * * autoheader, since we want to limit which symbols get defined here. + * */ + +/* Define to the name of a signed 64-bit integer type. */ +#define PG_INT64_TYPE long long int diff --git a/contrib/postgres-cmake/pg_config_os.h b/contrib/postgres-cmake/pg_config_os.h new file mode 100644 index 00000000000..ecfed0d7b19 --- /dev/null +++ b/contrib/postgres-cmake/pg_config_os.h @@ -0,0 +1,34 @@ +#if defined(OS_DARWIN) + +/* src/include/port/darwin.h */ +#define __darwin__ 1 + +#if HAVE_DECL_F_FULLFSYNC /* not present before macOS 10.3 */ +#define HAVE_FSYNC_WRITETHROUGH +#endif + +#else +/* src/include/port/linux.h */ +/* + * As of July 2007, all known versions of the Linux kernel will sometimes + * return EIDRM for a shmctl() operation when EINVAL is correct (it happens + * when the low-order 15 bits of the supplied shm ID match the slot number + * assigned to a newer shmem segment). We deal with this by assuming that + * EIDRM means EINVAL in PGSharedMemoryIsInUse(). This is reasonably safe + * since in fact Linux has no excuse for ever returning EIDRM; it doesn't + * track removed segments in a way that would allow distinguishing them from + * private ones. But someday that code might get upgraded, and we'd have + * to have a kernel version test here. + */ +#define HAVE_LINUX_EIDRM_BUG + +/* + * Set the default wal_sync_method to fdatasync. With recent Linux versions, + * xlogdefs.h's normal rules will prefer open_datasync, which (a) doesn't + * perform better and (b) causes outright failures on ext4 data=journal + * filesystems, because those don't support O_DIRECT. + */ +#define PLATFORM_DEFAULT_SYNC_METHOD SYNC_METHOD_FDATASYNC + +#endif + diff --git a/contrib/postgres-cmake/pg_config_paths.h b/contrib/postgres-cmake/pg_config_paths.h new file mode 100644 index 00000000000..b366e3994b3 --- /dev/null +++ b/contrib/postgres-cmake/pg_config_paths.h @@ -0,0 +1,12 @@ +#define PGBINDIR "/bin" +#define PGSHAREDIR "/share" +#define SYSCONFDIR "/etc" +#define INCLUDEDIR "/include" +#define PKGINCLUDEDIR "/include" +#define INCLUDEDIRSERVER "/include/server" +#define LIBDIR "/lib" +#define PKGLIBDIR "/lib" +#define LOCALEDIR "/share/locale" +#define DOCDIR "/doc" +#define HTMLDIR "/doc" +#define MANDIR "/man" diff --git a/contrib/postgres-cmake/utils/errcodes.h b/contrib/postgres-cmake/utils/errcodes.h new file mode 100644 index 00000000000..e69de29bb2d From 2e82e06330502b7c64a9bf5812f66a6147e79539 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 12 Sep 2024 16:59:25 +0000 Subject: [PATCH 053/103] Update tests --- tests/queries/0_stateless/00000_test.reference | 2 ++ tests/queries/0_stateless/00000_test.sh | 12 ++++++++++++ tests/queries/0_stateless/00000_test.sql | 14 ++++++++++++++ .../02156_storage_merge_prewhere.reference | 14 +++++++------- .../02842_move_pk_to_end_of_prewhere.reference | 10 +++++----- ...tics_delayed_materialization_in_merge.reference | 4 ++-- .../0_stateless/02864_statistics_usage.reference | 8 ++++---- 7 files changed, 46 insertions(+), 18 deletions(-) create mode 100644 tests/queries/0_stateless/00000_test.reference create mode 100755 tests/queries/0_stateless/00000_test.sh create mode 100644 tests/queries/0_stateless/00000_test.sql diff --git a/tests/queries/0_stateless/00000_test.reference b/tests/queries/0_stateless/00000_test.reference new file mode 100644 index 00000000000..676c099a485 --- /dev/null +++ b/tests/queries/0_stateless/00000_test.reference @@ -0,0 +1,2 @@ +42 +select 42 diff --git a/tests/queries/0_stateless/00000_test.sh b/tests/queries/0_stateless/00000_test.sh new file mode 100755 index 00000000000..ae20733a597 --- /dev/null +++ b/tests/queries/0_stateless/00000_test.sh @@ -0,0 +1,12 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +$CLICKHOUSE_CLIENT --query_id="my_id_$CLICKHOUSE_TEST_UNIQUE_NAME" -q "select 42"; +$CLICKHOUSE_CLIENT -q "system flush logs"; +$CLICKHOUSE_CLIENT -q "select query from system.query_log where query_id='my_id_$CLICKHOUSE_TEST_UNIQUE_NAME' and type='QueryFinish'" + diff --git a/tests/queries/0_stateless/00000_test.sql b/tests/queries/0_stateless/00000_test.sql new file mode 100644 index 00000000000..266af42f3db --- /dev/null +++ b/tests/queries/0_stateless/00000_test.sql @@ -0,0 +1,14 @@ +-- Tags: long, no-tsan, no-msan, no-ubsan, no-asan + +set allow_experimental_dynamic_type = 1; +set merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 1; + +drop table if exists test; + +create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, use_adaptive_write_buffer_for_dynamic_subcolumns=1, min_bytes_for_full_part_storage=100000000000; + +insert into test select number, if (number % 5 == 1, ('str_' || number)::LowCardinality(String)::Dynamic, number::Dynamic) from numbers(100000) settings min_insert_block_size_rows=50000; + +select count() from test where dynamicType(d) == 'UInt64'; + +drop table test; diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere.reference b/tests/queries/0_stateless/02156_storage_merge_prewhere.reference index 876cee60baa..8aa1f0b59d3 100644 --- a/tests/queries/0_stateless/02156_storage_merge_prewhere.reference +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere.reference @@ -1,25 +1,25 @@ Prewhere info Prewhere filter - Prewhere filter column: and(notEmpty(v), equals(k, 3)) (removed) + Prewhere filter column: and(equals(k, 3), notEmpty(v)) (removed) Prewhere info Prewhere filter - Prewhere filter column: and(notEmpty(v), equals(k, 3)) (removed) + Prewhere filter column: and(equals(k, 3), notEmpty(v)) (removed) Prewhere info Prewhere filter - Prewhere filter column: and(notEmpty(v), equals(k, 3)) (removed) + Prewhere filter column: and(equals(k, 3), notEmpty(v)) (removed) Prewhere info Prewhere filter - Prewhere filter column: and(notEmpty(v), equals(k, 3)) (removed) + Prewhere filter column: and(equals(k, 3), notEmpty(v)) (removed) 2 Filter column: and(equals(k, 3), notEmpty(v)) (removed) Prewhere info Prewhere filter - Prewhere filter column: and(notEmpty(v), equals(k, 3)) (removed) + Prewhere filter column: and(equals(k, 3), notEmpty(v)) (removed) 2 Prewhere info Prewhere filter - Prewhere filter column: and(notEmpty(v), equals(k, 3)) (removed) + Prewhere filter column: and(equals(k, 3), notEmpty(v)) (removed) Prewhere info Prewhere filter - Prewhere filter column: and(notEmpty(v), equals(k, 3)) (removed) + Prewhere filter column: and(equals(k, 3), notEmpty(v)) (removed) 2 diff --git a/tests/queries/0_stateless/02842_move_pk_to_end_of_prewhere.reference b/tests/queries/0_stateless/02842_move_pk_to_end_of_prewhere.reference index b91a4dd2f68..254e59d479a 100644 --- a/tests/queries/0_stateless/02842_move_pk_to_end_of_prewhere.reference +++ b/tests/queries/0_stateless/02842_move_pk_to_end_of_prewhere.reference @@ -1,15 +1,15 @@ Prewhere filter - Prewhere filter column: and(notEmpty(v), equals(k, 3)) (removed) + Prewhere filter column: and(equals(k, 3), notEmpty(v)) (removed) 1 Prewhere filter - Prewhere filter column: and(like(d, \'%es%\'), less(c, 20), equals(b, \'3\'), equals(a, 3)) (removed) + Prewhere filter column: and(equals(a, 3), equals(b, \'3\'), less(c, 20), like(d, \'%es%\')) (removed) 1 Prewhere filter - Prewhere filter column: and(like(d, \'%es%\'), less(c, 20), greater(c, 0), equals(a, 3)) (removed) + Prewhere filter column: and(equals(a, 3), less(c, 20), greater(c, 0), like(d, \'%es%\')) (removed) 1 Prewhere filter - Prewhere filter column: and(like(d, \'%es%\'), equals(b, \'3\'), less(c, 20)) (removed) + Prewhere filter column: and(equals(b, \'3\'), less(c, 20), like(d, \'%es%\')) (removed) 1 Prewhere filter - Prewhere filter column: and(like(d, \'%es%\'), equals(b, \'3\'), equals(a, 3)) (removed) + Prewhere filter column: and(equals(a, 3), equals(b, \'3\'), like(d, \'%es%\')) (removed) 1 diff --git a/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference b/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference index eb5e685597c..c4ef127ebc0 100644 --- a/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference +++ b/tests/queries/0_stateless/02864_statistics_delayed_materialization_in_merge.reference @@ -5,8 +5,8 @@ After insert After merge Prewhere info Prewhere filter - Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) + Prewhere filter column: and(less(b, 10_UInt8), less(a, 10_UInt8)) (removed) After truncate, insert, and materialize Prewhere info Prewhere filter - Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) + Prewhere filter column: and(less(b, 10_UInt8), less(a, 10_UInt8)) (removed) diff --git a/tests/queries/0_stateless/02864_statistics_usage.reference b/tests/queries/0_stateless/02864_statistics_usage.reference index a9f669b88c1..fd4181a59c3 100644 --- a/tests/queries/0_stateless/02864_statistics_usage.reference +++ b/tests/queries/0_stateless/02864_statistics_usage.reference @@ -1,7 +1,7 @@ After insert Prewhere info Prewhere filter - Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) + Prewhere filter column: and(less(b, 10_UInt8), less(a, 10_UInt8)) (removed) After drop statistic Prewhere info Prewhere filter @@ -9,12 +9,12 @@ After drop statistic After add and materialize statistic Prewhere info Prewhere filter - Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) + Prewhere filter column: and(less(b, 10_UInt8), less(a, 10_UInt8)) (removed) After merge Prewhere info Prewhere filter - Prewhere filter column: and(less(a, 10_UInt8), less(b, 10_UInt8)) (removed) + Prewhere filter column: and(less(b, 10_UInt8), less(a, 10_UInt8)) (removed) After rename Prewhere info Prewhere filter - Prewhere filter column: and(less(a, 10_UInt8), less(c, 10_UInt8)) (removed) + Prewhere filter column: and(less(c, 10_UInt8), less(a, 10_UInt8)) (removed) From 9c1f4f4545dae17824a7d772cb9d1fab11c4f0db Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 12 Sep 2024 17:21:28 +0000 Subject: [PATCH 054/103] Remove bad files --- tests/queries/0_stateless/00000_test.reference | 2 -- tests/queries/0_stateless/00000_test.sh | 12 ------------ tests/queries/0_stateless/00000_test.sql | 14 -------------- 3 files changed, 28 deletions(-) delete mode 100644 tests/queries/0_stateless/00000_test.reference delete mode 100755 tests/queries/0_stateless/00000_test.sh delete mode 100644 tests/queries/0_stateless/00000_test.sql diff --git a/tests/queries/0_stateless/00000_test.reference b/tests/queries/0_stateless/00000_test.reference deleted file mode 100644 index 676c099a485..00000000000 --- a/tests/queries/0_stateless/00000_test.reference +++ /dev/null @@ -1,2 +0,0 @@ -42 -select 42 diff --git a/tests/queries/0_stateless/00000_test.sh b/tests/queries/0_stateless/00000_test.sh deleted file mode 100755 index ae20733a597..00000000000 --- a/tests/queries/0_stateless/00000_test.sh +++ /dev/null @@ -1,12 +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 - - -$CLICKHOUSE_CLIENT --query_id="my_id_$CLICKHOUSE_TEST_UNIQUE_NAME" -q "select 42"; -$CLICKHOUSE_CLIENT -q "system flush logs"; -$CLICKHOUSE_CLIENT -q "select query from system.query_log where query_id='my_id_$CLICKHOUSE_TEST_UNIQUE_NAME' and type='QueryFinish'" - diff --git a/tests/queries/0_stateless/00000_test.sql b/tests/queries/0_stateless/00000_test.sql deleted file mode 100644 index 266af42f3db..00000000000 --- a/tests/queries/0_stateless/00000_test.sql +++ /dev/null @@ -1,14 +0,0 @@ --- Tags: long, no-tsan, no-msan, no-ubsan, no-asan - -set allow_experimental_dynamic_type = 1; -set merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 1; - -drop table if exists test; - -create table test (id UInt64, d Dynamic) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, use_adaptive_write_buffer_for_dynamic_subcolumns=1, min_bytes_for_full_part_storage=100000000000; - -insert into test select number, if (number % 5 == 1, ('str_' || number)::LowCardinality(String)::Dynamic, number::Dynamic) from numbers(100000) settings min_insert_block_size_rows=50000; - -select count() from test where dynamicType(d) == 'UInt64'; - -drop table test; From 8a89d7b2b956372e3c7a7e94ee599f1dfcae8aed Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 13 Sep 2024 10:46:38 +0800 Subject: [PATCH 055/103] allow empty inputs for arrayZip or arrayZipUnaligned --- src/Functions/array/arrayZip.cpp | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Functions/array/arrayZip.cpp b/src/Functions/array/arrayZip.cpp index 6e1cc0f7788..36ab56006b3 100644 --- a/src/Functions/array/arrayZip.cpp +++ b/src/Functions/array/arrayZip.cpp @@ -38,13 +38,6 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (arguments.empty()) - throw Exception( - ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, - "Function {} needs at least one argument; passed {}.", - getName(), - arguments.size()); - DataTypes arguments_types; for (size_t index = 0; index < arguments.size(); ++index) { @@ -68,9 +61,16 @@ public: } ColumnPtr - executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override + executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { size_t num_arguments = arguments.size(); + if (num_arguments == 0) + { + auto res_col = result_type->createColumn(); + res_col->insertDefault(); + return ColumnConst::create(std::move(res_col), input_rows_count); + } + Columns holders(num_arguments); Columns tuple_columns(num_arguments); From f9335a2fd5e4746a8b3fa9bb704640e1a4d3e61c Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 13 Sep 2024 10:50:50 +0800 Subject: [PATCH 056/103] update uts --- tests/queries/0_stateless/01045_array_zip.reference | 1 + tests/queries/0_stateless/01045_array_zip.sql | 2 +- tests/queries/0_stateless/03230_array_zip_unaligned.reference | 1 + tests/queries/0_stateless/03230_array_zip_unaligned.sql | 2 +- 4 files changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01045_array_zip.reference b/tests/queries/0_stateless/01045_array_zip.reference index 955ed98033e..85b53dd0c9f 100644 --- a/tests/queries/0_stateless/01045_array_zip.reference +++ b/tests/queries/0_stateless/01045_array_zip.reference @@ -1,2 +1,3 @@ [('a','d'),('b','e'),('c','f')] [('a','d','g'),('b','e','h'),('c','f','i')] +[()] diff --git a/tests/queries/0_stateless/01045_array_zip.sql b/tests/queries/0_stateless/01045_array_zip.sql index 0bf77747123..801df5a3230 100644 --- a/tests/queries/0_stateless/01045_array_zip.sql +++ b/tests/queries/0_stateless/01045_array_zip.sql @@ -2,7 +2,7 @@ SELECT arrayZip(['a', 'b', 'c'], ['d', 'e', 'f']); SELECT arrayZip(['a', 'b', 'c'], ['d', 'e', 'f'], ['g', 'h', 'i']); -SELECT arrayZip(); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION } +SELECT arrayZip(); SELECT arrayZip('a', 'b', 'c'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } diff --git a/tests/queries/0_stateless/03230_array_zip_unaligned.reference b/tests/queries/0_stateless/03230_array_zip_unaligned.reference index 7067f8788e5..7327d25eab8 100644 --- a/tests/queries/0_stateless/03230_array_zip_unaligned.reference +++ b/tests/queries/0_stateless/03230_array_zip_unaligned.reference @@ -1,5 +1,6 @@ [('a','d'),('b','e'),('c','f')] Array(Tuple(Nullable(String), Nullable(String))) [('a','d','g'),('b','e','h'),('c','f','i')] +[()] [('a','d'),('b','e'),('c','f'),(NULL,'g')] [('a',1),(NULL,2),(NULL,3)] [('a',1,1.1),('b',2,2.2),('c',NULL,3.3),(NULL,NULL,4.4)] diff --git a/tests/queries/0_stateless/03230_array_zip_unaligned.sql b/tests/queries/0_stateless/03230_array_zip_unaligned.sql index 90b7aa47bfd..08d77737e54 100644 --- a/tests/queries/0_stateless/03230_array_zip_unaligned.sql +++ b/tests/queries/0_stateless/03230_array_zip_unaligned.sql @@ -2,7 +2,7 @@ SELECT arrayZipUnaligned(['a', 'b', 'c'], ['d', 'e', 'f']) as x, toTypeName(x); SELECT arrayZipUnaligned(['a', 'b', 'c'], ['d', 'e', 'f'], ['g', 'h', 'i']); -SELECT arrayZipUnaligned(); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION } +SELECT arrayZipUnaligned(); SELECT arrayZipUnaligned('a', 'b', 'c'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } From 11d2963497ad62d729b9f445f97083a490c30a43 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Fri, 13 Sep 2024 11:56:47 +0800 Subject: [PATCH 057/103] fix style --- src/Functions/array/arrayZip.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/array/arrayZip.cpp b/src/Functions/array/arrayZip.cpp index 36ab56006b3..2f8c9a3af02 100644 --- a/src/Functions/array/arrayZip.cpp +++ b/src/Functions/array/arrayZip.cpp @@ -15,7 +15,6 @@ namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int SIZES_OF_ARRAYS_DONT_MATCH; -extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_COLUMN; } From 97c8d2897cec67ddf566fc50876d9c6d394b6b28 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 13 Sep 2024 09:30:19 +0000 Subject: [PATCH 058/103] Bump to v14.13 --- contrib/postgres | 2 +- contrib/postgres-cmake/pg_config.h | 26 +++++++++++++++++++++----- 2 files changed, 22 insertions(+), 6 deletions(-) diff --git a/contrib/postgres b/contrib/postgres index f0740b4eb82..665ff8c164d 160000 --- a/contrib/postgres +++ b/contrib/postgres @@ -1 +1 @@ -Subproject commit f0740b4eb8210cf3fd44de078ff9692d0b09bdb5 +Subproject commit 665ff8c164d56d012e359735efe4d400c0564b44 diff --git a/contrib/postgres-cmake/pg_config.h b/contrib/postgres-cmake/pg_config.h index a812e7f0776..ce16eab2239 100644 --- a/contrib/postgres-cmake/pg_config.h +++ b/contrib/postgres-cmake/pg_config.h @@ -296,9 +296,6 @@ /* Define to 1 if you have the `ldap' library (-lldap). */ //#define HAVE_LIBLDAP 0 -/* Define to 1 if you have the `ldap_r' library (-lldap_r). */ -#define HAVE_LIBLDAP_R 0 - /* Define to 1 if you have the `m' library (-lm). */ #define HAVE_LIBM 1 @@ -384,6 +381,15 @@ /* Define to 1 if you have the `posix_fadvise' function. */ #define HAVE_POSIX_FADVISE 1 +/* Define to 1 if you have the declaration of `preadv', and to 0 if you don't. */ +/* #undef HAVE_DECL_PREADV */ + +/* Define to 1 if you have the declaration of `pwritev', and to 0 if you don't. */ +/* #define HAVE_DECL_PWRITEV */ + +/* Define to 1 if you have the `X509_get_signature_info' function. */ +/* #undef HAVE_X509_GET_SIGNATURE_INFO */ + /* Define to 1 if you have the POSIX signal interface. */ #define HAVE_POSIX_SIGNALS 1 @@ -463,6 +469,9 @@ /* Define to 1 if you have the `srandom' function. */ #define HAVE_SRANDOM 1 +/* Define to 1 if you have the `SSL_CTX_set_num_tickets' function. */ +/* #define HAVE_SSL_CTX_SET_NUM_TICKETS */ + /* Define to 1 if you have the `SSL_get_current_compression' function. */ #define HAVE_SSL_GET_CURRENT_COMPRESSION 0 @@ -551,6 +560,9 @@ /* Define to 1 if you have the header file. */ #define HAVE_SYS_IPC_H 1 +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SYS_PERSONALITY_H */ + /* Define to 1 if you have the header file. */ #define HAVE_SYS_POLL_H 1 @@ -569,6 +581,9 @@ /* Define to 1 if you have the header file. */ #define HAVE_SYS_SHM_H 1 +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SYS_SIGNALFD_H */ + /* Define to 1 if you have the header file. */ #define HAVE_SYS_SOCKET_H 1 @@ -684,6 +699,9 @@ /* Define to 1 if your compiler understands __builtin_constant_p. */ #define HAVE__BUILTIN_CONSTANT_P 1 +/* Define to 1 if your compiler understands __builtin_frame_address. */ +/* #undef HAVE__BUILTIN_FRAME_ADDRESS */ + /* Define to 1 if your compiler understands __builtin_types_compatible_p. */ #define HAVE__BUILTIN_TYPES_COMPATIBLE_P 1 @@ -835,8 +853,6 @@ #define USE_OPENSSL_RANDOM 0 -#define HAVE_BIO_GET_DATA 0 - #define FRONTEND 1 /* Define to 1 to build with PAM support. (--with-pam) */ From 53e1975833a833951b110b21462a6a501cea48a6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Fri, 13 Sep 2024 18:15:47 +0800 Subject: [PATCH 059/103] fix 01045_array_zip --- tests/queries/0_stateless/01045_array_zip.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01045_array_zip.reference b/tests/queries/0_stateless/01045_array_zip.reference index 85b53dd0c9f..154afa7eb89 100644 --- a/tests/queries/0_stateless/01045_array_zip.reference +++ b/tests/queries/0_stateless/01045_array_zip.reference @@ -1,3 +1,3 @@ [('a','d'),('b','e'),('c','f')] [('a','d','g'),('b','e','h'),('c','f','i')] -[()] +[] From 71553022e0c36b7740445d1806f132613a394066 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Fri, 13 Sep 2024 18:16:13 +0800 Subject: [PATCH 060/103] fix 03230_array_zip_unaligned --- tests/queries/0_stateless/03230_array_zip_unaligned.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03230_array_zip_unaligned.reference b/tests/queries/0_stateless/03230_array_zip_unaligned.reference index 7327d25eab8..d373cf47a9c 100644 --- a/tests/queries/0_stateless/03230_array_zip_unaligned.reference +++ b/tests/queries/0_stateless/03230_array_zip_unaligned.reference @@ -1,6 +1,6 @@ [('a','d'),('b','e'),('c','f')] Array(Tuple(Nullable(String), Nullable(String))) [('a','d','g'),('b','e','h'),('c','f','i')] -[()] +[] [('a','d'),('b','e'),('c','f'),(NULL,'g')] [('a',1),(NULL,2),(NULL,3)] [('a',1,1.1),('b',2,2.2),('c',NULL,3.3),(NULL,NULL,4.4)] From 3c47f3df4b11b6824aec87a9cffb51c97683cccb Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 13 Sep 2024 13:34:21 +0200 Subject: [PATCH 061/103] Support more advanced SSL options for Keeper internal communication --- .../poco/Crypto/include/Poco/Crypto/EVPPKey.h | 5 +- .../NetSSL_OpenSSL/include/Poco/Net/Context.h | 10 + base/poco/NetSSL_OpenSSL/src/Context.cpp | 5 + src/Coordination/KeeperServer.cpp | 60 +++++- src/Server/CertificateReloader.cpp | 12 +- src/Server/CertificateReloader.h | 3 + .../configs/WithPassPhrase.crt | 20 ++ .../configs/WithPassPhrase.key | 30 +++ .../{server.crt => WithoutPassPhrase.crt} | 0 .../{server.key => WithoutPassPhrase.key} | 0 .../configs/enable_secure_keeper1.xml | 1 + .../configs/enable_secure_keeper2.xml | 1 + .../configs/enable_secure_keeper3.xml | 1 + .../configs/ssl_conf.xml | 15 -- .../configs/ssl_conf.yml | 10 + .../configs/ssl_conf_password.yml | 14 ++ .../test_keeper_internal_secure/ssl_conf.yml | 0 .../test_keeper_internal_secure/test.py | 190 +++++++++++++----- 18 files changed, 303 insertions(+), 74 deletions(-) create mode 100644 tests/integration/test_keeper_internal_secure/configs/WithPassPhrase.crt create mode 100644 tests/integration/test_keeper_internal_secure/configs/WithPassPhrase.key rename tests/integration/test_keeper_internal_secure/configs/{server.crt => WithoutPassPhrase.crt} (100%) rename tests/integration/test_keeper_internal_secure/configs/{server.key => WithoutPassPhrase.key} (100%) delete mode 100644 tests/integration/test_keeper_internal_secure/configs/ssl_conf.xml create mode 100644 tests/integration/test_keeper_internal_secure/configs/ssl_conf.yml create mode 100644 tests/integration/test_keeper_internal_secure/configs/ssl_conf_password.yml create mode 100644 tests/integration/test_keeper_internal_secure/ssl_conf.yml diff --git a/base/poco/Crypto/include/Poco/Crypto/EVPPKey.h b/base/poco/Crypto/include/Poco/Crypto/EVPPKey.h index acc79ec92b2..c33e0ae847f 100644 --- a/base/poco/Crypto/include/Poco/Crypto/EVPPKey.h +++ b/base/poco/Crypto/include/Poco/Crypto/EVPPKey.h @@ -188,8 +188,9 @@ namespace Crypto pFile = fopen(keyFile.c_str(), "r"); if (pFile) { - pem_password_cb * pCB = pass.empty() ? (pem_password_cb *)0 : &passCB; - void * pPassword = pass.empty() ? (void *)0 : (void *)pass.c_str(); + pem_password_cb * pCB = &passCB; + static constexpr char * no_password = ""; + void * pPassword = pass.empty() ? (void *)no_password : (void *)pass.c_str(); if (readFunc(pFile, &pKey, pCB, pPassword)) { fclose(pFile); diff --git a/base/poco/NetSSL_OpenSSL/include/Poco/Net/Context.h b/base/poco/NetSSL_OpenSSL/include/Poco/Net/Context.h index c19eecf5c73..2c56875835e 100644 --- a/base/poco/NetSSL_OpenSSL/include/Poco/Net/Context.h +++ b/base/poco/NetSSL_OpenSSL/include/Poco/Net/Context.h @@ -248,6 +248,9 @@ namespace Net SSL_CTX * sslContext() const; /// Returns the underlying OpenSSL SSL Context object. + SSL_CTX * takeSslContext(); + /// Takes ownership of the underlying OpenSSL SSL Context object. + Usage usage() const; /// Returns whether the context is for use by a client or by a server /// and whether TLSv1 is required. @@ -401,6 +404,13 @@ namespace Net return _pSSLContext; } + inline SSL_CTX * Context::takeSslContext() + { + auto * result = _pSSLContext; + _pSSLContext = nullptr; + return result; + } + inline bool Context::extendedCertificateVerificationEnabled() const { diff --git a/base/poco/NetSSL_OpenSSL/src/Context.cpp b/base/poco/NetSSL_OpenSSL/src/Context.cpp index da1c121286b..69c88eef63a 100644 --- a/base/poco/NetSSL_OpenSSL/src/Context.cpp +++ b/base/poco/NetSSL_OpenSSL/src/Context.cpp @@ -106,6 +106,11 @@ Context::Context( Context::~Context() { + if (_pSSLContext == nullptr) + { + return; + } + try { SSL_CTX_free(_pSSLContext); diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index f09ea56391a..8bfb6629fee 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -28,6 +28,15 @@ #include #include +#if USE_SSL +# include +# include +# include +# include +# include +# include +#endif + #include #include #include @@ -48,6 +57,7 @@ namespace ErrorCodes extern const int SUPPORT_IS_DISABLED; extern const int LOGICAL_ERROR; extern const int INVALID_CONFIG_PARAMETER; + extern const int BAD_ARGUMENTS; } using namespace std::chrono_literals; @@ -56,6 +66,16 @@ namespace { #if USE_SSL + +int callSetCertificate(SSL * ssl, void * arg) +{ + if (!arg) + return -1; + + const CertificateReloader::Data * data = reinterpret_cast(arg); + return setCertificateCallback(ssl, data, getLogger("SSLContext")); +} + void setSSLParams(nuraft::asio_service::options & asio_opts) { const Poco::Util::LayeredConfiguration & config = Poco::Util::Application::instance().config(); @@ -69,18 +89,42 @@ void setSSLParams(nuraft::asio_service::options & asio_opts) if (!config.has(private_key_file_property)) throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Server private key file is not set."); - asio_opts.enable_ssl_ = true; - asio_opts.server_cert_file_ = config.getString(certificate_file_property); - asio_opts.server_key_file_ = config.getString(private_key_file_property); + Poco::Net::Context::Params params; + params.certificateFile = config.getString(certificate_file_property); + if (params.certificateFile.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Server certificate file in config '{}' is empty", certificate_file_property); + + params.privateKeyFile = config.getString(private_key_file_property); + if (params.privateKeyFile.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Server key file in config '{}' is empty", private_key_file_property); + + auto pass_phrase = config.getString("openSSL.server.privateKeyPassphraseHandler.options.password", ""); + auto certificate_data = std::make_shared(params.certificateFile, params.privateKeyFile, pass_phrase); if (config.has(root_ca_file_property)) - asio_opts.root_cert_file_ = config.getString(root_ca_file_property); + params.caLocation = config.getString(root_ca_file_property); - if (config.getBool("openSSL.server.loadDefaultCAFile", false)) - asio_opts.load_default_ca_file_ = true; + params.loadDefaultCAs = config.getBool("openSSL.server.loadDefaultCAFile", false); + params.verificationMode = Poco::Net::Utility::convertVerificationMode(config.getString("openSSL.server.verificationMode", "none")); - if (config.getString("openSSL.server.verificationMode", "none") == "none") - asio_opts.skip_verification_ = true; + asio_opts.ssl_context_provider_server_ = [ctx_params = params, certificate_data] + { + Poco::Net::Context context(Poco::Net::Context::Usage::TLSV1_2_SERVER_USE, ctx_params); + SSL_CTX * ssl_ctx = context.takeSslContext(); + uint64_t options = 0; + options |= SSL_OP_ALL; + options |= SSL_OP_NO_SSLv2; + options |= SSL_OP_SINGLE_DH_USE; + SSL_CTX_set_options(ssl_ctx, options); + SSL_CTX_set_cert_cb(ssl_ctx, callSetCertificate, reinterpret_cast(certificate_data.get())); + return ssl_ctx; + }; + + asio_opts.ssl_context_provider_client_ = [ctx_params = std::move(params)] + { + Poco::Net::Context context(Poco::Net::Context::Usage::TLSV1_2_CLIENT_USE, ctx_params); + return context.takeSslContext(); + }; } #endif diff --git a/src/Server/CertificateReloader.cpp b/src/Server/CertificateReloader.cpp index df7b6e7fbd7..5b981fc7a87 100644 --- a/src/Server/CertificateReloader.cpp +++ b/src/Server/CertificateReloader.cpp @@ -34,8 +34,12 @@ int CertificateReloader::setCertificate(SSL * ssl, const CertificateReloader::Mu auto current = pdata->data.get(); if (!current) return -1; + return setCertificateCallback(ssl, current.get(), log); +} - if (current->certs_chain.empty()) +int setCertificateCallback(SSL * ssl, const CertificateReloader::Data * current_data, LoggerPtr log) +{ + if (current_data->certs_chain.empty()) return -1; if (auto err = SSL_clear_chain_certs(ssl); err != 1) @@ -43,12 +47,12 @@ int CertificateReloader::setCertificate(SSL * ssl, const CertificateReloader::Mu LOG_ERROR(log, "Clear certificates {}", Poco::Net::Utility::getLastError()); return -1; } - if (auto err = SSL_use_certificate(ssl, const_cast(current->certs_chain[0].certificate())); err != 1) + if (auto err = SSL_use_certificate(ssl, const_cast(current_data->certs_chain[0].certificate())); err != 1) { LOG_ERROR(log, "Use certificate {}", Poco::Net::Utility::getLastError()); return -1; } - for (auto cert = current->certs_chain.begin() + 1; cert != current->certs_chain.end(); cert++) + for (auto cert = current_data->certs_chain.begin() + 1; cert != current_data->certs_chain.end(); cert++) { if (auto err = SSL_add1_chain_cert(ssl, const_cast(cert->certificate())); err != 1) { @@ -56,7 +60,7 @@ int CertificateReloader::setCertificate(SSL * ssl, const CertificateReloader::Mu return -1; } } - if (auto err = SSL_use_PrivateKey(ssl, const_cast(static_cast(current->key))); err != 1) + if (auto err = SSL_use_PrivateKey(ssl, const_cast(static_cast(current_data->key))); err != 1) { LOG_ERROR(log, "Use private key {}", Poco::Net::Utility::getLastError()); return -1; diff --git a/src/Server/CertificateReloader.h b/src/Server/CertificateReloader.h index 7472d2f6baa..28737988fdd 100644 --- a/src/Server/CertificateReloader.h +++ b/src/Server/CertificateReloader.h @@ -104,6 +104,9 @@ private: mutable std::mutex data_mutex; }; +/// A callback for OpenSSL +int setCertificateCallback(SSL * ssl, const CertificateReloader::Data * current_data, LoggerPtr log); + } #endif diff --git a/tests/integration/test_keeper_internal_secure/configs/WithPassPhrase.crt b/tests/integration/test_keeper_internal_secure/configs/WithPassPhrase.crt new file mode 100644 index 00000000000..cabc53fa809 --- /dev/null +++ b/tests/integration/test_keeper_internal_secure/configs/WithPassPhrase.crt @@ -0,0 +1,20 @@ +-----BEGIN CERTIFICATE----- +MIIDPDCCAiQCFBXNOvsLA+dqmX/TkYG9JXdD5m72MA0GCSqGSIb3DQEBCwUAMFox +CzAJBgNVBAYTAkFVMRMwEQYDVQQIDApTb21lLVN0YXRlMSEwHwYDVQQKDBhJbnRl +cm5ldCBXaWRnaXRzIFB0eSBMdGQxEzARBgNVBAMMCmNsaWNraG91c2UwIBcNMjIw +NDIxMTAzNDU1WhgPMjEyMjAzMjgxMDM0NTVaMFkxCzAJBgNVBAYTAkFVMRMwEQYD +VQQIDApTb21lLVN0YXRlMSEwHwYDVQQKDBhJbnRlcm5ldCBXaWRnaXRzIFB0eSBM +dGQxEjAQBgNVBAMMCWxvY2FsaG9zdDCCASIwDQYJKoZIhvcNAQEBBQADggEPADCC +AQoCggEBAKaXz596N4NC2zZdIqdwZbSYAtNdBCsBVPt5YT9F640aF5zOogPZyxGP +ENyOZwABi/7HhwFbH657xyRvi8lTau8dZL+0tbakyoIn1Tw6j+/3GXTjLduJSy6C +mOf4OzsrFC8mYgU+7p5ijvWVlO9h5NMbLdAPSIB5WSHhmSORH5LgjoK6oMOYdRod +GmfHqSbwPVwy3Li5SXlniCQmJsM0zl64LFbJ/NU+13qETmhBiDgmh0Svi+wzSzqZ +q1PIX92T3k44IXNZbvF7lKbUOS9Xb3BoxA4cDqRcTx4x73xRDwodSmqiuQOC99HI +A0C/tZJ25VNAGjLKukPSHqYscq2PAsUCAwEAATANBgkqhkiG9w0BAQsFAAOCAQEA +IDQwjf/ja3TfOXrz+Gn1eErSKnWS3asjRT9rYWQsy3tzVUkMIcszrG+FqTR16g5H +ZWyuEOi6KIRmda3SYKdLKmtQLrgx6/d/jvH5TQ0LTFZrp6vh0lo3pV+L6fLo1ZRD +V1i8jW/7HHNyqJamUXOjwA0DpPOMkdtwuyV+rJ+2bTG1ZSK33O4Ae2CY5+dad6zy +YI6b1c9flWfDznuNEMH7jDDjKgXwjZGeU53FiuuhHiNyRchsr/B9eIBsom8oykiD +kch4cnAxx2E+m3hLYzupkXHOVQ5CNpVk8PGUCIGcyqDxPt+fOj1WbDQ9laEcfhmV +kR+vHmzOqWZnHU4QeMqDig== +-----END CERTIFICATE----- diff --git a/tests/integration/test_keeper_internal_secure/configs/WithPassPhrase.key b/tests/integration/test_keeper_internal_secure/configs/WithPassPhrase.key new file mode 100644 index 00000000000..1e29a4c8fa1 --- /dev/null +++ b/tests/integration/test_keeper_internal_secure/configs/WithPassPhrase.key @@ -0,0 +1,30 @@ +-----BEGIN RSA PRIVATE KEY----- +Proc-Type: 4,ENCRYPTED +DEK-Info: AES-256-CBC,4E14FF586022476CD22AAFB662BB0E40 + +dpJZKq5k+fMuC7XECfTSRjPeOEl9wNuVtZkcjEWaHN8ky4umND7ARyRyuU1Nk7cy +fpCFlFKOqDfCkT5zVK/fB6pF32wqAI7sqeSuYPfQY0+L77yRdwM6L46WslzVKZYE +lXD1AmqKT/LgF3+eBY5slkAAJo10zYDgKEwnoQVBp31YW2/+6oAGaY/O6x3p7aTG +dw9CP+SFc0o8lPl1lsSovdNXDUiVCftvClog7hwyDv8AhHyGgynw3UJXX8UlyWu+ +Zz5zpgrvB2gvDLeoZZ6qjMGvtpEwlYBh4de9ZOsvQUpXEEfkQFtJV0j613OCQune +LoTxKpYV1V/mZX4HPaJ1oC0OJ7XcliAOSS9K49YobTuz7Kg5Wg3bVGo9xRfYDjch +rVeMP4u5RXSGuHL23FPMfK6EqcldrFyRwLaY/IV1Yl6UNUMKAphn/WMtWVuT3TiT +QMCI2VRt7ItwZwwFn5RgyDweWdFf5v3AmN/lOhATDBqosahkPxDZ1VZ6OBPoJLPM +UrDWH/lqrByeEjtAOwr5UsWKwLuJ8qUxQ4TchHwFKOwy6VsrRwMQ3ZWi2govPF9I +W0sfLj5Ulfjx6zHdqnF48a1Elit4JH6inBEWFuj7bmlOotq+PHoeT61zAwW+gnrG +3JTo3XnaE2WwRDpqvKYHWLv/J218rq8PtIaq9gjr55odPfIt8lkJ1XzF4WQ21rIJ +GNWZ3xz4fxpvrKnQyAKGu0ZcdjA1nqs16oiVr+UnJoXmkM5yBCic4fZYwPTSQHYS +ZxwaTzEjfeGxrSeLrN9CgoweucvogOvUjJOBcW/py80du8vWz0YyzMhg3o0YeGME +C+Kts/YWxmyfw4DaWt8RtWCKl85hEmz8RODvkMLGtLzvVoSyLQWqp1NhGIlFtzXs +7sPLulUeyD2avTC/RB/Pu9Nk80c0368BxCoeYbiFWZpaN70SJmCUE5H59J2d0olw +5v2RVjLBi8wqnzoa0+2L8wnG7IQGadS97dj0eBR+JXNtoJhVrurS80RJ6B0bNxdu +gX8otfnJYsZyK5hbEhcQqLdnyGhDEE8YHe7Hv9stWwLAFOfOMzyzC06lFS1eNiw4 +FJyXJUhDieb8EqetouAC8dNVXz4Q1zOTlGuAbGoKm5v0U5IhCQap9GUSW5QiUgOQ +AEMs9aGfd91R+IcDf19mZptsQLYA6MGBN6fm+3O2iZImKIbF+ZZo0S6liFFmn6lm +M+diTzaoiqgEkiXOuRhdQUMaiGV8BMZxv8qUH6/vyC3gSueoTio0f9PfASDYfvXD +A3GuI87P6LF1it2UlN6ssFoXTZdfQQZwRmNuqOqw+BJOJHrR6trcXOCZOQ77Qnvd +M5a348gIzluVUkExAPGCsySQWMx4Of5NBF28jEC3+TAwkRqBV2ZHmfGLWnvwaB+A +YUeKtpWblfG1lsrDAdwL2dilU95oby+35sExX7M2dCrL9Y2P5oTCW3u12//ZSLeL +Yhi1Rzol6LAuesZCVF0Zv/YYDhzAckJfT/qXK5B5pz9saswxCUBEpiKlLpVsjOFJ +2bHm8NgOMD5b3cdh1kvts4wZe+giry7LHsn46f+9VqN+gA6XxeVsPyb4uO1KW3SN +-----END RSA PRIVATE KEY----- diff --git a/tests/integration/test_keeper_internal_secure/configs/server.crt b/tests/integration/test_keeper_internal_secure/configs/WithoutPassPhrase.crt similarity index 100% rename from tests/integration/test_keeper_internal_secure/configs/server.crt rename to tests/integration/test_keeper_internal_secure/configs/WithoutPassPhrase.crt diff --git a/tests/integration/test_keeper_internal_secure/configs/server.key b/tests/integration/test_keeper_internal_secure/configs/WithoutPassPhrase.key similarity index 100% rename from tests/integration/test_keeper_internal_secure/configs/server.key rename to tests/integration/test_keeper_internal_secure/configs/WithoutPassPhrase.key diff --git a/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper1.xml b/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper1.xml index 986b503ebe3..dabf280bc36 100644 --- a/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper1.xml +++ b/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper1.xml @@ -1,5 +1,6 @@ + 0 9181 1 /var/lib/clickhouse/coordination/log diff --git a/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper2.xml b/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper2.xml index 652b1992f46..21d8f1e0eb3 100644 --- a/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper2.xml +++ b/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper2.xml @@ -1,5 +1,6 @@ + 0 9181 2 /var/lib/clickhouse/coordination/log diff --git a/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper3.xml b/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper3.xml index 6507f97473b..5d8cfb8b3e7 100644 --- a/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper3.xml +++ b/tests/integration/test_keeper_internal_secure/configs/enable_secure_keeper3.xml @@ -1,5 +1,6 @@ + 0 9181 3 /var/lib/clickhouse/coordination/log diff --git a/tests/integration/test_keeper_internal_secure/configs/ssl_conf.xml b/tests/integration/test_keeper_internal_secure/configs/ssl_conf.xml deleted file mode 100644 index 37eb2624b1b..00000000000 --- a/tests/integration/test_keeper_internal_secure/configs/ssl_conf.xml +++ /dev/null @@ -1,15 +0,0 @@ - - - - - /etc/clickhouse-server/config.d/server.crt - /etc/clickhouse-server/config.d/server.key - /etc/clickhouse-server/config.d/rootCA.pem - true - none - true - sslv2,sslv3 - true - - - diff --git a/tests/integration/test_keeper_internal_secure/configs/ssl_conf.yml b/tests/integration/test_keeper_internal_secure/configs/ssl_conf.yml new file mode 100644 index 00000000000..a444122b09d --- /dev/null +++ b/tests/integration/test_keeper_internal_secure/configs/ssl_conf.yml @@ -0,0 +1,10 @@ +openSSL: + server: + certificateFile: '/etc/clickhouse-server/config.d/WithoutPassPhrase.crt' + privateKeyFile: '/etc/clickhouse-server/config.d/WithoutPassPhrase.key' + caConfig: '/etc/clickhouse-server/config.d/rootCA.pem' + loadDefaultCAFile: true + verificationMode: 'none' + cacheSessions: true + disableProtocols: 'sslv2,sslv3' + preferServerCiphers: true diff --git a/tests/integration/test_keeper_internal_secure/configs/ssl_conf_password.yml b/tests/integration/test_keeper_internal_secure/configs/ssl_conf_password.yml new file mode 100644 index 00000000000..51b65c5253a --- /dev/null +++ b/tests/integration/test_keeper_internal_secure/configs/ssl_conf_password.yml @@ -0,0 +1,14 @@ +openSSL: + server: + certificateFile: '/etc/clickhouse-server/config.d/WithoutPassPhrase.crt' + privateKeyFile: '/etc/clickhouse-server/config.d/WithoutPassPhrase.key' + privateKeyPassphraseHandler: + name: KeyFileHandler + options: + password: 'PASSWORD' + caConfig: '/etc/clickhouse-server/config.d/rootCA.pem' + loadDefaultCAFile: true + verificationMode: 'none' + cacheSessions: true + disableProtocols: 'sslv2,sslv3' + preferServerCiphers: true diff --git a/tests/integration/test_keeper_internal_secure/ssl_conf.yml b/tests/integration/test_keeper_internal_secure/ssl_conf.yml new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_keeper_internal_secure/test.py b/tests/integration/test_keeper_internal_secure/test.py index 2d45e95e4ff..af511a60636 100644 --- a/tests/integration/test_keeper_internal_secure/test.py +++ b/tests/integration/test_keeper_internal_secure/test.py @@ -2,44 +2,55 @@ import pytest from helpers.cluster import ClickHouseCluster -import random -import string +import helpers.keeper_utils as ku +from multiprocessing.dummy import Pool import os -import time +CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance( - "node1", - main_configs=[ - "configs/enable_secure_keeper1.xml", - "configs/ssl_conf.xml", - "configs/server.crt", - "configs/server.key", - "configs/rootCA.pem", - ], -) -node2 = cluster.add_instance( - "node2", - main_configs=[ - "configs/enable_secure_keeper2.xml", - "configs/ssl_conf.xml", - "configs/server.crt", - "configs/server.key", - "configs/rootCA.pem", - ], -) -node3 = cluster.add_instance( - "node3", - main_configs=[ - "configs/enable_secure_keeper3.xml", - "configs/ssl_conf.xml", - "configs/server.crt", - "configs/server.key", - "configs/rootCA.pem", - ], -) +nodes = [ + cluster.add_instance( + "node1", + main_configs=[ + "configs/enable_secure_keeper1.xml", + "configs/ssl_conf.yml", + "configs/WithoutPassPhrase.crt", + "configs/WithoutPassPhrase.key", + "configs/WithPassPhrase.crt", + "configs/WithPassPhrase.key", + "configs/rootCA.pem", + ], + stay_alive=True, + ), + cluster.add_instance( + "node2", + main_configs=[ + "configs/enable_secure_keeper2.xml", + "configs/ssl_conf.yml", + "configs/WithoutPassPhrase.crt", + "configs/WithoutPassPhrase.key", + "configs/WithPassPhrase.crt", + "configs/WithPassPhrase.key", + "configs/rootCA.pem", + ], + stay_alive=True, + ), + cluster.add_instance( + "node3", + main_configs=[ + "configs/enable_secure_keeper3.xml", + "configs/ssl_conf.yml", + "configs/WithoutPassPhrase.crt", + "configs/WithoutPassPhrase.key", + "configs/WithPassPhrase.crt", + "configs/WithPassPhrase.key", + "configs/rootCA.pem", + ], + stay_alive=True, + ), +] -from kazoo.client import KazooClient, KazooState +from kazoo.client import KazooClient @pytest.fixture(scope="module") @@ -61,23 +72,112 @@ def get_fake_zk(nodename, timeout=30.0): return _fake_zk_instance -def test_secure_raft_works(started_cluster): +def run_test(): + node_zks = [] try: - node1_zk = get_fake_zk("node1") - node2_zk = get_fake_zk("node2") - node3_zk = get_fake_zk("node3") + for node in nodes: + node_zks.append(get_fake_zk(node.name)) - node1_zk.create("/test_node", b"somedata1") - node2_zk.sync("/test_node") - node3_zk.sync("/test_node") + node_zks[0].create("/test_node", b"somedata1") + node_zks[1].sync("/test_node") + node_zks[2].sync("/test_node") - assert node1_zk.exists("/test_node") is not None - assert node2_zk.exists("/test_node") is not None - assert node3_zk.exists("/test_node") is not None + for node_zk in node_zks: + assert node_zk.exists("/test_node") is not None finally: try: - for zk_conn in [node1_zk, node2_zk, node3_zk]: + for zk_conn in node_zks: + if zk_conn is None: + continue zk_conn.stop() zk_conn.close() except: pass + + +def setupSsl(node, filename, password): + if password is None: + node.copy_file_to_container( + os.path.join(CURRENT_TEST_DIR, "configs/ssl_conf.yml"), + "/etc/clickhouse-server/config.d/ssl_conf.yml", + ) + + node.replace_in_config( + "/etc/clickhouse-server/config.d/ssl_conf.yml", + "WithoutPassPhrase", + filename, + ) + return + + node.copy_file_to_container( + os.path.join(CURRENT_TEST_DIR, "configs/ssl_conf_password.yml"), + "/etc/clickhouse-server/config.d/ssl_conf.yml", + ) + + node.replace_in_config( + "/etc/clickhouse-server/config.d/ssl_conf.yml", + "WithoutPassPhrase", + filename, + ) + + node.replace_in_config( + "/etc/clickhouse-server/config.d/ssl_conf.yml", + "PASSWORD", + password, + ) + + +def stop_all_clickhouse(): + for node in nodes: + node.stop_clickhouse() + + for node in nodes: + node.exec_in_container(["rm", "-rf", "/var/lib/clickhouse/coordination"]) + + +def start_clickhouse(node): + node.start_clickhouse() + + +def start_all_clickhouse(): + p = Pool(3) + waiters = [] + + for node in nodes: + waiters.append(p.apply_async(start_clickhouse, args=(node,))) + + for waiter in waiters: + waiter.wait() + + for node in nodes: + ku.wait_until_connected(cluster, node) + + +def check_valid_configuration(filename, password): + stop_all_clickhouse() + for node in nodes: + setupSsl(node, filename, password) + start_all_clickhouse() + run_test() + + +def test_secure_raft_works(started_cluster): + check_valid_configuration("WithoutPassPhrase", None) + + +def test_secure_raft_works_with_password(started_cluster): + def check_invalid_configuration(filename, password): + stop_all_clickhouse() + for node in nodes: + setupSsl(node, filename, password) + + nodes[0].start_clickhouse(expected_to_fail=True) + nodes[0].contains_in_log( + "OpenSSLException: EVPKey::loadKey(string): error:0480006C:PEM routines::no start line" + ) + + check_valid_configuration("WithoutPassPhrase", "unusedpassword") + check_invalid_configuration("WithPassPhrase", "wrongpassword") + check_invalid_configuration("WithPassPhrase", "") + check_valid_configuration("WithPassPhrase", "test") + check_invalid_configuration("WithPassPhrase", None) From 2812953a8ac38cddab356d2e24856ea8c8eb7ab1 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 13 Sep 2024 13:37:42 +0000 Subject: [PATCH 062/103] Try to fix tests --- tests/queries/1_stateful/00091_prewhere_two_conditions.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/1_stateful/00091_prewhere_two_conditions.sql b/tests/queries/1_stateful/00091_prewhere_two_conditions.sql index cd88743160c..649c63f2ec9 100644 --- a/tests/queries/1_stateful/00091_prewhere_two_conditions.sql +++ b/tests/queries/1_stateful/00091_prewhere_two_conditions.sql @@ -7,7 +7,7 @@ SET optimize_move_to_prewhere = 1; SET enable_multiple_prewhere_read_steps = 1; SELECT uniq(URL) FROM test.hits WHERE toTimeZone(EventTime, 'Asia/Dubai') >= '2014-03-20 00:00:00' AND toTimeZone(EventTime, 'Asia/Dubai') < '2014-03-21 00:00:00'; -SELECT uniq(URL) FROM test.hits WHERE toTimeZone(EventTime, 'Asia/Dubai') >= '2014-03-20 00:00:00' AND URL != '' AND toTimeZone(EventTime, 'Asia/Dubai') < '2014-03-21 00:00:00'; +SELECT uniq(URL) FROM test.hits WHERE toTimeZone(EventTime, 'Asia/Dubai') >= '2014-03-20 00:00:00' AND toTimeZone(EventTime, 'Asia/Dubai') < '2014-03-21 00:00:00' AND URL != ''; SELECT uniq(*) FROM test.hits WHERE toTimeZone(EventTime, 'Asia/Dubai') >= '2014-03-20 00:00:00' AND toTimeZone(EventTime, 'Asia/Dubai') < '2014-03-21 00:00:00' AND EventDate = '2014-03-21'; WITH toTimeZone(EventTime, 'Asia/Dubai') AS xyz SELECT uniq(*) FROM test.hits WHERE xyz >= '2014-03-20 00:00:00' AND xyz < '2014-03-21 00:00:00' AND EventDate = '2014-03-21'; From ddc506a677253b206922b1faa9b72f36a866d6f2 Mon Sep 17 00:00:00 2001 From: marco-vb Date: Fri, 13 Sep 2024 13:21:35 +0000 Subject: [PATCH 063/103] Corrected implementation for check of new settings and fix lint of settings change history. --- src/Core/SettingsChangesHistory.cpp | 4 +- .../MergeTree/MergeTreeDataWriter.cpp | 48 +++++++++++-------- 2 files changed, 29 insertions(+), 23 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 03d0ef8ea76..490198c2376 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -82,9 +82,9 @@ static std::initializer_list #include #include -#include #include #include #include @@ -61,6 +60,7 @@ namespace ErrorCodes extern const int ABORTED; extern const int LOGICAL_ERROR; extern const int TOO_MANY_PARTS; + extern const int NOT_ENOUGH_SPACE; } namespace @@ -554,6 +554,31 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( VolumePtr volume = data.getStoragePolicy()->getVolume(0); VolumePtr data_part_volume = createVolumeFromReservation(reservation, volume); + const auto & data_settings = data.getSettings(); + + { + const UInt64 min_bytes = data_settings->min_free_disk_bytes_to_throw_insert; + const Float64 min_ratio = data_settings->min_free_disk_ratio_to_throw_insert; + + const auto disk = data_part_volume->getDisk(); + const UInt64 total_disk_bytes = *disk->getTotalSpace(); + const UInt64 free_disk_bytes = *disk->getUnreservedSpace(); + + const UInt64 min_bytes_from_ratio = static_cast(min_ratio * total_disk_bytes); + const UInt64 needed_free_bytes = std::max(min_bytes, min_bytes_from_ratio); + + if (needed_free_bytes > free_disk_bytes) + { + throw Exception( + ErrorCodes::NOT_ENOUGH_SPACE, + "Could not perform insert: less than {} free bytes in disk space. " + "Configure this limit with user settings {} or {}", + needed_free_bytes, + "min_free_disk_bytes_to_throw_insert", + "min_free_disk_ratio_to_throw_insert"); + } + } + auto new_data_part = data.getDataPartBuilder(part_name, data_part_volume, part_dir) .withPartFormat(data.choosePartFormat(expected_size, block.rows())) .withPartInfo(new_part_info) @@ -565,8 +590,6 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( if (data.storage_settings.get()->assign_part_uuids) new_data_part->uuid = UUIDHelpers::generateV4(); - const auto & data_settings = data.getSettings(); - SerializationInfo::Settings settings{data_settings->ratio_of_defaults_for_sparse_serialization, true}; SerializationInfoByName infos(columns, settings); infos.add(block); @@ -690,25 +713,8 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( /// Size of part would not be greater than block.bytes() + epsilon size_t expected_size = block.bytes(); - // If not temporary insert try to reserve respecting min free disk bytes - size_t reserve_extra = 0; - - if (!is_temp) - { - const auto context = CurrentThread::getQueryContext(); - const auto * settings = context ? &context->getSettingsRef() : nullptr; - - const UInt64 min_bytes = settings->min_free_disk_bytes_to_throw_insert; - const Float64 min_ratio = settings->min_free_disk_ratio_to_throw_insert; - - const auto total_disk_space = parent_part->getDataPartStorage().calculateTotalSizeOnDisk(); - const UInt64 min_bytes_from_ratio = static_cast(min_ratio * total_disk_space); - reserve_extra = std::min(min_bytes, min_bytes_from_ratio); - } - // just check if there is enough space on parent volume - // down the line in reserving space there is concurrency safety so no need to worry about 'over-reserving' - MergeTreeData::reserveSpace(expected_size + reserve_extra, parent_part->getDataPartStorage()); + MergeTreeData::reserveSpace(expected_size, parent_part->getDataPartStorage()); part_type = data.choosePartFormatOnDisk(expected_size, block.rows()).part_type; auto new_data_part = parent_part->getProjectionPartBuilder(part_name, is_temp).withPartType(part_type).build(); From 9a31fc385d6335d2f21adfbfef2bb609044510ad Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 13 Sep 2024 15:58:17 +0200 Subject: [PATCH 064/103] Fixes --- src/Coordination/KeeperServer.cpp | 26 ++++++++++++++++++++------ 1 file changed, 20 insertions(+), 6 deletions(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 48c377a0c0b..e0a94b1a00c 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -35,6 +35,7 @@ # include # include # include +# include #endif #include @@ -107,15 +108,28 @@ void setSSLParams(nuraft::asio_service::options & asio_opts) params.loadDefaultCAs = config.getBool("openSSL.server.loadDefaultCAFile", false); params.verificationMode = Poco::Net::Utility::convertVerificationMode(config.getString("openSSL.server.verificationMode", "none")); - asio_opts.ssl_context_provider_server_ = [ctx_params = params, certificate_data] + std::string disabled_protocols_list = config.getString("openSSL.server.disableProtocols", ""); + Poco::StringTokenizer dp_tok(disabled_protocols_list, ";,", Poco::StringTokenizer::TOK_TRIM | Poco::StringTokenizer::TOK_IGNORE_EMPTY); + int disabled_protocols = 0; + for (const auto & token : dp_tok) + { + if (token == "sslv2") + disabled_protocols |= Poco::Net::Context::PROTO_SSLV2; + else if (token == "sslv3") + disabled_protocols |= Poco::Net::Context::PROTO_SSLV3; + else if (token == "tlsv1") + disabled_protocols |= Poco::Net::Context::PROTO_TLSV1; + else if (token == "tlsv1_1") + disabled_protocols |= Poco::Net::Context::PROTO_TLSV1_1; + else if (token == "tlsv1_2") + disabled_protocols |= Poco::Net::Context::PROTO_TLSV1_2; + } + + asio_opts.ssl_context_provider_server_ = [ctx_params = params, certificate_data, disabled_protocols] { Poco::Net::Context context(Poco::Net::Context::Usage::TLSV1_2_SERVER_USE, ctx_params); + context.disableProtocols(disabled_protocols); SSL_CTX * ssl_ctx = context.takeSslContext(); - uint64_t options = 0; - options |= SSL_OP_ALL; - options |= SSL_OP_NO_SSLv2; - options |= SSL_OP_SINGLE_DH_USE; - SSL_CTX_set_options(ssl_ctx, options); SSL_CTX_set_cert_cb(ssl_ctx, callSetCertificate, reinterpret_cast(certificate_data.get())); return ssl_ctx; }; From 2650a2062899f5f232176ee56814cce66c800139 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 13 Sep 2024 16:21:17 +0200 Subject: [PATCH 065/103] Make dedup logic O(n*log(n)) instead of O(n^2) --- .../ParallelReplicasReadingCoordinator.cpp | 29 ++++++++++++++----- 1 file changed, 22 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 8abf735b49f..c9fb09cd0ba 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -869,8 +869,7 @@ void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRa /// To get rid of duplicates for (auto && part: announcement.description) { - auto the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(), - [&part] (const Part & other) { return other.description.info == part.info; }); + auto the_same_it = all_parts_to_read.find(Part{.description = part}); /// We have the same part - add the info about presence on the corresponding replica to it if (the_same_it != all_parts_to_read.end()) @@ -882,12 +881,28 @@ void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRa if (state_initialized) continue; - auto covering_or_the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(), - [&part] (const Part & other) { return other.description.info.contains(part.info) || part.info.contains(other.description.info); }); + /// Look for the first part >= current + auto covering_it = all_parts_to_read.lower_bound(Part{.description = part}); - /// It is covering part or we have covering - skip it - if (covering_or_the_same_it != all_parts_to_read.end()) - continue; + if (covering_it != all_parts_to_read.end()) + { + /// Checks if other part covers this one or this one covers the other + auto is_covered_or_covering = [&part] (const Part & other) + { + return other.description.info.contains(part.info) || part.info.contains(other.description.info); + }; + + if (is_covered_or_covering(*covering_it)) + continue; + + /// Also look at the previous part, it could be covering the current one + if (covering_it != all_parts_to_read.begin()) + { + --covering_it; + if (is_covered_or_covering(*covering_it)) + continue; + } + } new_rows_to_read += part.rows; From e13247b67ee66d510af988cf0799a7286dab4ea4 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 13 Sep 2024 16:50:43 +0200 Subject: [PATCH 066/103] Fix clang-18 build --- .../MergeTree/ParallelReplicasReadingCoordinator.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index c9fb09cd0ba..ddbed5db7dc 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -869,7 +869,7 @@ void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRa /// To get rid of duplicates for (auto && part: announcement.description) { - auto the_same_it = all_parts_to_read.find(Part{.description = part}); + auto the_same_it = all_parts_to_read.find(Part{.description = part, .replicas = {}}); /// We have the same part - add the info about presence on the corresponding replica to it if (the_same_it != all_parts_to_read.end()) @@ -882,14 +882,14 @@ void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRa continue; /// Look for the first part >= current - auto covering_it = all_parts_to_read.lower_bound(Part{.description = part}); + auto covering_it = all_parts_to_read.lower_bound(Part{.description = part, .replicas = {}}); if (covering_it != all_parts_to_read.end()) { /// Checks if other part covers this one or this one covers the other auto is_covered_or_covering = [&part] (const Part & other) { - return other.description.info.contains(part.info) || part.info.contains(other.description.info); + return other.description.info.contains(part.info) || part.info.contains(other.description.info); }; if (is_covered_or_covering(*covering_it)) From 5cc12ca9eed1eca78a89bacdf8b824105a089aa9 Mon Sep 17 00:00:00 2001 From: marco-vb Date: Fri, 13 Sep 2024 17:16:16 +0000 Subject: [PATCH 067/103] Added integration testing for newly implemented settings. --- .../MergeTree/MergeTreeDataWriter.cpp | 5 +- .../__init__.py | 0 .../config.d/storage_configuration.xml | 19 +++++++ .../test.py | 51 +++++++++++++++++++ 4 files changed, 73 insertions(+), 2 deletions(-) create mode 100644 tests/integration/test_stop_insert_when_disk_close_to_full/__init__.py create mode 100644 tests/integration/test_stop_insert_when_disk_close_to_full/configs/config.d/storage_configuration.xml create mode 100644 tests/integration/test_stop_insert_when_disk_close_to_full/test.py diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 66422dd621e..b606bff7faa 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -562,7 +562,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( const auto disk = data_part_volume->getDisk(); const UInt64 total_disk_bytes = *disk->getTotalSpace(); - const UInt64 free_disk_bytes = *disk->getUnreservedSpace(); + const UInt64 free_disk_bytes = *disk->getAvailableSpace(); const UInt64 min_bytes_from_ratio = static_cast(min_ratio * total_disk_bytes); const UInt64 needed_free_bytes = std::max(min_bytes, min_bytes_from_ratio); @@ -571,9 +571,10 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( { throw Exception( ErrorCodes::NOT_ENOUGH_SPACE, - "Could not perform insert: less than {} free bytes in disk space. " + "Could not perform insert: less than {} free bytes in disk space ({}). " "Configure this limit with user settings {} or {}", needed_free_bytes, + free_disk_bytes, "min_free_disk_bytes_to_throw_insert", "min_free_disk_ratio_to_throw_insert"); } diff --git a/tests/integration/test_stop_insert_when_disk_close_to_full/__init__.py b/tests/integration/test_stop_insert_when_disk_close_to_full/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_stop_insert_when_disk_close_to_full/configs/config.d/storage_configuration.xml b/tests/integration/test_stop_insert_when_disk_close_to_full/configs/config.d/storage_configuration.xml new file mode 100644 index 00000000000..d4031ff656c --- /dev/null +++ b/tests/integration/test_stop_insert_when_disk_close_to_full/configs/config.d/storage_configuration.xml @@ -0,0 +1,19 @@ + + + + + local + /disk1/ + + + + + +
+ disk1 +
+
+
+
+
+
diff --git a/tests/integration/test_stop_insert_when_disk_close_to_full/test.py b/tests/integration/test_stop_insert_when_disk_close_to_full/test.py new file mode 100644 index 00000000000..d8533ba90bc --- /dev/null +++ b/tests/integration/test_stop_insert_when_disk_close_to_full/test.py @@ -0,0 +1,51 @@ +import pytest +from helpers.cluster import ClickHouseCluster, ClickHouseInstance +from helpers.client import QueryRuntimeException + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", + main_configs=["configs/config.d/storage_configuration.xml"], + tmpfs=["/disk1:size=100M"], + macros={"shard": 0, "replica": 1}, +) + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + +def test_insert_stops_when_disk_full(start_cluster): + min_free_bytes = 3 * 1024 * 1024 # 3 MiB + + node.query(f""" + CREATE TABLE test_table ( + id UInt32, + data String + ) ENGINE = MergeTree() + ORDER BY id + SETTINGS storage_policy = 'only_disk1', min_free_disk_bytes_to_throw_insert = {min_free_bytes} + """) + + count = 0 + + # Insert data to fill up disk + try: + for _ in range(100000): + node.query("INSERT INTO test_table SELECT number, repeat('a', 1000 * 1000) FROM numbers(1)") + count += 1 + except QueryRuntimeException as e: + assert "Could not perform insert" in str(e) + assert "free bytes in disk space" in str(e) + + free_space = int(node.query("SELECT free_space FROM system.disks WHERE name = 'disk1'").strip()) + assert free_space <= min_free_bytes, f"Free space ({free_space}) is less than min_free_bytes ({min_free_bytes})" + + rows = int(node.query("SELECT count() from test_table").strip()) + assert rows == count + + node.query("DROP TABLE test_table") \ No newline at end of file From 31ddfc6f5f8b2cca0f003102ee296738b4ad9789 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 13 Sep 2024 19:19:21 +0200 Subject: [PATCH 068/103] Check time limit while analyzing indexes --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 17 +++++++++++++++-- .../MergeTree/MergeTreeDataSelectExecutor.h | 3 ++- 2 files changed, 17 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 03db96dd016..5d246293396 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include #include #include @@ -528,6 +529,8 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( } auto query_context = context->hasQueryContext() ? context->getQueryContext() : context; + QueryStatusPtr query_status = context->getProcessListElement(); + PartFilterCounters part_filter_counters; if (query_context->getSettingsRef().allow_experimental_query_deduplication) selectPartsToReadWithUUIDFilter( @@ -549,7 +552,8 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( minmax_columns_types, partition_pruner, max_block_numbers_to_read, - part_filter_counters); + part_filter_counters, + query_status); index_stats.emplace_back(ReadFromMergeTree::IndexStat{ .type = ReadFromMergeTree::IndexType::None, @@ -649,8 +653,13 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd auto mark_cache = context->getIndexMarkCache(); auto uncompressed_cache = context->getIndexUncompressedCache(); + auto query_status = context->getProcessListElement(); + auto process_part = [&](size_t part_index) { + if (query_status) + query_status->checkTimeLimit(); + auto & part = parts[part_index]; RangesInDataPart ranges(part, part_index); @@ -1545,13 +1554,17 @@ void MergeTreeDataSelectExecutor::selectPartsToRead( const DataTypes & minmax_columns_types, const std::optional & partition_pruner, const PartitionIdToMaxBlock * max_block_numbers_to_read, - PartFilterCounters & counters) + PartFilterCounters & counters, + QueryStatusPtr query_status) { MergeTreeData::DataPartsVector prev_parts; std::swap(prev_parts, parts); for (const auto & part_or_projection : prev_parts) { + if (query_status) + query_status->checkTimeLimit(); + const auto * part = part_or_projection->isProjectionPart() ? part_or_projection->getParentPart() : part_or_projection.get(); if (part_values && part_values->find(part->name) == part_values->end()) continue; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 3668eb0ad90..70536b7aa54 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -126,7 +126,8 @@ private: const DataTypes & minmax_columns_types, const std::optional & partition_pruner, const PartitionIdToMaxBlock * max_block_numbers_to_read, - PartFilterCounters & counters); + PartFilterCounters & counters, + QueryStatusPtr query_status); /// Same as previous but also skip parts uuids if any to the query context, or skip parts which uuids marked as excluded. static void selectPartsToReadWithUUIDFilter( From 56f3030b1795a4c3afddfec600cb22afda5a204f Mon Sep 17 00:00:00 2001 From: marco-vb Date: Fri, 13 Sep 2024 17:32:33 +0000 Subject: [PATCH 069/103] Black formatting python test. --- .../test.py | 22 ++++++++++++++----- 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_stop_insert_when_disk_close_to_full/test.py b/tests/integration/test_stop_insert_when_disk_close_to_full/test.py index d8533ba90bc..9b8943705fd 100644 --- a/tests/integration/test_stop_insert_when_disk_close_to_full/test.py +++ b/tests/integration/test_stop_insert_when_disk_close_to_full/test.py @@ -11,6 +11,7 @@ node = cluster.add_instance( macros={"shard": 0, "replica": 1}, ) + @pytest.fixture(scope="module") def start_cluster(): try: @@ -19,33 +20,42 @@ def start_cluster(): finally: cluster.shutdown() + def test_insert_stops_when_disk_full(start_cluster): min_free_bytes = 3 * 1024 * 1024 # 3 MiB - node.query(f""" + node.query( + f""" CREATE TABLE test_table ( id UInt32, data String ) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'only_disk1', min_free_disk_bytes_to_throw_insert = {min_free_bytes} - """) + """ + ) count = 0 # Insert data to fill up disk try: for _ in range(100000): - node.query("INSERT INTO test_table SELECT number, repeat('a', 1000 * 1000) FROM numbers(1)") + node.query( + "INSERT INTO test_table SELECT number, repeat('a', 1000 * 1000) FROM numbers(1)" + ) count += 1 except QueryRuntimeException as e: assert "Could not perform insert" in str(e) assert "free bytes in disk space" in str(e) - free_space = int(node.query("SELECT free_space FROM system.disks WHERE name = 'disk1'").strip()) - assert free_space <= min_free_bytes, f"Free space ({free_space}) is less than min_free_bytes ({min_free_bytes})" + free_space = int( + node.query("SELECT free_space FROM system.disks WHERE name = 'disk1'").strip() + ) + assert ( + free_space <= min_free_bytes + ), f"Free space ({free_space}) is less than min_free_bytes ({min_free_bytes})" rows = int(node.query("SELECT count() from test_table").strip()) assert rows == count - node.query("DROP TABLE test_table") \ No newline at end of file + node.query("DROP TABLE test_table") From dbb1d043fec40cb3ef403645483c3aad622b4bee Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 14 Sep 2024 05:46:08 +0000 Subject: [PATCH 070/103] unification of FunctionSecretArgumentsFinder --- .../FunctionSecretArgumentsFinderTreeNode.h | 408 ++---------- src/Parsers/FunctionSecretArgumentsFinder.h | 512 ++++++++++++++- .../FunctionSecretArgumentsFinderAST.h | 583 +++--------------- 3 files changed, 642 insertions(+), 861 deletions(-) diff --git a/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h b/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h index 439ddffe5e5..3c0e5974d16 100644 --- a/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h +++ b/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h @@ -3,370 +3,84 @@ #include #include #include -#include #include -#include -#include -#include - -#include namespace DB { +class FunctionTreeNode : public AbstractFunction +{ +public: + class ArgumentTreeNode : public Argument + { + public: + explicit ArgumentTreeNode(const IQueryTreeNode * argument_) : argument(argument_) {} + std::unique_ptr getFunction() const override { return std::make_unique(*argument->as()); } + bool isIdentifier() const override { return argument->as(); } + bool tryGetString(String * res, bool allow_identifier = true) const override + { + if (const auto * literal = argument->as()) + { + if (literal->getValue().getType() != Field::Types::String) + return false; + if (res) + *res = literal->getValue().safeGet(); + return true; + } + + if (allow_identifier) + { + if (const auto * id = argument->as()) + { + if (res) + *res = id->getIdentifier().getFullName(); + return true; + } + } + + return false; + } + private: + const IQueryTreeNode * argument = nullptr; + }; + + class ArgumentsTreeNode : public Arguments + { + public: + explicit ArgumentsTreeNode(const QueryTreeNodes * arguments_) : arguments(arguments_) {} + size_t size() const override { return arguments ? arguments->size() : 0; } + std::unique_ptr at(size_t n) const override { return std::make_unique(arguments->at(n).get()); } + private: + const QueryTreeNodes * arguments = nullptr; + }; + + explicit FunctionTreeNode(const FunctionNode & function_) : function(&function_) + { + if (const auto & nodes = function->getArguments().getNodes(); !nodes.empty()) + arguments = std::make_unique(&nodes); + } + String name() const override { return function->getFunctionName(); } +private: + const FunctionNode * function = nullptr; +}; + /// Finds arguments of a specified function which should not be displayed for most users for security reasons. /// That involves passwords and secret keys. -class FunctionSecretArgumentsFinderTreeNode +class FunctionSecretArgumentsFinderTreeNode : public FunctionSecretArgumentsFinder { public: - explicit FunctionSecretArgumentsFinderTreeNode(const FunctionNode & function_) : function(function_), arguments(function.getArguments()) + explicit FunctionSecretArgumentsFinderTreeNode(const FunctionNode & function_) + : FunctionSecretArgumentsFinder(std::make_unique(function_)) { - if (arguments.getNodes().empty()) + if (!function->hasArguments()) return; - findFunctionSecretArguments(); + findOrdinaryFunctionSecretArguments(); } - struct Result - { - /// Result constructed by default means no arguments will be hidden. - size_t start = static_cast(-1); - size_t count = 0; /// Mostly it's either 0 or 1. There are only a few cases where `count` can be greater than 1 (e.g. see `encrypt`). - /// In all known cases secret arguments are consecutive - bool are_named = false; /// Arguments like `password = 'password'` are considered as named arguments. - /// E.g. "headers" in `url('..', headers('foo' = '[HIDDEN]'))` - std::vector nested_maps; - - bool hasSecrets() const - { - return count != 0 || !nested_maps.empty(); - } - }; - FunctionSecretArgumentsFinder::Result getResult() const { return result; } - -private: - const FunctionNode & function; - const ListNode & arguments; - FunctionSecretArgumentsFinder::Result result; - - void markSecretArgument(size_t index, bool argument_is_named = false) - { - if (index >= arguments.getNodes().size()) - return; - if (!result.count) - { - result.start = index; - result.are_named = argument_is_named; - } - chassert(index >= result.start); /// We always check arguments consecutively - result.count = index + 1 - result.start; - if (!argument_is_named) - result.are_named = false; - } - - void findFunctionSecretArguments() - { - const auto & name = function.getFunctionName(); - - if ((name == "mysql") || (name == "postgresql") || (name == "mongodb")) - { - /// mysql('host:port', 'database', 'table', 'user', 'password', ...) - /// postgresql('host:port', 'database', 'table', 'user', 'password', ...) - /// mongodb('host:port', 'database', 'collection', 'user', 'password', ...) - findMySQLFunctionSecretArguments(); - } - else if ((name == "s3") || (name == "cosn") || (name == "oss") || - (name == "deltaLake") || (name == "hudi") || (name == "iceberg")) - { - /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) - findS3FunctionSecretArguments(/* is_cluster_function= */ false); - } - else if (name == "s3Cluster") - { - /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', ...) - findS3FunctionSecretArguments(/* is_cluster_function= */ true); - } - else if ((name == "remote") || (name == "remoteSecure")) - { - /// remote('addresses_expr', 'db', 'table', 'user', 'password', ...) - findRemoteFunctionSecretArguments(); - } - else if ((name == "encrypt") || (name == "decrypt") || - (name == "aes_encrypt_mysql") || (name == "aes_decrypt_mysql") || - (name == "tryDecrypt")) - { - /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) - findEncryptionFunctionSecretArguments(); - } - else if (name == "url") - { - findURLSecretArguments(); - } - } - - void findMySQLFunctionSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// mysql(named_collection, ..., password = 'password', ...) - findSecretNamedArgument("password", 1); - } - else - { - /// mysql('host:port', 'database', 'table', 'user', 'password', ...) - markSecretArgument(4); - } - } - - /// Returns the number of arguments excluding "headers" and "extra_credentials" (which should - /// always be at the end). Marks "headers" as secret, if found. - size_t excludeS3OrURLNestedMaps() - { - const auto & nodes = arguments.getNodes(); - size_t count = nodes.size(); - while (count > 0) - { - const FunctionNode * f = nodes.at(count - 1)->as(); - if (!f) - break; - if (f->getFunctionName() == "headers") - result.nested_maps.push_back(f->getFunctionName()); - else if (f->getFunctionName() != "extra_credentials") - break; - count -= 1; - } - return count; - } - - void findS3FunctionSecretArguments(bool is_cluster_function) - { - /// s3Cluster('cluster_name', 'url', ...) has 'url' as its second argument. - size_t url_arg_idx = is_cluster_function ? 1 : 0; - - if (!is_cluster_function && isNamedCollectionName(0)) - { - /// s3(named_collection, ..., secret_access_key = 'secret_access_key', ...) - findSecretNamedArgument("secret_access_key", 1); - return; - } - - /// We should check other arguments first because we don't need to do any replacement in case of - /// s3('url', NOSIGN, 'format' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) - /// s3('url', 'format', 'structure' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) - size_t count = excludeS3OrURLNestedMaps(); - if ((url_arg_idx + 3 <= count) && (count <= url_arg_idx + 4)) - { - String second_arg; - if (tryGetStringFromArgument(url_arg_idx + 1, &second_arg)) - { - if (boost::iequals(second_arg, "NOSIGN")) - return; /// The argument after 'url' is "NOSIGN". - - if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) - return; /// The argument after 'url' is a format: s3('url', 'format', ...) - } - } - - /// We're going to replace 'aws_secret_access_key' with '[HIDDEN]' for the following signatures: - /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) - /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') - if (url_arg_idx + 2 < count) - markSecretArgument(url_arg_idx + 2); - } - - void findURLSecretArguments() - { - if (!isNamedCollectionName(0)) - excludeS3OrURLNestedMaps(); - } - - bool tryGetStringFromArgument(size_t arg_idx, String * res, bool allow_identifier = true) const - { - if (arg_idx >= arguments.getNodes().size()) - return false; - - return tryGetStringFromArgument(arguments.getNodes()[arg_idx], res, allow_identifier); - } - - static bool tryGetStringFromArgument(const QueryTreeNodePtr argument, String * res, bool allow_identifier = true) - { - if (const auto * literal = argument->as()) - { - if (literal->getValue().getType() != Field::Types::String) - return false; - if (res) - *res = literal->getValue().safeGet(); - return true; - } - - if (allow_identifier) - { - if (const auto * id = argument->as()) - { - if (res) - *res = id->getIdentifier().getFullName(); - return true; - } - } - - return false; - } - - void findRemoteFunctionSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// remote(named_collection, ..., password = 'password', ...) - findSecretNamedArgument("password", 1); - return; - } - - /// We're going to replace 'password' with '[HIDDEN'] for the following signatures: - /// remote('addresses_expr', db.table, 'user' [, 'password'] [, sharding_key]) - /// remote('addresses_expr', 'db', 'table', 'user' [, 'password'] [, sharding_key]) - /// remote('addresses_expr', table_function(), 'user' [, 'password'] [, sharding_key]) - - /// But we should check the number of arguments first because we don't need to do any replacements in case of - /// remote('addresses_expr', db.table) - if (arguments.getNodes().size() < 3) - return; - - size_t arg_num = 1; - - /// Skip 1 or 2 arguments with table_function() or db.table or 'db', 'table'. - const auto * table_function = arguments.getNodes()[arg_num]->as(); - if (table_function && KnownTableFunctionNames::instance().exists(table_function->getFunctionName())) - { - ++arg_num; - } - else - { - std::optional database; - std::optional qualified_table_name; - if (!tryGetDatabaseNameOrQualifiedTableName(arg_num, database, qualified_table_name)) - { - /// We couldn't evaluate the argument so we don't know whether it is 'db.table' or just 'db'. - /// Hence we can't figure out whether we should skip one argument 'user' or two arguments 'table', 'user' - /// before the argument 'password'. So it's safer to wipe two arguments just in case. - /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string - /// before wiping it (because the `password` argument is always a literal string). - if (tryGetStringFromArgument(arg_num + 2, nullptr, /* allow_identifier= */ false)) - { - /// Wipe either `password` or `user`. - markSecretArgument(arg_num + 2); - } - if (tryGetStringFromArgument(arg_num + 3, nullptr, /* allow_identifier= */ false)) - { - /// Wipe either `password` or `sharding_key`. - markSecretArgument(arg_num + 3); - } - return; - } - - /// Skip the current argument (which is either a database name or a qualified table name). - ++arg_num; - if (database) - { - /// Skip the 'table' argument if the previous argument was a database name. - ++arg_num; - } - } - - /// Skip username. - ++arg_num; - - /// Do our replacement: - /// remote('addresses_expr', db.table, 'user', 'password', ...) -> remote('addresses_expr', db.table, 'user', '[HIDDEN]', ...) - /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string - /// before wiping it (because the `password` argument is always a literal string). - bool can_be_password = tryGetStringFromArgument(arg_num, nullptr, /* allow_identifier= */ false); - if (can_be_password) - markSecretArgument(arg_num); - } - - /// Tries to get either a database name or a qualified table name from an argument. - /// Empty string is also allowed (it means the default database). - /// The function is used by findRemoteFunctionSecretArguments() to determine how many arguments to skip before a password. - bool tryGetDatabaseNameOrQualifiedTableName( - size_t arg_idx, - std::optional & res_database, - std::optional & res_qualified_table_name) const - { - res_database.reset(); - res_qualified_table_name.reset(); - - String str; - if (!tryGetStringFromArgument(arg_idx, &str, /* allow_identifier= */ true)) - return false; - - if (str.empty()) - { - res_database = ""; - return true; - } - - auto qualified_table_name = QualifiedTableName::tryParseFromString(str); - if (!qualified_table_name) - return false; - - if (qualified_table_name->database.empty()) - res_database = std::move(qualified_table_name->table); - else - res_qualified_table_name = std::move(qualified_table_name); - return true; - } - - void findEncryptionFunctionSecretArguments() - { - if (arguments.getNodes().empty()) - return; - - /// We replace all arguments after 'mode' with '[HIDDEN]': - /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) -> encrypt('mode', '[HIDDEN]') - result.start = 1; - result.count = arguments.getNodes().size() - 1; - } - - - /// Whether a specified argument can be the name of a named collection? - bool isNamedCollectionName(size_t arg_idx) const - { - if (arguments.getNodes().size() <= arg_idx) - return false; - - const auto * identifier = arguments.getNodes()[arg_idx]->as(); - return identifier != nullptr; - } - - /// Looks for a secret argument with a specified name. This function looks for arguments in format `key=value` where the key is specified. - void findSecretNamedArgument(const std::string_view & key, size_t start = 0) - { - for (size_t i = start; i < arguments.getNodes().size(); ++i) - { - const auto & argument = arguments.getNodes()[i]; - const auto * equals_func = argument->as(); - if (!equals_func || (equals_func->getFunctionName() != "equals")) - continue; - - const auto * expr_list = equals_func->getArguments().as(); - if (!expr_list) - continue; - - const auto & equal_args = expr_list->getNodes(); - if (equal_args.size() != 2) - continue; - - String found_key; - if (!tryGetStringFromArgument(equal_args[0], &found_key)) - continue; - - if (found_key == key) - markSecretArgument(i, /* argument_is_named= */ true); - } - } }; } diff --git a/src/Parsers/FunctionSecretArgumentsFinder.h b/src/Parsers/FunctionSecretArgumentsFinder.h index 002ad94f6ea..77facd715f8 100644 --- a/src/Parsers/FunctionSecretArgumentsFinder.h +++ b/src/Parsers/FunctionSecretArgumentsFinder.h @@ -1,10 +1,42 @@ #pragma once -#include +#include +#include +#include +#include + namespace DB { +class AbstractFunction +{ + friend class FunctionSecretArgumentsFinder; +public: + class Argument + { + public: + virtual ~Argument() = default; + virtual std::unique_ptr getFunction() const = 0; + virtual bool isIdentifier() const = 0; + virtual bool tryGetString(String * res, bool allow_identifier) const = 0; + }; + class Arguments + { + public: + virtual ~Arguments() = default; + virtual size_t size() const = 0; + virtual std::unique_ptr at(size_t n) const = 0; + }; + + virtual ~AbstractFunction() = default; + virtual String name() const = 0; + bool hasArguments() const { return !!arguments; } + +protected: + std::unique_ptr arguments; +}; + class FunctionSecretArgumentsFinder { public: @@ -23,6 +55,484 @@ public: return count != 0 || !nested_maps.empty(); } }; + + explicit FunctionSecretArgumentsFinder(std::unique_ptr && function_) : function(std::move(function_)) {} + + FunctionSecretArgumentsFinder::Result getResult() const { return result; } + +protected: + const std::unique_ptr function; + Result result; + + void markSecretArgument(size_t index, bool argument_is_named = false) + { + if (index >= function->arguments->size()) + return; + if (!result.count) + { + result.start = index; + result.are_named = argument_is_named; + } + chassert(index >= result.start); /// We always check arguments consecutively + result.count = index + 1 - result.start; + if (!argument_is_named) + result.are_named = false; + } + + void findOrdinaryFunctionSecretArguments() + { + if ((function->name() == "mysql") || (function->name() == "postgresql") || (function->name() == "mongodb")) + { + /// mysql('host:port', 'database', 'table', 'user', 'password', ...) + /// postgresql('host:port', 'database', 'table', 'user', 'password', ...) + /// mongodb('host:port', 'database', 'collection', 'user', 'password', ...) + findMySQLFunctionSecretArguments(); + } + else if ((function->name() == "s3") || (function->name() == "cosn") || (function->name() == "oss") || + (function->name() == "deltaLake") || (function->name() == "hudi") || (function->name() == "iceberg")) + { + /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) + findS3FunctionSecretArguments(/* is_cluster_function= */ false); + } + else if (function->name() == "s3Cluster") + { + /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', ...) + findS3FunctionSecretArguments(/* is_cluster_function= */ true); + } + else if (function->name() == "azureBlobStorage") + { + /// azureBlobStorage(connection_string|storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure) + findAzureBlobStorageFunctionSecretArguments(/* is_cluster_function= */ false); + } + else if (function->name() == "azureBlobStorageCluster") + { + /// azureBlobStorageCluster(cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]) + findAzureBlobStorageFunctionSecretArguments(/* is_cluster_function= */ true); + } + else if ((function->name() == "remote") || (function->name() == "remoteSecure")) + { + /// remote('addresses_expr', 'db', 'table', 'user', 'password', ...) + findRemoteFunctionSecretArguments(); + } + else if ((function->name() == "encrypt") || (function->name() == "decrypt") || + (function->name() == "aes_encrypt_mysql") || (function->name() == "aes_decrypt_mysql") || + (function->name() == "tryDecrypt")) + { + /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) + findEncryptionFunctionSecretArguments(); + } + else if (function->name() == "url") + { + findURLSecretArguments(); + } + } + + void findMySQLFunctionSecretArguments() + { + if (isNamedCollectionName(0)) + { + /// mysql(named_collection, ..., password = 'password', ...) + findSecretNamedArgument("password", 1); + } + else + { + /// mysql('host:port', 'database', 'table', 'user', 'password', ...) + markSecretArgument(4); + } + } + + /// Returns the number of arguments excluding "headers" and "extra_credentials" (which should + /// always be at the end). Marks "headers" as secret, if found. + size_t excludeS3OrURLNestedMaps() + { + size_t count = function->arguments->size(); + while (count > 0) + { + const auto f = function->arguments->at(count - 1)->getFunction(); + if (!f) + break; + if (f->name() == "headers") + result.nested_maps.push_back(f->name()); + else if (f->name() != "extra_credentials") + break; + count -= 1; + } + return count; + } + + void findS3FunctionSecretArguments(bool is_cluster_function) + { + /// s3Cluster('cluster_name', 'url', ...) has 'url' as its second argument. + size_t url_arg_idx = is_cluster_function ? 1 : 0; + + if (!is_cluster_function && isNamedCollectionName(0)) + { + /// s3(named_collection, ..., secret_access_key = 'secret_access_key', ...) + findSecretNamedArgument("secret_access_key", 1); + return; + } + + /// We should check other arguments first because we don't need to do any replacement in case of + /// s3('url', NOSIGN, 'format' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) + /// s3('url', 'format', 'structure' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) + size_t count = excludeS3OrURLNestedMaps(); + if ((url_arg_idx + 3 <= count) && (count <= url_arg_idx + 4)) + { + String second_arg; + if (tryGetStringFromArgument(url_arg_idx + 1, &second_arg)) + { + if (boost::iequals(second_arg, "NOSIGN")) + return; /// The argument after 'url' is "NOSIGN". + + if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) + return; /// The argument after 'url' is a format: s3('url', 'format', ...) + } + } + + /// We're going to replace 'aws_secret_access_key' with '[HIDDEN]' for the following signatures: + /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) + /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') + if (url_arg_idx + 2 < count) + markSecretArgument(url_arg_idx + 2); + } + + void findAzureBlobStorageFunctionSecretArguments(bool is_cluster_function) + { + /// azureBlobStorage('cluster_name', 'conn_string/storage_account_url', ...) has 'conn_string/storage_account_url' as its second argument. + size_t url_arg_idx = is_cluster_function ? 1 : 0; + + if (!is_cluster_function && isNamedCollectionName(0)) + { + /// azureBlobStorage(named_collection, ..., account_key = 'account_key', ...) + findSecretNamedArgument("account_key", 1); + return; + } + else if (is_cluster_function && isNamedCollectionName(1)) + { + /// azureBlobStorageCluster(cluster, named_collection, ..., account_key = 'account_key', ...) + findSecretNamedArgument("account_key", 2); + return; + } + + /// We should check other arguments first because we don't need to do any replacement in case storage_account_url is not used + /// azureBlobStorage(connection_string|storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure) + /// azureBlobStorageCluster(cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]) + size_t count = function->arguments->size(); + if ((url_arg_idx + 4 <= count) && (count <= url_arg_idx + 7)) + { + String second_arg; + if (tryGetStringFromArgument(url_arg_idx + 3, &second_arg)) + { + if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) + return; /// The argument after 'url' is a format: s3('url', 'format', ...) + } + } + + /// We're going to replace 'account_key' with '[HIDDEN]' if account_key is used in the signature + if (url_arg_idx + 4 < count) + markSecretArgument(url_arg_idx + 4); + } + + void findURLSecretArguments() + { + if (!isNamedCollectionName(0)) + excludeS3OrURLNestedMaps(); + } + + bool tryGetStringFromArgument(size_t arg_idx, String * res, bool allow_identifier = true) const + { + if (arg_idx >= function->arguments->size()) + return false; + + return tryGetStringFromArgument(*function->arguments->at(arg_idx), res, allow_identifier); + } + + static bool tryGetStringFromArgument(const AbstractFunction::Argument & argument, String * res, bool allow_identifier = true) + { + return argument.tryGetString(res, allow_identifier); + } + + void findRemoteFunctionSecretArguments() + { + if (isNamedCollectionName(0)) + { + /// remote(named_collection, ..., password = 'password', ...) + findSecretNamedArgument("password", 1); + return; + } + + /// We're going to replace 'password' with '[HIDDEN'] for the following signatures: + /// remote('addresses_expr', db.table, 'user' [, 'password'] [, sharding_key]) + /// remote('addresses_expr', 'db', 'table', 'user' [, 'password'] [, sharding_key]) + /// remote('addresses_expr', table_function(), 'user' [, 'password'] [, sharding_key]) + + /// But we should check the number of arguments first because we don't need to do any replacements in case of + /// remote('addresses_expr', db.table) + if (function->arguments->size() < 3) + return; + + size_t arg_num = 1; + + /// Skip 1 or 2 arguments with table_function() or db.table or 'db', 'table'. + auto table_function = function->arguments->at(arg_num)->getFunction(); + if (table_function && KnownTableFunctionNames::instance().exists(table_function->name())) + { + ++arg_num; + } + else + { + std::optional database; + std::optional qualified_table_name; + if (!tryGetDatabaseNameOrQualifiedTableName(arg_num, database, qualified_table_name)) + { + /// We couldn't evaluate the argument so we don't know whether it is 'db.table' or just 'db'. + /// Hence we can't figure out whether we should skip one argument 'user' or two arguments 'table', 'user' + /// before the argument 'password'. So it's safer to wipe two arguments just in case. + /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string + /// before wiping it (because the `password` argument is always a literal string). + if (tryGetStringFromArgument(arg_num + 2, nullptr, /* allow_identifier= */ false)) + { + /// Wipe either `password` or `user`. + markSecretArgument(arg_num + 2); + } + if (tryGetStringFromArgument(arg_num + 3, nullptr, /* allow_identifier= */ false)) + { + /// Wipe either `password` or `sharding_key`. + markSecretArgument(arg_num + 3); + } + return; + } + + /// Skip the current argument (which is either a database name or a qualified table name). + ++arg_num; + if (database) + { + /// Skip the 'table' argument if the previous argument was a database name. + ++arg_num; + } + } + + /// Skip username. + ++arg_num; + + /// Do our replacement: + /// remote('addresses_expr', db.table, 'user', 'password', ...) -> remote('addresses_expr', db.table, 'user', '[HIDDEN]', ...) + /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string + /// before wiping it (because the `password` argument is always a literal string). + bool can_be_password = tryGetStringFromArgument(arg_num, nullptr, /* allow_identifier= */ false); + if (can_be_password) + markSecretArgument(arg_num); + } + + /// Tries to get either a database name or a qualified table name from an argument. + /// Empty string is also allowed (it means the default database). + /// The function is used by findRemoteFunctionSecretArguments() to determine how many arguments to skip before a password. + bool tryGetDatabaseNameOrQualifiedTableName( + size_t arg_idx, + std::optional & res_database, + std::optional & res_qualified_table_name) const + { + res_database.reset(); + res_qualified_table_name.reset(); + + String str; + if (!tryGetStringFromArgument(arg_idx, &str, /* allow_identifier= */ true)) + return false; + + if (str.empty()) + { + res_database = ""; + return true; + } + + auto qualified_table_name = QualifiedTableName::tryParseFromString(str); + if (!qualified_table_name) + return false; + + if (qualified_table_name->database.empty()) + res_database = std::move(qualified_table_name->table); + else + res_qualified_table_name = std::move(qualified_table_name); + return true; + } + + void findEncryptionFunctionSecretArguments() + { + if (function->arguments->size() == 0) + return; + + /// We replace all arguments after 'mode' with '[HIDDEN]': + /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) -> encrypt('mode', '[HIDDEN]') + result.start = 1; + result.count = function->arguments->size() - 1; + } + + void findTableEngineSecretArguments() + { + const String & engine_name = function->name(); + if (engine_name == "ExternalDistributed") + { + /// ExternalDistributed('engine', 'host:port', 'database', 'table', 'user', 'password') + findExternalDistributedTableEngineSecretArguments(); + } + else if ((engine_name == "MySQL") || (engine_name == "PostgreSQL") || + (engine_name == "MaterializedPostgreSQL") || (engine_name == "MongoDB")) + { + /// MySQL('host:port', 'database', 'table', 'user', 'password', ...) + /// PostgreSQL('host:port', 'database', 'table', 'user', 'password', ...) + /// MaterializedPostgreSQL('host:port', 'database', 'table', 'user', 'password', ...) + /// MongoDB('host:port', 'database', 'collection', 'user', 'password', ...) + findMySQLFunctionSecretArguments(); + } + else if ((engine_name == "S3") || (engine_name == "COSN") || (engine_name == "OSS") || + (engine_name == "DeltaLake") || (engine_name == "Hudi") || (engine_name == "Iceberg") || (engine_name == "S3Queue")) + { + /// S3('url', ['aws_access_key_id', 'aws_secret_access_key',] ...) + findS3TableEngineSecretArguments(); + } + else if (engine_name == "URL") + { + findURLSecretArguments(); + } + } + + void findExternalDistributedTableEngineSecretArguments() + { + if (isNamedCollectionName(1)) + { + /// ExternalDistributed('engine', named_collection, ..., password = 'password', ...) + findSecretNamedArgument("password", 2); + } + else + { + /// ExternalDistributed('engine', 'host:port', 'database', 'table', 'user', 'password') + markSecretArgument(5); + } + } + + void findS3TableEngineSecretArguments() + { + if (isNamedCollectionName(0)) + { + /// S3(named_collection, ..., secret_access_key = 'secret_access_key') + findSecretNamedArgument("secret_access_key", 1); + return; + } + + /// We should check other arguments first because we don't need to do any replacement in case of + /// S3('url', NOSIGN, 'format' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) + /// S3('url', 'format', 'compression' [, extra_credentials(..)] [, headers(..)]) + size_t count = excludeS3OrURLNestedMaps(); + if ((3 <= count) && (count <= 4)) + { + String second_arg; + if (tryGetStringFromArgument(1, &second_arg)) + { + if (boost::iequals(second_arg, "NOSIGN")) + return; /// The argument after 'url' is "NOSIGN". + + if (count == 3) + { + if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) + return; /// The argument after 'url' is a format: S3('url', 'format', ...) + } + } + } + + /// We replace 'aws_secret_access_key' with '[HIDDEN]' for the following signatures: + /// S3('url', 'aws_access_key_id', 'aws_secret_access_key') + /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format') + /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') + if (2 < count) + markSecretArgument(2); + } + + void findDatabaseEngineSecretArguments() + { + const String & engine_name = function->name(); + if ((engine_name == "MySQL") || (engine_name == "MaterializeMySQL") || + (engine_name == "MaterializedMySQL") || (engine_name == "PostgreSQL") || + (engine_name == "MaterializedPostgreSQL")) + { + /// MySQL('host:port', 'database', 'user', 'password') + /// PostgreSQL('host:port', 'database', 'user', 'password') + findMySQLDatabaseSecretArguments(); + } + else if (engine_name == "S3") + { + /// S3('url', 'access_key_id', 'secret_access_key') + findS3DatabaseSecretArguments(); + } + } + + void findMySQLDatabaseSecretArguments() + { + if (isNamedCollectionName(0)) + { + /// MySQL(named_collection, ..., password = 'password', ...) + findSecretNamedArgument("password", 1); + } + else + { + /// MySQL('host:port', 'database', 'user', 'password') + markSecretArgument(3); + } + } + + void findS3DatabaseSecretArguments() + { + if (isNamedCollectionName(0)) + { + /// S3(named_collection, ..., secret_access_key = 'password', ...) + findSecretNamedArgument("secret_access_key", 1); + } + else + { + /// S3('url', 'access_key_id', 'secret_access_key') + markSecretArgument(2); + } + } + + void findBackupNameSecretArguments() + { + const String & engine_name = function->name(); + if (engine_name == "S3") + { + /// BACKUP ... TO S3(url, [aws_access_key_id, aws_secret_access_key]) + markSecretArgument(2); + } + } + + /// Whether a specified argument can be the name of a named collection? + bool isNamedCollectionName(size_t arg_idx) const + { + if (function->arguments->size() <= arg_idx) + return false; + + return function->arguments->at(arg_idx)->isIdentifier(); + } + + /// Looks for a secret argument with a specified name. This function looks for arguments in format `key=value` where the key is specified. + void findSecretNamedArgument(const std::string_view & key, size_t start = 0) + { + for (size_t i = start; i < function->arguments->size(); ++i) + { + const auto & argument = function->arguments->at(i); + const auto equals_func = argument->getFunction(); + if (!equals_func || (equals_func->name() != "equals")) + continue; + + if (!equals_func->arguments || equals_func->arguments->size() != 2) + continue; + + String found_key; + if (!tryGetStringFromArgument(*equals_func->arguments->at(0), &found_key)) + continue; + + if (found_key == key) + markSecretArgument(i, /* argument_is_named= */ true); + } + } }; } diff --git a/src/Parsers/FunctionSecretArgumentsFinderAST.h b/src/Parsers/FunctionSecretArgumentsFinderAST.h index 94da30922cc..d2d7a63aff1 100644 --- a/src/Parsers/FunctionSecretArgumentsFinderAST.h +++ b/src/Parsers/FunctionSecretArgumentsFinderAST.h @@ -1,35 +1,91 @@ #pragma once #include -#include #include #include #include -#include - -#include namespace DB { - -/// Finds arguments of a specified function which should not be displayed for most users for security reasons. -/// That involves passwords and secret keys. -class FunctionSecretArgumentsFinderAST +class FunctionAST : public AbstractFunction { public: - explicit FunctionSecretArgumentsFinderAST(const ASTFunction & function_) : function(function_) + class ArgumentAST : public Argument + { + public: + explicit ArgumentAST(const IAST * argument_) : argument(argument_) {} + std::unique_ptr getFunction() const override { return std::make_unique(*argument->as()); } + bool isIdentifier() const override { return argument->as(); } + bool tryGetString(String * res, bool allow_identifier = true) const override + { + if (const auto * literal = argument->as()) + { + if (literal->value.getType() != Field::Types::String) + return false; + if (res) + *res = literal->value.safeGet(); + return true; + } + + if (allow_identifier) + { + if (const auto * id = argument->as()) + { + if (res) + *res = id->name(); + return true; + } + } + + return false; + } + private: + const IAST * argument = nullptr; + }; + class ArgumentsAST : public Arguments { - if (!function.arguments) + public: + explicit ArgumentsAST(const ASTs * arguments_) : arguments(arguments_) {} + size_t size() const override { return arguments ? arguments->size() : 0; } + std::unique_ptr at(size_t n) const override + { + return std::make_unique(arguments->at(n).get()); + } + private: + const ASTs * arguments = nullptr; + }; + + explicit FunctionAST(const ASTFunction & function_) : function(&function_) + { + if (!function->arguments) return; - const auto * expr_list = function.arguments->as(); + const auto * expr_list = function->arguments->as(); if (!expr_list) return; - arguments = &expr_list->children; - switch (function.kind) + arguments = std::make_unique(&expr_list->children); + } + + String name() const override { return function->name; } +private: + const ASTFunction * function = nullptr; +}; + +/// Finds arguments of a specified function which should not be displayed for most users for security reasons. +/// That involves passwords and secret keys. +class FunctionSecretArgumentsFinderAST : public FunctionSecretArgumentsFinder +{ +public: + explicit FunctionSecretArgumentsFinderAST(const ASTFunction & function_) + : FunctionSecretArgumentsFinder(std::make_unique(function_)) + { + if (!function->hasArguments()) + return; + + switch (function_.kind) { case ASTFunction::Kind::ORDINARY_FUNCTION: findOrdinaryFunctionSecretArguments(); break; case ASTFunction::Kind::WINDOW_FUNCTION: break; @@ -43,506 +99,7 @@ public: } FunctionSecretArgumentsFinder::Result getResult() const { return result; } - -private: - const ASTFunction & function; - const ASTs * arguments = nullptr; - FunctionSecretArgumentsFinder::Result result; - - void markSecretArgument(size_t index, bool argument_is_named = false) - { - if (index >= arguments->size()) - return; - if (!result.count) - { - result.start = index; - result.are_named = argument_is_named; - } - chassert(index >= result.start); /// We always check arguments consecutively - result.count = index + 1 - result.start; - if (!argument_is_named) - result.are_named = false; - } - - void findOrdinaryFunctionSecretArguments() - { - if ((function.name == "mysql") || (function.name == "postgresql") || (function.name == "mongodb")) - { - /// mysql('host:port', 'database', 'table', 'user', 'password', ...) - /// postgresql('host:port', 'database', 'table', 'user', 'password', ...) - /// mongodb('host:port', 'database', 'collection', 'user', 'password', ...) - findMySQLFunctionSecretArguments(); - } - else if ((function.name == "s3") || (function.name == "cosn") || (function.name == "oss") || - (function.name == "deltaLake") || (function.name == "hudi") || (function.name == "iceberg")) - { - /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) - findS3FunctionSecretArguments(/* is_cluster_function= */ false); - } - else if (function.name == "s3Cluster") - { - /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', ...) - findS3FunctionSecretArguments(/* is_cluster_function= */ true); - } - else if (function.name == "azureBlobStorage") - { - /// azureBlobStorage(connection_string|storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure) - findAzureBlobStorageFunctionSecretArguments(/* is_cluster_function= */ false); - } - else if (function.name == "azureBlobStorageCluster") - { - /// azureBlobStorageCluster(cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]) - findAzureBlobStorageFunctionSecretArguments(/* is_cluster_function= */ true); - } - else if ((function.name == "remote") || (function.name == "remoteSecure")) - { - /// remote('addresses_expr', 'db', 'table', 'user', 'password', ...) - findRemoteFunctionSecretArguments(); - } - else if ((function.name == "encrypt") || (function.name == "decrypt") || - (function.name == "aes_encrypt_mysql") || (function.name == "aes_decrypt_mysql") || - (function.name == "tryDecrypt")) - { - /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) - findEncryptionFunctionSecretArguments(); - } - else if (function.name == "url") - { - findURLSecretArguments(); - } - } - - void findMySQLFunctionSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// mysql(named_collection, ..., password = 'password', ...) - findSecretNamedArgument("password", 1); - } - else - { - /// mysql('host:port', 'database', 'table', 'user', 'password', ...) - markSecretArgument(4); - } - } - - /// Returns the number of arguments excluding "headers" and "extra_credentials" (which should - /// always be at the end). Marks "headers" as secret, if found. - size_t excludeS3OrURLNestedMaps() - { - size_t count = arguments->size(); - while (count > 0) - { - const ASTFunction * f = arguments->at(count - 1)->as(); - if (!f) - break; - if (f->name == "headers") - result.nested_maps.push_back(f->name); - else if (f->name != "extra_credentials") - break; - count -= 1; - } - return count; - } - - void findS3FunctionSecretArguments(bool is_cluster_function) - { - /// s3Cluster('cluster_name', 'url', ...) has 'url' as its second argument. - size_t url_arg_idx = is_cluster_function ? 1 : 0; - - if (!is_cluster_function && isNamedCollectionName(0)) - { - /// s3(named_collection, ..., secret_access_key = 'secret_access_key', ...) - findSecretNamedArgument("secret_access_key", 1); - return; - } - - /// We should check other arguments first because we don't need to do any replacement in case of - /// s3('url', NOSIGN, 'format' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) - /// s3('url', 'format', 'structure' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) - size_t count = excludeS3OrURLNestedMaps(); - if ((url_arg_idx + 3 <= count) && (count <= url_arg_idx + 4)) - { - String second_arg; - if (tryGetStringFromArgument(url_arg_idx + 1, &second_arg)) - { - if (boost::iequals(second_arg, "NOSIGN")) - return; /// The argument after 'url' is "NOSIGN". - - if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) - return; /// The argument after 'url' is a format: s3('url', 'format', ...) - } - } - - /// We're going to replace 'aws_secret_access_key' with '[HIDDEN]' for the following signatures: - /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) - /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') - if (url_arg_idx + 2 < count) - markSecretArgument(url_arg_idx + 2); - } - - void findAzureBlobStorageFunctionSecretArguments(bool is_cluster_function) - { - /// azureBlobStorage('cluster_name', 'conn_string/storage_account_url', ...) has 'conn_string/storage_account_url' as its second argument. - size_t url_arg_idx = is_cluster_function ? 1 : 0; - - if (!is_cluster_function && isNamedCollectionName(0)) - { - /// azureBlobStorage(named_collection, ..., account_key = 'account_key', ...) - findSecretNamedArgument("account_key", 1); - return; - } - else if (is_cluster_function && isNamedCollectionName(1)) - { - /// azureBlobStorageCluster(cluster, named_collection, ..., account_key = 'account_key', ...) - findSecretNamedArgument("account_key", 2); - return; - } - - /// We should check other arguments first because we don't need to do any replacement in case storage_account_url is not used - /// azureBlobStorage(connection_string|storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure) - /// azureBlobStorageCluster(cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]) - size_t count = arguments->size(); - if ((url_arg_idx + 4 <= count) && (count <= url_arg_idx + 7)) - { - String second_arg; - if (tryGetStringFromArgument(url_arg_idx + 3, &second_arg)) - { - if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) - return; /// The argument after 'url' is a format: s3('url', 'format', ...) - } - } - - /// We're going to replace 'account_key' with '[HIDDEN]' if account_key is used in the signature - if (url_arg_idx + 4 < count) - markSecretArgument(url_arg_idx + 4); - } - - void findURLSecretArguments() - { - if (!isNamedCollectionName(0)) - excludeS3OrURLNestedMaps(); - } - - bool tryGetStringFromArgument(size_t arg_idx, String * res, bool allow_identifier = true) const - { - if (arg_idx >= arguments->size()) - return false; - - return tryGetStringFromArgument(*(*arguments)[arg_idx], res, allow_identifier); - } - - static bool tryGetStringFromArgument(const IAST & argument, String * res, bool allow_identifier = true) - { - if (const auto * literal = argument.as()) - { - if (literal->value.getType() != Field::Types::String) - return false; - if (res) - *res = literal->value.safeGet(); - return true; - } - - if (allow_identifier) - { - if (const auto * id = argument.as()) - { - if (res) - *res = id->name(); - return true; - } - } - - return false; - } - - void findRemoteFunctionSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// remote(named_collection, ..., password = 'password', ...) - findSecretNamedArgument("password", 1); - return; - } - - /// We're going to replace 'password' with '[HIDDEN'] for the following signatures: - /// remote('addresses_expr', db.table, 'user' [, 'password'] [, sharding_key]) - /// remote('addresses_expr', 'db', 'table', 'user' [, 'password'] [, sharding_key]) - /// remote('addresses_expr', table_function(), 'user' [, 'password'] [, sharding_key]) - - /// But we should check the number of arguments first because we don't need to do any replacements in case of - /// remote('addresses_expr', db.table) - if (arguments->size() < 3) - return; - - size_t arg_num = 1; - - /// Skip 1 or 2 arguments with table_function() or db.table or 'db', 'table'. - const auto * table_function = (*arguments)[arg_num]->as(); - if (table_function && KnownTableFunctionNames::instance().exists(table_function->name)) - { - ++arg_num; - } - else - { - std::optional database; - std::optional qualified_table_name; - if (!tryGetDatabaseNameOrQualifiedTableName(arg_num, database, qualified_table_name)) - { - /// We couldn't evaluate the argument so we don't know whether it is 'db.table' or just 'db'. - /// Hence we can't figure out whether we should skip one argument 'user' or two arguments 'table', 'user' - /// before the argument 'password'. So it's safer to wipe two arguments just in case. - /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string - /// before wiping it (because the `password` argument is always a literal string). - if (tryGetStringFromArgument(arg_num + 2, nullptr, /* allow_identifier= */ false)) - { - /// Wipe either `password` or `user`. - markSecretArgument(arg_num + 2); - } - if (tryGetStringFromArgument(arg_num + 3, nullptr, /* allow_identifier= */ false)) - { - /// Wipe either `password` or `sharding_key`. - markSecretArgument(arg_num + 3); - } - return; - } - - /// Skip the current argument (which is either a database name or a qualified table name). - ++arg_num; - if (database) - { - /// Skip the 'table' argument if the previous argument was a database name. - ++arg_num; - } - } - - /// Skip username. - ++arg_num; - - /// Do our replacement: - /// remote('addresses_expr', db.table, 'user', 'password', ...) -> remote('addresses_expr', db.table, 'user', '[HIDDEN]', ...) - /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string - /// before wiping it (because the `password` argument is always a literal string). - bool can_be_password = tryGetStringFromArgument(arg_num, nullptr, /* allow_identifier= */ false); - if (can_be_password) - markSecretArgument(arg_num); - } - - /// Tries to get either a database name or a qualified table name from an argument. - /// Empty string is also allowed (it means the default database). - /// The function is used by findRemoteFunctionSecretArguments() to determine how many arguments to skip before a password. - bool tryGetDatabaseNameOrQualifiedTableName( - size_t arg_idx, - std::optional & res_database, - std::optional & res_qualified_table_name) const - { - res_database.reset(); - res_qualified_table_name.reset(); - - String str; - if (!tryGetStringFromArgument(arg_idx, &str, /* allow_identifier= */ true)) - return false; - - if (str.empty()) - { - res_database = ""; - return true; - } - - auto qualified_table_name = QualifiedTableName::tryParseFromString(str); - if (!qualified_table_name) - return false; - - if (qualified_table_name->database.empty()) - res_database = std::move(qualified_table_name->table); - else - res_qualified_table_name = std::move(qualified_table_name); - return true; - } - - void findEncryptionFunctionSecretArguments() - { - if (arguments->empty()) - return; - - /// We replace all arguments after 'mode' with '[HIDDEN]': - /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) -> encrypt('mode', '[HIDDEN]') - result.start = 1; - result.count = arguments->size() - 1; - } - - void findTableEngineSecretArguments() - { - const String & engine_name = function.name; - if (engine_name == "ExternalDistributed") - { - /// ExternalDistributed('engine', 'host:port', 'database', 'table', 'user', 'password') - findExternalDistributedTableEngineSecretArguments(); - } - else if ((engine_name == "MySQL") || (engine_name == "PostgreSQL") || - (engine_name == "MaterializedPostgreSQL") || (engine_name == "MongoDB")) - { - /// MySQL('host:port', 'database', 'table', 'user', 'password', ...) - /// PostgreSQL('host:port', 'database', 'table', 'user', 'password', ...) - /// MaterializedPostgreSQL('host:port', 'database', 'table', 'user', 'password', ...) - /// MongoDB('host:port', 'database', 'collection', 'user', 'password', ...) - findMySQLFunctionSecretArguments(); - } - else if ((engine_name == "S3") || (engine_name == "COSN") || (engine_name == "OSS") || - (engine_name == "DeltaLake") || (engine_name == "Hudi") || (engine_name == "Iceberg") || (engine_name == "S3Queue")) - { - /// S3('url', ['aws_access_key_id', 'aws_secret_access_key',] ...) - findS3TableEngineSecretArguments(); - } - else if (engine_name == "URL") - { - findURLSecretArguments(); - } - } - - void findExternalDistributedTableEngineSecretArguments() - { - if (isNamedCollectionName(1)) - { - /// ExternalDistributed('engine', named_collection, ..., password = 'password', ...) - findSecretNamedArgument("password", 2); - } - else - { - /// ExternalDistributed('engine', 'host:port', 'database', 'table', 'user', 'password') - markSecretArgument(5); - } - } - - void findS3TableEngineSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// S3(named_collection, ..., secret_access_key = 'secret_access_key') - findSecretNamedArgument("secret_access_key", 1); - return; - } - - /// We should check other arguments first because we don't need to do any replacement in case of - /// S3('url', NOSIGN, 'format' [, 'compression'] [, extra_credentials(..)] [, headers(..)]) - /// S3('url', 'format', 'compression' [, extra_credentials(..)] [, headers(..)]) - size_t count = excludeS3OrURLNestedMaps(); - if ((3 <= count) && (count <= 4)) - { - String second_arg; - if (tryGetStringFromArgument(1, &second_arg)) - { - if (boost::iequals(second_arg, "NOSIGN")) - return; /// The argument after 'url' is "NOSIGN". - - if (count == 3) - { - if (second_arg == "auto" || KnownFormatNames::instance().exists(second_arg)) - return; /// The argument after 'url' is a format: S3('url', 'format', ...) - } - } - } - - /// We replace 'aws_secret_access_key' with '[HIDDEN]' for the following signatures: - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key') - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format') - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') - if (2 < count) - markSecretArgument(2); - } - - void findDatabaseEngineSecretArguments() - { - const String & engine_name = function.name; - if ((engine_name == "MySQL") || (engine_name == "MaterializeMySQL") || - (engine_name == "MaterializedMySQL") || (engine_name == "PostgreSQL") || - (engine_name == "MaterializedPostgreSQL")) - { - /// MySQL('host:port', 'database', 'user', 'password') - /// PostgreSQL('host:port', 'database', 'user', 'password') - findMySQLDatabaseSecretArguments(); - } - else if (engine_name == "S3") - { - /// S3('url', 'access_key_id', 'secret_access_key') - findS3DatabaseSecretArguments(); - } - } - - void findMySQLDatabaseSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// MySQL(named_collection, ..., password = 'password', ...) - findSecretNamedArgument("password", 1); - } - else - { - /// MySQL('host:port', 'database', 'user', 'password') - markSecretArgument(3); - } - } - - void findS3DatabaseSecretArguments() - { - if (isNamedCollectionName(0)) - { - /// S3(named_collection, ..., secret_access_key = 'password', ...) - findSecretNamedArgument("secret_access_key", 1); - } - else - { - /// S3('url', 'access_key_id', 'secret_access_key') - markSecretArgument(2); - } - } - - void findBackupNameSecretArguments() - { - const String & engine_name = function.name; - if (engine_name == "S3") - { - /// BACKUP ... TO S3(url, [aws_access_key_id, aws_secret_access_key]) - markSecretArgument(2); - } - } - - /// Whether a specified argument can be the name of a named collection? - bool isNamedCollectionName(size_t arg_idx) const - { - if (arguments->size() <= arg_idx) - return false; - - const auto * identifier = (*arguments)[arg_idx]->as(); - return identifier != nullptr; - } - - /// Looks for a secret argument with a specified name. This function looks for arguments in format `key=value` where the key is specified. - void findSecretNamedArgument(const std::string_view & key, size_t start = 0) - { - for (size_t i = start; i < arguments->size(); ++i) - { - const auto & argument = (*arguments)[i]; - const auto * equals_func = argument->as(); - if (!equals_func || (equals_func->name != "equals")) - continue; - - const auto * expr_list = equals_func->arguments->as(); - if (!expr_list) - continue; - - const auto & equal_args = expr_list->children; - if (equal_args.size() != 2) - continue; - - String found_key; - if (!tryGetStringFromArgument(*equal_args[0], &found_key)) - continue; - - if (found_key == key) - markSecretArgument(i, /* argument_is_named= */ true); - } - } }; + } From 56cfa74a14f36e89c7ae9b7ab5f27f4b042cab48 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 14 Sep 2024 13:32:52 +0000 Subject: [PATCH 071/103] fix --- src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h | 7 ++++++- src/Parsers/FunctionSecretArgumentsFinderAST.h | 7 ++++++- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h b/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h index 3c0e5974d16..8e8f56760e4 100644 --- a/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h +++ b/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h @@ -16,7 +16,12 @@ public: { public: explicit ArgumentTreeNode(const IQueryTreeNode * argument_) : argument(argument_) {} - std::unique_ptr getFunction() const override { return std::make_unique(*argument->as()); } + std::unique_ptr getFunction() const override + { + if (const auto * f = argument->as()) + return std::make_unique(*f); + return nullptr; + } bool isIdentifier() const override { return argument->as(); } bool tryGetString(String * res, bool allow_identifier = true) const override { diff --git a/src/Parsers/FunctionSecretArgumentsFinderAST.h b/src/Parsers/FunctionSecretArgumentsFinderAST.h index d2d7a63aff1..9430053d6a5 100644 --- a/src/Parsers/FunctionSecretArgumentsFinderAST.h +++ b/src/Parsers/FunctionSecretArgumentsFinderAST.h @@ -16,7 +16,12 @@ public: { public: explicit ArgumentAST(const IAST * argument_) : argument(argument_) {} - std::unique_ptr getFunction() const override { return std::make_unique(*argument->as()); } + std::unique_ptr getFunction() const override + { + if (const auto * f = argument->as()) + return std::make_unique(*f); + return nullptr; + } bool isIdentifier() const override { return argument->as(); } bool tryGetString(String * res, bool allow_identifier = true) const override { From 6f63a7b213170b5b561b904d44c4cafa62a88dbb Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 14 Sep 2024 16:46:48 +0000 Subject: [PATCH 072/103] fix tidy --- src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h | 2 +- src/Parsers/FunctionSecretArgumentsFinderAST.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h b/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h index 8e8f56760e4..c598adfd98e 100644 --- a/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h +++ b/src/Analyzer/FunctionSecretArgumentsFinderTreeNode.h @@ -23,7 +23,7 @@ public: return nullptr; } bool isIdentifier() const override { return argument->as(); } - bool tryGetString(String * res, bool allow_identifier = true) const override + bool tryGetString(String * res, bool allow_identifier) const override { if (const auto * literal = argument->as()) { diff --git a/src/Parsers/FunctionSecretArgumentsFinderAST.h b/src/Parsers/FunctionSecretArgumentsFinderAST.h index 9430053d6a5..b89007c619d 100644 --- a/src/Parsers/FunctionSecretArgumentsFinderAST.h +++ b/src/Parsers/FunctionSecretArgumentsFinderAST.h @@ -23,7 +23,7 @@ public: return nullptr; } bool isIdentifier() const override { return argument->as(); } - bool tryGetString(String * res, bool allow_identifier = true) const override + bool tryGetString(String * res, bool allow_identifier) const override { if (const auto * literal = argument->as()) { From 63577507c968e82f51c35e5f893864191b86ccf6 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Sat, 14 Sep 2024 21:43:27 +0100 Subject: [PATCH 073/103] fix build --- src/Interpreters/ClusterProxy/executeQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 771c6a89caa..1f854c41873 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -532,7 +532,7 @@ void executeQueryWithParallelReplicas( max_replicas_to_use = shard.getAllNodeCount(); } - auto coordinator = std::make_shared(max_replicas_to_use, settings.parallel_replicas_mark_segment_size); + auto coordinator = std::make_shared(max_replicas_to_use); auto external_tables = new_context->getExternalTables(); From 038f56cb5e4fec023cc56a9cc1688e6985857230 Mon Sep 17 00:00:00 2001 From: marco-vb Date: Sat, 14 Sep 2024 21:04:12 +0000 Subject: [PATCH 074/103] Only make checks to stop inserts if settings are being used. --- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index b606bff7faa..e766ae01dfc 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -555,11 +555,11 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( VolumePtr data_part_volume = createVolumeFromReservation(reservation, volume); const auto & data_settings = data.getSettings(); + const UInt64 min_bytes = data_settings->min_free_disk_bytes_to_throw_insert; + const Float64 min_ratio = data_settings->min_free_disk_ratio_to_throw_insert; + if (min_bytes > 0 || min_ratio > 0.0) { - const UInt64 min_bytes = data_settings->min_free_disk_bytes_to_throw_insert; - const Float64 min_ratio = data_settings->min_free_disk_ratio_to_throw_insert; - const auto disk = data_part_volume->getDisk(); const UInt64 total_disk_bytes = *disk->getTotalSpace(); const UInt64 free_disk_bytes = *disk->getAvailableSpace(); From 03737ddcab8e2355e6f6dd17348e0123272466da Mon Sep 17 00:00:00 2001 From: marco-vb Date: Sat, 14 Sep 2024 22:24:17 +0000 Subject: [PATCH 075/103] Reduced disk size on test for faster execution. --- .../test_stop_insert_when_disk_close_to_full/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_stop_insert_when_disk_close_to_full/test.py b/tests/integration/test_stop_insert_when_disk_close_to_full/test.py index 9b8943705fd..328de674de1 100644 --- a/tests/integration/test_stop_insert_when_disk_close_to_full/test.py +++ b/tests/integration/test_stop_insert_when_disk_close_to_full/test.py @@ -7,7 +7,7 @@ cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", main_configs=["configs/config.d/storage_configuration.xml"], - tmpfs=["/disk1:size=100M"], + tmpfs=["/disk1:size=7M"], macros={"shard": 0, "replica": 1}, ) From cf9200f1d0b154f9f313138aa3b2c67ec6a5b167 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 6 Sep 2024 13:50:13 +0000 Subject: [PATCH 076/103] Randomize integration tests settings --- tests/integration/helpers/cluster.py | 4 +++ tests/integration/helpers/random_settings.py | 28 ++++++++++++++++ .../test_settings_randomization/__init__.py | 0 .../config/users.xml | 7 ++++ .../test_settings_randomization/test.py | 32 +++++++++++++++++++ 5 files changed, 71 insertions(+) create mode 100644 tests/integration/helpers/random_settings.py create mode 100644 tests/integration/test_settings_randomization/__init__.py create mode 100644 tests/integration/test_settings_randomization/config/users.xml create mode 100644 tests/integration/test_settings_randomization/test.py diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 821bb887435..469263a8e45 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -19,6 +19,7 @@ import urllib.parse import shlex import urllib3 import requests +from pathlib import Path try: # Please, add modules that required for specific tests only here. @@ -52,6 +53,7 @@ from helpers.client import QueryRuntimeException import docker from .client import Client +from .random_settings import write_random_settings_config from .retry_decorator import retry from .config_cluster import * @@ -4602,6 +4604,8 @@ class ClickHouseInstance: if len(self.custom_dictionaries_paths): write_embedded_config("0_common_enable_dictionaries.xml", self.config_d_dir) + write_random_settings_config(Path(users_d_dir) / "0_random_settings.xml") + version = None version_parts = self.tag.split(".") if version_parts[0].isdigit() and version_parts[1].isdigit(): diff --git a/tests/integration/helpers/random_settings.py b/tests/integration/helpers/random_settings.py new file mode 100644 index 00000000000..b2319561fd7 --- /dev/null +++ b/tests/integration/helpers/random_settings.py @@ -0,0 +1,28 @@ +import random + + +def randomize_settings(): + yield "max_joined_block_size_rows", random.randint(8000, 100000) + if random.random() < 0.5: + yield "max_block_size", random.randint(8000, 100000) + + +def write_random_settings_config(destination): + with open(destination, "w") as f: + f.write( + """ + + + +""" + ) + for setting, value in randomize_settings(): + f.write(f"<{setting}>{value}\n") + + f.write( + """ + + + +""" + ) diff --git a/tests/integration/test_settings_randomization/__init__.py b/tests/integration/test_settings_randomization/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_settings_randomization/config/users.xml b/tests/integration/test_settings_randomization/config/users.xml new file mode 100644 index 00000000000..a7e5d5f15f0 --- /dev/null +++ b/tests/integration/test_settings_randomization/config/users.xml @@ -0,0 +1,7 @@ + + + + 59999 + + + diff --git a/tests/integration/test_settings_randomization/test.py b/tests/integration/test_settings_randomization/test.py new file mode 100644 index 00000000000..d4074baf6c6 --- /dev/null +++ b/tests/integration/test_settings_randomization/test.py @@ -0,0 +1,32 @@ +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance("node1", user_configs=["config/users.xml"]) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_settings_randomization(started_cluster): + """ + See tests/integration/helpers/random_settings.py + """ + + def q(field, name): + return int( + node.query( + f"SELECT {field} FROM system.settings WHERE name = '{name}'" + ).strip() + ) + + assert q("value", "max_block_size") == 59999 + assert q("changed", "max_joined_block_size_rows") == 1 + assert 8000 <= q("value", "max_joined_block_size_rows") <= 100000 From 056c7af356fe6410a38651be1b615b86017314bf Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 6 Sep 2024 14:35:33 +0000 Subject: [PATCH 077/103] comment --- tests/integration/test_settings_randomization/test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/test_settings_randomization/test.py b/tests/integration/test_settings_randomization/test.py index d4074baf6c6..f93a0a15984 100644 --- a/tests/integration/test_settings_randomization/test.py +++ b/tests/integration/test_settings_randomization/test.py @@ -27,6 +27,9 @@ def test_settings_randomization(started_cluster): ).strip() ) + # setting set in test config is not overriden assert q("value", "max_block_size") == 59999 + + # some setting is randomized assert q("changed", "max_joined_block_size_rows") == 1 assert 8000 <= q("value", "max_joined_block_size_rows") <= 100000 From 1963e971f3940b2e6b84e6a1ace3c5785e36efd7 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 16 Sep 2024 09:57:20 +0000 Subject: [PATCH 078/103] fix --- tests/integration/helpers/cluster.py | 15 +++++++++++---- tests/integration/test_settings_profile/test.py | 4 +++- 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 469263a8e45..b39b859a807 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -62,6 +62,9 @@ HELPERS_DIR = p.dirname(__file__) CLICKHOUSE_ROOT_DIR = p.join(p.dirname(__file__), "../../..") LOCAL_DOCKER_COMPOSE_DIR = p.join(CLICKHOUSE_ROOT_DIR, "tests/integration/compose/") DEFAULT_ENV_NAME = ".env" +DEFAULT_BASE_CONFIG_DIR = os.environ.get( + "CLICKHOUSE_TESTS_BASE_CONFIG_DIR", "/etc/clickhouse-server/" +) SANITIZER_SIGN = "==================" @@ -446,9 +449,7 @@ class ClickHouseCluster: self.base_dir = p.dirname(base_path) self.name = name if name is not None else extract_test_name(base_path) - self.base_config_dir = base_config_dir or os.environ.get( - "CLICKHOUSE_TESTS_BASE_CONFIG_DIR", "/etc/clickhouse-server/" - ) + self.base_config_dir = base_config_dir or DEFAULT_BASE_CONFIG_DIR self.server_bin_path = p.realpath( server_bin_path or os.environ.get("CLICKHOUSE_TESTS_SERVER_BIN_PATH", "/usr/bin/clickhouse") @@ -1743,6 +1744,7 @@ class ClickHouseCluster: copy_common_configs=True, config_root_name="clickhouse", extra_configs=[], + randomize_settings=True, ) -> "ClickHouseInstance": """Add an instance to the cluster. @@ -1847,6 +1849,7 @@ class ClickHouseCluster: mem_limit=mem_limit, config_root_name=config_root_name, extra_configs=extra_configs, + randomize_settings=randomize_settings, ) docker_compose_yml_dir = get_docker_compose_path() @@ -3465,6 +3468,7 @@ class ClickHouseInstance: mem_limit=None, config_root_name="clickhouse", extra_configs=[], + randomize_settings=True, ): self.name = name self.base_cmd = cluster.base_cmd @@ -3533,6 +3537,7 @@ class ClickHouseInstance: self.with_coredns = with_coredns self.coredns_config_dir = p.abspath(p.join(base_path, "coredns_config")) self.use_old_analyzer = use_old_analyzer + self.randomize_settings = randomize_settings self.main_config_name = main_config_name self.users_config_name = users_config_name @@ -4604,7 +4609,9 @@ class ClickHouseInstance: if len(self.custom_dictionaries_paths): write_embedded_config("0_common_enable_dictionaries.xml", self.config_d_dir) - write_random_settings_config(Path(users_d_dir) / "0_random_settings.xml") + if self.randomize_settings and self.base_config_dir == DEFAULT_BASE_CONFIG_DIR: + # If custom main config is used, do not apply random settings to it + write_random_settings_config(Path(users_d_dir) / "0_random_settings.xml") version = None version_parts = self.tag.split(".") diff --git a/tests/integration/test_settings_profile/test.py b/tests/integration/test_settings_profile/test.py index 4800ab798bf..1f37e17a60b 100644 --- a/tests/integration/test_settings_profile/test.py +++ b/tests/integration/test_settings_profile/test.py @@ -3,7 +3,9 @@ from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance("instance") + +# `randomize_settings` is set tot `False` to maake result of `SHOW CREATE SETTINGS PROFILE` consistent +instance = cluster.add_instance("instance", randomize_settings=False) def system_settings_profile(profile_name): From 6863dc764756878a1e985254503200450539b910 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 16 Sep 2024 12:48:01 +0200 Subject: [PATCH 079/103] init --- .../0_stateless/02706_show_columns.sql | 30 +++++++-------- .../0_stateless/02724_show_indexes.sql | 38 +++++++++---------- 2 files changed, 34 insertions(+), 34 deletions(-) diff --git a/tests/queries/0_stateless/02706_show_columns.sql b/tests/queries/0_stateless/02706_show_columns.sql index a5164ce8022..bf33d2eb4dc 100644 --- a/tests/queries/0_stateless/02706_show_columns.sql +++ b/tests/queries/0_stateless/02706_show_columns.sql @@ -55,6 +55,21 @@ CREATE TABLE NULL (c String) ENGINE = MergeTree ORDER BY c; SHOW COLUMNS FROM NULL; DROP TABLE NULL; +DROP TABLE IF EXISTS `tab.with.dots`; +CREATE TABLE `tab.with.dots` +( + `uint64` UInt64, + `int32` Nullable(Int32) COMMENT 'example comment', + `str` String, + INDEX idx str TYPE set(1000) +) +ENGINE = MergeTree +PRIMARY KEY (uint64) +ORDER BY (uint64, str); +SELECT '--- SHOW COLUMNS FROM table with dots'; +SHOW COLUMNS FROM `tab.with.dots`; +DROP TABLE `tab.with.dots`; + DROP DATABASE IF EXISTS `'`; CREATE DATABASE `'`; CREATE TABLE `'`.`'` (c String) ENGINE = MergeTree ORDER BY c; @@ -90,18 +105,3 @@ SHOW COLUMNS FROM database_123456789abcde.tab; DROP DATABASE database_123456789abcde; DROP TABLE tab; - -DROP TABLE IF EXISTS `tab.with.dots`; -CREATE TABLE `tab.with.dots` -( - `uint64` UInt64, - `int32` Nullable(Int32) COMMENT 'example comment', - `str` String, - INDEX idx str TYPE set(1000) -) -ENGINE = MergeTree -PRIMARY KEY (uint64) -ORDER BY (uint64, str); -SELECT '--- SHOW COLUMNS FROM table with dots'; -SHOW COLUMNS FROM `tab.with.dots`; -DROP TABLE `tab.with.dots`; diff --git a/tests/queries/0_stateless/02724_show_indexes.sql b/tests/queries/0_stateless/02724_show_indexes.sql index f7eb978b375..ed5b6b4e25a 100644 --- a/tests/queries/0_stateless/02724_show_indexes.sql +++ b/tests/queries/0_stateless/02724_show_indexes.sql @@ -43,6 +43,25 @@ CREATE TABLE NULL (c String) ENGINE = MergeTree ORDER BY c; SHOW INDEX FROM NULL; DROP TABLE NULL; +DROP TABLE IF EXISTS `tab.with.dots`; +CREATE TABLE `tab.with.dots` +( + a UInt64, + b UInt64, + c UInt64, + d UInt64, + e UInt64, + INDEX mm1_idx (a, c, d) TYPE minmax, + INDEX mm2_idx (c, d, e) TYPE minmax, + INDEX set_idx (e) TYPE set(100), + INDEX blf_idx (d, b) TYPE bloom_filter(0.8) +) +ENGINE = MergeTree +PRIMARY KEY (c, a); +SELECT '--- SHOW INDEX FROM table with dots'; +SHOW INDEX FROM `tab.with.dots`; +DROP TABLE `tab.with.dots`; + DROP DATABASE IF EXISTS `'`; CREATE DATABASE `'`; CREATE TABLE `'`.`'` (c String) ENGINE = MergeTree ORDER BY c; @@ -78,22 +97,3 @@ SHOW INDEX FROM database_123456789abcde.tbl; DROP DATABASE database_123456789abcde; DROP TABLE tbl; - -DROP TABLE IF EXISTS `tab.with.dots`; -CREATE TABLE `tab.with.dots` -( - a UInt64, - b UInt64, - c UInt64, - d UInt64, - e UInt64, - INDEX mm1_idx (a, c, d) TYPE minmax, - INDEX mm2_idx (c, d, e) TYPE minmax, - INDEX set_idx (e) TYPE set(100), - INDEX blf_idx (d, b) TYPE bloom_filter(0.8) -) -ENGINE = MergeTree -PRIMARY KEY (c, a); -SELECT '--- SHOW INDEX FROM table with dots'; -SHOW INDEX FROM `tab.with.dots`; -DROP TABLE `tab.with.dots`; From 8cdcc431fe5b3cb001523b697d1348abe111d88c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 16 Sep 2024 09:56:31 +0200 Subject: [PATCH 080/103] Fix --- .../poco/Crypto/include/Poco/Crypto/EVPPKey.h | 14 +++++++++++++ src/Coordination/KeeperServer.cpp | 4 ++-- tests/integration/helpers/cluster.py | 2 +- .../test_keeper_internal_secure/ssl_conf.yml | 0 .../test_keeper_internal_secure/test.py | 21 ++++++++++--------- 5 files changed, 28 insertions(+), 13 deletions(-) delete mode 100644 tests/integration/test_keeper_internal_secure/ssl_conf.yml diff --git a/base/poco/Crypto/include/Poco/Crypto/EVPPKey.h b/base/poco/Crypto/include/Poco/Crypto/EVPPKey.h index c33e0ae847f..6e44d9f45b7 100644 --- a/base/poco/Crypto/include/Poco/Crypto/EVPPKey.h +++ b/base/poco/Crypto/include/Poco/Crypto/EVPPKey.h @@ -226,6 +226,13 @@ namespace Crypto error: if (pFile) fclose(pFile); + if (*ppKey) + { + if constexpr (std::is_same_v) + EVP_PKEY_free(*ppKey); + else + EC_KEY_free(*ppKey); + } throw OpenSSLException("EVPKey::loadKey(string)"); } @@ -287,6 +294,13 @@ namespace Crypto error: if (pBIO) BIO_free(pBIO); + if (*ppKey) + { + if constexpr (std::is_same_v) + EVP_PKEY_free(*ppKey); + else + EC_KEY_free(*ppKey); + } throw OpenSSLException("EVPKey::loadKey(stream)"); } diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index e0a94b1a00c..2eada508e22 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -125,9 +125,9 @@ void setSSLParams(nuraft::asio_service::options & asio_opts) disabled_protocols |= Poco::Net::Context::PROTO_TLSV1_2; } - asio_opts.ssl_context_provider_server_ = [ctx_params = params, certificate_data, disabled_protocols] + asio_opts.ssl_context_provider_server_ = [params, certificate_data, disabled_protocols] { - Poco::Net::Context context(Poco::Net::Context::Usage::TLSV1_2_SERVER_USE, ctx_params); + Poco::Net::Context context(Poco::Net::Context::Usage::TLSV1_2_SERVER_USE, params); context.disableProtocols(disabled_protocols); SSL_CTX * ssl_ctx = context.takeSslContext(); SSL_CTX_set_cert_cb(ssl_ctx, callSetCertificate, reinterpret_cast(certificate_data.get())); diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 821bb887435..4ef2699ea3b 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -4093,7 +4093,7 @@ class ClickHouseInstance: exclusion_substring="", ): if from_host: - # We check fist file exists but want to look for all rotated logs as well + # We check first file exists but want to look for all rotated logs as well result = subprocess_check_call( [ "bash", diff --git a/tests/integration/test_keeper_internal_secure/ssl_conf.yml b/tests/integration/test_keeper_internal_secure/ssl_conf.yml deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_keeper_internal_secure/test.py b/tests/integration/test_keeper_internal_secure/test.py index af511a60636..8cab03b6e2d 100644 --- a/tests/integration/test_keeper_internal_secure/test.py +++ b/tests/integration/test_keeper_internal_secure/test.py @@ -161,21 +161,22 @@ def check_valid_configuration(filename, password): run_test() +def check_invalid_configuration(filename, password): + stop_all_clickhouse() + for node in nodes: + setupSsl(node, filename, password) + + nodes[0].start_clickhouse(expected_to_fail=True) + nodes[0].wait_for_log_line( + "OpenSSLException: EVPKey::loadKey.*error:0480006C:PEM routines::no start line", + ) + + def test_secure_raft_works(started_cluster): check_valid_configuration("WithoutPassPhrase", None) def test_secure_raft_works_with_password(started_cluster): - def check_invalid_configuration(filename, password): - stop_all_clickhouse() - for node in nodes: - setupSsl(node, filename, password) - - nodes[0].start_clickhouse(expected_to_fail=True) - nodes[0].contains_in_log( - "OpenSSLException: EVPKey::loadKey(string): error:0480006C:PEM routines::no start line" - ) - check_valid_configuration("WithoutPassPhrase", "unusedpassword") check_invalid_configuration("WithPassPhrase", "wrongpassword") check_invalid_configuration("WithPassPhrase", "") From 4af369fbc4bada857e267f34440df72f1aaba425 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 16 Sep 2024 13:34:01 +0200 Subject: [PATCH 081/103] Failpoint for testing slow index analysis --- src/Common/FailPoint.cpp | 1 + .../MergeTree/MergeTreeDataSelectExecutor.cpp | 12 ++++++++++++ 2 files changed, 13 insertions(+) diff --git a/src/Common/FailPoint.cpp b/src/Common/FailPoint.cpp index b2fcbc77c56..1b65447efc3 100644 --- a/src/Common/FailPoint.cpp +++ b/src/Common/FailPoint.cpp @@ -63,6 +63,7 @@ static struct InitFiu REGULAR(keepermap_fail_drop_data) \ REGULAR(lazy_pipe_fds_fail_close) \ PAUSEABLE(infinite_sleep) \ + REGULAR(slowdown_index_analysis) \ namespace FailPoints diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 5d246293396..6e0ae8f7cca 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -41,6 +41,8 @@ #include #include #include +#include +#include #include #include #include @@ -75,6 +77,11 @@ namespace ErrorCodes extern const int DUPLICATED_PART_UUIDS; } +namespace FailPoints +{ + extern const char slowdown_index_analysis[]; +} + MergeTreeDataSelectExecutor::MergeTreeDataSelectExecutor(const MergeTreeData & data_) : data(data_), log(getLogger(data.getLogName() + " (SelectExecutor)")) @@ -1565,6 +1572,11 @@ void MergeTreeDataSelectExecutor::selectPartsToRead( if (query_status) query_status->checkTimeLimit(); + fiu_do_on(FailPoints::slowdown_index_analysis, + { + sleepForMilliseconds(1000); + }); + const auto * part = part_or_projection->isProjectionPart() ? part_or_projection->getParentPart() : part_or_projection.get(); if (part_values && part_values->find(part->name) == part_values->end()) continue; From f5b9d5ad34e6e3c7143ac26f4cc871db3b69604d Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 16 Sep 2024 13:34:40 +0200 Subject: [PATCH 082/103] Test for checking time limit in index analysis --- ..._check_timeout_in_index_analysis.reference | 4 +++ .../03176_check_timeout_in_index_analysis.sql | 32 +++++++++++++++++++ 2 files changed, 36 insertions(+) create mode 100644 tests/queries/0_stateless/03176_check_timeout_in_index_analysis.reference create mode 100644 tests/queries/0_stateless/03176_check_timeout_in_index_analysis.sql diff --git a/tests/queries/0_stateless/03176_check_timeout_in_index_analysis.reference b/tests/queries/0_stateless/03176_check_timeout_in_index_analysis.reference new file mode 100644 index 00000000000..05a83c81dae --- /dev/null +++ b/tests/queries/0_stateless/03176_check_timeout_in_index_analysis.reference @@ -0,0 +1,4 @@ +5 +03176_q1 1 0 0 +03176_q2 1 2 0 +03176_q3 0 1 1 diff --git a/tests/queries/0_stateless/03176_check_timeout_in_index_analysis.sql b/tests/queries/0_stateless/03176_check_timeout_in_index_analysis.sql new file mode 100644 index 00000000000..4163ad58c4e --- /dev/null +++ b/tests/queries/0_stateless/03176_check_timeout_in_index_analysis.sql @@ -0,0 +1,32 @@ +-- Tags: no-parallel, no-tsan, no-asan, no-ubsan, no-msan, no-debug, no-fasttest +-- no-parallel because the test uses failpoint + +CREATE TABLE t_03176(k UInt64, v UInt64) ENGINE=MergeTree() ORDER BY k PARTITION BY k; + +INSERT INTO t_03176 SELECT number, number FROM numbers(5); + +-- Table is partitioned by k to so it will have 5 partitions +SELECT count() FROM system.parts WHERE database = currentDatabase() AND table = 't_03176' AND active; + +-- This query is fast without failpoint: should take < 1 sec +EXPLAIN indexes = 1 SELECT * FROM t_03176 ORDER BY k LIMIT 5 SETTINGS log_comment = '03176_q1' FORMAT Null; + +SYSTEM ENABLE FAILPOINT slowdown_index_analysis; + +-- Check that failpont actually works: the query should take >= 5 sec +EXPLAIN indexes = 1 SELECT * FROM t_03176 ORDER BY k LIMIT 5 SETTINGS log_comment = '03176_q2' FORMAT Null; + +-- Now the query should be cancelled after about 1 sec +EXPLAIN indexes = 1 SELECT * FROM t_03176 ORDER BY k LIMIT 5 SETTINGS log_comment = '03176_q3', max_execution_time = 1.1 FORMAT Null; -- { serverError TIMEOUT_EXCEEDED } + +SYSTEM DISABLE FAILPOINT slowdown_index_analysis; + +SYSTEM FLUSH LOGS; + +-- Check that q1 was fast, q2 was slow and q3 had timeout +SELECT log_comment, type = 'QueryFinish', intDiv(query_duration_ms, 2000), length(exception) > 0 +FROM system.query_log +WHERE current_database = currentDatabase() AND log_comment LIKE '03176_q_' AND type IN ('QueryFinish', 'ExceptionBeforeStart') +ORDER BY log_comment; + +DROP TABLE t_03176; From dc02b168a0c6c87ee923040e0b24720b147f6a23 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 16 Sep 2024 13:53:47 +0200 Subject: [PATCH 083/103] fix references + remove index --- .../0_stateless/02706_show_columns.reference | 5 +---- tests/queries/0_stateless/02706_show_columns.sql | 12 +----------- .../0_stateless/02724_show_indexes.reference | 13 ++++++------- tests/queries/0_stateless/02724_show_indexes.sql | 1 - 4 files changed, 8 insertions(+), 23 deletions(-) diff --git a/tests/queries/0_stateless/02706_show_columns.reference b/tests/queries/0_stateless/02706_show_columns.reference index 32cae685a59..693a2fc2eb5 100644 --- a/tests/queries/0_stateless/02706_show_columns.reference +++ b/tests/queries/0_stateless/02706_show_columns.reference @@ -33,6 +33,7 @@ c String NO PRI SOR \N c String NO PRI SOR \N c String NO PRI SOR \N c String NO PRI SOR \N +c String NO PRI SOR \N --- Original table int32 Nullable(Int32) YES \N str String NO SOR \N @@ -45,7 +46,3 @@ uint64 UInt64 NO PRI SOR \N int32 Int32 NO \N str String NO \N uint64 UInt64 NO PRI SOR \N ---- SHOW COLUMNS FROM table with dots -int32 Nullable(Int32) YES \N -str String NO SOR \N -uint64 UInt64 NO PRI SOR \N diff --git a/tests/queries/0_stateless/02706_show_columns.sql b/tests/queries/0_stateless/02706_show_columns.sql index bf33d2eb4dc..57d80a1c580 100644 --- a/tests/queries/0_stateless/02706_show_columns.sql +++ b/tests/queries/0_stateless/02706_show_columns.sql @@ -56,17 +56,7 @@ SHOW COLUMNS FROM NULL; DROP TABLE NULL; DROP TABLE IF EXISTS `tab.with.dots`; -CREATE TABLE `tab.with.dots` -( - `uint64` UInt64, - `int32` Nullable(Int32) COMMENT 'example comment', - `str` String, - INDEX idx str TYPE set(1000) -) -ENGINE = MergeTree -PRIMARY KEY (uint64) -ORDER BY (uint64, str); -SELECT '--- SHOW COLUMNS FROM table with dots'; +CREATE TABLE `tab.with.dots` (c String) ENGINE = MergeTree ORDER BY c; SHOW COLUMNS FROM `tab.with.dots`; DROP TABLE `tab.with.dots`; diff --git a/tests/queries/0_stateless/02724_show_indexes.reference b/tests/queries/0_stateless/02724_show_indexes.reference index 27802b0991c..f4308db121f 100644 --- a/tests/queries/0_stateless/02724_show_indexes.reference +++ b/tests/queries/0_stateless/02724_show_indexes.reference @@ -34,6 +34,12 @@ tbl 1 set_idx 1 \N 0 \N \N \N SET YES e --- Check with weird table names $4@^7 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES NULL 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES +tab.with.dots 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b +tab.with.dots 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d +tab.with.dots 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e +tab.with.dots 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES +tab.with.dots 1 PRIMARY 2 a A 0 \N \N \N PRIMARY YES +tab.with.dots 1 set_idx 1 \N 0 \N \N \N SET YES e \' 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES \' 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES --- Original table @@ -49,10 +55,3 @@ tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES --- Short form tbl 1 mmi_idx 1 \N 0 \N \N \N MINMAX YES b tbl 1 PRIMARY 1 a A 0 \N \N \N PRIMARY YES ---- SHOW INDEX FROM table with dots -tab.with.dots 1 blf_idx 1 \N 0 \N \N \N BLOOM_FILTER YES d, b -tab.with.dots 1 mm1_idx 1 \N 0 \N \N \N MINMAX YES a, c, d -tab.with.dots 1 mm2_idx 1 \N 0 \N \N \N MINMAX YES c, d, e -tab.with.dots 1 PRIMARY 1 c A 0 \N \N \N PRIMARY YES -tab.with.dots 1 PRIMARY 2 a A 0 \N \N \N PRIMARY YES -tab.with.dots 1 set_idx 1 \N 0 \N \N \N SET YES e diff --git a/tests/queries/0_stateless/02724_show_indexes.sql b/tests/queries/0_stateless/02724_show_indexes.sql index ed5b6b4e25a..a8d699ddb47 100644 --- a/tests/queries/0_stateless/02724_show_indexes.sql +++ b/tests/queries/0_stateless/02724_show_indexes.sql @@ -58,7 +58,6 @@ CREATE TABLE `tab.with.dots` ) ENGINE = MergeTree PRIMARY KEY (c, a); -SELECT '--- SHOW INDEX FROM table with dots'; SHOW INDEX FROM `tab.with.dots`; DROP TABLE `tab.with.dots`; From d223c4547faf36ba909e57c8a7d1094f33d44ea3 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Mon, 16 Sep 2024 08:35:05 -0400 Subject: [PATCH 084/103] fix after master merge --- src/Parsers/FunctionSecretArgumentsFinder.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Parsers/FunctionSecretArgumentsFinder.h b/src/Parsers/FunctionSecretArgumentsFinder.h index 77facd715f8..434d027c9fc 100644 --- a/src/Parsers/FunctionSecretArgumentsFinder.h +++ b/src/Parsers/FunctionSecretArgumentsFinder.h @@ -89,7 +89,8 @@ protected: findMySQLFunctionSecretArguments(); } else if ((function->name() == "s3") || (function->name() == "cosn") || (function->name() == "oss") || - (function->name() == "deltaLake") || (function->name() == "hudi") || (function->name() == "iceberg")) + (function->name() == "deltaLake") || (function->name() == "hudi") || (function->name() == "iceberg") || + (function.name == "gcs")) { /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) findS3FunctionSecretArguments(/* is_cluster_function= */ false); From 19e219758290e80261700bbe55adb48a607a0309 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Mon, 16 Sep 2024 10:38:28 -0400 Subject: [PATCH 085/103] fix --- src/Parsers/FunctionSecretArgumentsFinder.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/FunctionSecretArgumentsFinder.h b/src/Parsers/FunctionSecretArgumentsFinder.h index 434d027c9fc..7836a863920 100644 --- a/src/Parsers/FunctionSecretArgumentsFinder.h +++ b/src/Parsers/FunctionSecretArgumentsFinder.h @@ -90,7 +90,7 @@ protected: } else if ((function->name() == "s3") || (function->name() == "cosn") || (function->name() == "oss") || (function->name() == "deltaLake") || (function->name() == "hudi") || (function->name() == "iceberg") || - (function.name == "gcs")) + (function->name() == "gcs")) { /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) findS3FunctionSecretArguments(/* is_cluster_function= */ false); From 5aaff37b36fd6b8b4f7aaf3f14a622a1816eb5b1 Mon Sep 17 00:00:00 2001 From: Michael Stetsyuk Date: Tue, 17 Sep 2024 09:16:52 +0000 Subject: [PATCH 086/103] sync changes to InterpreterDropQuery::executeToDatabaseImpl from the private fork --- src/Interpreters/InterpreterDropQuery.cpp | 165 +++++++++++----------- 1 file changed, 82 insertions(+), 83 deletions(-) diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index ef560ec3405..5161fd19d87 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -380,100 +380,99 @@ BlockIO InterpreterDropQuery::executeToDatabase(const ASTDropQuery & query) BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, DatabasePtr & database, std::vector & uuids_to_wait) { + if (query.kind != ASTDropQuery::Kind::Detach && query.kind != ASTDropQuery::Kind::Drop && query.kind != ASTDropQuery::Kind::Truncate) + return {}; + const auto & database_name = query.getDatabase(); auto ddl_guard = DatabaseCatalog::instance().getDDLGuard(database_name, ""); database = tryGetDatabase(database_name, query.if_exists); - if (database) + if (!database) + return {}; + + bool drop = query.kind == ASTDropQuery::Kind::Drop; + bool truncate = query.kind == ASTDropQuery::Kind::Truncate; + + getContext()->checkAccess(AccessType::DROP_DATABASE, database_name); + + if (query.kind == ASTDropQuery::Kind::Detach && query.permanently) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "DETACH PERMANENTLY is not implemented for databases"); + + if (query.if_empty) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "DROP IF EMPTY is not implemented for databases"); + + if (!truncate && database->hasReplicationThread()) + database->stopReplication(); + + if (database->shouldBeEmptyOnDetach()) { - if (query.kind == ASTDropQuery::Kind::Detach || query.kind == ASTDropQuery::Kind::Drop - || query.kind == ASTDropQuery::Kind::Truncate) + /// Cancel restarting replicas in that database, wait for remaining RESTART queries to finish. + /// So it will not startup tables concurrently with the flushAndPrepareForShutdown call below. + auto restart_replica_lock = DatabaseCatalog::instance().getLockForDropDatabase(database_name); + + ASTDropQuery query_for_table; + query_for_table.kind = query.kind; + // For truncate operation on database, drop the tables + if (truncate) + query_for_table.kind = query.has_all_tables ? ASTDropQuery::Kind::Truncate : ASTDropQuery::Kind::Drop; + query_for_table.if_exists = true; + query_for_table.if_empty = false; + query_for_table.setDatabase(database_name); + query_for_table.sync = query.sync; + + /// Flush should not be done if shouldBeEmptyOnDetach() == false, + /// since in this case getTablesIterator() may do some additional work, + /// see DatabaseMaterializedMySQL::getTablesIterator() + auto table_context = Context::createCopy(getContext()); + table_context->setInternalQuery(true); + /// Do not hold extra shared pointers to tables + std::vector> tables_to_drop; + // NOTE: This means we wait for all tables to be loaded inside getTablesIterator() call in case of `async_load_databases = true`. + for (auto iterator = database->getTablesIterator(table_context); iterator->isValid(); iterator->next()) { - bool drop = query.kind == ASTDropQuery::Kind::Drop; - bool truncate = query.kind == ASTDropQuery::Kind::Truncate; + auto table_ptr = iterator->table(); + tables_to_drop.push_back({table_ptr->getStorageID(), table_ptr->isDictionary()}); + } - getContext()->checkAccess(AccessType::DROP_DATABASE, database_name); - - if (query.kind == ASTDropQuery::Kind::Detach && query.permanently) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "DETACH PERMANENTLY is not implemented for databases"); - - if (query.if_empty) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "DROP IF EMPTY is not implemented for databases"); - - if (!truncate && database->hasReplicationThread()) - database->stopReplication(); - - if (database->shouldBeEmptyOnDetach()) + /// Prepare tables for shutdown in parallel. + ThreadPoolCallbackRunnerLocal runner(getDatabaseCatalogDropTablesThreadPool().get(), "DropTables"); + for (const auto & [name, _] : tables_to_drop) + { + auto table_ptr = DatabaseCatalog::instance().getTable(name, table_context); + runner([my_table_ptr = std::move(table_ptr)]() { - /// Cancel restarting replicas in that database, wait for remaining RESTART queries to finish. - /// So it will not startup tables concurrently with the flushAndPrepareForShutdown call below. - auto restart_replica_lock = DatabaseCatalog::instance().getLockForDropDatabase(database_name); + my_table_ptr->flushAndPrepareForShutdown(); + }); + } + runner.waitForAllToFinishAndRethrowFirstError(); - ASTDropQuery query_for_table; - query_for_table.kind = query.kind; - // For truncate operation on database, drop the tables - if (truncate) - query_for_table.kind = query.has_all_tables ? ASTDropQuery::Kind::Truncate : ASTDropQuery::Kind::Drop; - query_for_table.if_exists = true; - query_for_table.if_empty = false; - query_for_table.setDatabase(database_name); - query_for_table.sync = query.sync; - - /// Flush should not be done if shouldBeEmptyOnDetach() == false, - /// since in this case getTablesIterator() may do some additional work, - /// see DatabaseMaterializedMySQL::getTablesIterator() - auto table_context = Context::createCopy(getContext()); - table_context->setInternalQuery(true); - /// Do not hold extra shared pointers to tables - std::vector> tables_to_drop; - // NOTE: This means we wait for all tables to be loaded inside getTablesIterator() call in case of `async_load_databases = true`. - for (auto iterator = database->getTablesIterator(table_context); iterator->isValid(); iterator->next()) - { - auto table_ptr = iterator->table(); - tables_to_drop.push_back({table_ptr->getStorageID(), table_ptr->isDictionary()}); - } - - /// Prepare tables for shutdown in parallel. - ThreadPoolCallbackRunnerLocal runner(getDatabaseCatalogDropTablesThreadPool().get(), "DropTables"); - for (const auto & [name, _] : tables_to_drop) - { - auto table_ptr = DatabaseCatalog::instance().getTable(name, table_context); - runner([my_table_ptr = std::move(table_ptr)]() - { - my_table_ptr->flushAndPrepareForShutdown(); - }); - } - runner.waitForAllToFinishAndRethrowFirstError(); - - for (const auto & table : tables_to_drop) - { - query_for_table.setTable(table.first.getTableName()); - query_for_table.is_dictionary = table.second; - DatabasePtr db; - UUID table_to_wait = UUIDHelpers::Nil; - executeToTableImpl(table_context, query_for_table, db, table_to_wait); - uuids_to_wait.push_back(table_to_wait); - } - } - // only if operation is DETACH - if ((!drop || !truncate) && query.sync) - { - /// Avoid "some tables are still in use" when sync mode is enabled - for (const auto & table_uuid : uuids_to_wait) - database->waitDetachedTableNotInUse(table_uuid); - } - - /// Protects from concurrent CREATE TABLE queries - auto db_guard = DatabaseCatalog::instance().getExclusiveDDLGuardForDatabase(database_name); - // only if operation is DETACH - if (!drop || !truncate) - database->assertCanBeDetached(true); - - /// DETACH or DROP database itself. If TRUNCATE skip dropping/erasing the database. - if (!truncate) - DatabaseCatalog::instance().detachDatabase(getContext(), database_name, drop, database->shouldBeEmptyOnDetach()); + for (const auto & table : tables_to_drop) + { + query_for_table.setTable(table.first.getTableName()); + query_for_table.is_dictionary = table.second; + DatabasePtr db; + UUID table_to_wait = UUIDHelpers::Nil; + executeToTableImpl(table_context, query_for_table, db, table_to_wait); + uuids_to_wait.push_back(table_to_wait); } } + // only if operation is DETACH + if ((!drop || !truncate) && query.sync) + { + /// Avoid "some tables are still in use" when sync mode is enabled + for (const auto & table_uuid : uuids_to_wait) + database->waitDetachedTableNotInUse(table_uuid); + } + + /// Protects from concurrent CREATE TABLE queries + auto db_guard = DatabaseCatalog::instance().getExclusiveDDLGuardForDatabase(database_name); + // only if operation is DETACH + if (!drop || !truncate) + database->assertCanBeDetached(true); + + /// DETACH or DROP database itself. If TRUNCATE skip dropping/erasing the database. + if (!truncate) + DatabaseCatalog::instance().detachDatabase(getContext(), database_name, drop, database->shouldBeEmptyOnDetach()); return {}; } From 1ccd461c97da60689bf5ab1cb646df7cc59b781e Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 4 Sep 2024 23:43:53 +0200 Subject: [PATCH 087/103] Fix restoring access entities dependant on existing ones. --- src/Access/AccessBackup.cpp | 75 +++++++++++++++-------- src/Access/AccessBackup.h | 4 +- src/Access/AccessControl.cpp | 4 +- src/Access/AccessControl.h | 2 +- src/Access/DiskAccessStorage.cpp | 41 +++++-------- src/Access/DiskAccessStorage.h | 5 +- src/Access/IAccessEntity.cpp | 24 ++++++++ src/Access/IAccessEntity.h | 5 +- src/Access/IAccessStorage.cpp | 57 ++++++++++++++--- src/Access/IAccessStorage.h | 10 ++- src/Access/MemoryAccessStorage.cpp | 43 +++++-------- src/Access/MemoryAccessStorage.h | 5 +- src/Access/MultipleAccessStorage.cpp | 4 +- src/Access/MultipleAccessStorage.h | 2 +- src/Access/Quota.cpp | 2 +- src/Access/Quota.h | 2 +- src/Access/ReplicatedAccessStorage.cpp | 85 ++++++++++++++------------ src/Access/ReplicatedAccessStorage.h | 7 ++- src/Access/Role.cpp | 2 +- src/Access/Role.h | 2 +- src/Access/RowPolicy.cpp | 2 +- src/Access/RowPolicy.h | 2 +- src/Access/SettingsProfile.cpp | 2 +- src/Access/SettingsProfile.h | 2 +- src/Access/User.cpp | 2 +- src/Access/User.h | 2 +- 26 files changed, 238 insertions(+), 155 deletions(-) diff --git a/src/Access/AccessBackup.cpp b/src/Access/AccessBackup.cpp index d9ee89b45ce..e8ea21852b5 100644 --- a/src/Access/AccessBackup.cpp +++ b/src/Access/AccessBackup.cpp @@ -29,6 +29,7 @@ namespace DB namespace ErrorCodes { extern const int CANNOT_RESTORE_TABLE; + extern const int ACCESS_ENTITY_ALREADY_EXISTS; extern const int LOGICAL_ERROR; } @@ -175,9 +176,46 @@ namespace return res; } - std::unordered_map resolveDependencies(const std::unordered_map> & dependencies, const AccessControl & access_control, bool allow_unresolved_dependencies) + /// Checks if new entities (which we're going to restore) already exist, + /// and either skips them or throws an exception depending on the restore settings. + void checkExistingEntities(std::vector> & entities, + std::unordered_map & old_to_new_id, + const AccessControl & access_control, + RestoreAccessCreationMode creation_mode) + { + if (creation_mode == RestoreAccessCreationMode::kReplace) + return; + + auto should_skip = [&](const std::pair & id_and_entity) + { + const auto & id = id_and_entity.first; + const auto & entity = *id_and_entity.second; + auto existing_id = access_control.find(entity.getType(), entity.getName()); + if (!existing_id) + { + return false; + } + else if (creation_mode == RestoreAccessCreationMode::kCreateIfNotExists) + { + old_to_new_id[id] = *existing_id; + return true; + } + else + { + throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "Cannot restore {} because it already exists", entity.formatTypeWithName()); + } + }; + + std::erase_if(entities, should_skip); + } + + /// If new entities (which we're going to restore) depend on other entities which are not going to be restored or not present in the backup + /// then we should try to replace those dependencies with already existing entities. + void resolveDependencies(const std::unordered_map> & dependencies, + std::unordered_map & old_to_new_ids, + const AccessControl & access_control, + bool allow_unresolved_dependencies) { - std::unordered_map old_to_new_ids; for (const auto & [id, name_and_type] : dependencies) { std::optional new_id; @@ -188,9 +226,9 @@ namespace if (new_id) old_to_new_ids.emplace(id, *new_id); } - return old_to_new_ids; } + /// Generates random IDs for the new entities. void generateRandomIDs(std::vector> & entities, std::unordered_map & old_to_new_ids) { Poco::UUIDGenerator generator; @@ -203,27 +241,12 @@ namespace } } - void replaceDependencies(std::vector> & entities, const std::unordered_map & old_to_new_ids) + /// Updates dependencies of the new entities using a specified map. + void replaceDependencies(std::vector> & entities, + const std::unordered_map & old_to_new_ids) { for (auto & entity : entities | boost::adaptors::map_values) - { - bool need_replace = false; - for (const auto & dependency : entity->findDependencies()) - { - if (old_to_new_ids.contains(dependency)) - { - need_replace = true; - break; - } - } - - if (!need_replace) - continue; - - auto new_entity = entity->clone(); - new_entity->replaceDependencies(old_to_new_ids); - entity = new_entity; - } + IAccessEntity::replaceDependencies(entity, old_to_new_ids); } AccessRightsElements getRequiredAccessToRestore(const std::vector> & entities) @@ -314,7 +337,9 @@ std::pair makeBackupEntryForAccess( AccessRestorerFromBackup::AccessRestorerFromBackup( const BackupPtr & backup_, const RestoreSettings & restore_settings_) - : backup(backup_), allow_unresolved_access_dependencies(restore_settings_.allow_unresolved_access_dependencies) + : backup(backup_) + , creation_mode(restore_settings_.create_access) + , allow_unresolved_dependencies(restore_settings_.allow_unresolved_access_dependencies) { } @@ -362,7 +387,9 @@ std::vector> AccessRestorerFromBackup::getAcces { auto new_entities = entities; - auto old_to_new_ids = resolveDependencies(dependencies, access_control, allow_unresolved_access_dependencies); + std::unordered_map old_to_new_ids; + checkExistingEntities(new_entities, old_to_new_ids, access_control, creation_mode); + resolveDependencies(dependencies, old_to_new_ids, access_control, allow_unresolved_dependencies); generateRandomIDs(new_entities, old_to_new_ids); replaceDependencies(new_entities, old_to_new_ids); diff --git a/src/Access/AccessBackup.h b/src/Access/AccessBackup.h index aa59d6bf201..51a1112e5d5 100644 --- a/src/Access/AccessBackup.h +++ b/src/Access/AccessBackup.h @@ -17,6 +17,7 @@ using BackupPtr = std::shared_ptr; class IBackupEntry; using BackupEntryPtr = std::shared_ptr; struct RestoreSettings; +enum class RestoreAccessCreationMode : uint8_t; /// Makes a backup of access entities of a specified type. @@ -45,7 +46,8 @@ public: private: BackupPtr backup; - bool allow_unresolved_access_dependencies = false; + RestoreAccessCreationMode creation_mode; + bool allow_unresolved_dependencies = false; std::vector> entities; std::unordered_map> dependencies; std::unordered_set data_paths; diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 95a467bbbe5..ec513f0692d 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -544,9 +544,9 @@ scope_guard AccessControl::subscribeForChanges(const std::vector & ids, co return changes_notifier->subscribeForChanges(ids, handler); } -bool AccessControl::insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) +bool AccessControl::insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) { - if (MultipleAccessStorage::insertImpl(id, entity, replace_if_exists, throw_if_exists)) + if (MultipleAccessStorage::insertImpl(id, entity, replace_if_exists, throw_if_exists, conflicting_id)) { changes_notifier->sendNotifications(); return true; diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h index bfaf256ad48..0c3bb9352f0 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -243,7 +243,7 @@ private: class CustomSettingsPrefixes; class PasswordComplexityRules; - bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index ee422f7d8ff..046c532cf5c 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -1,8 +1,6 @@ #include #include #include -#include -#include #include #include #include @@ -418,7 +416,7 @@ void DiskAccessStorage::setAllInMemory(const std::vector & ids_to_keep) @@ -507,14 +505,14 @@ std::optional> DiskAccessStorage::readNameWi } -bool DiskAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) +bool DiskAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) { std::lock_guard lock{mutex}; - return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists, /* write_on_disk = */ true); + return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists, conflicting_id, /* write_on_disk = */ true); } -bool DiskAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, bool write_on_disk) +bool DiskAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id, bool write_on_disk) { const String & name = new_entity->getName(); AccessEntityType type = new_entity->getType(); @@ -533,9 +531,15 @@ bool DiskAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & ne if (name_collision && !replace_if_exists) { if (throw_if_exists) + { throwNameCollisionCannotInsert(type, name); + } else + { + if (conflicting_id) + *conflicting_id = id_by_name; return false; + } } auto it_by_id = entries_by_id.find(id); @@ -548,7 +552,11 @@ bool DiskAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & ne throwIDCollisionCannotInsert(id, type, name, existing_entry.type, existing_entry.name); } else + { + if (conflicting_id) + *conflicting_id = id; return false; + } } if (write_on_disk) @@ -727,25 +735,4 @@ void DiskAccessStorage::deleteAccessEntityOnDisk(const UUID & id) const throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Couldn't delete {}", file_path); } - -void DiskAccessStorage::restoreFromBackup(RestorerFromBackup & restorer) -{ - if (!isRestoreAllowed()) - throwRestoreNotAllowed(); - - auto entities = restorer.getAccessEntitiesToRestore(); - if (entities.empty()) - return; - - auto create_access = restorer.getRestoreSettings().create_access; - bool replace_if_exists = (create_access == RestoreAccessCreationMode::kReplace); - bool throw_if_exists = (create_access == RestoreAccessCreationMode::kCreate); - - restorer.addDataRestoreTask([this, my_entities = std::move(entities), replace_if_exists, throw_if_exists] - { - for (const auto & [id, entity] : my_entities) - insert(id, entity, replace_if_exists, throw_if_exists); - }); -} - } diff --git a/src/Access/DiskAccessStorage.h b/src/Access/DiskAccessStorage.h index 38172b26970..40f2017dd97 100644 --- a/src/Access/DiskAccessStorage.h +++ b/src/Access/DiskAccessStorage.h @@ -34,14 +34,13 @@ public: bool exists(const UUID & id) const override; bool isBackupAllowed() const override { return backup_allowed; } - void restoreFromBackup(RestorerFromBackup & restorer) override; private: std::optional findImpl(AccessEntityType type, const String & name) const override; std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const override; - bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; @@ -55,7 +54,7 @@ private: void listsWritingThreadFunc() TSA_NO_THREAD_SAFETY_ANALYSIS; void stopListsWritingThread(); - bool insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, bool write_on_disk) TSA_REQUIRES(mutex); + bool insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id, bool write_on_disk) TSA_REQUIRES(mutex); bool updateNoLock(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists, bool write_on_disk) TSA_REQUIRES(mutex); bool removeNoLock(const UUID & id, bool throw_if_not_exists, bool write_on_disk) TSA_REQUIRES(mutex); diff --git a/src/Access/IAccessEntity.cpp b/src/Access/IAccessEntity.cpp index 5dc566fe456..9afa1b73597 100644 --- a/src/Access/IAccessEntity.cpp +++ b/src/Access/IAccessEntity.cpp @@ -9,4 +9,28 @@ bool IAccessEntity::equal(const IAccessEntity & other) const return (name == other.name) && (getType() == other.getType()); } +void IAccessEntity::replaceDependencies(std::shared_ptr & entity, const std::unordered_map & old_to_new_ids) +{ + if (old_to_new_ids.empty()) + return; + + bool need_replace_dependencies = false; + auto dependencies = entity->findDependencies(); + for (const auto & dependency : dependencies) + { + if (old_to_new_ids.contains(dependency)) + { + need_replace_dependencies = true; + break; + } + } + + if (!need_replace_dependencies) + return; + + auto new_entity = entity->clone(); + new_entity->replaceDependencies(old_to_new_ids); + entity = new_entity; +} + } diff --git a/src/Access/IAccessEntity.h b/src/Access/IAccessEntity.h index 5614a172f6f..2c2df7353c5 100644 --- a/src/Access/IAccessEntity.h +++ b/src/Access/IAccessEntity.h @@ -50,7 +50,8 @@ struct IAccessEntity virtual std::vector findDependencies() const { return {}; } /// Replaces dependencies according to a specified map. - virtual void replaceDependencies(const std::unordered_map & /* old_to_new_ids */) {} + void replaceDependencies(const std::unordered_map & old_to_new_ids) { doReplaceDependencies(old_to_new_ids); } + static void replaceDependencies(std::shared_ptr & entity, const std::unordered_map & old_to_new_ids); /// Whether this access entity should be written to a backup. virtual bool isBackupAllowed() const { return false; } @@ -66,6 +67,8 @@ protected: { return std::make_shared(typeid_cast(*this)); } + + virtual void doReplaceDependencies(const std::unordered_map & /* old_to_new_ids */) {} }; using AccessEntityPtr = std::shared_ptr; diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index e8e5d57722d..6c9b027fe53 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -4,6 +4,8 @@ #include #include #include +#include +#include #include #include #include @@ -14,10 +16,11 @@ #include #include #include +#include #include +#include #include - namespace DB { namespace ErrorCodes @@ -178,20 +181,20 @@ UUID IAccessStorage::insert(const AccessEntityPtr & entity) return *insert(entity, /* replace_if_exists = */ false, /* throw_if_exists = */ true); } -std::optional IAccessStorage::insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) +std::optional IAccessStorage::insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) { auto id = generateRandomID(); - if (insert(id, entity, replace_if_exists, throw_if_exists)) + if (insert(id, entity, replace_if_exists, throw_if_exists, conflicting_id)) return id; return std::nullopt; } -bool IAccessStorage::insert(const DB::UUID & id, const DB::AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) +bool IAccessStorage::insert(const DB::UUID & id, const DB::AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) { - return insertImpl(id, entity, replace_if_exists, throw_if_exists); + return insertImpl(id, entity, replace_if_exists, throw_if_exists, conflicting_id); } @@ -285,7 +288,7 @@ std::vector IAccessStorage::insertOrReplace(const std::vectorgetType(), entity->getName()); @@ -611,12 +614,50 @@ void IAccessStorage::backup(BackupEntriesCollector & backup_entries_collector, c } -void IAccessStorage::restoreFromBackup(RestorerFromBackup &) +void IAccessStorage::restoreFromBackup(RestorerFromBackup & restorer) { if (!isRestoreAllowed()) throwRestoreNotAllowed(); - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "restoreFromBackup() is not implemented in {}", getStorageType()); + if (isReplicated() && !acquireReplicatedRestore(restorer)) + return; + + auto entities = restorer.getAccessEntitiesToRestore(); + if (entities.empty()) + return; + + auto create_access = restorer.getRestoreSettings().create_access; + bool replace_if_exists = (create_access == RestoreAccessCreationMode::kReplace); + bool throw_if_exists = (create_access == RestoreAccessCreationMode::kCreate); + + restorer.addDataRestoreTask([this, my_entities = std::move(entities), replace_if_exists, throw_if_exists] mutable + { + std::unordered_map new_to_existing_ids; + for (auto & [id, entity] : my_entities) + { + UUID existing_entity_id; + if (!insert(id, entity, replace_if_exists, throw_if_exists, &existing_entity_id)) + { + /// Couldn't insert `entity` because there is an existing entity with the same name. + new_to_existing_ids[id] = existing_entity_id; + } + } + + if (!new_to_existing_ids.empty()) + { + /// If new entities restored from backup have dependencies on other entities from backup which were not restored because they existed, + /// then we should correct those dependencies. + auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr + { + auto res = entity; + IAccessEntity::replaceDependencies(res, new_to_existing_ids); + return res; + }; + std::vector ids; + boost::copy(my_entities | boost::adaptors::map_keys, std::back_inserter(ids)); + tryUpdate(ids, update_func); + } + }); } diff --git a/src/Access/IAccessStorage.h b/src/Access/IAccessStorage.h index e193b65e77a..a8ac75075d3 100644 --- a/src/Access/IAccessStorage.h +++ b/src/Access/IAccessStorage.h @@ -64,6 +64,9 @@ public: /// Returns true if this entity is readonly. virtual bool isReadOnly(const UUID &) const { return isReadOnly(); } + /// Returns true if this storage is replicated. + virtual bool isReplicated() const { return false; } + /// Starts periodic reloading and updating of entities in this storage. virtual void startPeriodicReloading() {} @@ -153,8 +156,8 @@ public: /// Inserts an entity to the storage. Returns ID of a new entry in the storage. /// Throws an exception if the specified name already exists. UUID insert(const AccessEntityPtr & entity); - std::optional insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); - bool insert(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); + std::optional insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id = nullptr); + bool insert(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id = nullptr); std::vector insert(const std::vector & multiple_entities, bool replace_if_exists = false, bool throw_if_exists = true); std::vector insert(const std::vector & multiple_entities, const std::vector & ids, bool replace_if_exists = false, bool throw_if_exists = true); @@ -218,7 +221,7 @@ protected: virtual std::vector findAllImpl(AccessEntityType type) const = 0; virtual AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const = 0; virtual std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const; - virtual bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); + virtual bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id); virtual bool removeImpl(const UUID & id, bool throw_if_not_exists); virtual bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists); virtual std::optional authenticateImpl( @@ -240,6 +243,7 @@ protected: LoggerPtr getLogger() const; static String formatEntityTypeWithName(AccessEntityType type, const String & name) { return AccessEntityTypeInfo::get(type).formatEntityNameWithType(name); } static void clearConflictsInEntitiesList(std::vector> & entities, LoggerPtr log_); + virtual bool acquireReplicatedRestore(RestorerFromBackup &) const { return false; } [[noreturn]] void throwNotFound(const UUID & id) const; [[noreturn]] void throwNotFound(AccessEntityType type, const String & name) const; [[noreturn]] static void throwBadCast(const UUID & id, AccessEntityType type, const String & name, AccessEntityType required_type); diff --git a/src/Access/MemoryAccessStorage.cpp b/src/Access/MemoryAccessStorage.cpp index 791030b9b12..3b5a987fc6e 100644 --- a/src/Access/MemoryAccessStorage.cpp +++ b/src/Access/MemoryAccessStorage.cpp @@ -1,7 +1,5 @@ #include #include -#include -#include #include #include #include @@ -63,14 +61,14 @@ AccessEntityPtr MemoryAccessStorage::readImpl(const UUID & id, bool throw_if_not } -bool MemoryAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) +bool MemoryAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) { std::lock_guard lock{mutex}; - return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists); + return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists, conflicting_id); } -bool MemoryAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) +bool MemoryAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) { const String & name = new_entity->getName(); AccessEntityType type = new_entity->getType(); @@ -86,9 +84,15 @@ bool MemoryAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & if (name_collision && !replace_if_exists) { if (throw_if_exists) + { throwNameCollisionCannotInsert(type, name); + } else + { + if (conflicting_id) + *conflicting_id = id_by_name; return false; + } } auto it_by_id = entries_by_id.find(id); @@ -97,9 +101,15 @@ bool MemoryAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & { const auto & existing_entry = it_by_id->second; if (throw_if_exists) + { throwIDCollisionCannotInsert(id, type, name, existing_entry.entity->getType(), existing_entry.entity->getName()); + } else + { + if (conflicting_id) + *conflicting_id = id; return false; + } } /// Remove collisions if necessary. @@ -270,28 +280,7 @@ void MemoryAccessStorage::setAll(const std::vector findImpl(AccessEntityType type, const String & name) const override; std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; - bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; - bool insertNoLock(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); + bool insertNoLock(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id); bool removeNoLock(const UUID & id, bool throw_if_not_exists); bool updateNoLock(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists); diff --git a/src/Access/MultipleAccessStorage.cpp b/src/Access/MultipleAccessStorage.cpp index fda6601e4c6..f1da8359d48 100644 --- a/src/Access/MultipleAccessStorage.cpp +++ b/src/Access/MultipleAccessStorage.cpp @@ -353,7 +353,7 @@ void MultipleAccessStorage::reload(ReloadMode reload_mode) } -bool MultipleAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) +bool MultipleAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) { std::shared_ptr storage_for_insertion; @@ -376,7 +376,7 @@ bool MultipleAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & getStorageName()); } - if (storage_for_insertion->insert(id, entity, replace_if_exists, throw_if_exists)) + if (storage_for_insertion->insert(id, entity, replace_if_exists, throw_if_exists, conflicting_id)) { std::lock_guard lock{mutex}; ids_cache.set(id, storage_for_insertion); diff --git a/src/Access/MultipleAccessStorage.h b/src/Access/MultipleAccessStorage.h index e1543c59b67..352cc7f7457 100644 --- a/src/Access/MultipleAccessStorage.h +++ b/src/Access/MultipleAccessStorage.h @@ -67,7 +67,7 @@ protected: std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const override; - bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; std::optional authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const override; diff --git a/src/Access/Quota.cpp b/src/Access/Quota.cpp index 87b15e722c3..ead5f77ce57 100644 --- a/src/Access/Quota.cpp +++ b/src/Access/Quota.cpp @@ -24,7 +24,7 @@ std::vector Quota::findDependencies() const return to_roles.findDependencies(); } -void Quota::replaceDependencies(const std::unordered_map & old_to_new_ids) +void Quota::doReplaceDependencies(const std::unordered_map & old_to_new_ids) { to_roles.replaceDependencies(old_to_new_ids); } diff --git a/src/Access/Quota.h b/src/Access/Quota.h index eb9edb14fb0..69ec2eb53a5 100644 --- a/src/Access/Quota.h +++ b/src/Access/Quota.h @@ -47,7 +47,7 @@ struct Quota : public IAccessEntity AccessEntityType getType() const override { return TYPE; } std::vector findDependencies() const override; - void replaceDependencies(const std::unordered_map & old_to_new_ids) override; + void doReplaceDependencies(const std::unordered_map & old_to_new_ids) override; bool isBackupAllowed() const override { return true; } }; diff --git a/src/Access/ReplicatedAccessStorage.cpp b/src/Access/ReplicatedAccessStorage.cpp index ed114327041..9039a3b98b7 100644 --- a/src/Access/ReplicatedAccessStorage.cpp +++ b/src/Access/ReplicatedAccessStorage.cpp @@ -5,10 +5,9 @@ #include #include #include -#include -#include #include #include +#include #include #include #include @@ -120,7 +119,7 @@ static void retryOnZooKeeperUserError(size_t attempts, Func && function) } } -bool ReplicatedAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) +bool ReplicatedAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) { const AccessEntityTypeInfo type_info = AccessEntityTypeInfo::get(new_entity->getType()); const String & name = new_entity->getName(); @@ -128,7 +127,7 @@ bool ReplicatedAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr auto zookeeper = getZooKeeper(); bool ok = false; - retryOnZooKeeperUserError(10, [&]{ ok = insertZooKeeper(zookeeper, id, new_entity, replace_if_exists, throw_if_exists); }); + retryOnZooKeeperUserError(10, [&]{ ok = insertZooKeeper(zookeeper, id, new_entity, replace_if_exists, throw_if_exists, conflicting_id); }); if (!ok) return false; @@ -143,7 +142,8 @@ bool ReplicatedAccessStorage::insertZooKeeper( const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, - bool throw_if_exists) + bool throw_if_exists, + UUID * conflicting_id) { const String & name = new_entity->getName(); const AccessEntityType type = new_entity->getType(); @@ -167,27 +167,52 @@ bool ReplicatedAccessStorage::insertZooKeeper( if (res == Coordination::Error::ZNODEEXISTS) { - if (!throw_if_exists && !replace_if_exists) - return false; /// Couldn't insert a new entity. - - if (throw_if_exists) + if (!replace_if_exists) { if (responses[0]->error == Coordination::Error::ZNODEEXISTS) { - /// To fail with a nice error message, we need info about what already exists. - /// This itself could fail if the conflicting uuid disappears in the meantime. - /// If that happens, then we'll just retry from the start. - String existing_entity_definition = zookeeper->get(entity_path); + /// Couldn't insert the new entity because there is an existing entity with such UUID. + if (throw_if_exists) + { + /// To fail with a nice error message, we need info about what already exists. + /// This itself can fail if the conflicting uuid disappears in the meantime. + /// If that happens, then retryOnZooKeeperUserError() will just retry the operation from the start. + String existing_entity_definition = zookeeper->get(entity_path); - AccessEntityPtr existing_entity = deserializeAccessEntity(existing_entity_definition, entity_path); - AccessEntityType existing_type = existing_entity->getType(); - String existing_name = existing_entity->getName(); - throwIDCollisionCannotInsert(id, type, name, existing_type, existing_name); + AccessEntityPtr existing_entity = deserializeAccessEntity(existing_entity_definition, entity_path); + AccessEntityType existing_type = existing_entity->getType(); + String existing_name = existing_entity->getName(); + throwIDCollisionCannotInsert(id, type, name, existing_type, existing_name); + } + else + { + if (conflicting_id) + *conflicting_id = id; + return false; + } + } + else if (responses[1]->error == Coordination::Error::ZNODEEXISTS) + { + /// Couldn't insert the new entity because there is an existing entity with the same name. + if (throw_if_exists) + { + throwNameCollisionCannotInsert(type, name); + } + else + { + if (conflicting_id) + { + /// Get UUID of the existing entry with the same name. + /// This itself can fail if the conflicting name disappears in the meantime. + /// If that happens, then retryOnZooKeeperUserError() will just retry the operation from the start. + *conflicting_id = parseUUID(zookeeper->get(name_path)); + } + return false; + } } else { - /// Couldn't insert the new entity because there is an existing entity with such name. - throwNameCollisionCannotInsert(type, name); + zkutil::KeeperMultiException::check(res, ops, responses); } } @@ -693,28 +718,10 @@ void ReplicatedAccessStorage::backup(BackupEntriesCollector & backup_entries_col } -void ReplicatedAccessStorage::restoreFromBackup(RestorerFromBackup & restorer) +bool ReplicatedAccessStorage::acquireReplicatedRestore(RestorerFromBackup & restorer) const { - if (!isRestoreAllowed()) - throwRestoreNotAllowed(); - auto restore_coordination = restorer.getRestoreCoordination(); - if (!restore_coordination->acquireReplicatedAccessStorage(zookeeper_path)) - return; - - auto entities = restorer.getAccessEntitiesToRestore(); - if (entities.empty()) - return; - - auto create_access = restorer.getRestoreSettings().create_access; - bool replace_if_exists = (create_access == RestoreAccessCreationMode::kReplace); - bool throw_if_exists = (create_access == RestoreAccessCreationMode::kCreate); - - restorer.addDataRestoreTask([this, my_entities = std::move(entities), replace_if_exists, throw_if_exists] - { - for (const auto & [id, entity] : my_entities) - insert(id, entity, replace_if_exists, throw_if_exists); - }); + return restore_coordination->acquireReplicatedAccessStorage(zookeeper_path); } } diff --git a/src/Access/ReplicatedAccessStorage.h b/src/Access/ReplicatedAccessStorage.h index f8518226997..528dbb31c24 100644 --- a/src/Access/ReplicatedAccessStorage.h +++ b/src/Access/ReplicatedAccessStorage.h @@ -26,6 +26,7 @@ public: void shutdown() override; const char * getStorageType() const override { return STORAGE_TYPE; } + bool isReplicated() const override { return true; } void startPeriodicReloading() override { startWatchingThread(); } void stopPeriodicReloading() override { stopWatchingThread(); } @@ -35,7 +36,6 @@ public: bool isBackupAllowed() const override { return backup_allowed; } void backup(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, AccessEntityType type) const override; - void restoreFromBackup(RestorerFromBackup & restorer) override; private: String zookeeper_path; @@ -48,11 +48,11 @@ private: std::unique_ptr watching_thread; std::shared_ptr> watched_queue; - bool insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; - bool insertZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); + bool insertZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, UUID * conflicting_id); bool removeZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, bool throw_if_not_exists); bool updateZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists); @@ -80,6 +80,7 @@ private: std::optional findImpl(AccessEntityType type, const String & name) const override; std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; + bool acquireReplicatedRestore(RestorerFromBackup & restorer) const override; mutable std::mutex mutex; MemoryAccessStorage memory_storage TSA_GUARDED_BY(mutex); diff --git a/src/Access/Role.cpp b/src/Access/Role.cpp index 089488e7aba..f6250594103 100644 --- a/src/Access/Role.cpp +++ b/src/Access/Role.cpp @@ -21,7 +21,7 @@ std::vector Role::findDependencies() const return res; } -void Role::replaceDependencies(const std::unordered_map & old_to_new_ids) +void Role::doReplaceDependencies(const std::unordered_map & old_to_new_ids) { granted_roles.replaceDependencies(old_to_new_ids); settings.replaceDependencies(old_to_new_ids); diff --git a/src/Access/Role.h b/src/Access/Role.h index b2f879dc357..c7f98585a6c 100644 --- a/src/Access/Role.h +++ b/src/Access/Role.h @@ -21,7 +21,7 @@ struct Role : public IAccessEntity AccessEntityType getType() const override { return TYPE; } std::vector findDependencies() const override; - void replaceDependencies(const std::unordered_map & old_to_new_ids) override; + void doReplaceDependencies(const std::unordered_map & old_to_new_ids) override; bool isBackupAllowed() const override { return settings.isBackupAllowed(); } }; diff --git a/src/Access/RowPolicy.cpp b/src/Access/RowPolicy.cpp index d25b9e259b1..8724d0f513c 100644 --- a/src/Access/RowPolicy.cpp +++ b/src/Access/RowPolicy.cpp @@ -63,7 +63,7 @@ std::vector RowPolicy::findDependencies() const return to_roles.findDependencies(); } -void RowPolicy::replaceDependencies(const std::unordered_map & old_to_new_ids) +void RowPolicy::doReplaceDependencies(const std::unordered_map & old_to_new_ids) { to_roles.replaceDependencies(old_to_new_ids); } diff --git a/src/Access/RowPolicy.h b/src/Access/RowPolicy.h index 9c190458620..5cfe85c186a 100644 --- a/src/Access/RowPolicy.h +++ b/src/Access/RowPolicy.h @@ -50,7 +50,7 @@ struct RowPolicy : public IAccessEntity AccessEntityType getType() const override { return TYPE; } std::vector findDependencies() const override; - void replaceDependencies(const std::unordered_map & old_to_new_ids) override; + void doReplaceDependencies(const std::unordered_map & old_to_new_ids) override; bool isBackupAllowed() const override { return true; } /// Which roles or users should use this row policy. diff --git a/src/Access/SettingsProfile.cpp b/src/Access/SettingsProfile.cpp index 48aa48040ab..632bd97fbf5 100644 --- a/src/Access/SettingsProfile.cpp +++ b/src/Access/SettingsProfile.cpp @@ -21,7 +21,7 @@ std::vector SettingsProfile::findDependencies() const return res; } -void SettingsProfile::replaceDependencies(const std::unordered_map & old_to_new_ids) +void SettingsProfile::doReplaceDependencies(const std::unordered_map & old_to_new_ids) { elements.replaceDependencies(old_to_new_ids); to_roles.replaceDependencies(old_to_new_ids); diff --git a/src/Access/SettingsProfile.h b/src/Access/SettingsProfile.h index f85630d324d..6bcaf6fef30 100644 --- a/src/Access/SettingsProfile.h +++ b/src/Access/SettingsProfile.h @@ -22,7 +22,7 @@ struct SettingsProfile : public IAccessEntity AccessEntityType getType() const override { return TYPE; } std::vector findDependencies() const override; - void replaceDependencies(const std::unordered_map & old_to_new_ids) override; + void doReplaceDependencies(const std::unordered_map & old_to_new_ids) override; bool isBackupAllowed() const override { return elements.isBackupAllowed(); } }; diff --git a/src/Access/User.cpp b/src/Access/User.cpp index 0bd11000775..2052527f4ae 100644 --- a/src/Access/User.cpp +++ b/src/Access/User.cpp @@ -49,7 +49,7 @@ std::vector User::findDependencies() const return res; } -void User::replaceDependencies(const std::unordered_map & old_to_new_ids) +void User::doReplaceDependencies(const std::unordered_map & old_to_new_ids) { default_roles.replaceDependencies(old_to_new_ids); granted_roles.replaceDependencies(old_to_new_ids); diff --git a/src/Access/User.h b/src/Access/User.h index 28f16a76b0c..7f91c1e3756 100644 --- a/src/Access/User.h +++ b/src/Access/User.h @@ -32,7 +32,7 @@ struct User : public IAccessEntity void setName(const String & name_) override; std::vector findDependencies() const override; - void replaceDependencies(const std::unordered_map & old_to_new_ids) override; + void doReplaceDependencies(const std::unordered_map & old_to_new_ids) override; bool isBackupAllowed() const override { return settings.isBackupAllowed(); } }; From f8f72ccb00d3bc35212ec94a272e055b1fe32c76 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 6 Sep 2024 20:56:09 +0200 Subject: [PATCH 088/103] Add test. --- ..._restore_user_with_existing_role.reference | 6 ++ .../03231_restore_user_with_existing_role.sh | 77 +++++++++++++++++++ 2 files changed, 83 insertions(+) create mode 100644 tests/queries/0_stateless/03231_restore_user_with_existing_role.reference create mode 100755 tests/queries/0_stateless/03231_restore_user_with_existing_role.sh diff --git a/tests/queries/0_stateless/03231_restore_user_with_existing_role.reference b/tests/queries/0_stateless/03231_restore_user_with_existing_role.reference new file mode 100644 index 00000000000..cad1bf13574 --- /dev/null +++ b/tests/queries/0_stateless/03231_restore_user_with_existing_role.reference @@ -0,0 +1,6 @@ +Everything dropped +User dropped +Nothing dropped +Nothing dropped, mode=replace +Nothing dropped, mode=create +ACCESS_ENTITY_ALREADY_EXISTS diff --git a/tests/queries/0_stateless/03231_restore_user_with_existing_role.sh b/tests/queries/0_stateless/03231_restore_user_with_existing_role.sh new file mode 100755 index 00000000000..7862911de04 --- /dev/null +++ b/tests/queries/0_stateless/03231_restore_user_with_existing_role.sh @@ -0,0 +1,77 @@ +#!/usr/bin/env bash +# Tags: no-parallel + +# Disabled parallel since RESTORE can only restore either all users or no users +# (it can't restore only users added by the current test run), +# so a RESTORE from a parallel test run could recreate our users before we expect that. + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +user_a="user_a_${CLICKHOUSE_TEST_UNIQUE_NAME}" +role_b="role_b_${CLICKHOUSE_TEST_UNIQUE_NAME}" + +${CLICKHOUSE_CLIENT} -m --query " +CREATE ROLE ${role_b} SETTINGS custom_x=1; +CREATE USER ${user_a} DEFAULT ROLE ${role_b} SETTINGS custom_x=2; +" + +backup_name="Disk('backups', '${CLICKHOUSE_TEST_UNIQUE_NAME}')" + +${CLICKHOUSE_CLIENT} --query "BACKUP TABLE system.users, TABLE system.roles TO ${backup_name} FORMAT Null" +${CLICKHOUSE_CLIENT} --query "RESTORE ALL FROM ${backup_name} FORMAT Null" + +do_check() +{ + local replacements + replacements="s/${user_a}/user_a/g; s/${role_b}/role_b/g" + local check_info + check_info=$(${CLICKHOUSE_CLIENT} -mq " + SHOW CREATE USER ${user_a}; + SHOW GRANTS FOR ${user_a}; + SHOW CREATE ROLE ${role_b}; + SHOW GRANTS FOR ${role_b}; + " | sed "${replacements}") + local expected + expected=$'CREATE USER user_a DEFAULT ROLE role_b SETTINGS custom_x = 2\nGRANT role_b TO user_a\nCREATE ROLE role_b SETTINGS custom_x = 1' + if [[ "${check_info}" != "${expected}" ]]; then + echo "Assertion failed:" + echo "\"${check_info}\"" + echo "!=" + echo "\"${expected}\"" + echo "Test database: ${CLICKHOUSE_DATABASE}" >&2 + fi +} + +echo "Everything dropped" +${CLICKHOUSE_CLIENT} --query "DROP USER ${user_a}" +${CLICKHOUSE_CLIENT} --query "DROP ROLE ${role_b}" +${CLICKHOUSE_CLIENT} --query "RESTORE ALL FROM ${backup_name} FORMAT Null" +do_check + +echo "User dropped" +${CLICKHOUSE_CLIENT} --query "DROP USER ${user_a}" +${CLICKHOUSE_CLIENT} --query "RESTORE ALL FROM ${backup_name} FORMAT Null" +do_check + +# TODO: Cannot restore a dropped role granted to an existing user. The result after RESTORE ALL below is the following: +# CREATE USER user_a DEFAULT ROLE NONE SETTINGS custom_x = 2; GRANT NONE TO user_a; CREATE ROLE role_b SETTINGS custom_x = 1 +# because `role_b` is restored but not granted to existing user `user_a`. +# +# echo "Role dropped" +# ${CLICKHOUSE_CLIENT} --query "DROP ROLE ${role_b}" +# ${CLICKHOUSE_CLIENT} --query "RESTORE ALL FROM ${backup_name} FORMAT Null" +# do_check + +echo "Nothing dropped" +${CLICKHOUSE_CLIENT} --query "RESTORE ALL FROM ${backup_name} FORMAT Null" +do_check + +echo "Nothing dropped, mode=replace" +${CLICKHOUSE_CLIENT} --query "RESTORE ALL FROM ${backup_name} SETTINGS create_access='replace' FORMAT Null" +do_check + +echo "Nothing dropped, mode=create" +${CLICKHOUSE_CLIENT} --query "RESTORE ALL FROM ${backup_name} SETTINGS create_access='create' FORMAT Null" 2>&1 | grep -om1 "ACCESS_ENTITY_ALREADY_EXISTS" +do_check From 983b061b58c0d0c8421b07e2974795da16c7be0f Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 17 Sep 2024 12:56:10 +0200 Subject: [PATCH 089/103] Corrections after review. --- src/Access/IAccessStorage.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index 6c9b027fe53..29475461c45 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -630,10 +630,10 @@ void IAccessStorage::restoreFromBackup(RestorerFromBackup & restorer) bool replace_if_exists = (create_access == RestoreAccessCreationMode::kReplace); bool throw_if_exists = (create_access == RestoreAccessCreationMode::kCreate); - restorer.addDataRestoreTask([this, my_entities = std::move(entities), replace_if_exists, throw_if_exists] mutable + restorer.addDataRestoreTask([this, entities_to_restore = std::move(entities), replace_if_exists, throw_if_exists] mutable { std::unordered_map new_to_existing_ids; - for (auto & [id, entity] : my_entities) + for (auto & [id, entity] : entities_to_restore) { UUID existing_entity_id; if (!insert(id, entity, replace_if_exists, throw_if_exists, &existing_entity_id)) @@ -654,7 +654,8 @@ void IAccessStorage::restoreFromBackup(RestorerFromBackup & restorer) return res; }; std::vector ids; - boost::copy(my_entities | boost::adaptors::map_keys, std::back_inserter(ids)); + ids.reserve(entities_to_restore.size()); + boost::copy(entities_to_restore | boost::adaptors::map_keys, std::back_inserter(ids)); tryUpdate(ids, update_func); } }); From f768717be86d0198847783f10b4c9a8f662d78d5 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 17 Sep 2024 13:05:02 +0200 Subject: [PATCH 090/103] Fix test. --- .../0_stateless/03231_restore_user_with_existing_role.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03231_restore_user_with_existing_role.sh b/tests/queries/0_stateless/03231_restore_user_with_existing_role.sh index 7862911de04..04f907b719d 100755 --- a/tests/queries/0_stateless/03231_restore_user_with_existing_role.sh +++ b/tests/queries/0_stateless/03231_restore_user_with_existing_role.sh @@ -34,7 +34,7 @@ do_check() SHOW GRANTS FOR ${role_b}; " | sed "${replacements}") local expected - expected=$'CREATE USER user_a DEFAULT ROLE role_b SETTINGS custom_x = 2\nGRANT role_b TO user_a\nCREATE ROLE role_b SETTINGS custom_x = 1' + expected=$'CREATE USER user_a IDENTIFIED WITH no_password DEFAULT ROLE role_b SETTINGS custom_x = 2\nGRANT role_b TO user_a\nCREATE ROLE role_b SETTINGS custom_x = 1' if [[ "${check_info}" != "${expected}" ]]; then echo "Assertion failed:" echo "\"${check_info}\"" From 3a05282bced2b8e4a557ef452afdd5c3fc62471f Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 17 Sep 2024 14:26:31 +0200 Subject: [PATCH 091/103] Update assert --- src/Interpreters/Cache/FileCache.cpp | 7 ++++++- tests/clickhouse-test | 2 +- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 60db406ca72..ffe9a611014 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -718,7 +718,12 @@ FileCache::getOrSet( } } - chassert(file_segments_limit ? file_segments.back()->range().left <= result_range.right : file_segments.back()->range().contains(result_range.right)); + chassert(file_segments_limit + ? file_segments.back()->range().left <= result_range.right + : file_segments.back()->range().contains(result_range.right), + fmt::format("Unexpected state. Back: {}, result range: {}, limit: {}", + file_segments.back()->range().toString(), result_range.toString(), file_segments_limit)); + chassert(!file_segments_limit || file_segments.size() <= file_segments_limit); return std::make_unique(std::move(file_segments)); diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 75a180f356b..a005fdcc439 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -835,7 +835,7 @@ class SettingsRandomizer: ), "remote_filesystem_read_method": lambda: random.choice(["read", "threadpool"]), "local_filesystem_read_prefetch": lambda: random.randint(0, 1), - "filesystem_cache_segments_batch_size": lambda: random.choice([0, 3, 10, 50]), + "filesystem_cache_segments_batch_size": lambda: random.choice([0, 1, 2, 3, 5, 10, 50, 100]), "read_from_filesystem_cache_if_exists_otherwise_bypass_cache": lambda: random.randint( 0, 1 ), From 813bcd896f170b3403088aad01890b07b1065f2b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 17 Sep 2024 12:30:12 +0000 Subject: [PATCH 092/103] Bump to v18.8 --- contrib/postgres | 2 +- contrib/postgres-cmake/pg_config.h | 46 ++++++++++++++++-------------- 2 files changed, 25 insertions(+), 23 deletions(-) diff --git a/contrib/postgres b/contrib/postgres index 665ff8c164d..cfd77000af2 160000 --- a/contrib/postgres +++ b/contrib/postgres @@ -1 +1 @@ -Subproject commit 665ff8c164d56d012e359735efe4d400c0564b44 +Subproject commit cfd77000af28469fcb650485bad65a35e7649e41 diff --git a/contrib/postgres-cmake/pg_config.h b/contrib/postgres-cmake/pg_config.h index ce16eab2239..0060d4960db 100644 --- a/contrib/postgres-cmake/pg_config.h +++ b/contrib/postgres-cmake/pg_config.h @@ -1,18 +1,6 @@ /* src/include/pg_config.h. Generated from pg_config.h.in by configure. */ /* src/include/pg_config.h.in. Generated from configure.in by autoheader. */ -/* Define to the type of arg 1 of 'accept' */ -#define ACCEPT_TYPE_ARG1 int - -/* Define to the type of arg 2 of 'accept' */ -#define ACCEPT_TYPE_ARG2 struct sockaddr * - -/* Define to the type of arg 3 of 'accept' */ -#define ACCEPT_TYPE_ARG3 size_t - -/* Define to the return type of 'accept' */ -#define ACCEPT_TYPE_RETURN int - /* Define if building universal (internal helper macro) */ /* #undef AC_APPLE_UNIVERSAL_BUILD */ @@ -49,6 +37,9 @@ /* Define to the default TCP port number as a string constant. */ #define DEF_PGPORT_STR "5432" +/* Define to the file name extension of dynamically-loadable modules. */ +#define DLSUFFIX ".so" + /* Define to build with GSSAPI support. (--with-gssapi) */ //#define ENABLE_GSS 0 @@ -122,6 +113,9 @@ don't. */ #define HAVE_DECL_SNPRINTF 1 +/* Define to 1 if you have the declaration of `sigwait', and to 0 if you don't. */ +#define HAVE_DECL_SIGWAIT 1 + /* Define to 1 if you have the declaration of `strlcat', and to 0 if you don't. */ #if OS_DARWIN @@ -257,6 +251,9 @@ /* Define to 1 if you have the `inet_aton' function. */ #define HAVE_INET_ATON 1 +/* Define to 1 if you have the `inet_pton' function. */ +#define HAVE_INET_PTON 1 + /* Define to 1 if the system has the type `int64'. */ /* #undef HAVE_INT64 */ @@ -323,6 +320,9 @@ /* Define to 1 if you have the `z' library (-lz). */ #define HAVE_LIBZ 1 +/* Define to 1 if you have the `zstd' library (-lzstd). */ +/* #undef HAVE_LIBZSTD */ + /* Define to 1 if constants of type 'long long int' should have the suffix LL. */ #define HAVE_LL_CONSTANTS 1 @@ -378,6 +378,9 @@ /* Define to 1 if you have the header file. */ #define HAVE_POLL_H 1 +/* Define to 1 if you have a POSIX-conforming sigwait declaration. */ +/* #undef HAVE_POSIX_DECL_SIGWAIT */ + /* Define to 1 if you have the `posix_fadvise' function. */ #define HAVE_POSIX_FADVISE 1 @@ -408,9 +411,6 @@ /* Define to 1 if you have the header file. */ #define HAVE_PWD_H 1 -/* Define to 1 if you have the `random' function. */ -#define HAVE_RANDOM 1 - /* Define to 1 if you have the header file. */ /* #undef HAVE_READLINE_H */ @@ -426,10 +426,6 @@ /* Define to 1 if you have the `rint' function. */ #define HAVE_RINT 1 -/* Define to 1 if you have the global variable - 'rl_completion_append_character'. */ -/* #undef HAVE_RL_COMPLETION_APPEND_CHARACTER */ - /* Define to 1 if you have the `rl_completion_matches' function. */ #define HAVE_RL_COMPLETION_MATCHES 1 @@ -439,6 +435,9 @@ /* Define to 1 if you have the `rl_reset_screen_size' function. */ /* #undef HAVE_RL_RESET_SCREEN_SIZE */ +/* Define to 1 if you have the `rl_variable_bind' function. */ +#define HAVE_RL_VARIABLE_BIND 1 + /* Define to 1 if you have the header file. */ #define HAVE_SECURITY_PAM_APPL_H 1 @@ -451,6 +450,9 @@ /* Define to 1 if you have the `shm_open' function. */ #define HAVE_SHM_OPEN 1 +/* Define to 1 if the system has the type `socklen_t'. */ +#define HAVE_SOCKLEN_T 1 + /* Define to 1 if you have the `sigprocmask' function. */ #define HAVE_SIGPROCMASK 1 @@ -466,9 +468,6 @@ /* Define to 1 if you have spinlocks. */ #define HAVE_SPINLOCKS 1 -/* Define to 1 if you have the `srandom' function. */ -#define HAVE_SRANDOM 1 - /* Define to 1 if you have the `SSL_CTX_set_num_tickets' function. */ /* #define HAVE_SSL_CTX_SET_NUM_TICKETS */ @@ -885,6 +884,9 @@ /* Define to select Win32-style shared memory. */ /* #undef USE_WIN32_SHARED_MEMORY */ +/* Define to 1 to build with ZSTD support. (--with-zstd) */ +/* #undef USE_ZSTD */ + /* Define to 1 if `wcstombs_l' requires . */ /* #undef WCSTOMBS_L_IN_XLOCALE */ From 5ce8604869e9c21bc5b707a985bf4247c1c5a0fd Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 17 Sep 2024 12:37:23 +0000 Subject: [PATCH 093/103] Automatic style fix --- tests/clickhouse-test | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a005fdcc439..810bae86cb0 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -835,7 +835,9 @@ class SettingsRandomizer: ), "remote_filesystem_read_method": lambda: random.choice(["read", "threadpool"]), "local_filesystem_read_prefetch": lambda: random.randint(0, 1), - "filesystem_cache_segments_batch_size": lambda: random.choice([0, 1, 2, 3, 5, 10, 50, 100]), + "filesystem_cache_segments_batch_size": lambda: random.choice( + [0, 1, 2, 3, 5, 10, 50, 100] + ), "read_from_filesystem_cache_if_exists_otherwise_bypass_cache": lambda: random.randint( 0, 1 ), From 4b69d8e2ca2a68e2030d31151d6adb63a79de836 Mon Sep 17 00:00:00 2001 From: NikBarykin Date: Tue, 17 Sep 2024 15:52:20 +0300 Subject: [PATCH 094/103] Fix CE --- src/Databases/MySQL/DatabaseMaterializedMySQL.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp index 50c7a5bf588..2b728039632 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp @@ -291,11 +291,11 @@ void registerDatabaseMaterializedMySQL(DatabaseFactory & factory) std::move(materialize_mode_settings)); }; - DatabaseFactory::Features features{ + DatabaseFactory::EngineFeatures features{ .supports_arguments = true, .supports_settings = true, .supports_table_overrides = true, - } + }; factory.registerDatabase("MaterializeMySQL", create_fn, features); factory.registerDatabase("MaterializedMySQL", create_fn, features); } From aba7de5091ffc52c864cc004c44ff4be966bb126 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 17 Sep 2024 16:53:32 +0200 Subject: [PATCH 095/103] Verify that there are no intersecting parts in the resulting all_parts_to_read --- .../ParallelReplicasReadingCoordinator.cpp | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index ddbed5db7dc..0a25874cfd9 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -911,6 +911,24 @@ void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRa std::sort(ranges.begin(), ranges.end()); } +#ifndef NDEBUG + /// Double check that there are no intersecting parts + { + auto part_it = all_parts_to_read.begin(); + auto next_part_it = part_it; + if (next_part_it != all_parts_to_read.end()) + ++next_part_it; + while (next_part_it != all_parts_to_read.end()) + { + chassert(part_it->description.info.isDisjoint(next_part_it->description.info), + fmt::format("Parts {} and {} intersect", + part_it->description.info.getPartNameV1(), next_part_it->description.info.getPartNameV1())); + ++part_it; + ++next_part_it; + } + } +#endif + state_initialized = true; // progress_callback is not set when local plan is used for initiator From 190d3f04c9fb03e8a8c64ce1b25d5536e7835ad7 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 17 Sep 2024 16:54:49 +0200 Subject: [PATCH 096/103] More optimal check for intrsecting parts in DefaultCoordinator init --- .../ParallelReplicasReadingCoordinator.cpp | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 0a25874cfd9..603584af6c2 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -376,17 +376,20 @@ void DefaultCoordinator::initializeReadingState(InitialAllRangesAnnouncement ann if (state_initialized) return; - for (auto && part : announcement.description) { - auto intersecting_it = std::find_if( - all_parts_to_read.begin(), - all_parts_to_read.end(), - [&part](const Part & other) { return !other.description.info.isDisjoint(part.info); }); + /// To speedup search for adjacent parts + Parts known_parts(all_parts_to_read.begin(), all_parts_to_read.end()); - if (intersecting_it != all_parts_to_read.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Intersecting parts found in announcement"); + for (auto && part : announcement.description) + { + auto intersecting_it = known_parts.lower_bound(Part{.description = part, .replicas = {}}); - all_parts_to_read.push_back(Part{.description = std::move(part), .replicas = {announcement.replica_num}}); + if (intersecting_it != known_parts.end() && !intersecting_it->description.info.isDisjoint(part.info)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Intersecting parts found in announcement"); + + all_parts_to_read.push_back(Part{.description = std::move(part), .replicas = {announcement.replica_num}}); + known_parts.emplace(Part{.description = part, .replicas = {}}); + } } std::ranges::sort( From 3674c97ebba63bc88c6bc03f630124afa314053a Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 17 Sep 2024 17:49:02 +0200 Subject: [PATCH 097/103] Fix for using part after std::move from it --- src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 603584af6c2..98f28430ecc 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -387,8 +387,8 @@ void DefaultCoordinator::initializeReadingState(InitialAllRangesAnnouncement ann if (intersecting_it != known_parts.end() && !intersecting_it->description.info.isDisjoint(part.info)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Intersecting parts found in announcement"); - all_parts_to_read.push_back(Part{.description = std::move(part), .replicas = {announcement.replica_num}}); known_parts.emplace(Part{.description = part, .replicas = {}}); + all_parts_to_read.push_back(Part{.description = std::move(part), .replicas = {announcement.replica_num}}); } } From 574a26c63ba24e4632b428827642b40db48424e4 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 17 Sep 2024 17:56:44 +0200 Subject: [PATCH 098/103] Use adjacent_find to check adjacent parts --- .../ParallelReplicasReadingCoordinator.cpp | 21 ++++++++----------- 1 file changed, 9 insertions(+), 12 deletions(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 98f28430ecc..26f2273d196 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -917,18 +917,15 @@ void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRa #ifndef NDEBUG /// Double check that there are no intersecting parts { - auto part_it = all_parts_to_read.begin(); - auto next_part_it = part_it; - if (next_part_it != all_parts_to_read.end()) - ++next_part_it; - while (next_part_it != all_parts_to_read.end()) - { - chassert(part_it->description.info.isDisjoint(next_part_it->description.info), - fmt::format("Parts {} and {} intersect", - part_it->description.info.getPartNameV1(), next_part_it->description.info.getPartNameV1())); - ++part_it; - ++next_part_it; - } + auto intersecting_part_it = std::adjacent_find(all_parts_to_read.begin(), all_parts_to_read.end(), + [] (const Part & lhs, const Part & rhs) + { + return !lhs.description.info.isDisjoint(rhs.description.info); + }); + + if (intersecting_part_it != all_parts_to_read.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Parts {} and {} intersect", + intersecting_part_it->description.info.getPartNameV1(), std::next(intersecting_part_it)->description.info.getPartNameV1()); } #endif From 665f362601a2ee1869cd5c91102d48818f7f0145 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 17 Sep 2024 16:10:03 +0000 Subject: [PATCH 099/103] Prohibit ALTER TABLE ... ADD INDEX ... TYPE inverted if setting = 0 --- src/Storages/AlterCommands.cpp | 10 +++ src/Storages/AlterCommands.h | 3 +- src/Storages/MergeTree/MergeTreeData.cpp | 4 ++ ...02346_inverted_index_experimental_flag.sql | 66 +++++++++++++++---- 4 files changed, 71 insertions(+), 12 deletions(-) diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index ef76bc691ec..68778243371 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -1142,6 +1142,16 @@ bool AlterCommands::hasFullTextIndex(const StorageInMemoryMetadata & metadata) return false; } +bool AlterCommands::hasLegacyInvertedIndex(const StorageInMemoryMetadata & metadata) +{ + for (const auto & index : metadata.secondary_indices) + { + if (index.type == INVERTED_INDEX_NAME) + return true; + } + return false; +} + bool AlterCommands::hasVectorSimilarityIndex(const StorageInMemoryMetadata & metadata) { for (const auto & index : metadata.secondary_indices) diff --git a/src/Storages/AlterCommands.h b/src/Storages/AlterCommands.h index c4c792e7dec..be1b31f3d20 100644 --- a/src/Storages/AlterCommands.h +++ b/src/Storages/AlterCommands.h @@ -235,8 +235,9 @@ public: /// additional mutation command (MATERIALIZE_TTL) will be returned. MutationCommands getMutationCommands(StorageInMemoryMetadata metadata, bool materialize_ttl, ContextPtr context, bool with_alters=false) const; - /// Check if commands have any full-text index + /// Check if commands have any full-text index or a (legacy) inverted index static bool hasFullTextIndex(const StorageInMemoryMetadata & metadata); + static bool hasLegacyInvertedIndex(const StorageInMemoryMetadata & metadata); /// Check if commands have any vector similarity index static bool hasVectorSimilarityIndex(const StorageInMemoryMetadata & metadata); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ca619d4d208..80d61058d08 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3230,6 +3230,10 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental full-text index feature is not enabled (turn on setting 'allow_experimental_full_text_index')"); + if (AlterCommands::hasLegacyInvertedIndex(new_metadata) && !settings.allow_experimental_inverted_index) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "Experimental inverted index feature is not enabled (turn on setting 'allow_experimental_inverted_index')"); + if (AlterCommands::hasVectorSimilarityIndex(new_metadata) && !settings.allow_experimental_vector_similarity_index) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental vector similarity index is disabled (turn on setting 'allow_experimental_vector_similarity_index')"); diff --git a/tests/queries/0_stateless/02346_inverted_index_experimental_flag.sql b/tests/queries/0_stateless/02346_inverted_index_experimental_flag.sql index f2d294ff9e4..84188337a8d 100644 --- a/tests/queries/0_stateless/02346_inverted_index_experimental_flag.sql +++ b/tests/queries/0_stateless/02346_inverted_index_experimental_flag.sql @@ -1,16 +1,60 @@ --- Tests that the inverted index can only be supported when allow_experimental_full_text_index = 1. - -SET allow_experimental_full_text_index = 0; +-- Tests that CREATE TABLE and ADD INDEX respect settings 'allow_experimental_full_text_index' and `allow_experimental_inverted_index` DROP TABLE IF EXISTS tab; -CREATE TABLE tab -( - `key` UInt64, - `str` String -) -ENGINE = MergeTree -ORDER BY key; -ALTER TABLE tab ADD INDEX inv_idx(str) TYPE full_text(0); -- { serverError SUPPORT_IS_DISABLED } +-- Test CREATE TABLE + full_text index setting +SET allow_experimental_full_text_index = 0; +CREATE TABLE tab (id UInt32, str String, INDEX idx str TYPE full_text(0)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError SUPPORT_IS_DISABLED } +CREATE TABLE tab (id UInt32, str String, INDEX idx str TYPE inverted(0)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError ILLEGAL_INDEX } + +SET allow_experimental_full_text_index = 1; +CREATE TABLE tab (id UInt32, str String, INDEX idx str TYPE full_text(0)) ENGINE = MergeTree ORDER BY tuple(); +CREATE TABLE tab (id UInt32, str String, INDEX idx str TYPE inverted(0)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError ILLEGAL_INDEX } DROP TABLE tab; + +SET allow_experimental_full_text_index = 0; -- reset to default + +-- Test CREATE TABLE + inverted index setting + +SET allow_experimental_inverted_index = 0; +CREATE TABLE tab (id UInt32, str String, INDEX idx str TYPE full_text(0)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError SUPPORT_IS_DISABLED } +CREATE TABLE tab (id UInt32, str String, INDEX idx str TYPE inverted(0)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError ILLEGAL_INDEX } + +SET allow_experimental_inverted_index = 1; +CREATE TABLE tab (id UInt32, str String, INDEX idx str TYPE full_text(0)) ENGINE = MergeTree ORDER BY tuple(); -- { serverError SUPPORT_IS_DISABLED } +CREATE TABLE tab (id UInt32, str String, INDEX idx str TYPE inverted(0)) ENGINE = MergeTree ORDER BY tuple(); +DROP TABLE tab; + +SET allow_experimental_inverted_index = 0; -- reset to default + +-- Test ADD INDEX + full_text index setting + +SET allow_experimental_full_text_index = 0; +CREATE TABLE tab (id UInt32, str String) ENGINE = MergeTree ORDER BY tuple(); +ALTER TABLE tab ADD INDEX idx1 str TYPE full_text(0); -- { serverError SUPPORT_IS_DISABLED } +ALTER TABLE tab ADD INDEX idx2 str TYPE inverted(0); -- { serverError SUPPORT_IS_DISABLED } +DROP TABLE tab; + +SET allow_experimental_full_text_index = 1; +CREATE TABLE tab (id UInt32, str String) ENGINE = MergeTree ORDER BY tuple(); +ALTER TABLE tab ADD INDEX idx1 str TYPE full_text(0); +ALTER TABLE tab ADD INDEX idx2 str TYPE inverted(0); -- { serverError SUPPORT_IS_DISABLED } +DROP TABLE tab; +SET allow_experimental_full_text_index = 0; -- reset to default + + +-- Test ADD INDEX + inverted index setting + +SET allow_experimental_inverted_index = 0; +CREATE TABLE tab (id UInt32, str String) ENGINE = MergeTree ORDER BY tuple(); +ALTER TABLE tab ADD INDEX idx1 str TYPE full_text(0); -- { serverError SUPPORT_IS_DISABLED } +ALTER TABLE tab ADD INDEX idx2 str TYPE inverted(0); -- { serverError SUPPORT_IS_DISABLED } +DROP TABLE tab; + +SET allow_experimental_inverted_index = 1; +CREATE TABLE tab (id UInt32, str String) ENGINE = MergeTree ORDER BY tuple(); +ALTER TABLE tab ADD INDEX idx1 str TYPE full_text(0); -- { serverError SUPPORT_IS_DISABLED } +ALTER TABLE tab ADD INDEX idx2 str TYPE inverted(0); +DROP TABLE tab; +SET allow_experimental_inverted_index = 0; -- reset to default From 13e82d6439bd92f3e025eee7acd635a17d098a36 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Tue, 17 Sep 2024 17:45:04 +0000 Subject: [PATCH 100/103] fix double visit of uncommitted changes --- src/Coordination/KeeperStorage.cpp | 6 ++- src/Coordination/tests/gtest_coordination.cpp | 40 +++++++++++++++++++ 2 files changed, 44 insertions(+), 2 deletions(-) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 63dc39092cf..a3ba7402296 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -1777,7 +1777,8 @@ private: auto child_path = (root_fs_path / child_name).generic_string(); const auto actual_child_node_ptr = storage.uncommitted_state.getActualNodeView(child_path, child_node); - if (actual_child_node_ptr == nullptr) /// node was deleted in previous step of multi transaction + /// if node was changed in previous step of multi transaction - skip until the uncommitted state visit + if (actual_child_node_ptr != &child_node) continue; if (checkLimits(actual_child_node_ptr)) @@ -1811,7 +1812,8 @@ private: const auto actual_child_node_ptr = storage.uncommitted_state.getActualNodeView(child_path, child_node); - if (actual_child_node_ptr == nullptr) /// node was deleted in previous step of multi transaction + /// if node was changed in previous step of multi transaction - skip until the uncommitted state visit + if (actual_child_node_ptr != &child_node) continue; if (checkLimits(actual_child_node_ptr)) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 46f36fe0039..15d5f460e94 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -3738,6 +3738,46 @@ TYPED_TEST(CoordinationTest, TestRemoveRecursiveInMultiRequest) ASSERT_FALSE(exists("/A/B")); ASSERT_FALSE(exists("/A/B/D")); } + + { + SCOPED_TRACE("Recursive Remove For Subtree With Updated Node"); + int create_zxid = ++zxid; + auto ops = prepare_create_tree(); + + /// First create nodes + const auto create_request = std::make_shared(ops, ACLs{}); + storage.preprocessRequest(create_request, 1, 0, create_zxid); + auto create_responses = storage.processRequest(create_request, 1, create_zxid); + ASSERT_EQ(create_responses.size(), 1); + ASSERT_TRUE(is_multi_ok(create_responses[0].response)); + + /// Small limit + int remove_zxid = ++zxid; + ops = { + zkutil::makeSetRequest("/A/B", "", -1), + zkutil::makeRemoveRecursiveRequest("/A", 3), + }; + auto remove_request = std::make_shared(ops, ACLs{}); + storage.preprocessRequest(remove_request, 1, 0, remove_zxid); + auto remove_responses = storage.processRequest(remove_request, 1, remove_zxid); + + ASSERT_EQ(remove_responses.size(), 1); + ASSERT_FALSE(is_multi_ok(remove_responses[0].response)); + + /// Big limit + remove_zxid = ++zxid; + ops[1] = zkutil::makeRemoveRecursiveRequest("/A", 4); + remove_request = std::make_shared(ops, ACLs{}); + storage.preprocessRequest(remove_request, 1, 0, remove_zxid); + remove_responses = storage.processRequest(remove_request, 1, remove_zxid); + + ASSERT_EQ(remove_responses.size(), 1); + ASSERT_TRUE(is_multi_ok(remove_responses[0].response)); + ASSERT_FALSE(exists("/A")); + ASSERT_FALSE(exists("/A/C")); + ASSERT_FALSE(exists("/A/B")); + ASSERT_FALSE(exists("/A/B/D")); + } } TYPED_TEST(CoordinationTest, TestRemoveRecursiveWatches) From 9c185374e460cfab452783c86964300e880145b3 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Tue, 17 Sep 2024 18:14:47 +0000 Subject: [PATCH 101/103] fix level sorting --- src/Coordination/KeeperStorage.h | 8 +++--- src/Coordination/tests/gtest_coordination.cpp | 26 +++++++++++++++++++ 2 files changed, 31 insertions(+), 3 deletions(-) diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 6fbc4c2b168..ec502a38a72 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -613,13 +613,15 @@ public: struct PathCmp { - using is_transparent = std::true_type; - auto operator()(const std::string_view a, const std::string_view b) const { - return a.size() < b.size() || (a.size() == b.size() && a < b); + size_t level_a = std::count(a.begin(), a.end(), '/'); + size_t level_b = std::count(b.begin(), b.end(), '/'); + return level_a < level_b || (level_a == level_b && a < b); } + + using is_transparent = void; // required to make find() work with different type than key_type }; mutable std::map nodes; diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 15d5f460e94..4272e504da1 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -3778,6 +3778,32 @@ TYPED_TEST(CoordinationTest, TestRemoveRecursiveInMultiRequest) ASSERT_FALSE(exists("/A/B")); ASSERT_FALSE(exists("/A/B/D")); } + + { + SCOPED_TRACE("[BUG] Recursive Remove Level Sorting"); + int new_zxid = ++zxid; + + Coordination::Requests ops = { + zkutil::makeCreateRequest("/a", "", zkutil::CreateMode::Persistent), + zkutil::makeCreateRequest("/a/bbbbbb", "", zkutil::CreateMode::Persistent), + zkutil::makeCreateRequest("/A", "", zkutil::CreateMode::Persistent), + zkutil::makeCreateRequest("/A/B", "", zkutil::CreateMode::Persistent), + zkutil::makeCreateRequest("/A/CCCCCCCCCCCC", "", zkutil::CreateMode::Persistent), + zkutil::makeRemoveRecursiveRequest("/A", 3), + }; + auto remove_request = std::make_shared(ops, ACLs{}); + storage.preprocessRequest(remove_request, 1, 0, new_zxid); + auto remove_responses = storage.processRequest(remove_request, 1, new_zxid); + + ASSERT_EQ(remove_responses.size(), 1); + ASSERT_TRUE(is_multi_ok(remove_responses[0].response)); + ASSERT_TRUE(exists("/a")); + ASSERT_TRUE(exists("/a/bbbbbb")); + ASSERT_FALSE(exists("/A")); + ASSERT_FALSE(exists("/A/B")); + ASSERT_FALSE(exists("/A/CCCCCCCCCCCC")); + } + } TYPED_TEST(CoordinationTest, TestRemoveRecursiveWatches) From 474499d240425872fadbd59ee97750a67f41390f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 17 Sep 2024 21:48:19 +0200 Subject: [PATCH 102/103] Revert "Add user-level settings min_free_diskspace_bytes_to_throw_insert and min_free_diskspace_ratio_to_throw_insert" --- .../settings/merge-tree-settings.md | 20 ------ src/Core/Settings.h | 2 - src/Core/SettingsChangesHistory.cpp | 4 +- .../MergeTree/MergeTreeDataWriter.cpp | 30 +-------- src/Storages/MergeTree/MergeTreeSettings.h | 2 - .../__init__.py | 0 .../config.d/storage_configuration.xml | 19 ------ .../test.py | 61 ------------------- 8 files changed, 3 insertions(+), 135 deletions(-) delete mode 100644 tests/integration/test_stop_insert_when_disk_close_to_full/__init__.py delete mode 100644 tests/integration/test_stop_insert_when_disk_close_to_full/configs/config.d/storage_configuration.xml delete mode 100644 tests/integration/test_stop_insert_when_disk_close_to_full/test.py diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 376c1c66ad5..a13aacc76e6 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -156,26 +156,6 @@ Default value: 1000. ClickHouse artificially executes `INSERT` longer (adds ‘sleep’) so that the background merge process can merge parts faster than they are added. -## min_free_disk_bytes_to_throw_insert {#min_free_disk_bytes_to_throw_insert} - -The minimum number of bytes that should be free in disk space in order to insert data. If the number of available free bytes - `keep_free_space_bytes` is less than `min_free_disk_bytes_to_throw_insert` then an exception is thrown and the insert is not executed. Note that this setting does not take into account the amount of data that will be written by the `INSERT` operation. - -Possible values: - -- Any positive integer. - -Default value: 0 bytes. - -## min_free_disk_ratio_to_throw_insert {#min_free_disk_ratio_to_throw_insert} - -The minimum free to total disk space ratio to perform an `INSERT`. The free space is calculated by subtracting `keep_free_space_bytes` from the total available space in disk. - -Possible values: - -- Float, 0.0 - 1.0 - -Default value: 0.0 - ## inactive_parts_to_throw_insert {#inactive-parts-to-throw-insert} If the number of inactive parts in a single partition more than the `inactive_parts_to_throw_insert` value, `INSERT` is interrupted with the "Too many inactive parts (N). Parts cleaning are processing significantly slower than inserts" exception. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 95f585d1a27..fadc4079fe0 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -343,8 +343,6 @@ class IColumn; M(Int64, distributed_ddl_task_timeout, 180, "Timeout for DDL query responses from all hosts in cluster. If a ddl request has not been performed on all hosts, a response will contain a timeout error and a request will be executed in an async mode. Negative value means infinite. Zero means async mode.", 0) \ M(Milliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.", 0) \ M(Milliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.", 0) \ - M(UInt64, min_free_disk_bytes_to_throw_insert, 0, "Minimum free disk space bytes to throw an insert.", 0) \ - M(Double, min_free_disk_ratio_to_throw_insert, 0.0, "Minimum free disk space ratio to throw an insert.", 0) \ \ M(Bool, final, false, "Query with the FINAL modifier by default. If the engine does not support final, it does not have any effect. On queries with multiple tables final is applied only on those that support it. It also works on distributed tables", 0) \ \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index da0fbfad255..560f144866b 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -85,9 +85,7 @@ static std::initializer_listgetVolume(0); VolumePtr data_part_volume = createVolumeFromReservation(reservation, volume); - const auto & data_settings = data.getSettings(); - const UInt64 min_bytes = data_settings->min_free_disk_bytes_to_throw_insert; - const Float64 min_ratio = data_settings->min_free_disk_ratio_to_throw_insert; - - if (min_bytes > 0 || min_ratio > 0.0) - { - const auto disk = data_part_volume->getDisk(); - const UInt64 total_disk_bytes = *disk->getTotalSpace(); - const UInt64 free_disk_bytes = *disk->getAvailableSpace(); - - const UInt64 min_bytes_from_ratio = static_cast(min_ratio * total_disk_bytes); - const UInt64 needed_free_bytes = std::max(min_bytes, min_bytes_from_ratio); - - if (needed_free_bytes > free_disk_bytes) - { - throw Exception( - ErrorCodes::NOT_ENOUGH_SPACE, - "Could not perform insert: less than {} free bytes in disk space ({}). " - "Configure this limit with user settings {} or {}", - needed_free_bytes, - free_disk_bytes, - "min_free_disk_bytes_to_throw_insert", - "min_free_disk_ratio_to_throw_insert"); - } - } - auto new_data_part = data.getDataPartBuilder(part_name, data_part_volume, part_dir) .withPartFormat(data.choosePartFormat(expected_size, block.rows())) .withPartInfo(new_part_info) @@ -591,6 +564,8 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( if (data.storage_settings.get()->assign_part_uuids) new_data_part->uuid = UUIDHelpers::generateV4(); + const auto & data_settings = data.getSettings(); + SerializationInfo::Settings settings{data_settings->ratio_of_defaults_for_sparse_serialization, true}; SerializationInfoByName infos(columns, settings); infos.add(block); @@ -713,7 +688,6 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( MergeTreeDataPartType part_type; /// Size of part would not be greater than block.bytes() + epsilon size_t expected_size = block.bytes(); - // just check if there is enough space on parent volume MergeTreeData::reserveSpace(expected_size, parent_part->getDataPartStorage()); part_type = data.choosePartFormatOnDisk(expected_size, block.rows()).part_type; diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index b2ebfa1dfda..dcb18155114 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -99,8 +99,6 @@ struct Settings; M(Bool, add_implicit_sign_column_constraint_for_collapsing_engine, false, "If true, add implicit constraint for sign column for CollapsingMergeTree engine.", 0) \ M(Milliseconds, sleep_before_commit_local_part_in_replicated_table_ms, 0, "For testing. Do not change it.", 0) \ M(Bool, optimize_row_order, false, "Allow reshuffling of rows during part inserts and merges to improve the compressibility of the new part", 0) \ - M(UInt64, min_free_disk_bytes_to_throw_insert, 0, "Minimum free disk space bytes to throw an insert.", 0) \ - M(Double, min_free_disk_ratio_to_throw_insert, 0.0, "Minimum free disk space ratio to throw an insert.", 0) \ M(Bool, use_adaptive_write_buffer_for_dynamic_subcolumns, true, "Allow to use adaptive writer buffers during writing dynamic subcolumns to reduce memory usage", 0) \ M(UInt64, adaptive_write_buffer_initial_size, 16 * 1024, "Initial size of an adaptive write buffer", 0) \ \ diff --git a/tests/integration/test_stop_insert_when_disk_close_to_full/__init__.py b/tests/integration/test_stop_insert_when_disk_close_to_full/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_stop_insert_when_disk_close_to_full/configs/config.d/storage_configuration.xml b/tests/integration/test_stop_insert_when_disk_close_to_full/configs/config.d/storage_configuration.xml deleted file mode 100644 index d4031ff656c..00000000000 --- a/tests/integration/test_stop_insert_when_disk_close_to_full/configs/config.d/storage_configuration.xml +++ /dev/null @@ -1,19 +0,0 @@ - - - - - local - /disk1/ - - - - - -
- disk1 -
-
-
-
-
-
diff --git a/tests/integration/test_stop_insert_when_disk_close_to_full/test.py b/tests/integration/test_stop_insert_when_disk_close_to_full/test.py deleted file mode 100644 index 328de674de1..00000000000 --- a/tests/integration/test_stop_insert_when_disk_close_to_full/test.py +++ /dev/null @@ -1,61 +0,0 @@ -import pytest -from helpers.cluster import ClickHouseCluster, ClickHouseInstance -from helpers.client import QueryRuntimeException - -cluster = ClickHouseCluster(__file__) - -node = cluster.add_instance( - "node", - main_configs=["configs/config.d/storage_configuration.xml"], - tmpfs=["/disk1:size=7M"], - macros={"shard": 0, "replica": 1}, -) - - -@pytest.fixture(scope="module") -def start_cluster(): - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - - -def test_insert_stops_when_disk_full(start_cluster): - min_free_bytes = 3 * 1024 * 1024 # 3 MiB - - node.query( - f""" - CREATE TABLE test_table ( - id UInt32, - data String - ) ENGINE = MergeTree() - ORDER BY id - SETTINGS storage_policy = 'only_disk1', min_free_disk_bytes_to_throw_insert = {min_free_bytes} - """ - ) - - count = 0 - - # Insert data to fill up disk - try: - for _ in range(100000): - node.query( - "INSERT INTO test_table SELECT number, repeat('a', 1000 * 1000) FROM numbers(1)" - ) - count += 1 - except QueryRuntimeException as e: - assert "Could not perform insert" in str(e) - assert "free bytes in disk space" in str(e) - - free_space = int( - node.query("SELECT free_space FROM system.disks WHERE name = 'disk1'").strip() - ) - assert ( - free_space <= min_free_bytes - ), f"Free space ({free_space}) is less than min_free_bytes ({min_free_bytes})" - - rows = int(node.query("SELECT count() from test_table").strip()) - assert rows == count - - node.query("DROP TABLE test_table") From 1bcdde3e628c570183880027770004fff196e8f1 Mon Sep 17 00:00:00 2001 From: Tanya Bragin Date: Tue, 17 Sep 2024 19:48:48 -0700 Subject: [PATCH 103/103] Update README.md - Meetups --- README.md | 20 +++++++++----------- 1 file changed, 9 insertions(+), 11 deletions(-) diff --git a/README.md b/README.md index 5fa04fe29e7..e74fb303d4f 100644 --- a/README.md +++ b/README.md @@ -40,17 +40,8 @@ Every month we get together with the community (users, contributors, customers, Keep an eye out for upcoming meetups and events around the world. Somewhere else you want us to be? Please feel free to reach out to tyler `` clickhouse `` com. You can also peruse [ClickHouse Events](https://clickhouse.com/company/news-events) for a list of all upcoming trainings, meetups, speaking engagements, etc. -The following upcoming meetups are featuring creator of ClickHouse & CTO, Alexey Milovidov: +Upcoming meetups -* [Raleigh Meetup (Deutsche Bank)](https://www.meetup.com/triangletechtalks/events/302723486/) - September 9 -* [New York Meetup (Rokt)](https://www.meetup.com/clickhouse-new-york-user-group/events/302575342) - September 10 -* [Chicago Meetup (Jump Capital)](https://lu.ma/43tvmrfw) - September 12 - -Other upcoming meetups - -* [Toronto Meetup (Shopify)](https://www.meetup.com/clickhouse-toronto-user-group/events/301490855/) - September 10 -* [Austin Meetup](https://www.meetup.com/clickhouse-austin-user-group/events/302558689/) - September 17 -* [London Meetup](https://www.meetup.com/clickhouse-london-user-group/events/302977267) - September 17 * [Bangalore Meetup](https://www.meetup.com/clickhouse-bangalore-user-group/events/303208274/) - September 18 * [Tel Aviv Meetup](https://www.meetup.com/clickhouse-meetup-israel/events/303095121) - September 22 * [Jakarta Meetup](https://www.meetup.com/clickhouse-indonesia-user-group/events/303191359/) - October 1 @@ -62,13 +53,20 @@ Other upcoming meetups * [Dubai Meetup](https://www.meetup.com/clickhouse-dubai-meetup-group/events/303096989/) - November 21 * [Paris Meetup](https://www.meetup.com/clickhouse-france-user-group/events/303096434) - November 26 -Recently completed events +Recently completed meetups + * [ClickHouse Guangzhou User Group Meetup](https://mp.weixin.qq.com/s/GSvo-7xUoVzCsuUvlLTpCw) - August 25 * [Seattle Meetup (Statsig)](https://www.meetup.com/clickhouse-seattle-user-group/events/302518075/) - August 27 * [Melbourne Meetup](https://www.meetup.com/clickhouse-australia-user-group/events/302732666/) - August 27 * [Sydney Meetup](https://www.meetup.com/clickhouse-australia-user-group/events/302862966/) - September 5 * [Zurich Meetup](https://www.meetup.com/clickhouse-switzerland-meetup-group/events/302267429/) - September 5 * [San Francisco Meetup (Cloudflare)](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/302540575) - September 5 +* [Raleigh Meetup (Deutsche Bank)](https://www.meetup.com/triangletechtalks/events/302723486/) - September 9 +* [New York Meetup (Rokt)](https://www.meetup.com/clickhouse-new-york-user-group/events/302575342) - September 10 +* [Toronto Meetup (Shopify)](https://www.meetup.com/clickhouse-toronto-user-group/events/301490855/) - September 10 +* [Chicago Meetup (Jump Capital)](https://lu.ma/43tvmrfw) - September 12 +* [London Meetup](https://www.meetup.com/clickhouse-london-user-group/events/302977267) - September 17 +* [Austin Meetup](https://www.meetup.com/clickhouse-austin-user-group/events/302558689/) - September 17 ## Recent Recordings * **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments"